Introduce IStorageComponentProvider

Change-Id: If86750cdb2436c713f6598e54d4aaaf23d9f7bbf
Reviewed-on: https://asterix-gerrit.ics.uci.edu/1451
Tested-by: Jenkins <jenkins@fulliautomatix.ics.uci.edu>
Reviewed-by: Yingyi Bu <buyingyi@gmail.com>
Integration-Tests: Jenkins <jenkins@fulliautomatix.ics.uci.edu>
diff --git a/asterixdb/asterix-algebra/pom.xml b/asterixdb/asterix-algebra/pom.xml
index 1209f5a..8588381 100644
--- a/asterixdb/asterix-algebra/pom.xml
+++ b/asterixdb/asterix-algebra/pom.xml
@@ -153,11 +153,6 @@
       </exclusions>
     </dependency>
     <dependency>
-      <groupId>org.apache.asterix</groupId>
-      <artifactId>asterix-transactions</artifactId>
-      <version>${project.version}</version>
-    </dependency>
-    <dependency>
       <groupId>org.apache.hyracks</groupId>
       <artifactId>algebricks-compiler</artifactId>
     </dependency>
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/algebra/operators/physical/BTreeSearchPOperator.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/algebra/operators/physical/BTreeSearchPOperator.java
index 4db5430..9dd57d5 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/algebra/operators/physical/BTreeSearchPOperator.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/algebra/operators/physical/BTreeSearchPOperator.java
@@ -47,7 +47,6 @@
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.AbstractUnnestMapOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.IOperatorSchema;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.OrderOperator.IOrder.OrderKind;
-import org.apache.hyracks.algebricks.core.algebra.operators.logical.visitors.VariableUtilities;
 import org.apache.hyracks.algebricks.core.algebra.properties.BroadcastPartitioningProperty;
 import org.apache.hyracks.algebricks.core.algebra.properties.ILocalStructuralProperty;
 import org.apache.hyracks.algebricks.core.algebra.properties.INodeDomain;
@@ -98,7 +97,7 @@
     @Override
     public void contributeRuntimeOperator(IHyracksJobBuilder builder, JobGenContext context, ILogicalOperator op,
             IOperatorSchema opSchema, IOperatorSchema[] inputSchemas, IOperatorSchema outerPlanSchema)
-                    throws AlgebricksException {
+            throws AlgebricksException {
         AbstractUnnestMapOperator unnestMap = (AbstractUnnestMapOperator) op;
         ILogicalExpression unnestExpr = unnestMap.getExpressionRef().getValue();
         if (unnestExpr.getExpressionTag() != LogicalExpressionTag.FUNCTION_CALL) {
@@ -120,20 +119,12 @@
         MetadataProvider metadataProvider = (MetadataProvider) context.getMetadataProvider();
         Dataset dataset = metadataProvider.findDataset(jobGenParams.getDataverseName(), jobGenParams.getDatasetName());
         IVariableTypeEnvironment typeEnv = context.getTypeEnvironment(op);
-        List<LogicalVariable> outputVars = unnestMap.getVariables();
-        if (jobGenParams.getRetainInput()) {
-            outputVars = new ArrayList<LogicalVariable>();
-            VariableUtilities.getLiveVariables(unnestMap, outputVars);
-        }
-        boolean retainNull = false;
-        if (op.getOperatorTag() == LogicalOperatorTag.LEFT_OUTER_UNNEST_MAP) {
-            // By nature, LEFT_OUTER_UNNEST_MAP should generate null values for non-matching tuples.
-            retainNull = true;
-        }
+        // By nature, LEFT_OUTER_UNNEST_MAP should generate null values for non-matching tuples.
+        boolean retainMissing = op.getOperatorTag() == LogicalOperatorTag.LEFT_OUTER_UNNEST_MAP;
         Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> btreeSearch = metadataProvider.buildBtreeRuntime(
-                builder.getJobSpec(), outputVars, opSchema, typeEnv, context, jobGenParams.getRetainInput(), retainNull,
+                builder.getJobSpec(), opSchema, typeEnv, context, jobGenParams.getRetainInput(), retainMissing,
                 dataset, jobGenParams.getIndexName(), lowKeyIndexes, highKeyIndexes, jobGenParams.isLowKeyInclusive(),
-                jobGenParams.isHighKeyInclusive(), implConfig, minFilterFieldIndexes, maxFilterFieldIndexes);
+                jobGenParams.isHighKeyInclusive(), minFilterFieldIndexes, maxFilterFieldIndexes);
 
         builder.contributeHyracksOperator(unnestMap, btreeSearch.first);
         builder.contributeAlgebricksPartitionConstraint(btreeSearch.first, btreeSearch.second);
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/algebra/operators/physical/CommitRuntime.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/algebra/operators/physical/CommitRuntime.java
index 8c61300..63a91ac 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/algebra/operators/physical/CommitRuntime.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/algebra/operators/physical/CommitRuntime.java
@@ -41,7 +41,7 @@
 import org.apache.hyracks.dataflow.common.data.accessors.FrameTupleReference;
 import org.apache.hyracks.dataflow.common.data.accessors.ITupleReference;
 import org.apache.hyracks.dataflow.common.io.MessagingFrameTupleAppender;
-import org.apache.hyracks.dataflow.common.util.TaskUtils;
+import org.apache.hyracks.dataflow.common.utils.TaskUtil;
 import org.apache.hyracks.storage.am.bloomfilter.impls.MurmurHash128Bit;
 
 public class CommitRuntime extends AbstractOneInputOneOutputOneFramePushRuntime {
@@ -65,8 +65,8 @@
     public CommitRuntime(IHyracksTaskContext ctx, JobId jobId, int datasetId, int[] primaryKeyFields,
             boolean isTemporaryDatasetWriteJob, boolean isWriteTransaction, int resourcePartition, boolean isSink) {
         this.ctx = ctx;
-        IAppRuntimeContext runtimeCtx = (IAppRuntimeContext) ctx.getJobletContext()
-                .getApplicationContext().getApplicationObject();
+        IAppRuntimeContext runtimeCtx =
+                (IAppRuntimeContext) ctx.getJobletContext().getApplicationContext().getApplicationObject();
         this.transactionManager = runtimeCtx.getTransactionSubsystem().getTransactionManager();
         this.logMgr = runtimeCtx.getTransactionSubsystem().getLogManager();
         this.jobId = jobId;
@@ -85,8 +85,8 @@
         try {
             transactionContext = transactionManager.getTransactionContext(jobId, false);
             transactionContext.setWriteTxn(isWriteTransaction);
-            ILogMarkerCallback callback = TaskUtils.<ILogMarkerCallback> get(ILogMarkerCallback.KEY_MARKER_CALLBACK,
-                    ctx);
+            ILogMarkerCallback callback =
+                    TaskUtil.<ILogMarkerCallback> get(ILogMarkerCallback.KEY_MARKER_CALLBACK, ctx);
             logRecord = new LogRecord(callback);
             if (isSink) {
                 return;
@@ -126,7 +126,7 @@
                 }
             }
         }
-        VSizeFrame message = TaskUtils.<VSizeFrame> get(HyracksConstants.KEY_MESSAGE, ctx);
+        VSizeFrame message = TaskUtil.<VSizeFrame> get(HyracksConstants.KEY_MESSAGE, ctx);
         if (message != null
                 && MessagingFrameTupleAppender.getMessageType(message) == MessagingFrameTupleAppender.MARKER_MESSAGE) {
             try {
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/algebra/operators/physical/ExternalDataLookupPOperator.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/algebra/operators/physical/ExternalDataLookupPOperator.java
index 98bf45f..1d79455 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/algebra/operators/physical/ExternalDataLookupPOperator.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/algebra/operators/physical/ExternalDataLookupPOperator.java
@@ -18,7 +18,6 @@
  */
 package org.apache.asterix.algebra.operators.physical;
 
-import java.util.ArrayList;
 import java.util.List;
 
 import org.apache.asterix.metadata.declared.DataSource;
@@ -27,7 +26,6 @@
 import org.apache.asterix.metadata.declared.DatasetDataSource;
 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.om.functions.BuiltinFunctions;
 import org.apache.asterix.om.types.ARecordType;
 import org.apache.hyracks.algebricks.common.constraints.AlgebricksPartitionConstraint;
@@ -47,7 +45,6 @@
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.AbstractScanOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.IOperatorSchema;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.UnnestMapOperator;
-import org.apache.hyracks.algebricks.core.algebra.operators.logical.visitors.VariableUtilities;
 import org.apache.hyracks.algebricks.core.algebra.operators.physical.AbstractScanPOperator;
 import org.apache.hyracks.algebricks.core.algebra.properties.BroadcastPartitioningProperty;
 import org.apache.hyracks.algebricks.core.algebra.properties.IPartitioningRequirementsCoordinator;
@@ -63,22 +60,19 @@
     private DataSourceId datasetId;
     private Dataset dataset;
     private ARecordType recordType;
-    private Index secondaryIndex;
     private boolean requiresBroadcast;
     private boolean retainInput;
-    private boolean retainNull;
+    private boolean retainMissing;
 
     public ExternalDataLookupPOperator(DataSourceId datasetId, Dataset dataset, ARecordType recordType,
-            Index secondaryIndex, List<LogicalVariable> ridVarList, boolean requiresBroadcast, boolean retainInput,
-            boolean retainNull) {
+            List<LogicalVariable> ridVarList, boolean requiresBroadcast, boolean retainInput, boolean retainMissing) {
         this.datasetId = datasetId;
         this.dataset = dataset;
         this.recordType = recordType;
-        this.secondaryIndex = secondaryIndex;
         this.ridVarList = ridVarList;
         this.requiresBroadcast = requiresBroadcast;
         this.retainInput = retainInput;
-        this.retainNull = retainNull;
+        this.retainMissing = retainMissing;
     }
 
     public Dataset getDataset() {
@@ -137,17 +131,10 @@
         }
         int[] ridIndexes = getKeyIndexes(ridVarList, inputSchemas);
         IVariableTypeEnvironment typeEnv = context.getTypeEnvironment(op);
-        List<LogicalVariable> outputVars = new ArrayList<LogicalVariable>();
-        if (retainInput) {
-            VariableUtilities.getLiveVariables(unnestMap, outputVars);
-        } else {
-            VariableUtilities.getProducedVariables(unnestMap, outputVars);
-        }
-
         MetadataProvider metadataProvider = (MetadataProvider) context.getMetadataProvider();
-        Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> externalLoopup = metadataProvider
-                .buildExternalDataLookupRuntime(builder.getJobSpec(), dataset, secondaryIndex, ridIndexes, retainInput,
-                        typeEnv, outputVars, opSchema, context, metadataProvider, retainNull);
+        Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> externalLoopup =
+                metadataProvider.buildExternalDataLookupRuntime(builder.getJobSpec(), dataset, ridIndexes, retainInput,
+                        typeEnv, opSchema, context, metadataProvider, retainMissing);
         builder.contributeHyracksOperator(unnestMap, externalLoopup.first);
         builder.contributeAlgebricksPartitionConstraint(externalLoopup.first, externalLoopup.second);
         ILogicalOperator srcExchange = unnestMap.getInputs().get(0).getValue();
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/algebra/operators/physical/InvertedIndexPOperator.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/algebra/operators/physical/InvertedIndexPOperator.java
index 7d8dcd1..86faa6c 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/algebra/operators/physical/InvertedIndexPOperator.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/algebra/operators/physical/InvertedIndexPOperator.java
@@ -22,31 +22,25 @@
 import java.util.List;
 import java.util.Map;
 
-import org.apache.asterix.common.config.StorageProperties;
-import org.apache.asterix.common.context.AsterixVirtualBufferCacheProvider;
 import org.apache.asterix.common.dataflow.IApplicationContextInfo;
-import org.apache.asterix.common.dataflow.LSMIndexUtil;
-import org.apache.asterix.common.ioopcallbacks.LSMInvertedIndexIOOperationCallbackFactory;
 import org.apache.asterix.metadata.MetadataException;
 import org.apache.asterix.metadata.MetadataManager;
 import org.apache.asterix.metadata.declared.DataSourceId;
 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.utils.DatasetUtils;
+import org.apache.asterix.metadata.utils.DatasetUtil;
 import org.apache.asterix.om.base.IAObject;
 import org.apache.asterix.om.constants.AsterixConstantValue;
 import org.apache.asterix.om.functions.BuiltinFunctions;
 import org.apache.asterix.om.types.ARecordType;
 import org.apache.asterix.om.types.ATypeTag;
 import org.apache.asterix.om.types.IAType;
-import org.apache.asterix.om.util.NonTaggedFormatUtil;
+import org.apache.asterix.om.utils.NonTaggedFormatUtil;
 import org.apache.asterix.optimizer.rules.am.InvertedIndexAccessMethod;
 import org.apache.asterix.optimizer.rules.am.InvertedIndexAccessMethod.SearchModifierType;
 import org.apache.asterix.optimizer.rules.am.InvertedIndexJobGenParams;
-import org.apache.asterix.runtime.util.AppContextInfo;
-import org.apache.asterix.runtime.util.RuntimeComponentsProvider;
-import org.apache.asterix.transaction.management.opcallbacks.SecondaryIndexOperationTrackerProvider;
+import org.apache.asterix.runtime.job.listener.JobEventListenerFactory;
 import org.apache.hyracks.algebricks.common.constraints.AlgebricksPartitionConstraint;
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
 import org.apache.hyracks.algebricks.common.utils.Pair;
@@ -76,12 +70,10 @@
 import org.apache.hyracks.data.std.primitive.ShortPointable;
 import org.apache.hyracks.dataflow.std.file.IFileSplitProvider;
 import org.apache.hyracks.storage.am.common.dataflow.IIndexDataflowHelperFactory;
-import org.apache.hyracks.storage.am.common.impls.NoOpOperationCallbackFactory;
+import org.apache.hyracks.storage.am.common.ophelpers.IndexOperation;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMMergePolicyFactory;
 import org.apache.hyracks.storage.am.lsm.invertedindex.api.IInvertedIndexSearchModifierFactory;
-import org.apache.hyracks.storage.am.lsm.invertedindex.dataflow.LSMInvertedIndexDataflowHelperFactory;
 import org.apache.hyracks.storage.am.lsm.invertedindex.dataflow.LSMInvertedIndexSearchOperatorDescriptor;
-import org.apache.hyracks.storage.am.lsm.invertedindex.dataflow.PartitionedLSMInvertedIndexDataflowHelperFactory;
 import org.apache.hyracks.storage.am.lsm.invertedindex.tokenizers.IBinaryTokenizerFactory;
 
 /**
@@ -140,12 +132,12 @@
             retainNull = true;
         }
         // Build runtime.
-        Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> invIndexSearch = buildInvertedIndexRuntime(
-                metadataProvider, context, builder.getJobSpec(), unnestMapOp, opSchema, jobGenParams.getRetainInput(),
-                retainNull, jobGenParams.getDatasetName(), dataset, jobGenParams.getIndexName(),
-                jobGenParams.getSearchKeyType(), keyIndexes, jobGenParams.getSearchModifierType(),
-                jobGenParams.getSimilarityThreshold(), minFilterFieldIndexes, maxFilterFieldIndexes,
-                jobGenParams.getIsFullTextSearch());
+        Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> invIndexSearch =
+                buildInvertedIndexRuntime(metadataProvider, context, builder.getJobSpec(), unnestMapOp, opSchema,
+                        jobGenParams.getRetainInput(), retainNull, jobGenParams.getDatasetName(), dataset,
+                        jobGenParams.getIndexName(), jobGenParams.getSearchKeyType(), keyIndexes,
+                        jobGenParams.getSearchModifierType(), jobGenParams.getSimilarityThreshold(),
+                        minFilterFieldIndexes, maxFilterFieldIndexes, jobGenParams.getIsFullTextSearch());
 
         // Contribute operator in hyracks job.
         builder.contributeHyracksOperator(unnestMapOp, invIndexSearch.first);
@@ -161,12 +153,11 @@
             SearchModifierType searchModifierType, IAlgebricksConstantValue similarityThreshold,
             int[] minFilterFieldIndexes, int[] maxFilterFieldIndexes, boolean isFullTextSearchQuery)
             throws AlgebricksException {
-
         try {
             IAObject simThresh = ((AsterixConstantValue) similarityThreshold).getObject();
             IAType itemType = MetadataManager.INSTANCE.getDatatype(metadataProvider.getMetadataTxnContext(),
                     dataset.getItemTypeDataverseName(), dataset.getItemTypeName()).getDatatype();
-            int numPrimaryKeys = DatasetUtils.getPartitioningKeys(dataset).size();
+            int numPrimaryKeys = DatasetUtil.getPartitioningKeys(dataset).size();
             Index secondaryIndex = MetadataManager.INSTANCE.getIndex(metadataProvider.getMetadataTxnContext(),
                     dataset.getDataverseName(), dataset.getDatasetName(), indexName);
             if (secondaryIndex == null) {
@@ -177,8 +168,9 @@
             List<IAType> secondaryKeyTypeEntries = secondaryIndex.getKeyFieldTypes();
             int numSecondaryKeys = secondaryKeyFieldEntries.size();
             if (numSecondaryKeys != 1) {
-                throw new AlgebricksException("Cannot use " + numSecondaryKeys
-                        + " fields as a key for an inverted index. There can be only one field as a key for the inverted index index.");
+                throw new AlgebricksException(
+                        "Cannot use " + numSecondaryKeys + " fields as a key for an inverted index. "
+                                + "There can be only one field as a key for the inverted index index.");
             }
             if (itemType.getTypeTag() != ATypeTag.RECORD) {
                 throw new AlgebricksException("Only record types can be indexed.");
@@ -205,8 +197,8 @@
             }
             if (isPartitioned) {
                 // The partitioning field is hardcoded to be a short *without* an Asterix type tag.
-                tokenComparatorFactories[numSecondaryKeys] = PointableBinaryComparatorFactory
-                        .of(ShortPointable.FACTORY);
+                tokenComparatorFactories[numSecondaryKeys] =
+                        PointableBinaryComparatorFactory.of(ShortPointable.FACTORY);
                 tokenTypeTraits[numSecondaryKeys] = ShortPointable.TYPE_TRAITS;
             }
 
@@ -221,17 +213,13 @@
             int start = outputRecDesc.getFieldCount() - numPrimaryKeys;
             IBinaryComparatorFactory[] invListsComparatorFactories = JobGenHelper
                     .variablesToAscBinaryComparatorFactories(outputVars, start, numPrimaryKeys, typeEnv, context);
-            ITypeTraits[] invListsTypeTraits = JobGenHelper.variablesToTypeTraits(outputVars, start, numPrimaryKeys,
-                    typeEnv, context);
-
-            ITypeTraits[] filterTypeTraits = DatasetUtils.computeFilterTypeTraits(dataset, recordType);
-            IBinaryComparatorFactory[] filterCmpFactories = DatasetUtils.computeFilterBinaryComparatorFactories(dataset,
-                    recordType, context.getBinaryComparatorFactoryProvider());
-
-            int[] filterFields = null;
-            int[] invertedIndexFields = null;
-            int[] filterFieldsForNonBulkLoadOps = null;
-            int[] invertedIndexFieldsForNonBulkLoadOps = null;
+            ITypeTraits[] invListsTypeTraits =
+                    JobGenHelper.variablesToTypeTraits(outputVars, start, numPrimaryKeys, typeEnv, context);
+            ITypeTraits[] filterTypeTraits = DatasetUtil.computeFilterTypeTraits(dataset, recordType);
+            int[] filterFields;
+            int[] invertedIndexFields;
+            int[] filterFieldsForNonBulkLoadOps;
+            int[] invertedIndexFieldsForNonBulkLoadOps;
             if (filterTypeTraits != null) {
                 filterFields = new int[1];
                 filterFields[0] = numTokenKeys + numPrimaryKeys;
@@ -247,50 +235,37 @@
                     invertedIndexFieldsForNonBulkLoadOps[k] = k;
                 }
             }
-
             IApplicationContextInfo appContext = (IApplicationContextInfo) context.getAppContext();
-            Pair<IFileSplitProvider, AlgebricksPartitionConstraint> secondarySplitsAndConstraint = metadataProvider
-                    .splitProviderAndPartitionConstraintsForDataset(dataset.getDataverseName(), datasetName, indexName,
+            Pair<IFileSplitProvider, AlgebricksPartitionConstraint> secondarySplitsAndConstraint =
+                    metadataProvider.getSplitProviderAndConstraints(dataset.getDataverseName(), datasetName, indexName,
                             dataset.getDatasetDetails().isTemp());
             // TODO: Here we assume there is only one search key field.
             int queryField = keyFields[0];
             // Get tokenizer and search modifier factories.
-            IInvertedIndexSearchModifierFactory searchModifierFactory = InvertedIndexAccessMethod
-                    .getSearchModifierFactory(searchModifierType, simThresh, secondaryIndex);
+            IInvertedIndexSearchModifierFactory searchModifierFactory =
+                    InvertedIndexAccessMethod.getSearchModifierFactory(searchModifierType, simThresh, secondaryIndex);
             IBinaryTokenizerFactory queryTokenizerFactory = InvertedIndexAccessMethod
                     .getBinaryTokenizerFactory(searchModifierType, searchKeyType, secondaryIndex);
-            IIndexDataflowHelperFactory dataflowHelperFactory;
-
-            StorageProperties storageProperties = AppContextInfo.INSTANCE.getStorageProperties();
-            Pair<ILSMMergePolicyFactory, Map<String, String>> compactionInfo = DatasetUtils
-                    .getMergePolicyFactory(dataset, metadataProvider.getMetadataTxnContext());
-            boolean temp = dataset.getDatasetDetails().isTemp();
-            if (!isPartitioned) {
-                dataflowHelperFactory = new LSMInvertedIndexDataflowHelperFactory(
-                        new AsterixVirtualBufferCacheProvider(dataset.getDatasetId()), compactionInfo.first,
-                        compactionInfo.second, new SecondaryIndexOperationTrackerProvider(dataset.getDatasetId()),
-                        RuntimeComponentsProvider.RUNTIME_PROVIDER,
-                        LSMInvertedIndexIOOperationCallbackFactory.INSTANCE,
-                        storageProperties.getBloomFilterFalsePositiveRate(), invertedIndexFields, filterTypeTraits,
-                        filterCmpFactories, filterFields, filterFieldsForNonBulkLoadOps,
-                        invertedIndexFieldsForNonBulkLoadOps, !temp);
-            } else {
-                dataflowHelperFactory = new PartitionedLSMInvertedIndexDataflowHelperFactory(
-                        new AsterixVirtualBufferCacheProvider(dataset.getDatasetId()), compactionInfo.first,
-                        compactionInfo.second, new SecondaryIndexOperationTrackerProvider(dataset.getDatasetId()),
-                        RuntimeComponentsProvider.RUNTIME_PROVIDER,
-                        LSMInvertedIndexIOOperationCallbackFactory.INSTANCE,
-                        storageProperties.getBloomFilterFalsePositiveRate(), invertedIndexFields, filterTypeTraits,
-                        filterCmpFactories, filterFields, filterFieldsForNonBulkLoadOps,
-                        invertedIndexFieldsForNonBulkLoadOps, !temp);
-            }
+            ARecordType metaType = dataset.hasMetaPart()
+                    ? (ARecordType) metadataProvider
+                            .findType(dataset.getMetaItemTypeDataverseName(), dataset.getMetaItemTypeName()).getType()
+                    : null;
+            Pair<ILSMMergePolicyFactory, Map<String, String>> compactionInfo =
+                    DatasetUtil.getMergePolicyFactory(dataset, metadataProvider.getMetadataTxnContext());
+            IIndexDataflowHelperFactory dataflowHelperFactory = dataset.getIndexDataflowHelperFactory(metadataProvider,
+                    secondaryIndex, recordType, metaType, compactionInfo.first, compactionInfo.second);
             LSMInvertedIndexSearchOperatorDescriptor invIndexSearchOp = new LSMInvertedIndexSearchOperatorDescriptor(
-                    jobSpec, queryField, appContext.getStorageManagerInterface(), secondarySplitsAndConstraint.first,
+                    jobSpec, queryField, appContext.getStorageManager(), secondarySplitsAndConstraint.first,
                     appContext.getIndexLifecycleManagerProvider(), tokenTypeTraits, tokenComparatorFactories,
                     invListsTypeTraits, invListsComparatorFactories, dataflowHelperFactory, queryTokenizerFactory,
-                    searchModifierFactory, outputRecDesc, retainInput, retainMissing, context.getMissingWriterFactory(),
-                    NoOpOperationCallbackFactory.INSTANCE, minFilterFieldIndexes, maxFilterFieldIndexes,
-                    LSMIndexUtil.getMetadataPageManagerFactory(), isFullTextSearchQuery);
+                    searchModifierFactory, outputRecDesc, retainInput, retainMissing,
+                    context.getMissingWriterFactory(),
+                    dataset.getSearchCallbackFactory(metadataProvider.getStorageComponentProvider(), secondaryIndex,
+                            ((JobEventListenerFactory) jobSpec.getJobletEventListenerFactory()).getJobId(),
+                            IndexOperation.SEARCH, null),
+                    minFilterFieldIndexes, maxFilterFieldIndexes,
+                    metadataProvider.getStorageComponentProvider().getMetadataPageManagerFactory(),
+                    isFullTextSearchQuery);
             return new Pair<>(invIndexSearchOp, secondarySplitsAndConstraint.second);
         } catch (MetadataException e) {
             throw new AlgebricksException(e);
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/ByNameToByIndexFieldAccessRule.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/ByNameToByIndexFieldAccessRule.java
index be8a14b..c7d5ad2 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/ByNameToByIndexFieldAccessRule.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/ByNameToByIndexFieldAccessRule.java
@@ -23,7 +23,6 @@
 import java.util.Collections;
 
 import org.apache.asterix.algebra.base.OperatorAnnotation;
-import org.apache.asterix.om.util.ConstantExpressionUtil;
 import org.apache.asterix.lang.common.util.FunctionUtil;
 import org.apache.asterix.om.base.AInt32;
 import org.apache.asterix.om.constants.AsterixConstantValue;
@@ -32,6 +31,7 @@
 import org.apache.asterix.om.types.ATypeTag;
 import org.apache.asterix.om.types.AUnionType;
 import org.apache.asterix.om.types.IAType;
+import org.apache.asterix.om.utils.ConstantExpressionUtil;
 import org.apache.commons.lang3.mutable.Mutable;
 import org.apache.commons.lang3.mutable.MutableObject;
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/CheckFilterExpressionTypeRule.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/CheckFilterExpressionTypeRule.java
index 1400950..ab359bf 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/CheckFilterExpressionTypeRule.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/CheckFilterExpressionTypeRule.java
@@ -22,7 +22,7 @@
 import org.apache.asterix.om.types.ATypeTag;
 import org.apache.asterix.om.types.AUnionType;
 import org.apache.asterix.om.types.IAType;
-import org.apache.asterix.om.util.NonTaggedFormatUtil;
+import org.apache.asterix.om.utils.NonTaggedFormatUtil;
 import org.apache.commons.lang3.mutable.Mutable;
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
 import org.apache.hyracks.algebricks.core.algebra.base.ILogicalExpression;
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/ConstantFoldingRule.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/ConstantFoldingRule.java
index 3ef5f2b..98e9f9d 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/ConstantFoldingRule.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/ConstantFoldingRule.java
@@ -42,7 +42,7 @@
 import org.apache.asterix.om.types.ARecordType;
 import org.apache.asterix.om.types.ATypeTag;
 import org.apache.asterix.om.types.AbstractCollectionType;
-import org.apache.asterix.om.util.ConstantExpressionUtil;
+import org.apache.asterix.om.utils.ConstantExpressionUtil;
 import org.apache.commons.lang3.mutable.Mutable;
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
 import org.apache.hyracks.algebricks.common.utils.Pair;
@@ -56,7 +56,7 @@
 import org.apache.hyracks.algebricks.core.algebra.expressions.AggregateFunctionCallExpression;
 import org.apache.hyracks.algebricks.core.algebra.expressions.ConstantExpression;
 import org.apache.hyracks.algebricks.core.algebra.expressions.IVariableTypeEnvironment;
-import org.apache.hyracks.algebricks.core.algebra.expressions.LogicalExpressionJobGenToExpressionRuntimeProviderAdapter;
+import org.apache.hyracks.algebricks.core.algebra.expressions.ExpressionRuntimeProvider;
 import org.apache.hyracks.algebricks.core.algebra.expressions.ScalarFunctionCallExpression;
 import org.apache.hyracks.algebricks.core.algebra.expressions.StatefulFunctionCallExpression;
 import org.apache.hyracks.algebricks.core.algebra.expressions.UnnestingFunctionCallExpression;
@@ -126,7 +126,7 @@
             BinaryHashFunctionFamilyProvider.INSTANCE, BinaryComparatorFactoryProvider.INSTANCE,
             TypeTraitProvider.INSTANCE, BinaryBooleanInspector.FACTORY, BinaryIntegerInspector.FACTORY,
             ADMPrinterFactoryProvider.INSTANCE, MissingWriterFactory.INSTANCE, null,
-            new LogicalExpressionJobGenToExpressionRuntimeProviderAdapter(QueryLogicalExpressionJobGen.INSTANCE),
+            new ExpressionRuntimeProvider(QueryLogicalExpressionJobGen.INSTANCE),
             ExpressionTypeComputer.INSTANCE, null, null, null, null, GlobalConfig.DEFAULT_FRAME_SIZE, null);
 
     private static final IOperatorSchema[] _emptySchemas = new IOperatorSchema[] {};
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/FullTextContainsParameterCheckRule.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/FullTextContainsParameterCheckRule.java
index e589065..20c043a 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/FullTextContainsParameterCheckRule.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/FullTextContainsParameterCheckRule.java
@@ -26,7 +26,7 @@
 import org.apache.asterix.om.constants.AsterixConstantValue;
 import org.apache.asterix.om.functions.BuiltinFunctions;
 import org.apache.asterix.om.types.ATypeTag;
-import org.apache.asterix.om.util.ConstantExpressionUtil;
+import org.apache.asterix.om.utils.ConstantExpressionUtil;
 import org.apache.asterix.runtime.evaluators.functions.FullTextContainsDescriptor;
 import org.apache.commons.lang3.mutable.Mutable;
 import org.apache.commons.lang3.mutable.MutableObject;
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/IntroduceDynamicTypeCastForExternalFunctionRule.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/IntroduceDynamicTypeCastForExternalFunctionRule.java
index 1b3acea..a813285 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/IntroduceDynamicTypeCastForExternalFunctionRule.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/IntroduceDynamicTypeCastForExternalFunctionRule.java
@@ -27,7 +27,7 @@
 import org.apache.asterix.om.types.ARecordType;
 import org.apache.asterix.om.types.AUnionType;
 import org.apache.asterix.om.types.IAType;
-import org.apache.asterix.om.util.NonTaggedFormatUtil;
+import org.apache.asterix.om.utils.NonTaggedFormatUtil;
 import org.apache.commons.lang3.mutable.Mutable;
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
 import org.apache.hyracks.algebricks.core.algebra.base.ILogicalExpression;
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/IntroduceDynamicTypeCastRule.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/IntroduceDynamicTypeCastRule.java
index e9eb6e8..535edbe 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/IntroduceDynamicTypeCastRule.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/IntroduceDynamicTypeCastRule.java
@@ -31,7 +31,7 @@
 import org.apache.asterix.om.types.ATypeTag;
 import org.apache.asterix.om.types.AUnionType;
 import org.apache.asterix.om.types.IAType;
-import org.apache.asterix.om.util.NonTaggedFormatUtil;
+import org.apache.asterix.om.utils.NonTaggedFormatUtil;
 import org.apache.commons.lang3.mutable.Mutable;
 import org.apache.commons.lang3.mutable.MutableObject;
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/IntroduceSecondaryIndexInsertDeleteRule.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/IntroduceSecondaryIndexInsertDeleteRule.java
index 38e5984..949786a 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/IntroduceSecondaryIndexInsertDeleteRule.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/IntroduceSecondaryIndexInsertDeleteRule.java
@@ -47,7 +47,7 @@
 import org.apache.asterix.om.types.ATypeTag;
 import org.apache.asterix.om.types.BuiltinType;
 import org.apache.asterix.om.types.IAType;
-import org.apache.asterix.om.util.NonTaggedFormatUtil;
+import org.apache.asterix.om.utils.NonTaggedFormatUtil;
 import org.apache.commons.lang3.mutable.Mutable;
 import org.apache.commons.lang3.mutable.MutableObject;
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/LoadRecordFieldsRule.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/LoadRecordFieldsRule.java
index a0fe4de..7336181 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/LoadRecordFieldsRule.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/LoadRecordFieldsRule.java
@@ -30,7 +30,7 @@
 import org.apache.asterix.om.constants.AsterixConstantValue;
 import org.apache.asterix.om.functions.BuiltinFunctions;
 import org.apache.asterix.om.types.ARecordType;
-import org.apache.asterix.om.util.ConstantExpressionUtil;
+import org.apache.asterix.om.utils.ConstantExpressionUtil;
 import org.apache.asterix.optimizer.base.AnalysisUtil;
 import org.apache.commons.lang3.mutable.Mutable;
 import org.apache.commons.lang3.mutable.MutableObject;
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/PushFieldAccessRule.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/PushFieldAccessRule.java
index 73057ae..6ca822a 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/PushFieldAccessRule.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/PushFieldAccessRule.java
@@ -30,7 +30,7 @@
 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.utils.DatasetUtils;
+import org.apache.asterix.metadata.utils.DatasetUtil;
 import org.apache.asterix.om.base.AInt32;
 import org.apache.asterix.om.base.AString;
 import org.apache.asterix.om.base.IAObject;
@@ -38,7 +38,7 @@
 import org.apache.asterix.om.types.ARecordType;
 import org.apache.asterix.om.types.ATypeTag;
 import org.apache.asterix.om.types.IAType;
-import org.apache.asterix.om.util.ConstantExpressionUtil;
+import org.apache.asterix.om.utils.ConstantExpressionUtil;
 import org.apache.asterix.optimizer.base.AnalysisUtil;
 import org.apache.commons.lang3.mutable.Mutable;
 import org.apache.commons.lang3.mutable.MutableObject;
@@ -332,7 +332,7 @@
                                 }
                                 fldName = rt.getFieldNames()[pos];
                             }
-                            int p = DatasetUtils.getPositionOfPartitioningKeyField(dataset, fldName);
+                            int p = DatasetUtil.getPositionOfPartitioningKeyField(dataset, fldName);
                             if (p < 0) { // not one of the partitioning fields
                                 setAsFinal(access, context, finalAnnot);
                                 return false;
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/RemoveUnusedOneToOneEquiJoinRule.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/RemoveUnusedOneToOneEquiJoinRule.java
index 03c7663..a20a33d 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/RemoveUnusedOneToOneEquiJoinRule.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/RemoveUnusedOneToOneEquiJoinRule.java
@@ -26,7 +26,7 @@
 
 import org.apache.asterix.metadata.declared.DatasetDataSource;
 import org.apache.asterix.metadata.entities.InternalDatasetDetails;
-import org.apache.asterix.metadata.utils.DatasetUtils;
+import org.apache.asterix.metadata.utils.DatasetUtil;
 import org.apache.commons.lang3.mutable.Mutable;
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
 import org.apache.hyracks.algebricks.core.algebra.base.ILogicalExpression;
@@ -231,7 +231,7 @@
         DatasetDataSource datasetDataSource = (DatasetDataSource) dataScan.getDataSource();
         pkVars.clear();
         if (datasetDataSource.getDataset().getDatasetDetails() instanceof InternalDatasetDetails) {
-            int numPKs = DatasetUtils.getPartitioningKeys(datasetDataSource.getDataset()).size();
+            int numPKs = DatasetUtil.getPartitioningKeys(datasetDataSource.getDataset()).size();
             for (int i = 0; i < numPKs; i++) {
                 pkVars.add(dataScan.getVariables().get(i));
             }
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/ResolveVariableRule.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/ResolveVariableRule.java
index f8e57a1..4ae26fc 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/ResolveVariableRule.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/ResolveVariableRule.java
@@ -34,7 +34,7 @@
 import org.apache.asterix.om.types.ATypeTag;
 import org.apache.asterix.om.types.AUnionType;
 import org.apache.asterix.om.types.IAType;
-import org.apache.asterix.om.util.ConstantExpressionUtil;
+import org.apache.asterix.om.utils.ConstantExpressionUtil;
 import org.apache.commons.lang3.mutable.Mutable;
 import org.apache.commons.lang3.mutable.MutableObject;
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/UnnestToDataScanRule.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/UnnestToDataScanRule.java
index cdabeaa..74e6ec5 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/UnnestToDataScanRule.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/UnnestToDataScanRule.java
@@ -36,14 +36,14 @@
 import org.apache.asterix.metadata.entities.FeedPolicyEntity;
 import org.apache.asterix.metadata.entities.InternalDatasetDetails;
 import org.apache.asterix.metadata.feeds.BuiltinFeedPolicies;
-import org.apache.asterix.metadata.utils.DatasetUtils;
+import org.apache.asterix.metadata.utils.DatasetUtil;
 import org.apache.asterix.om.base.AString;
 import org.apache.asterix.om.constants.AsterixConstantValue;
 import org.apache.asterix.om.functions.BuiltinFunctions;
 import org.apache.asterix.om.types.ARecordType;
 import org.apache.asterix.om.types.ATypeTag;
 import org.apache.asterix.om.types.IAType;
-import org.apache.asterix.om.util.ConstantExpressionUtil;
+import org.apache.asterix.om.utils.ConstantExpressionUtil;
 import org.apache.asterix.optimizer.rules.util.EquivalenceClassUtils;
 import org.apache.asterix.translator.util.PlanTranslationUtil;
 import org.apache.commons.lang3.mutable.Mutable;
@@ -123,7 +123,7 @@
             DataSourceId asid = new DataSourceId(dataverseName, datasetName);
             List<LogicalVariable> variables = new ArrayList<>();
             if (dataset.getDatasetType() == DatasetType.INTERNAL) {
-                int numPrimaryKeys = DatasetUtils.getPartitioningKeys(dataset).size();
+                int numPrimaryKeys = DatasetUtil.getPartitioningKeys(dataset).size();
                 for (int i = 0; i < numPrimaryKeys; i++) {
                     variables.add(context.newVar());
                 }
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/AbstractIntroduceAccessMethodRule.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/AbstractIntroduceAccessMethodRule.java
index e6cf12d..ca042f1 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/AbstractIntroduceAccessMethodRule.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/AbstractIntroduceAccessMethodRule.java
@@ -28,10 +28,10 @@
 
 import org.apache.asterix.common.config.DatasetConfig.IndexType;
 import org.apache.asterix.dataflow.data.common.ExpressionTypeComputer;
-import org.apache.asterix.metadata.api.IMetadataEntity;
 import org.apache.asterix.metadata.declared.MetadataProvider;
 import org.apache.asterix.metadata.entities.Index;
-import org.apache.asterix.metadata.utils.DatasetUtils;
+import org.apache.asterix.metadata.utils.DatasetUtil;
+import org.apache.asterix.metadata.utils.MetadataUtil;
 import org.apache.asterix.om.base.AOrderedList;
 import org.apache.asterix.om.base.AString;
 import org.apache.asterix.om.constants.AsterixConstantValue;
@@ -42,7 +42,7 @@
 import org.apache.asterix.om.types.BuiltinType;
 import org.apache.asterix.om.types.IAType;
 import org.apache.asterix.om.types.hierachy.ATypeHierarchy;
-import org.apache.asterix.om.util.ConstantExpressionUtil;
+import org.apache.asterix.om.utils.ConstantExpressionUtil;
 import org.apache.asterix.optimizer.rules.am.OptimizableOperatorSubTree.DataSourceType;
 import org.apache.commons.lang3.mutable.Mutable;
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
@@ -76,15 +76,13 @@
     private MetadataProvider metadataProvider;
 
     // Function Identifier sets that retain the original field variable through each function's arguments
-    private final ImmutableSet<FunctionIdentifier> funcIDSetThatRetainFieldName = ImmutableSet.of(
-            BuiltinFunctions.WORD_TOKENS, BuiltinFunctions.GRAM_TOKENS, BuiltinFunctions.SUBSTRING,
-            BuiltinFunctions.SUBSTRING_BEFORE, BuiltinFunctions.SUBSTRING_AFTER,
-            BuiltinFunctions.CREATE_POLYGON, BuiltinFunctions.CREATE_MBR,
-            BuiltinFunctions.CREATE_RECTANGLE, BuiltinFunctions.CREATE_CIRCLE,
-            BuiltinFunctions.CREATE_LINE, BuiltinFunctions.CREATE_POINT,
-            BuiltinFunctions.NUMERIC_ADD, BuiltinFunctions.NUMERIC_SUBTRACT,
-            BuiltinFunctions.NUMERIC_MULTIPLY, BuiltinFunctions.NUMERIC_DIVIDE,
-            BuiltinFunctions.NUMERIC_MOD);
+    private final ImmutableSet<FunctionIdentifier> funcIDSetThatRetainFieldName =
+            ImmutableSet.of(BuiltinFunctions.WORD_TOKENS, BuiltinFunctions.GRAM_TOKENS, BuiltinFunctions.SUBSTRING,
+                    BuiltinFunctions.SUBSTRING_BEFORE, BuiltinFunctions.SUBSTRING_AFTER,
+                    BuiltinFunctions.CREATE_POLYGON, BuiltinFunctions.CREATE_MBR, BuiltinFunctions.CREATE_RECTANGLE,
+                    BuiltinFunctions.CREATE_CIRCLE, BuiltinFunctions.CREATE_LINE, BuiltinFunctions.CREATE_POINT,
+                    BuiltinFunctions.NUMERIC_ADD, BuiltinFunctions.NUMERIC_SUBTRACT, BuiltinFunctions.NUMERIC_MULTIPLY,
+                    BuiltinFunctions.NUMERIC_DIVIDE, BuiltinFunctions.NUMERIC_MOD);
 
     public abstract Map<FunctionIdentifier, List<IAccessMethod>> getAccessMethods();
 
@@ -94,7 +92,7 @@
         for (FunctionIdentifier funcIdent : funcs) {
             List<IAccessMethod> l = accessMethods.get(funcIdent);
             if (l == null) {
-                l = new ArrayList<IAccessMethod>();
+                l = new ArrayList<>();
                 accessMethods.put(funcIdent, l);
             }
             l.add(accessMethod);
@@ -112,7 +110,7 @@
 
     protected void fillSubTreeIndexExprs(OptimizableOperatorSubTree subTree,
             Map<IAccessMethod, AccessMethodAnalysisContext> analyzedAMs, IOptimizationContext context)
-                    throws AlgebricksException {
+            throws AlgebricksException {
         Iterator<Map.Entry<IAccessMethod, AccessMethodAnalysisContext>> amIt = analyzedAMs.entrySet().iterator();
         // Check applicability of indexes by access method type.
         while (amIt.hasNext()) {
@@ -159,15 +157,15 @@
      */
     protected List<Pair<IAccessMethod, Index>> chooseAllIndex(
             Map<IAccessMethod, AccessMethodAnalysisContext> analyzedAMs) {
-        List<Pair<IAccessMethod, Index>> result = new ArrayList<Pair<IAccessMethod, Index>>();
+        List<Pair<IAccessMethod, Index>> result = new ArrayList<>();
         // Use variables (fields) to the index types map to check which type of indexes are applied for the vars.
         Map<List<Pair<Integer, Integer>>, List<IndexType>> resultVarsToIndexTypesMap = new HashMap<>();
         Iterator<Map.Entry<IAccessMethod, AccessMethodAnalysisContext>> amIt = analyzedAMs.entrySet().iterator();
         while (amIt.hasNext()) {
             Map.Entry<IAccessMethod, AccessMethodAnalysisContext> amEntry = amIt.next();
             AccessMethodAnalysisContext analysisCtx = amEntry.getValue();
-            Iterator<Map.Entry<Index, List<Pair<Integer, Integer>>>> indexIt = analysisCtx.indexExprsAndVars.entrySet()
-                    .iterator();
+            Iterator<Map.Entry<Index, List<Pair<Integer, Integer>>>> indexIt =
+                    analysisCtx.indexExprsAndVars.entrySet().iterator();
             while (indexIt.hasNext()) {
                 Map.Entry<Index, List<Pair<Integer, Integer>>> indexEntry = indexIt.next();
                 IAccessMethod chosenAccessMethod = amEntry.getKey();
@@ -184,13 +182,13 @@
                         List<IndexType> appliedIndexTypes = resultVarsToIndexTypesMap.get(indexEntry.getValue());
                         if (!appliedIndexTypes.contains(indexType)) {
                             appliedIndexTypes.add(indexType);
-                            result.add(new Pair<IAccessMethod, Index>(chosenAccessMethod, chosenIndex));
+                            result.add(new Pair<>(chosenAccessMethod, chosenIndex));
                         }
                     } else {
                         List<IndexType> addedIndexTypes = new ArrayList<>();
                         addedIndexTypes.add(indexType);
                         resultVarsToIndexTypesMap.put(indexEntry.getValue(), addedIndexTypes);
-                        result.add(new Pair<IAccessMethod, Index>(chosenAccessMethod, chosenIndex));
+                        result.add(new Pair<>(chosenAccessMethod, chosenIndex));
                     }
                 }
             }
@@ -210,11 +208,11 @@
      */
     public void pruneIndexCandidates(IAccessMethod accessMethod, AccessMethodAnalysisContext analysisCtx,
             IOptimizationContext context, IVariableTypeEnvironment typeEnvironment) throws AlgebricksException {
-        Iterator<Map.Entry<Index, List<Pair<Integer, Integer>>>> indexExprAndVarIt = analysisCtx.indexExprsAndVars
-                .entrySet().iterator();
+        Iterator<Map.Entry<Index, List<Pair<Integer, Integer>>>> indexExprAndVarIt =
+                analysisCtx.indexExprsAndVars.entrySet().iterator();
         // Used to keep track of matched expressions (added for prefix search)
         int numMatchedKeys = 0;
-        ArrayList<Integer> matchedExpressions = new ArrayList<Integer>();
+        ArrayList<Integer> matchedExpressions = new ArrayList<>();
         while (indexExprAndVarIt.hasNext()) {
             Map.Entry<Index, List<Pair<Integer, Integer>>> indexExprAndVarEntry = indexExprAndVarIt.next();
             Index index = indexExprAndVarEntry.getKey();
@@ -252,8 +250,7 @@
                     if (matchedTypes.size() < 2 && optFuncExpr.getNumLogicalVars() == 1) {
                         matchedTypes
                                 .add((IAType) ExpressionTypeComputer.INSTANCE.getType(optFuncExpr.getConstantExpr(0),
-                                        context.getMetadataProvider(),
-                                typeEnvironment));
+                                        context.getMetadataProvider(), typeEnvironment));
                     }
 
                     //infer type of logicalExpr based on index keyType
@@ -271,7 +268,7 @@
                                 @Override
                                 public Object getVarType(LogicalVariable var, List<LogicalVariable> nonNullVariables,
                                         List<List<LogicalVariable>> correlatedNullableVariableLists)
-                                                throws AlgebricksException {
+                                        throws AlgebricksException {
                                     if (var.equals(optFuncExpr.getSourceVar(exprAndVarIdx.second))) {
                                         return keyType;
                                     }
@@ -321,8 +318,8 @@
 
                     // Check if any field name in the optFuncExpr matches.
                     if (optFuncExpr.findFieldName(keyField) != -1) {
-                        foundKeyField = typeMatch
-                                && optFuncExpr.getOperatorSubTree(exprAndVarIdx.second).hasDataSourceScan();
+                        foundKeyField =
+                                typeMatch && optFuncExpr.getOperatorSubTree(exprAndVarIdx.second).hasDataSourceScan();
                         if (foundKeyField) {
                             matchedExpressions.add(exprAndVarIdx.first);
                             numMatchedKeys++;
@@ -401,8 +398,8 @@
                 continue;
             }
             AbstractFunctionCallExpression argFuncExpr = (AbstractFunctionCallExpression) argExpr;
-            boolean matchFound = analyzeFunctionExpr(argFuncExpr, assignsAndUnnests, analyzedAMs, context,
-                    typeEnvironment);
+            boolean matchFound =
+                    analyzeFunctionExpr(argFuncExpr, assignsAndUnnests, analyzedAMs, context, typeEnvironment);
             found = found || matchFound;
         }
         return found;
@@ -467,13 +464,13 @@
     protected boolean fillIndexExprs(List<Index> datasetIndexes, List<String> fieldName, IAType fieldType,
             IOptimizableFuncExpr optFuncExpr, int matchedFuncExprIndex, int varIdx,
             OptimizableOperatorSubTree matchedSubTree, AccessMethodAnalysisContext analysisCtx)
-                    throws AlgebricksException {
-        List<Index> indexCandidates = new ArrayList<Index>();
+            throws AlgebricksException {
+        List<Index> indexCandidates = new ArrayList<>();
         // Add an index to the candidates if one of the indexed fields is
         // fieldName
         for (Index index : datasetIndexes) {
             // Need to also verify the index is pending no op
-            if (index.getKeyFieldNames().contains(fieldName) && index.getPendingOp() == IMetadataEntity.PENDING_NO_OP) {
+            if (index.getKeyFieldNames().contains(fieldName) && index.getPendingOp() == MetadataUtil.PENDING_NO_OP) {
                 indexCandidates.add(index);
                 if (optFuncExpr.getFieldType(varIdx) == BuiltinType.AMISSING
                         || optFuncExpr.getFieldType(varIdx) == BuiltinType.ANY) {
@@ -493,7 +490,7 @@
     protected void fillAllIndexExprs(OptimizableOperatorSubTree subTree, AccessMethodAnalysisContext analysisCtx,
             IOptimizationContext context) throws AlgebricksException {
         int optFuncExprIndex = 0;
-        List<Index> datasetIndexes = new ArrayList<Index>();
+        List<Index> datasetIndexes = new ArrayList<>();
         LogicalVariable datasetMetaVar = null;
         LogicalVariable datasetRecordVar = null;
         if (subTree.getDataSourceType() != DataSourceType.COLLECTION_SCAN) {
@@ -532,7 +529,7 @@
             List<LogicalVariable> additionalDsVarList = null;
 
             if (subTree.hasIxJoinOuterAdditionalDataSource()) {
-                additionalDsVarList = new ArrayList<LogicalVariable>();
+                additionalDsVarList = new ArrayList<>();
                 for (int i = 0; i < subTree.getIxJoinOuterAdditionalDataSourceRefs().size(); i++) {
                     additionalDsVarList.addAll(subTree.getIxJoinOuterAdditionalDataSourceVariables(i));
                 }
@@ -571,8 +568,8 @@
                 return;
             }
         }
-        IAType fieldType = (IAType) context.getOutputTypeEnvironment(unnestOp)
-                .getType(optFuncExpr.getLogicalExpr(funcVarIndex));
+        IAType fieldType =
+                (IAType) context.getOutputTypeEnvironment(unnestOp).getType(optFuncExpr.getLogicalExpr(funcVarIndex));
         // Set the fieldName in the corresponding matched function
         // expression.
         optFuncExpr.setFieldName(funcVarIndex, fieldName);
@@ -624,7 +621,7 @@
     private void matchVarsFromOptFuncExprToDataSourceScan(IOptimizableFuncExpr optFuncExpr, int optFuncExprIndex,
             List<Index> datasetIndexes, List<LogicalVariable> dsVarList, OptimizableOperatorSubTree subTree,
             AccessMethodAnalysisContext analysisCtx, IOptimizationContext context, boolean fromAdditionalDataSource)
-                    throws AlgebricksException {
+            throws AlgebricksException {
         for (int varIndex = 0; varIndex < dsVarList.size(); varIndex++) {
             LogicalVariable var = dsVarList.get(varIndex);
             int funcVarIndex = optFuncExpr.findLogicalVar(var);
@@ -638,7 +635,7 @@
             List<List<String>> subTreePKs = null;
 
             if (!fromAdditionalDataSource) {
-                subTreePKs = DatasetUtils.getPartitioningKeys(subTree.getDataset());
+                subTreePKs = DatasetUtil.getPartitioningKeys(subTree.getDataset());
                 // Check whether this variable is PK, not a record variable.
                 if (varIndex <= subTreePKs.size() - 1) {
                     fieldName = subTreePKs.get(varIndex);
@@ -649,8 +646,8 @@
                 // Need to check additional dataset one by one
                 for (int i = 0; i < subTree.getIxJoinOuterAdditionalDatasets().size(); i++) {
                     if (subTree.getIxJoinOuterAdditionalDatasets().get(i) != null) {
-                        subTreePKs = DatasetUtils
-                                .getPartitioningKeys(subTree.getIxJoinOuterAdditionalDatasets().get(i));
+                        subTreePKs =
+                                DatasetUtil.getPartitioningKeys(subTree.getIxJoinOuterAdditionalDatasets().get(i));
 
                         // Check whether this variable is PK, not a record variable.
                         if (subTreePKs.contains(var) && varIndex <= subTreePKs.size() - 1) {
@@ -693,10 +690,10 @@
      *
      * @throws AlgebricksException
      */
-    protected List<String> getFieldNameFromSubTree(IOptimizableFuncExpr optFuncExpr, OptimizableOperatorSubTree subTree,
-            int opIndex, int assignVarIndex, ARecordType recordType, int funcVarIndex,
-            ILogicalExpression parentFuncExpr, LogicalVariable recordVar, ARecordType metaType, LogicalVariable metaVar)
-                    throws AlgebricksException {
+    protected List<String> getFieldNameFromSubTree(IOptimizableFuncExpr optFuncExpr,
+            OptimizableOperatorSubTree subTree, int opIndex, int assignVarIndex, ARecordType recordType,
+            int funcVarIndex, ILogicalExpression parentFuncExpr, LogicalVariable recordVar, ARecordType metaType,
+            LogicalVariable metaVar) throws AlgebricksException {
         // Get expression corresponding to opVar at varIndex.
         AbstractLogicalExpression expr = null;
         AbstractFunctionCallExpression childFuncExpr = null;
@@ -752,9 +749,9 @@
                 return Collections.emptyList();
             }
             ConstantExpression constExpr = (ConstantExpression) nameArg;
-            AOrderedList orderedNestedFieldName = (AOrderedList) ((AsterixConstantValue) constExpr.getValue())
-                    .getObject();
-            nestedAccessFieldName = new ArrayList<String>();
+            AOrderedList orderedNestedFieldName =
+                    (AOrderedList) ((AsterixConstantValue) constExpr.getValue()).getObject();
+            nestedAccessFieldName = new ArrayList<>();
             for (int i = 0; i < orderedNestedFieldName.size(); i++) {
                 nestedAccessFieldName.add(((AString) orderedNestedFieldName.getItem(i)).getStringValue());
             }
@@ -762,8 +759,8 @@
             isByName = true;
         }
         if (isFieldAccess) {
-            LogicalVariable sourceVar = ((VariableReferenceExpression) funcExpr.getArguments().get(0).getValue())
-                    .getVariableReference();
+            LogicalVariable sourceVar =
+                    ((VariableReferenceExpression) funcExpr.getArguments().get(0).getValue()).getVariableReference();
             optFuncExpr.setLogicalExpr(funcVarIndex, parentFuncExpr);
             int[] assignAndExpressionIndexes = null;
 
@@ -785,7 +782,7 @@
                 //Go through variables in assign to check for match
                 for (int varIndex = 0; varIndex < varList.size(); varIndex++) {
                     LogicalVariable var = varList.get(varIndex);
-                    ArrayList<LogicalVariable> parentVars = new ArrayList<LogicalVariable>();
+                    ArrayList<LogicalVariable> parentVars = new ArrayList<>();
                     expr.getUsedVariables(parentVars);
 
                     if (parentVars.contains(var)) {
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/AccessMethodJobGenParams.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/AccessMethodJobGenParams.java
index 331922d..e541a79 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/AccessMethodJobGenParams.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/AccessMethodJobGenParams.java
@@ -35,6 +35,7 @@
  * and from a list of function arguments, typically of an unnest-map.
  */
 public class AccessMethodJobGenParams {
+    private static final int NUM_PARAMS = 6;
     protected String indexName;
     protected IndexType indexType;
     protected String dataverseName;
@@ -43,9 +44,8 @@
     protected boolean requiresBroadcast;
     protected boolean isPrimaryIndex;
 
-    private final int NUM_PARAMS = 6;
-
     public AccessMethodJobGenParams() {
+        // Enable creation of an empty object and fill members using setters
     }
 
     public AccessMethodJobGenParams(String indexName, IndexType indexType, String dataverseName, String datasetName,
@@ -60,12 +60,12 @@
     }
 
     public void writeToFuncArgs(List<Mutable<ILogicalExpression>> funcArgs) {
-        funcArgs.add(new MutableObject<ILogicalExpression>(AccessMethodUtils.createStringConstant(indexName)));
-        funcArgs.add(new MutableObject<ILogicalExpression>(AccessMethodUtils.createInt32Constant(indexType.ordinal())));
-        funcArgs.add(new MutableObject<ILogicalExpression>(AccessMethodUtils.createStringConstant(dataverseName)));
-        funcArgs.add(new MutableObject<ILogicalExpression>(AccessMethodUtils.createStringConstant(datasetName)));
-        funcArgs.add(new MutableObject<ILogicalExpression>(AccessMethodUtils.createBooleanConstant(retainInput)));
-        funcArgs.add(new MutableObject<ILogicalExpression>(AccessMethodUtils.createBooleanConstant(requiresBroadcast)));
+        funcArgs.add(new MutableObject<>(AccessMethodUtils.createStringConstant(indexName)));
+        funcArgs.add(new MutableObject<>(AccessMethodUtils.createInt32Constant(indexType.ordinal())));
+        funcArgs.add(new MutableObject<>(AccessMethodUtils.createStringConstant(dataverseName)));
+        funcArgs.add(new MutableObject<>(AccessMethodUtils.createStringConstant(datasetName)));
+        funcArgs.add(new MutableObject<>(AccessMethodUtils.createBooleanConstant(retainInput)));
+        funcArgs.add(new MutableObject<>(AccessMethodUtils.createBooleanConstant(requiresBroadcast)));
     }
 
     public void readFromFuncArgs(List<Mutable<ILogicalExpression>> funcArgs) {
@@ -103,12 +103,11 @@
     }
 
     protected void writeVarList(List<LogicalVariable> varList, List<Mutable<ILogicalExpression>> funcArgs) {
-        Mutable<ILogicalExpression> numKeysRef = new MutableObject<ILogicalExpression>(
-                new ConstantExpression(new AsterixConstantValue(new AInt32(varList.size()))));
+        Mutable<ILogicalExpression> numKeysRef =
+                new MutableObject<>(new ConstantExpression(new AsterixConstantValue(new AInt32(varList.size()))));
         funcArgs.add(numKeysRef);
         for (LogicalVariable keyVar : varList) {
-            Mutable<ILogicalExpression> keyVarRef = new MutableObject<ILogicalExpression>(
-                    new VariableReferenceExpression(keyVar));
+            Mutable<ILogicalExpression> keyVarRef = new MutableObject<>(new VariableReferenceExpression(keyVar));
             funcArgs.add(keyVarRef);
         }
     }
@@ -117,8 +116,8 @@
         int numLowKeys = AccessMethodUtils.getInt32Constant(funcArgs.get(index));
         if (numLowKeys > 0) {
             for (int i = 0; i < numLowKeys; i++) {
-                LogicalVariable var = ((VariableReferenceExpression) funcArgs.get(index + 1 + i).getValue())
-                        .getVariableReference();
+                LogicalVariable var =
+                        ((VariableReferenceExpression) funcArgs.get(index + 1 + i).getValue()).getVariableReference();
                 varList.add(var);
             }
         }
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/AccessMethodUtils.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/AccessMethodUtils.java
index 189f37e..7743d9f 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/AccessMethodUtils.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/AccessMethodUtils.java
@@ -36,8 +36,8 @@
 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.metadata.utils.DatasetUtils;
-import org.apache.asterix.metadata.utils.KeyFieldTypeUtils;
+import org.apache.asterix.metadata.utils.DatasetUtil;
+import org.apache.asterix.metadata.utils.KeyFieldTypeUtil;
 import org.apache.asterix.om.base.ABoolean;
 import org.apache.asterix.om.base.AInt32;
 import org.apache.asterix.om.base.AString;
@@ -50,7 +50,7 @@
 import org.apache.asterix.om.types.BuiltinType;
 import org.apache.asterix.om.types.IAType;
 import org.apache.asterix.om.types.hierachy.ATypeHierarchy;
-import org.apache.asterix.om.util.ConstantExpressionUtil;
+import org.apache.asterix.om.utils.ConstantExpressionUtil;
 import org.apache.commons.lang3.mutable.Mutable;
 import org.apache.commons.lang3.mutable.MutableObject;
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
@@ -76,7 +76,6 @@
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.GroupByOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.LeftOuterUnnestMapOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.OrderOperator;
-import org.apache.hyracks.algebricks.core.algebra.operators.logical.OrderOperator.IOrder;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.SelectOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.UnnestMapOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.visitors.VariableUtilities;
@@ -93,7 +92,7 @@
             List<Object> target) throws AlgebricksException {
         ARecordType recordType = (ARecordType) itemType;
         ARecordType metaRecordType = (ARecordType) metaItemType;
-        target.addAll(KeyFieldTypeUtils.getPartitoningKeyTypes(dataset, recordType, metaRecordType));
+        target.addAll(KeyFieldTypeUtil.getPartitoningKeyTypes(dataset, recordType, metaRecordType));
         // Adds data record type.
         target.add(itemType);
         // Adds meta record type if any.
@@ -281,10 +280,10 @@
         if (!primaryKeysOnly) {
             switch (index.getIndexType()) {
                 case BTREE:
-                    dest.addAll(KeyFieldTypeUtils.getBTreeIndexKeyTypes(index, recordType, metaRecordType));
+                    dest.addAll(KeyFieldTypeUtil.getBTreeIndexKeyTypes(index, recordType, metaRecordType));
                     break;
                 case RTREE:
-                    dest.addAll(KeyFieldTypeUtils.getRTreeIndexKeyTypes(index, recordType, metaRecordType));
+                    dest.addAll(KeyFieldTypeUtil.getRTreeIndexKeyTypes(index, recordType, metaRecordType));
                     break;
                 case SINGLE_PARTITION_WORD_INVIX:
                 case SINGLE_PARTITION_NGRAM_INVIX:
@@ -303,7 +302,7 @@
                 throw new AlgebricksException(e);
             }
         } else {
-            dest.addAll(KeyFieldTypeUtils.getPartitoningKeyTypes(dataset, recordType, metaRecordType));
+            dest.addAll(KeyFieldTypeUtil.getPartitoningKeyTypes(dataset, recordType, metaRecordType));
         }
     }
 
@@ -315,9 +314,9 @@
             numPrimaryKeys = IndexingConstants
                     .getRIDSize(((ExternalDatasetDetails) dataset.getDatasetDetails()).getProperties());
         } else {
-            numPrimaryKeys = DatasetUtils.getPartitioningKeys(dataset).size();
+            numPrimaryKeys = DatasetUtil.getPartitioningKeys(dataset).size();
         }
-        int numSecondaryKeys = KeyFieldTypeUtils.getNumSecondaryKeys(index, recordType, metaRecordType);
+        int numSecondaryKeys = KeyFieldTypeUtil.getNumSecondaryKeys(index, recordType, metaRecordType);
         int numVars = (primaryKeysOnly) ? numPrimaryKeys : numPrimaryKeys + numSecondaryKeys;
         for (int i = 0; i < numVars; i++) {
             dest.add(context.newVar());
@@ -331,9 +330,9 @@
             numPrimaryKeys = IndexingConstants
                     .getRIDSize(((ExternalDatasetDetails) dataset.getDatasetDetails()).getProperties());
         } else {
-            numPrimaryKeys = DatasetUtils.getPartitioningKeys(dataset).size();
+            numPrimaryKeys = DatasetUtil.getPartitioningKeys(dataset).size();
         }
-        List<LogicalVariable> primaryKeyVars = new ArrayList<LogicalVariable>();
+        List<LogicalVariable> primaryKeyVars = new ArrayList<>();
         List<LogicalVariable> sourceVars = null;
 
         sourceVars = ((AbstractUnnestMapOperator) unnestMapOp).getVariables();
@@ -349,8 +348,8 @@
 
     public static List<LogicalVariable> getPrimaryKeyVarsFromPrimaryUnnestMap(Dataset dataset,
             ILogicalOperator unnestMapOp) {
-        int numPrimaryKeys = DatasetUtils.getPartitioningKeys(dataset).size();
-        List<LogicalVariable> primaryKeyVars = new ArrayList<LogicalVariable>();
+        int numPrimaryKeys = DatasetUtil.getPartitioningKeys(dataset).size();
+        List<LogicalVariable> primaryKeyVars = new ArrayList<>();
         List<LogicalVariable> sourceVars = null;
 
         // For a left outer join case, LEFT_OUTER_UNNEST_MAP operator is placed
@@ -467,11 +466,11 @@
             IOptimizationContext context, boolean outputPrimaryKeysOnly, boolean retainInput, boolean retainNull)
             throws AlgebricksException {
         // The job gen parameters are transferred to the actual job gen via the UnnestMapOperator's function arguments.
-        ArrayList<Mutable<ILogicalExpression>> secondaryIndexFuncArgs = new ArrayList<Mutable<ILogicalExpression>>();
+        ArrayList<Mutable<ILogicalExpression>> secondaryIndexFuncArgs = new ArrayList<>();
         jobGenParams.writeToFuncArgs(secondaryIndexFuncArgs);
         // Variables and types coming out of the secondary-index search.
-        List<LogicalVariable> secondaryIndexUnnestVars = new ArrayList<LogicalVariable>();
-        List<Object> secondaryIndexOutputTypes = new ArrayList<Object>();
+        List<LogicalVariable> secondaryIndexUnnestVars = new ArrayList<>();
+        List<Object> secondaryIndexOutputTypes = new ArrayList<>();
         // Append output variables/types generated by the secondary-index search (not forwarded from input).
         appendSecondaryIndexOutputVars(dataset, recordType, metaRecordType, index, outputPrimaryKeysOnly, context,
                 secondaryIndexUnnestVars);
@@ -492,7 +491,7 @@
                 LeftOuterUnnestMapOperator secondaryIndexLeftOuterUnnestOp = new LeftOuterUnnestMapOperator(
                         secondaryIndexUnnestVars, new MutableObject<ILogicalExpression>(secondaryIndexSearchFunc),
                         secondaryIndexOutputTypes, true);
-                secondaryIndexLeftOuterUnnestOp.getInputs().add(new MutableObject<ILogicalOperator>(inputOp));
+                secondaryIndexLeftOuterUnnestOp.getInputs().add(new MutableObject<>(inputOp));
                 context.computeAndSetTypeEnvironmentForOperator(secondaryIndexLeftOuterUnnestOp);
                 secondaryIndexLeftOuterUnnestOp.setExecutionMode(ExecutionMode.PARTITIONED);
                 return secondaryIndexLeftOuterUnnestOp;
@@ -505,7 +504,7 @@
             UnnestMapOperator secondaryIndexUnnestOp = new UnnestMapOperator(secondaryIndexUnnestVars,
                     new MutableObject<ILogicalExpression>(secondaryIndexSearchFunc), secondaryIndexOutputTypes,
                     retainInput);
-            secondaryIndexUnnestOp.getInputs().add(new MutableObject<ILogicalOperator>(inputOp));
+            secondaryIndexUnnestOp.getInputs().add(new MutableObject<>(inputOp));
             context.computeAndSetTypeEnvironmentForOperator(secondaryIndexUnnestOp);
             secondaryIndexUnnestOp.setExecutionMode(ExecutionMode.PARTITIONED);
             return secondaryIndexUnnestOp;
@@ -523,18 +522,18 @@
         if (sortPrimaryKeys) {
             order = new OrderOperator();
             for (LogicalVariable pkVar : primaryKeyVars) {
-                Mutable<ILogicalExpression> vRef = new MutableObject<ILogicalExpression>(
+                Mutable<ILogicalExpression> vRef = new MutableObject<>(
                         new VariableReferenceExpression(pkVar));
                 order.getOrderExpressions()
-                        .add(new Pair<IOrder, Mutable<ILogicalExpression>>(OrderOperator.ASC_ORDER, vRef));
+                        .add(new Pair<>(OrderOperator.ASC_ORDER, vRef));
             }
             // The secondary-index search feeds into the sort.
-            order.getInputs().add(new MutableObject<ILogicalOperator>(inputOp));
+            order.getInputs().add(new MutableObject<>(inputOp));
             order.setExecutionMode(ExecutionMode.LOCAL);
             context.computeAndSetTypeEnvironmentForOperator(order);
         }
         // The job gen parameters are transferred to the actual job gen via the UnnestMapOperator's function arguments.
-        List<Mutable<ILogicalExpression>> primaryIndexFuncArgs = new ArrayList<Mutable<ILogicalExpression>>();
+        List<Mutable<ILogicalExpression>> primaryIndexFuncArgs = new ArrayList<>();
         BTreeJobGenParams jobGenParams = new BTreeJobGenParams(dataset.getDatasetName(), IndexType.BTREE,
                 dataset.getDataverseName(), dataset.getDatasetName(), retainInput, requiresBroadcast);
         // Set low/high inclusive to true for a point lookup.
@@ -545,8 +544,8 @@
         jobGenParams.setIsEqCondition(true);
         jobGenParams.writeToFuncArgs(primaryIndexFuncArgs);
         // Variables and types coming out of the primary-index search.
-        List<LogicalVariable> primaryIndexUnnestVars = new ArrayList<LogicalVariable>();
-        List<Object> primaryIndexOutputTypes = new ArrayList<Object>();
+        List<LogicalVariable> primaryIndexUnnestVars = new ArrayList<>();
+        List<Object> primaryIndexOutputTypes = new ArrayList<>();
         // Append output variables/types generated by the primary-index search (not forwarded from input).
         primaryIndexUnnestVars.addAll(dataSourceOp.getVariables());
         appendPrimaryIndexTypes(dataset, recordType, metaRecordType, primaryIndexOutputTypes);
@@ -575,7 +574,7 @@
         if (sortPrimaryKeys) {
             primaryIndexUnnestOp.getInputs().add(new MutableObject<ILogicalOperator>(order));
         } else {
-            primaryIndexUnnestOp.getInputs().add(new MutableObject<ILogicalOperator>(inputOp));
+            primaryIndexUnnestOp.getInputs().add(new MutableObject<>(inputOp));
         }
         context.computeAndSetTypeEnvironmentForOperator(primaryIndexUnnestOp);
         primaryIndexUnnestOp.setExecutionMode(ExecutionMode.PARTITIONED);
@@ -653,41 +652,41 @@
     }
 
     private static void writeVarList(List<LogicalVariable> varList, List<Mutable<ILogicalExpression>> funcArgs) {
-        Mutable<ILogicalExpression> numKeysRef = new MutableObject<ILogicalExpression>(
+        Mutable<ILogicalExpression> numKeysRef = new MutableObject<>(
                 new ConstantExpression(new AsterixConstantValue(new AInt32(varList.size()))));
         funcArgs.add(numKeysRef);
         for (LogicalVariable keyVar : varList) {
-            Mutable<ILogicalExpression> keyVarRef = new MutableObject<ILogicalExpression>(
+            Mutable<ILogicalExpression> keyVarRef = new MutableObject<>(
                     new VariableReferenceExpression(keyVar));
             funcArgs.add(keyVarRef);
         }
     }
 
     private static void addStringArg(String argument, List<Mutable<ILogicalExpression>> funcArgs) {
-        Mutable<ILogicalExpression> stringRef = new MutableObject<ILogicalExpression>(
+        Mutable<ILogicalExpression> stringRef = new MutableObject<>(
                 new ConstantExpression(new AsterixConstantValue(new AString(argument))));
         funcArgs.add(stringRef);
     }
 
     public static UnnestMapOperator createExternalDataLookupUnnestMap(AbstractDataSourceOperator dataSourceOp,
             Dataset dataset, ARecordType recordType, ILogicalOperator inputOp, IOptimizationContext context,
-            Index secondaryIndex, boolean retainInput, boolean retainNull) throws AlgebricksException {
+            boolean retainInput, boolean retainNull) throws AlgebricksException {
         List<LogicalVariable> primaryKeyVars = AccessMethodUtils.getPrimaryKeyVarsFromSecondaryUnnestMap(dataset,
                 inputOp);
 
         // add a sort on the RID fields before fetching external data.
         OrderOperator order = new OrderOperator();
         for (LogicalVariable pkVar : primaryKeyVars) {
-            Mutable<ILogicalExpression> vRef = new MutableObject<ILogicalExpression>(
+            Mutable<ILogicalExpression> vRef = new MutableObject<>(
                     new VariableReferenceExpression(pkVar));
             order.getOrderExpressions()
-                    .add(new Pair<IOrder, Mutable<ILogicalExpression>>(OrderOperator.ASC_ORDER, vRef));
+                    .add(new Pair<>(OrderOperator.ASC_ORDER, vRef));
         }
         // The secondary-index search feeds into the sort.
-        order.getInputs().add(new MutableObject<ILogicalOperator>(inputOp));
+        order.getInputs().add(new MutableObject<>(inputOp));
         order.setExecutionMode(ExecutionMode.LOCAL);
         context.computeAndSetTypeEnvironmentForOperator(order);
-        List<Mutable<ILogicalExpression>> externalLookupArgs = new ArrayList<Mutable<ILogicalExpression>>();
+        List<Mutable<ILogicalExpression>> externalLookupArgs = new ArrayList<>();
         //Add dataverse to the arguments
         AccessMethodUtils.addStringArg(dataset.getDataverseName(), externalLookupArgs);
         //Add dataset to the arguments
@@ -696,8 +695,8 @@
         AccessMethodUtils.writeVarList(primaryKeyVars, externalLookupArgs);
 
         // Variables and types coming out of the external access.
-        List<LogicalVariable> externalUnnestVars = new ArrayList<LogicalVariable>();
-        List<Object> outputTypes = new ArrayList<Object>();
+        List<LogicalVariable> externalUnnestVars = new ArrayList<>();
+        List<Object> outputTypes = new ArrayList<>();
         // Append output variables/types generated by the data scan (not forwarded from input).
         externalUnnestVars.addAll(dataSourceOp.getVariables());
         appendExternalRecTypes(dataset, recordType, outputTypes);
@@ -715,7 +714,7 @@
 
         //set the physical operator
         DataSourceId dataSourceId = new DataSourceId(dataset.getDataverseName(), dataset.getDatasetName());
-        unnestOp.setPhysicalOperator(new ExternalDataLookupPOperator(dataSourceId, dataset, recordType, secondaryIndex,
+        unnestOp.setPhysicalOperator(new ExternalDataLookupPOperator(dataSourceId, dataset, recordType,
                 primaryKeyVars, false, retainInput, retainNull));
         return unnestOp;
     }
@@ -723,7 +722,7 @@
     //If the expression is constant at runtime, return the type
     public static IAType constantRuntimeResultType(ILogicalExpression expr, IOptimizationContext context,
             IVariableTypeEnvironment typeEnvironment) throws AlgebricksException {
-        Set<LogicalVariable> usedVariables = new HashSet<LogicalVariable>();
+        Set<LogicalVariable> usedVariables = new HashSet<>();
         expr.getUsedVariables(usedVariables);
         if (usedVariables.size() > 0) {
             return null;
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/BTreeAccessMethod.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/BTreeAccessMethod.java
index 3384ad7..79ef433 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/BTreeAccessMethod.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/BTreeAccessMethod.java
@@ -148,9 +148,9 @@
             select.getInputs().clear();
             if (op != null) {
                 subTree.getDataSourceRef().setValue(primaryIndexUnnestOp);
-                select.getInputs().add(new MutableObject<ILogicalOperator>(op));
+                select.getInputs().add(new MutableObject<>(op));
             } else {
-                select.getInputs().add(new MutableObject<ILogicalOperator>(primaryIndexUnnestOp));
+                select.getInputs().add(new MutableObject<>(primaryIndexUnnestOp));
             }
         } else {
             ((AbstractLogicalOperator) primaryIndexUnnestOp).setExecutionMode(ExecutionMode.PARTITIONED);
@@ -465,10 +465,10 @@
         // Here we generate vars and funcs for assigning the secondary-index keys to be fed into the secondary-index
         // search.
         // List of variables for the assign.
-        ArrayList<LogicalVariable> keyVarList = new ArrayList<LogicalVariable>();
+        ArrayList<LogicalVariable> keyVarList = new ArrayList<>();
         // List of variables and expressions for the assign.
-        ArrayList<LogicalVariable> assignKeyVarList = new ArrayList<LogicalVariable>();
-        ArrayList<Mutable<ILogicalExpression>> assignKeyExprList = new ArrayList<Mutable<ILogicalExpression>>();
+        ArrayList<LogicalVariable> assignKeyVarList = new ArrayList<>();
+        ArrayList<Mutable<ILogicalExpression>> assignKeyExprList = new ArrayList<>();
         int numLowKeys = createKeyVarsAndExprs(numSecondaryKeys, lowKeyLimits, lowKeyExprs, assignKeyVarList,
                 assignKeyExprList, keyVarList, context, constantAtRuntimeExpressions, constAtRuntimeExprVars);
         int numHighKeys = createKeyVarsAndExprs(numSecondaryKeys, highKeyLimits, highKeyExprs, assignKeyVarList,
@@ -487,7 +487,7 @@
             // Assign operator that sets the constant secondary-index search-key fields if necessary.
             AssignOperator assignConstantSearchKeys = new AssignOperator(assignKeyVarList, assignKeyExprList);
             // Input to this assign is the EmptyTupleSource (which the dataSourceScan also must have had as input).
-            assignConstantSearchKeys.getInputs().add(new MutableObject<ILogicalOperator>(
+            assignConstantSearchKeys.getInputs().add(new MutableObject<>(
                     OperatorManipulationUtil.deepCopy(dataSourceOp.getInputs().get(0).getValue())));
             assignConstantSearchKeys.setExecutionMode(dataSourceOp.getExecutionMode());
             inputOp = assignConstantSearchKeys;
@@ -518,7 +518,7 @@
         if (dataset.getDatasetType() == DatasetType.EXTERNAL) {
             // External dataset
             UnnestMapOperator externalDataAccessOp = AccessMethodUtils.createExternalDataLookupUnnestMap(dataSourceOp,
-                    dataset, recordType, secondaryIndexUnnestOp, context, chosenIndex, retainInput, retainNull);
+                    dataset, recordType, secondaryIndexUnnestOp, context, retainInput, retainNull);
             indexSubTree.getDataSourceRef().setValue(externalDataAccessOp);
             return externalDataAccessOp;
         } else if (!isPrimaryIndex) {
@@ -530,7 +530,7 @@
             EquivalenceClassUtils.addEquivalenceClassesForPrimaryIndexAccess(primaryIndexUnnestOp,
                     dataSourceOp.getVariables(), recordType, metaRecordType, dataset, context);
         } else {
-            List<Object> primaryIndexOutputTypes = new ArrayList<Object>();
+            List<Object> primaryIndexOutputTypes = new ArrayList<>();
             AccessMethodUtils.appendPrimaryIndexTypes(dataset, recordType, metaRecordType, primaryIndexOutputTypes);
             List<LogicalVariable> scanVariables = dataSourceOp.getVariables();
 
@@ -540,7 +540,7 @@
             // operator will be removed.
             // If not, we create a new condition based on remaining ones.
             if (!primaryIndexPostProccessingIsNeeded) {
-                List<Mutable<ILogicalExpression>> remainingFuncExprs = new ArrayList<Mutable<ILogicalExpression>>();
+                List<Mutable<ILogicalExpression>> remainingFuncExprs = new ArrayList<>();
                 getNewConditionExprs(conditionRef, replacedFuncExprs, remainingFuncExprs);
                 // Generate new condition.
                 if (!remainingFuncExprs.isEmpty()) {
@@ -562,7 +562,7 @@
             if (conditionRef.getValue() != null) {
                 // The job gen parameters are transferred to the actual job gen
                 // via the UnnestMapOperator's function arguments.
-                List<Mutable<ILogicalExpression>> primaryIndexFuncArgs = new ArrayList<Mutable<ILogicalExpression>>();
+                List<Mutable<ILogicalExpression>> primaryIndexFuncArgs = new ArrayList<>();
                 jobGenParams.writeToFuncArgs(primaryIndexFuncArgs);
                 // An index search is expressed as an unnest-map over an
                 // index-search function.
@@ -591,7 +591,7 @@
                 }
             }
 
-            primaryIndexUnnestOp.getInputs().add(new MutableObject<ILogicalOperator>(inputOp));
+            primaryIndexUnnestOp.getInputs().add(new MutableObject<>(inputOp));
 
             // Adds equivalence classes --- one equivalent class between a primary key
             // variable and a record field-access expression.
@@ -615,12 +615,12 @@
             LogicalVariable keyVar = null;
             if (searchKeyExpr.getExpressionTag() == LogicalExpressionTag.CONSTANT) {
                 keyVar = context.newVar();
-                assignKeyExprList.add(new MutableObject<ILogicalExpression>(searchKeyExpr));
+                assignKeyExprList.add(new MutableObject<>(searchKeyExpr));
                 assignKeyVarList.add(keyVar);
             } else {
                 keyVar = ((VariableReferenceExpression) searchKeyExpr).getVariableReference();
                 if (constExpression != null) {
-                    assignKeyExprList.add(new MutableObject<ILogicalExpression>(constExpression));
+                    assignKeyExprList.add(new MutableObject<>(constExpression));
                     assignKeyVarList.add(constExprVars[i]);
                 }
             }
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/IntroduceLSMComponentFilterRule.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/IntroduceLSMComponentFilterRule.java
index 57af64f..5f9b4ab 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/IntroduceLSMComponentFilterRule.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/IntroduceLSMComponentFilterRule.java
@@ -29,13 +29,13 @@
 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.utils.DatasetUtils;
-import org.apache.asterix.metadata.utils.KeyFieldTypeUtils;
+import org.apache.asterix.metadata.utils.DatasetUtil;
+import org.apache.asterix.metadata.utils.KeyFieldTypeUtil;
 import org.apache.asterix.om.functions.BuiltinFunctions;
 import org.apache.asterix.om.types.ARecordType;
 import org.apache.asterix.om.types.ATypeTag;
 import org.apache.asterix.om.types.IAType;
-import org.apache.asterix.om.util.ConstantExpressionUtil;
+import org.apache.asterix.om.utils.ConstantExpressionUtil;
 import org.apache.commons.lang3.mutable.Mutable;
 import org.apache.commons.lang3.mutable.MutableObject;
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
@@ -91,7 +91,7 @@
         List<String> filterFieldName = null;
         ARecordType recType = null;
         if (dataset != null && dataset.getDatasetType() == DatasetType.INTERNAL) {
-            filterFieldName = DatasetUtils.getFilterField(dataset);
+            filterFieldName = DatasetUtil.getFilterField(dataset);
             IAType itemType = ((MetadataProvider) context.getMetadataProvider())
                     .findType(dataset.getItemTypeDataverseName(), dataset.getItemTypeName());
             if (itemType.getTypeTag() == ATypeTag.RECORD) {
@@ -335,7 +335,7 @@
                         continue;
                     }
                     // The variable value is one of the partitioning fields.
-                    List<String> fieldName = DatasetUtils.getPartitioningKeys(dataset).get(varIndex);
+                    List<String> fieldName = DatasetUtil.getPartitioningKeys(dataset).get(varIndex);
                     if (fieldName == null) {
                         return false;
                     }
@@ -375,10 +375,10 @@
                     IAType metaItemType = ((MetadataProvider) context.getMetadataProvider())
                             .findType(dataset.getMetaItemTypeDataverseName(), dataset.getMetaItemTypeName());
                     ARecordType metaRecType = (ARecordType) metaItemType;
-                    int numSecondaryKeys = KeyFieldTypeUtils.getNumSecondaryKeys(index, recType, metaRecType);
+                    int numSecondaryKeys = KeyFieldTypeUtil.getNumSecondaryKeys(index, recType, metaRecType);
                     List<String> fieldName;
                     if (varIndex >= numSecondaryKeys) {
-                        fieldName = DatasetUtils.getPartitioningKeys(dataset).get(varIndex - numSecondaryKeys);
+                        fieldName = DatasetUtil.getPartitioningKeys(dataset).get(varIndex - numSecondaryKeys);
                     } else {
                         fieldName = index.getKeyFieldNames().get(varIndex);
                     }
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/InvertedIndexAccessMethod.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/InvertedIndexAccessMethod.java
index 30866cf..7328f97 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/InvertedIndexAccessMethod.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/InvertedIndexAccessMethod.java
@@ -47,7 +47,7 @@
 import org.apache.asterix.om.types.AUnionType;
 import org.apache.asterix.om.types.IAType;
 import org.apache.asterix.om.types.hierachy.ATypeHierarchy;
-import org.apache.asterix.om.util.ConstantExpressionUtil;
+import org.apache.asterix.om.utils.ConstantExpressionUtil;
 import org.apache.asterix.runtime.evaluators.functions.FullTextContainsDescriptor;
 import org.apache.commons.lang3.mutable.Mutable;
 import org.apache.commons.lang3.mutable.MutableObject;
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/InvertedIndexJobGenParams.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/InvertedIndexJobGenParams.java
index 05122cd..1ddd1b5 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/InvertedIndexJobGenParams.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/InvertedIndexJobGenParams.java
@@ -118,7 +118,7 @@
         // Read full-text search information.
         isFullTextSearchQuery = AccessMethodUtils.getBooleanConstant(funcArgs.get(index + IS_FULLTEXT_SEARCH_INDEX));
         // Read key var list.
-        keyVarList = new ArrayList<LogicalVariable>();
+        keyVarList = new ArrayList<>();
         readVarList(funcArgs, index + KEY_VAR_INDEX, keyVarList);
         // TODO: We could possibly simplify things if we did read the non-key var list here.
         // We don't need to read the non-key var list.
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/OptimizableOperatorSubTree.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/OptimizableOperatorSubTree.java
index a96bc56..3d7e959 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/OptimizableOperatorSubTree.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/OptimizableOperatorSubTree.java
@@ -24,7 +24,7 @@
 import org.apache.asterix.metadata.declared.DataSource;
 import org.apache.asterix.metadata.declared.MetadataProvider;
 import org.apache.asterix.metadata.entities.Dataset;
-import org.apache.asterix.metadata.utils.DatasetUtils;
+import org.apache.asterix.metadata.utils.DatasetUtil;
 import org.apache.asterix.om.functions.BuiltinFunctions;
 import org.apache.asterix.om.types.ARecordType;
 import org.apache.asterix.om.types.ATypeTag;
@@ -371,7 +371,7 @@
         switch (getDataSourceType()) {
             case DATASOURCE_SCAN:
                 DataSourceScanOperator dataSourceScan = (DataSourceScanOperator) getDataSourceRef().getValue();
-                int numPrimaryKeys = DatasetUtils.getPartitioningKeys(getDataset()).size();
+                int numPrimaryKeys = DatasetUtil.getPartitioningKeys(getDataset()).size();
                 for (int i = 0; i < numPrimaryKeys; i++) {
                     target.add(dataSourceScan.getVariables().get(i));
                 }
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/RTreeAccessMethod.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/RTreeAccessMethod.java
index dc6e91d..44a53e1 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/RTreeAccessMethod.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/RTreeAccessMethod.java
@@ -32,7 +32,7 @@
 import org.apache.asterix.om.functions.BuiltinFunctions;
 import org.apache.asterix.om.types.ARecordType;
 import org.apache.asterix.om.types.IAType;
-import org.apache.asterix.om.util.NonTaggedFormatUtil;
+import org.apache.asterix.om.utils.NonTaggedFormatUtil;
 import org.apache.commons.lang3.mutable.Mutable;
 import org.apache.commons.lang3.mutable.MutableObject;
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
@@ -216,7 +216,7 @@
             AbstractFunctionCallExpression createMBR = new ScalarFunctionCallExpression(
                     FunctionUtil.getFunctionInfo(BuiltinFunctions.CREATE_MBR));
             // Spatial object is the constant from the func expr we are optimizing.
-            createMBR.getArguments().add(new MutableObject<ILogicalExpression>(searchKeyExpr));
+            createMBR.getArguments().add(new MutableObject<>(searchKeyExpr));
             // The number of dimensions.
             createMBR.getArguments().add(new MutableObject<ILogicalExpression>(
                     new ConstantExpression(new AsterixConstantValue(new AInt32(numDimensions)))));
@@ -235,7 +235,7 @@
         if (probeSubTree == null) {
             // We are optimizing a selection query.
             // Input to this assign is the EmptyTupleSource (which the dataSourceScan also must have had as input).
-            assignSearchKeys.getInputs().add(new MutableObject<ILogicalOperator>(
+            assignSearchKeys.getInputs().add(new MutableObject<>(
                     OperatorManipulationUtil.deepCopy(dataSourceOp.getInputs().get(0).getValue())));
             assignSearchKeys.setExecutionMode(dataSourceOp.getExecutionMode());
         } else {
@@ -249,7 +249,7 @@
         // Generate the rest of the upstream plan which feeds the search results into the primary index.
         return dataset.getDatasetType() == DatasetType.EXTERNAL
                 ? AccessMethodUtils.createExternalDataLookupUnnestMap(dataSourceOp, dataset, recordType,
-                        secondaryIndexUnnestOp, context, chosenIndex, retainInput, retainNull)
+                        secondaryIndexUnnestOp, context, retainInput, retainNull)
                 : AccessMethodUtils.createPrimaryIndexUnnestMap(dataSourceOp, dataset, recordType, metaRecordType,
                         secondaryIndexUnnestOp, context, true, retainInput, false, false);
     }
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/typecast/StaticTypeCastUtil.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/typecast/StaticTypeCastUtil.java
index adee58a..2a569b9 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/typecast/StaticTypeCastUtil.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/typecast/StaticTypeCastUtil.java
@@ -38,7 +38,7 @@
 import org.apache.asterix.om.types.AbstractCollectionType;
 import org.apache.asterix.om.types.BuiltinType;
 import org.apache.asterix.om.types.IAType;
-import org.apache.asterix.om.util.NonTaggedFormatUtil;
+import org.apache.asterix.om.utils.NonTaggedFormatUtil;
 import org.apache.commons.lang3.mutable.Mutable;
 import org.apache.commons.lang3.mutable.MutableObject;
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/AbstractLangTranslator.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/AbstractLangTranslator.java
index 37be55b..634824c 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/AbstractLangTranslator.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/AbstractLangTranslator.java
@@ -35,8 +35,8 @@
 import org.apache.asterix.metadata.dataset.hints.DatasetHints;
 import org.apache.asterix.metadata.entities.Dataverse;
 import org.apache.asterix.metadata.utils.MetadataConstants;
-import org.apache.asterix.runtime.util.AppContextInfo;
-import org.apache.asterix.runtime.util.ClusterStateManager;
+import org.apache.asterix.runtime.utils.AppContextInfo;
+import org.apache.asterix.runtime.utils.ClusterStateManager;
 import org.apache.hyracks.algebricks.common.utils.Pair;
 
 /**
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/AqlPlusExpressionToPlanTranslator.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/AqlPlusExpressionToPlanTranslator.java
index 09d8055..5eb5a5f 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/AqlPlusExpressionToPlanTranslator.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/AqlPlusExpressionToPlanTranslator.java
@@ -106,7 +106,7 @@
 import org.apache.asterix.metadata.declared.FileSplitSinkId;
 import org.apache.asterix.metadata.declared.MetadataProvider;
 import org.apache.asterix.metadata.entities.Dataset;
-import org.apache.asterix.metadata.utils.DatasetUtils;
+import org.apache.asterix.metadata.utils.DatasetUtil;
 import org.apache.asterix.om.base.AString;
 import org.apache.asterix.om.constants.AsterixConstantValue;
 import org.apache.asterix.om.functions.BuiltinFunctions;
@@ -271,7 +271,7 @@
             }
             ARecordType itemType =
                     (ARecordType) metadata.findType(dataset.getItemTypeDataverseName(), dataset.getItemTypeName());
-            List<List<String>> partitioningKeys = DatasetUtils.getPartitioningKeys(dataset);
+            List<List<String>> partitioningKeys = DatasetUtil.getPartitioningKeys(dataset);
             ArrayList<LogicalVariable> vars = new ArrayList<LogicalVariable>();
             ArrayList<Mutable<ILogicalExpression>> exprs = new ArrayList<Mutable<ILogicalExpression>>();
             List<Mutable<ILogicalExpression>> varRefsForLoading = new ArrayList<Mutable<ILogicalExpression>>();
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/CompiledStatements.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/CompiledStatements.java
index c21b5db..403c26b 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/CompiledStatements.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/CompiledStatements.java
@@ -18,16 +18,15 @@
  */
 package org.apache.asterix.translator;
 
-import java.util.List;
 import java.util.Map;
 
-import org.apache.asterix.common.config.DatasetConfig.IndexType;
 import org.apache.asterix.external.feed.management.FeedConnectionRequest;
 import org.apache.asterix.lang.common.base.Expression;
 import org.apache.asterix.lang.common.base.Statement;
 import org.apache.asterix.lang.common.expression.VariableExpr;
 import org.apache.asterix.lang.common.statement.Query;
-import org.apache.asterix.om.types.IAType;
+import org.apache.asterix.metadata.entities.Dataset;
+import org.apache.asterix.metadata.entities.Index;
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
 
 /**
@@ -182,62 +181,30 @@
     }
 
     public static class CompiledCreateIndexStatement implements ICompiledDmlStatement {
-        private final String indexName;
-        private final String dataverseName;
-        private final String datasetName;
-        private final List<List<String>> keyFields;
-        private final List<IAType> keyTypes;
-        private final boolean isEnforced;
-        private final IndexType indexType;
+        private final Dataset dataset;
+        private final Index index;
 
-        // Specific to NGram index.
-        private final int gramLength;
-
-        public CompiledCreateIndexStatement(String indexName, String dataverseName, String datasetName,
-                List<List<String>> keyFields, List<IAType> keyTypes, boolean isEnforced, int gramLength,
-                IndexType indexType) {
-            this.indexName = indexName;
-            this.dataverseName = dataverseName;
-            this.datasetName = datasetName;
-            this.keyFields = keyFields;
-            this.keyTypes = keyTypes;
-            this.gramLength = gramLength;
-            this.isEnforced = isEnforced;
-            this.indexType = indexType;
+        public CompiledCreateIndexStatement(Dataset dataset, Index index) {
+            this.dataset = dataset;
+            this.index = index;
         }
 
         @Override
         public String getDatasetName() {
-            return datasetName;
+            return index.getDatasetName();
         }
 
         @Override
         public String getDataverseName() {
-            return dataverseName;
+            return index.getDataverseName();
         }
 
-        public String getIndexName() {
-            return indexName;
+        public Index getIndex() {
+            return index;
         }
 
-        public List<List<String>> getKeyFields() {
-            return keyFields;
-        }
-
-        public List<IAType> getKeyFieldTypes() {
-            return keyTypes;
-        }
-
-        public IndexType getIndexType() {
-            return indexType;
-        }
-
-        public int getGramLength() {
-            return gramLength;
-        }
-
-        public boolean isEnforced() {
-            return isEnforced;
+        public Dataset getDataset() {
+            return dataset;
         }
 
         @Override
@@ -538,49 +505,21 @@
     }
 
     public static class CompiledIndexCompactStatement extends CompiledCompactStatement {
-        private final String indexName;
-        private final List<List<String>> keyFields;
-        private final List<IAType> keyTypes;
-        private final IndexType indexType;
-        private final boolean isEnforced;
+        private Dataset dataset;
+        private Index index;
 
-        // Specific to NGram index.
-        private final int gramLength;
-
-        public CompiledIndexCompactStatement(String dataverseName, String datasetName, String indexName,
-                List<List<String>> keyFields, List<IAType> keyTypes, boolean isEnforced, int gramLength,
-                IndexType indexType) {
-            super(dataverseName, datasetName);
-            this.indexName = indexName;
-            this.keyFields = keyFields;
-            this.keyTypes = keyTypes;
-            this.gramLength = gramLength;
-            this.indexType = indexType;
-            this.isEnforced = isEnforced;
+        public CompiledIndexCompactStatement(Dataset dataset, Index index) {
+            super(dataset.getDataverseName(), dataset.getDatasetName());
+            this.dataset = dataset;
+            this.index = index;
         }
 
-        public String getIndexName() {
-            return indexName;
+        public Dataset getDataset() {
+            return dataset;
         }
 
-        public List<List<String>> getKeyFields() {
-            return keyFields;
-        }
-
-        public List<IAType> getKeyTypes() {
-            return keyTypes;
-        }
-
-        public IndexType getIndexType() {
-            return indexType;
-        }
-
-        public int getGramLength() {
-            return gramLength;
-        }
-
-        public boolean isEnforced() {
-            return isEnforced;
+        public Index getIndex() {
+            return index;
         }
     }
 }
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/IStatementExecutorFactory.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/IStatementExecutorFactory.java
index 05fde09..cf54921 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/IStatementExecutorFactory.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/IStatementExecutorFactory.java
@@ -20,6 +20,7 @@
 
 import java.util.List;
 
+import org.apache.asterix.common.context.IStorageComponentProvider;
 import org.apache.asterix.compiler.provider.ILangCompilationProvider;
 import org.apache.asterix.lang.common.base.Statement;
 
@@ -38,8 +39,11 @@
      * @param conf
      *            request configuration
      * @param compilationProvider
-     * @return an implementation of {@code IStatementExecutor} that is used to execute the passed list of statements
+     *            provides query language related components
+     * @param storageComponentProvider
+     *            provides storage related components
+     * @return an implementation of {@code IStatementExecutor} thaxt is used to execute the passed list of statements
      */
     IStatementExecutor create(List<Statement> statements, SessionConfig conf,
-            ILangCompilationProvider compilationProvider);
+            ILangCompilationProvider compilationProvider, IStorageComponentProvider storageComponentProvider);
 }
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/LangExpressionToPlanTranslator.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/LangExpressionToPlanTranslator.java
index dc1cb65..741ce56 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/LangExpressionToPlanTranslator.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/LangExpressionToPlanTranslator.java
@@ -85,7 +85,7 @@
 import org.apache.asterix.metadata.entities.Function;
 import org.apache.asterix.metadata.entities.InternalDatasetDetails;
 import org.apache.asterix.metadata.functions.ExternalFunctionCompilerUtil;
-import org.apache.asterix.metadata.utils.DatasetUtils;
+import org.apache.asterix.metadata.utils.DatasetUtil;
 import org.apache.asterix.om.base.AInt64;
 import org.apache.asterix.om.base.AString;
 import org.apache.asterix.om.constants.AsterixConstantValue;
@@ -94,7 +94,7 @@
 import org.apache.asterix.om.types.ARecordType;
 import org.apache.asterix.om.types.IAType;
 import org.apache.asterix.runtime.formats.FormatUtils;
-import org.apache.asterix.runtime.util.AppContextInfo;
+import org.apache.asterix.runtime.utils.AppContextInfo;
 import org.apache.asterix.translator.CompiledStatements.CompiledInsertStatement;
 import org.apache.asterix.translator.CompiledStatements.CompiledLoadFromFileStatement;
 import org.apache.asterix.translator.CompiledStatements.CompiledSubscribeFeedStatement;
@@ -199,11 +199,11 @@
                     "Unable to load dataset " + clffs.getDatasetName() + " since it does not exist");
         }
         IAType itemType = metadataProvider.findType(dataset.getItemTypeDataverseName(), dataset.getItemTypeName());
-        IAType metaItemType = metadataProvider.findType(dataset.getMetaItemTypeDataverseName(),
-                dataset.getMetaItemTypeName());
-        DatasetDataSource targetDatasource = validateDatasetInfo(metadataProvider, stmt.getDataverseName(),
-                stmt.getDatasetName());
-        List<List<String>> partitionKeys = DatasetUtils.getPartitioningKeys(targetDatasource.getDataset());
+        IAType metaItemType =
+                metadataProvider.findType(dataset.getMetaItemTypeDataverseName(), dataset.getMetaItemTypeName());
+        DatasetDataSource targetDatasource =
+                validateDatasetInfo(metadataProvider, stmt.getDataverseName(), stmt.getDatasetName());
+        List<List<String>> partitionKeys = DatasetUtil.getPartitioningKeys(targetDatasource.getDataset());
         if (dataset.hasMetaPart()) {
             throw new AlgebricksException(
                     dataset.getDatasetName() + ": load dataset is not supported on Datasets with Meta records");
@@ -254,7 +254,7 @@
             assign.setExplicitOrderingProperty(new LocalOrderProperty(orderColumns));
         }
 
-        List<String> additionalFilteringField = DatasetUtils.getFilterField(targetDatasource.getDataset());
+        List<String> additionalFilteringField = DatasetUtil.getFilterField(targetDatasource.getDataset());
         List<LogicalVariable> additionalFilteringVars;
         List<Mutable<ILogicalExpression>> additionalFilteringAssignExpressions;
         List<Mutable<ILogicalExpression>> additionalFilteringExpressions = null;
@@ -265,8 +265,8 @@
             additionalFilteringExpressions = new ArrayList<>();
             PlanTranslationUtil.prepareVarAndExpression(additionalFilteringField, payloadVar, additionalFilteringVars,
                     additionalFilteringAssignExpressions, additionalFilteringExpressions, context);
-            additionalFilteringAssign = new AssignOperator(additionalFilteringVars,
-                    additionalFilteringAssignExpressions);
+            additionalFilteringAssign =
+                    new AssignOperator(additionalFilteringVars, additionalFilteringAssignExpressions);
         }
 
         InsertDeleteUpsertOperator insertOp = new InsertDeleteUpsertOperator(targetDatasource, payloadRef,
@@ -345,14 +345,15 @@
             ProjectOperator projectOperator = (ProjectOperator) topOp;
             projectOperator.getVariables().set(0, seqVar);
             resVar = seqVar;
-            DatasetDataSource targetDatasource = validateDatasetInfo(metadataProvider, stmt.getDataverseName(),
-                    stmt.getDatasetName());
-            List<Integer> keySourceIndicator = ((InternalDatasetDetails) targetDatasource.getDataset()
-                    .getDatasetDetails()).getKeySourceIndicator();
+            DatasetDataSource targetDatasource =
+                    validateDatasetInfo(metadataProvider, stmt.getDataverseName(), stmt.getDatasetName());
+            List<Integer> keySourceIndicator =
+                    ((InternalDatasetDetails) targetDatasource.getDataset().getDatasetDetails())
+                            .getKeySourceIndicator();
             ArrayList<LogicalVariable> vars = new ArrayList<>();
             ArrayList<Mutable<ILogicalExpression>> exprs = new ArrayList<>();
             List<Mutable<ILogicalExpression>> varRefsForLoading = new ArrayList<>();
-            List<List<String>> partitionKeys = DatasetUtils.getPartitioningKeys(targetDatasource.getDataset());
+            List<List<String>> partitionKeys = DatasetUtil.getPartitioningKeys(targetDatasource.getDataset());
             int numOfPrimaryKeys = partitionKeys.size();
             for (int i = 0; i < numOfPrimaryKeys; i++) {
                 if (keySourceIndicator == null || keySourceIndicator.get(i).intValue() == 0) {
@@ -367,7 +368,7 @@
             }
 
             AssignOperator assign = new AssignOperator(vars, exprs);
-            List<String> additionalFilteringField = DatasetUtils.getFilterField(targetDatasource.getDataset());
+            List<String> additionalFilteringField = DatasetUtil.getFilterField(targetDatasource.getDataset());
             List<LogicalVariable> additionalFilteringVars;
             List<Mutable<ILogicalExpression>> additionalFilteringAssignExpressions;
             List<Mutable<ILogicalExpression>> additionalFilteringExpressions = null;
@@ -380,8 +381,8 @@
                 PlanTranslationUtil.prepareVarAndExpression(additionalFilteringField, resVar, additionalFilteringVars,
                         additionalFilteringAssignExpressions, additionalFilteringExpressions, context);
 
-                additionalFilteringAssign = new AssignOperator(additionalFilteringVars,
-                        additionalFilteringAssignExpressions);
+                additionalFilteringAssign =
+                        new AssignOperator(additionalFilteringVars, additionalFilteringAssignExpressions);
                 additionalFilteringAssign.getInputs().add(new MutableObject<>(topOp));
                 assign.getInputs().add(new MutableObject<>(additionalFilteringAssign));
             } else {
@@ -439,7 +440,7 @@
     private ILogicalOperator translateDelete(DatasetDataSource targetDatasource, Mutable<ILogicalExpression> varRef,
             List<Mutable<ILogicalExpression>> varRefsForLoading,
             List<Mutable<ILogicalExpression>> additionalFilteringExpressions, ILogicalOperator assign)
-                    throws AlgebricksException {
+            throws AlgebricksException {
         if (targetDatasource.getDataset().hasMetaPart()) {
             throw new AlgebricksException(targetDatasource.getDataset().getDatasetName()
                     + ": delete from dataset is not supported on Datasets with Meta records");
@@ -526,8 +527,8 @@
                 feedModificationOp.getInputs().add(assign.getInputs().get(0));
             }
         } else {
-            final InsertDeleteUpsertOperator.Kind opKind = isUpsertFeed ? InsertDeleteUpsertOperator.Kind.UPSERT
-                    : InsertDeleteUpsertOperator.Kind.INSERT;
+            final InsertDeleteUpsertOperator.Kind opKind =
+                    isUpsertFeed ? InsertDeleteUpsertOperator.Kind.UPSERT : InsertDeleteUpsertOperator.Kind.INSERT;
             feedModificationOp = new InsertDeleteUpsertOperator(targetDatasource, varRef, varRefsForLoading,
                     metaExpSingletonList, opKind, false);
             if (isUpsertFeed) {
@@ -553,7 +554,7 @@
             List<String> additionalFilteringField, LogicalVariable unnestVar, ILogicalOperator topOp,
             List<Mutable<ILogicalExpression>> exprs, LogicalVariable resVar, AssignOperator additionalFilteringAssign,
             ICompiledDmlStatement stmt) throws AlgebricksException {
-        if (!targetDatasource.getDataset().allow(topOp, Dataset.OP_UPSERT)) {
+        if (!targetDatasource.getDataset().allow(topOp, DatasetUtil.OP_UPSERT)) {
             throw new AlgebricksException(targetDatasource.getDataset().getDatasetName()
                     + ": upsert into dataset is not supported on Datasets with Meta records");
         }
@@ -597,8 +598,8 @@
                 }
             }
             // A change feed, we don't need the assign to access PKs
-            upsertOp = new InsertDeleteUpsertOperator(targetDatasource, varRef, varRefsForLoading, metaExpSingletonList,
-                    InsertDeleteUpsertOperator.Kind.UPSERT, false);
+            upsertOp = new InsertDeleteUpsertOperator(targetDatasource, varRef, varRefsForLoading,
+                    metaExpSingletonList, InsertDeleteUpsertOperator.Kind.UPSERT, false);
             // Create and add a new variable used for representing the original record
             upsertOp.setPrevRecordVar(context.newVar());
             upsertOp.setPrevRecordType(targetDatasource.getItemType());
@@ -664,8 +665,8 @@
         // Adds the commit operator.
         CompiledInsertStatement compiledInsert = (CompiledInsertStatement) stmt;
         Expression returnExpression = compiledInsert.getReturnExpression();
-        ILogicalOperator rootOperator = new DelegateOperator(
-                new CommitOperator(returnExpression == null ? true : false));
+        ILogicalOperator rootOperator =
+                new DelegateOperator(new CommitOperator(returnExpression == null ? true : false));
         rootOperator.getInputs().add(new MutableObject<>(insertOp));
 
         // Compiles the return expression.
@@ -685,8 +686,8 @@
         context.newVar(compiledInsert.getVar());
         context.setVar(compiledInsert.getVar(),
                 ((VariableReferenceExpression) insertOp.getPayloadExpression().getValue()).getVariableReference());
-        Pair<ILogicalExpression, Mutable<ILogicalOperator>> p = langExprToAlgExpression(returnExpression,
-                    new MutableObject<>(rootOperator));
+        Pair<ILogicalExpression, Mutable<ILogicalOperator>> p =
+                langExprToAlgExpression(returnExpression, new MutableObject<>(rootOperator));
 
         // Adds an assign operator for the returning expression.
         LogicalVariable resultVar = context.newVar();
@@ -714,8 +715,8 @@
         }
         DataSourceId sourceId = new DataSourceId(dataverseName, datasetName);
         IAType itemType = metadataProvider.findType(dataset.getItemTypeDataverseName(), dataset.getItemTypeName());
-        IAType metaItemType = metadataProvider.findType(dataset.getMetaItemTypeDataverseName(),
-                dataset.getMetaItemTypeName());
+        IAType metaItemType =
+                metadataProvider.findType(dataset.getMetaItemTypeDataverseName(), dataset.getMetaItemTypeName());
         INodeDomain domain = metadataProvider.findNodeDomain(dataset.getNodeGroupName());
         return new DatasetDataSource(sourceId, dataset, itemType, metaItemType, DataSource.Type.INTERNAL_DATASET,
                 dataset.getDatasetDetails(), domain);
@@ -723,8 +724,8 @@
 
     private FileSplit getDefaultOutputFileLocation() throws MetadataException {
         String outputDir = System.getProperty("java.io.tmpDir");
-        String filePath = outputDir + System.getProperty("file.separator") + OUTPUT_FILE_PREFIX
-                + outputFileID.incrementAndGet();
+        String filePath =
+                outputDir + System.getProperty("file.separator") + OUTPUT_FILE_PREFIX + outputFileID.incrementAndGet();
         MetadataProperties metadataProperties = AppContextInfo.INSTANCE.getMetadataProperties();
         return new ManagedFileSplit(metadataProperties.getMetadataNodeName(), filePath);
     }
@@ -741,8 +742,8 @@
             returnedOp.getInputs().add(tupSource);
         } else {
             v = context.newVar(lc.getVarExpr());
-            Pair<ILogicalExpression, Mutable<ILogicalOperator>> eo = langExprToAlgExpression(lc.getBindingExpr(),
-                    tupSource);
+            Pair<ILogicalExpression, Mutable<ILogicalOperator>> eo =
+                    langExprToAlgExpression(lc.getBindingExpr(), tupSource);
             returnedOp = new AssignOperator(v, new MutableObject<>(eo.first));
             returnedOp.getInputs().add(eo.second);
         }
@@ -754,11 +755,11 @@
             throws CompilationException {
         Pair<ILogicalExpression, Mutable<ILogicalOperator>> p = langExprToAlgExpression(fa.getExpr(), tupSource);
         LogicalVariable v = context.newVar();
-        AbstractFunctionCallExpression fldAccess = new ScalarFunctionCallExpression(
-                FunctionUtil.getFunctionInfo(BuiltinFunctions.FIELD_ACCESS_BY_NAME));
+        AbstractFunctionCallExpression fldAccess =
+                new ScalarFunctionCallExpression(FunctionUtil.getFunctionInfo(BuiltinFunctions.FIELD_ACCESS_BY_NAME));
         fldAccess.getArguments().add(new MutableObject<>(p.first));
-        ILogicalExpression faExpr = new ConstantExpression(
-                new AsterixConstantValue(new AString(fa.getIdent().getValue())));
+        ILogicalExpression faExpr =
+                new ConstantExpression(new AsterixConstantValue(new AString(fa.getIdent().getValue())));
         fldAccess.getArguments().add(new MutableObject<>(faExpr));
         AssignOperator a = new AssignOperator(v, new MutableObject<>(fldAccess));
         a.getInputs().add(p.second);
@@ -772,12 +773,11 @@
         LogicalVariable v = context.newVar();
         AbstractFunctionCallExpression f;
         if (ia.isAny()) {
-            f = new ScalarFunctionCallExpression(
-                    FunctionUtil.getFunctionInfo(BuiltinFunctions.ANY_COLLECTION_MEMBER));
+            f = new ScalarFunctionCallExpression(FunctionUtil.getFunctionInfo(BuiltinFunctions.ANY_COLLECTION_MEMBER));
             f.getArguments().add(new MutableObject<>(p.first));
         } else {
-            Pair<ILogicalExpression, Mutable<ILogicalOperator>> indexPair = langExprToAlgExpression(ia.getIndexExpr(),
-                    tupSource);
+            Pair<ILogicalExpression, Mutable<ILogicalOperator>> indexPair =
+                    langExprToAlgExpression(ia.getIndexExpr(), tupSource);
             f = new ScalarFunctionCallExpression(FunctionUtil.getFunctionInfo(BuiltinFunctions.GET_ITEM));
             f.getArguments().add(new MutableObject<>(p.first));
             f.getArguments().add(new MutableObject<>(indexPair.first));
@@ -884,8 +884,8 @@
         if (BuiltinFunctions.isBuiltinAggregateFunction(fi)) {
             f = BuiltinFunctions.makeAggregateFunctionExpression(fi, args);
         } else if (BuiltinFunctions.isBuiltinUnnestingFunction(fi)) {
-            UnnestingFunctionCallExpression ufce = new UnnestingFunctionCallExpression(FunctionUtil.getFunctionInfo(fi),
-                    args);
+            UnnestingFunctionCallExpression ufce =
+                    new UnnestingFunctionCallExpression(FunctionUtil.getFunctionInfo(fi), args);
             ufce.setReturnsUniqueValues(BuiltinFunctions.returnsUniqueValues(fi));
             f = ufce;
         } else {
@@ -907,8 +907,9 @@
             List<Pair<Expression, Identifier>> groupFieldList = gc.getGroupFieldList();
             List<Mutable<ILogicalExpression>> groupRecordConstructorArgList = new ArrayList<>();
             for (Pair<Expression, Identifier> groupField : groupFieldList) {
-                ILogicalExpression groupFieldNameExpr = langExprToAlgExpression(
-                        new LiteralExpr(new StringLiteral(groupField.second.getValue())), topOp).first;
+                ILogicalExpression groupFieldNameExpr =
+                        langExprToAlgExpression(new LiteralExpr(new StringLiteral(groupField.second.getValue())),
+                                topOp).first;
                 groupRecordConstructorArgList.add(new MutableObject<>(groupFieldNameExpr));
                 ILogicalExpression groupFieldExpr = langExprToAlgExpression(groupField.first, topOp).first;
                 groupRecordConstructorArgList.add(new MutableObject<>(groupFieldExpr));
@@ -944,8 +945,8 @@
                     new MutableObject<>(new NestedTupleSourceOperator(new MutableObject<>(gOp))));
             List<Mutable<ILogicalExpression>> flArgs = new ArrayList<>(1);
             flArgs.add(new MutableObject<>(listifyInput.first));
-            AggregateFunctionCallExpression fListify = BuiltinFunctions
-                    .makeAggregateFunctionExpression(BuiltinFunctions.LISTIFY, flArgs);
+            AggregateFunctionCallExpression fListify =
+                    BuiltinFunctions.makeAggregateFunctionExpression(BuiltinFunctions.LISTIFY, flArgs);
             LogicalVariable aggVar = context.newVar();
             AggregateOperator agg = new AggregateOperator(mkSingletonArrayList(aggVar),
                     mkSingletonArrayList(new MutableObject<>(fListify)));
@@ -1005,14 +1006,14 @@
         LogicalVariable unnestVar = context.newVar();
         UnnestOperator unnestOp = new UnnestOperator(unnestVar,
                 new MutableObject<>(new UnnestingFunctionCallExpression(
-                        FunctionUtil.getFunctionInfo(BuiltinFunctions.SCAN_COLLECTION),
-                        Collections.singletonList(new MutableObject<>(new VariableReferenceExpression(selectVar))))));
+                        FunctionUtil.getFunctionInfo(BuiltinFunctions.SCAN_COLLECTION), Collections
+                                .singletonList(new MutableObject<>(new VariableReferenceExpression(selectVar))))));
         unnestOp.getInputs().add(new MutableObject<>(assignOp));
 
         // Produces the final result.
         LogicalVariable resultVar = context.newVar();
-        AssignOperator finalAssignOp = new AssignOperator(resultVar,
-                new MutableObject<>(new VariableReferenceExpression(unnestVar)));
+        AssignOperator finalAssignOp =
+                new AssignOperator(resultVar, new MutableObject<>(new VariableReferenceExpression(unnestVar)));
         finalAssignOp.getInputs().add(new MutableObject<>(unnestOp));
         return new Pair<>(finalAssignOp, resultVar);
     }
@@ -1167,7 +1168,8 @@
         firstOp.getInputs().add(topOp);
         topOp = lastOp;
 
-        Pair<ILogicalExpression, Mutable<ILogicalOperator>> eo2 = langExprToAlgExpression(qe.getSatisfiesExpr(), topOp);
+        Pair<ILogicalExpression, Mutable<ILogicalOperator>> eo2 =
+                langExprToAlgExpression(qe.getSatisfiesExpr(), topOp);
 
         AggregateFunctionCallExpression fAgg;
         SelectOperator s;
@@ -1182,8 +1184,7 @@
             s = new SelectOperator(new MutableObject<>(new ScalarFunctionCallExpression(
                     FunctionUtil.getFunctionInfo(AlgebricksBuiltinFunctions.NOT), satExprList)), false, null);
             s.getInputs().add(eo2.second);
-            fAgg = BuiltinFunctions.makeAggregateFunctionExpression(BuiltinFunctions.EMPTY_STREAM,
-                    new ArrayList<>());
+            fAgg = BuiltinFunctions.makeAggregateFunctionExpression(BuiltinFunctions.EMPTY_STREAM, new ArrayList<>());
         }
         LogicalVariable qeVar = context.newVar();
         AggregateOperator a = new AggregateOperator(mkSingletonArrayList(qeVar),
@@ -1210,7 +1211,8 @@
             Pair<ILogicalExpression, Mutable<ILogicalOperator>> eo1 = langExprToAlgExpression(fb.getLeftExpr(), topOp);
             f.getArguments().add(new MutableObject<>(eo1.first));
             topOp = eo1.second;
-            Pair<ILogicalExpression, Mutable<ILogicalOperator>> eo2 = langExprToAlgExpression(fb.getRightExpr(), topOp);
+            Pair<ILogicalExpression, Mutable<ILogicalOperator>> eo2 =
+                    langExprToAlgExpression(fb.getRightExpr(), topOp);
             f.getArguments().add(new MutableObject<>(eo2.first));
             topOp = eo2.second;
         }
@@ -1379,8 +1381,8 @@
             Mutable<ILogicalOperator> topOpRef) throws CompilationException {
         switch (expr.getKind()) {
             case VARIABLE_EXPRESSION:
-                VariableReferenceExpression ve = new VariableReferenceExpression(
-                        context.getVar(((VariableExpr) expr).getVar().getId()));
+                VariableReferenceExpression ve =
+                        new VariableReferenceExpression(context.getVar(((VariableExpr) expr).getVar().getId()));
                 return new Pair<>(ve, topOpRef);
             case LITERAL_EXPRESSION:
                 LiteralExpr val = (LiteralExpr) expr;
@@ -1417,8 +1419,8 @@
 
     protected Pair<ILogicalOperator, LogicalVariable> aggListifyForSubquery(LogicalVariable var,
             Mutable<ILogicalOperator> opRef, boolean bProject) {
-        AggregateFunctionCallExpression funAgg = BuiltinFunctions
-                .makeAggregateFunctionExpression(BuiltinFunctions.LISTIFY, new ArrayList<>());
+        AggregateFunctionCallExpression funAgg =
+                BuiltinFunctions.makeAggregateFunctionExpression(BuiltinFunctions.LISTIFY, new ArrayList<>());
         funAgg.getArguments().add(new MutableObject<>(new VariableReferenceExpression(var)));
 
         LogicalVariable varListified = context.newSubplanOutputVar();
@@ -1575,10 +1577,10 @@
                     // There is a shared operator reference in the query plan.
                     // Deep copies the child plan.
                     LogicalOperatorDeepCopyWithNewVariablesVisitor visitor =
-                        new LogicalOperatorDeepCopyWithNewVariablesVisitor(context, null);
+                            new LogicalOperatorDeepCopyWithNewVariablesVisitor(context, null);
                     ILogicalOperator newChild = childRef.getValue().accept(visitor, null);
-                    LinkedHashMap<LogicalVariable, LogicalVariable> cloneVarMap = visitor
-                            .getInputToOutputVariableMapping();
+                    LinkedHashMap<LogicalVariable, LogicalVariable> cloneVarMap =
+                            visitor.getInputToOutputVariableMapping();
 
                     // Substitute variables according to the deep copy which generates new variables.
                     VariableUtilities.substituteVariables(currentOperator, cloneVarMap, null);
@@ -1591,8 +1593,8 @@
 
                 // Recursively eliminate shared operator reference for the operator subtree,
                 // even if it is a deep copy of some other one.
-                LinkedHashMap<LogicalVariable, LogicalVariable> childVarMap = eliminateSharedOperatorReference(childRef,
-                        opRefSet);
+                LinkedHashMap<LogicalVariable, LogicalVariable> childVarMap =
+                        eliminateSharedOperatorReference(childRef, opRefSet);
                 // Substitute variables according to the new subtree.
                 VariableUtilities.substituteVariables(currentOperator, childVarMap, null);
 
@@ -1635,8 +1637,8 @@
         context.enterSubplan();
         SubplanOperator subplanOp = new SubplanOperator();
         subplanOp.getInputs().add(new MutableObject<>(inputOp));
-        Mutable<ILogicalOperator> nestedSource = new MutableObject<>(
-                new NestedTupleSourceOperator(new MutableObject<>(subplanOp)));
+        Mutable<ILogicalOperator> nestedSource =
+                new MutableObject<>(new NestedTupleSourceOperator(new MutableObject<>(subplanOp)));
         SelectOperator select = new SelectOperator(selectExpr, false, null);
         // The select operator cannot be moved up and down, otherwise it will cause typing issues (ASTERIXDB-1203).
         OperatorPropertiesUtil.markMovable(select, false);
@@ -1656,8 +1658,8 @@
 
     // Processes EXISTS and NOT EXISTS.
     private AssignOperator processExists(ILogicalExpression inputExpr, LogicalVariable v1, boolean not) {
-        AbstractFunctionCallExpression count = new ScalarFunctionCallExpression(
-                FunctionUtil.getFunctionInfo(BuiltinFunctions.SCALAR_COUNT));
+        AbstractFunctionCallExpression count =
+                new ScalarFunctionCallExpression(FunctionUtil.getFunctionInfo(BuiltinFunctions.SCALAR_COUNT));
         count.getArguments().add(new MutableObject<>(inputExpr));
         AbstractFunctionCallExpression comparison = new ScalarFunctionCallExpression(
                 FunctionUtil.getFunctionInfo(not ? BuiltinFunctions.EQ : BuiltinFunctions.NEQ));
@@ -1678,9 +1680,8 @@
         }
         Mutable<ILogicalExpression> hasBeenExecutedExprRef = new MutableObject<>(
                 new ScalarFunctionCallExpression(FunctionUtil.getFunctionInfo(BuiltinFunctions.OR), arguments));
-        return new MutableObject<>(
-                new ScalarFunctionCallExpression(FunctionUtil.getFunctionInfo(BuiltinFunctions.NOT),
-                        new ArrayList<>(Collections.singletonList(hasBeenExecutedExprRef))));
+        return new MutableObject<>(new ScalarFunctionCallExpression(FunctionUtil.getFunctionInfo(BuiltinFunctions.NOT),
+                new ArrayList<>(Collections.singletonList(hasBeenExecutedExprRef))));
     }
 
     // For an input expression `expr`, return `expr AND expr IS NOT UNKOWN`.
@@ -1729,8 +1730,8 @@
         while (inputOpRefIterator.hasNext()) {
             // Generates the variable triple <leftVar, rightVar, outputVar> .
             topUnionVar = context.newVar();
-            Triple<LogicalVariable, LogicalVariable, LogicalVariable> varTriple = new Triple<>(leftInputVar,
-                    inputVarIterator.next(), topUnionVar);
+            Triple<LogicalVariable, LogicalVariable, LogicalVariable> varTriple =
+                    new Triple<>(leftInputVar, inputVarIterator.next(), topUnionVar);
             List<Triple<LogicalVariable, LogicalVariable, LogicalVariable>> varTriples = new ArrayList<>();
             varTriples.add(varTriple);
 
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/util/ValidateUtil.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/util/ValidateUtil.java
index 065b1b0..3d0315c 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/util/ValidateUtil.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/util/ValidateUtil.java
@@ -23,34 +23,44 @@
 
 import org.apache.asterix.common.config.DatasetConfig.IndexType;
 import org.apache.asterix.common.exceptions.AsterixException;
-import org.apache.asterix.metadata.utils.KeyFieldTypeUtils;
+import org.apache.asterix.common.exceptions.CompilationException;
+import org.apache.asterix.common.exceptions.ErrorCode;
+import org.apache.asterix.metadata.utils.KeyFieldTypeUtil;
 import org.apache.asterix.om.types.ARecordType;
 import org.apache.asterix.om.types.ATypeTag;
 import org.apache.asterix.om.types.BuiltinType;
 import org.apache.asterix.om.types.IAType;
+import org.apache.asterix.om.utils.RecordUtil;
+import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
 
 /**
  * A util that can verify if a filter field, a list of partitioning expressions,
  * or a list of key fields are valid in a record type.
  */
 public class ValidateUtil {
+    private ValidateUtil() {
+    }
 
     /**
      * Validates the field that will be used as filter for the components of an LSM index.
      *
-     * @param recType
+     * @param dataset
+     *            the dataset
+     * @param recordType
      *            the record type
-     * @param keyFieldNames
-     *            a list of key fields that will be validated
-     * @param indexType
-     *            the type of the index that its key fields is being validated
-     * @throws AsterixException
-     *             (if the validation failed), IOException
+     * @param filterField
+     *            the full name of the field
+     * @throws AlgebricksException
+     *             if field is not found in record.
+     *             if field type can't be a filter type.
+     *             if field type is nullable.
      */
-    public static void validateFilterField(ARecordType recType, List<String> filterField) throws AsterixException {
-        IAType fieldType = recType.getSubFieldType(filterField);
+    public static void validateFilterField(ARecordType recordType, List<String> filterField)
+            throws AlgebricksException {
+        IAType fieldType = recordType.getSubFieldType(filterField);
         if (fieldType == null) {
-            throw new AsterixException("A field with this name  \"" + filterField + "\" could not be found.");
+            throw new CompilationException(ErrorCode.COMPILATION_FIELD_NOT_FOUND,
+                    RecordUtil.toFullyQualifiedName(filterField));
         }
         switch (fieldType.getTypeTag()) {
             case INT8:
@@ -69,47 +79,59 @@
             case DAYTIMEDURATION:
                 break;
             case UNION:
-                throw new AsterixException("The filter field \"" + filterField + "\" cannot be nullable");
+                throw new CompilationException(ErrorCode.COMPILATION_FILTER_CANNOT_BE_NULLABLE,
+                        RecordUtil.toFullyQualifiedName(filterField));
             default:
-                throw new AsterixException("The field \"" + filterField + "\" which is of type "
-                        + fieldType.getTypeTag() + " cannot be used as a filter for a dataset.");
+                throw new CompilationException(ErrorCode.COMPILATION_ILLEGAL_FILTER_TYPE,
+                        fieldType.getTypeTag().name());
         }
     }
 
     /**
      * Validates the partitioning expression that will be used to partition a dataset and returns expression type.
      *
+     * @param recType
+     *            the record type
+     * @param metaRecType
+     *            the meta record type
      * @param partitioningExprs
      *            a list of partitioning expressions that will be validated
+     * @param keySourceIndicators
+     *            the key sources (record vs. meta)
+     * @param autogenerated
+     *            true if auto generated, false otherwise
      * @return a list of partitioning expressions types
-     * @throws AsterixException
-     *             (if the validation failed), IOException
+     * @throws AlgebricksException
+     *             if composite key is autogenerated.
+     *             if autogenerated and of a type that can't be autogenerated.
+     *             if a field could not be found in its record type.
+     *             if partitioning key is nullable.
+     *             if the field type can't be a primary key.
      */
     public static List<IAType> validatePartitioningExpressions(ARecordType recType, ARecordType metaRecType,
             List<List<String>> partitioningExprs, List<Integer> keySourceIndicators, boolean autogenerated)
-                    throws AsterixException {
-        List<IAType> partitioningExprTypes = new ArrayList<IAType>(partitioningExprs.size());
+            throws AlgebricksException {
+        List<IAType> partitioningExprTypes = new ArrayList<>(partitioningExprs.size());
         if (autogenerated) {
             if (partitioningExprs.size() > 1) {
-                throw new AsterixException("Cannot autogenerate a composite primary key");
+                throw new CompilationException(ErrorCode.COMPILATION_CANNOT_AUTOGENERATE_COMPOSITE_PRIMARY_KEY);
             }
             List<String> fieldName = partitioningExprs.get(0);
             IAType fieldType = recType.getSubFieldType(fieldName);
             partitioningExprTypes.add(fieldType);
-
             ATypeTag pkTypeTag = fieldType.getTypeTag();
             if (pkTypeTag != ATypeTag.UUID) {
-                throw new AsterixException("Cannot autogenerate a primary key for type " + pkTypeTag
-                        + ". Autogenerated primary keys must be of type " + ATypeTag.UUID + ".");
+                throw new CompilationException(ErrorCode.COMPILATION_ILLEGAL_AUTOGENERATED_TYPE, pkTypeTag.name(),
+                        ATypeTag.UUID.name());
             }
         } else {
-            partitioningExprTypes = KeyFieldTypeUtils.getKeyTypes(recType, metaRecType, partitioningExprs,
-                    keySourceIndicators);
+            partitioningExprTypes =
+                    KeyFieldTypeUtil.getKeyTypes(recType, metaRecType, partitioningExprs, keySourceIndicators);
             for (int fidx = 0; fidx < partitioningExprTypes.size(); ++fidx) {
                 IAType fieldType = partitioningExprTypes.get(fidx);
                 if (fieldType == null) {
-                    throw new AsterixException(
-                            "Type not found for partitioning key " + partitioningExprs.get(fidx));
+                    throw new CompilationException(ErrorCode.COMPILATION_FIELD_NOT_FOUND,
+                            RecordUtil.toFullyQualifiedName(partitioningExprs.get(fidx)));
                 }
                 switch (fieldType.getTypeTag()) {
                     case INT8:
@@ -128,11 +150,11 @@
                     case DAYTIMEDURATION:
                         break;
                     case UNION:
-                        throw new AsterixException(
-                                "The partitioning key " + partitioningExprs.get(fidx) + " cannot be nullable");
+                        throw new CompilationException(ErrorCode.COMPILATION_PRIMARY_KEY_CANNOT_BE_NULLABLE,
+                                RecordUtil.toFullyQualifiedName(partitioningExprs.get(fidx)));
                     default:
-                        throw new AsterixException("The partitioning key " + partitioningExprs.get(fidx)
-                                + " cannot be of type " + fieldType.getTypeTag() + ".");
+                        throw new CompilationException(ErrorCode.COMPILATION_ILLEGAL_PRIMARY_KEY_TYPE,
+                                fieldType.getTypeTag().name());
                 }
             }
         }
@@ -150,14 +172,13 @@
      *            a map of key types (if provided) that will be validated
      * @param indexType
      *            the type of the index that its key fields is being validated
-     * @throws AsterixException
-     *             (if the validation failed), IOException
+     * @throws AlgebricksException
      */
-    public static void validateKeyFields(ARecordType recType, ARecordType metaRecType, List<List<String>> keyFieldNames,
-            List<Integer> keySourceIndicators, List<IAType> keyFieldTypes, IndexType indexType)
-                    throws AsterixException {
-        List<IAType> fieldTypes = KeyFieldTypeUtils.getKeyTypes(recType, metaRecType, keyFieldNames,
-                keySourceIndicators);
+    public static void validateKeyFields(ARecordType recType, ARecordType metaRecType,
+            List<List<String>> keyFieldNames, List<Integer> keySourceIndicators, List<IAType> keyFieldTypes,
+            IndexType indexType) throws AlgebricksException {
+        List<IAType> fieldTypes =
+                KeyFieldTypeUtil.getKeyTypes(recType, metaRecType, keyFieldNames, keySourceIndicators);
         int pos = 0;
         boolean openFieldCompositeIdx = false;
         for (IAType fieldType : fieldTypes) {
diff --git a/asterixdb/asterix-app/pom.xml b/asterixdb/asterix-app/pom.xml
index 77ca6ef..282181c 100644
--- a/asterixdb/asterix-app/pom.xml
+++ b/asterixdb/asterix-app/pom.xml
@@ -147,7 +147,6 @@
       <plugin>
         <groupId>org.apache.maven.plugins</groupId>
         <artifactId>maven-dependency-plugin</artifactId>
-        <version>2.10</version>
         <configuration>
           <ignoredUsedUndeclaredDependencies>
             <ignoredUsedUndeclaredDependency>commons-logging:commons-logging-api:*</ignoredUsedUndeclaredDependency>
@@ -351,10 +350,6 @@
       <scope>test</scope>
     </dependency>
     <dependency>
-      <groupId>org.apache.hyracks</groupId>
-      <artifactId>hyracks-storage-am-lsm-rtree</artifactId>
-    </dependency>
-    <dependency>
       <groupId>org.apache.commons</groupId>
       <artifactId>commons-compress</artifactId>
       <version>1.4.1</version>
@@ -409,10 +404,6 @@
       <scope>test</scope>
     </dependency>
     <dependency>
-      <groupId>org.apache.hyracks</groupId>
-      <artifactId>hyracks-storage-am-rtree</artifactId>
-    </dependency>
-    <dependency>
       <groupId>commons-lang</groupId>
       <artifactId>commons-lang</artifactId>
     </dependency>
@@ -435,10 +426,6 @@
     </dependency>
     <dependency>
       <groupId>org.apache.hyracks</groupId>
-      <artifactId>hyracks-storage-am-lsm-invertedindex</artifactId>
-    </dependency>
-    <dependency>
-      <groupId>org.apache.hyracks</groupId>
       <artifactId>hyracks-storage-am-btree</artifactId>
     </dependency>
     <dependency>
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/common/APIFramework.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/common/APIFramework.java
index c4535cf..0f16179 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/common/APIFramework.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/common/APIFramework.java
@@ -31,7 +31,6 @@
 
 import org.apache.asterix.algebra.base.ILangExpressionToPlanTranslator;
 import org.apache.asterix.algebra.base.ILangExpressionToPlanTranslatorFactory;
-import org.apache.asterix.api.common.Job.SubmissionMode;
 import org.apache.asterix.app.result.ResultUtil;
 import org.apache.asterix.common.config.CompilerProperties;
 import org.apache.asterix.common.config.ExternalProperties;
@@ -40,6 +39,8 @@
 import org.apache.asterix.common.config.PropertyInterpreters;
 import org.apache.asterix.common.exceptions.ACIDException;
 import org.apache.asterix.common.exceptions.CompilationException;
+import org.apache.asterix.common.utils.Job;
+import org.apache.asterix.common.utils.Job.SubmissionMode;
 import org.apache.asterix.compiler.provider.ILangCompilationProvider;
 import org.apache.asterix.compiler.provider.IRuleSetFactory;
 import org.apache.asterix.dataflow.data.common.ConflictingTypeResolver;
@@ -59,12 +60,12 @@
 import org.apache.asterix.lang.common.statement.Query;
 import org.apache.asterix.metadata.declared.MetadataProvider;
 import org.apache.asterix.runtime.job.listener.JobEventListenerFactory;
-import org.apache.asterix.runtime.util.AppContextInfo;
+import org.apache.asterix.runtime.utils.AppContextInfo;
 import org.apache.asterix.transaction.management.service.transaction.JobIdFactory;
 import org.apache.asterix.translator.CompiledStatements.ICompiledDmlStatement;
 import org.apache.asterix.translator.IStatementExecutor.Stats;
+import org.apache.asterix.utils.ResourceUtils;
 import org.apache.asterix.translator.SessionConfig;
-import org.apache.asterix.util.ResourceUtils;
 import org.apache.hyracks.algebricks.common.constraints.AlgebricksAbsolutePartitionConstraint;
 import org.apache.hyracks.algebricks.common.constraints.AlgebricksPartitionConstraint;
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
@@ -79,7 +80,7 @@
 import org.apache.hyracks.algebricks.core.algebra.expressions.IExpressionTypeComputer;
 import org.apache.hyracks.algebricks.core.algebra.expressions.IMergeAggregationExpressionFactory;
 import org.apache.hyracks.algebricks.core.algebra.expressions.IMissableTypeComputer;
-import org.apache.hyracks.algebricks.core.algebra.expressions.LogicalExpressionJobGenToExpressionRuntimeProviderAdapter;
+import org.apache.hyracks.algebricks.core.algebra.expressions.ExpressionRuntimeProvider;
 import org.apache.hyracks.algebricks.core.algebra.prettyprint.AlgebricksAppendable;
 import org.apache.hyracks.algebricks.core.algebra.prettyprint.LogicalOperatorPrettyPrintVisitor;
 import org.apache.hyracks.algebricks.core.algebra.prettyprint.PlanPlotter;
@@ -126,7 +127,8 @@
                 IExpressionEvalSizeComputer expressionEvalSizeComputer,
                 IMergeAggregationExpressionFactory mergeAggregationExpressionFactory,
                 IExpressionTypeComputer expressionTypeComputer, IMissableTypeComputer missableTypeComputer,
-                IConflictingTypeResolver conflictingTypeResolver, PhysicalOptimizationConfig physicalOptimizationConfig,
+                IConflictingTypeResolver conflictingTypeResolver,
+                PhysicalOptimizationConfig physicalOptimizationConfig,
                 AlgebricksPartitionConstraint clusterLocations) {
             return new AlgebricksOptimizationContext(varCounter, expressionEvalSizeComputer,
                     mergeAggregationExpressionFactory, expressionTypeComputer, missableTypeComputer,
@@ -181,8 +183,8 @@
 
         org.apache.asterix.common.transactions.JobId asterixJobId = JobIdFactory.generateJobId();
         metadataProvider.setJobId(asterixJobId);
-        ILangExpressionToPlanTranslator t = translatorFactory.createExpressionToPlanTranslator(metadataProvider,
-                varCounter);
+        ILangExpressionToPlanTranslator t =
+                translatorFactory.createExpressionToPlanTranslator(metadataProvider, varCounter);
 
         ILogicalPlan plan;
         // statement = null when it's a query
@@ -224,8 +226,8 @@
         OptimizationConfUtil.getPhysicalOptimizationConfig().setMaxFramesExternalGroupBy(groupFrameLimit);
         OptimizationConfUtil.getPhysicalOptimizationConfig().setMaxFramesForJoin(joinFrameLimit);
 
-        HeuristicCompilerFactoryBuilder builder = new HeuristicCompilerFactoryBuilder(
-                OptimizationContextFactory.INSTANCE);
+        HeuristicCompilerFactoryBuilder builder =
+                new HeuristicCompilerFactoryBuilder(OptimizationContextFactory.INSTANCE);
         builder.setPhysicalOptimizationConfig(OptimizationConfUtil.getPhysicalOptimizationConfig());
         builder.setLogicalRewrites(ruleSetFactory.getLogicalRewrites());
         builder.setPhysicalRewrites(ruleSetFactory.getPhysicalRewrites());
@@ -240,8 +242,8 @@
 
         int parallelism = getParallelism(querySpecificConfig.get(CompilerProperties.COMPILER_PARALLELISM_KEY),
                 compilerProperties.getParallelism());
-        AlgebricksAbsolutePartitionConstraint computationLocations = chooseLocations(clusterInfoCollector, parallelism,
-                metadataProvider.getClusterLocations());
+        AlgebricksAbsolutePartitionConstraint computationLocations =
+                chooseLocations(clusterInfoCollector, parallelism, metadataProvider.getClusterLocations());
         builder.setClusterLocations(computationLocations);
 
         ICompiler compiler = compilerFactory.createCompiler(plan, metadataProvider, t.getVarCounter());
@@ -285,7 +287,7 @@
         builder.setBinaryIntegerInspectorFactory(format.getBinaryIntegerInspectorFactory());
         builder.setComparatorFactoryProvider(format.getBinaryComparatorFactoryProvider());
         builder.setExpressionRuntimeProvider(
-                new LogicalExpressionJobGenToExpressionRuntimeProviderAdapter(QueryLogicalExpressionJobGen.INSTANCE));
+                new ExpressionRuntimeProvider(QueryLogicalExpressionJobGen.INSTANCE));
         builder.setHashFunctionFactoryProvider(format.getBinaryHashFunctionFactoryProvider());
         builder.setHashFunctionFamilyProvider(format.getBinaryHashFunctionFamilyProvider());
         builder.setMissingWriterFactory(format.getMissingWriterFactory());
@@ -313,8 +315,8 @@
         builder.setTypeTraitProvider(format.getTypeTraitProvider());
         builder.setNormalizedKeyComputerFactoryProvider(format.getNormalizedKeyComputerFactoryProvider());
 
-        JobEventListenerFactory jobEventListenerFactory = new JobEventListenerFactory(asterixJobId,
-                metadataProvider.isWriteTransaction());
+        JobEventListenerFactory jobEventListenerFactory =
+                new JobEventListenerFactory(asterixJobId, metadataProvider.isWriteTransaction());
         JobSpecification spec = compiler.createJob(AppContextInfo.INSTANCE, jobEventListenerFactory);
 
         // When the top-level statement is a query, the statement parameter is null.
@@ -427,10 +429,10 @@
         for (Map.Entry<String, NodeControllerInfo> entry : ncMap.entrySet()) {
             String nodeId = entry.getKey();
             int availableCores = entry.getValue().getNumAvailableCores();
-            int nodeParallelism = selectedNodesWithOneMorePartition.contains(nodeId) ? perNodeParallelismMax
-                    : perNodeParallelismMin;
-            int coresToUse = nodeParallelism >= 0 && nodeParallelism < availableCores ? nodeParallelism
-                    : availableCores;
+            int nodeParallelism =
+                    selectedNodesWithOneMorePartition.contains(nodeId) ? perNodeParallelismMax : perNodeParallelismMin;
+            int coresToUse =
+                    nodeParallelism >= 0 && nodeParallelism < availableCores ? nodeParallelism : availableCores;
             for (int count = 0; count < coresToUse; ++count) {
                 locations.add(nodeId);
             }
@@ -450,15 +452,15 @@
     // Gets the frame limit.
     private int getFrameLimit(String parameter, long memBudgetInConfiguration, int frameSize) {
         IPropertyInterpreter<Long> longBytePropertyInterpreter = PropertyInterpreters.getLongBytePropertyInterpreter();
-        long memBudget = parameter == null ? memBudgetInConfiguration
-                : longBytePropertyInterpreter.interpret(parameter);
+        long memBudget =
+                parameter == null ? memBudgetInConfiguration : longBytePropertyInterpreter.interpret(parameter);
         return (int) (memBudget / frameSize);
     }
 
     // Gets the parallelism parameter.
     private int getParallelism(String parameter, int parallelismInConfiguration) {
-        IPropertyInterpreter<Integer> integerIPropertyInterpreter = PropertyInterpreters
-                .getIntegerPropertyInterpreter();
+        IPropertyInterpreter<Integer> integerIPropertyInterpreter =
+                PropertyInterpreters.getIntegerPropertyInterpreter();
         return parameter == null ? parallelismInConfiguration : integerIPropertyInterpreter.interpret(parameter);
     }
 }
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/ApiServlet.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/ApiServlet.java
index ab05f10..95eb6fb 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/ApiServlet.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/ApiServlet.java
@@ -38,6 +38,7 @@
 import org.apache.asterix.app.result.ResultReader;
 import org.apache.asterix.app.result.ResultUtil;
 import org.apache.asterix.common.config.GlobalConfig;
+import org.apache.asterix.common.context.IStorageComponentProvider;
 import org.apache.asterix.common.exceptions.AsterixException;
 import org.apache.asterix.compiler.provider.ILangCompilationProvider;
 import org.apache.asterix.lang.aql.parser.TokenMgrError;
@@ -69,14 +70,16 @@
     private final ILangCompilationProvider aqlCompilationProvider;
     private final ILangCompilationProvider sqlppCompilationProvider;
     private final IStatementExecutorFactory statementExectorFactory;
+    private final IStorageComponentProvider componentProvider;
 
     public ApiServlet(ConcurrentMap<String, Object> ctx, String[] paths,
             ILangCompilationProvider aqlCompilationProvider, ILangCompilationProvider sqlppCompilationProvider,
-            IStatementExecutorFactory statementExecutorFactory) {
+            IStatementExecutorFactory statementExecutorFactory, IStorageComponentProvider componentProvider) {
         super(ctx, paths);
         this.aqlCompilationProvider = aqlCompilationProvider;
         this.sqlppCompilationProvider = sqlppCompilationProvider;
         this.statementExectorFactory = statementExecutorFactory;
+        this.componentProvider = componentProvider;
     }
 
     public void doPost(IServletRequest request, IServletResponse response) {
@@ -135,8 +138,8 @@
             sessionConfig.setOOBData(isSet(printExprParam), isSet(printRewrittenExprParam),
                     isSet(printLogicalPlanParam), isSet(printOptimizedLogicalPlanParam), isSet(printJob));
             MetadataManager.INSTANCE.init();
-            IStatementExecutor translator =
-                    statementExectorFactory.create(aqlStatements, sessionConfig, compilationProvider);
+            IStatementExecutor translator = statementExectorFactory.create(aqlStatements, sessionConfig,
+                    compilationProvider, componentProvider);
             double duration;
             long startTime = System.currentTimeMillis();
             translator.compileAndExecute(hcc, hds, IStatementExecutor.ResultDelivery.IMMEDIATE);
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/ClusterApiServlet.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/ClusterApiServlet.java
index 038ed2f..e93108d 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/ClusterApiServlet.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/ClusterApiServlet.java
@@ -31,7 +31,7 @@
 import org.apache.asterix.common.config.AbstractProperties;
 import org.apache.asterix.common.config.ReplicationProperties;
 import org.apache.asterix.common.utils.JSONUtil;
-import org.apache.asterix.runtime.util.ClusterStateManager;
+import org.apache.asterix.runtime.utils.ClusterStateManager;
 import org.apache.hyracks.http.api.IServlet;
 import org.apache.hyracks.http.api.IServletRequest;
 import org.apache.hyracks.http.api.IServletResponse;
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/ConnectorApiServlet.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/ConnectorApiServlet.java
index caa00f1..66064aa 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/ConnectorApiServlet.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/ConnectorApiServlet.java
@@ -28,13 +28,14 @@
 import java.util.logging.Level;
 import java.util.logging.Logger;
 
+import org.apache.asterix.file.StorageComponentProvider;
 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.utils.DatasetUtils;
+import org.apache.asterix.metadata.utils.DatasetUtil;
 import org.apache.asterix.om.types.ARecordType;
-import org.apache.asterix.util.FlushDatasetUtils;
+import org.apache.asterix.utils.FlushDatasetUtil;
 import org.apache.hyracks.api.client.IHyracksClientConnection;
 import org.apache.hyracks.api.client.NodeControllerInfo;
 import org.apache.hyracks.api.io.FileSplit;
@@ -97,7 +98,7 @@
             MetadataTransactionContext mdTxnCtx = MetadataManager.INSTANCE.beginTransaction();
 
             // Retrieves file splits of the dataset.
-            MetadataProvider metadataProvider = new MetadataProvider(null);
+            MetadataProvider metadataProvider = new MetadataProvider(null, new StorageComponentProvider());
             metadataProvider.setMetadataTxnContext(mdTxnCtx);
             Dataset dataset = metadataProvider.findDataset(dataverseName, datasetName);
             if (dataset == null) {
@@ -112,7 +113,7 @@
                     metadataProvider.splitsForDataset(mdTxnCtx, dataverseName, datasetName, datasetName, temp);
             ARecordType recordType = (ARecordType) metadataProvider.findType(dataset.getItemTypeDataverseName(),
                     dataset.getItemTypeName());
-            List<List<String>> primaryKeys = DatasetUtils.getPartitioningKeys(dataset);
+            List<List<String>> primaryKeys = DatasetUtil.getPartitioningKeys(dataset);
             StringBuilder pkStrBuf = new StringBuilder();
             for (List<String> keys : primaryKeys) {
                 for (String key : keys) {
@@ -126,7 +127,7 @@
                     hcc.getNodeControllerInfos());
 
             // Flush the cached contents of the dataset to file system.
-            FlushDatasetUtils.flushDataset(hcc, metadataProvider, mdTxnCtx, dataverseName, datasetName, datasetName);
+            FlushDatasetUtil.flushDataset(hcc, metadataProvider, dataverseName, datasetName, datasetName);
 
             // Metadata transaction commits.
             MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/DdlApiServlet.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/DdlApiServlet.java
index be6e280..a994470 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/DdlApiServlet.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/DdlApiServlet.java
@@ -20,6 +20,7 @@
 
 import java.util.concurrent.ConcurrentMap;
 
+import org.apache.asterix.common.context.IStorageComponentProvider;
 import org.apache.asterix.compiler.provider.ILangCompilationProvider;
 import org.apache.asterix.lang.common.base.Statement;
 import org.apache.asterix.translator.IStatementExecutorFactory;
@@ -30,8 +31,9 @@
             Statement.Category.QUERY | Statement.Category.UPDATE | Statement.Category.DDL;
 
     public DdlApiServlet(ConcurrentMap<String, Object> ctx, String[] paths,
-            ILangCompilationProvider compilationProvider, IStatementExecutorFactory statementExecutorFactory) {
-        super(ctx, paths, compilationProvider, statementExecutorFactory);
+            ILangCompilationProvider compilationProvider, IStatementExecutorFactory statementExecutorFactory,
+            IStorageComponentProvider componentProvider) {
+        super(ctx, paths, compilationProvider, statementExecutorFactory, componentProvider);
     }
 
     @Override
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/DiagnosticsApiServlet.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/DiagnosticsApiServlet.java
index e6a32a3..709db4b 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/DiagnosticsApiServlet.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/DiagnosticsApiServlet.java
@@ -35,7 +35,7 @@
 import java.util.logging.Logger;
 
 import org.apache.asterix.api.http.servlet.ServletConstants;
-import org.apache.asterix.runtime.util.AppContextInfo;
+import org.apache.asterix.runtime.utils.AppContextInfo;
 import org.apache.hyracks.api.client.IHyracksClientConnection;
 import org.apache.hyracks.http.api.IServlet;
 import org.apache.hyracks.http.api.IServletRequest;
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/FullApiServlet.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/FullApiServlet.java
index 9c08fbd..eafae35 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/FullApiServlet.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/FullApiServlet.java
@@ -20,6 +20,7 @@
 
 import java.util.concurrent.ConcurrentMap;
 
+import org.apache.asterix.common.context.IStorageComponentProvider;
 import org.apache.asterix.compiler.provider.ILangCompilationProvider;
 import org.apache.asterix.lang.common.base.Statement;
 import org.apache.asterix.translator.IStatementExecutorFactory;
@@ -32,8 +33,9 @@
             | Statement.Category.DDL | Statement.Category.PROCEDURE;
 
     public FullApiServlet(ConcurrentMap<String, Object> ctx, String[] paths,
-            ILangCompilationProvider compilationProvider, IStatementExecutorFactory statementExecutorFactory) {
-        super(ctx, paths, compilationProvider, statementExecutorFactory);
+            ILangCompilationProvider compilationProvider, IStatementExecutorFactory statementExecutorFactory,
+            IStorageComponentProvider componentProvider) {
+        super(ctx, paths, compilationProvider, statementExecutorFactory, componentProvider);
     }
 
     @Override
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/NodeControllerDetailsApiServlet.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/NodeControllerDetailsApiServlet.java
index 2dbaa54..f6b80fc4 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/NodeControllerDetailsApiServlet.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/NodeControllerDetailsApiServlet.java
@@ -29,7 +29,7 @@
 import java.util.logging.Level;
 import java.util.logging.Logger;
 
-import org.apache.asterix.runtime.util.ClusterStateManager;
+import org.apache.asterix.runtime.utils.ClusterStateManager;
 import org.apache.hyracks.api.client.IHyracksClientConnection;
 import org.apache.hyracks.http.api.IServlet;
 import org.apache.hyracks.http.api.IServletRequest;
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/QueryApiServlet.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/QueryApiServlet.java
index 160c801..5075795 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/QueryApiServlet.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/QueryApiServlet.java
@@ -20,6 +20,7 @@
 
 import java.util.concurrent.ConcurrentMap;
 
+import org.apache.asterix.common.context.IStorageComponentProvider;
 import org.apache.asterix.compiler.provider.ILangCompilationProvider;
 import org.apache.asterix.lang.common.base.Statement;
 import org.apache.asterix.translator.IStatementExecutorFactory;
@@ -29,8 +30,9 @@
     private static final byte ALLOWED_CATEGORIES = Statement.Category.QUERY;
 
     public QueryApiServlet(ConcurrentMap<String, Object> ctx, String[] paths,
-            ILangCompilationProvider compilationProvider, IStatementExecutorFactory statementExecutorFactory) {
-        super(ctx, paths, compilationProvider, statementExecutorFactory);
+            ILangCompilationProvider compilationProvider, IStatementExecutorFactory statementExecutorFactory,
+            IStorageComponentProvider componentProvider) {
+        super(ctx, paths, compilationProvider, statementExecutorFactory, componentProvider);
     }
 
     @Override
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/QueryServiceServlet.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/QueryServiceServlet.java
index 43530ea..410fd1e 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/QueryServiceServlet.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/QueryServiceServlet.java
@@ -36,6 +36,7 @@
 import org.apache.asterix.app.translator.QueryTranslator;
 import org.apache.asterix.common.api.IClusterManagementWork;
 import org.apache.asterix.common.config.GlobalConfig;
+import org.apache.asterix.common.context.IStorageComponentProvider;
 import org.apache.asterix.common.exceptions.AsterixException;
 import org.apache.asterix.common.utils.JSONUtil;
 import org.apache.asterix.compiler.provider.ILangCompilationProvider;
@@ -43,7 +44,7 @@
 import org.apache.asterix.lang.common.base.IParser;
 import org.apache.asterix.lang.common.base.Statement;
 import org.apache.asterix.metadata.MetadataManager;
-import org.apache.asterix.runtime.util.ClusterStateManager;
+import org.apache.asterix.runtime.utils.ClusterStateManager;
 import org.apache.asterix.translator.IStatementExecutor;
 import org.apache.asterix.translator.IStatementExecutor.Stats;
 import org.apache.asterix.translator.IStatementExecutorFactory;
@@ -73,12 +74,15 @@
     private static final Logger LOGGER = Logger.getLogger(QueryServiceServlet.class.getName());
     private final ILangCompilationProvider compilationProvider;
     private final IStatementExecutorFactory statementExecutorFactory;
+    private final IStorageComponentProvider componentProvider;
 
     public QueryServiceServlet(ConcurrentMap<String, Object> ctx, String[] paths,
-            ILangCompilationProvider compilationProvider, IStatementExecutorFactory statementExecutorFactory) {
+            ILangCompilationProvider compilationProvider, IStatementExecutorFactory statementExecutorFactory,
+            IStorageComponentProvider componentProvider) {
         super(ctx, paths);
         this.compilationProvider = compilationProvider;
         this.statementExecutorFactory = statementExecutorFactory;
+        this.componentProvider = componentProvider;
     }
 
     @Override
@@ -310,19 +314,10 @@
             }
         };
 
-        SessionConfig.ResultDecorator resultPostfix = (AlgebricksAppendable app) -> app.append("\t,\n");
-
-        SessionConfig.ResultDecorator handlePrefix = new SessionConfig.ResultDecorator() {
-            @Override
-            public AlgebricksAppendable append(AlgebricksAppendable app) throws AlgebricksException {
-                app.append("\t\"");
-                app.append(ResultFields.HANDLE.str());
-                app.append("\": ");
-                return app;
-            }
-        };
-
-        SessionConfig.ResultDecorator handlePostfix = (AlgebricksAppendable app) -> app.append(",\n");
+        SessionConfig.ResultDecorator resultPostfix = app -> app.append("\t,\n");
+        SessionConfig.ResultDecorator handlePrefix =
+                app -> app.append("\t\"").append(ResultFields.HANDLE.str()).append("\": ");
+        SessionConfig.ResultDecorator handlePostfix = app -> app.append(",\n");
 
         SessionConfig.OutputFormat format = getFormat(param.format);
         SessionConfig sessionConfig =
@@ -518,7 +513,7 @@
             List<Statement> statements = parser.parse();
             MetadataManager.INSTANCE.init();
             IStatementExecutor translator =
-                    statementExecutorFactory.create(statements, sessionConfig, compilationProvider);
+                    statementExecutorFactory.create(statements, sessionConfig, compilationProvider, componentProvider);
             execStart = System.nanoTime();
             translator.compileAndExecute(hcc, hds, delivery, stats);
             execEnd = System.nanoTime();
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/QueryWebInterfaceServlet.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/QueryWebInterfaceServlet.java
index fac5883..96df30f 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/QueryWebInterfaceServlet.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/QueryWebInterfaceServlet.java
@@ -27,7 +27,7 @@
 import java.util.logging.Logger;
 
 import org.apache.asterix.common.config.ExternalProperties;
-import org.apache.asterix.runtime.util.AppContextInfo;
+import org.apache.asterix.runtime.utils.AppContextInfo;
 import org.apache.commons.io.IOUtils;
 import org.apache.hyracks.http.api.IServlet;
 import org.apache.hyracks.http.api.IServletRequest;
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/RestApiServlet.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/RestApiServlet.java
index 787ff47..b069efe 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/RestApiServlet.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/RestApiServlet.java
@@ -32,6 +32,7 @@
 import org.apache.asterix.app.result.ResultUtil;
 import org.apache.asterix.app.translator.QueryTranslator;
 import org.apache.asterix.common.config.GlobalConfig;
+import org.apache.asterix.common.context.IStorageComponentProvider;
 import org.apache.asterix.common.exceptions.AsterixException;
 import org.apache.asterix.compiler.provider.ILangCompilationProvider;
 import org.apache.asterix.lang.aql.parser.TokenMgrError;
@@ -66,13 +67,16 @@
     private final ILangCompilationProvider compilationProvider;
     private final IParserFactory parserFactory;
     private final IStatementExecutorFactory statementExecutorFactory;
+    private final IStorageComponentProvider componentProvider;
 
     public RestApiServlet(ConcurrentMap<String, Object> ctx, String[] paths,
-            ILangCompilationProvider compilationProvider, IStatementExecutorFactory statementExecutorFactory) {
+            ILangCompilationProvider compilationProvider, IStatementExecutorFactory statementExecutorFactory,
+            IStorageComponentProvider componentProvider) {
         super(ctx, paths);
         this.compilationProvider = compilationProvider;
         this.parserFactory = compilationProvider.getParserFactory();
         this.statementExecutorFactory = statementExecutorFactory;
+        this.componentProvider = componentProvider;
     }
 
     /**
@@ -189,8 +193,8 @@
             List<Statement> aqlStatements = parser.parse();
             validate(aqlStatements);
             MetadataManager.INSTANCE.init();
-            IStatementExecutor translator =
-                    statementExecutorFactory.create(aqlStatements, sessionConfig, compilationProvider);
+            IStatementExecutor translator = statementExecutorFactory.create(aqlStatements, sessionConfig,
+                    compilationProvider, componentProvider);
             translator.compileAndExecute(hcc, hds, resultDelivery);
         } catch (AsterixException | TokenMgrError | org.apache.asterix.aqlplus.parser.TokenMgrError pe) {
             response.setStatus(HttpResponseStatus.INTERNAL_SERVER_ERROR);
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/ShutdownApiServlet.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/ShutdownApiServlet.java
index dc48288..5c569ed 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/ShutdownApiServlet.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/ShutdownApiServlet.java
@@ -28,7 +28,7 @@
 import java.util.logging.Logger;
 
 import org.apache.asterix.common.config.GlobalConfig;
-import org.apache.asterix.runtime.util.ClusterStateManager;
+import org.apache.asterix.runtime.utils.ClusterStateManager;
 import org.apache.hyracks.api.client.IHyracksClientConnection;
 import org.apache.hyracks.http.api.IServlet;
 import org.apache.hyracks.http.api.IServletRequest;
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/UpdateApiServlet.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/UpdateApiServlet.java
index 0a0e680..ad2c128 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/UpdateApiServlet.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/UpdateApiServlet.java
@@ -20,6 +20,7 @@
 
 import java.util.concurrent.ConcurrentMap;
 
+import org.apache.asterix.common.context.IStorageComponentProvider;
 import org.apache.asterix.compiler.provider.ILangCompilationProvider;
 import org.apache.asterix.lang.common.base.Statement;
 import org.apache.asterix.translator.IStatementExecutorFactory;
@@ -29,8 +30,9 @@
     private static final byte ALLOWED_CATEGORIES = Statement.Category.QUERY | Statement.Category.UPDATE;
 
     public UpdateApiServlet(ConcurrentMap<String, Object> ctx, String[] paths,
-            ILangCompilationProvider compilationProvider, IStatementExecutorFactory statementExecutorFactory) {
-        super(ctx, paths, compilationProvider, statementExecutorFactory);
+            ILangCompilationProvider compilationProvider, IStatementExecutorFactory statementExecutorFactory,
+            IStorageComponentProvider componentProvider) {
+        super(ctx, paths, compilationProvider, statementExecutorFactory, componentProvider);
     }
 
     @Override
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/VersionApiServlet.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/VersionApiServlet.java
index 5899660..1b2c2e6 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/VersionApiServlet.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/VersionApiServlet.java
@@ -27,7 +27,7 @@
 import java.util.logging.Level;
 import java.util.logging.Logger;
 
-import org.apache.asterix.runtime.util.AppContextInfo;
+import org.apache.asterix.runtime.utils.AppContextInfo;
 import org.apache.hyracks.http.api.IServlet;
 import org.apache.hyracks.http.api.IServletRequest;
 import org.apache.hyracks.http.api.IServletResponse;
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/java/AsterixJavaClient.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/java/AsterixJavaClient.java
index 3d240f8..c09f8cb 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/java/AsterixJavaClient.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/java/AsterixJavaClient.java
@@ -23,8 +23,9 @@
 import java.util.List;
 
 import org.apache.asterix.api.common.APIFramework;
-import org.apache.asterix.api.common.Job;
 import org.apache.asterix.app.translator.QueryTranslator;
+import org.apache.asterix.common.context.IStorageComponentProvider;
+import org.apache.asterix.common.utils.Job;
 import org.apache.asterix.compiler.provider.ILangCompilationProvider;
 import org.apache.asterix.lang.common.base.IParser;
 import org.apache.asterix.lang.common.base.IParserFactory;
@@ -49,21 +50,28 @@
     private final IParserFactory parserFactory;
     private final APIFramework apiFramework;
     private final IStatementExecutorFactory statementExecutorFactory;
+    private final IStorageComponentProvider storageComponentProvider;
 
     public AsterixJavaClient(IHyracksClientConnection hcc, Reader queryText, PrintWriter writer,
-            ILangCompilationProvider compilationProvider, IStatementExecutorFactory statementExecutorFactory) {
+            ILangCompilationProvider compilationProvider, IStatementExecutorFactory statementExecutorFactory,
+            IStorageComponentProvider storageComponentProvider) {
         this.hcc = hcc;
         this.queryText = queryText;
         this.writer = writer;
         this.compilationProvider = compilationProvider;
-        this.apiFramework = new APIFramework(compilationProvider);
         this.statementExecutorFactory = statementExecutorFactory;
+        this.storageComponentProvider = storageComponentProvider;
+        apiFramework = new APIFramework(compilationProvider);
         parserFactory = compilationProvider.getParserFactory();
     }
 
     public AsterixJavaClient(IHyracksClientConnection hcc, Reader queryText,
-            ILangCompilationProvider compilationProvider, IStatementExecutorFactory statementExecutorFactory) {
-        this(hcc, queryText, new PrintWriter(System.out, true), compilationProvider, statementExecutorFactory);
+            ILangCompilationProvider compilationProvider, IStatementExecutorFactory statementExecutorFactory,
+            IStorageComponentProvider storageComponentProvider) {
+        this(hcc, queryText,
+                // This is a commandline client and so System.out is appropriate
+                new PrintWriter(System.out, true), // NOSONAR
+                compilationProvider, statementExecutorFactory, storageComponentProvider);
     }
 
     public void compile() throws Exception {
@@ -94,7 +102,8 @@
             conf.set(SessionConfig.FORMAT_ONLY_PHYSICAL_OPS, true);
         }
 
-        IStatementExecutor translator = statementExecutorFactory.create(statements, conf, compilationProvider);
+        IStatementExecutor translator =
+                statementExecutorFactory.create(statements, conf, compilationProvider, storageComponentProvider);
         translator.compileAndExecute(hcc, null, QueryTranslator.ResultDelivery.IMMEDIATE);
         writer.flush();
     }
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/cc/CompilerExtensionManager.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/cc/CCExtensionManager.java
similarity index 76%
rename from asterixdb/asterix-app/src/main/java/org/apache/asterix/app/cc/CompilerExtensionManager.java
rename to asterixdb/asterix-app/src/main/java/org/apache/asterix/app/cc/CCExtensionManager.java
index 95ce100..31ace22 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/cc/CompilerExtensionManager.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/cc/CCExtensionManager.java
@@ -35,6 +35,7 @@
 import org.apache.asterix.compiler.provider.ILangCompilationProvider;
 import org.apache.asterix.compiler.provider.SqlppCompilationProvider;
 import org.apache.asterix.translator.IStatementExecutorFactory;
+import org.apache.asterix.utils.ExtensionUtil;
 import org.apache.hyracks.algebricks.common.utils.Pair;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 
@@ -42,7 +43,7 @@
  * AsterixDB's implementation of {@code IAlgebraExtensionManager} which takes care of
  * initializing extensions for App and Compilation purposes
  */
-public class CompilerExtensionManager implements IAlgebraExtensionManager {
+public class CCExtensionManager implements IAlgebraExtensionManager {
 
     private final Map<ExtensionId, IExtension> extensions = new HashMap<>();
 
@@ -60,13 +61,12 @@
      * @throws ClassNotFoundException
      * @throws HyracksDataException
      */
-    public CompilerExtensionManager(List<AsterixExtension> list)
+    public CCExtensionManager(List<AsterixExtension> list)
             throws InstantiationException, IllegalAccessException, ClassNotFoundException, HyracksDataException {
         Pair<ExtensionId, ILangCompilationProvider> aqlcp = null;
         Pair<ExtensionId, ILangCompilationProvider> sqlppcp = null;
         IStatementExecutorExtension see = null;
         defaultQueryTranslatorFactory = new DefaultStatementExecutorFactory();
-
         if (list != null) {
             for (AsterixExtension extensionConf : list) {
                 IExtension extension = (IExtension) Class.forName(extensionConf.getClassName()).newInstance();
@@ -77,12 +77,12 @@
                 extensions.put(extension.getId(), extension);
                 switch (extension.getExtensionKind()) {
                     case STATEMENT_EXECUTOR:
-                        see = extendStatementExecutor(see, (IStatementExecutorExtension) extension);
+                        see = ExtensionUtil.extendStatementExecutor(see, (IStatementExecutorExtension) extension);
                         break;
                     case LANG:
                         ILangExtension le = (ILangExtension) extension;
-                        aqlcp = extendLangCompilationProvider(Language.AQL, aqlcp, le);
-                        sqlppcp = extendLangCompilationProvider(Language.SQLPP, sqlppcp, le);
+                        aqlcp = ExtensionUtil.extendLangCompilationProvider(Language.AQL, aqlcp, le);
+                        sqlppcp = ExtensionUtil.extendLangCompilationProvider(Language.SQLPP, sqlppcp, le);
                         break;
                     default:
                         break;
@@ -94,25 +94,6 @@
         this.sqlppCompilationProvider = sqlppcp == null ? new SqlppCompilationProvider() : sqlppcp.second;
     }
 
-    private Pair<ExtensionId, ILangCompilationProvider> extendLangCompilationProvider(Language lang,
-            Pair<ExtensionId, ILangCompilationProvider> cp, ILangExtension le) throws HyracksDataException {
-        if (cp != null && le.getLangCompilationProvider(lang) != null) {
-            throw new RuntimeDataException(ErrorCode.EXTENSION_COMPONENT_CONFLICT, le.getId(), cp.first,
-                    lang.toString());
-        }
-        return (le.getLangCompilationProvider(lang) != null)
-                ? new Pair<>(le.getId(), le.getLangCompilationProvider(lang)) : cp;
-    }
-
-    private IStatementExecutorExtension extendStatementExecutor(IStatementExecutorExtension qte,
-            IStatementExecutorExtension extension) throws HyracksDataException {
-        if (qte != null) {
-            throw new RuntimeDataException(ErrorCode.EXTENSION_COMPONENT_CONFLICT, qte.getId(), extension.getId(),
-                    IStatementExecutorFactory.class.getSimpleName());
-        }
-        return extension;
-    }
-
     public IStatementExecutorFactory getQueryTranslatorFactory() {
         return statementExecutorExtension == null ? defaultQueryTranslatorFactory
                 : statementExecutorExtension.getQueryTranslatorFactory();
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/cc/IStatementExecutorExtension.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/cc/IStatementExecutorExtension.java
index a4b2345..f7b6842 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/cc/IStatementExecutorExtension.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/cc/IStatementExecutorExtension.java
@@ -25,6 +25,7 @@
  * An interface for extensions of {@code IStatementExecutor}
  */
 public interface IStatementExecutorExtension extends IExtension {
+
     @Override
     default ExtensionKind getExtensionKind() {
         return ExtensionKind.STATEMENT_EXECUTOR;
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/cc/ResourceIdManager.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/cc/ResourceIdManager.java
index 7b01169..f43092b 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/cc/ResourceIdManager.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/cc/ResourceIdManager.java
@@ -23,7 +23,7 @@
 import java.util.concurrent.atomic.AtomicLong;
 
 import org.apache.asterix.common.transactions.IResourceIdManager;
-import org.apache.asterix.runtime.util.ClusterStateManager;
+import org.apache.asterix.runtime.utils.ClusterStateManager;
 
 public class ResourceIdManager implements IResourceIdManager {
 
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/external/ExternalIndexingOperations.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/external/ExternalIndexingOperations.java
deleted file mode 100644
index a2518ec..0000000
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/external/ExternalIndexingOperations.java
+++ /dev/null
@@ -1,771 +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.app.external;
-
-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.ExternalDatasetTransactionState;
-import org.apache.asterix.common.config.DatasetConfig.ExternalFilePendingOp;
-import org.apache.asterix.common.config.DatasetConfig.IndexType;
-import org.apache.asterix.common.config.IPropertiesProvider;
-import org.apache.asterix.common.config.StorageProperties;
-import org.apache.asterix.common.context.AsterixVirtualBufferCacheProvider;
-import org.apache.asterix.common.dataflow.LSMIndexUtil;
-import org.apache.asterix.common.exceptions.AsterixException;
-import org.apache.asterix.common.ioopcallbacks.LSMBTreeIOOperationCallbackFactory;
-import org.apache.asterix.common.ioopcallbacks.LSMBTreeWithBuddyIOOperationCallbackFactory;
-import org.apache.asterix.common.ioopcallbacks.LSMRTreeIOOperationCallbackFactory;
-import org.apache.asterix.common.transactions.IResourceFactory;
-import org.apache.asterix.dataflow.data.nontagged.valueproviders.PrimitiveValueProviderFactory;
-import org.apache.asterix.external.api.IAdapterFactory;
-import org.apache.asterix.external.indexing.ExternalFile;
-import org.apache.asterix.external.indexing.FilesIndexDescription;
-import org.apache.asterix.external.indexing.IndexingConstants;
-import org.apache.asterix.external.operators.ExternalDataScanOperatorDescriptor;
-import org.apache.asterix.external.operators.ExternalDatasetIndexesAbortOperatorDescriptor;
-import org.apache.asterix.external.operators.ExternalDatasetIndexesCommitOperatorDescriptor;
-import org.apache.asterix.external.operators.ExternalDatasetIndexesRecoverOperatorDescriptor;
-import org.apache.asterix.external.operators.ExternalFilesIndexOperatorDescriptor;
-import org.apache.asterix.external.operators.IndexInfoOperatorDescriptor;
-import org.apache.asterix.external.provider.AdapterFactoryProvider;
-import org.apache.asterix.external.util.ExternalDataConstants;
-import org.apache.asterix.file.IndexOperations;
-import org.apache.asterix.file.JobSpecificationUtils;
-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.metadata.MetadataException;
-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.metadata.utils.DatasetUtils;
-import org.apache.asterix.metadata.utils.ExternalDatasetsRegistry;
-import org.apache.asterix.om.types.ARecordType;
-import org.apache.asterix.om.types.ATypeTag;
-import org.apache.asterix.om.types.BuiltinType;
-import org.apache.asterix.om.types.IAType;
-import org.apache.asterix.om.util.NonTaggedFormatUtil;
-import org.apache.asterix.runtime.util.AppContextInfo;
-import org.apache.asterix.runtime.util.RuntimeComponentsProvider;
-import org.apache.asterix.transaction.management.opcallbacks.SecondaryIndexOperationTrackerProvider;
-import org.apache.asterix.transaction.management.resource.ExternalBTreeLocalResourceMetadataFactory;
-import org.apache.asterix.transaction.management.resource.PersistentLocalResourceFactoryProvider;
-import org.apache.asterix.translator.CompiledStatements.CompiledCreateIndexStatement;
-import org.apache.asterix.translator.CompiledStatements.CompiledIndexDropStatement;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FileStatus;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.hdfs.DistributedFileSystem;
-import org.apache.hyracks.algebricks.common.constraints.AlgebricksPartitionConstraint;
-import org.apache.hyracks.algebricks.common.constraints.AlgebricksPartitionConstraintHelper;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
-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.ISerializerDeserializer;
-import org.apache.hyracks.api.dataflow.value.ITypeTraits;
-import org.apache.hyracks.api.dataflow.value.RecordDescriptor;
-import org.apache.hyracks.api.exceptions.HyracksDataException;
-import org.apache.hyracks.api.job.JobSpecification;
-import org.apache.hyracks.dataflow.std.file.IFileSplitProvider;
-import org.apache.hyracks.storage.am.common.api.IPrimitiveValueProviderFactory;
-import org.apache.hyracks.storage.am.common.dataflow.IndexDropOperatorDescriptor;
-import org.apache.hyracks.storage.am.common.impls.NoOpOperationCallbackFactory;
-import org.apache.hyracks.storage.am.lsm.btree.dataflow.ExternalBTreeDataflowHelperFactory;
-import org.apache.hyracks.storage.am.lsm.btree.dataflow.ExternalBTreeWithBuddyDataflowHelperFactory;
-import org.apache.hyracks.storage.am.lsm.btree.dataflow.LSMBTreeDataflowHelperFactory;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMMergePolicyFactory;
-import org.apache.hyracks.storage.am.lsm.common.dataflow.LSMTreeIndexCompactOperatorDescriptor;
-import org.apache.hyracks.storage.am.lsm.rtree.dataflow.ExternalRTreeDataflowHelperFactory;
-import org.apache.hyracks.storage.am.rtree.frames.RTreePolicyType;
-import org.apache.hyracks.storage.common.file.LocalResource;
-
-public class ExternalIndexingOperations {
-
-    public static final List<List<String>> FILE_INDEX_FIELD_NAMES = Collections
-            .singletonList(Collections.singletonList(""));
-    public static final List<IAType> FILE_INDEX_FIELD_TYPES = 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() != ExternalDatasetTransactionState.COMMIT;
-    }
-
-    public static boolean isValidIndexName(String datasetName, String indexName) {
-        return (!datasetName.concat(IndexingConstants.EXTERNAL_FILE_INDEX_NAME_SUFFIX).equals(indexName));
-    }
-
-    public static String getFilesIndexName(String datasetName) {
-        return datasetName.concat(IndexingConstants.EXTERNAL_FILE_INDEX_NAME_SUFFIX);
-    }
-
-    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 ArrayList<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();
-                    if (fileStatuses[i].isDirectory()) {
-                        listSubFiles(dataset, fs, fileStatuses[i], files);
-                    } else {
-                        files.add(new ExternalFile(dataset.getDataverseName(), dataset.getDatasetName(), nextFileNumber,
-                                fileStatuses[i].getPath().toUri().getPath(),
-                                new Date(fileStatuses[i].getModificationTime()), fileStatuses[i].getLen(),
-                                ExternalFilePendingOp.PENDING_NO_OP));
-                    }
-                }
-            }
-            // Close file system
-            fs.close();
-            if (files.size() == 0) {
-                throw new AlgebricksException("File Snapshot retrieved from external file system is empty");
-            }
-            return files;
-        } catch (Exception e) {
-            e.printStackTrace();
-            throw new AlgebricksException("Unable to get list of HDFS files " + e);
-        }
-    }
-
-    /* list all files under the directory
-     * src is expected to be a folder
-     */
-    private static void listSubFiles(Dataset dataset, FileSystem srcFs, FileStatus src, ArrayList<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.PENDING_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 buildFilesIndexReplicationJobSpec(Dataset dataset,
-            ArrayList<ExternalFile> externalFilesSnapshot, MetadataProvider metadataProvider, boolean createIndex)
-            throws MetadataException, AlgebricksException {
-        JobSpecification spec = JobSpecificationUtils.createJobSpecification();
-        IPropertiesProvider asterixPropertiesProvider = AppContextInfo.INSTANCE;
-        StorageProperties storageProperties = asterixPropertiesProvider.getStorageProperties();
-        Pair<ILSMMergePolicyFactory, Map<String, String>> compactionInfo = DatasetUtils.getMergePolicyFactory(dataset,
-                metadataProvider.getMetadataTxnContext());
-        ILSMMergePolicyFactory mergePolicyFactory = compactionInfo.first;
-        Map<String, String> mergePolicyFactoryProperties = compactionInfo.second;
-        Pair<IFileSplitProvider, AlgebricksPartitionConstraint> secondarySplitsAndConstraint = metadataProvider
-                .splitProviderAndPartitionConstraintsForFilesIndex(dataset.getDataverseName(), dataset.getDatasetName(),
-                        getFilesIndexName(dataset.getDatasetName()), true);
-        IFileSplitProvider secondaryFileSplitProvider = secondarySplitsAndConstraint.first;
-        FilesIndexDescription filesIndexDescription = new FilesIndexDescription();
-        IResourceFactory localResourceMetadata = new ExternalBTreeLocalResourceMetadataFactory(
-                filesIndexDescription.EXTERNAL_FILE_INDEX_TYPE_TRAITS, filesIndexDescription.FILES_INDEX_COMP_FACTORIES,
-                new int[] { 0 }, false, dataset.getDatasetId(), mergePolicyFactory, mergePolicyFactoryProperties);
-        PersistentLocalResourceFactoryProvider localResourceFactoryProvider =
-                new PersistentLocalResourceFactoryProvider(
-                        localResourceMetadata, LocalResource.ExternalBTreeResource);
-        ExternalBTreeDataflowHelperFactory indexDataflowHelperFactory = new ExternalBTreeDataflowHelperFactory(
-                mergePolicyFactory, mergePolicyFactoryProperties,
-                new SecondaryIndexOperationTrackerProvider(dataset.getDatasetId()),
-                RuntimeComponentsProvider.RUNTIME_PROVIDER, LSMBTreeIOOperationCallbackFactory.INSTANCE,
-                storageProperties.getBloomFilterFalsePositiveRate(),
-                ExternalDatasetsRegistry.INSTANCE.getDatasetVersion(dataset), true);
-        ExternalFilesIndexOperatorDescriptor externalFilesOp = new ExternalFilesIndexOperatorDescriptor(spec,
-                RuntimeComponentsProvider.RUNTIME_PROVIDER, RuntimeComponentsProvider.RUNTIME_PROVIDER,
-                secondaryFileSplitProvider, indexDataflowHelperFactory, localResourceFactoryProvider,
-                externalFilesSnapshot, createIndex, LSMIndexUtil.getMetadataPageManagerFactory());
-        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
-     * @return
-     * @throws AlgebricksException
-     * @throws HyracksDataException
-     * @throws Exception
-     */
-    private static Pair<ExternalDataScanOperatorDescriptor, AlgebricksPartitionConstraint>
-            getExternalDataIndexingOperator(
-                    MetadataProvider metadataProvider, JobSpecification jobSpec, IAType itemType, Dataset dataset,
-                    List<ExternalFile> files, RecordDescriptor indexerDesc)
-                    throws HyracksDataException, AlgebricksException {
-        ExternalDatasetDetails externalDatasetDetails = (ExternalDatasetDetails) dataset.getDatasetDetails();
-        Map<String, String> configuration = externalDatasetDetails.getProperties();
-        IAdapterFactory adapterFactory = AdapterFactoryProvider.getIndexingAdapterFactory(
-                metadataProvider.getLibraryManager(), externalDatasetDetails.getAdapter(), configuration,
-                (ARecordType) itemType, files, true, null);
-        return new Pair<>(new ExternalDataScanOperatorDescriptor(jobSpec, indexerDesc, adapterFactory),
-                adapterFactory.getPartitionConstraint());
-    }
-
-    public static Pair<ExternalDataScanOperatorDescriptor, AlgebricksPartitionConstraint> createExternalIndexingOp(
-            JobSpecification spec, MetadataProvider metadataProvider, Dataset dataset, ARecordType itemType,
-            RecordDescriptor indexerDesc, List<ExternalFile> files) throws HyracksDataException, AlgebricksException {
-        if (files == null) {
-            files = MetadataManager.INSTANCE.getDatasetExternalFiles(metadataProvider.getMetadataTxnContext(), dataset);
-        }
-        return getExternalDataIndexingOperator(metadataProvider, spec, itemType, dataset, files, indexerDesc);
-    }
-
-    /**
-     * 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 MetadataException
-     * @throws AlgebricksException
-     */
-    public static boolean isDatasetUptodate(Dataset dataset, List<ExternalFile> metadataFiles,
-            List<ExternalFile> addedFiles, List<ExternalFile> deletedFiles, List<ExternalFile> appendedFiles)
-            throws MetadataException, AlgebricksException {
-        boolean uptodate = true;
-        int newFileNumber = metadataFiles.get(metadataFiles.size() - 1).getFileNumber() + 1;
-
-        ArrayList<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())) {
-                    // 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.PENDING_APPEND_OP);
-                            fileSystemFile.setPendingOp(ExternalFilePendingOp.PENDING_APPEND_OP);
-                            appendedFiles.add(fileSystemFile);
-                            fileFound = true;
-                            uptodate = false;
-                        }
-                    } else {
-                        // Same file name, Different file mod date -> delete and add
-                        metadataFile.setPendingOp(ExternalFilePendingOp.PENDING_DROP_OP);
-                        deletedFiles
-                                .add(new ExternalFile(metadataFile.getDataverseName(), metadataFile.getDatasetName(), 0,
-                                        metadataFile.getFileName(), metadataFile.getLastModefiedTime(),
-                                        metadataFile.getSize(), ExternalFilePendingOp.PENDING_DROP_OP));
-                        fileSystemFile.setPendingOp(ExternalFilePendingOp.PENDING_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.PENDING_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.PENDING_NO_OP) {
-                metadataFile.setPendingOp(ExternalFilePendingOp.PENDING_DROP_OP);
-                deletedFiles.add(new ExternalFile(metadataFile.getDataverseName(), metadataFile.getDatasetName(),
-                        newFileNumber, metadataFile.getFileName(), metadataFile.getLastModefiedTime(),
-                        metadataFile.getSize(), metadataFile.getPendingOp()));
-                newFileNumber++;
-                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(), ExternalDatasetTransactionState.BEGIN);
-        Dataset transactionDatset = 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());
-        return transactionDatset;
-    }
-
-    public static boolean isFileIndex(Index index) {
-        return (index.getIndexName().equals(getFilesIndexName(index.getDatasetName())));
-    }
-
-    public static JobSpecification buildDropFilesIndexJobSpec(CompiledIndexDropStatement indexDropStmt,
-            MetadataProvider metadataProvider, Dataset dataset) throws AlgebricksException, MetadataException {
-        String dataverseName = indexDropStmt.getDataverseName() == null ? metadataProvider.getDefaultDataverseName()
-                : indexDropStmt.getDataverseName();
-        String datasetName = indexDropStmt.getDatasetName();
-        String indexName = indexDropStmt.getIndexName();
-        boolean temp = dataset.getDatasetDetails().isTemp();
-        JobSpecification spec = JobSpecificationUtils.createJobSpecification();
-        Pair<IFileSplitProvider, AlgebricksPartitionConstraint> splitsAndConstraint = metadataProvider
-                .splitProviderAndPartitionConstraintsForFilesIndex(dataverseName, datasetName, indexName, true);
-        StorageProperties storageProperties = AppContextInfo.INSTANCE.getStorageProperties();
-        Pair<ILSMMergePolicyFactory, Map<String, String>> compactionInfo = DatasetUtils.getMergePolicyFactory(dataset,
-                metadataProvider.getMetadataTxnContext());
-        IndexDropOperatorDescriptor btreeDrop = new IndexDropOperatorDescriptor(spec,
-                RuntimeComponentsProvider.RUNTIME_PROVIDER, RuntimeComponentsProvider.RUNTIME_PROVIDER,
-                splitsAndConstraint.first,
-                new LSMBTreeDataflowHelperFactory(new AsterixVirtualBufferCacheProvider(dataset.getDatasetId()),
-                        compactionInfo.first, compactionInfo.second,
-                        new SecondaryIndexOperationTrackerProvider(dataset.getDatasetId()),
-                        RuntimeComponentsProvider.RUNTIME_PROVIDER, LSMBTreeIOOperationCallbackFactory.INSTANCE,
-                        storageProperties.getBloomFilterFalsePositiveRate(), false, null, null, null, null, !temp),
-                LSMIndexUtil.getMetadataPageManagerFactory());
-        AlgebricksPartitionConstraintHelper.setPartitionConstraintInJobSpec(spec, btreeDrop,
-                splitsAndConstraint.second);
-        spec.addRoot(btreeDrop);
-
-        return spec;
-    }
-
-    public static JobSpecification buildFilesIndexUpdateOp(Dataset ds, List<ExternalFile> metadataFiles,
-            List<ExternalFile> deletedFiles, List<ExternalFile> addedFiles, List<ExternalFile> appendedFiles,
-            MetadataProvider metadataProvider) throws MetadataException, AlgebricksException {
-        ArrayList<ExternalFile> files = new ArrayList<>();
-        for (ExternalFile file : metadataFiles) {
-            if (file.getPendingOp() == ExternalFilePendingOp.PENDING_DROP_OP) {
-                files.add(file);
-            } else if (file.getPendingOp() == ExternalFilePendingOp.PENDING_APPEND_OP) {
-                for (ExternalFile appendedFile : appendedFiles) {
-                    if (appendedFile.getFileName().equals(file.getFileName())) {
-                        files.add(new ExternalFile(file.getDataverseName(), file.getDatasetName(), file.getFileNumber(),
-                                file.getFileName(), file.getLastModefiedTime(), appendedFile.getSize(),
-                                ExternalFilePendingOp.PENDING_NO_OP));
-                    }
-                }
-            }
-        }
-        for (ExternalFile file : addedFiles) {
-            files.add(file);
-        }
-        Collections.sort(files);
-        return buildFilesIndexReplicationJobSpec(ds, files, metadataProvider, false);
-    }
-
-    public static JobSpecification buildIndexUpdateOp(Dataset ds, Index index, List<ExternalFile> metadataFiles,
-            List<ExternalFile> deletedFiles, List<ExternalFile> addedFiles, List<ExternalFile> appendedFiles,
-            MetadataProvider metadataProvider) throws AsterixException, AlgebricksException {
-        // Create files list
-        ArrayList<ExternalFile> files = new ArrayList<>();
-
-        for (ExternalFile metadataFile : metadataFiles) {
-            if (metadataFile.getPendingOp() != ExternalFilePendingOp.PENDING_APPEND_OP) {
-                files.add(metadataFile);
-            } else {
-                metadataFile.setPendingOp(ExternalFilePendingOp.PENDING_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);
-        }
-
-        CompiledCreateIndexStatement ccis = new CompiledCreateIndexStatement(index.getIndexName(),
-                index.getDataverseName(), index.getDatasetName(), index.getKeyFieldNames(), index.getKeyFieldTypes(),
-                index.isEnforcingKeyFileds(), index.getGramLength(), index.getIndexType());
-        return IndexOperations.buildSecondaryIndexLoadingJobSpec(ccis, null, null, null, null, metadataProvider, files);
-    }
-
-    public static JobSpecification buildCommitJob(Dataset ds, List<Index> indexes, MetadataProvider metadataProvider)
-            throws AlgebricksException, AsterixException {
-        JobSpecification spec = JobSpecificationUtils.createJobSpecification();
-        IPropertiesProvider asterixPropertiesProvider = AppContextInfo.INSTANCE;
-        StorageProperties storageProperties = asterixPropertiesProvider.getStorageProperties();
-        Pair<ILSMMergePolicyFactory, Map<String, String>> compactionInfo = DatasetUtils.getMergePolicyFactory(ds,
-                metadataProvider.getMetadataTxnContext());
-        boolean temp = ds.getDatasetDetails().isTemp();
-        ILSMMergePolicyFactory mergePolicyFactory = compactionInfo.first;
-        Map<String, String> mergePolicyFactoryProperties = compactionInfo.second;
-        Pair<IFileSplitProvider, AlgebricksPartitionConstraint> filesIndexSplitsAndConstraint = metadataProvider
-                .splitProviderAndPartitionConstraintsForDataset(ds.getDataverseName(), ds.getDatasetName(),
-                        getFilesIndexName(ds.getDatasetName()), temp);
-        IFileSplitProvider filesIndexSplitProvider = filesIndexSplitsAndConstraint.first;
-        ExternalBTreeDataflowHelperFactory filesIndexDataflowHelperFactory = getFilesIndexDataflowHelperFactory(ds,
-                mergePolicyFactory, mergePolicyFactoryProperties, storageProperties, spec);
-        IndexInfoOperatorDescriptor filesIndexInfo = new IndexInfoOperatorDescriptor(filesIndexSplitProvider,
-                RuntimeComponentsProvider.RUNTIME_PROVIDER, RuntimeComponentsProvider.RUNTIME_PROVIDER);
-
-        ArrayList<ExternalBTreeWithBuddyDataflowHelperFactory> btreeDataflowHelperFactories = new ArrayList<>();
-        ArrayList<IndexInfoOperatorDescriptor> btreeInfos = new ArrayList<>();
-        ArrayList<ExternalRTreeDataflowHelperFactory> rtreeDataflowHelperFactories = new ArrayList<>();
-        ArrayList<IndexInfoOperatorDescriptor> rtreeInfos = new ArrayList<>();
-
-        for (Index index : indexes) {
-            if (isValidIndexName(index.getDatasetName(), index.getIndexName())) {
-                Pair<IFileSplitProvider, AlgebricksPartitionConstraint> indexSplitsAndConstraint = metadataProvider
-                        .splitProviderAndPartitionConstraintsForDataset(ds.getDataverseName(), ds.getDatasetName(),
-                                index.getIndexName(), temp);
-                if (index.getIndexType() == IndexType.BTREE) {
-                    btreeDataflowHelperFactories.add(getBTreeDataflowHelperFactory(ds, index, mergePolicyFactory,
-                            mergePolicyFactoryProperties, storageProperties, spec));
-                    btreeInfos.add(new IndexInfoOperatorDescriptor(indexSplitsAndConstraint.first,
-                            RuntimeComponentsProvider.RUNTIME_PROVIDER,
-                            RuntimeComponentsProvider.RUNTIME_PROVIDER));
-                } else if (index.getIndexType() == IndexType.RTREE) {
-                    rtreeDataflowHelperFactories.add(getRTreeDataflowHelperFactory(ds, index, mergePolicyFactory,
-                            mergePolicyFactoryProperties, storageProperties, metadataProvider, spec));
-                    rtreeInfos.add(new IndexInfoOperatorDescriptor(indexSplitsAndConstraint.first,
-                            RuntimeComponentsProvider.RUNTIME_PROVIDER,
-                            RuntimeComponentsProvider.RUNTIME_PROVIDER));
-                }
-            }
-        }
-
-        ExternalDatasetIndexesCommitOperatorDescriptor op = new ExternalDatasetIndexesCommitOperatorDescriptor(spec,
-                filesIndexDataflowHelperFactory, filesIndexInfo, btreeDataflowHelperFactories, btreeInfos,
-                rtreeDataflowHelperFactories, rtreeInfos);
-
-        spec.addRoot(op);
-        AlgebricksPartitionConstraintHelper.setPartitionConstraintInJobSpec(spec, op,
-                filesIndexSplitsAndConstraint.second);
-        spec.setConnectorPolicyAssignmentPolicy(new ConnectorPolicyAssignmentPolicy());
-        return spec;
-    }
-
-    private static ExternalBTreeDataflowHelperFactory getFilesIndexDataflowHelperFactory(Dataset ds,
-            ILSMMergePolicyFactory mergePolicyFactory, Map<String, String> mergePolicyFactoryProperties,
-            StorageProperties storageProperties, JobSpecification spec) {
-        return new ExternalBTreeDataflowHelperFactory(mergePolicyFactory, mergePolicyFactoryProperties,
-                new SecondaryIndexOperationTrackerProvider(ds.getDatasetId()),
-                RuntimeComponentsProvider.RUNTIME_PROVIDER, LSMBTreeIOOperationCallbackFactory.INSTANCE,
-                storageProperties.getBloomFilterFalsePositiveRate(),
-                ExternalDatasetsRegistry.INSTANCE.getDatasetVersion(ds), true);
-    }
-
-    private static ExternalBTreeWithBuddyDataflowHelperFactory getBTreeDataflowHelperFactory(Dataset ds, Index index,
-            ILSMMergePolicyFactory mergePolicyFactory, Map<String, String> mergePolicyFactoryProperties,
-            StorageProperties storageProperties, JobSpecification spec) {
-        return new ExternalBTreeWithBuddyDataflowHelperFactory(mergePolicyFactory, mergePolicyFactoryProperties,
-                new SecondaryIndexOperationTrackerProvider(ds.getDatasetId()),
-                RuntimeComponentsProvider.RUNTIME_PROVIDER, LSMBTreeWithBuddyIOOperationCallbackFactory.INSTANCE,
-                storageProperties.getBloomFilterFalsePositiveRate(), new int[] { index.getKeyFieldNames().size() },
-                ExternalDatasetsRegistry.INSTANCE.getDatasetVersion(ds), true);
-    }
-
-    @SuppressWarnings("rawtypes")
-    private static ExternalRTreeDataflowHelperFactory getRTreeDataflowHelperFactory(Dataset ds, Index index,
-            ILSMMergePolicyFactory mergePolicyFactory, Map<String, String> mergePolicyFactoryProperties,
-            StorageProperties storageProperties, MetadataProvider metadataProvider, JobSpecification spec)
-            throws AlgebricksException, AsterixException {
-        int numPrimaryKeys = getRIDSize(ds);
-        List<List<String>> secondaryKeyFields = index.getKeyFieldNames();
-        secondaryKeyFields.size();
-        ARecordType itemType = (ARecordType) metadataProvider.findType(ds.getItemTypeDataverseName(),
-                ds.getItemTypeName());
-        Pair<IAType, Boolean> spatialTypePair = Index.getNonNullableKeyFieldType(secondaryKeyFields.get(0), itemType);
-        IAType spatialType = spatialTypePair.first;
-        if (spatialType == null) {
-            throw new AsterixException("Could not find field " + secondaryKeyFields.get(0) + " in the schema.");
-        }
-        boolean isPointMBR = spatialType.getTypeTag() == ATypeTag.POINT || spatialType.getTypeTag() == ATypeTag.POINT3D;
-        int numDimensions = NonTaggedFormatUtil.getNumDimensions(spatialType.getTypeTag());
-        int numNestedSecondaryKeyFields = numDimensions * 2;
-        IPrimitiveValueProviderFactory[] valueProviderFactories =
-                new IPrimitiveValueProviderFactory[numNestedSecondaryKeyFields];
-        IBinaryComparatorFactory[] secondaryComparatorFactories =
-                new IBinaryComparatorFactory[numNestedSecondaryKeyFields];
-
-        ISerializerDeserializer[] secondaryRecFields = new ISerializerDeserializer[numPrimaryKeys
-                + numNestedSecondaryKeyFields];
-        ITypeTraits[] secondaryTypeTraits = new ITypeTraits[numNestedSecondaryKeyFields + numPrimaryKeys];
-        IAType nestedKeyType = NonTaggedFormatUtil.getNestedSpatialType(spatialType.getTypeTag());
-        ATypeTag keyType = nestedKeyType.getTypeTag();
-
-        keyType = nestedKeyType.getTypeTag();
-        for (int i = 0; i < numNestedSecondaryKeyFields; i++) {
-            ISerializerDeserializer keySerde = SerializerDeserializerProvider.INSTANCE
-                    .getSerializerDeserializer(nestedKeyType);
-            secondaryRecFields[i] = keySerde;
-
-            secondaryComparatorFactories[i] = BinaryComparatorFactoryProvider.INSTANCE
-                    .getBinaryComparatorFactory(nestedKeyType, true);
-            secondaryTypeTraits[i] = TypeTraitProvider.INSTANCE.getTypeTrait(nestedKeyType);
-            valueProviderFactories[i] = PrimitiveValueProviderFactory.INSTANCE;
-        }
-        // Add serializers and comparators for primary index fields.
-        for (int i = 0; i < numPrimaryKeys; i++) {
-            secondaryRecFields[numNestedSecondaryKeyFields + i] = IndexingConstants.getSerializerDeserializer(i);
-            secondaryTypeTraits[numNestedSecondaryKeyFields + i] = IndexingConstants.getTypeTraits(i);
-        }
-        int[] primaryKeyFields = new int[numPrimaryKeys];
-        for (int i = 0; i < primaryKeyFields.length; i++) {
-            primaryKeyFields[i] = i + numNestedSecondaryKeyFields;
-        }
-
-        return new ExternalRTreeDataflowHelperFactory(valueProviderFactories, RTreePolicyType.RTREE,
-                getBuddyBtreeComparatorFactories(), mergePolicyFactory, mergePolicyFactoryProperties,
-                new SecondaryIndexOperationTrackerProvider(ds.getDatasetId()),
-                RuntimeComponentsProvider.RUNTIME_PROVIDER, LSMRTreeIOOperationCallbackFactory.INSTANCE,
-                MetadataProvider.proposeLinearizer(keyType, secondaryComparatorFactories.length),
-                storageProperties.getBloomFilterFalsePositiveRate(), new int[] { index.getKeyFieldNames().size() },
-                ExternalDatasetsRegistry.INSTANCE.getDatasetVersion(ds), true, isPointMBR);
-    }
-
-    public static JobSpecification buildAbortOp(Dataset ds, List<Index> indexes, MetadataProvider metadataProvider)
-            throws AlgebricksException, AsterixException {
-        JobSpecification spec = JobSpecificationUtils.createJobSpecification();
-        IPropertiesProvider asterixPropertiesProvider = AppContextInfo.INSTANCE;
-        StorageProperties storageProperties = asterixPropertiesProvider.getStorageProperties();
-        Pair<ILSMMergePolicyFactory, Map<String, String>> compactionInfo = DatasetUtils.getMergePolicyFactory(ds,
-                metadataProvider.getMetadataTxnContext());
-        ILSMMergePolicyFactory mergePolicyFactory = compactionInfo.first;
-        Map<String, String> mergePolicyFactoryProperties = compactionInfo.second;
-
-        boolean temp = ds.getDatasetDetails().isTemp();
-        Pair<IFileSplitProvider, AlgebricksPartitionConstraint> filesIndexSplitsAndConstraint = metadataProvider
-                .splitProviderAndPartitionConstraintsForDataset(ds.getDataverseName(), ds.getDatasetName(),
-                        getFilesIndexName(ds.getDatasetName()), temp);
-        IFileSplitProvider filesIndexSplitProvider = filesIndexSplitsAndConstraint.first;
-        ExternalBTreeDataflowHelperFactory filesIndexDataflowHelperFactory = getFilesIndexDataflowHelperFactory(ds,
-                mergePolicyFactory, mergePolicyFactoryProperties, storageProperties, spec);
-        IndexInfoOperatorDescriptor filesIndexInfo = new IndexInfoOperatorDescriptor(filesIndexSplitProvider,
-                RuntimeComponentsProvider.RUNTIME_PROVIDER, RuntimeComponentsProvider.RUNTIME_PROVIDER);
-
-        ArrayList<ExternalBTreeWithBuddyDataflowHelperFactory> btreeDataflowHelperFactories = new ArrayList<>();
-        ArrayList<IndexInfoOperatorDescriptor> btreeInfos = new ArrayList<>();
-        ArrayList<ExternalRTreeDataflowHelperFactory> rtreeDataflowHelperFactories = new ArrayList<>();
-        ArrayList<IndexInfoOperatorDescriptor> rtreeInfos = new ArrayList<>();
-
-        for (Index index : indexes) {
-            if (isValidIndexName(index.getDatasetName(), index.getIndexName())) {
-                Pair<IFileSplitProvider, AlgebricksPartitionConstraint> indexSplitsAndConstraint = metadataProvider
-                        .splitProviderAndPartitionConstraintsForDataset(ds.getDataverseName(), ds.getDatasetName(),
-                                index.getIndexName(), temp);
-                if (index.getIndexType() == IndexType.BTREE) {
-                    btreeDataflowHelperFactories.add(getBTreeDataflowHelperFactory(ds, index, mergePolicyFactory,
-                            mergePolicyFactoryProperties, storageProperties, spec));
-                    btreeInfos.add(new IndexInfoOperatorDescriptor(indexSplitsAndConstraint.first,
-                            RuntimeComponentsProvider.RUNTIME_PROVIDER,
-                            RuntimeComponentsProvider.RUNTIME_PROVIDER));
-                } else if (index.getIndexType() == IndexType.RTREE) {
-                    rtreeDataflowHelperFactories.add(getRTreeDataflowHelperFactory(ds, index, mergePolicyFactory,
-                            mergePolicyFactoryProperties, storageProperties, metadataProvider, spec));
-                    rtreeInfos.add(new IndexInfoOperatorDescriptor(indexSplitsAndConstraint.first,
-                            RuntimeComponentsProvider.RUNTIME_PROVIDER,
-                            RuntimeComponentsProvider.RUNTIME_PROVIDER));
-                }
-            }
-        }
-
-        ExternalDatasetIndexesAbortOperatorDescriptor op = new ExternalDatasetIndexesAbortOperatorDescriptor(spec,
-                filesIndexDataflowHelperFactory, filesIndexInfo, btreeDataflowHelperFactories, btreeInfos,
-                rtreeDataflowHelperFactories, rtreeInfos);
-
-        spec.addRoot(op);
-        AlgebricksPartitionConstraintHelper.setPartitionConstraintInJobSpec(spec, op,
-                filesIndexSplitsAndConstraint.second);
-        spec.setConnectorPolicyAssignmentPolicy(new ConnectorPolicyAssignmentPolicy());
-        return spec;
-
-    }
-
-    public static JobSpecification buildRecoverOp(Dataset ds, List<Index> indexes, MetadataProvider metadataProvider)
-            throws AlgebricksException, AsterixException {
-        JobSpecification spec = JobSpecificationUtils.createJobSpecification();
-        IPropertiesProvider asterixPropertiesProvider = AppContextInfo.INSTANCE;
-        StorageProperties storageProperties = asterixPropertiesProvider.getStorageProperties();
-        Pair<ILSMMergePolicyFactory, Map<String, String>> compactionInfo = DatasetUtils.getMergePolicyFactory(ds,
-                metadataProvider.getMetadataTxnContext());
-        ILSMMergePolicyFactory mergePolicyFactory = compactionInfo.first;
-        Map<String, String> mergePolicyFactoryProperties = compactionInfo.second;
-        boolean temp = ds.getDatasetDetails().isTemp();
-
-        Pair<IFileSplitProvider, AlgebricksPartitionConstraint> filesIndexSplitsAndConstraint = metadataProvider
-                .splitProviderAndPartitionConstraintsForDataset(ds.getDataverseName(), ds.getDatasetName(),
-                        getFilesIndexName(ds.getDatasetName()), temp);
-        IFileSplitProvider filesIndexSplitProvider = filesIndexSplitsAndConstraint.first;
-        ExternalBTreeDataflowHelperFactory filesIndexDataflowHelperFactory = getFilesIndexDataflowHelperFactory(ds,
-                mergePolicyFactory, mergePolicyFactoryProperties, storageProperties, spec);
-        IndexInfoOperatorDescriptor filesIndexInfo = new IndexInfoOperatorDescriptor(filesIndexSplitProvider,
-                RuntimeComponentsProvider.RUNTIME_PROVIDER, RuntimeComponentsProvider.RUNTIME_PROVIDER);
-
-        ArrayList<ExternalBTreeWithBuddyDataflowHelperFactory> btreeDataflowHelperFactories = new ArrayList<>();
-        ArrayList<IndexInfoOperatorDescriptor> btreeInfos = new ArrayList<>();
-        ArrayList<ExternalRTreeDataflowHelperFactory> rtreeDataflowHelperFactories = new ArrayList<>();
-        ArrayList<IndexInfoOperatorDescriptor> rtreeInfos = new ArrayList<>();
-
-        for (Index index : indexes) {
-            if (isValidIndexName(index.getDatasetName(), index.getIndexName())) {
-                Pair<IFileSplitProvider, AlgebricksPartitionConstraint> indexSplitsAndConstraint = metadataProvider
-                        .splitProviderAndPartitionConstraintsForDataset(ds.getDataverseName(), ds.getDatasetName(),
-                                index.getIndexName(), temp);
-                if (index.getIndexType() == IndexType.BTREE) {
-                    btreeDataflowHelperFactories.add(getBTreeDataflowHelperFactory(ds, index, mergePolicyFactory,
-                            mergePolicyFactoryProperties, storageProperties, spec));
-                    btreeInfos.add(new IndexInfoOperatorDescriptor(indexSplitsAndConstraint.first,
-                            RuntimeComponentsProvider.RUNTIME_PROVIDER,
-                            RuntimeComponentsProvider.RUNTIME_PROVIDER));
-                } else if (index.getIndexType() == IndexType.RTREE) {
-                    rtreeDataflowHelperFactories.add(getRTreeDataflowHelperFactory(ds, index, mergePolicyFactory,
-                            mergePolicyFactoryProperties, storageProperties, metadataProvider, spec));
-                    rtreeInfos.add(new IndexInfoOperatorDescriptor(indexSplitsAndConstraint.first,
-                            RuntimeComponentsProvider.RUNTIME_PROVIDER,
-                            RuntimeComponentsProvider.RUNTIME_PROVIDER));
-                }
-            }
-        }
-
-        ExternalDatasetIndexesRecoverOperatorDescriptor op = new ExternalDatasetIndexesRecoverOperatorDescriptor(spec,
-                filesIndexDataflowHelperFactory, filesIndexInfo, btreeDataflowHelperFactories, btreeInfos,
-                rtreeDataflowHelperFactories, rtreeInfos);
-
-        spec.addRoot(op);
-        AlgebricksPartitionConstraintHelper.setPartitionConstraintInJobSpec(spec, op,
-                filesIndexSplitsAndConstraint.second);
-        spec.setConnectorPolicyAssignmentPolicy(new ConnectorPolicyAssignmentPolicy());
-        return spec;
-    }
-
-    public static JobSpecification compactFilesIndexJobSpec(Dataset dataset, MetadataProvider metadataProvider)
-            throws MetadataException, AlgebricksException {
-        JobSpecification spec = JobSpecificationUtils.createJobSpecification();
-        IPropertiesProvider asterixPropertiesProvider = AppContextInfo.INSTANCE;
-        StorageProperties storageProperties = asterixPropertiesProvider.getStorageProperties();
-        Pair<ILSMMergePolicyFactory, Map<String, String>> compactionInfo = DatasetUtils.getMergePolicyFactory(dataset,
-                metadataProvider.getMetadataTxnContext());
-        ILSMMergePolicyFactory mergePolicyFactory = compactionInfo.first;
-        Map<String, String> mergePolicyFactoryProperties = compactionInfo.second;
-        Pair<IFileSplitProvider, AlgebricksPartitionConstraint> secondarySplitsAndConstraint = metadataProvider
-                .splitProviderAndPartitionConstraintsForFilesIndex(dataset.getDataverseName(), dataset.getDatasetName(),
-                        getFilesIndexName(dataset.getDatasetName()), true);
-        IFileSplitProvider secondaryFileSplitProvider = secondarySplitsAndConstraint.first;
-        ExternalBTreeDataflowHelperFactory indexDataflowHelperFactory = new ExternalBTreeDataflowHelperFactory(
-                mergePolicyFactory, mergePolicyFactoryProperties,
-                new SecondaryIndexOperationTrackerProvider(dataset.getDatasetId()),
-                RuntimeComponentsProvider.RUNTIME_PROVIDER, LSMBTreeIOOperationCallbackFactory.INSTANCE,
-                storageProperties.getBloomFilterFalsePositiveRate(),
-                ExternalDatasetsRegistry.INSTANCE.getDatasetVersion(dataset), true);
-        FilesIndexDescription filesIndexDescription = new FilesIndexDescription();
-        LSMTreeIndexCompactOperatorDescriptor compactOp = new LSMTreeIndexCompactOperatorDescriptor(spec,
-                RuntimeComponentsProvider.RUNTIME_PROVIDER, RuntimeComponentsProvider.RUNTIME_PROVIDER,
-                secondaryFileSplitProvider, filesIndexDescription.EXTERNAL_FILE_INDEX_TYPE_TRAITS,
-                filesIndexDescription.FILES_INDEX_COMP_FACTORIES, new int[] { 0 }, indexDataflowHelperFactory,
-                NoOpOperationCallbackFactory.INSTANCE, LSMIndexUtil.getMetadataPageManagerFactory());
-        spec.addRoot(compactOp);
-        AlgebricksPartitionConstraintHelper.setPartitionConstraintInJobSpec(spec, compactOp,
-                secondarySplitsAndConstraint.second);
-        spec.setConnectorPolicyAssignmentPolicy(new ConnectorPolicyAssignmentPolicy());
-        return spec;
-    }
-}
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/external/ExternalLibraryUtils.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/external/ExternalLibraryUtils.java
index 95fe68c..4680465 100755
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/external/ExternalLibraryUtils.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/external/ExternalLibraryUtils.java
@@ -43,11 +43,11 @@
 import org.apache.asterix.external.library.LibraryFunction;
 import org.apache.asterix.metadata.MetadataManager;
 import org.apache.asterix.metadata.MetadataTransactionContext;
-import org.apache.asterix.metadata.api.IMetadataEntity;
 import org.apache.asterix.metadata.entities.DatasourceAdapter;
 import org.apache.asterix.metadata.entities.Dataverse;
 import org.apache.asterix.metadata.entities.Function;
 import org.apache.asterix.metadata.entities.Library;
+import org.apache.asterix.metadata.utils.MetadataUtil;
 import org.apache.asterix.runtime.formats.NonTaggedDataFormat;
 
 public class ExternalLibraryUtils {
@@ -81,7 +81,7 @@
                         // get library file
                         File libraryDir = new File(installLibDir.getAbsolutePath() + File.separator + dataverse
                                 + File.separator + library);
-                        // install if needed (i,e, add the functions, adapters, datasources, parsers to the metadata) <Not required for use>
+                        // install if needed (i,e, add the functions, adapters, datasources, parsers to the metadata)
                         installLibraryIfNeeded(dataverse, libraryDir, uninstalledLibs);
                     }
                 }
@@ -96,7 +96,7 @@
      * @throws Exception
      */
     private static Map<String, List<String>> uninstallLibraries() throws Exception {
-        Map<String, List<String>> uninstalledLibs = new HashMap<String, List<String>>();
+        Map<String, List<String>> uninstalledLibs = new HashMap<>();
         // get the directory of the un-install libraries
         File uninstallLibDir = getLibraryUninstallDir();
         String[] uninstallLibNames;
@@ -116,7 +116,7 @@
                 // add the library to the list of uninstalled libraries
                 List<String> uinstalledLibsInDv = uninstalledLibs.get(dataverse);
                 if (uinstalledLibsInDv == null) {
-                    uinstalledLibsInDv = new ArrayList<String>();
+                    uinstalledLibsInDv = new ArrayList<>();
                     uninstalledLibs.put(dataverse, uinstalledLibsInDv);
                 }
                 uinstalledLibsInDv.add(libName);
@@ -172,7 +172,8 @@
                 // belong to the library?
                 if (adapter.getAdapterIdentifier().getName().startsWith(libraryName + "#")) {
                     // remove adapter <! we didn't check if there are feeds which use this adapter>
-                    MetadataManager.INSTANCE.dropAdapter(mdTxnCtx, dataverse, adapter.getAdapterIdentifier().getName());
+                    MetadataManager.INSTANCE.dropAdapter(mdTxnCtx, dataverse,
+                            adapter.getAdapterIdentifier().getName());
                 }
             }
             // drop the library itself
@@ -203,7 +204,8 @@
             Library libraryInMetadata = MetadataManager.INSTANCE.getLibrary(mdTxnCtx, dataverse, libraryName);
             if (libraryInMetadata != null && !wasUninstalled) {
                 // exists in metadata and was not un-installed, we return.
-                // Another place which shows that our metadata transactions are broken (we didn't call commit before!!!)
+                // Another place which shows that our metadata transactions are broken
+                // (we didn't call commit before!!!)
                 MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
                 return;
             }
@@ -235,13 +237,13 @@
             Dataverse dv = MetadataManager.INSTANCE.getDataverse(mdTxnCtx, dataverse);
             if (dv == null) {
                 MetadataManager.INSTANCE.addDataverse(mdTxnCtx, new Dataverse(dataverse,
-                        NonTaggedDataFormat.NON_TAGGED_DATA_FORMAT, IMetadataEntity.PENDING_NO_OP));
+                        NonTaggedDataFormat.NON_TAGGED_DATA_FORMAT, MetadataUtil.PENDING_NO_OP));
             }
             // Add functions
             if (library.getLibraryFunctions() != null) {
                 for (LibraryFunction function : library.getLibraryFunctions().getLibraryFunction()) {
                     String[] fargs = function.getArguments().trim().split(",");
-                    List<String> args = new ArrayList<String>();
+                    List<String> args = new ArrayList<>();
                     for (String arg : fargs) {
                         args.add(arg);
                     }
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/external/FeedWorkCollection.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/external/FeedWorkCollection.java
index 6be7af9..ec7c239 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/external/FeedWorkCollection.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/external/FeedWorkCollection.java
@@ -24,17 +24,19 @@
 
 import org.apache.asterix.app.translator.DefaultStatementExecutorFactory;
 import org.apache.asterix.app.translator.QueryTranslator;
+import org.apache.asterix.common.context.IStorageComponentProvider;
 import org.apache.asterix.compiler.provider.AqlCompilationProvider;
 import org.apache.asterix.compiler.provider.ILangCompilationProvider;
 import org.apache.asterix.external.feed.api.IFeedWork;
 import org.apache.asterix.external.feed.api.IFeedWorkEventListener;
 import org.apache.asterix.external.feed.management.FeedConnectionRequest;
 import org.apache.asterix.external.feed.management.FeedConnectionRequest.ConnectionStatus;
+import org.apache.asterix.file.StorageComponentProvider;
 import org.apache.asterix.lang.aql.statement.SubscribeFeedStatement;
 import org.apache.asterix.lang.common.base.Statement;
 import org.apache.asterix.lang.common.statement.DataverseDecl;
 import org.apache.asterix.lang.common.struct.Identifier;
-import org.apache.asterix.runtime.util.AppContextInfo;
+import org.apache.asterix.runtime.utils.AppContextInfo;
 import org.apache.asterix.translator.IStatementExecutor;
 import org.apache.asterix.translator.SessionConfig;
 import org.apache.asterix.translator.SessionConfig.OutputFormat;
@@ -48,6 +50,7 @@
 
     private static Logger LOGGER = Logger.getLogger(FeedWorkCollection.class.getName());
     private static final ILangCompilationProvider compilationProvider = new AqlCompilationProvider();
+    private static final IStorageComponentProvider storageComponentProvider = new StorageComponentProvider();
 
     /**
      * The task of subscribing to a feed to obtain data.
@@ -91,7 +94,8 @@
                     List<Statement> statements = new ArrayList<>();
                     statements.add(dataverseDecl);
                     statements.add(subscribeStmt);
-                    IStatementExecutor translator = qtFactory.create(statements, pc, compilationProvider);
+                    IStatementExecutor translator = qtFactory.create(statements, pc, compilationProvider,
+                            storageComponentProvider);
                     translator.compileAndExecute(AppContextInfo.INSTANCE.getHcc(), null,
                             QueryTranslator.ResultDelivery.IMMEDIATE);
                     if (LOGGER.isEnabledFor(Level.INFO)) {
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/nc/NCAppRuntimeContext.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/nc/NCAppRuntimeContext.java
index b1ca062b..b114e8c 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/nc/NCAppRuntimeContext.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/nc/NCAppRuntimeContext.java
@@ -29,26 +29,26 @@
 
 import org.apache.asterix.active.ActiveManager;
 import org.apache.asterix.api.common.AppRuntimeContextProviderForRecovery;
-import org.apache.asterix.common.api.ThreadExecutor;
 import org.apache.asterix.common.api.IAppRuntimeContext;
 import org.apache.asterix.common.api.IDatasetLifecycleManager;
+import org.apache.asterix.common.api.ThreadExecutor;
 import org.apache.asterix.common.cluster.ClusterPartition;
-import org.apache.asterix.common.config.BuildProperties;
-import org.apache.asterix.common.config.CompilerProperties;
 import org.apache.asterix.common.config.AsterixExtension;
+import org.apache.asterix.common.config.BuildProperties;
+import org.apache.asterix.common.config.ClusterProperties;
+import org.apache.asterix.common.config.CompilerProperties;
 import org.apache.asterix.common.config.ExtensionProperties;
 import org.apache.asterix.common.config.ExternalProperties;
 import org.apache.asterix.common.config.FeedProperties;
+import org.apache.asterix.common.config.MessagingProperties;
 import org.apache.asterix.common.config.MetadataProperties;
 import org.apache.asterix.common.config.PropertiesAccessor;
 import org.apache.asterix.common.config.ReplicationProperties;
 import org.apache.asterix.common.config.StorageProperties;
 import org.apache.asterix.common.config.TransactionProperties;
-import org.apache.asterix.common.config.ClusterProperties;
-import org.apache.asterix.common.config.IPropertiesProvider;
-import org.apache.asterix.common.config.MessagingProperties;
-import org.apache.asterix.common.context.FileMapManager;
 import org.apache.asterix.common.context.DatasetLifecycleManager;
+import org.apache.asterix.common.context.FileMapManager;
+import org.apache.asterix.common.context.IStorageComponentProvider;
 import org.apache.asterix.common.exceptions.ACIDException;
 import org.apache.asterix.common.exceptions.AsterixException;
 import org.apache.asterix.common.library.ILibraryManager;
@@ -62,6 +62,7 @@
 import org.apache.asterix.common.transactions.IRecoveryManager.SystemState;
 import org.apache.asterix.common.transactions.ITransactionSubsystem;
 import org.apache.asterix.external.library.ExternalLibraryManager;
+import org.apache.asterix.file.StorageComponentProvider;
 import org.apache.asterix.metadata.MetadataManager;
 import org.apache.asterix.metadata.MetadataNode;
 import org.apache.asterix.metadata.api.IAsterixStateProxy;
@@ -74,7 +75,6 @@
 import org.apache.asterix.runtime.transaction.GlobalResourceIdFactoryProvider;
 import org.apache.asterix.transaction.management.resource.PersistentLocalResourceRepository;
 import org.apache.asterix.transaction.management.resource.PersistentLocalResourceRepositoryFactory;
-import org.apache.asterix.transaction.management.service.transaction.TransactionSubsystem;
 import org.apache.hyracks.api.application.INCApplicationContext;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.api.io.IIOManager;
@@ -99,13 +99,12 @@
 import org.apache.hyracks.storage.common.file.ILocalResourceRepositoryFactory;
 import org.apache.hyracks.storage.common.file.IResourceIdFactory;
 
-public class NCAppRuntimeContext implements IAppRuntimeContext, IPropertiesProvider {
+public class NCAppRuntimeContext implements IAppRuntimeContext {
     private static final Logger LOGGER = Logger.getLogger(NCAppRuntimeContext.class.getName());
 
     private ILSMMergePolicyFactory metadataMergePolicyFactory;
     private final INCApplicationContext ncApplicationContext;
     private final IResourceIdFactory resourceIdFactory;
-
     private CompilerProperties compilerProperties;
     private ExternalProperties externalProperties;
     private MetadataProperties metadataProperties;
@@ -115,7 +114,6 @@
     private BuildProperties buildProperties;
     private ReplicationProperties replicationProperties;
     private MessagingProperties messagingProperties;
-
     private ThreadExecutor threadExecutor;
     private IDatasetLifecycleManager datasetLifecycleManager;
     private IFileMapManager fileMapManager;
@@ -136,14 +134,14 @@
 
     private final ILibraryManager libraryManager;
     private final NCExtensionManager ncExtensionManager;
+    private final IStorageComponentProvider componentProvider;
 
     public NCAppRuntimeContext(INCApplicationContext ncApplicationContext, List<AsterixExtension> extensions)
-            throws AsterixException, InstantiationException, IllegalAccessException,
-            ClassNotFoundException, IOException {
+            throws AsterixException, InstantiationException, IllegalAccessException, ClassNotFoundException,
+            IOException {
         List<AsterixExtension> allExtensions = new ArrayList<>();
         this.ncApplicationContext = ncApplicationContext;
-        PropertiesAccessor propertiesAccessor =
-                PropertiesAccessor.getInstance(ncApplicationContext.getAppConfig());
+        PropertiesAccessor propertiesAccessor = PropertiesAccessor.getInstance(ncApplicationContext.getAppConfig());
         compilerProperties = new CompilerProperties(propertiesAccessor);
         externalProperties = new ExternalProperties(propertiesAccessor);
         metadataProperties = new MetadataProperties(propertiesAccessor);
@@ -159,6 +157,7 @@
         }
         allExtensions.addAll(new ExtensionProperties(propertiesAccessor).getExtensions());
         ncExtensionManager = new NCExtensionManager(allExtensions);
+        componentProvider = new StorageComponentProvider();
         resourceIdFactory = new GlobalResourceIdFactoryProvider(ncApplicationContext).createResourceIdFactory();
     }
 
@@ -181,16 +180,15 @@
         metadataMergePolicyFactory = new PrefixMergePolicyFactory();
 
         ILocalResourceRepositoryFactory persistentLocalResourceRepositoryFactory =
-                new PersistentLocalResourceRepositoryFactory(
-                        ioManager, ncApplicationContext.getNodeId(), metadataProperties);
+                new PersistentLocalResourceRepositoryFactory(ioManager, ncApplicationContext.getNodeId(),
+                        metadataProperties);
 
-        localResourceRepository = (PersistentLocalResourceRepository) persistentLocalResourceRepositoryFactory
-                .createRepository();
+        localResourceRepository =
+                (PersistentLocalResourceRepository) persistentLocalResourceRepositoryFactory.createRepository();
 
-        IAppRuntimeContextProvider asterixAppRuntimeContextProvider =
-                new AppRuntimeContextProviderForRecovery(this);
-        txnSubsystem = new TransactionSubsystem(ncApplicationContext.getNodeId(), asterixAppRuntimeContextProvider,
-                txnProperties);
+        IAppRuntimeContextProvider asterixAppRuntimeContextProvider = new AppRuntimeContextProviderForRecovery(this);
+        txnSubsystem = new TransactionSubsystem(ncApplicationContext, ncApplicationContext.getNodeId(),
+                asterixAppRuntimeContextProvider, txnProperties);
 
         IRecoveryManager recoveryMgr = txnSubsystem.getRecoveryManager();
         SystemState systemState = recoveryMgr.getSystemState();
@@ -448,9 +446,9 @@
         // This way we can delay the registration of the metadataNode until
         // it is completely initialized.
         MetadataManager.initialize(proxy, MetadataNode.INSTANCE);
-        MetadataBootstrap.startUniverse(this, ncApplicationContext, newUniverse);
+        MetadataBootstrap.startUniverse(ncApplicationContext, newUniverse);
         MetadataBootstrap.startDDLRecovery();
-        ncExtensionManager.initializeMetadata();
+        ncExtensionManager.initializeMetadata(ncApplicationContext);
 
         if (LOGGER.isLoggable(Level.INFO)) {
             LOGGER.info("Metadata node bound");
@@ -473,4 +471,9 @@
         return ncExtensionManager;
     }
 
+    @Override
+    public IStorageComponentProvider getStorageComponentProvider() {
+        return componentProvider;
+    }
+
 }
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/nc/NCExtensionManager.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/nc/NCExtensionManager.java
index 9e32b7e..9937479 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/nc/NCExtensionManager.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/nc/NCExtensionManager.java
@@ -25,17 +25,18 @@
 import org.apache.asterix.common.api.IExtension;
 import org.apache.asterix.common.config.AsterixExtension;
 import org.apache.asterix.common.exceptions.ACIDException;
-import org.apache.asterix.common.exceptions.RuntimeDataException;
-import org.apache.asterix.common.exceptions.ErrorCode;
 import org.apache.asterix.metadata.api.IMetadataExtension;
+import org.apache.asterix.metadata.api.INCExtensionManager;
 import org.apache.asterix.metadata.entitytupletranslators.MetadataTupleTranslatorProvider;
+import org.apache.asterix.utils.ExtensionUtil;
+import org.apache.hyracks.api.application.INCApplicationContext;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 
 /**
  * AsterixDB's implementation of {@code INCExtensionManager} which takes care of
  * initializing extensions on Node Controllers
  */
-public class NCExtensionManager {
+public class NCExtensionManager implements INCExtensionManager {
 
     private final MetadataTupleTranslatorProvider tupleTranslatorProvider;
     private final List<IMetadataExtension> mdExtensions;
@@ -44,15 +45,19 @@
      * Initialize {@code CCExtensionManager} from configuration
      *
      * @param list
+     *            list of user configured extensions
      * @throws InstantiationException
+     *             if an extension couldn't be created
      * @throws IllegalAccessException
+     *             if user doesn't have enough acess priveleges
      * @throws ClassNotFoundException
+     *             if a class was not found
      * @throws HyracksDataException
+     *             if two extensions conlict with each other
      */
     public NCExtensionManager(List<AsterixExtension> list)
             throws InstantiationException, IllegalAccessException, ClassNotFoundException, HyracksDataException {
-        MetadataTupleTranslatorProvider ttp = null;
-        IMetadataExtension tupleTranslatorExtension = null;
+        IMetadataExtension tupleTranslatorProviderExtension = null;
         mdExtensions = new ArrayList<>();
         if (list != null) {
             for (AsterixExtension extensionConf : list) {
@@ -62,31 +67,23 @@
                     case METADATA:
                         IMetadataExtension mde = (IMetadataExtension) extension;
                         mdExtensions.add(mde);
-                        ttp = extendTupleTranslator(ttp, tupleTranslatorExtension, mde);
-                        tupleTranslatorExtension = ttp == null ? null : mde;
+                        tupleTranslatorProviderExtension =
+                                ExtensionUtil.extendTupleTranslatorProvider(tupleTranslatorProviderExtension, mde);
                         break;
                     default:
                         break;
                 }
             }
         }
-        this.tupleTranslatorProvider = ttp == null ? new MetadataTupleTranslatorProvider() : ttp;
-    }
-
-    private MetadataTupleTranslatorProvider extendTupleTranslator(MetadataTupleTranslatorProvider ttp,
-            IMetadataExtension tupleTranslatorExtension, IMetadataExtension mde) throws HyracksDataException {
-        if (ttp != null) {
-            throw new RuntimeDataException(ErrorCode.EXTENSION_COMPONENT_CONFLICT,
-                    tupleTranslatorExtension.getId(),
-                    mde.getId(), IMetadataExtension.class.getSimpleName());
-        }
-        return mde.getMetadataTupleTranslatorProvider();
+        this.tupleTranslatorProvider = tupleTranslatorProviderExtension == null ? new MetadataTupleTranslatorProvider()
+                : tupleTranslatorProviderExtension.getMetadataTupleTranslatorProvider();
     }
 
     public List<IMetadataExtension> getMetadataExtensions() {
         return mdExtensions;
     }
 
+    @Override
     public MetadataTupleTranslatorProvider getMetadataTupleTranslatorProvider() {
         return tupleTranslatorProvider;
     }
@@ -94,13 +91,15 @@
     /**
      * Called on bootstrap of metadata node allowing extensions to instantiate their Metadata artifacts
      *
+     * @param ncApplicationContext
+     *            the node controller application context
      * @throws HyracksDataException
      */
-    public void initializeMetadata() throws HyracksDataException {
+    public void initializeMetadata(INCApplicationContext appCtx) throws HyracksDataException {
         if (mdExtensions != null) {
             for (IMetadataExtension mdExtension : mdExtensions) {
                 try {
-                    mdExtension.initializeMetadata();
+                    mdExtension.initializeMetadata(appCtx);
                 } catch (RemoteException | ACIDException e) {
                     throw new HyracksDataException(e);
                 }
diff --git a/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/recovery/RecoveryManager.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/nc/RecoveryManager.java
similarity index 88%
rename from asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/recovery/RecoveryManager.java
rename to asterixdb/asterix-app/src/main/java/org/apache/asterix/app/nc/RecoveryManager.java
index f8b6384..4edf991 100644
--- a/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/recovery/RecoveryManager.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/nc/RecoveryManager.java
@@ -16,7 +16,7 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.asterix.transaction.management.service.recovery;
+package org.apache.asterix.app.nc;
 
 import java.io.File;
 import java.io.FileInputStream;
@@ -54,13 +54,16 @@
 import org.apache.asterix.common.transactions.ILogRecord;
 import org.apache.asterix.common.transactions.IRecoveryManager;
 import org.apache.asterix.common.transactions.ITransactionContext;
+import org.apache.asterix.common.transactions.ITransactionSubsystem;
 import org.apache.asterix.common.transactions.LogType;
 import org.apache.asterix.common.transactions.Resource;
 import org.apache.asterix.transaction.management.resource.PersistentLocalResourceRepository;
 import org.apache.asterix.transaction.management.service.logging.LogManager;
+import org.apache.asterix.transaction.management.service.recovery.AbstractCheckpointManager;
+import org.apache.asterix.transaction.management.service.recovery.TxnId;
 import org.apache.asterix.transaction.management.service.transaction.TransactionManagementConstants;
-import org.apache.asterix.transaction.management.service.transaction.TransactionSubsystem;
 import org.apache.commons.io.FileUtils;
+import org.apache.hyracks.api.application.INCApplicationContext;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.api.lifecycle.ILifeCycleComponent;
 import org.apache.hyracks.storage.am.common.api.IIndex;
@@ -79,7 +82,7 @@
 
     public static final boolean IS_DEBUG_MODE = false;
     private static final Logger LOGGER = Logger.getLogger(RecoveryManager.class.getName());
-    private final TransactionSubsystem txnSubsystem;
+    private final ITransactionSubsystem txnSubsystem;
     private final LogManager logMgr;
     private final boolean replicationEnabled;
     private static final String RECOVERY_FILES_DIR_NAME = "recovery_temp";
@@ -88,13 +91,15 @@
     private final PersistentLocalResourceRepository localResourceRepository;
     private final ICheckpointManager checkpointManager;
     private SystemState state;
+    private final INCApplicationContext appCtx;
 
-    public RecoveryManager(TransactionSubsystem txnSubsystem) {
+    public RecoveryManager(ITransactionSubsystem txnSubsystem, INCApplicationContext appCtx) {
+        this.appCtx = appCtx;
         this.txnSubsystem = txnSubsystem;
         logMgr = (LogManager) txnSubsystem.getLogManager();
         replicationEnabled = ClusterProperties.INSTANCE.isReplicationEnabled();
-        localResourceRepository = (PersistentLocalResourceRepository) txnSubsystem.getAsterixAppRuntimeContextProvider()
-                .getLocalResourceRepository();
+        localResourceRepository = (PersistentLocalResourceRepository) txnSubsystem
+                .getAsterixAppRuntimeContextProvider().getLocalResourceRepository();
         cachedEntityCommitsPerJobSize = txnSubsystem.getTransactionProperties().getJobRecoveryMemorySize();
         checkpointManager = txnSubsystem.getCheckpointManager();
     }
@@ -122,12 +127,9 @@
         }
 
         if (replicationEnabled) {
-            if (checkpointObject.getMinMCTFirstLsn() == AbstractCheckpointManager.SHARP_CHECKPOINT_LSN) {
-                //no logs exist
-                state = SystemState.HEALTHY;
-                return state;
-            } else if (checkpointObject.getCheckpointLsn() == logMgr.getAppendLSN() && checkpointObject.isSharp()) {
-                //only remote logs exist
+            if (checkpointObject.getMinMCTFirstLsn() == AbstractCheckpointManager.SHARP_CHECKPOINT_LSN
+                    || (checkpointObject.getCheckpointLsn() == logMgr.getAppendLSN() && checkpointObject.isSharp())) {
+                //no logs exist or only remote logs exist
                 state = SystemState.HEALTHY;
                 return state;
             } else {
@@ -194,18 +196,11 @@
         int entityCommitLogCount = 0;
         int jobCommitLogCount = 0;
         int abortLogCount = 0;
-        int jobId = -1;
-
         Set<Integer> winnerJobSet = new HashSet<>();
         jobId2WinnerEntitiesMap = new HashMap<>();
-
         //set log reader to the lowWaterMarkLsn
-        ILogRecord logRecord = null;
+        ILogRecord logRecord;
         logReader.initializeScan(lowWaterMarkLSN);
-
-        //collect all committed Lsn
-        JobEntityCommits jobEntityWinners = null;
-
         logRecord = logReader.next();
         while (logRecord != null) {
             if (IS_DEBUG_MODE) {
@@ -218,32 +213,14 @@
                     }
                     break;
                 case LogType.JOB_COMMIT:
-                    jobId = logRecord.getJobId();
-                    winnerJobSet.add(jobId);
-                    if (jobId2WinnerEntitiesMap.containsKey(jobId)) {
-                        jobEntityWinners = jobId2WinnerEntitiesMap.get(jobId);
-                        //to delete any spilled files as well
-                        jobEntityWinners.clear();
-                        jobId2WinnerEntitiesMap.remove(jobId);
-                    }
+                    winnerJobSet.add(logRecord.getJobId());
+                    cleanupJobCommits(logRecord.getJobId());
                     jobCommitLogCount++;
                     break;
                 case LogType.ENTITY_COMMIT:
                 case LogType.UPSERT_ENTITY_COMMIT:
                     if (partitions.contains(logRecord.getResourcePartition())) {
-                        jobId = logRecord.getJobId();
-                        if (!jobId2WinnerEntitiesMap.containsKey(jobId)) {
-                            jobEntityWinners = new JobEntityCommits(jobId);
-                            if (needToFreeMemory()) {
-                                //if we don't have enough memory for one more job, we will force all jobs to spill their cached entities to disk.
-                                //This could happen only when we have many jobs with small number of records and none of them have job commit.
-                                freeJobsCachedEntities(jobId);
-                            }
-                            jobId2WinnerEntitiesMap.put(jobId, jobEntityWinners);
-                        } else {
-                            jobEntityWinners = jobId2WinnerEntitiesMap.get(jobId);
-                        }
-                        jobEntityWinners.add(logRecord);
+                        analyzeEntityCommitLog(logRecord);
                         entityCommitLogCount++;
                     }
                     break;
@@ -272,6 +249,34 @@
         return winnerJobSet;
     }
 
+    private void cleanupJobCommits(int jobId) {
+        if (jobId2WinnerEntitiesMap.containsKey(jobId)) {
+            JobEntityCommits jobEntityWinners = jobId2WinnerEntitiesMap.get(jobId);
+            //to delete any spilled files as well
+            jobEntityWinners.clear();
+            jobId2WinnerEntitiesMap.remove(jobId);
+        }
+    }
+
+    private void analyzeEntityCommitLog(ILogRecord logRecord) throws IOException {
+        int jobId = logRecord.getJobId();
+        JobEntityCommits jobEntityWinners;
+        if (!jobId2WinnerEntitiesMap.containsKey(jobId)) {
+            jobEntityWinners = new JobEntityCommits(jobId);
+            if (needToFreeMemory()) {
+                // If we don't have enough memory for one more job,
+                // we will force all jobs to spill their cached entities to disk.
+                // This could happen only when we have many jobs with small
+                // number of records and none of them have job commit.
+                freeJobsCachedEntities(jobId);
+            }
+            jobId2WinnerEntitiesMap.put(jobId, jobEntityWinners);
+        } else {
+            jobEntityWinners = jobId2WinnerEntitiesMap.get(jobId);
+        }
+        jobEntityWinners.add(logRecord);
+    }
+
     private synchronized void startRecoveryRedoPhase(Set<Integer> partitions, ILogReader logReader,
             long lowWaterMarkLSN, Set<Integer> winnerJobSet) throws IOException, ACIDException {
         int redoCount = 0;
@@ -279,7 +284,7 @@
 
         long resourceId;
         long maxDiskLastLsn;
-        long LSN = -1;
+        long lsn = -1;
         ILSMIndex index = null;
         LocalResource localResource = null;
         Resource localResourceMetadata = null;
@@ -301,7 +306,7 @@
                 if (IS_DEBUG_MODE) {
                     LOGGER.info(logRecord.getLogRecordForDisplay());
                 }
-                LSN = logRecord.getLSN();
+                lsn = logRecord.getLSN();
                 jobId = logRecord.getJobId();
                 foundWinner = false;
                 switch (logRecord.getLogType()) {
@@ -313,7 +318,7 @@
                                 jobEntityWinners = jobId2WinnerEntitiesMap.get(jobId);
                                 tempKeyTxnId.setTxnId(jobId, logRecord.getDatasetId(), logRecord.getPKHashValue(),
                                         logRecord.getPKValue(), logRecord.getPKValueSize());
-                                if (jobEntityWinners.containsEntityCommitForTxnId(LSN, tempKeyTxnId)) {
+                                if (jobEntityWinners.containsEntityCommitForTxnId(lsn, tempKeyTxnId)) {
                                     foundWinner = true;
                                 }
                             }
@@ -347,15 +352,15 @@
                                 index = (ILSMIndex) datasetLifecycleManager.get(localResource.getPath());
                                 if (index == null) {
                                     //#. create index instance and register to indexLifeCycleManager
-                                    index = localResourceMetadata.createIndexInstance(appRuntimeContext, localResource);
+                                    index = localResourceMetadata.createIndexInstance(appCtx, localResource);
                                     datasetLifecycleManager.register(localResource.getPath(), index);
                                     datasetLifecycleManager.open(localResource.getPath());
 
                                     //#. get maxDiskLastLSN
                                     ILSMIndex lsmIndex = index;
                                     try {
-                                        maxDiskLastLsn = ((AbstractLSMIOOperationCallback) lsmIndex
-                                                .getIOOperationCallback())
+                                        maxDiskLastLsn =
+                                                ((AbstractLSMIOOperationCallback) lsmIndex.getIOOperationCallback())
                                                         .getComponentLSN(lsmIndex.getImmutableComponents());
                                     } catch (HyracksDataException e) {
                                         datasetLifecycleManager.close(localResource.getPath());
@@ -368,7 +373,7 @@
                                     maxDiskLastLsn = resourceId2MaxLSNMap.get(resourceId);
                                 }
 
-                                if (LSN > maxDiskLastLsn) {
+                                if (lsn > maxDiskLastLsn) {
                                     redo(logRecord, datasetLifecycleManager);
                                     redoCount++;
                                 }
@@ -418,17 +423,16 @@
 
     @Override
     public long getLocalMinFirstLSN() throws HyracksDataException {
-        IDatasetLifecycleManager datasetLifecycleManager = txnSubsystem.getAsterixAppRuntimeContextProvider()
-                .getDatasetLifecycleManager();
+        IDatasetLifecycleManager datasetLifecycleManager =
+                txnSubsystem.getAsterixAppRuntimeContextProvider().getDatasetLifecycleManager();
         List<IIndex> openIndexList = datasetLifecycleManager.getOpenResources();
         long firstLSN;
         //the min first lsn can only be the current append or smaller
         long minFirstLSN = logMgr.getAppendLSN();
-        if (openIndexList.size() > 0) {
+        if (!openIndexList.isEmpty()) {
             for (IIndex index : openIndexList) {
-                AbstractLSMIOOperationCallback ioCallback = (AbstractLSMIOOperationCallback) ((ILSMIndex) index)
-                        .getIOOperationCallback();
-
+                AbstractLSMIOOperationCallback ioCallback =
+                        (AbstractLSMIOOperationCallback) ((ILSMIndex) index).getIOOperationCallback();
                 if (!((AbstractLSMIndex) index).isCurrentMutableComponentEmpty() || ioCallback.hasPendingFlush()) {
                     firstLSN = ioCallback.getFirstLSN();
                     minFirstLSN = Math.min(minFirstLSN, firstLSN);
@@ -439,27 +443,27 @@
     }
 
     private long getRemoteMinFirstLSN() {
-        IReplicaResourcesManager remoteResourcesManager = txnSubsystem.getAsterixAppRuntimeContextProvider()
-                .getAppContext().getReplicaResourcesManager();
-        long minRemoteLSN = remoteResourcesManager.getPartitionsMinLSN(localResourceRepository.getInactivePartitions());
-        return minRemoteLSN;
+        IReplicaResourcesManager remoteResourcesManager =
+                txnSubsystem.getAsterixAppRuntimeContextProvider().getAppContext().getReplicaResourcesManager();
+        return remoteResourcesManager.getPartitionsMinLSN(localResourceRepository.getInactivePartitions());
     }
 
     @Override
     public File createJobRecoveryFile(int jobId, String fileName) throws IOException {
         String recoveryDirPath = getRecoveryDirPath();
-        Path JobRecoveryFolder = Paths.get(recoveryDirPath + File.separator + jobId);
-        if (!Files.exists(JobRecoveryFolder)) {
-            Files.createDirectories(JobRecoveryFolder);
+        Path jobRecoveryFolder = Paths.get(recoveryDirPath + File.separator + jobId);
+        if (!Files.exists(jobRecoveryFolder)) {
+            Files.createDirectories(jobRecoveryFolder);
         }
 
-        File jobRecoveryFile = new File(JobRecoveryFolder.toString() + File.separator + fileName);
+        File jobRecoveryFile = new File(jobRecoveryFolder.toString() + File.separator + fileName);
         if (!jobRecoveryFile.exists()) {
-            jobRecoveryFile.createNewFile();
+            if (!jobRecoveryFile.createNewFile()) {
+                throw new IOException("Failed to create file: " + fileName + " for job id(" + jobId + ")");
+            }
         } else {
             throw new IOException("File: " + fileName + " for job id(" + jobId + ") already exists");
         }
-
         return jobRecoveryFile;
     }
 
@@ -513,8 +517,7 @@
         // check if the transaction actually wrote some logs.
         if (firstLSN == TransactionManagementConstants.LogManagerConstants.TERMINAL_LSN || firstLSN > lastLSN) {
             if (LOGGER.isLoggable(Level.INFO)) {
-                LOGGER.info(
-                        "no need to roll back as there were no operations by the transaction " + txnContext.getJobId());
+                LOGGER.info("no need to roll back as there were no operations by the job " + txnContext.getJobId());
             }
             return;
         }
@@ -579,8 +582,8 @@
                             jobLoserEntity2LSNsMap.remove(tempKeyTxnId);
                             entityCommitLogCount++;
                             if (IS_DEBUG_MODE) {
-                                LOGGER.info(Thread.currentThread().getId() + "======> entity_commit[" + currentLSN + "]"
-                                        + tempKeyTxnId);
+                                LOGGER.info(Thread.currentThread().getId() + "======> entity_commit[" + currentLSN
+                                        + "]" + tempKeyTxnId);
                             }
                         }
                         break;
@@ -605,8 +608,8 @@
             //undo loserTxn's effect
             LOGGER.log(Level.INFO, "undoing loser transaction's effect");
 
-            IDatasetLifecycleManager datasetLifecycleManager = txnSubsystem.getAsterixAppRuntimeContextProvider()
-                    .getDatasetLifecycleManager();
+            IDatasetLifecycleManager datasetLifecycleManager =
+                    txnSubsystem.getAsterixAppRuntimeContextProvider().getDatasetLifecycleManager();
             //TODO sort loser entities by smallest LSN to undo in one pass.
             Iterator<Entry<TxnId, List<Long>>> iter = jobLoserEntity2LSNsMap.entrySet().iterator();
             int undoCount = 0;
@@ -658,10 +661,10 @@
 
     private static void undo(ILogRecord logRecord, IDatasetLifecycleManager datasetLifecycleManager) {
         try {
-            ILSMIndex index = (ILSMIndex) datasetLifecycleManager.getIndex(logRecord.getDatasetId(),
-                    logRecord.getResourceId());
-            ILSMIndexAccessor indexAccessor = index.createAccessor(NoOpOperationCallback.INSTANCE,
-                    NoOpOperationCallback.INSTANCE);
+            ILSMIndex index =
+                    (ILSMIndex) datasetLifecycleManager.getIndex(logRecord.getDatasetId(), logRecord.getResourceId());
+            ILSMIndexAccessor indexAccessor =
+                    index.createAccessor(NoOpOperationCallback.INSTANCE, NoOpOperationCallback.INSTANCE);
             if (logRecord.getNewOp() == IndexOperation.INSERT.ordinal()) {
                 indexAccessor.forceDelete(logRecord.getNewValue());
             } else if (logRecord.getNewOp() == IndexOperation.DELETE.ordinal()) {
@@ -679,8 +682,8 @@
             int datasetId = logRecord.getDatasetId();
             long resourceId = logRecord.getResourceId();
             ILSMIndex index = (ILSMIndex) datasetLifecycleManager.getIndex(datasetId, resourceId);
-            ILSMIndexAccessor indexAccessor = index.createAccessor(NoOpOperationCallback.INSTANCE,
-                    NoOpOperationCallback.INSTANCE);
+            ILSMIndexAccessor indexAccessor =
+                    index.createAccessor(NoOpOperationCallback.INSTANCE, NoOpOperationCallback.INSTANCE);
             if (logRecord.getNewOp() == IndexOperation.INSERT.ordinal()) {
                 indexAccessor.forceInsert(logRecord.getNewValue());
             } else if (logRecord.getNewOp() == IndexOperation.DELETE.ordinal()) {
@@ -778,7 +781,8 @@
             ArrayList<File> candidiatePartitions = new ArrayList<>();
             for (File partition : jobEntitCommitOnDiskPartitionsFiles) {
                 String partitionName = partition.getName();
-                //entity commit log must come after the update log, therefore, consider only partitions with max LSN > logLSN
+                // entity commit log must come after the update log,
+                // therefore, consider only partitions with max LSN > logLSN
                 if (getPartitionMaxLSNFromName(partitionName) > logLSN) {
                     candidiatePartitions.add(partition);
                 }
@@ -813,7 +817,8 @@
         }
 
         private void writeCurrentPartitionToDisk() throws IOException {
-            //if we don't have enough memory to allocate for this partition, we will ask recovery manager to free memory
+            //if we don't have enough memory to allocate for this partition,
+            // we will ask recovery manager to free memory
             if (needToFreeMemory()) {
                 freeJobsCachedEntities(jobId);
             }
diff --git a/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/transaction/TransactionSubsystem.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/nc/TransactionSubsystem.java
similarity index 88%
rename from asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/transaction/TransactionSubsystem.java
rename to asterixdb/asterix-app/src/main/java/org/apache/asterix/app/nc/TransactionSubsystem.java
index 09183fe..808a252 100644
--- a/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/transaction/TransactionSubsystem.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/nc/TransactionSubsystem.java
@@ -16,10 +16,9 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.asterix.transaction.management.service.transaction;
+package org.apache.asterix.app.nc;
 
 import java.util.concurrent.Callable;
-import java.util.concurrent.Future;
 import java.util.logging.Logger;
 
 import org.apache.asterix.common.config.ClusterProperties;
@@ -37,11 +36,13 @@
 import org.apache.asterix.common.transactions.ITransactionManager;
 import org.apache.asterix.common.transactions.ITransactionSubsystem;
 import org.apache.asterix.common.utils.StorageConstants;
+import org.apache.asterix.common.utils.TransactionUtil;
 import org.apache.asterix.transaction.management.service.locking.ConcurrentLockManager;
 import org.apache.asterix.transaction.management.service.logging.LogManager;
 import org.apache.asterix.transaction.management.service.logging.LogManagerWithReplication;
 import org.apache.asterix.transaction.management.service.recovery.CheckpointManagerFactory;
-import org.apache.asterix.transaction.management.service.recovery.RecoveryManager;
+import org.apache.asterix.transaction.management.service.transaction.TransactionManager;
+import org.apache.hyracks.api.application.INCApplicationContext;
 
 /**
  * Provider for all the sub-systems (transaction/lock/log/recovery) managers.
@@ -58,13 +59,12 @@
     private final ICheckpointManager checkpointManager;
 
     //for profiling purpose
-    public static final boolean IS_PROFILE_MODE = false;//true
-    public long profilerEntityCommitLogCount = 0;
+    private long profilerEntityCommitLogCount = 0;
     private EntityCommitProfiler ecp;
-    private Future<Object> fecp;
 
-    public TransactionSubsystem(String id, IAppRuntimeContextProvider asterixAppRuntimeContextProvider,
-            TransactionProperties txnProperties) throws ACIDException {
+    public TransactionSubsystem(INCApplicationContext appCtx, String id,
+            IAppRuntimeContextProvider asterixAppRuntimeContextProvider, TransactionProperties txnProperties)
+            throws ACIDException {
         this.asterixAppRuntimeContextProvider = asterixAppRuntimeContextProvider;
         this.id = id;
         this.txnProperties = txnProperties;
@@ -82,8 +82,8 @@
 
         ReplicationProperties asterixReplicationProperties = null;
         if (asterixAppRuntimeContextProvider != null) {
-            asterixReplicationProperties = ((IPropertiesProvider) asterixAppRuntimeContextProvider
-                    .getAppContext()).getReplicationProperties();
+            asterixReplicationProperties = ((IPropertiesProvider) asterixAppRuntimeContextProvider.getAppContext())
+                    .getReplicationProperties();
         }
 
         if (asterixReplicationProperties != null && replicationEnabled) {
@@ -91,11 +91,11 @@
         } else {
             this.logManager = new LogManager(this);
         }
-        this.recoveryManager = new RecoveryManager(this);
+        this.recoveryManager = new RecoveryManager(this, appCtx);
 
-        if (IS_PROFILE_MODE) {
+        if (TransactionUtil.PROFILE_MODE) {
             ecp = new EntityCommitProfiler(this, this.txnProperties.getCommitProfilerReportInterval());
-            fecp = (Future<Object>) getAsterixAppRuntimeContextProvider().getThreadExecutor().submit(ecp);
+            getAsterixAppRuntimeContextProvider().getThreadExecutor().submit(ecp);
         }
     }
 
@@ -124,6 +124,7 @@
         return asterixAppRuntimeContextProvider;
     }
 
+    @Override
     public TransactionProperties getTransactionProperties() {
         return txnProperties;
     }
@@ -133,6 +134,7 @@
         return id;
     }
 
+    @Override
     public void incrementEntityCommitCount() {
         ++profilerEntityCommitLogCount;
     }
@@ -163,7 +165,7 @@
             Thread.currentThread().setName("EntityCommitProfiler-Thread");
             this.txnSubsystem = txnSubsystem;
             this.reportIntervalInSeconds = reportIntervalInSeconds;
-            this.reportIntervalInMillisec = reportIntervalInSeconds * 1000;
+            this.reportIntervalInMillisec = reportIntervalInSeconds * 1000L;
             lastEntityCommitCount = txnSubsystem.profilerEntityCommitLogCount;
         }
 
@@ -176,7 +178,6 @@
                         startTimeStamp = System.currentTimeMillis();
                         firstReport = false;
                     }
-                    //output the count
                     outputCount();
                 }
             }
@@ -186,8 +187,8 @@
             long currentTimeStamp = System.currentTimeMillis();
             long currentEntityCommitCount = txnSubsystem.profilerEntityCommitLogCount;
 
-            LOGGER.severe("EntityCommitProfiler ReportRound[" + reportRound + "], AbsoluteTimeStamp[" + currentTimeStamp
-                    + "], ActualRelativeTimeStamp[" + (currentTimeStamp - startTimeStamp)
+            LOGGER.severe("EntityCommitProfiler ReportRound[" + reportRound + "], AbsoluteTimeStamp["
+                    + currentTimeStamp + "], ActualRelativeTimeStamp[" + (currentTimeStamp - startTimeStamp)
                     + "], ExpectedRelativeTimeStamp[" + (reportIntervalInSeconds * reportRound) + "], IIPS["
                     + ((currentEntityCommitCount - lastEntityCommitCount) / reportIntervalInSeconds) + "], IPS["
                     + (currentEntityCommitCount / (reportRound * reportIntervalInSeconds)) + "]");
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/result/ResultPrinter.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/result/ResultPrinter.java
index 9b1a3db..f401576 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/result/ResultPrinter.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/result/ResultPrinter.java
@@ -27,6 +27,7 @@
 import com.fasterxml.jackson.databind.ObjectMapper;
 import com.fasterxml.jackson.databind.SerializationFeature;
 import com.fasterxml.jackson.databind.node.ObjectNode;
+
 import org.apache.asterix.common.utils.JSONUtil;
 import org.apache.asterix.om.types.ARecordType;
 import org.apache.asterix.translator.IStatementExecutor.Stats;
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/result/ResultReader.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/result/ResultReader.java
index bffaf1f..b0677d8 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/result/ResultReader.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/result/ResultReader.java
@@ -18,7 +18,7 @@
  */
 package org.apache.asterix.app.result;
 
-import org.apache.asterix.runtime.util.AppContextInfo;
+import org.apache.asterix.runtime.utils.AppContextInfo;
 import org.apache.hyracks.api.comm.IFrame;
 import org.apache.hyracks.api.comm.IFrameTupleAccessor;
 import org.apache.hyracks.api.dataset.DatasetJobRecord.Status;
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/translator/DefaultStatementExecutorFactory.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/translator/DefaultStatementExecutorFactory.java
index 6cdf329..99dcc83 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/translator/DefaultStatementExecutorFactory.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/translator/DefaultStatementExecutorFactory.java
@@ -20,17 +20,18 @@
 
 import java.util.List;
 
+import org.apache.asterix.common.context.IStorageComponentProvider;
 import org.apache.asterix.compiler.provider.ILangCompilationProvider;
 import org.apache.asterix.lang.common.base.Statement;
+import org.apache.asterix.translator.IStatementExecutor;
 import org.apache.asterix.translator.IStatementExecutorFactory;
 import org.apache.asterix.translator.SessionConfig;
 
 public class DefaultStatementExecutorFactory implements IStatementExecutorFactory {
 
     @Override
-    public QueryTranslator create(List<Statement> aqlStatements, SessionConfig conf,
-            ILangCompilationProvider compilationProvider) {
-        return new QueryTranslator(aqlStatements, conf, compilationProvider);
+    public IStatementExecutor create(List<Statement> statements, SessionConfig conf,
+            ILangCompilationProvider compilationProvider, IStorageComponentProvider storageComponentProvider) {
+        return new QueryTranslator(statements, conf, compilationProvider, storageComponentProvider);
     }
-
 }
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 bb65b74..6fbf2a5 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
@@ -25,13 +25,10 @@
 import java.io.InputStream;
 import java.io.InputStreamReader;
 import java.rmi.RemoteException;
-import java.util.ArrayDeque;
 import java.util.ArrayList;
 import java.util.Date;
-import java.util.Deque;
 import java.util.HashMap;
 import java.util.Iterator;
-import java.util.LinkedHashMap;
 import java.util.List;
 import java.util.Map;
 import java.util.Map.Entry;
@@ -47,22 +44,23 @@
 import org.apache.asterix.algebra.extension.IExtensionStatement;
 import org.apache.asterix.api.common.APIFramework;
 import org.apache.asterix.api.http.server.ApiServlet;
-import org.apache.asterix.app.external.ExternalIndexingOperations;
 import org.apache.asterix.app.external.FeedJoint;
-import org.apache.asterix.app.external.FeedOperations;
 import org.apache.asterix.app.result.ResultHandle;
 import org.apache.asterix.app.result.ResultReader;
 import org.apache.asterix.app.result.ResultUtil;
 import org.apache.asterix.common.config.ClusterProperties;
 import org.apache.asterix.common.config.DatasetConfig.DatasetType;
-import org.apache.asterix.common.config.DatasetConfig.ExternalDatasetTransactionState;
 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.ExternalProperties;
 import org.apache.asterix.common.config.GlobalConfig;
+import org.apache.asterix.common.context.IStorageComponentProvider;
 import org.apache.asterix.common.exceptions.ACIDException;
 import org.apache.asterix.common.exceptions.CompilationException;
 import org.apache.asterix.common.functions.FunctionSignature;
+import org.apache.asterix.common.utils.JobUtils;
+import org.apache.asterix.common.utils.JobUtils.ProgressState;
 import org.apache.asterix.compiler.provider.ILangCompilationProvider;
 import org.apache.asterix.external.api.IAdapterFactory;
 import org.apache.asterix.external.feed.api.IActiveLifecycleEventSubscriber;
@@ -81,10 +79,9 @@
 import org.apache.asterix.external.feed.watch.FeedConnectJobInfo;
 import org.apache.asterix.external.feed.watch.FeedIntakeInfo;
 import org.apache.asterix.external.indexing.ExternalFile;
+import org.apache.asterix.external.indexing.IndexingConstants;
 import org.apache.asterix.external.util.FeedUtils.FeedRuntimeType;
-import org.apache.asterix.file.DatasetOperations;
-import org.apache.asterix.file.DataverseOperations;
-import org.apache.asterix.file.IndexOperations;
+import org.apache.asterix.file.StorageComponentProvider;
 import org.apache.asterix.formats.nontagged.TypeTraitProvider;
 import org.apache.asterix.lang.aql.statement.SubscribeFeedStatement;
 import org.apache.asterix.lang.common.base.IReturningStatement;
@@ -132,7 +129,6 @@
 import org.apache.asterix.metadata.MetadataException;
 import org.apache.asterix.metadata.MetadataManager;
 import org.apache.asterix.metadata.MetadataTransactionContext;
-import org.apache.asterix.metadata.api.IMetadataEntity;
 import org.apache.asterix.metadata.bootstrap.MetadataBuiltinEntities;
 import org.apache.asterix.metadata.dataset.hints.DatasetHints;
 import org.apache.asterix.metadata.dataset.hints.DatasetHints.DatasetNodegroupCardinalityHint;
@@ -151,26 +147,25 @@
 import org.apache.asterix.metadata.entities.NodeGroup;
 import org.apache.asterix.metadata.feeds.BuiltinFeedPolicies;
 import org.apache.asterix.metadata.feeds.FeedMetadataUtil;
-import org.apache.asterix.metadata.utils.DatasetUtils;
+import org.apache.asterix.metadata.feeds.FeedOperations;
+import org.apache.asterix.metadata.utils.DatasetUtil;
 import org.apache.asterix.metadata.utils.ExternalDatasetsRegistry;
-import org.apache.asterix.metadata.utils.KeyFieldTypeUtils;
+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;
 import org.apache.asterix.metadata.utils.MetadataLockManager;
+import org.apache.asterix.metadata.utils.MetadataUtil;
+import org.apache.asterix.metadata.utils.TypeUtil;
 import org.apache.asterix.om.types.ARecordType;
 import org.apache.asterix.om.types.ATypeTag;
-import org.apache.asterix.om.types.AUnionType;
 import org.apache.asterix.om.types.IAType;
 import org.apache.asterix.om.types.TypeSignature;
-import org.apache.asterix.om.types.hierachy.ATypeHierarchy;
-import org.apache.asterix.runtime.util.AppContextInfo;
+import org.apache.asterix.runtime.utils.AppContextInfo;
 import org.apache.asterix.transaction.management.service.transaction.DatasetIdFactory;
 import org.apache.asterix.translator.AbstractLangTranslator;
 import org.apache.asterix.translator.CompiledStatements.CompiledConnectFeedStatement;
-import org.apache.asterix.translator.CompiledStatements.CompiledCreateIndexStatement;
-import org.apache.asterix.translator.CompiledStatements.CompiledDatasetDropStatement;
 import org.apache.asterix.translator.CompiledStatements.CompiledDeleteStatement;
-import org.apache.asterix.translator.CompiledStatements.CompiledIndexCompactStatement;
-import org.apache.asterix.translator.CompiledStatements.CompiledIndexDropStatement;
 import org.apache.asterix.translator.CompiledStatements.CompiledInsertStatement;
 import org.apache.asterix.translator.CompiledStatements.CompiledLoadFromFileStatement;
 import org.apache.asterix.translator.CompiledStatements.CompiledSubscribeFeedStatement;
@@ -180,9 +175,8 @@
 import org.apache.asterix.translator.SessionConfig;
 import org.apache.asterix.translator.TypeTranslator;
 import org.apache.asterix.translator.util.ValidateUtil;
-import org.apache.asterix.util.FlushDatasetUtils;
-import org.apache.asterix.util.JobUtils;
-import org.apache.commons.lang3.ArrayUtils;
+import org.apache.asterix.utils.DataverseUtil;
+import org.apache.asterix.utils.FlushDatasetUtil;
 import org.apache.commons.lang3.StringUtils;
 import org.apache.commons.lang3.mutable.MutableBoolean;
 import org.apache.commons.lang3.mutable.MutableObject;
@@ -216,27 +210,24 @@
 
     private static final Logger LOGGER = Logger.getLogger(QueryTranslator.class.getName());
 
-    protected enum ProgressState {
-        NO_PROGRESS,
-        ADDED_PENDINGOP_RECORD_TO_METADATA
-    }
-
     public static final boolean IS_DEBUG_MODE = false;// true
     protected final List<Statement> statements;
     protected final SessionConfig sessionConfig;
-    protected Dataverse activeDefaultDataverse;
+    protected Dataverse activeDataverse;
     protected final List<FunctionDecl> declaredFunctions;
     protected final APIFramework apiFramework;
     protected final IRewriterFactory rewriterFactory;
+    protected final IStorageComponentProvider componentProvider;
 
     public QueryTranslator(List<Statement> statements, SessionConfig conf,
-            ILangCompilationProvider compliationProvider) {
+            ILangCompilationProvider compliationProvider, IStorageComponentProvider componentProvider) {
         this.statements = statements;
         this.sessionConfig = conf;
-        this.declaredFunctions = getDeclaredFunctions(statements);
-        this.apiFramework = new APIFramework(compliationProvider);
-        this.rewriterFactory = compliationProvider.getRewriterFactory();
-        activeDefaultDataverse = MetadataBuiltinEntities.DEFAULT_DATAVERSE;
+        this.componentProvider = componentProvider;
+        declaredFunctions = getDeclaredFunctions(statements);
+        apiFramework = new APIFramework(compliationProvider);
+        rewriterFactory = compliationProvider.getRewriterFactory();
+        activeDataverse = MetadataBuiltinEntities.DEFAULT_DATAVERSE;
     }
 
     protected List<FunctionDecl> getDeclaredFunctions(List<Statement> statements) {
@@ -286,9 +277,9 @@
                 if (sessionConfig.is(SessionConfig.FORMAT_HTML)) {
                     sessionConfig.out().println(ApiServlet.HTML_STATEMENT_SEPARATOR);
                 }
-                validateOperation(activeDefaultDataverse, stmt);
+                validateOperation(activeDataverse, stmt);
                 rewriteStatement(stmt); // Rewrite the statement's AST.
-                MetadataProvider metadataProvider = new MetadataProvider(activeDefaultDataverse);
+                MetadataProvider metadataProvider = new MetadataProvider(activeDataverse, componentProvider);
                 metadataProvider.setWriterFactory(writerFactory);
                 metadataProvider.setResultSerializerFactoryProvider(resultSerializerFactoryProvider);
                 metadataProvider.setOutputFile(outputFile);
@@ -298,7 +289,7 @@
                         handleSetStatement(stmt, config);
                         break;
                     case Statement.Kind.DATAVERSE_DECL:
-                        activeDefaultDataverse = handleUseDataverseStatement(metadataProvider, stmt);
+                        activeDataverse = handleUseDataverseStatement(metadataProvider, stmt);
                         break;
                     case Statement.Kind.CREATE_DATAVERSE:
                         handleCreateDataverseStatement(metadataProvider, stmt);
@@ -375,8 +366,8 @@
                         break;
                     case Statement.Kind.QUERY:
                         metadataProvider.setResultSetId(new ResultSetId(resultSetIdCounter++));
-                        metadataProvider.setResultAsyncMode(resultDelivery == ResultDelivery.ASYNC
-                                || resultDelivery == ResultDelivery.DEFERRED);
+                        metadataProvider.setResultAsyncMode(
+                                resultDelivery == ResultDelivery.ASYNC || resultDelivery == ResultDelivery.DEFERRED);
                         handleQuery(metadataProvider, (Query) stmt, hcc, hdc, resultDelivery, stats);
                         break;
                     case Statement.Kind.COMPACT:
@@ -450,8 +441,7 @@
         }
     }
 
-    protected void handleCreateDataverseStatement(MetadataProvider metadataProvider, Statement stmt)
-            throws Exception {
+    protected void handleCreateDataverseStatement(MetadataProvider metadataProvider, Statement stmt) throws Exception {
 
         CreateDataverseStatement stmtCreateDataverse = (CreateDataverseStatement) stmt;
         String dvName = stmtCreateDataverse.getDataverseName().getValue();
@@ -470,7 +460,7 @@
                 }
             }
             MetadataManager.INSTANCE.addDataverse(metadataProvider.getMetadataTxnContext(),
-                    new Dataverse(dvName, stmtCreateDataverse.getFormat(), IMetadataEntity.PENDING_NO_OP));
+                    new Dataverse(dvName, stmtCreateDataverse.getFormat(), MetadataUtil.PENDING_NO_OP));
             MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
         } catch (Exception e) {
             abort(e, e, mdTxnCtx);
@@ -488,8 +478,8 @@
             throw new CompilationException("Unknown compaction policy: " + compactionPolicy);
         }
         String compactionPolicyFactoryClassName = compactionPolicyEntity.getClassName();
-        ILSMMergePolicyFactory mergePolicyFactory = (ILSMMergePolicyFactory) Class
-                .forName(compactionPolicyFactoryClassName).newInstance();
+        ILSMMergePolicyFactory mergePolicyFactory =
+                (ILSMMergePolicyFactory) Class.forName(compactionPolicyFactoryClassName).newInstance();
         if (isExternalDataset && mergePolicyFactory.getName().compareTo("correlated-prefix") == 0) {
             throw new CompilationException("The correlated-prefix merge policy cannot be used with external dataset.");
         }
@@ -538,6 +528,7 @@
                 metaItemTypeDataverseName + "." + metaItemTypeName, nodegroupName, compactionPolicy,
                 dataverseName + "." + datasetName, defaultCompactionPolicy);
         Dataset dataset = null;
+        Index primaryIndex = null;
         try {
 
             IDatasetDetails datasetDetails = null;
@@ -556,8 +547,8 @@
             if (dt == null) {
                 throw new AlgebricksException(": type " + itemTypeName + " could not be found.");
             }
-            String ngName = ngNameId != null ? ngNameId.getValue()
-                    : configureNodegroupForDataset(dd, dataverseName, mdTxnCtx);
+            String ngName =
+                    ngNameId != null ? ngNameId.getValue() : configureNodegroupForDataset(dd, dataverseName, mdTxnCtx);
 
             if (compactionPolicy == null) {
                 compactionPolicy = GlobalConfig.DEFAULT_COMPACTION_POLICY_NAME;
@@ -581,10 +572,10 @@
                     }
                     ARecordType metaRecType = (ARecordType) metaItemType;
 
-                    List<List<String>> partitioningExprs = ((InternalDetailsDecl) dd.getDatasetDetailsDecl())
-                            .getPartitioningExprs();
-                    List<Integer> keySourceIndicators = ((InternalDetailsDecl) dd.getDatasetDetailsDecl())
-                            .getKeySourceIndicators();
+                    List<List<String>> partitioningExprs =
+                            ((InternalDetailsDecl) dd.getDatasetDetailsDecl()).getPartitioningExprs();
+                    List<Integer> keySourceIndicators =
+                            ((InternalDetailsDecl) dd.getDatasetDetailsDecl()).getKeySourceIndicators();
                     boolean autogenerated = ((InternalDetailsDecl) dd.getDatasetDetailsDecl()).isAutogenerated();
                     ARecordType aRecordType = (ARecordType) itemType;
                     List<IAType> partitioningTypes = ValidateUtil.validatePartitioningExpressions(aRecordType,
@@ -607,10 +598,11 @@
                     break;
                 case EXTERNAL:
                     String adapter = ((ExternalDetailsDecl) dd.getDatasetDetailsDecl()).getAdapter();
-                    Map<String, String> properties = ((ExternalDetailsDecl) dd.getDatasetDetailsDecl()).getProperties();
+                    Map<String, String> properties =
+                            ((ExternalDetailsDecl) dd.getDatasetDetailsDecl()).getProperties();
 
-                    datasetDetails = new ExternalDatasetDetails(adapter, properties, new Date(),
-                            ExternalDatasetTransactionState.COMMIT);
+                    datasetDetails =
+                            new ExternalDatasetDetails(adapter, properties, new Date(), TransactionState.COMMIT);
                     break;
                 default:
                     throw new CompilationException("Unknown datatype " + dd.getDatasetType());
@@ -625,14 +617,13 @@
             dataset = new Dataset(dataverseName, datasetName, itemTypeDataverseName, itemTypeName,
                     metaItemTypeDataverseName, metaItemTypeName, ngName, compactionPolicy, compactionPolicyProperties,
                     datasetDetails, dd.getHints(), dsType, DatasetIdFactory.generateDatasetId(),
-                    IMetadataEntity.PENDING_ADD_OP);
+                    MetadataUtil.PENDING_ADD_OP);
             MetadataManager.INSTANCE.addDataset(metadataProvider.getMetadataTxnContext(), dataset);
-
+            primaryIndex = MetadataManager.INSTANCE.getIndex(mdTxnCtx, dataverseName, datasetName, datasetName);
             if (dd.getDatasetType() == DatasetType.INTERNAL) {
-                Dataverse dataverse = MetadataManager.INSTANCE.getDataverse(metadataProvider.getMetadataTxnContext(),
-                        dataverseName);
-                JobSpecification jobSpec = DatasetOperations.createDatasetJobSpec(dataverse, datasetName,
-                        metadataProvider);
+                Dataverse dataverse =
+                        MetadataManager.INSTANCE.getDataverse(metadataProvider.getMetadataTxnContext(), dataverseName);
+                JobSpecification jobSpec = DatasetUtil.createDatasetJobSpec(dataverse, datasetName, metadataProvider);
 
                 // #. make metadataTxn commit before calling runJob.
                 MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
@@ -650,7 +641,7 @@
 
             // #. add a new dataset with PendingNoOp after deleting the dataset with PendingAddOp
             MetadataManager.INSTANCE.dropDataset(metadataProvider.getMetadataTxnContext(), dataverseName, datasetName);
-            dataset.setPendingOp(IMetadataEntity.PENDING_NO_OP);
+            dataset.setPendingOp(MetadataUtil.PENDING_NO_OP);
             MetadataManager.INSTANCE.addDataset(metadataProvider.getMetadataTxnContext(), dataset);
             MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
         } catch (Exception e) {
@@ -668,9 +659,8 @@
                 mdTxnCtx = MetadataManager.INSTANCE.beginTransaction();
                 bActiveTxn = true;
                 metadataProvider.setMetadataTxnContext(mdTxnCtx);
-                CompiledDatasetDropStatement cds = new CompiledDatasetDropStatement(dataverseName, datasetName);
                 try {
-                    JobSpecification jobSpec = DatasetOperations.createDropDatasetJobSpec(cds, metadataProvider);
+                    JobSpecification jobSpec = DatasetUtil.dropDatasetJobSpec(dataset, primaryIndex, metadataProvider);
                     MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
                     bActiveTxn = false;
                     JobUtils.runJob(hcc, jobSpec, true);
@@ -735,8 +725,8 @@
         }
     }
 
-    protected String configureNodegroupForDataset(DatasetDecl dd, String dataverse, MetadataTransactionContext mdTxnCtx)
-            throws CompilationException {
+    protected String configureNodegroupForDataset(DatasetDecl dd, String dataverse,
+            MetadataTransactionContext mdTxnCtx) throws CompilationException {
         int nodegroupCardinality;
         String nodegroupName;
         String hintValue = dd.getHints().get(DatasetNodegroupCardinalityHint.NAME);
@@ -802,11 +792,12 @@
         JobSpecification spec = null;
         Dataset ds = null;
         // For external datasets
-        ArrayList<ExternalFile> externalFilesSnapshot = null;
+        List<ExternalFile> externalFilesSnapshot = null;
         boolean firstExternalDatasetIndex = false;
         boolean filesIndexReplicated = false;
         Index filesIndex = null;
         boolean datasetLocked = false;
+        Index index = null;
         try {
             ds = MetadataManager.INSTANCE.getDataset(metadataProvider.getMetadataTxnContext(), dataverseName,
                     datasetName);
@@ -816,7 +807,7 @@
             }
 
             indexName = stmtCreateIndex.getIndexName().getValue();
-            Index idx = MetadataManager.INSTANCE.getIndex(metadataProvider.getMetadataTxnContext(), dataverseName,
+            index = MetadataManager.INSTANCE.getIndex(metadataProvider.getMetadataTxnContext(), dataverseName,
                     datasetName, indexName);
             Datatype dt = MetadataManager.INSTANCE.getDatatype(metadataProvider.getMetadataTxnContext(),
                     ds.getItemTypeDataverseName(), ds.getItemTypeName());
@@ -833,8 +824,8 @@
             int keyIndex = 0;
             for (Pair<List<String>, TypeExpression> fieldExpr : stmtCreateIndex.getFieldExprs()) {
                 IAType fieldType = null;
-                ARecordType subType = KeyFieldTypeUtils.chooseSource(keySourceIndicators, keyIndex, aRecordType,
-                        metaRecordType);
+                ARecordType subType =
+                        KeyFieldTypeUtil.chooseSource(keySourceIndicators, keyIndex, aRecordType, metaRecordType);
                 boolean isOpen = subType.isOpen();
                 int i = 0;
                 if (fieldExpr.first.size() > 1 && !isOpen) {
@@ -858,8 +849,8 @@
                     if (stmtCreateIndex.hasMetaField()) {
                         throw new AlgebricksException("Typed open index can only be created on the record part");
                     }
-                    Map<TypeSignature, IAType> typeMap = TypeTranslator.computeTypes(mdTxnCtx, fieldExpr.second,
-                            indexName, dataverseName);
+                    Map<TypeSignature, IAType> typeMap =
+                            TypeTranslator.computeTypes(mdTxnCtx, fieldExpr.second, indexName, dataverseName);
                     TypeSignature typeSignature = new TypeSignature(dataverseName, indexName);
                     fieldType = typeMap.get(typeSignature);
                 }
@@ -876,7 +867,7 @@
             ValidateUtil.validateKeyFields(aRecordType, metaRecordType, indexFields, keySourceIndicators,
                     indexFieldTypes, stmtCreateIndex.getIndexType());
 
-            if (idx != null) {
+            if (index != null) {
                 if (stmtCreateIndex.getIfNotExists()) {
                     MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
                     return;
@@ -893,7 +884,7 @@
                     || stmtCreateIndex.getIndexType() == IndexType.SINGLE_PARTITION_NGRAM_INVIX
                     || stmtCreateIndex.getIndexType() == IndexType.LENGTH_PARTITIONED_WORD_INVIX
                     || stmtCreateIndex.getIndexType() == IndexType.LENGTH_PARTITIONED_NGRAM_INVIX) {
-                List<List<String>> partitioningKeys = DatasetUtils.getPartitioningKeys(ds);
+                List<List<String>> partitioningKeys = DatasetUtil.getPartitioningKeys(ds);
                 for (List<String> partitioningKey : partitioningKeys) {
                     IAType keyType = aRecordType.getSubFieldType(partitioningKey);
                     ITypeTraits typeTrait = TypeTraitProvider.INSTANCE.getTypeTrait(keyType);
@@ -925,7 +916,7 @@
 
                 // Check if the files index exist
                 filesIndex = MetadataManager.INSTANCE.getIndex(metadataProvider.getMetadataTxnContext(), dataverseName,
-                        datasetName, ExternalIndexingOperations.getFilesIndexName(datasetName));
+                        datasetName, IndexingConstants.getFilesIndexName(datasetName));
                 firstExternalDatasetIndex = filesIndex == null;
                 // Lock external dataset
                 ExternalDatasetsRegistry.INSTANCE.buildIndexBegin(ds, firstExternalDatasetIndex);
@@ -933,7 +924,7 @@
                 if (firstExternalDatasetIndex) {
                     // Verify that no one has created an index before we acquire the lock
                     filesIndex = MetadataManager.INSTANCE.getIndex(metadataProvider.getMetadataTxnContext(),
-                            dataverseName, datasetName, ExternalIndexingOperations.getFilesIndexName(datasetName));
+                            dataverseName, datasetName, IndexingConstants.getFilesIndexName(datasetName));
                     if (filesIndex != null) {
                         ExternalDatasetsRegistry.INSTANCE.buildIndexEnd(ds, firstExternalDatasetIndex);
                         firstExternalDatasetIndex = false;
@@ -944,11 +935,11 @@
                     // Get snapshot from External File System
                     externalFilesSnapshot = ExternalIndexingOperations.getSnapshotFromExternalFileSystem(ds);
                     // Add an entry for the files index
-                    filesIndex = new Index(dataverseName, datasetName,
-                            ExternalIndexingOperations.getFilesIndexName(datasetName), IndexType.BTREE,
-                            ExternalIndexingOperations.FILE_INDEX_FIELD_NAMES, null,
-                            ExternalIndexingOperations.FILE_INDEX_FIELD_TYPES, false, false,
-                            IMetadataEntity.PENDING_ADD_OP);
+                    filesIndex =
+                            new Index(dataverseName, datasetName, IndexingConstants.getFilesIndexName(datasetName),
+                                    IndexType.BTREE, ExternalIndexingOperations.FILE_INDEX_FIELD_NAMES, null,
+                                    ExternalIndexingOperations.FILE_INDEX_FIELD_TYPES, false, false,
+                                    MetadataUtil.PENDING_ADD_OP);
                     MetadataManager.INSTANCE.addIndex(metadataProvider.getMetadataTxnContext(), filesIndex);
                     // Add files to the external files index
                     for (ExternalFile file : externalFilesSnapshot) {
@@ -970,33 +961,35 @@
             if (stmtCreateIndex.isEnforced()) {
                 List<Index> indexes = MetadataManager.INSTANCE
                         .getDatasetIndexes(metadataProvider.getMetadataTxnContext(), dataverseName, datasetName);
-                for (Index index : indexes) {
-                    if (index.getKeyFieldNames().equals(indexFields)
-                            && !index.getKeyFieldTypes().equals(indexFieldTypes) && index.isEnforcingKeyFileds()) {
+                for (Index existingIndex : indexes) {
+                    if (existingIndex.getKeyFieldNames().equals(indexFields)
+                            && !existingIndex.getKeyFieldTypes().equals(indexFieldTypes)
+                            && existingIndex.isEnforcingKeyFileds()) {
                         throw new CompilationException("Cannot create index " + indexName + " , enforced index "
-                                + index.getIndexName() + " on field \"" + StringUtils.join(indexFields, ',')
-                                + "\" is already defined with type \"" + index.getKeyFieldTypes() + "\"");
+                                + existingIndex.getIndexName() + " on field \"" + StringUtils.join(indexFields, ',')
+                                + "\" is already defined with type \"" + existingIndex.getKeyFieldTypes() + "\"");
                     }
                 }
             }
 
             // #. add a new index with PendingAddOp
-            Index index = new Index(dataverseName, datasetName, indexName, stmtCreateIndex.getIndexType(), indexFields,
-                    keySourceIndicators, indexFieldTypes, stmtCreateIndex.getGramLength(), stmtCreateIndex.isEnforced(),
-                    false, IMetadataEntity.PENDING_ADD_OP);
+            index = new Index(dataverseName, datasetName, indexName, stmtCreateIndex.getIndexType(), indexFields,
+                    keySourceIndicators, indexFieldTypes, stmtCreateIndex.getGramLength(),
+                    stmtCreateIndex.isEnforced(), false, MetadataUtil.PENDING_ADD_OP);
             MetadataManager.INSTANCE.addIndex(metadataProvider.getMetadataTxnContext(), index);
 
             ARecordType enforcedType = null;
+            ARecordType enforcedMetaType = null;
             if (stmtCreateIndex.isEnforced()) {
-                enforcedType = createEnforcedType(aRecordType, Lists.newArrayList(index));
+                Pair<ARecordType, ARecordType> enforcedTypes =
+                        TypeUtil.createEnforcedType(aRecordType, metaRecordType, Lists.newArrayList(index));
+                enforcedType = enforcedTypes.first;
+                enforcedMetaType = enforcedTypes.second;
             }
 
             // #. prepare to create the index artifact in NC.
-            CompiledCreateIndexStatement cis = new CompiledCreateIndexStatement(index.getIndexName(), dataverseName,
-                    index.getDatasetName(), index.getKeyFieldNames(), index.getKeyFieldTypes(),
-                    index.isEnforcingKeyFileds(), index.getGramLength(), index.getIndexType());
-            spec = IndexOperations.buildSecondaryIndexCreationJobSpec(cis, aRecordType, metaRecordType,
-                    keySourceIndicators, enforcedType, metadataProvider);
+            spec = IndexUtil.buildSecondaryIndexCreationJobSpec(ds, index, aRecordType, metaRecordType, enforcedType,
+                    enforcedMetaType, metadataProvider);
             if (spec == null) {
                 throw new CompilationException("Failed to create job spec for creating index '"
                         + stmtCreateIndex.getDatasetName() + "." + stmtCreateIndex.getIndexName() + "'");
@@ -1014,12 +1007,8 @@
             metadataProvider.setMetadataTxnContext(mdTxnCtx);
 
             // #. load data into the index in NC.
-            cis = new CompiledCreateIndexStatement(index.getIndexName(), dataverseName, index.getDatasetName(),
-                    index.getKeyFieldNames(), index.getKeyFieldTypes(), index.isEnforcingKeyFileds(),
-                    index.getGramLength(), index.getIndexType());
-
-            spec = IndexOperations.buildSecondaryIndexLoadingJobSpec(cis, aRecordType, metaRecordType,
-                    keySourceIndicators, enforcedType, metadataProvider);
+            spec = IndexUtil.buildSecondaryIndexLoadingJobSpec(ds, index, aRecordType, metaRecordType, enforcedType,
+                    enforcedMetaType, metadataProvider);
             MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
             bActiveTxn = false;
 
@@ -1033,14 +1022,14 @@
             // #. add another new index with PendingNoOp after deleting the index with PendingAddOp
             MetadataManager.INSTANCE.dropIndex(metadataProvider.getMetadataTxnContext(), dataverseName, datasetName,
                     indexName);
-            index.setPendingOp(IMetadataEntity.PENDING_NO_OP);
+            index.setPendingOp(MetadataUtil.PENDING_NO_OP);
             MetadataManager.INSTANCE.addIndex(metadataProvider.getMetadataTxnContext(), index);
             // add another new files index with PendingNoOp after deleting the index with
             // PendingAddOp
             if (firstExternalDatasetIndex) {
-                MetadataManager.INSTANCE.dropIndex(metadataProvider.getMetadataTxnContext(), dataverseName, datasetName,
-                        filesIndex.getIndexName());
-                filesIndex.setPendingOp(IMetadataEntity.PENDING_NO_OP);
+                MetadataManager.INSTANCE.dropIndex(metadataProvider.getMetadataTxnContext(), dataverseName,
+                        datasetName, filesIndex.getIndexName());
+                filesIndex.setPendingOp(MetadataUtil.PENDING_NO_OP);
                 MetadataManager.INSTANCE.addIndex(metadataProvider.getMetadataTxnContext(), filesIndex);
                 // update transaction timestamp
                 ((ExternalDatasetDetails) ds.getDatasetDetails()).setRefreshTimestamp(new Date());
@@ -1056,11 +1045,9 @@
             if (filesIndexReplicated) {
                 mdTxnCtx = MetadataManager.INSTANCE.beginTransaction();
                 bActiveTxn = true;
-                CompiledIndexDropStatement cds = new CompiledIndexDropStatement(dataverseName, datasetName,
-                        ExternalIndexingOperations.getFilesIndexName(datasetName));
                 try {
-                    JobSpecification jobSpec = ExternalIndexingOperations.buildDropFilesIndexJobSpec(cds,
-                            metadataProvider, ds);
+                    JobSpecification jobSpec =
+                            ExternalIndexingOperations.buildDropFilesIndexJobSpec(metadataProvider, ds);
                     MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
                     bActiveTxn = false;
                     JobUtils.runJob(hcc, jobSpec, true);
@@ -1078,11 +1065,8 @@
                 mdTxnCtx = MetadataManager.INSTANCE.beginTransaction();
                 bActiveTxn = true;
                 metadataProvider.setMetadataTxnContext(mdTxnCtx);
-                CompiledIndexDropStatement cds = new CompiledIndexDropStatement(dataverseName, datasetName, indexName);
                 try {
-                    JobSpecification jobSpec = IndexOperations.buildDropSecondaryIndexJobSpec(cds, metadataProvider,
-                            ds);
-
+                    JobSpecification jobSpec = IndexUtil.buildDropSecondaryIndexJobSpec(index, metadataProvider, ds);
                     MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
                     bActiveTxn = false;
                     JobUtils.runJob(hcc, jobSpec, true);
@@ -1111,13 +1095,13 @@
                     try {
                         // Drop the files index from metadata
                         MetadataManager.INSTANCE.dropIndex(metadataProvider.getMetadataTxnContext(), dataverseName,
-                                datasetName, ExternalIndexingOperations.getFilesIndexName(datasetName));
+                                datasetName, IndexingConstants.getFilesIndexName(datasetName));
                         MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
                     } catch (Exception e2) {
                         e.addSuppressed(e2);
                         abort(e, e2, mdTxnCtx);
                         throw new IllegalStateException("System is inconsistent state: pending index(" + dataverseName
-                                + "." + datasetName + "." + ExternalIndexingOperations.getFilesIndexName(datasetName)
+                                + "." + datasetName + "." + IndexingConstants.getFilesIndexName(datasetName)
                                 + ") couldn't be removed from the metadata", e);
                     }
                 }
@@ -1219,8 +1203,8 @@
                         disconnectFeedBeforeDelete(dvId, activeEntityId, conn, metadataProvider, hcc);
                     }
                     // prepare job to remove feed log storage
-                    jobsToExecute.add(FeedOperations.buildRemoveFeedStorageJob(
-                            MetadataManager.INSTANCE.getFeed(mdTxnCtx, dataverseName, activeEntityId.getEntityName())));
+                    jobsToExecute.add(FeedOperations.buildRemoveFeedStorageJob(MetadataManager.INSTANCE
+                            .getFeed(mdTxnCtx, dataverseName, activeEntityId.getEntityName())));
                 }
             }
 
@@ -1230,47 +1214,41 @@
                 String datasetName = datasets.get(j).getDatasetName();
                 DatasetType dsType = datasets.get(j).getDatasetType();
                 if (dsType == DatasetType.INTERNAL) {
-                    List<Index> indexes = MetadataManager.INSTANCE.getDatasetIndexes(mdTxnCtx, dataverseName,
-                            datasetName);
+                    List<Index> indexes =
+                            MetadataManager.INSTANCE.getDatasetIndexes(mdTxnCtx, dataverseName, datasetName);
                     for (int k = 0; k < indexes.size(); k++) {
                         if (indexes.get(k).isSecondaryIndex()) {
-                            CompiledIndexDropStatement cds = new CompiledIndexDropStatement(dataverseName, datasetName,
-                                    indexes.get(k).getIndexName());
-                            jobsToExecute.add(IndexOperations.buildDropSecondaryIndexJobSpec(cds, metadataProvider,
-                                    datasets.get(j)));
+                            jobsToExecute.add(IndexUtil.buildDropSecondaryIndexJobSpec(indexes.get(k),
+                                    metadataProvider, datasets.get(j)));
                         }
                     }
-
-                    CompiledDatasetDropStatement cds = new CompiledDatasetDropStatement(dataverseName, datasetName);
-                    jobsToExecute.add(DatasetOperations.createDropDatasetJobSpec(cds, metadataProvider));
+                    Index primaryIndex =
+                            MetadataManager.INSTANCE.getIndex(mdTxnCtx, dataverseName, datasetName, datasetName);
+                    jobsToExecute.add(DatasetUtil.dropDatasetJobSpec(datasets.get(j), primaryIndex, metadataProvider));
                 } else {
                     // External dataset
-                    List<Index> indexes = MetadataManager.INSTANCE.getDatasetIndexes(mdTxnCtx, dataverseName,
-                            datasetName);
+                    List<Index> indexes =
+                            MetadataManager.INSTANCE.getDatasetIndexes(mdTxnCtx, dataverseName, datasetName);
                     for (int k = 0; k < indexes.size(); k++) {
                         if (ExternalIndexingOperations.isFileIndex(indexes.get(k))) {
-                            CompiledIndexDropStatement cds = new CompiledIndexDropStatement(dataverseName, datasetName,
-                                    indexes.get(k).getIndexName());
-                            jobsToExecute.add(ExternalIndexingOperations.buildDropFilesIndexJobSpec(cds,
-                                    metadataProvider, datasets.get(j)));
-                        } else {
-                            CompiledIndexDropStatement cds = new CompiledIndexDropStatement(dataverseName, datasetName,
-                                    indexes.get(k).getIndexName());
-                            jobsToExecute.add(IndexOperations.buildDropSecondaryIndexJobSpec(cds, metadataProvider,
+                            jobsToExecute.add(ExternalIndexingOperations.buildDropFilesIndexJobSpec(metadataProvider,
                                     datasets.get(j)));
+                        } else {
+                            jobsToExecute.add(IndexUtil.buildDropSecondaryIndexJobSpec(indexes.get(k),
+                                    metadataProvider, datasets.get(j)));
                         }
                     }
                     ExternalDatasetsRegistry.INSTANCE.removeDatasetInfo(datasets.get(j));
                 }
             }
-            jobsToExecute.add(DataverseOperations.createDropDataverseJobSpec(dv, metadataProvider));
+            jobsToExecute.add(DataverseUtil.dropDataverseJobSpec(dv, metadataProvider));
             // #. mark PendingDropOp on the dataverse record by
             // first, deleting the dataverse record from the DATAVERSE_DATASET
             // second, inserting the dataverse record with the PendingDropOp value into the
             // DATAVERSE_DATASET
             MetadataManager.INSTANCE.dropDataverse(mdTxnCtx, dataverseName);
             MetadataManager.INSTANCE.addDataverse(mdTxnCtx,
-                    new Dataverse(dataverseName, dv.getDataFormat(), IMetadataEntity.PENDING_DROP_OP));
+                    new Dataverse(dataverseName, dv.getDataFormat(), MetadataUtil.PENDING_DROP_OP));
 
             MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
             bActiveTxn = false;
@@ -1286,8 +1264,8 @@
 
             // #. finally, delete the dataverse.
             MetadataManager.INSTANCE.dropDataverse(mdTxnCtx, dataverseName);
-            if (activeDefaultDataverse != null && activeDefaultDataverse.getDataverseName() == dataverseName) {
-                activeDefaultDataverse = null;
+            if (activeDataverse != null && activeDataverse.getDataverseName() == dataverseName) {
+                activeDataverse = null;
             }
             MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
         } catch (Exception e) {
@@ -1296,8 +1274,8 @@
             }
 
             if (progress == ProgressState.ADDED_PENDINGOP_RECORD_TO_METADATA) {
-                if (activeDefaultDataverse != null && activeDefaultDataverse.getDataverseName() == dataverseName) {
-                    activeDefaultDataverse = null;
+                if (activeDataverse != null && activeDataverse.getDataverseName() == dataverseName) {
+                    activeDataverse = null;
                 }
 
                 // #. execute compensation operations
@@ -1327,6 +1305,7 @@
             throw e;
         } finally {
             MetadataLockManager.INSTANCE.releaseDataverseWriteLock(dataverseName);
+            ExternalDatasetsRegistry.INSTANCE.releaseAcquiredLocks(metadataProvider);
         }
     }
 
@@ -1359,8 +1338,8 @@
         String dataverseName = getActiveDataverse(stmtDelete.getDataverseName());
         String datasetName = stmtDelete.getDatasetName().getValue();
         MutableObject<ProgressState> progress = new MutableObject<>(ProgressState.NO_PROGRESS);
-        MutableObject<MetadataTransactionContext> mdTxnCtx = new MutableObject<>(
-                MetadataManager.INSTANCE.beginTransaction());
+        MutableObject<MetadataTransactionContext> mdTxnCtx =
+                new MutableObject<>(MetadataManager.INSTANCE.beginTransaction());
         MutableBoolean bActiveTxn = new MutableBoolean(true);
         metadataProvider.setMetadataTxnContext(mdTxnCtx.getValue());
         MetadataLockManager.INSTANCE.dropDatasetBegin(dataverseName, dataverseName + "." + datasetName);
@@ -1372,13 +1351,11 @@
                     MetadataManager.INSTANCE.commitTransaction(mdTxnCtx.getValue());
                     return;
                 } else {
-                    throw new AlgebricksException("There is no dataset with this name " + datasetName + " in dataverse "
-                            + dataverseName + ".");
+                    throw new AlgebricksException("There is no dataset with this name " + datasetName
+                            + " in dataverse " + dataverseName + ".");
                 }
             }
-
-            doDropDataset(ds, datasetName, metadataProvider, mdTxnCtx, jobsToExecute, dataverseName, bActiveTxn,
-                    progress, hcc);
+            ds.drop(metadataProvider, mdTxnCtx, jobsToExecute, bActiveTxn, progress, hcc);
             MetadataManager.INSTANCE.commitTransaction(mdTxnCtx.getValue());
         } catch (Exception e) {
             if (bActiveTxn.booleanValue()) {
@@ -1415,6 +1392,7 @@
             throw e;
         } finally {
             MetadataLockManager.INSTANCE.dropDatasetEnd(dataverseName, dataverseName + "." + datasetName);
+            ExternalDatasetsRegistry.INSTANCE.releaseAcquiredLocks(metadataProvider);
         }
     }
 
@@ -1434,25 +1412,23 @@
             }
 
             // #. prepare jobs to drop the datatset and the indexes in NC
-            List<Index> indexes = MetadataManager.INSTANCE.getDatasetIndexes(mdTxnCtx.getValue(), dataverseName,
-                    datasetName);
+            List<Index> indexes =
+                    MetadataManager.INSTANCE.getDatasetIndexes(mdTxnCtx.getValue(), dataverseName, datasetName);
             for (int j = 0; j < indexes.size(); j++) {
                 if (indexes.get(j).isSecondaryIndex()) {
-                    CompiledIndexDropStatement cds = new CompiledIndexDropStatement(dataverseName, datasetName,
-                            indexes.get(j).getIndexName());
-                    jobsToExecute.add(IndexOperations.buildDropSecondaryIndexJobSpec(cds, metadataProvider, ds));
+                    jobsToExecute.add(IndexUtil.buildDropSecondaryIndexJobSpec(indexes.get(j), metadataProvider, ds));
                 }
             }
-            CompiledDatasetDropStatement cds = new CompiledDatasetDropStatement(dataverseName, datasetName);
-            jobsToExecute.add(DatasetOperations.createDropDatasetJobSpec(cds, metadataProvider));
-
+            Index primaryIndex =
+                    MetadataManager.INSTANCE.getIndex(mdTxnCtx.getValue(), dataverseName, datasetName, datasetName);
+            jobsToExecute.add(DatasetUtil.dropDatasetJobSpec(ds, primaryIndex, metadataProvider));
             // #. mark the existing dataset as PendingDropOp
             MetadataManager.INSTANCE.dropDataset(mdTxnCtx.getValue(), dataverseName, datasetName);
             MetadataManager.INSTANCE.addDataset(mdTxnCtx.getValue(),
                     new Dataset(dataverseName, datasetName, ds.getItemTypeDataverseName(), ds.getItemTypeName(),
                             ds.getMetaItemTypeDataverseName(), ds.getMetaItemTypeName(), ds.getNodeGroupName(),
                             ds.getCompactionPolicy(), ds.getCompactionPolicyProperties(), ds.getDatasetDetails(),
-                            ds.getHints(), ds.getDatasetType(), ds.getDatasetId(), IMetadataEntity.PENDING_DROP_OP));
+                            ds.getHints(), ds.getDatasetType(), ds.getDatasetId(), MetadataUtil.PENDING_DROP_OP));
 
             MetadataManager.INSTANCE.commitTransaction(mdTxnCtx.getValue());
             bActiveTxn.setValue(false);
@@ -1475,17 +1451,13 @@
             // External dataset
             ExternalDatasetsRegistry.INSTANCE.removeDatasetInfo(ds);
             // #. prepare jobs to drop the datatset and the indexes in NC
-            List<Index> indexes = MetadataManager.INSTANCE.getDatasetIndexes(mdTxnCtx.getValue(), dataverseName,
-                    datasetName);
+            List<Index> indexes =
+                    MetadataManager.INSTANCE.getDatasetIndexes(mdTxnCtx.getValue(), dataverseName, datasetName);
             for (int j = 0; j < indexes.size(); j++) {
                 if (ExternalIndexingOperations.isFileIndex(indexes.get(j))) {
-                    CompiledIndexDropStatement cds = new CompiledIndexDropStatement(dataverseName, datasetName,
-                            indexes.get(j).getIndexName());
-                    jobsToExecute.add(IndexOperations.buildDropSecondaryIndexJobSpec(cds, metadataProvider, ds));
+                    jobsToExecute.add(IndexUtil.buildDropSecondaryIndexJobSpec(indexes.get(j), metadataProvider, ds));
                 } else {
-                    CompiledIndexDropStatement cds = new CompiledIndexDropStatement(dataverseName, datasetName,
-                            indexes.get(j).getIndexName());
-                    jobsToExecute.add(ExternalIndexingOperations.buildDropFilesIndexJobSpec(cds, metadataProvider, ds));
+                    jobsToExecute.add(ExternalIndexingOperations.buildDropFilesIndexJobSpec(metadataProvider, ds));
                 }
             }
 
@@ -1495,7 +1467,7 @@
                     new Dataset(dataverseName, datasetName, ds.getItemTypeDataverseName(), ds.getItemTypeName(),
                             ds.getNodeGroupName(), ds.getCompactionPolicy(), ds.getCompactionPolicyProperties(),
                             ds.getDatasetDetails(), ds.getHints(), ds.getDatasetType(), ds.getDatasetId(),
-                            IMetadataEntity.PENDING_DROP_OP));
+                            MetadataUtil.PENDING_DROP_OP));
 
             MetadataManager.INSTANCE.commitTransaction(mdTxnCtx.getValue());
             bActiveTxn.setValue(false);
@@ -1573,15 +1545,15 @@
                     }
                 }
                 // #. prepare a job to drop the index in NC.
-                CompiledIndexDropStatement cds = new CompiledIndexDropStatement(dataverseName, datasetName, indexName);
-                jobsToExecute.add(IndexOperations.buildDropSecondaryIndexJobSpec(cds, metadataProvider, ds));
+                jobsToExecute.add(IndexUtil.buildDropSecondaryIndexJobSpec(index, metadataProvider, ds));
 
                 // #. mark PendingDropOp on the existing index
                 MetadataManager.INSTANCE.dropIndex(mdTxnCtx, dataverseName, datasetName, indexName);
                 MetadataManager.INSTANCE.addIndex(mdTxnCtx,
-                        new Index(dataverseName, datasetName, indexName, index.getIndexType(), index.getKeyFieldNames(),
-                                index.getKeyFieldSourceIndicators(), index.getKeyFieldTypes(),
-                                index.isEnforcingKeyFileds(), index.isPrimaryIndex(), IMetadataEntity.PENDING_DROP_OP));
+                        new Index(dataverseName, datasetName, indexName, index.getIndexType(),
+                                index.getKeyFieldNames(), index.getKeyFieldSourceIndicators(),
+                                index.getKeyFieldTypes(), index.isEnforcingKeyFileds(), index.isPrimaryIndex(),
+                                MetadataUtil.PENDING_DROP_OP));
 
                 // #. commit the existing transaction before calling runJob.
                 MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
@@ -1614,19 +1586,16 @@
                     throw new AlgebricksException("Dropping a dataset's files index is not allowed.");
                 }
                 // #. prepare a job to drop the index in NC.
-                CompiledIndexDropStatement cds = new CompiledIndexDropStatement(dataverseName, datasetName, indexName);
-                jobsToExecute.add(IndexOperations.buildDropSecondaryIndexJobSpec(cds, metadataProvider, ds));
-                List<Index> datasetIndexes = MetadataManager.INSTANCE.getDatasetIndexes(mdTxnCtx, dataverseName,
-                        datasetName);
+                jobsToExecute.add(IndexUtil.buildDropSecondaryIndexJobSpec(index, metadataProvider, ds));
+                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)) {
-                            cds = new CompiledIndexDropStatement(dataverseName, datasetName,
-                                    externalIndex.getIndexName());
-                            jobsToExecute.add(
-                                    ExternalIndexingOperations.buildDropFilesIndexJobSpec(cds, metadataProvider, ds));
+                            jobsToExecute
+                                    .add(ExternalIndexingOperations.buildDropFilesIndexJobSpec(metadataProvider, ds));
                             // #. mark PendingDropOp on the existing files index
                             MetadataManager.INSTANCE.dropIndex(mdTxnCtx, dataverseName, datasetName,
                                     externalIndex.getIndexName());
@@ -1635,7 +1604,7 @@
                                             externalIndex.getIndexType(), externalIndex.getKeyFieldNames(),
                                             externalIndex.getKeyFieldSourceIndicators(), index.getKeyFieldTypes(),
                                             index.isEnforcingKeyFileds(), externalIndex.isPrimaryIndex(),
-                                            IMetadataEntity.PENDING_DROP_OP));
+                                            MetadataUtil.PENDING_DROP_OP));
                         }
                     }
                 }
@@ -1643,9 +1612,10 @@
                 // #. mark PendingDropOp on the existing index
                 MetadataManager.INSTANCE.dropIndex(mdTxnCtx, dataverseName, datasetName, indexName);
                 MetadataManager.INSTANCE.addIndex(mdTxnCtx,
-                        new Index(dataverseName, datasetName, indexName, index.getIndexType(), index.getKeyFieldNames(),
-                                index.getKeyFieldSourceIndicators(), index.getKeyFieldTypes(),
-                                index.isEnforcingKeyFileds(), index.isPrimaryIndex(), IMetadataEntity.PENDING_DROP_OP));
+                        new Index(dataverseName, datasetName, indexName, index.getIndexType(),
+                                index.getKeyFieldNames(), index.getKeyFieldSourceIndicators(),
+                                index.getKeyFieldTypes(), index.isEnforcingKeyFileds(), index.isPrimaryIndex(),
+                                MetadataUtil.PENDING_DROP_OP));
 
                 // #. commit the existing transaction before calling runJob.
                 MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
@@ -1666,7 +1636,7 @@
                 if (dropFilesIndex) {
                     // delete the files index too
                     MetadataManager.INSTANCE.dropIndex(mdTxnCtx, dataverseName, datasetName,
-                            ExternalIndexingOperations.getFilesIndexName(datasetName));
+                            IndexingConstants.getFilesIndexName(datasetName));
                     MetadataManager.INSTANCE.dropDatasetExternalFiles(mdTxnCtx, ds);
                     ExternalDatasetsRegistry.INSTANCE.removeDatasetInfo(ds);
                 }
@@ -1698,14 +1668,14 @@
                             datasetName, indexName);
                     if (dropFilesIndex) {
                         MetadataManager.INSTANCE.dropIndex(metadataProvider.getMetadataTxnContext(), dataverseName,
-                                datasetName, ExternalIndexingOperations.getFilesIndexName(datasetName));
+                                datasetName, IndexingConstants.getFilesIndexName(datasetName));
                     }
                     MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
                 } catch (Exception e2) {
                     e.addSuppressed(e2);
                     abort(e, e2, mdTxnCtx);
-                    throw new IllegalStateException("System is inconsistent state: pending index(" + dataverseName + "."
-                            + datasetName + "." + indexName + ") couldn't be removed from the metadata", e);
+                    throw new IllegalStateException("System is inconsistent state: pending index(" + dataverseName
+                            + "." + datasetName + "." + indexName + ") couldn't be removed from the metadata", e);
                 }
             }
 
@@ -1713,6 +1683,7 @@
 
         } finally {
             MetadataLockManager.INSTANCE.dropIndexEnd(dataverseName, dataverseName + "." + datasetName);
+            ExternalDatasetsRegistry.INSTANCE.releaseAcquiredLocks(metadataProvider);
         }
     }
 
@@ -1769,8 +1740,7 @@
         }
     }
 
-    protected void handleCreateFunctionStatement(MetadataProvider metadataProvider, Statement stmt)
-            throws Exception {
+    protected void handleCreateFunctionStatement(MetadataProvider metadataProvider, Statement stmt) throws Exception {
         CreateFunctionStatement cfs = (CreateFunctionStatement) stmt;
         String dataverse = getActiveDataverseName(cfs.getSignature().getNamespace());
         cfs.getSignature().setNamespace(dataverse);
@@ -1826,8 +1796,8 @@
         }
     }
 
-    protected void handleLoadStatement(MetadataProvider metadataProvider, Statement stmt,
-            IHyracksClientConnection hcc) throws Exception {
+    protected void handleLoadStatement(MetadataProvider metadataProvider, Statement stmt, IHyracksClientConnection hcc)
+            throws Exception {
         LoadStatement loadStmt = (LoadStatement) stmt;
         String dataverseName = getActiveDataverse(loadStmt.getDataverseName());
         String datasetName = loadStmt.getDatasetName().getValue();
@@ -1836,11 +1806,11 @@
         metadataProvider.setMetadataTxnContext(mdTxnCtx);
         MetadataLockManager.INSTANCE.modifyDatasetBegin(dataverseName, dataverseName + "." + datasetName);
         try {
-            CompiledLoadFromFileStatement cls = new CompiledLoadFromFileStatement(dataverseName,
-                    loadStmt.getDatasetName().getValue(), loadStmt.getAdapter(), loadStmt.getProperties(),
-                    loadStmt.dataIsAlreadySorted());
-            JobSpecification spec = apiFramework.compileQuery(hcc, metadataProvider, null, 0, null, sessionConfig,
-                    cls);
+            CompiledLoadFromFileStatement cls =
+                    new CompiledLoadFromFileStatement(dataverseName, loadStmt.getDatasetName().getValue(),
+                            loadStmt.getAdapter(), loadStmt.getProperties(), loadStmt.dataIsAlreadySorted());
+            JobSpecification spec =
+                    apiFramework.compileQuery(hcc, metadataProvider, null, 0, null, sessionConfig, cls);
             MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
             bActiveTxn = false;
             if (spec != null) {
@@ -1894,8 +1864,8 @@
         }
     }
 
-    public void handleDeleteStatement(MetadataProvider metadataProvider, Statement stmt,
-            IHyracksClientConnection hcc) throws Exception {
+    public void handleDeleteStatement(MetadataProvider metadataProvider, Statement stmt, IHyracksClientConnection hcc)
+            throws Exception {
 
         DeleteStatement stmtDelete = (DeleteStatement) stmt;
         String dataverseName = getActiveDataverse(stmtDelete.getDataverseName());
@@ -1934,13 +1904,12 @@
 
     @Override
     public JobSpecification rewriteCompileQuery(IClusterInfoCollector clusterInfoCollector,
-            MetadataProvider metadataProvider, Query query,
-            ICompiledDmlStatement stmt)
+            MetadataProvider metadataProvider, Query query, ICompiledDmlStatement stmt)
             throws RemoteException, AlgebricksException, ACIDException {
 
         // Query Rewriting (happens under the same ongoing metadata transaction)
-        Pair<IReturningStatement, Integer> rewrittenResult = apiFramework.reWriteQuery(declaredFunctions,
-                metadataProvider, query, sessionConfig);
+        Pair<IReturningStatement, Integer> rewrittenResult =
+                apiFramework.reWriteQuery(declaredFunctions, metadataProvider, query, sessionConfig);
 
         // Query Compilation (happens under the same ongoing metadata transaction)
         return apiFramework.compileQuery(clusterInfoCollector, metadataProvider, (Query) rewrittenResult.first,
@@ -1952,8 +1921,8 @@
             throws RemoteException, AlgebricksException, ACIDException {
 
         // Insert/upsert statement rewriting (happens under the same ongoing metadata transaction)
-        Pair<IReturningStatement, Integer> rewrittenResult = apiFramework.reWriteQuery(declaredFunctions,
-                metadataProvider, insertUpsert, sessionConfig);
+        Pair<IReturningStatement, Integer> rewrittenResult =
+                apiFramework.reWriteQuery(declaredFunctions, metadataProvider, insertUpsert, sessionConfig);
 
         InsertStatement rewrittenInsertUpsert = (InsertStatement) rewrittenResult.first;
         String dataverseName = getActiveDataverse(rewrittenInsertUpsert.getDataverseName());
@@ -2049,8 +2018,8 @@
             boolean extendingExisting = cfps.getSourcePolicyName() != null;
             String description = cfps.getDescription() == null ? "" : cfps.getDescription();
             if (extendingExisting) {
-                FeedPolicyEntity sourceFeedPolicy = MetadataManager.INSTANCE
-                        .getFeedPolicy(metadataProvider.getMetadataTxnContext(), dataverse, cfps.getSourcePolicyName());
+                FeedPolicyEntity sourceFeedPolicy = MetadataManager.INSTANCE.getFeedPolicy(
+                        metadataProvider.getMetadataTxnContext(), dataverse, cfps.getSourcePolicyName());
                 if (sourceFeedPolicy == null) {
                     sourceFeedPolicy = MetadataManager.INSTANCE.getFeedPolicy(metadataProvider.getMetadataTxnContext(),
                             MetadataConstants.METADATA_DATAVERSE_NAME, cfps.getSourcePolicyName());
@@ -2105,8 +2074,8 @@
             }
 
             EntityId feedId = new EntityId(Feed.EXTENSION_NAME, dataverseName, feedName);
-            FeedEventsListener listener = (FeedEventsListener) ActiveJobNotificationHandler.INSTANCE
-                    .getActiveEntityListener(feedId);
+            FeedEventsListener listener =
+                    (FeedEventsListener) ActiveJobNotificationHandler.INSTANCE.getActiveEntityListener(feedId);
             if (listener != null) {
                 StringBuilder builder = new StringBuilder();
                 for (FeedConnectionId connectionId : listener.getConnections()) {
@@ -2134,8 +2103,7 @@
         }
     }
 
-    protected void handleDropFeedPolicyStatement(MetadataProvider metadataProvider, Statement stmt)
-            throws Exception {
+    protected void handleDropFeedPolicyStatement(MetadataProvider metadataProvider, Statement stmt) throws Exception {
         MetadataTransactionContext mdTxnCtx = MetadataManager.INSTANCE.beginTransaction();
         metadataProvider.setMetadataTxnContext(mdTxnCtx);
         FeedPolicyDropStatement stmtFeedPolicyDrop = (FeedPolicyDropStatement) stmt;
@@ -2176,8 +2144,8 @@
         IActiveLifecycleEventSubscriber eventSubscriber = new ActiveLifecycleEventSubscriber();
         FeedConnectionId feedConnId = null;
         EntityId entityId = new EntityId(Feed.EXTENSION_NAME, dataverseName, cfs.getFeedName());
-        FeedEventsListener listener = (FeedEventsListener) ActiveJobNotificationHandler.INSTANCE
-                .getActiveEntityListener(entityId);
+        FeedEventsListener listener =
+                (FeedEventsListener) ActiveJobNotificationHandler.INSTANCE.getActiveEntityListener(entityId);
         MetadataLockManager.INSTANCE.connectFeedBegin(dataverseName, dataverseName + "." + datasetName,
                 dataverseName + "." + feedName);
         try {
@@ -2196,26 +2164,26 @@
                 throw new CompilationException("Feed " + cfs.getFeedName() + " is already connected to dataset "
                         + cfs.getDatasetName().getValue());
             }
-            FeedPolicyEntity feedPolicy = FeedMetadataUtil.validateIfPolicyExists(dataverseName, cbfs.getPolicyName(),
-                    mdTxnCtx);
+            FeedPolicyEntity feedPolicy =
+                    FeedMetadataUtil.validateIfPolicyExists(dataverseName, cbfs.getPolicyName(), mdTxnCtx);
             // All Metadata checks have passed. Feed connect request is valid. //
             if (listener == null) {
                 listener = new FeedEventsListener(entityId);
                 ActiveJobNotificationHandler.INSTANCE.registerListener(listener);
             }
             FeedPolicyAccessor policyAccessor = new FeedPolicyAccessor(feedPolicy.getProperties());
-            Triple<FeedConnectionRequest, Boolean, List<IFeedJoint>> triple = getFeedConnectionRequest(dataverseName,
-                    feed, cbfs.getDatasetName(), feedPolicy, mdTxnCtx);
+            Triple<FeedConnectionRequest, Boolean, List<IFeedJoint>> triple =
+                    getFeedConnectionRequest(dataverseName, feed, cbfs.getDatasetName(), feedPolicy, mdTxnCtx);
             FeedConnectionRequest connectionRequest = triple.first;
             boolean createFeedIntakeJob = triple.second;
             listener.registerFeedEventSubscriber(eventSubscriber);
             subscriberRegistered = true;
             if (createFeedIntakeJob) {
                 EntityId feedId = connectionRequest.getFeedJointKey().getFeedId();
-                Feed primaryFeed = MetadataManager.INSTANCE.getFeed(mdTxnCtx, feedId.getDataverse(),
-                        feedId.getEntityName());
-                Pair<JobSpecification, IAdapterFactory> pair = FeedOperations.buildFeedIntakeJobSpec(primaryFeed,
-                        metadataProvider, policyAccessor);
+                Feed primaryFeed =
+                        MetadataManager.INSTANCE.getFeed(mdTxnCtx, feedId.getDataverse(), feedId.getEntityName());
+                Pair<JobSpecification, IAdapterFactory> pair =
+                        FeedOperations.buildFeedIntakeJobSpec(primaryFeed, metadataProvider, policyAccessor);
                 // adapter configuration are valid at this stage
                 // register the feed joints (these are auto-de-registered)
                 int numOfPrividers = pair.second.getPartitionConstraint().getLocations().length;
@@ -2276,8 +2244,8 @@
         FeedRuntimeType connectionLocation;
         FeedJointKey feedJointKey = getFeedJointKey(feed, mdTxnCtx);
         EntityId entityId = new EntityId(Feed.EXTENSION_NAME, dataverse, feed.getFeedName());
-        FeedEventsListener listener = (FeedEventsListener) ActiveJobNotificationHandler.INSTANCE
-                .getActiveEntityListener(entityId);
+        FeedEventsListener listener =
+                (FeedEventsListener) ActiveJobNotificationHandler.INSTANCE.getActiveEntityListener(entityId);
         if (listener == null) {
             throw new CompilationException("Feed Listener is not registered");
         }
@@ -2339,8 +2307,8 @@
             if (sourceFeed.getAppliedFunction() != null) {
                 appliedFunctions.add(0, sourceFeed.getAppliedFunction().getName());
             }
-            Feed parentFeed = MetadataManager.INSTANCE.getFeed(ctx, feed.getDataverseName(),
-                    sourceFeed.getSourceFeedName());
+            Feed parentFeed =
+                    MetadataManager.INSTANCE.getFeed(ctx, feed.getDataverseName(), sourceFeed.getSourceFeedName());
             sourceFeed = parentFeed;
         }
 
@@ -2365,12 +2333,11 @@
         EntityId entityId = new EntityId(Feed.EXTENSION_NAME, feed.getDataverseName(), feed.getFeedName());
         FeedConnectionId connectionId = new FeedConnectionId(feed.getFeedId(), cfs.getDatasetName().getValue());
         IActiveLifecycleEventSubscriber eventSubscriber = new ActiveLifecycleEventSubscriber();
-        FeedEventsListener listener = (FeedEventsListener) ActiveJobNotificationHandler.INSTANCE
-                .getActiveEntityListener(entityId);
+        FeedEventsListener listener =
+                (FeedEventsListener) ActiveJobNotificationHandler.INSTANCE.getActiveEntityListener(entityId);
         if (listener == null || !listener.isEntityUsingDataset(dataverseName, datasetName)) {
-            throw new CompilationException(
-                    "Feed " + feed.getFeedId().getEntityName() + " is currently not connected to "
-                    + cfs.getDatasetName().getValue() + ". Invalid operation!");
+            throw new CompilationException("Feed " + feed.getFeedId().getEntityName()
+                    + " is currently not connected to " + cfs.getDatasetName().getValue() + ". Invalid operation!");
         }
         listener.registerFeedEventSubscriber(eventSubscriber);
         MetadataLockManager.INSTANCE.disconnectFeedBegin(dataverseName, dataverseName + "." + datasetName,
@@ -2382,8 +2349,8 @@
                 throw new CompilationException(
                         "Unknown dataset :" + cfs.getDatasetName().getValue() + " in dataverse " + dataverseName);
             }
-            Pair<JobSpecification, Boolean> specDisconnectType = FeedOperations
-                    .buildDisconnectFeedJobSpec(metadataProvider, connectionId);
+            Pair<JobSpecification, Boolean> specDisconnectType =
+                    FeedOperations.buildDisconnectFeedJobSpec(connectionId);
             JobSpecification jobSpec = specDisconnectType.first;
             MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
             bActiveTxn = false;
@@ -2415,8 +2382,8 @@
         SubscribeFeedStatement bfs = (SubscribeFeedStatement) stmt;
         bfs.initialize(metadataProvider.getMetadataTxnContext());
 
-        CompiledSubscribeFeedStatement csfs = new CompiledSubscribeFeedStatement(bfs.getSubscriptionRequest(),
-                bfs.getVarCounter());
+        CompiledSubscribeFeedStatement csfs =
+                new CompiledSubscribeFeedStatement(bfs.getSubscriptionRequest(), bfs.getVarCounter());
         metadataProvider.getConfig().put(FunctionUtil.IMPORT_PRIVATE_FUNCTIONS, "" + Boolean.TRUE);
         metadataProvider.getConfig().put(FeedActivityDetails.FEED_POLICY_NAME, "" + bfs.getPolicy());
         metadataProvider.getConfig().put(FeedActivityDetails.COLLECT_LOCATIONS,
@@ -2445,8 +2412,8 @@
             if (jobSpec != null) {
                 FeedEventsListener listener = (FeedEventsListener) ActiveJobNotificationHandler.INSTANCE
                         .getActiveEntityListener(bfs.getSubscriptionRequest().getReceivingFeedId());
-                FeedConnectJobInfo activeJob = new FeedConnectJobInfo(bfs.getSubscriptionRequest().getReceivingFeedId(),
-                        null, ActivityState.ACTIVE,
+                FeedConnectJobInfo activeJob = new FeedConnectJobInfo(
+                        bfs.getSubscriptionRequest().getReceivingFeedId(), null, ActivityState.ACTIVE,
                         new FeedConnectionId(bfs.getSubscriptionRequest().getReceivingFeedId(), dataset),
                         listener.getSourceFeedJoint(), null, alteredJobSpec, policy.getProperties());
                 alteredJobSpec.setProperty(ActiveJobNotificationHandler.ACTIVE_ENTITY_PROPERTY_NAME, activeJob);
@@ -2486,10 +2453,9 @@
                     ds.getItemTypeDataverseName(), itemTypeName);
             ARecordType metaRecordType = null;
             if (ds.hasMetaPart()) {
-                metaRecordType = (ARecordType) MetadataManager.INSTANCE
-                        .getDatatype(metadataProvider.getMetadataTxnContext(), ds.getMetaItemTypeDataverseName(),
-                                ds.getMetaItemTypeName())
-                        .getDatatype();
+                metaRecordType =
+                        (ARecordType) MetadataManager.INSTANCE.getDatatype(metadataProvider.getMetadataTxnContext(),
+                                ds.getMetaItemTypeDataverseName(), ds.getMetaItemTypeName()).getDatatype();
             }
             // Prepare jobs to compact the datatset and its indexes
             List<Index> indexes = MetadataManager.INSTANCE.getDatasetIndexes(mdTxnCtx, dataverseName, datasetName);
@@ -2497,27 +2463,24 @@
                 throw new AlgebricksException(
                         "Cannot compact the extrenal dataset " + datasetName + " because it has no indexes");
             }
-            Dataverse dataverse = MetadataManager.INSTANCE.getDataverse(metadataProvider.getMetadataTxnContext(),
-                    dataverseName);
-            jobsToExecute.add(DatasetOperations.compactDatasetJobSpec(dataverse, datasetName, metadataProvider));
+            Dataverse dataverse =
+                    MetadataManager.INSTANCE.getDataverse(metadataProvider.getMetadataTxnContext(), dataverseName);
+            jobsToExecute.add(DatasetUtil.compactDatasetJobSpec(dataverse, datasetName, metadataProvider));
             ARecordType aRecordType = (ARecordType) dt.getDatatype();
-            ARecordType enforcedType = createEnforcedType(aRecordType, indexes);
+            Pair<ARecordType, ARecordType> enforcedTypes =
+                    TypeUtil.createEnforcedType(aRecordType, metaRecordType, indexes);
+            ARecordType enforcedType = enforcedTypes.first;
+            ARecordType enforcedMeta = enforcedTypes.second;
             if (ds.getDatasetType() == DatasetType.INTERNAL) {
                 for (int j = 0; j < indexes.size(); j++) {
                     if (indexes.get(j).isSecondaryIndex()) {
-                        CompiledIndexCompactStatement cics = new CompiledIndexCompactStatement(dataverseName,
-                                datasetName, indexes.get(j).getIndexName(), indexes.get(j).getKeyFieldNames(),
-                                indexes.get(j).getKeyFieldTypes(), indexes.get(j).isEnforcingKeyFileds(),
-                                indexes.get(j).getGramLength(), indexes.get(j).getIndexType());
-                        List<Integer> keySourceIndicators = indexes.get(j).getKeyFieldSourceIndicators();
-
-                        jobsToExecute.add(IndexOperations.buildSecondaryIndexCompactJobSpec(cics, aRecordType,
-                                metaRecordType, keySourceIndicators, enforcedType, metadataProvider));
+                        jobsToExecute.add(IndexUtil.buildSecondaryIndexCompactJobSpec(ds, indexes.get(j), aRecordType,
+                                metaRecordType, enforcedType, enforcedMeta, metadataProvider));
                     }
                 }
             } else {
-                prepareCompactJobsForExternalDataset(indexes, dataverseName, datasetName, ds, jobsToExecute,
-                        aRecordType, metaRecordType, metadataProvider, enforcedType);
+                prepareCompactJobsForExternalDataset(indexes, ds, jobsToExecute, aRecordType, metaRecordType,
+                        metadataProvider, enforcedType, enforcedMeta);
             }
             MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
             bActiveTxn = false;
@@ -2533,28 +2496,23 @@
             throw e;
         } finally {
             MetadataLockManager.INSTANCE.compactEnd(dataverseName, dataverseName + "." + datasetName);
+            ExternalDatasetsRegistry.INSTANCE.releaseAcquiredLocks(metadataProvider);
         }
     }
 
-    protected void prepareCompactJobsForExternalDataset(List<Index> indexes, String dataverseName, String datasetName,
-            Dataset ds, List<JobSpecification> jobsToExecute, ARecordType aRecordType, ARecordType metaRecordType,
-            MetadataProvider metadataProvider, ARecordType enforcedType) throws AlgebricksException {
+    protected void prepareCompactJobsForExternalDataset(List<Index> indexes, Dataset ds,
+            List<JobSpecification> jobsToExecute, ARecordType aRecordType, ARecordType metaRecordType,
+            MetadataProvider metadataProvider, ARecordType enforcedType, ARecordType enforcedMeta)
+            throws AlgebricksException {
         for (int j = 0; j < indexes.size(); j++) {
             if (!ExternalIndexingOperations.isFileIndex(indexes.get(j))) {
-                CompiledIndexCompactStatement cics = new CompiledIndexCompactStatement(dataverseName, datasetName,
-                        indexes.get(j).getIndexName(), indexes.get(j).getKeyFieldNames(),
-                        indexes.get(j).getKeyFieldTypes(), indexes.get(j).isEnforcingKeyFileds(),
-                        indexes.get(j).getGramLength(), indexes.get(j).getIndexType());
-                List<Integer> keySourceIndicators = null;
-                if (ds.hasMetaPart()) {
-                    keySourceIndicators = indexes.get(j).getKeyFieldSourceIndicators();
-                }
-                jobsToExecute.add(IndexOperations.buildSecondaryIndexCompactJobSpec(cics, aRecordType, metaRecordType,
-                        keySourceIndicators, enforcedType, metadataProvider));
+                jobsToExecute.add(IndexUtil.buildSecondaryIndexCompactJobSpec(ds, indexes.get(j), aRecordType,
+                        metaRecordType, enforcedType, enforcedMeta, metadataProvider));
             }
 
         }
-        jobsToExecute.add(ExternalIndexingOperations.compactFilesIndexJobSpec(ds, metadataProvider));
+        jobsToExecute.add(ExternalIndexingOperations.compactFilesIndexJobSpec(ds, metadataProvider,
+                new StorageComponentProvider()));
     }
 
     protected JobSpecification handleQuery(MetadataProvider metadataProvider, Query query,
@@ -2563,7 +2521,7 @@
         MetadataTransactionContext mdTxnCtx = MetadataManager.INSTANCE.beginTransaction();
         boolean bActiveTxn = true;
         metadataProvider.setMetadataTxnContext(mdTxnCtx);
-        MetadataLockManager.INSTANCE.queryBegin(activeDefaultDataverse, query.getDataverses(), query.getDatasets());
+        MetadataLockManager.INSTANCE.queryBegin(activeDataverse, query.getDataverses(), query.getDatasets());
         try {
             JobSpecification jobSpec = rewriteCompileQuery(hcc, metadataProvider, query, null);
 
@@ -2598,8 +2556,8 @@
         ResultHandle hand;
         switch (resultDelivery) {
             case ASYNC:
-                hand = new ResultHandle(jobId,metadataProvider.getResultSetId());
-                ResultUtil.printResultHandle(hand,sessionConfig);
+                hand = new ResultHandle(jobId, metadataProvider.getResultSetId());
+                ResultUtil.printResultHandle(hand, sessionConfig);
                 hcc.waitForCompletion(jobId);
                 sessionConfig.out().flush();
                 break;
@@ -2611,8 +2569,8 @@
                 break;
             case DEFERRED:
                 hcc.waitForCompletion(jobId);
-                hand = new ResultHandle(jobId,metadataProvider.getResultSetId());
-                ResultUtil.printResultHandle(hand,sessionConfig);
+                hand = new ResultHandle(jobId, metadataProvider.getResultSetId());
+                ResultUtil.printResultHandle(hand, sessionConfig);
                 sessionConfig.out().flush();
                 break;
             default:
@@ -2620,8 +2578,7 @@
         }
     }
 
-    protected void handleCreateNodeGroupStatement(MetadataProvider metadataProvider, Statement stmt)
-            throws Exception {
+    protected void handleCreateNodeGroupStatement(MetadataProvider metadataProvider, Statement stmt) throws Exception {
 
         NodegroupDecl stmtCreateNodegroup = (NodegroupDecl) stmt;
         String ngName = stmtCreateNodegroup.getNodegroupName().getValue();
@@ -2658,7 +2615,7 @@
         RefreshExternalDatasetStatement stmtRefresh = (RefreshExternalDatasetStatement) stmt;
         String dataverseName = getActiveDataverse(stmtRefresh.getDataverseName());
         String datasetName = stmtRefresh.getDatasetName().getValue();
-        ExternalDatasetTransactionState transactionState = ExternalDatasetTransactionState.COMMIT;
+        TransactionState transactionState = TransactionState.COMMIT;
         MetadataTransactionContext mdTxnCtx = MetadataManager.INSTANCE.beginTransaction();
         MetadataLockManager.INSTANCE.refreshDatasetBegin(dataverseName, dataverseName + "." + datasetName);
         boolean bActiveTxn = true;
@@ -2738,20 +2695,20 @@
             }
 
             // Create the files index update job
-            spec = ExternalIndexingOperations.buildFilesIndexUpdateOp(ds, metadataFiles, deletedFiles, addedFiles,
-                    appendedFiles, metadataProvider);
+            spec = ExternalIndexingOperations.buildFilesIndexUpdateOp(ds, metadataFiles, addedFiles, appendedFiles,
+                    metadataProvider);
 
             MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
             bActiveTxn = false;
-            transactionState = ExternalDatasetTransactionState.BEGIN;
+            transactionState = TransactionState.BEGIN;
 
             // run the files update job
             JobUtils.runJob(hcc, spec, true);
 
             for (Index index : indexes) {
                 if (!ExternalIndexingOperations.isFileIndex(index)) {
-                    spec = ExternalIndexingOperations.buildIndexUpdateOp(ds, index, metadataFiles, deletedFiles,
-                            addedFiles, appendedFiles, metadataProvider);
+                    spec = ExternalIndexingOperations.buildIndexUpdateOp(ds, index, metadataFiles, addedFiles,
+                            appendedFiles, metadataProvider);
                     // run the files update job
                     JobUtils.runJob(hcc, spec, true);
                 }
@@ -2765,12 +2722,12 @@
             metadataProvider.setMetadataTxnContext(mdTxnCtx);
             bActiveTxn = true;
             ((ExternalDatasetDetails) transactionDataset.getDatasetDetails())
-                    .setState(ExternalDatasetTransactionState.READY_TO_COMMIT);
+                    .setState(TransactionState.READY_TO_COMMIT);
             ((ExternalDatasetDetails) transactionDataset.getDatasetDetails()).setRefreshTimestamp(txnTime);
             MetadataManager.INSTANCE.updateDataset(mdTxnCtx, transactionDataset);
             MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
             bActiveTxn = false;
-            transactionState = ExternalDatasetTransactionState.READY_TO_COMMIT;
+            transactionState = TransactionState.READY_TO_COMMIT;
             // We don't release the latch since this job is expected to be quick
             JobUtils.runJob(hcc, spec, true);
             // Start a new metadata transaction to record the final state of the transaction
@@ -2779,9 +2736,9 @@
             bActiveTxn = true;
 
             for (ExternalFile file : metadataFiles) {
-                if (file.getPendingOp() == ExternalFilePendingOp.PENDING_DROP_OP) {
+                if (file.getPendingOp() == ExternalFilePendingOp.DROP_OP) {
                     MetadataManager.INSTANCE.dropExternalFile(mdTxnCtx, file);
-                } else if (file.getPendingOp() == ExternalFilePendingOp.PENDING_NO_OP) {
+                } else if (file.getPendingOp() == ExternalFilePendingOp.NO_OP) {
                     Iterator<ExternalFile> iterator = appendedFiles.iterator();
                     while (iterator.hasNext()) {
                         ExternalFile appendedFile = iterator.next();
@@ -2792,7 +2749,7 @@
                             MetadataManager.INSTANCE.dropExternalFile(mdTxnCtx, appendedFile);
                             // add the original file with appended information
                             appendedFile.setFileNumber(file.getFileNumber());
-                            appendedFile.setPendingOp(ExternalFilePendingOp.PENDING_NO_OP);
+                            appendedFile.setPendingOp(ExternalFilePendingOp.NO_OP);
                             MetadataManager.INSTANCE.addExternalFile(mdTxnCtx, appendedFile);
                             iterator.remove();
                         }
@@ -2808,13 +2765,12 @@
             // insert new files
             for (ExternalFile file : addedFiles) {
                 MetadataManager.INSTANCE.dropExternalFile(mdTxnCtx, file);
-                file.setPendingOp(ExternalFilePendingOp.PENDING_NO_OP);
+                file.setPendingOp(ExternalFilePendingOp.NO_OP);
                 MetadataManager.INSTANCE.addExternalFile(mdTxnCtx, file);
             }
 
             // mark the transaction as complete
-            ((ExternalDatasetDetails) transactionDataset.getDatasetDetails())
-                    .setState(ExternalDatasetTransactionState.COMMIT);
+            ((ExternalDatasetDetails) transactionDataset.getDatasetDetails()).setState(TransactionState.COMMIT);
             MetadataManager.INSTANCE.updateDataset(mdTxnCtx, transactionDataset);
 
             // commit metadata transaction
@@ -2824,15 +2780,15 @@
             if (bActiveTxn) {
                 abort(e, e, mdTxnCtx);
             }
-            if (transactionState == ExternalDatasetTransactionState.READY_TO_COMMIT) {
+            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 == ExternalDatasetTransactionState.COMMIT) {
+            if (transactionState == TransactionState.COMMIT) {
                 // Nothing to do , everything should be clean
                 throw e;
             }
-            if (transactionState == ExternalDatasetTransactionState.BEGIN) {
+            if (transactionState == TransactionState.BEGIN) {
                 // transaction failed, need to do the following
                 // clean NCs removing transaction components
                 mdTxnCtx = MetadataManager.INSTANCE.beginTransaction();
@@ -2878,8 +2834,8 @@
         }
     }
 
-    protected void handleRunStatement(MetadataProvider metadataProvider, Statement stmt,
-            IHyracksClientConnection hcc) throws CompilationException, Exception {
+    protected void handleRunStatement(MetadataProvider metadataProvider, Statement stmt, IHyracksClientConnection hcc)
+            throws CompilationException, Exception {
         RunStatement runStmt = (RunStatement) stmt;
         switch (runStmt.getSystem()) {
             case "pregel":
@@ -2887,8 +2843,8 @@
                 handlePregelixStatement(metadataProvider, runStmt, hcc);
                 break;
             default:
-                throw new AlgebricksException(
-                        "The system \"" + runStmt.getSystem() + "\" specified in your run statement is not supported.");
+                throw new AlgebricksException("The system \"" + runStmt.getSystem()
+                        + "\" specified in your run statement is not supported.");
         }
 
     }
@@ -2966,9 +2922,8 @@
         // Validates the source/sink dataverses and datasets.
         Dataset fromDataset = metadataProvider.findDataset(dataverseNameFrom, datasetNameFrom);
         if (fromDataset == null) {
-            throw new CompilationException(
-                    "The source dataset " + datasetNameFrom + " in dataverse " + dataverseNameFrom
-                    + " could not be found for the Run command");
+            throw new CompilationException("The source dataset " + datasetNameFrom + " in dataverse "
+                    + dataverseNameFrom + " could not be found for the Run command");
         }
         Dataset toDataset = metadataProvider.findDataset(dataverseNameTo, datasetNameTo);
         if (toDataset == null) {
@@ -2991,8 +2946,8 @@
                 throw new AlgebricksException("Tried to access non-existing dataset: " + datasetNameTo);
             }
             // Cleans up the sink dataset -- Drop and then Create.
-            DropDatasetStatement dropStmt = new DropDatasetStatement(new Identifier(dataverseNameTo),
-                    pregelixStmt.getDatasetNameTo(), true);
+            DropDatasetStatement dropStmt =
+                    new DropDatasetStatement(new Identifier(dataverseNameTo), pregelixStmt.getDatasetNameTo(), true);
             this.handleDatasetDropStatement(metadataProvider, dropStmt, hcc);
             IDatasetDetailsDecl idd = new InternalDetailsDecl(toIndex.getKeyFieldNames(),
                     toIndex.getKeyFieldSourceIndicators(), false, null, toDataset.getDatasetDetails().isTemp());
@@ -3010,8 +2965,7 @@
         }
 
         // Flushes source dataset.
-        FlushDatasetUtils.flushDataset(hcc, metadataProvider, mdTxnCtx, dataverseNameFrom, datasetNameFrom,
-                datasetNameFrom);
+        FlushDatasetUtil.flushDataset(hcc, metadataProvider, dataverseNameFrom, datasetNameFrom, datasetNameFrom);
     }
 
     // Executes external shell commands.
@@ -3110,7 +3064,7 @@
 
     @Override
     public String getActiveDataverseName(String dataverse) {
-        return (dataverse != null) ? dataverse : activeDefaultDataverse.getDataverseName();
+        return (dataverse != null) ? dataverse : activeDataverse.getDataverseName();
     }
 
     public String getActiveDataverse(Identifier dataverse) {
@@ -3142,106 +3096,4 @@
         IStatementRewriter rewriter = rewriterFactory.createStatementRewriter();
         rewriter.rewrite(stmt);
     }
-
-    /*
-     * Merges typed index fields with specified recordType, allowing indexed fields to be optional.
-     * I.e. the type { "personId":int32, "name": string, "address" : { "street": string } } with typed indexes
-     * on age:int32, address.state:string will be merged into type { "personId":int32, "name": string,
-     * "age": int32? "address" : { "street": string, "state": string? } } Used by open indexes to enforce
-     * the type of an indexed record
-     */
-    private static ARecordType createEnforcedType(ARecordType initialType, List<Index> indexes)
-            throws AlgebricksException {
-        ARecordType enforcedType = initialType;
-        for (Index index : indexes) {
-            if (!index.isSecondaryIndex() || !index.isEnforcingKeyFileds()) {
-                continue;
-            }
-            if (index.hasMetaFields()) {
-                throw new AlgebricksException("Indexing an open field is only supported on the record part");
-            }
-            for (int i = 0; i < index.getKeyFieldNames().size(); i++) {
-                Deque<Pair<ARecordType, String>> nestedTypeStack = new ArrayDeque<>();
-                List<String> splits = index.getKeyFieldNames().get(i);
-                ARecordType nestedFieldType = enforcedType;
-                boolean openRecords = false;
-                String bridgeName = nestedFieldType.getTypeName();
-                int j;
-                // Build the stack for the enforced type
-                for (j = 1; j < splits.size(); j++) {
-                    nestedTypeStack.push(new Pair<>(nestedFieldType, splits.get(j - 1)));
-                    bridgeName = nestedFieldType.getTypeName();
-                    nestedFieldType = (ARecordType) enforcedType.getSubFieldType(splits.subList(0, j));
-                    if (nestedFieldType == null) {
-                        openRecords = true;
-                        break;
-                    }
-                }
-                if (openRecords) {
-                    // create the smallest record
-                    enforcedType = new ARecordType(splits.get(splits.size() - 2),
-                            new String[] { splits.get(splits.size() - 1) },
-                            new IAType[] { AUnionType.createUnknownableType(index.getKeyFieldTypes().get(i)) }, true);
-                    // create the open part of the nested field
-                    for (int k = splits.size() - 3; k > (j - 2); k--) {
-                        enforcedType = new ARecordType(splits.get(k), new String[] { splits.get(k + 1) },
-                                new IAType[] { AUnionType.createUnknownableType(enforcedType) }, true);
-                    }
-                    // Bridge the gap
-                    Pair<ARecordType, String> gapPair = nestedTypeStack.pop();
-                    ARecordType parent = gapPair.first;
-
-                    IAType[] parentFieldTypes = ArrayUtils.addAll(parent.getFieldTypes().clone(),
-                            new IAType[] { AUnionType.createUnknownableType(enforcedType) });
-                    enforcedType = new ARecordType(bridgeName,
-                            ArrayUtils.addAll(parent.getFieldNames(), enforcedType.getTypeName()), parentFieldTypes,
-                            true);
-                } else {
-                    //Schema is closed all the way to the field
-                    //enforced fields are either null or strongly typed
-                    LinkedHashMap<String, IAType> recordNameTypesMap = createRecordNameTypeMap(nestedFieldType);
-                    // if a an enforced field already exists and the type is correct
-                    IAType enforcedFieldType = recordNameTypesMap.get(splits.get(splits.size() - 1));
-                    if (enforcedFieldType != null && enforcedFieldType.getTypeTag() == ATypeTag.UNION
-                            && ((AUnionType) enforcedFieldType).isUnknownableType()) {
-                        enforcedFieldType = ((AUnionType) enforcedFieldType).getActualType();
-                    }
-                    if (enforcedFieldType != null && !ATypeHierarchy.canPromote(enforcedFieldType.getTypeTag(),
-                            index.getKeyFieldTypes().get(i).getTypeTag())) {
-                        throw new AlgebricksException("Cannot enforce field " + index.getKeyFieldNames().get(i)
-                                + " to have type " + index.getKeyFieldTypes().get(i));
-                    }
-                    if (enforcedFieldType == null) {
-                        recordNameTypesMap.put(splits.get(splits.size() - 1),
-                                AUnionType.createUnknownableType(index.getKeyFieldTypes().get(i)));
-                    }
-                    enforcedType = new ARecordType(nestedFieldType.getTypeName(),
-                            recordNameTypesMap.keySet().toArray(new String[recordNameTypesMap.size()]),
-                            recordNameTypesMap.values().toArray(new IAType[recordNameTypesMap.size()]),
-                            nestedFieldType.isOpen());
-                }
-
-                // Create the enforced type for the nested fields in the schema, from the ground up
-                if (!nestedTypeStack.isEmpty()) {
-                    while (!nestedTypeStack.isEmpty()) {
-                        Pair<ARecordType, String> nestedTypePair = nestedTypeStack.pop();
-                        ARecordType nestedRecType = nestedTypePair.first;
-                        IAType[] nestedRecTypeFieldTypes = nestedRecType.getFieldTypes().clone();
-                        nestedRecTypeFieldTypes[nestedRecType.getFieldIndex(nestedTypePair.second)] = enforcedType;
-                        enforcedType = new ARecordType(nestedRecType.getTypeName() + "_enforced",
-                                nestedRecType.getFieldNames(), nestedRecTypeFieldTypes, nestedRecType.isOpen());
-                    }
-                }
-            }
-        }
-        return enforcedType;
-    }
-
-    private static LinkedHashMap<String, IAType> createRecordNameTypeMap(ARecordType nestedFieldType) {
-        LinkedHashMap<String, IAType> recordNameTypesMap = new LinkedHashMap<>();
-        for (int j = 0; j < nestedFieldType.getFieldNames().length; j++) {
-            recordNameTypesMap.put(nestedFieldType.getFieldNames()[j], nestedFieldType.getFieldTypes()[j]);
-        }
-        return recordNameTypesMap;
-    }
 }
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/drivers/AsterixCLI.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/drivers/AsterixCLI.java
index 9b9ab4a..d4f2129 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/drivers/AsterixCLI.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/drivers/AsterixCLI.java
@@ -29,6 +29,7 @@
 import org.apache.asterix.common.config.GlobalConfig;
 import org.apache.asterix.compiler.provider.AqlCompilationProvider;
 import org.apache.asterix.compiler.provider.ILangCompilationProvider;
+import org.apache.asterix.file.StorageComponentProvider;
 import org.apache.commons.io.FileUtils;
 import org.kohsuke.args4j.Argument;
 import org.kohsuke.args4j.CmdLineParser;
@@ -58,7 +59,7 @@
             for (String queryFile : options.args) {
                 Reader in = new FileReader(queryFile);
                 AsterixJavaClient ajc = new AsterixJavaClient(integrationUtil.getHyracksClientConnection(), in,
-                        compilationProvider, new DefaultStatementExecutorFactory());
+                        compilationProvider, new DefaultStatementExecutorFactory(), new StorageComponentProvider());
                 try {
                     ajc.compile(true, false, false, false, false, true, false);
                 } finally {
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/drivers/AsterixClientDriver.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/drivers/AsterixClientDriver.java
index f05b67d..f44aeb9 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/drivers/AsterixClientDriver.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/drivers/AsterixClientDriver.java
@@ -25,6 +25,7 @@
 import org.apache.asterix.app.translator.DefaultStatementExecutorFactory;
 import org.apache.asterix.compiler.provider.AqlCompilationProvider;
 import org.apache.asterix.compiler.provider.ILangCompilationProvider;
+import org.apache.asterix.file.StorageComponentProvider;
 import org.apache.hyracks.api.client.HyracksConnection;
 import org.apache.hyracks.api.client.IHyracksClientConnection;
 import org.kohsuke.args4j.CmdLineParser;
@@ -64,7 +65,7 @@
         ILangCompilationProvider compilationProvider = new AqlCompilationProvider();
         FileReader reader = new FileReader(filename);
         AsterixJavaClient q = new AsterixJavaClient(hcc, reader, compilationProvider,
-                new DefaultStatementExecutorFactory());
+                new DefaultStatementExecutorFactory(), new StorageComponentProvider());
         q.compile(optimize, true, true, true, onlyPhysical, createBinaryRuntime, createBinaryRuntime);
         return q;
     }
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/file/DatasetOperations.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/file/DatasetOperations.java
deleted file mode 100644
index 1c05a62..0000000
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/file/DatasetOperations.java
+++ /dev/null
@@ -1,259 +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.file;
-
-import java.io.File;
-import java.rmi.RemoteException;
-import java.util.Map;
-import java.util.logging.Logger;
-
-import org.apache.asterix.common.config.StorageProperties;
-import org.apache.asterix.common.config.DatasetConfig.DatasetType;
-import org.apache.asterix.common.context.AsterixVirtualBufferCacheProvider;
-import org.apache.asterix.common.dataflow.LSMIndexUtil;
-import org.apache.asterix.common.exceptions.ACIDException;
-import org.apache.asterix.common.exceptions.AsterixException;
-import org.apache.asterix.common.ioopcallbacks.LSMBTreeIOOperationCallbackFactory;
-import org.apache.asterix.common.transactions.IResourceFactory;
-import org.apache.asterix.formats.base.IDataFormat;
-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.Dataverse;
-import org.apache.asterix.metadata.utils.DatasetUtils;
-import org.apache.asterix.om.types.ARecordType;
-import org.apache.asterix.runtime.util.AppContextInfo;
-import org.apache.asterix.runtime.util.RuntimeComponentsProvider;
-import org.apache.asterix.transaction.management.opcallbacks.PrimaryIndexOperationTrackerProvider;
-import org.apache.asterix.transaction.management.resource.LSMBTreeLocalResourceMetadataFactory;
-import org.apache.asterix.transaction.management.resource.PersistentLocalResourceFactoryProvider;
-import org.apache.asterix.translator.CompiledStatements.CompiledDatasetDropStatement;
-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.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.FileSplit;
-import org.apache.hyracks.api.job.JobSpecification;
-import org.apache.hyracks.dataflow.std.file.IFileSplitProvider;
-import org.apache.hyracks.storage.am.common.dataflow.IndexDropOperatorDescriptor;
-import org.apache.hyracks.storage.am.common.dataflow.TreeIndexCreateOperatorDescriptor;
-import org.apache.hyracks.storage.am.common.impls.NoOpOperationCallbackFactory;
-import org.apache.hyracks.storage.am.lsm.btree.dataflow.LSMBTreeDataflowHelperFactory;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMMergePolicyFactory;
-import org.apache.hyracks.storage.am.lsm.common.dataflow.LSMTreeIndexCompactOperatorDescriptor;
-import org.apache.hyracks.storage.common.file.ILocalResourceFactoryProvider;
-import org.apache.hyracks.storage.common.file.LocalResource;
-
-public class DatasetOperations {
-
-    private static Logger LOGGER = Logger.getLogger(DatasetOperations.class.getName());
-
-    public static JobSpecification createDropDatasetJobSpec(CompiledDatasetDropStatement datasetDropStmt,
-            MetadataProvider metadataProvider)
-            throws AlgebricksException, HyracksDataException, RemoteException, ACIDException, AsterixException {
-
-        String dataverseName = null;
-        if (datasetDropStmt.getDataverseName() != null) {
-            dataverseName = datasetDropStmt.getDataverseName();
-        } else if (metadataProvider.getDefaultDataverse() != null) {
-            dataverseName = metadataProvider.getDefaultDataverse().getDataverseName();
-        }
-
-        String datasetName = datasetDropStmt.getDatasetName();
-        String datasetPath = dataverseName + File.separator + datasetName;
-
-        LOGGER.info("DROP DATASETPATH: " + datasetPath);
-
-        Dataset dataset = metadataProvider.findDataset(dataverseName, datasetName);
-        if (dataset == null) {
-            throw new AlgebricksException("DROP DATASET: No metadata for dataset " + datasetName);
-        }
-        if (dataset.getDatasetType() == DatasetType.EXTERNAL) {
-            return JobSpecificationUtils.createJobSpecification();
-        }
-        boolean temp = dataset.getDatasetDetails().isTemp();
-
-        Dataverse dataverse = MetadataManager.INSTANCE.getDataverse(metadataProvider.getMetadataTxnContext(),
-                dataverseName);
-        IDataFormat format;
-        try {
-            format = (IDataFormat) Class.forName(dataverse.getDataFormat()).newInstance();
-        } catch (Exception e) {
-            throw new AsterixException(e);
-        }
-
-        ARecordType itemType = (ARecordType) metadataProvider.findType(dataset.getItemTypeDataverseName(),
-                dataset.getItemTypeName());
-
-        ITypeTraits[] filterTypeTraits = DatasetUtils.computeFilterTypeTraits(dataset, itemType);
-        IBinaryComparatorFactory[] filterCmpFactories = DatasetUtils.computeFilterBinaryComparatorFactories(dataset,
-                itemType, format.getBinaryComparatorFactoryProvider());
-        int[] filterFields = DatasetUtils.createFilterFields(dataset);
-        int[] btreeFields = DatasetUtils.createBTreeFieldsWhenThereisAFilter(dataset);
-        JobSpecification specPrimary = JobSpecificationUtils.createJobSpecification();
-
-        Pair<IFileSplitProvider, AlgebricksPartitionConstraint> splitsAndConstraint = metadataProvider
-                .splitProviderAndPartitionConstraintsForDataset(dataset.getDataverseName(), datasetName, datasetName,
-                        temp);
-        StorageProperties storageProperties = AppContextInfo.INSTANCE.getStorageProperties();
-        Pair<ILSMMergePolicyFactory, Map<String, String>> compactionInfo = DatasetUtils.getMergePolicyFactory(dataset,
-                metadataProvider.getMetadataTxnContext());
-
-        IndexDropOperatorDescriptor primaryBtreeDrop = new IndexDropOperatorDescriptor(specPrimary,
-                RuntimeComponentsProvider.RUNTIME_PROVIDER, RuntimeComponentsProvider.RUNTIME_PROVIDER,
-                splitsAndConstraint.first,
-                new LSMBTreeDataflowHelperFactory(new AsterixVirtualBufferCacheProvider(dataset.getDatasetId()),
-                        compactionInfo.first, compactionInfo.second,
-                        new PrimaryIndexOperationTrackerProvider(dataset.getDatasetId()),
-                        RuntimeComponentsProvider.RUNTIME_PROVIDER, LSMBTreeIOOperationCallbackFactory.INSTANCE,
-                        storageProperties.getBloomFilterFalsePositiveRate(), true, filterTypeTraits, filterCmpFactories,
-                        btreeFields, filterFields, !temp), LSMIndexUtil.getMetadataPageManagerFactory());
-        AlgebricksPartitionConstraintHelper.setPartitionConstraintInJobSpec(specPrimary, primaryBtreeDrop,
-                splitsAndConstraint.second);
-
-        specPrimary.addRoot(primaryBtreeDrop);
-
-        return specPrimary;
-    }
-
-    public static JobSpecification createDatasetJobSpec(Dataverse dataverse, String datasetName,
-            MetadataProvider metadata) throws AsterixException, AlgebricksException {
-        String dataverseName = dataverse.getDataverseName();
-        IDataFormat format;
-        try {
-            format = (IDataFormat) Class.forName(dataverse.getDataFormat()).newInstance();
-        } catch (Exception e) {
-            throw new AsterixException(e);
-        }
-        Dataset dataset = metadata.findDataset(dataverseName, datasetName);
-        if (dataset == null) {
-            throw new AsterixException("Could not find dataset " + datasetName + " in dataverse " + dataverseName);
-        }
-        boolean temp = dataset.getDatasetDetails().isTemp();
-        ARecordType itemType = (ARecordType) metadata.findType(dataset.getItemTypeDataverseName(),
-                dataset.getItemTypeName());
-        // get meta item type
-        ARecordType metaItemType = null;
-        if (dataset.hasMetaPart()) {
-            metaItemType = (ARecordType) metadata.findType(dataset.getMetaItemTypeDataverseName(),
-                    dataset.getMetaItemTypeName());
-        }
-        JobSpecification spec = JobSpecificationUtils.createJobSpecification();
-        IBinaryComparatorFactory[] comparatorFactories = DatasetUtils.computeKeysBinaryComparatorFactories(dataset,
-                itemType, metaItemType, format.getBinaryComparatorFactoryProvider());
-        ITypeTraits[] typeTraits = DatasetUtils.computeTupleTypeTraits(dataset, itemType, metaItemType);
-        int[] bloomFilterKeyFields = DatasetUtils.createBloomFilterKeyFields(dataset);
-
-        ITypeTraits[] filterTypeTraits = DatasetUtils.computeFilterTypeTraits(dataset, itemType);
-        IBinaryComparatorFactory[] filterCmpFactories = DatasetUtils.computeFilterBinaryComparatorFactories(dataset,
-                itemType, format.getBinaryComparatorFactoryProvider());
-        int[] filterFields = DatasetUtils.createFilterFields(dataset);
-        int[] btreeFields = DatasetUtils.createBTreeFieldsWhenThereisAFilter(dataset);
-
-        Pair<IFileSplitProvider, AlgebricksPartitionConstraint> splitsAndConstraint = metadata
-                .splitProviderAndPartitionConstraintsForDataset(dataverseName, datasetName, datasetName, temp);
-        FileSplit[] fs = splitsAndConstraint.first.getFileSplits();
-        StringBuilder sb = new StringBuilder();
-        for (int i = 0; i < fs.length; i++) {
-            sb.append(fs[i] + " ");
-        }
-        LOGGER.info("CREATING File Splits: " + sb.toString());
-
-        Pair<ILSMMergePolicyFactory, Map<String, String>> compactionInfo = DatasetUtils.getMergePolicyFactory(dataset,
-                metadata.getMetadataTxnContext());
-        StorageProperties storageProperties = AppContextInfo.INSTANCE.getStorageProperties();
-        //prepare a LocalResourceMetadata which will be stored in NC's local resource repository
-        IResourceFactory localResourceMetadata = new LSMBTreeLocalResourceMetadataFactory(typeTraits,
-                comparatorFactories, bloomFilterKeyFields, true, dataset.getDatasetId(), compactionInfo.first,
-                compactionInfo.second, filterTypeTraits, filterCmpFactories, btreeFields, filterFields);
-        ILocalResourceFactoryProvider localResourceFactoryProvider = new PersistentLocalResourceFactoryProvider(
-                localResourceMetadata, LocalResource.LSMBTreeResource);
-
-        TreeIndexCreateOperatorDescriptor indexCreateOp = new TreeIndexCreateOperatorDescriptor(spec,
-                RuntimeComponentsProvider.RUNTIME_PROVIDER, RuntimeComponentsProvider.RUNTIME_PROVIDER,
-                splitsAndConstraint.first, typeTraits, comparatorFactories, bloomFilterKeyFields,
-                new LSMBTreeDataflowHelperFactory(new AsterixVirtualBufferCacheProvider(dataset.getDatasetId()),
-                        compactionInfo.first, compactionInfo.second,
-                        new PrimaryIndexOperationTrackerProvider(dataset.getDatasetId()),
-                        RuntimeComponentsProvider.RUNTIME_PROVIDER, LSMBTreeIOOperationCallbackFactory.INSTANCE,
-                        storageProperties.getBloomFilterFalsePositiveRate(), true, filterTypeTraits, filterCmpFactories,
-                        btreeFields, filterFields, !temp),
-                localResourceFactoryProvider, NoOpOperationCallbackFactory.INSTANCE, LSMIndexUtil
-                        .getMetadataPageManagerFactory());
-        AlgebricksPartitionConstraintHelper.setPartitionConstraintInJobSpec(spec, indexCreateOp,
-                splitsAndConstraint.second);
-        spec.addRoot(indexCreateOp);
-        return spec;
-    }
-
-    public static JobSpecification compactDatasetJobSpec(Dataverse dataverse, String datasetName,
-            MetadataProvider metadata) throws AsterixException, AlgebricksException {
-        String dataverseName = dataverse.getDataverseName();
-        IDataFormat format;
-        try {
-            format = (IDataFormat) Class.forName(dataverse.getDataFormat()).newInstance();
-        } catch (Exception e) {
-            throw new AsterixException(e);
-        }
-        Dataset dataset = metadata.findDataset(dataverseName, datasetName);
-        if (dataset == null) {
-            throw new AsterixException("Could not find dataset " + datasetName + " in dataverse " + dataverseName);
-        }
-        boolean temp = dataset.getDatasetDetails().isTemp();
-        ARecordType itemType = (ARecordType) metadata.findType(dataset.getItemTypeDataverseName(),
-                dataset.getItemTypeName());
-        ARecordType metaItemType = DatasetUtils.getMetaType(metadata, dataset);
-        JobSpecification spec = JobSpecificationUtils.createJobSpecification();
-        IBinaryComparatorFactory[] comparatorFactories = DatasetUtils.computeKeysBinaryComparatorFactories(dataset,
-                itemType, metaItemType, format.getBinaryComparatorFactoryProvider());
-        ITypeTraits[] typeTraits = DatasetUtils.computeTupleTypeTraits(dataset, itemType, metaItemType);
-        int[] blooFilterKeyFields = DatasetUtils.createBloomFilterKeyFields(dataset);
-        ITypeTraits[] filterTypeTraits = DatasetUtils.computeFilterTypeTraits(dataset, itemType);
-        IBinaryComparatorFactory[] filterCmpFactories = DatasetUtils.computeFilterBinaryComparatorFactories(dataset,
-                itemType, format.getBinaryComparatorFactoryProvider());
-        int[] filterFields = DatasetUtils.createFilterFields(dataset);
-        int[] btreeFields = DatasetUtils.createBTreeFieldsWhenThereisAFilter(dataset);
-        Pair<IFileSplitProvider, AlgebricksPartitionConstraint> splitsAndConstraint = metadata
-                .splitProviderAndPartitionConstraintsForDataset(dataverseName, datasetName, datasetName, temp);
-        StorageProperties storageProperties = AppContextInfo.INSTANCE.getStorageProperties();
-        Pair<ILSMMergePolicyFactory, Map<String, String>> compactionInfo = DatasetUtils.getMergePolicyFactory(dataset,
-                metadata.getMetadataTxnContext());
-        LSMTreeIndexCompactOperatorDescriptor compactOp = new LSMTreeIndexCompactOperatorDescriptor(spec,
-                RuntimeComponentsProvider.RUNTIME_PROVIDER, RuntimeComponentsProvider.RUNTIME_PROVIDER,
-                splitsAndConstraint.first, typeTraits, comparatorFactories, blooFilterKeyFields,
-                new LSMBTreeDataflowHelperFactory(new AsterixVirtualBufferCacheProvider(dataset.getDatasetId()),
-                        compactionInfo.first, compactionInfo.second,
-                        new PrimaryIndexOperationTrackerProvider(dataset.getDatasetId()),
-                        RuntimeComponentsProvider.RUNTIME_PROVIDER, LSMBTreeIOOperationCallbackFactory.INSTANCE,
-                        storageProperties.getBloomFilterFalsePositiveRate(), true, filterTypeTraits, filterCmpFactories,
-                        btreeFields, filterFields, !temp),
-                NoOpOperationCallbackFactory.INSTANCE, LSMIndexUtil.getMetadataPageManagerFactory());
-        AlgebricksPartitionConstraintHelper.setPartitionConstraintInJobSpec(spec, compactOp,
-                splitsAndConstraint.second);
-
-        AlgebricksPartitionConstraintHelper.setPartitionConstraintInJobSpec(spec, compactOp,
-                splitsAndConstraint.second);
-        spec.addRoot(compactOp);
-        return spec;
-    }
-}
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/file/IndexOperations.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/file/IndexOperations.java
deleted file mode 100644
index d1c0ae3..0000000
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/file/IndexOperations.java
+++ /dev/null
@@ -1,139 +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.file;
-
-import java.util.List;
-import java.util.Map;
-
-import org.apache.asterix.common.config.OptimizationConfUtil;
-import org.apache.asterix.common.config.StorageProperties;
-import org.apache.asterix.common.context.AsterixVirtualBufferCacheProvider;
-import org.apache.asterix.common.dataflow.LSMIndexUtil;
-import org.apache.asterix.common.exceptions.AsterixException;
-import org.apache.asterix.common.ioopcallbacks.LSMBTreeIOOperationCallbackFactory;
-import org.apache.asterix.external.indexing.ExternalFile;
-import org.apache.asterix.metadata.MetadataException;
-import org.apache.asterix.metadata.declared.MetadataProvider;
-import org.apache.asterix.metadata.entities.Dataset;
-import org.apache.asterix.metadata.utils.DatasetUtils;
-import org.apache.asterix.om.types.ARecordType;
-import org.apache.asterix.runtime.util.AppContextInfo;
-import org.apache.asterix.runtime.util.RuntimeComponentsProvider;
-import org.apache.asterix.transaction.management.opcallbacks.SecondaryIndexOperationTrackerProvider;
-import org.apache.asterix.translator.CompiledStatements.CompiledCreateIndexStatement;
-import org.apache.asterix.translator.CompiledStatements.CompiledIndexCompactStatement;
-import org.apache.asterix.translator.CompiledStatements.CompiledIndexDropStatement;
-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.rewriter.base.PhysicalOptimizationConfig;
-import org.apache.hyracks.api.job.JobSpecification;
-import org.apache.hyracks.dataflow.std.file.IFileSplitProvider;
-import org.apache.hyracks.storage.am.common.dataflow.IndexDropOperatorDescriptor;
-import org.apache.hyracks.storage.am.lsm.btree.dataflow.LSMBTreeDataflowHelperFactory;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMMergePolicyFactory;
-
-public class IndexOperations {
-
-    private static final PhysicalOptimizationConfig physicalOptimizationConfig =
-            OptimizationConfUtil.getPhysicalOptimizationConfig();
-
-    public static JobSpecification buildSecondaryIndexCreationJobSpec(CompiledCreateIndexStatement createIndexStmt,
-            ARecordType recType, ARecordType metaType, List<Integer> keySourceIndicators, ARecordType enforcedType,
-            MetadataProvider metadataProvider) throws AsterixException, AlgebricksException {
-        SecondaryIndexOperationsHelper secondaryIndexHelper = SecondaryIndexOperationsHelper
-                .createIndexOperationsHelper(createIndexStmt.getIndexType(), createIndexStmt.getDataverseName(),
-                        createIndexStmt.getDatasetName(), createIndexStmt.getIndexName(),
-                        createIndexStmt.getKeyFields(), createIndexStmt.getKeyFieldTypes(),
-                        createIndexStmt.isEnforced(), createIndexStmt.getGramLength(), metadataProvider,
-                        physicalOptimizationConfig, recType, metaType, keySourceIndicators, enforcedType);
-        return secondaryIndexHelper.buildCreationJobSpec();
-    }
-
-    public static JobSpecification buildSecondaryIndexLoadingJobSpec(CompiledCreateIndexStatement createIndexStmt,
-            ARecordType recType, ARecordType metaType, List<Integer> keySourceIndicators, ARecordType enforcedType,
-            MetadataProvider metadataProvider) throws AsterixException, AlgebricksException {
-        SecondaryIndexOperationsHelper secondaryIndexHelper = SecondaryIndexOperationsHelper
-                .createIndexOperationsHelper(createIndexStmt.getIndexType(), createIndexStmt.getDataverseName(),
-                        createIndexStmt.getDatasetName(), createIndexStmt.getIndexName(),
-                        createIndexStmt.getKeyFields(), createIndexStmt.getKeyFieldTypes(),
-                        createIndexStmt.isEnforced(), createIndexStmt.getGramLength(), metadataProvider,
-                        physicalOptimizationConfig, recType, metaType, keySourceIndicators, enforcedType);
-        return secondaryIndexHelper.buildLoadingJobSpec();
-    }
-
-    public static JobSpecification buildSecondaryIndexLoadingJobSpec(CompiledCreateIndexStatement createIndexStmt,
-            ARecordType recType, ARecordType metaType, List<Integer> keySourceIndicators, ARecordType enforcedType,
-            MetadataProvider metadataProvider, List<ExternalFile> files)
-            throws AsterixException, AlgebricksException {
-        SecondaryIndexOperationsHelper secondaryIndexHelper = SecondaryIndexOperationsHelper
-                .createIndexOperationsHelper(createIndexStmt.getIndexType(), createIndexStmt.getDataverseName(),
-                        createIndexStmt.getDatasetName(), createIndexStmt.getIndexName(),
-                        createIndexStmt.getKeyFields(), createIndexStmt.getKeyFieldTypes(),
-                        createIndexStmt.isEnforced(), createIndexStmt.getGramLength(), metadataProvider,
-                        physicalOptimizationConfig, recType, metaType, keySourceIndicators, enforcedType);
-        secondaryIndexHelper.setExternalFiles(files);
-        return secondaryIndexHelper.buildLoadingJobSpec();
-    }
-
-    public static JobSpecification buildDropSecondaryIndexJobSpec(CompiledIndexDropStatement indexDropStmt,
-            MetadataProvider metadataProvider, Dataset dataset) throws AlgebricksException, MetadataException {
-        String dataverseName = indexDropStmt.getDataverseName() == null ? metadataProvider.getDefaultDataverseName()
-                : indexDropStmt.getDataverseName();
-        String datasetName = indexDropStmt.getDatasetName();
-        String indexName = indexDropStmt.getIndexName();
-        JobSpecification spec = JobSpecificationUtils.createJobSpecification();
-        boolean temp = dataset.getDatasetDetails().isTemp();
-
-        Pair<IFileSplitProvider, AlgebricksPartitionConstraint> splitsAndConstraint = metadataProvider
-                .splitProviderAndPartitionConstraintsForDataset(dataverseName, datasetName, indexName, temp);
-        StorageProperties storageProperties = AppContextInfo.INSTANCE.getStorageProperties();
-        Pair<ILSMMergePolicyFactory, Map<String, String>> compactionInfo =
-                DatasetUtils.getMergePolicyFactory(dataset, metadataProvider.getMetadataTxnContext());
-
-        // The index drop operation should be persistent regardless of temp datasets or permanent dataset.
-        IndexDropOperatorDescriptor btreeDrop = new IndexDropOperatorDescriptor(spec,
-                RuntimeComponentsProvider.RUNTIME_PROVIDER, RuntimeComponentsProvider.RUNTIME_PROVIDER,
-                splitsAndConstraint.first,
-                new LSMBTreeDataflowHelperFactory(new AsterixVirtualBufferCacheProvider(dataset.getDatasetId()),
-                        compactionInfo.first, compactionInfo.second,
-                        new SecondaryIndexOperationTrackerProvider(dataset.getDatasetId()),
-                        RuntimeComponentsProvider.RUNTIME_PROVIDER, LSMBTreeIOOperationCallbackFactory.INSTANCE,
-                        storageProperties.getBloomFilterFalsePositiveRate(), false, null, null, null, null, !temp),
-                LSMIndexUtil.getMetadataPageManagerFactory());
-        AlgebricksPartitionConstraintHelper.setPartitionConstraintInJobSpec(spec, btreeDrop,
-                splitsAndConstraint.second);
-        spec.addRoot(btreeDrop);
-
-        return spec;
-    }
-
-    public static JobSpecification buildSecondaryIndexCompactJobSpec(CompiledIndexCompactStatement indexCompactStmt,
-            ARecordType recType, ARecordType metaType, List<Integer> keySourceIndicators, ARecordType enforcedType,
-            MetadataProvider metadataProvider) throws AsterixException, AlgebricksException {
-        SecondaryIndexOperationsHelper secondaryIndexHelper = SecondaryIndexOperationsHelper
-                .createIndexOperationsHelper(indexCompactStmt.getIndexType(), indexCompactStmt.getDataverseName(),
-                        indexCompactStmt.getDatasetName(), indexCompactStmt.getIndexName(),
-                        indexCompactStmt.getKeyFields(), indexCompactStmt.getKeyTypes(), indexCompactStmt.isEnforced(),
-                        indexCompactStmt.getGramLength(), metadataProvider, physicalOptimizationConfig, recType,
-                        metaType, keySourceIndicators, enforcedType);
-        return secondaryIndexHelper.buildCompactJobSpec();
-    }
-}
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/file/JobSpecificationUtils.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/file/JobSpecificationUtils.java
deleted file mode 100644
index 68ab182..0000000
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/file/JobSpecificationUtils.java
+++ /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.
- */
-package org.apache.asterix.file;
-
-import org.apache.asterix.common.config.CompilerProperties;
-import org.apache.asterix.runtime.util.AppContextInfo;
-import org.apache.hyracks.api.job.JobSpecification;
-
-public class JobSpecificationUtils {
-    public static JobSpecification createJobSpecification() {
-        CompilerProperties compilerProperties = AppContextInfo.INSTANCE.getCompilerProperties();
-        int frameSize = compilerProperties.getFrameSize();
-        JobSpecification spec = new JobSpecification(frameSize);
-        return spec;
-    }
-}
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/file/StorageComponentProvider.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/file/StorageComponentProvider.java
new file mode 100644
index 0000000..d0d14c8
--- /dev/null
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/file/StorageComponentProvider.java
@@ -0,0 +1,78 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.file;
+
+import org.apache.asterix.common.context.IStorageComponentProvider;
+import org.apache.asterix.common.context.ITransactionSubsystemProvider;
+import org.apache.asterix.common.context.TransactionSubsystemProvider;
+import org.apache.asterix.dataflow.data.nontagged.valueproviders.PrimitiveValueProviderFactory;
+import org.apache.asterix.formats.nontagged.BinaryComparatorFactoryProvider;
+import org.apache.asterix.formats.nontagged.TypeTraitProvider;
+import org.apache.asterix.runtime.utils.RuntimeComponentsProvider;
+import org.apache.hyracks.algebricks.data.IBinaryComparatorFactoryProvider;
+import org.apache.hyracks.storage.am.common.api.IIndexLifecycleManagerProvider;
+import org.apache.hyracks.storage.am.common.api.IMetadataPageManagerFactory;
+import org.apache.hyracks.storage.am.common.api.IPrimitiveValueProviderFactory;
+import org.apache.hyracks.storage.am.common.freepage.AppendOnlyLinkedMetadataPageManagerFactory;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationSchedulerProvider;
+import org.apache.hyracks.storage.common.IStorageManager;
+
+public class StorageComponentProvider implements IStorageComponentProvider {
+
+    @Override
+    public IPrimitiveValueProviderFactory getPrimitiveValueProviderFactory() {
+        return PrimitiveValueProviderFactory.INSTANCE;
+    }
+
+    @Override
+    public ITransactionSubsystemProvider getTransactionSubsystemProvider() {
+        return TransactionSubsystemProvider.INSTANCE;
+    }
+
+    @Override
+    public ILSMIOOperationSchedulerProvider getIoOperationSchedulerProvider() {
+        return RuntimeComponentsProvider.RUNTIME_PROVIDER;
+    }
+
+    @Override
+    public IMetadataPageManagerFactory getMetadataPageManagerFactory() {
+        return AppendOnlyLinkedMetadataPageManagerFactory.INSTANCE;
+    }
+
+    @Override
+    public IBinaryComparatorFactoryProvider getComparatorFactoryProvider() {
+        return BinaryComparatorFactoryProvider.INSTANCE;
+    }
+
+    @Override
+    public TypeTraitProvider getTypeTraitProvider() {
+        return TypeTraitProvider.INSTANCE;
+    }
+
+    @Override
+    public IIndexLifecycleManagerProvider getIndexLifecycleManagerProvider() {
+        return RuntimeComponentsProvider.RUNTIME_PROVIDER;
+    }
+
+    @Override
+    public IStorageManager getStorageManager() {
+        return RuntimeComponentsProvider.RUNTIME_PROVIDER;
+    }
+
+}
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/hyracks/bootstrap/CCApplicationEntryPoint.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/hyracks/bootstrap/CCApplicationEntryPoint.java
index b17a722..7955ad6 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/hyracks/bootstrap/CCApplicationEntryPoint.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/hyracks/bootstrap/CCApplicationEntryPoint.java
@@ -29,13 +29,13 @@
 
 import org.apache.asterix.active.ActiveLifecycleListener;
 import org.apache.asterix.api.http.server.ApiServlet;
-import org.apache.asterix.api.http.server.FullApiServlet;
 import org.apache.asterix.api.http.server.ClusterApiServlet;
 import org.apache.asterix.api.http.server.ClusterControllerDetailsApiServlet;
 import org.apache.asterix.api.http.server.ConnectorApiServlet;
 import org.apache.asterix.api.http.server.DdlApiServlet;
 import org.apache.asterix.api.http.server.DiagnosticsApiServlet;
 import org.apache.asterix.api.http.server.FeedServlet;
+import org.apache.asterix.api.http.server.FullApiServlet;
 import org.apache.asterix.api.http.server.NodeControllerDetailsApiServlet;
 import org.apache.asterix.api.http.server.QueryApiServlet;
 import org.apache.asterix.api.http.server.QueryResultApiServlet;
@@ -46,23 +46,25 @@
 import org.apache.asterix.api.http.server.UpdateApiServlet;
 import org.apache.asterix.api.http.server.VersionApiServlet;
 import org.apache.asterix.api.http.servlet.ServletConstants;
-import org.apache.asterix.app.cc.CompilerExtensionManager;
+import org.apache.asterix.app.cc.CCExtensionManager;
 import org.apache.asterix.app.cc.ResourceIdManager;
 import org.apache.asterix.app.external.ExternalLibraryUtils;
 import org.apache.asterix.common.api.AsterixThreadFactory;
 import org.apache.asterix.common.config.AsterixExtension;
 import org.apache.asterix.common.config.ExternalProperties;
 import org.apache.asterix.common.config.MetadataProperties;
+import org.apache.asterix.common.context.IStorageComponentProvider;
 import org.apache.asterix.common.library.ILibraryManager;
 import org.apache.asterix.common.utils.LetUtil.Lets;
 import org.apache.asterix.external.library.ExternalLibraryManager;
+import org.apache.asterix.file.StorageComponentProvider;
 import org.apache.asterix.messaging.CCMessageBroker;
 import org.apache.asterix.metadata.MetadataManager;
 import org.apache.asterix.metadata.api.IAsterixStateProxy;
 import org.apache.asterix.metadata.bootstrap.AsterixStateProxy;
 import org.apache.asterix.metadata.cluster.ClusterManagerProvider;
 import org.apache.asterix.runtime.job.resource.JobCapacityController;
-import org.apache.asterix.runtime.util.AppContextInfo;
+import org.apache.asterix.runtime.utils.AppContextInfo;
 import org.apache.hyracks.api.application.ICCApplicationContext;
 import org.apache.hyracks.api.application.ICCApplicationEntryPoint;
 import org.apache.hyracks.api.client.HyracksConnection;
@@ -83,7 +85,8 @@
     private static final Logger LOGGER = Logger.getLogger(CCApplicationEntryPoint.class.getName());
     private static IAsterixStateProxy proxy;
     protected ICCApplicationContext appCtx;
-    protected CompilerExtensionManager ccExtensionManager;
+    protected CCExtensionManager ccExtensionManager;
+    protected IStorageComponentProvider componentProvider;
     private IJobCapacityController jobCapacityController;
     protected WebManager webManager;
 
@@ -98,17 +101,16 @@
         }
 
         appCtx.setThreadFactory(new AsterixThreadFactory(appCtx.getThreadFactory(), new LifeCycleComponentManager()));
-        GlobalRecoveryManager.instantiate((HyracksConnection) getNewHyracksClientConnection());
         ILibraryManager libraryManager = new ExternalLibraryManager();
         ResourceIdManager resourceIdManager = new ResourceIdManager();
         ExternalLibraryUtils.setUpExternaLibraries(libraryManager, false);
-        AppContextInfo.initialize(appCtx, getNewHyracksClientConnection(), GlobalRecoveryManager.instance(),
-                libraryManager, resourceIdManager, () -> MetadataManager.INSTANCE);
-        ccExtensionManager = new CompilerExtensionManager(getExtensions());
+        componentProvider = new StorageComponentProvider();
+        GlobalRecoveryManager.instantiate((HyracksConnection) getNewHyracksClientConnection(), componentProvider);
+        AppContextInfo.initialize(appCtx, getNewHyracksClientConnection(), libraryManager, resourceIdManager,
+                () -> MetadataManager.INSTANCE, GlobalRecoveryManager.instance());
+        ccExtensionManager = new CCExtensionManager(getExtensions());
         AppContextInfo.INSTANCE.setExtensionManager(ccExtensionManager);
-
         final CCConfig ccConfig = controllerService.getCCConfig();
-
         if (System.getProperty("java.rmi.server.hostname") == null) {
             System.setProperty("java.rmi.server.hostname", ccConfig.clusterNetIpAddress);
         }
@@ -119,8 +121,7 @@
 
         MetadataManager.initialize(proxy, metadataProperties);
 
-        AppContextInfo.INSTANCE.getCCApplicationContext()
-                .addJobLifecycleListener(ActiveLifecycleListener.INSTANCE);
+        AppContextInfo.INSTANCE.getCCApplicationContext().addJobLifecycleListener(ActiveLifecycleListener.INSTANCE);
 
         // create event loop groups
         webManager = new WebManager();
@@ -161,19 +162,19 @@
     }
 
     protected HttpServer setupWebServer(ExternalProperties externalProperties) throws Exception {
-        HttpServer webServer = new HttpServer(webManager.getBosses(), webManager.getWorkers(), externalProperties
-                .getWebInterfacePort());
+        HttpServer webServer = new HttpServer(webManager.getBosses(), webManager.getWorkers(),
+                externalProperties.getWebInterfacePort());
         IHyracksClientConnection hcc = getNewHyracksClientConnection();
         webServer.setAttribute(HYRACKS_CONNECTION_ATTR, hcc);
-        webServer.addLet(new ApiServlet(webServer.ctx(), new String[] { "/*" }, ccExtensionManager
-                .getAqlCompilationProvider(), ccExtensionManager.getSqlppCompilationProvider(), ccExtensionManager
-                        .getQueryTranslatorFactory()));
+        webServer.addLet(new ApiServlet(webServer.ctx(), new String[] { "/*" },
+                ccExtensionManager.getAqlCompilationProvider(), ccExtensionManager.getSqlppCompilationProvider(),
+                ccExtensionManager.getQueryTranslatorFactory(), componentProvider));
         return webServer;
     }
 
     protected HttpServer setupJSONAPIServer(ExternalProperties externalProperties) throws Exception {
-        HttpServer jsonAPIServer = new HttpServer(webManager.getBosses(), webManager.getWorkers(), externalProperties
-                .getAPIServerPort());
+        HttpServer jsonAPIServer =
+                new HttpServer(webManager.getBosses(), webManager.getWorkers(), externalProperties.getAPIServerPort());
         IHyracksClientConnection hcc = getNewHyracksClientConnection();
         jsonAPIServer.setAttribute(HYRACKS_CONNECTION_ATTR, hcc);
         jsonAPIServer.setAttribute(ASTERIX_BUILD_PROP_ATTR, AppContextInfo.INSTANCE);
@@ -211,8 +212,8 @@
     }
 
     protected HttpServer setupQueryWebServer(ExternalProperties externalProperties) throws Exception {
-        HttpServer queryWebServer = new HttpServer(webManager.getBosses(), webManager.getWorkers(), externalProperties
-                .getQueryWebInterfacePort());
+        HttpServer queryWebServer = new HttpServer(webManager.getBosses(), webManager.getWorkers(),
+                externalProperties.getQueryWebInterfacePort());
         IHyracksClientConnection hcc = getNewHyracksClientConnection();
         queryWebServer.setAttribute(HYRACKS_CONNECTION_ATTR, hcc);
         queryWebServer.addLet(new QueryWebInterfaceServlet(queryWebServer.ctx(), new String[] { "/*" }));
@@ -229,8 +230,8 @@
     }
 
     protected HttpServer setupFeedServer(ExternalProperties externalProperties) throws Exception {
-        HttpServer feedServer = new HttpServer(webManager.getBosses(), webManager.getWorkers(), externalProperties
-                .getFeedServerPort());
+        HttpServer feedServer = new HttpServer(webManager.getBosses(), webManager.getWorkers(),
+                externalProperties.getFeedServerPort());
         feedServer.setAttribute(HYRACKS_CONNECTION_ATTR, getNewHyracksClientConnection());
         feedServer.addLet(new FeedServlet(feedServer.ctx(), new String[] { "/" }));
         return feedServer;
@@ -240,35 +241,35 @@
         switch (key) {
             case AQL:
                 return new FullApiServlet(server.ctx(), paths, ccExtensionManager.getAqlCompilationProvider(),
-                        ccExtensionManager.getQueryTranslatorFactory());
+                        ccExtensionManager.getQueryTranslatorFactory(), componentProvider);
             case AQL_QUERY:
                 return new QueryApiServlet(server.ctx(), paths, ccExtensionManager.getAqlCompilationProvider(),
-                        ccExtensionManager.getQueryTranslatorFactory());
+                        ccExtensionManager.getQueryTranslatorFactory(), componentProvider);
             case AQL_UPDATE:
                 return new UpdateApiServlet(server.ctx(), paths, ccExtensionManager.getAqlCompilationProvider(),
-                        ccExtensionManager.getQueryTranslatorFactory());
+                        ccExtensionManager.getQueryTranslatorFactory(), componentProvider);
             case AQL_DDL:
                 return new DdlApiServlet(server.ctx(), paths, ccExtensionManager.getAqlCompilationProvider(),
-                        ccExtensionManager.getQueryTranslatorFactory());
+                        ccExtensionManager.getQueryTranslatorFactory(), componentProvider);
             case SQLPP:
                 return new FullApiServlet(server.ctx(), paths, ccExtensionManager.getSqlppCompilationProvider(),
-                        ccExtensionManager.getQueryTranslatorFactory());
+                        ccExtensionManager.getQueryTranslatorFactory(), componentProvider);
             case SQLPP_QUERY:
                 return new QueryApiServlet(server.ctx(), paths, ccExtensionManager.getSqlppCompilationProvider(),
-                        ccExtensionManager.getQueryTranslatorFactory());
+                        ccExtensionManager.getQueryTranslatorFactory(), componentProvider);
             case SQLPP_UPDATE:
                 return new UpdateApiServlet(server.ctx(), paths, ccExtensionManager.getSqlppCompilationProvider(),
-                        ccExtensionManager.getQueryTranslatorFactory());
+                        ccExtensionManager.getQueryTranslatorFactory(), componentProvider);
             case SQLPP_DDL:
                 return new DdlApiServlet(server.ctx(), paths, ccExtensionManager.getSqlppCompilationProvider(),
-                        ccExtensionManager.getQueryTranslatorFactory());
+                        ccExtensionManager.getQueryTranslatorFactory(), componentProvider);
             case QUERY_STATUS:
                 return new QueryStatusApiServlet(server.ctx(), paths);
             case QUERY_RESULT:
                 return new QueryResultApiServlet(server.ctx(), paths);
             case QUERY_SERVICE:
-                return new QueryServiceServlet(server.ctx(), paths, ccExtensionManager
-                        .getSqlppCompilationProvider(), ccExtensionManager.getQueryTranslatorFactory());
+                return new QueryServiceServlet(server.ctx(), paths, ccExtensionManager.getSqlppCompilationProvider(),
+                        ccExtensionManager.getQueryTranslatorFactory(), componentProvider);
             case CONNECTOR:
                 return new ConnectorApiServlet(server.ctx(), paths);
             case SHUTDOWN:
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/hyracks/bootstrap/ClusterLifecycleListener.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/hyracks/bootstrap/ClusterLifecycleListener.java
index 41d8b0d..53b577a 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/hyracks/bootstrap/ClusterLifecycleListener.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/hyracks/bootstrap/ClusterLifecycleListener.java
@@ -41,7 +41,7 @@
 import org.apache.asterix.metadata.cluster.ClusterManagerProvider;
 import org.apache.asterix.metadata.cluster.RemoveNodeWork;
 import org.apache.asterix.metadata.cluster.RemoveNodeWorkResponse;
-import org.apache.asterix.runtime.util.ClusterStateManager;
+import org.apache.asterix.runtime.utils.ClusterStateManager;
 import org.apache.hyracks.api.application.IClusterLifecycleListener;
 import org.apache.hyracks.api.exceptions.HyracksException;
 
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/hyracks/bootstrap/ClusterWorkExecutor.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/hyracks/bootstrap/ClusterWorkExecutor.java
index b1d8dd3..71fff3f 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/hyracks/bootstrap/ClusterWorkExecutor.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/hyracks/bootstrap/ClusterWorkExecutor.java
@@ -30,7 +30,7 @@
 import org.apache.asterix.metadata.cluster.AddNodeWork;
 import org.apache.asterix.metadata.cluster.ClusterManagerProvider;
 import org.apache.asterix.metadata.cluster.RemoveNodeWork;
-import org.apache.asterix.runtime.util.ClusterStateManager;
+import org.apache.asterix.runtime.utils.ClusterStateManager;
 
 public class ClusterWorkExecutor implements Runnable {
 
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 d437b5b..002e270 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
@@ -25,14 +25,14 @@
 import java.util.logging.Level;
 import java.util.logging.Logger;
 
-import org.apache.asterix.app.external.ExternalIndexingOperations;
 import org.apache.asterix.common.api.IClusterManagementWork;
 import org.apache.asterix.common.api.IClusterManagementWork.ClusterState;
 import org.apache.asterix.common.api.IClusterManagementWorkResponse;
-import org.apache.asterix.common.cluster.IGlobalRecoveryMaanger;
+import org.apache.asterix.common.cluster.IGlobalRecoveryManager;
 import org.apache.asterix.common.config.DatasetConfig.DatasetType;
-import org.apache.asterix.common.config.DatasetConfig.ExternalDatasetTransactionState;
 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.indexing.ExternalFile;
 import org.apache.asterix.metadata.MetadataManager;
 import org.apache.asterix.metadata.MetadataTransactionContext;
@@ -41,22 +41,25 @@
 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.ExternalIndexingOperations;
 import org.apache.asterix.metadata.utils.MetadataConstants;
-import org.apache.asterix.runtime.util.ClusterStateManager;
+import org.apache.asterix.runtime.utils.ClusterStateManager;
 import org.apache.hyracks.api.client.HyracksConnection;
 import org.apache.hyracks.api.job.JobId;
 import org.apache.hyracks.api.job.JobSpecification;
 
-public class GlobalRecoveryManager implements IGlobalRecoveryMaanger {
+public class GlobalRecoveryManager implements IGlobalRecoveryManager {
 
     private static final Logger LOGGER = Logger.getLogger(GlobalRecoveryManager.class.getName());
     private static GlobalRecoveryManager instance;
     private static ClusterState state;
+    private final IStorageComponentProvider componentProvider;
     private HyracksConnection hcc;
 
-    private GlobalRecoveryManager(HyracksConnection hcc) {
+    private GlobalRecoveryManager(HyracksConnection hcc, IStorageComponentProvider componentProvider) {
         setState(ClusterState.UNUSABLE);
         this.hcc = hcc;
+        this.componentProvider = componentProvider;
     }
 
     @Override
@@ -107,7 +110,7 @@
                         List<Dataverse> dataverses = MetadataManager.INSTANCE.getDataverses(mdTxnCtx);
                         for (Dataverse dataverse : dataverses) {
                             if (!dataverse.getDataverseName().equals(MetadataConstants.METADATA_DATAVERSE_NAME)) {
-                                MetadataProvider metadataProvider = new MetadataProvider(dataverse);
+                                MetadataProvider metadataProvider = new MetadataProvider(dataverse, componentProvider);
                                 List<Dataset> datasets = MetadataManager.INSTANCE.getDataverseDatasets(mdTxnCtx,
                                         dataverse.getDataverseName());
                                 for (Dataset dataset : datasets) {
@@ -118,16 +121,16 @@
                                                 dataset.getDataverseName(), dataset.getDatasetName());
                                         if (!indexes.isEmpty()) {
                                             // Get the state of the dataset
-                                            ExternalDatasetDetails dsd = (ExternalDatasetDetails) dataset
-                                                    .getDatasetDetails();
-                                            ExternalDatasetTransactionState datasetState = dsd.getState();
-                                            if (datasetState == ExternalDatasetTransactionState.BEGIN) {
+                                            ExternalDatasetDetails dsd =
+                                                    (ExternalDatasetDetails) dataset.getDatasetDetails();
+                                            TransactionState datasetState = dsd.getState();
+                                            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.PENDING_NO_OP) {
+                                                    if (file.getPendingOp() != ExternalFilePendingOp.NO_OP) {
                                                         MetadataManager.INSTANCE.dropExternalFile(mdTxnCtx, file);
                                                     }
                                                 }
@@ -138,11 +141,11 @@
                                                 executeHyracksJob(jobSpec);
                                                 // 3. correct the dataset state
                                                 ((ExternalDatasetDetails) dataset.getDatasetDetails())
-                                                        .setState(ExternalDatasetTransactionState.COMMIT);
+                                                        .setState(TransactionState.COMMIT);
                                                 MetadataManager.INSTANCE.updateDataset(mdTxnCtx, dataset);
                                                 MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
                                                 mdTxnCtx = MetadataManager.INSTANCE.beginTransaction();
-                                            } else if (datasetState == ExternalDatasetTransactionState.READY_TO_COMMIT) {
+                                            } else if (datasetState == TransactionState.READY_TO_COMMIT) {
                                                 List<ExternalFile> files = MetadataManager.INSTANCE
                                                         .getDatasetExternalFiles(mdTxnCtx, dataset);
                                                 // if ready to commit, roll forward
@@ -153,15 +156,15 @@
                                                 executeHyracksJob(jobSpec);
                                                 // 2. add pending files in metadata
                                                 for (ExternalFile file : files) {
-                                                    if (file.getPendingOp() == ExternalFilePendingOp.PENDING_ADD_OP) {
+                                                    if (file.getPendingOp() == ExternalFilePendingOp.ADD_OP) {
                                                         MetadataManager.INSTANCE.dropExternalFile(mdTxnCtx, file);
-                                                        file.setPendingOp(ExternalFilePendingOp.PENDING_NO_OP);
+                                                        file.setPendingOp(ExternalFilePendingOp.NO_OP);
                                                         MetadataManager.INSTANCE.addExternalFile(mdTxnCtx, file);
-                                                    } else if (file
-                                                            .getPendingOp() == ExternalFilePendingOp.PENDING_DROP_OP) {
+                                                    } else if (file.getPendingOp() == ExternalFilePendingOp.DROP_OP) {
                                                         // find original file
                                                         for (ExternalFile originalFile : files) {
-                                                            if (originalFile.getFileName().equals(file.getFileName())) {
+                                                            if (originalFile.getFileName()
+                                                                    .equals(file.getFileName())) {
                                                                 MetadataManager.INSTANCE.dropExternalFile(mdTxnCtx,
                                                                         file);
                                                                 MetadataManager.INSTANCE.dropExternalFile(mdTxnCtx,
@@ -170,10 +173,11 @@
                                                             }
                                                         }
                                                     } else if (file
-                                                            .getPendingOp() == ExternalFilePendingOp.PENDING_APPEND_OP) {
+                                                            .getPendingOp() == ExternalFilePendingOp.APPEND_OP) {
                                                         // find original file
                                                         for (ExternalFile originalFile : files) {
-                                                            if (originalFile.getFileName().equals(file.getFileName())) {
+                                                            if (originalFile.getFileName()
+                                                                    .equals(file.getFileName())) {
                                                                 MetadataManager.INSTANCE.dropExternalFile(mdTxnCtx,
                                                                         file);
                                                                 MetadataManager.INSTANCE.dropExternalFile(mdTxnCtx,
@@ -187,7 +191,7 @@
                                                 }
                                                 // 3. correct the dataset state
                                                 ((ExternalDatasetDetails) dataset.getDatasetDetails())
-                                                        .setState(ExternalDatasetTransactionState.COMMIT);
+                                                        .setState(TransactionState.COMMIT);
                                                 MetadataManager.INSTANCE.updateDataset(mdTxnCtx, dataset);
                                                 MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
                                                 mdTxnCtx = MetadataManager.INSTANCE.beginTransaction();
@@ -225,8 +229,8 @@
         return instance;
     }
 
-    public static synchronized void instantiate(HyracksConnection hcc) {
-        instance = new GlobalRecoveryManager(hcc);
+    public static synchronized void instantiate(HyracksConnection hcc, IStorageComponentProvider componentProvider) {
+        instance = new GlobalRecoveryManager(hcc, componentProvider);
     }
 
     public static synchronized void setState(ClusterState state) {
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/file/DataverseOperations.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/utils/DataverseUtil.java
similarity index 73%
rename from asterixdb/asterix-app/src/main/java/org/apache/asterix/file/DataverseOperations.java
rename to asterixdb/asterix-app/src/main/java/org/apache/asterix/utils/DataverseUtil.java
index 4f9af13..48fd782 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/file/DataverseOperations.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/utils/DataverseUtil.java
@@ -16,10 +16,11 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.asterix.file;
+package org.apache.asterix.utils;
 
 import org.apache.asterix.metadata.declared.MetadataProvider;
 import org.apache.asterix.metadata.entities.Dataverse;
+import org.apache.asterix.runtime.utils.RuntimeUtils;
 import org.apache.hyracks.algebricks.common.constraints.AlgebricksPartitionConstraint;
 import org.apache.hyracks.algebricks.common.constraints.AlgebricksPartitionConstraintHelper;
 import org.apache.hyracks.algebricks.common.utils.Pair;
@@ -27,12 +28,17 @@
 import org.apache.hyracks.dataflow.std.file.FileRemoveOperatorDescriptor;
 import org.apache.hyracks.dataflow.std.file.IFileSplitProvider;
 
-public class DataverseOperations {
-    public static JobSpecification createDropDataverseJobSpec(Dataverse dataverse, MetadataProvider metadata) {
-        JobSpecification jobSpec = JobSpecificationUtils.createJobSpecification();
-        Pair<IFileSplitProvider, AlgebricksPartitionConstraint> splitsAndConstraint = metadata
-                .splitProviderAndPartitionConstraintsForDataverse(dataverse.getDataverseName());
-        FileRemoveOperatorDescriptor frod = new FileRemoveOperatorDescriptor(jobSpec, splitsAndConstraint.first, false);
+public class DataverseUtil {
+
+    private DataverseUtil() {
+    }
+
+    public static JobSpecification dropDataverseJobSpec(Dataverse dataverse, MetadataProvider metadata) {
+        JobSpecification jobSpec = RuntimeUtils.createJobSpecification();
+        Pair<IFileSplitProvider, AlgebricksPartitionConstraint> splitsAndConstraint =
+                metadata.splitAndConstraints(dataverse.getDataverseName());
+        FileRemoveOperatorDescriptor frod =
+                new FileRemoveOperatorDescriptor(jobSpec, splitsAndConstraint.first, false);
         AlgebricksPartitionConstraintHelper.setPartitionConstraintInJobSpec(jobSpec, frod, splitsAndConstraint.second);
         jobSpec.addRoot(frod);
         return jobSpec;
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/utils/ExtensionUtil.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/utils/ExtensionUtil.java
new file mode 100644
index 0000000..07eed0d
--- /dev/null
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/utils/ExtensionUtil.java
@@ -0,0 +1,102 @@
+/*
+ * 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.utils;
+
+import org.apache.asterix.algebra.base.ILangExtension;
+import org.apache.asterix.algebra.base.ILangExtension.Language;
+import org.apache.asterix.app.cc.IStatementExecutorExtension;
+import org.apache.asterix.common.api.ExtensionId;
+import org.apache.asterix.common.exceptions.ErrorCode;
+import org.apache.asterix.common.exceptions.RuntimeDataException;
+import org.apache.asterix.compiler.provider.ILangCompilationProvider;
+import org.apache.asterix.metadata.api.IMetadataExtension;
+import org.apache.asterix.translator.IStatementExecutorFactory;
+import org.apache.hyracks.algebricks.common.utils.Pair;
+
+/**
+ * Provide util methods dealing with extensions
+ */
+public class ExtensionUtil {
+
+    private ExtensionUtil() {
+    }
+
+    /**
+     * Verifies no conflict and return the language compilation provider
+     *
+     * @param lang
+     *            the language for the passed compilation provider
+     * @param cp
+     *            placeholder for compilation provider
+     * @param le
+     *            user defined extension for compilation provider
+     * @return a pair of extension id and extended compilation provider
+     * @throws RuntimeDataException
+     *             if there was a conflict between two extensions
+     */
+    public static Pair<ExtensionId, ILangCompilationProvider> extendLangCompilationProvider(Language lang,
+            Pair<ExtensionId, ILangCompilationProvider> cp, ILangExtension le) throws RuntimeDataException {
+        if (cp != null && le.getLangCompilationProvider(lang) != null) {
+            throw new RuntimeDataException(ErrorCode.EXTENSION_COMPONENT_CONFLICT, le.getId(), cp.first,
+                    lang.toString());
+        }
+        return (le.getLangCompilationProvider(lang) != null)
+                ? new Pair<>(le.getId(), le.getLangCompilationProvider(lang)) : cp;
+    }
+
+    /**
+     * Validate no extension conflict and return statement executor extension
+     *
+     * @param qte
+     *            place holder for statement executor extension
+     * @param extension
+     *            user defined extension
+     * @return the user defined extension
+     * @throws RuntimeDataException
+     *             if extension conflict was detected
+     */
+    public static IStatementExecutorExtension extendStatementExecutor(IStatementExecutorExtension qte,
+            IStatementExecutorExtension extension) throws RuntimeDataException {
+        if (qte != null) {
+            throw new RuntimeDataException(ErrorCode.EXTENSION_COMPONENT_CONFLICT, qte.getId(), extension.getId(),
+                    IStatementExecutorFactory.class.getSimpleName());
+        }
+        return extension;
+    }
+
+    /**
+     * Validate no extension conflict and extends tuple translator provider
+     *
+     * @param metadataExtension
+     *            place holder for tuple translator provider extension
+     * @param mde
+     *            user defined metadata extension
+     * @return the metadata extension if the extension defines a metadata tuple translator, null otherwise
+     * @throws RuntimeDataException
+     *             if an extension conflict was detected
+     */
+    public static IMetadataExtension extendTupleTranslatorProvider(IMetadataExtension metadataExtension,
+            IMetadataExtension mde) throws RuntimeDataException {
+        if (metadataExtension != null) {
+            throw new RuntimeDataException(ErrorCode.EXTENSION_COMPONENT_CONFLICT, metadataExtension.getId(),
+                    mde.getId(), IMetadataExtension.class.getSimpleName());
+        }
+        return mde.getMetadataTupleTranslatorProvider() == null ? null : mde;
+    }
+}
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/util/FlushDatasetUtils.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/utils/FlushDatasetUtil.java
similarity index 85%
rename from asterixdb/asterix-app/src/main/java/org/apache/asterix/util/FlushDatasetUtils.java
rename to asterixdb/asterix-app/src/main/java/org/apache/asterix/utils/FlushDatasetUtil.java
index 5a3419a..bc8a79e 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/util/FlushDatasetUtils.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/utils/FlushDatasetUtil.java
@@ -17,15 +17,15 @@
  * under the License.
  */
 
-package org.apache.asterix.util;
+package org.apache.asterix.utils;
 
 import org.apache.asterix.common.config.CompilerProperties;
-import org.apache.asterix.metadata.MetadataTransactionContext;
+import org.apache.asterix.common.utils.JobUtils;
 import org.apache.asterix.metadata.declared.MetadataProvider;
 import org.apache.asterix.metadata.entities.Dataset;
 import org.apache.asterix.runtime.job.listener.JobEventListenerFactory;
 import org.apache.asterix.runtime.operators.std.FlushDatasetOperatorDescriptor;
-import org.apache.asterix.runtime.util.AppContextInfo;
+import org.apache.asterix.runtime.utils.AppContextInfo;
 import org.apache.asterix.transaction.management.service.transaction.JobIdFactory;
 import org.apache.hyracks.algebricks.common.constraints.AlgebricksPartitionConstraint;
 import org.apache.hyracks.algebricks.common.constraints.AlgebricksPartitionConstraintHelper;
@@ -40,11 +40,12 @@
 import org.apache.hyracks.dataflow.std.connectors.OneToOneConnectorDescriptor;
 import org.apache.hyracks.dataflow.std.file.IFileSplitProvider;
 
-public class FlushDatasetUtils {
+public class FlushDatasetUtil {
+    private FlushDatasetUtil() {
+    }
 
     public static void flushDataset(IHyracksClientConnection hcc, MetadataProvider metadataProvider,
-            MetadataTransactionContext mdTxnCtx, String dataverseName, String datasetName, String indexName)
-            throws Exception {
+            String dataverseName, String datasetName, String indexName) throws Exception {
         CompilerProperties compilerProperties = AppContextInfo.INSTANCE.getCompilerProperties();
         int frameSize = compilerProperties.getFrameSize();
         JobSpecification spec = new JobSpecification(frameSize);
@@ -55,13 +56,13 @@
 
         org.apache.asterix.common.transactions.JobId jobId = JobIdFactory.generateJobId();
         Dataset dataset = metadataProvider.findDataset(dataverseName, datasetName);
-        FlushDatasetOperatorDescriptor flushOperator = new FlushDatasetOperatorDescriptor(spec, jobId,
-                dataset.getDatasetId());
+        FlushDatasetOperatorDescriptor flushOperator =
+                new FlushDatasetOperatorDescriptor(spec, jobId, dataset.getDatasetId());
 
         spec.connect(new OneToOneConnectorDescriptor(spec), emptySource, 0, flushOperator, 0);
 
-        Pair<IFileSplitProvider, AlgebricksPartitionConstraint> primarySplitsAndConstraint = metadataProvider
-                .splitProviderAndPartitionConstraintsForDataset(dataverseName, datasetName, indexName,
+        Pair<IFileSplitProvider, AlgebricksPartitionConstraint> primarySplitsAndConstraint =
+                metadataProvider.getSplitProviderAndConstraints(dataverseName, datasetName, indexName,
                         dataset.getDatasetDetails().isTemp());
         AlgebricksPartitionConstraint primaryPartitionConstraint = primarySplitsAndConstraint.second;
 
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/util/ResourceUtils.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/utils/ResourceUtils.java
similarity index 98%
rename from asterixdb/asterix-app/src/main/java/org/apache/asterix/util/ResourceUtils.java
rename to asterixdb/asterix-app/src/main/java/org/apache/asterix/utils/ResourceUtils.java
index 50a21bc..61c1dfe 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/util/ResourceUtils.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/utils/ResourceUtils.java
@@ -17,7 +17,7 @@
  * under the License.
  */
 
-package org.apache.asterix.util;
+package org.apache.asterix.utils;
 
 import org.apache.asterix.app.resource.RequiredCapacityVisitor;
 import org.apache.hyracks.algebricks.common.constraints.AlgebricksAbsolutePartitionConstraint;
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/api/http/servlet/ConnectorApiLetTest.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/api/http/servlet/ConnectorApiLetTest.java
index d34a9cf..d9a0a79 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/api/http/servlet/ConnectorApiLetTest.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/api/http/servlet/ConnectorApiLetTest.java
@@ -29,6 +29,7 @@
 import java.util.concurrent.ConcurrentHashMap;
 
 import org.apache.asterix.api.http.server.ConnectorApiServlet;
+import org.apache.asterix.file.StorageComponentProvider;
 import org.apache.asterix.metadata.MetadataManager;
 import org.apache.asterix.metadata.MetadataTransactionContext;
 import org.apache.asterix.metadata.declared.MetadataProvider;
@@ -36,7 +37,7 @@
 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.om.util.JSONDeserializerForTypes;
+import org.apache.asterix.om.utils.JSONDeserializerForTypes;
 import org.apache.asterix.test.runtime.SqlppExecutionTest;
 import org.apache.hyracks.api.client.IHyracksClientConnection;
 import org.apache.hyracks.api.client.NodeControllerInfo;
@@ -103,8 +104,7 @@
         Assert.assertFalse(temp);
         String primaryKey = actualResponse.get("keys").asText();
         Assert.assertEquals("DataverseName,DatasetName", primaryKey);
-        ARecordType recordType = (ARecordType) JSONDeserializerForTypes
-                .convertFromJSON(actualResponse.get("type"));
+        ARecordType recordType = (ARecordType) JSONDeserializerForTypes.convertFromJSON(actualResponse.get("type"));
         Assert.assertEquals(getMetadataRecordType("Metadata", "Dataset"), recordType);
 
         // Checks the correctness of results.
@@ -140,9 +140,11 @@
         // Calls ConnectorAPIServlet.formResponseObject.
         nodeMap.put("asterix_nc1", mockInfo1);
         nodeMap.put("asterix_nc2", mockInfo2);
-        PA.invokeMethod(let, "formResponseObject(" + ObjectNode.class.getName() + ", " + FileSplit.class.getName()
-                + "[], " + ARecordType.class.getName() + ", " + String.class.getName() + ", boolean, " + Map.class
-                        .getName() + ")", actualResponse, splits, recordType, primaryKey, true, nodeMap);
+        PA.invokeMethod(let,
+                "formResponseObject(" + ObjectNode.class.getName() + ", " + FileSplit.class.getName() + "[], "
+                        + ARecordType.class.getName() + ", " + String.class.getName() + ", boolean, "
+                        + Map.class.getName() + ")",
+                actualResponse, splits, recordType, primaryKey, true, nodeMap);
         // Constructs expected response.
         ObjectNode expectedResponse = om.createObjectNode();
         expectedResponse.put("temp", true);
@@ -166,11 +168,11 @@
     private ARecordType getMetadataRecordType(String dataverseName, String datasetName) throws Exception {
         MetadataTransactionContext mdTxnCtx = MetadataManager.INSTANCE.beginTransaction();
         // Retrieves file splits of the dataset.
-        MetadataProvider metadataProvider = new MetadataProvider(null);
+        MetadataProvider metadataProvider = new MetadataProvider(null, new StorageComponentProvider());
         metadataProvider.setMetadataTxnContext(mdTxnCtx);
         Dataset dataset = metadataProvider.findDataset(dataverseName, datasetName);
-        ARecordType recordType = (ARecordType) metadataProvider.findType(dataset.getItemTypeDataverseName(),
-                dataset.getItemTypeName());
+        ARecordType recordType =
+                (ARecordType) metadataProvider.findType(dataset.getItemTypeDataverseName(), dataset.getItemTypeName());
         // Metadata transaction commits.
         MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
         return recordType;
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/api/http/servlet/QueryServiceLetTest.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/api/http/servlet/QueryServiceServletTest.java
similarity index 98%
rename from asterixdb/asterix-app/src/test/java/org/apache/asterix/api/http/servlet/QueryServiceLetTest.java
rename to asterixdb/asterix-app/src/test/java/org/apache/asterix/api/http/servlet/QueryServiceServletTest.java
index 7b7c0d2..e0539ac 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/api/http/servlet/QueryServiceLetTest.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/api/http/servlet/QueryServiceServletTest.java
@@ -23,7 +23,7 @@
 import org.junit.Assert;
 import org.junit.Test;
 
-public class QueryServiceLetTest {
+public class QueryServiceServletTest {
 
     @Test
     public void testTimeUnitFormatNanos() throws Exception {
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/api/http/servlet/VersionApiLetTest.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/api/http/servlet/VersionApiLetTest.java
index b482948..619e7a5 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/api/http/servlet/VersionApiLetTest.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/api/http/servlet/VersionApiLetTest.java
@@ -32,7 +32,7 @@
 
 import org.apache.asterix.api.http.server.VersionApiServlet;
 import org.apache.asterix.common.config.BuildProperties;
-import org.apache.asterix.runtime.util.AppContextInfo;
+import org.apache.asterix.runtime.utils.AppContextInfo;
 import org.apache.asterix.test.runtime.SqlppExecutionTest;
 import org.apache.hyracks.api.client.IHyracksClientConnection;
 import org.apache.hyracks.http.api.IServletRequest;
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/app/bootstrap/TestNodeController.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/app/bootstrap/TestNodeController.java
index beadc66..bc18045 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/app/bootstrap/TestNodeController.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/app/bootstrap/TestNodeController.java
@@ -19,39 +19,45 @@
 package org.apache.asterix.app.bootstrap;
 
 import java.io.File;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
 import java.util.Map;
 import java.util.logging.Logger;
 
 import org.apache.asterix.algebra.operators.physical.CommitRuntime;
 import org.apache.asterix.app.external.TestLibrarian;
 import org.apache.asterix.app.nc.NCAppRuntimeContext;
+import org.apache.asterix.app.nc.TransactionSubsystem;
+import org.apache.asterix.common.config.DatasetConfig.IndexType;
 import org.apache.asterix.common.config.TransactionProperties;
-import org.apache.asterix.common.context.AsterixVirtualBufferCacheProvider;
 import org.apache.asterix.common.context.DatasetLifecycleManager;
+import org.apache.asterix.common.context.IStorageComponentProvider;
 import org.apache.asterix.common.context.TransactionSubsystemProvider;
-import org.apache.asterix.common.dataflow.LSMIndexUtil;
 import org.apache.asterix.common.dataflow.LSMInsertDeleteOperatorNodePushable;
 import org.apache.asterix.common.dataflow.LSMTreeInsertDeleteOperatorDescriptor;
-import org.apache.asterix.common.ioopcallbacks.LSMBTreeIOOperationCallbackFactory;
 import org.apache.asterix.common.transactions.IRecoveryManager.ResourceType;
 import org.apache.asterix.common.transactions.IResourceFactory;
 import org.apache.asterix.common.transactions.ITransactionManager;
+import org.apache.asterix.file.StorageComponentProvider;
 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.metadata.declared.MetadataProvider;
 import org.apache.asterix.metadata.entities.Dataset;
-import org.apache.asterix.metadata.utils.DatasetUtils;
+import org.apache.asterix.metadata.entities.Dataverse;
+import org.apache.asterix.metadata.entities.Index;
+import org.apache.asterix.metadata.utils.DatasetUtil;
+import org.apache.asterix.metadata.utils.MetadataUtil;
 import org.apache.asterix.om.types.ARecordType;
 import org.apache.asterix.om.types.IAType;
 import org.apache.asterix.runtime.formats.NonTaggedDataFormat;
-import org.apache.asterix.runtime.util.RuntimeComponentsProvider;
+import org.apache.asterix.runtime.utils.RuntimeComponentsProvider;
 import org.apache.asterix.test.runtime.ExecutionTestUtil;
 import org.apache.asterix.transaction.management.opcallbacks.PrimaryIndexModificationOperationCallbackFactory;
-import org.apache.asterix.transaction.management.opcallbacks.PrimaryIndexOperationTrackerProvider;
 import org.apache.asterix.transaction.management.resource.LSMBTreeLocalResourceMetadataFactory;
 import org.apache.asterix.transaction.management.resource.PersistentLocalResourceFactoryProvider;
 import org.apache.asterix.transaction.management.service.logging.LogReader;
-import org.apache.asterix.transaction.management.service.transaction.TransactionSubsystem;
 import org.apache.commons.lang3.StringUtils;
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
 import org.apache.hyracks.algebricks.runtime.base.IPushRuntime;
@@ -71,18 +77,18 @@
 import org.apache.hyracks.api.job.JobId;
 import org.apache.hyracks.api.job.JobSpecification;
 import org.apache.hyracks.api.util.HyracksConstants;
-import org.apache.hyracks.dataflow.common.util.TaskUtils;
+import org.apache.hyracks.dataflow.common.utils.TaskUtil;
 import org.apache.hyracks.dataflow.std.file.ConstantFileSplitProvider;
 import org.apache.hyracks.storage.am.btree.dataflow.BTreeSearchOperatorDescriptor;
 import org.apache.hyracks.storage.am.btree.dataflow.BTreeSearchOperatorNodePushable;
+import org.apache.hyracks.storage.am.common.api.IIndexDataflowHelper;
 import org.apache.hyracks.storage.am.common.api.IModificationOperationCallbackFactory;
-import org.apache.hyracks.storage.am.common.dataflow.IndexDataflowHelper;
+import org.apache.hyracks.storage.am.common.dataflow.IIndexDataflowHelperFactory;
 import org.apache.hyracks.storage.am.common.dataflow.TreeIndexCreateOperatorDescriptor;
 import org.apache.hyracks.storage.am.common.impls.NoOpOperationCallbackFactory;
 import org.apache.hyracks.storage.am.common.ophelpers.IndexOperation;
-import org.apache.hyracks.storage.am.lsm.btree.dataflow.LSMBTreeDataflowHelper;
-import org.apache.hyracks.storage.am.lsm.btree.dataflow.LSMBTreeDataflowHelperFactory;
 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.impls.NoMergePolicyFactory;
 import org.apache.hyracks.storage.common.file.ILocalResourceFactoryProvider;
 import org.apache.hyracks.storage.common.file.LocalResource;
@@ -95,8 +101,8 @@
     protected static final Logger LOGGER = Logger.getLogger(TestNodeController.class.getName());
 
     protected static final String PATH_ACTUAL = "unittest" + File.separator;
-    protected static final String PATH_BASE = StringUtils.join(new String[] { "src", "test", "resources", "nodetests" },
-            File.separator);
+    protected static final String PATH_BASE =
+            StringUtils.join(new String[] { "src", "test", "resources", "nodetests" }, File.separator);
 
     protected static final String TEST_CONFIG_FILE_NAME = "asterix-build-configuration.xml";
     protected static TransactionProperties txnProperties;
@@ -109,7 +115,7 @@
     public static final int KB32 = 32768;
     public static final int PARTITION = 0;
     public static final double BLOOM_FILTER_FALSE_POSITIVE_RATE = 0.01;
-    public static final TransactionSubsystemProvider TXN_SUBSYSTEM_PROVIDER = new TransactionSubsystemProvider();
+    public static final TransactionSubsystemProvider TXN_SUBSYSTEM_PROVIDER = TransactionSubsystemProvider.INSTANCE;
     // Mutables
     private JobId jobId;
     private long jobCounter = 0L;
@@ -157,23 +163,24 @@
 
     public LSMInsertDeleteOperatorNodePushable getInsertPipeline(IHyracksTaskContext ctx, Dataset dataset,
             IAType[] primaryKeyTypes, ARecordType recordType, ARecordType metaType,
-            ILSMMergePolicyFactory mergePolicyFactory, Map<String, String> mergePolicyProperties, int[] filterFields)
-            throws AlgebricksException, HyracksDataException {
+            ILSMMergePolicyFactory mergePolicyFactory, Map<String, String> mergePolicyProperties, int[] filterFields,
+            int[] primaryKeyIndexes, List<Integer> primaryKeyIndicators,
+            StorageComponentProvider storageComponentProvider) throws AlgebricksException, HyracksDataException {
         PrimaryIndexInfo primaryIndexInfo = new PrimaryIndexInfo(dataset, primaryKeyTypes, recordType, metaType,
-                mergePolicyFactory, mergePolicyProperties, filterFields);
+                mergePolicyFactory, mergePolicyProperties, filterFields, primaryKeyIndexes, primaryKeyIndicators,
+                storageComponentProvider);
         IndexOperation op = IndexOperation.INSERT;
-        IModificationOperationCallbackFactory modOpCallbackFactory = new PrimaryIndexModificationOperationCallbackFactory(
-                getTxnJobId(), dataset.getDatasetId(), primaryIndexInfo.primaryKeyIndexes, TXN_SUBSYSTEM_PROVIDER, op,
-                ResourceType.LSM_BTREE, true);
-        LSMTreeInsertDeleteOperatorDescriptor indexOpDesc = getInsertOpratorDesc(primaryIndexInfo,
-                modOpCallbackFactory);
-        LSMBTreeDataflowHelperFactory dataflowHelperFactory = getPrimaryIndexDataflowHelperFactory(ctx,
-                primaryIndexInfo);
+        IModificationOperationCallbackFactory modOpCallbackFactory =
+                new PrimaryIndexModificationOperationCallbackFactory(getTxnJobId(), dataset.getDatasetId(),
+                        primaryIndexInfo.primaryKeyIndexes, TXN_SUBSYSTEM_PROVIDER, op, ResourceType.LSM_BTREE, true);
+        LSMTreeInsertDeleteOperatorDescriptor indexOpDesc =
+                getInsertOpratorDesc(primaryIndexInfo, modOpCallbackFactory);
+        IIndexDataflowHelperFactory dataflowHelperFactory =
+                getPrimaryIndexDataflowHelperFactory(ctx, primaryIndexInfo, storageComponentProvider, dataset);
         Mockito.when(indexOpDesc.getIndexDataflowHelperFactory()).thenReturn(dataflowHelperFactory);
         IRecordDescriptorProvider recordDescProvider = primaryIndexInfo.getInsertRecordDescriptorProvider();
-        LSMInsertDeleteOperatorNodePushable insertOp = new LSMInsertDeleteOperatorNodePushable(
-                indexOpDesc, ctx, PARTITION, primaryIndexInfo.primaryIndexInsertFieldsPermutations, recordDescProvider,
-                op, true);
+        LSMInsertDeleteOperatorNodePushable insertOp = new LSMInsertDeleteOperatorNodePushable(indexOpDesc, ctx,
+                PARTITION, primaryIndexInfo.primaryIndexInsertFieldsPermutations, recordDescProvider, op, true);
         CommitRuntime commitOp = new CommitRuntime(ctx, getTxnJobId(), dataset.getDatasetId(),
                 primaryIndexInfo.primaryKeyIndexes, false, true, PARTITION, true);
         insertOp.setOutputFrameWriter(0, commitOp, primaryIndexInfo.rDesc);
@@ -183,21 +190,23 @@
 
     public IPushRuntime getFullScanPipeline(IFrameWriter countOp, IHyracksTaskContext ctx, Dataset dataset,
             IAType[] primaryKeyTypes, ARecordType recordType, ARecordType metaType,
-            NoMergePolicyFactory mergePolicyFactory, Map<String, String> mergePolicyProperties, int[] filterFields)
-            throws HyracksDataException, AlgebricksException {
+            NoMergePolicyFactory mergePolicyFactory, Map<String, String> mergePolicyProperties, int[] filterFields,
+            int[] primaryKeyIndexes, List<Integer> primaryKeyIndicators,
+            StorageComponentProvider storageComponentProvider) throws HyracksDataException, AlgebricksException {
         IPushRuntime emptyTupleOp = new EmptyTupleSourceRuntimeFactory().createPushRuntime(ctx);
         JobSpecification spec = new JobSpecification();
         PrimaryIndexInfo primaryIndexInfo = new PrimaryIndexInfo(dataset, primaryKeyTypes, recordType, metaType,
-                mergePolicyFactory, mergePolicyProperties, filterFields);
-        LSMBTreeDataflowHelperFactory indexDataflowHelperFactory = getPrimaryIndexDataflowHelperFactory(ctx,
-                primaryIndexInfo);
+                mergePolicyFactory, mergePolicyProperties, filterFields, primaryKeyIndexes, primaryKeyIndicators,
+                storageComponentProvider);
+        IIndexDataflowHelperFactory indexDataflowHelperFactory =
+                getPrimaryIndexDataflowHelperFactory(ctx, primaryIndexInfo, storageComponentProvider, dataset);
         BTreeSearchOperatorDescriptor searchOpDesc = new BTreeSearchOperatorDescriptor(spec, primaryIndexInfo.rDesc,
                 RuntimeComponentsProvider.RUNTIME_PROVIDER, RuntimeComponentsProvider.RUNTIME_PROVIDER,
                 primaryIndexInfo.fileSplitProvider, primaryIndexInfo.primaryIndexTypeTraits,
                 primaryIndexInfo.primaryIndexComparatorFactories, primaryIndexInfo.primaryIndexBloomFilterKeyFields,
                 primaryIndexInfo.primaryKeyIndexes, primaryIndexInfo.primaryKeyIndexes, true, true,
                 indexDataflowHelperFactory, false, false, null, NoOpOperationCallbackFactory.INSTANCE, filterFields,
-                filterFields, LSMIndexUtil.getMetadataPageManagerFactory());
+                filterFields, storageComponentProvider.getMetadataPageManagerFactory());
         BTreeSearchOperatorNodePushable searchOp = new BTreeSearchOperatorNodePushable(searchOpDesc, ctx, 0,
                 primaryIndexInfo.getSearchRecordDescriptorProvider(), /*primaryIndexInfo.primaryKeyIndexes*/null,
                 /*primaryIndexInfo.primaryKeyIndexes*/null, true, true, filterFields, filterFields);
@@ -218,10 +227,8 @@
 
     public LSMTreeInsertDeleteOperatorDescriptor getInsertOpratorDesc(PrimaryIndexInfo primaryIndexInfo,
             IModificationOperationCallbackFactory modOpCallbackFactory) {
-        LSMTreeInsertDeleteOperatorDescriptor indexOpDesc = Mockito
-                .mock(LSMTreeInsertDeleteOperatorDescriptor.class);
-        Mockito.when(indexOpDesc.getLifecycleManagerProvider())
-                .thenReturn(RuntimeComponentsProvider.RUNTIME_PROVIDER);
+        LSMTreeInsertDeleteOperatorDescriptor indexOpDesc = Mockito.mock(LSMTreeInsertDeleteOperatorDescriptor.class);
+        Mockito.when(indexOpDesc.getLifecycleManagerProvider()).thenReturn(RuntimeComponentsProvider.RUNTIME_PROVIDER);
         Mockito.when(indexOpDesc.getStorageManager()).thenReturn(RuntimeComponentsProvider.RUNTIME_PROVIDER);
         Mockito.when(indexOpDesc.getFileSplitProvider()).thenReturn(primaryIndexInfo.fileSplitProvider);
         Mockito.when(indexOpDesc.getLocalResourceFactoryProvider())
@@ -232,15 +239,14 @@
         Mockito.when(indexOpDesc.getTreeIndexBloomFilterKeyFields())
                 .thenReturn(primaryIndexInfo.primaryIndexBloomFilterKeyFields);
         Mockito.when(indexOpDesc.getModificationOpCallbackFactory()).thenReturn(modOpCallbackFactory);
-        Mockito.when(indexOpDesc.getPageManagerFactory()).thenReturn(LSMIndexUtil
-                .getMetadataPageManagerFactory());
+        Mockito.when(indexOpDesc.getPageManagerFactory())
+                .thenReturn(primaryIndexInfo.storageComponentProvider.getMetadataPageManagerFactory());
         return indexOpDesc;
     }
 
     public TreeIndexCreateOperatorDescriptor getIndexCreateOpDesc(PrimaryIndexInfo primaryIndexInfo) {
         TreeIndexCreateOperatorDescriptor indexOpDesc = Mockito.mock(TreeIndexCreateOperatorDescriptor.class);
-        Mockito.when(indexOpDesc.getLifecycleManagerProvider())
-                .thenReturn(RuntimeComponentsProvider.RUNTIME_PROVIDER);
+        Mockito.when(indexOpDesc.getLifecycleManagerProvider()).thenReturn(RuntimeComponentsProvider.RUNTIME_PROVIDER);
         Mockito.when(indexOpDesc.getStorageManager()).thenReturn(RuntimeComponentsProvider.RUNTIME_PROVIDER);
         Mockito.when(indexOpDesc.getFileSplitProvider()).thenReturn(primaryIndexInfo.fileSplitProvider);
         Mockito.when(indexOpDesc.getLocalResourceFactoryProvider())
@@ -250,8 +256,8 @@
                 .thenReturn(primaryIndexInfo.primaryIndexComparatorFactories);
         Mockito.when(indexOpDesc.getTreeIndexBloomFilterKeyFields())
                 .thenReturn(primaryIndexInfo.primaryIndexBloomFilterKeyFields);
-        Mockito.when(indexOpDesc.getPageManagerFactory()).thenReturn(LSMIndexUtil
-                .getMetadataPageManagerFactory());
+        Mockito.when(indexOpDesc.getPageManagerFactory())
+                .thenReturn(primaryIndexInfo.storageComponentProvider.getMetadataPageManagerFactory());
         return indexOpDesc;
     }
 
@@ -261,64 +267,66 @@
         return new ConstantFileSplitProvider(new FileSplit[] { fileSplit });
     }
 
-    public ILocalResourceFactoryProvider getPrimaryIndexLocalResourceMetadataProvider(Dataset dataset,
+    public ILocalResourceFactoryProvider getPrimaryIndexLocalResourceMetadataProvider(
+            IStorageComponentProvider storageComponentProvider, Index index, Dataset dataset,
             ITypeTraits[] primaryIndexTypeTraits, IBinaryComparatorFactory[] primaryIndexComparatorFactories,
             int[] primaryIndexBloomFilterKeyFields, ILSMMergePolicyFactory mergePolicyFactory,
             Map<String, String> mergePolicyProperties, ITypeTraits[] filterTypeTraits,
-            IBinaryComparatorFactory[] filterCmpFactories, int[] btreeFields, int[] filterFields) {
+            IBinaryComparatorFactory[] filterCmpFactories, int[] btreeFields, int[] filterFields,
+            ILSMOperationTrackerFactory opTrackerProvider) throws AlgebricksException {
         IResourceFactory localResourceMetadata = new LSMBTreeLocalResourceMetadataFactory(primaryIndexTypeTraits,
                 primaryIndexComparatorFactories, primaryIndexBloomFilterKeyFields, true, dataset.getDatasetId(),
                 mergePolicyFactory, mergePolicyProperties, filterTypeTraits, filterCmpFactories, btreeFields,
-                filterFields);
-        ILocalResourceFactoryProvider localResourceFactoryProvider = new PersistentLocalResourceFactoryProvider(
-                localResourceMetadata, LocalResource.LSMBTreeResource);
+                filterFields, opTrackerProvider, dataset.getIoOperationCallbackFactory(index),
+                storageComponentProvider.getMetadataPageManagerFactory());
+        ILocalResourceFactoryProvider localResourceFactoryProvider =
+                new PersistentLocalResourceFactoryProvider(localResourceMetadata, LocalResource.LSMBTreeResource);
         return localResourceFactoryProvider;
     }
 
-    public LSMBTreeDataflowHelper getPrimaryIndexDataflowHelper(IHyracksTaskContext ctx,
-            PrimaryIndexInfo primaryIndexInfo, TreeIndexCreateOperatorDescriptor indexOpDesc)
+    public IIndexDataflowHelper getPrimaryIndexDataflowHelper(IHyracksTaskContext ctx,
+            PrimaryIndexInfo primaryIndexInfo, TreeIndexCreateOperatorDescriptor indexOpDesc,
+            IStorageComponentProvider storageComponentProvider, Dataset dataset)
             throws AlgebricksException, HyracksDataException {
-        LSMBTreeDataflowHelperFactory dataflowHelperFactory = new LSMBTreeDataflowHelperFactory(
-                new AsterixVirtualBufferCacheProvider(primaryIndexInfo.dataset.getDatasetId()),
-                primaryIndexInfo.mergePolicyFactory, primaryIndexInfo.mergePolicyProperties,
-                new PrimaryIndexOperationTrackerProvider(primaryIndexInfo.dataset.getDatasetId()),
-                RuntimeComponentsProvider.RUNTIME_PROVIDER, LSMBTreeIOOperationCallbackFactory.INSTANCE,
-                BLOOM_FILTER_FALSE_POSITIVE_RATE, true, primaryIndexInfo.filterTypeTraits,
-                primaryIndexInfo.filterCmpFactories, primaryIndexInfo.btreeFields, primaryIndexInfo.filterFields, true);
-        IndexDataflowHelper dataflowHelper = dataflowHelperFactory.createIndexDataflowHelper(indexOpDesc, ctx,
-                PARTITION);
-        return (LSMBTreeDataflowHelper) dataflowHelper;
+        return getPrimaryIndexDataflowHelperFactory(ctx, primaryIndexInfo, storageComponentProvider, dataset)
+                .createIndexDataflowHelper(indexOpDesc, ctx, PARTITION);
     }
 
-    public LSMBTreeDataflowHelperFactory getPrimaryIndexDataflowHelperFactory(IHyracksTaskContext ctx,
-            PrimaryIndexInfo primaryIndexInfo) throws AlgebricksException {
-        return new LSMBTreeDataflowHelperFactory(
-                new AsterixVirtualBufferCacheProvider(primaryIndexInfo.dataset.getDatasetId()),
-                primaryIndexInfo.mergePolicyFactory, primaryIndexInfo.mergePolicyProperties,
-                new PrimaryIndexOperationTrackerProvider(primaryIndexInfo.dataset.getDatasetId()),
-                RuntimeComponentsProvider.RUNTIME_PROVIDER, LSMBTreeIOOperationCallbackFactory.INSTANCE,
-                BLOOM_FILTER_FALSE_POSITIVE_RATE, true, primaryIndexInfo.filterTypeTraits,
-                primaryIndexInfo.filterCmpFactories, primaryIndexInfo.btreeFields, primaryIndexInfo.filterFields, true);
+    public IIndexDataflowHelperFactory getPrimaryIndexDataflowHelperFactory(IHyracksTaskContext ctx,
+            PrimaryIndexInfo primaryIndexInfo, IStorageComponentProvider storageComponentProvider, Dataset dataset)
+            throws AlgebricksException {
+        Dataverse dataverse = new Dataverse(dataset.getDataverseName(), NonTaggedDataFormat.class.getName(),
+                MetadataUtil.PENDING_NO_OP);
+        Index index = primaryIndexInfo.getIndex();
+        MetadataProvider mdProvider = new MetadataProvider(dataverse, storageComponentProvider);
+        return dataset.getIndexDataflowHelperFactory(mdProvider, index, primaryIndexInfo.recordType,
+                primaryIndexInfo.metaType, primaryIndexInfo.mergePolicyFactory,
+                primaryIndexInfo.mergePolicyProperties);
     }
 
-    public LSMBTreeDataflowHelper getPrimaryIndexDataflowHelper(Dataset dataset, IAType[] primaryKeyTypes,
+    public IIndexDataflowHelper getPrimaryIndexDataflowHelper(Dataset dataset, IAType[] primaryKeyTypes,
             ARecordType recordType, ARecordType metaType, ILSMMergePolicyFactory mergePolicyFactory,
-            Map<String, String> mergePolicyProperties, int[] filterFields)
-            throws AlgebricksException, HyracksDataException {
+            Map<String, String> mergePolicyProperties, int[] filterFields,
+            IStorageComponentProvider storageComponentProvider, int[] primaryKeyIndexes,
+            List<Integer> primaryKeyIndicators) throws AlgebricksException, HyracksDataException {
         PrimaryIndexInfo primaryIndexInfo = new PrimaryIndexInfo(dataset, primaryKeyTypes, recordType, metaType,
-                mergePolicyFactory, mergePolicyProperties, filterFields);
+                mergePolicyFactory, mergePolicyProperties, filterFields, primaryKeyIndexes, primaryKeyIndicators,
+                storageComponentProvider);
         TreeIndexCreateOperatorDescriptor indexOpDesc = getIndexCreateOpDesc(primaryIndexInfo);
-        return getPrimaryIndexDataflowHelper(createTestContext(true), primaryIndexInfo, indexOpDesc);
+        return getPrimaryIndexDataflowHelper(createTestContext(true), primaryIndexInfo, indexOpDesc,
+                storageComponentProvider, dataset);
     }
 
     public void createPrimaryIndex(Dataset dataset, IAType[] primaryKeyTypes, ARecordType recordType,
             ARecordType metaType, ILSMMergePolicyFactory mergePolicyFactory, Map<String, String> mergePolicyProperties,
-            int[] filterFields) throws AlgebricksException, HyracksDataException {
+            int[] filterFields, IStorageComponentProvider storageComponentProvider, int[] primaryKeyIndexes,
+            List<Integer> primaryKeyIndicators) throws AlgebricksException, HyracksDataException {
         PrimaryIndexInfo primaryIndexInfo = new PrimaryIndexInfo(dataset, primaryKeyTypes, recordType, metaType,
-                mergePolicyFactory, mergePolicyProperties, filterFields);
+                mergePolicyFactory, mergePolicyProperties, filterFields, primaryKeyIndexes, primaryKeyIndicators,
+                storageComponentProvider);
         TreeIndexCreateOperatorDescriptor indexOpDesc = getIndexCreateOpDesc(primaryIndexInfo);
-        LSMBTreeDataflowHelper dataflowHelper = getPrimaryIndexDataflowHelper(createTestContext(true), primaryIndexInfo,
-                indexOpDesc);
+        IIndexDataflowHelper dataflowHelper = getPrimaryIndexDataflowHelper(createTestContext(true), primaryIndexInfo,
+                indexOpDesc, storageComponentProvider, dataset);
         dataflowHelper.create();
     }
 
@@ -331,10 +339,11 @@
     }
 
     private IBinaryComparatorFactory[] createPrimaryIndexComparatorFactories(IAType[] primaryKeyTypes) {
-        IBinaryComparatorFactory[] primaryIndexComparatorFactories = new IBinaryComparatorFactory[primaryKeyTypes.length];
+        IBinaryComparatorFactory[] primaryIndexComparatorFactories =
+                new IBinaryComparatorFactory[primaryKeyTypes.length];
         for (int j = 0; j < primaryKeyTypes.length; ++j) {
-            primaryIndexComparatorFactories[j] = BinaryComparatorFactoryProvider.INSTANCE
-                    .getBinaryComparatorFactory(primaryKeyTypes[j], true);
+            primaryIndexComparatorFactories[j] =
+                    BinaryComparatorFactoryProvider.INSTANCE.getBinaryComparatorFactory(primaryKeyTypes[j], true);
         }
         return primaryIndexComparatorFactories;
     }
@@ -344,8 +353,8 @@
         int i = 0;
         ISerializerDeserializer<?>[] primaryIndexSerdes = new ISerializerDeserializer<?>[primaryIndexNumOfTupleFields];
         for (; i < primaryKeyTypes.length; i++) {
-            primaryIndexSerdes[i] = SerializerDeserializerProvider.INSTANCE
-                    .getSerializerDeserializer(primaryKeyTypes[i]);
+            primaryIndexSerdes[i] =
+                    SerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(primaryKeyTypes[i]);
         }
         primaryIndexSerdes[i++] = SerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(recordType);
         if (metaType != null) {
@@ -371,12 +380,11 @@
     public IHyracksTaskContext createTestContext(boolean withMessaging) throws HyracksDataException {
         IHyracksTaskContext ctx = TestUtils.create(KB32);
         if (withMessaging) {
-            TaskUtils.putInSharedMap(HyracksConstants.KEY_MESSAGE, new VSizeFrame(ctx), ctx);
+            TaskUtil.putInSharedMap(HyracksConstants.KEY_MESSAGE, new VSizeFrame(ctx), ctx);
         }
         ctx = Mockito.spy(ctx);
         Mockito.when(ctx.getJobletContext()).thenReturn(jobletCtx);
-        Mockito.when(ctx.getIOManager())
-                .thenReturn(ExecutionTestUtil.integrationUtil.ncs[0].getIoManager());
+        Mockito.when(ctx.getIOManager()).thenReturn(ExecutionTestUtil.integrationUtil.ncs[0].getIoManager());
         return ctx;
     }
 
@@ -420,10 +428,18 @@
         private RecordDescriptor rDesc;
         private int[] primaryIndexInsertFieldsPermutations;
         private int[] primaryKeyIndexes;
+        private List<List<String>> keyFieldNames;
+        private List<Integer> keyFieldSourceIndicators;
+        private List<IAType> keyFieldTypes;
+        private Index index;
+        private IStorageComponentProvider storageComponentProvider;
 
-        public PrimaryIndexInfo(Dataset dataset, IAType[] primaryKeyTypes, ARecordType recordType, ARecordType metaType,
-                ILSMMergePolicyFactory mergePolicyFactory, Map<String, String> mergePolicyProperties,
-                int[] filterFields) throws AlgebricksException {
+        public PrimaryIndexInfo(Dataset dataset, IAType[] primaryKeyTypes, ARecordType recordType,
+                ARecordType metaType, ILSMMergePolicyFactory mergePolicyFactory,
+                Map<String, String> mergePolicyProperties, int[] filterFields, int[] primaryKeyIndexes,
+                List<Integer> primaryKeyIndicators, IStorageComponentProvider storageComponentProvider)
+                throws AlgebricksException {
+            this.storageComponentProvider = storageComponentProvider;
             this.dataset = dataset;
             this.primaryKeyTypes = primaryKeyTypes;
             this.recordType = recordType;
@@ -431,30 +447,44 @@
             this.mergePolicyFactory = mergePolicyFactory;
             this.mergePolicyProperties = mergePolicyProperties;
             this.filterFields = filterFields;
+            this.primaryKeyIndexes = primaryKeyIndexes;
             primaryIndexNumOfTupleFields = primaryKeyTypes.length + (1 + ((metaType == null) ? 0 : 1));
-            primaryIndexTypeTraits = createPrimaryIndexTypeTraits(primaryIndexNumOfTupleFields, primaryKeyTypes,
-                    recordType, metaType);
+            primaryIndexTypeTraits =
+                    createPrimaryIndexTypeTraits(primaryIndexNumOfTupleFields, primaryKeyTypes, recordType, metaType);
             primaryIndexComparatorFactories = createPrimaryIndexComparatorFactories(primaryKeyTypes);
             primaryIndexBloomFilterKeyFields = createPrimaryIndexBloomFilterFields(primaryKeyTypes.length);
-            filterTypeTraits = DatasetUtils.computeFilterTypeTraits(dataset, recordType);
-            filterCmpFactories = DatasetUtils.computeFilterBinaryComparatorFactories(dataset, recordType,
+            filterTypeTraits = DatasetUtil.computeFilterTypeTraits(dataset, recordType);
+            filterCmpFactories = DatasetUtil.computeFilterBinaryComparatorFactories(dataset, recordType,
                     NonTaggedDataFormat.INSTANCE.getBinaryComparatorFactoryProvider());
-            btreeFields = DatasetUtils.createBTreeFieldsWhenThereisAFilter(dataset);
-            localResourceFactoryProvider = getPrimaryIndexLocalResourceMetadataProvider(dataset, primaryIndexTypeTraits,
-                    primaryIndexComparatorFactories, primaryIndexBloomFilterKeyFields, mergePolicyFactory,
-                    mergePolicyProperties, filterTypeTraits, filterCmpFactories, btreeFields, filterFields);
+            btreeFields = DatasetUtil.createBTreeFieldsWhenThereisAFilter(dataset);
             fileSplitProvider = getFileSplitProvider(dataset);
-            primaryIndexSerdes = createPrimaryIndexSerdes(primaryIndexNumOfTupleFields, primaryKeyTypes, recordType,
-                    metaType);
+            primaryIndexSerdes =
+                    createPrimaryIndexSerdes(primaryIndexNumOfTupleFields, primaryKeyTypes, recordType, metaType);
             rDesc = new RecordDescriptor(primaryIndexSerdes, primaryIndexTypeTraits);
             primaryIndexInsertFieldsPermutations = new int[primaryIndexNumOfTupleFields];
             for (int i = 0; i < primaryIndexNumOfTupleFields; i++) {
                 primaryIndexInsertFieldsPermutations[i] = i;
             }
-            primaryKeyIndexes = new int[primaryKeyTypes.length];
-            for (int i = 0; i < primaryKeyIndexes.length; i++) {
-                primaryKeyIndexes[i] = i;
+            keyFieldSourceIndicators = primaryKeyIndicators;
+            keyFieldNames = new ArrayList<>();
+            keyFieldTypes = Arrays.asList(primaryKeyTypes);
+            for (int i = 0; i < keyFieldSourceIndicators.size(); i++) {
+                Integer indicator = keyFieldSourceIndicators.get(i);
+                String[] fieldNames =
+                        indicator == Index.RECORD_INDICATOR ? recordType.getFieldNames() : metaType.getFieldNames();
+                keyFieldNames.add(Arrays.asList(fieldNames[primaryKeyIndexes[i]]));
             }
+            index = new Index(dataset.getDataverseName(), dataset.getDatasetName(), dataset.getDatasetName(),
+                    IndexType.BTREE, keyFieldNames, keyFieldSourceIndicators, keyFieldTypes, false, true,
+                    MetadataUtil.PENDING_NO_OP);
+            localResourceFactoryProvider = getPrimaryIndexLocalResourceMetadataProvider(storageComponentProvider,
+                    index, dataset, primaryIndexTypeTraits, primaryIndexComparatorFactories,
+                    primaryIndexBloomFilterKeyFields, mergePolicyFactory, mergePolicyProperties, filterTypeTraits,
+                    filterCmpFactories, btreeFields, filterFields, dataset.getIndexOperationTrackerFactory(index));
+        }
+
+        public Index getIndex() {
+            return index;
         }
 
         public IRecordDescriptorProvider getInsertRecordDescriptorProvider() {
@@ -468,8 +498,8 @@
             ISerializerDeserializer<?>[] primaryKeySerdes = new ISerializerDeserializer<?>[primaryKeyTypes.length];
             for (int i = 0; i < primaryKeyTypes.length; i++) {
                 primaryKeyTypeTraits[i] = TypeTraitProvider.INSTANCE.getTypeTrait(primaryKeyTypes[i]);
-                primaryKeySerdes[i] = SerializerDeserializerProvider.INSTANCE
-                        .getSerializerDeserializer(primaryKeyTypes[i]);
+                primaryKeySerdes[i] =
+                        SerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(primaryKeyTypes[i]);
             }
             RecordDescriptor searcgRecDesc = new RecordDescriptor(primaryKeySerdes, primaryKeyTypeTraits);
             IRecordDescriptorProvider rDescProvider = Mockito.mock(IRecordDescriptorProvider.class);
@@ -481,10 +511,10 @@
 
     public RecordDescriptor getSearchOutputDesc(IAType[] keyTypes, ARecordType recordType, ARecordType metaType) {
         int primaryIndexNumOfTupleFields = keyTypes.length + (1 + ((metaType == null) ? 0 : 1));
-        ITypeTraits[] primaryIndexTypeTraits = createPrimaryIndexTypeTraits(primaryIndexNumOfTupleFields, keyTypes,
-                recordType, metaType);
-        ISerializerDeserializer<?>[] primaryIndexSerdes = createPrimaryIndexSerdes(primaryIndexNumOfTupleFields,
-                keyTypes, recordType, metaType);
+        ITypeTraits[] primaryIndexTypeTraits =
+                createPrimaryIndexTypeTraits(primaryIndexNumOfTupleFields, keyTypes, recordType, metaType);
+        ISerializerDeserializer<?>[] primaryIndexSerdes =
+                createPrimaryIndexSerdes(primaryIndexNumOfTupleFields, keyTypes, recordType, metaType);
         return new RecordDescriptor(primaryIndexSerdes, primaryIndexTypeTraits);
     }
 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/aql/translator/QueryTranslatorTest.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/aql/translator/QueryTranslatorTest.java
index b305e27..5cd8a63 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/aql/translator/QueryTranslatorTest.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/aql/translator/QueryTranslatorTest.java
@@ -29,14 +29,15 @@
 import java.util.List;
 
 import org.apache.asterix.app.translator.DefaultStatementExecutorFactory;
-import org.apache.asterix.common.config.ExternalProperties;
 import org.apache.asterix.common.config.ClusterProperties;
+import org.apache.asterix.common.config.ExternalProperties;
 import org.apache.asterix.compiler.provider.AqlCompilationProvider;
 import org.apache.asterix.event.schema.cluster.Cluster;
 import org.apache.asterix.event.schema.cluster.MasterNode;
+import org.apache.asterix.file.StorageComponentProvider;
 import org.apache.asterix.lang.common.base.Statement;
 import org.apache.asterix.lang.common.statement.RunStatement;
-import org.apache.asterix.runtime.util.AppContextInfo;
+import org.apache.asterix.runtime.utils.AppContextInfo;
 import org.apache.asterix.translator.IStatementExecutor;
 import org.apache.asterix.translator.SessionConfig;
 import org.junit.Assert;
@@ -49,7 +50,7 @@
 
     @Test
     public void test() throws Exception {
-        List<Statement> statements = new ArrayList<Statement>();
+        List<Statement> statements = new ArrayList<>();
         SessionConfig mockSessionConfig = mock(SessionConfig.class);
         RunStatement mockRunStatement = mock(RunStatement.class);
 
@@ -70,8 +71,8 @@
         when(mockMasterNode.getClientIp()).thenReturn("127.0.0.1");
 
         IStatementExecutor aqlTranslator = new DefaultStatementExecutorFactory().create(statements, mockSessionConfig,
-                new AqlCompilationProvider());
-        List<String> parameters = new ArrayList<String>();
+                new AqlCompilationProvider(), new StorageComponentProvider());
+        List<String> parameters = new ArrayList<>();
         parameters.add("examples/pregelix-example-jar-with-dependencies.jar");
         parameters.add("org.apache.pregelix.example.PageRankVertex");
         parameters.add("-ip 10.0.2.15 -port 3199");
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/ConnectorDescriptorWithMessagingTest.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/ConnectorDescriptorWithMessagingTest.java
index a253ac0..a3d6102 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/ConnectorDescriptorWithMessagingTest.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/ConnectorDescriptorWithMessagingTest.java
@@ -45,7 +45,7 @@
 import org.apache.hyracks.dataflow.common.data.marshalling.Integer64SerializerDeserializer;
 import org.apache.hyracks.dataflow.common.data.marshalling.UTF8StringSerializerDeserializer;
 import org.apache.hyracks.dataflow.common.io.MessagingFrameTupleAppender;
-import org.apache.hyracks.dataflow.common.util.TaskUtils;
+import org.apache.hyracks.dataflow.common.utils.TaskUtil;
 import org.apache.hyracks.dataflow.std.connectors.MToNPartitioningWithMessageConnectorDescriptor;
 import org.apache.hyracks.dataflow.std.connectors.PartitionWithMessageDataWriter;
 import org.apache.hyracks.test.support.TestUtils;
@@ -72,7 +72,7 @@
             IHyracksTaskContext ctx = TestUtils.create(DEFAULT_FRAME_SIZE);
             VSizeFrame message = new VSizeFrame(ctx);
             VSizeFrame tempBuffer = new VSizeFrame(ctx);
-            TaskUtils.putInSharedMap(HyracksConstants.KEY_MESSAGE, message, ctx);
+            TaskUtil.putInSharedMap(HyracksConstants.KEY_MESSAGE, message, ctx);
             message.getBuffer().clear();
             message.getBuffer().put(MessagingFrameTupleAppender.NULL_FEED_MESSAGE);
             message.getBuffer().flip();
@@ -81,8 +81,8 @@
                     BooleanSerializerDeserializer.INSTANCE, new UTF8StringSerializerDeserializer() };
             RecordDescriptor rDesc = new RecordDescriptor(serdes);
             TestPartitionWriterFactory partitionWriterFactory = new TestPartitionWriterFactory();
-            IFrameWriter partitioner = connector.createPartitioner(ctx, rDesc, partitionWriterFactory, CURRENT_PRODUCER,
-                    NUMBER_OF_CONSUMERS, NUMBER_OF_CONSUMERS);
+            IFrameWriter partitioner = connector.createPartitioner(ctx, rDesc, partitionWriterFactory,
+                    CURRENT_PRODUCER, NUMBER_OF_CONSUMERS, NUMBER_OF_CONSUMERS);
             partitioner.open();
             FrameTupleAccessor fta = new FrameTupleAccessor(rDesc);
             List<TestFrameWriter> recipients = new ArrayList<>();
@@ -146,15 +146,15 @@
             IHyracksTaskContext ctx = TestUtils.create(DEFAULT_FRAME_SIZE);
             VSizeFrame message = new VSizeFrame(ctx);
             VSizeFrame tempBuffer = new VSizeFrame(ctx);
-            TaskUtils.putInSharedMap(HyracksConstants.KEY_MESSAGE, message, ctx);
+            TaskUtil.putInSharedMap(HyracksConstants.KEY_MESSAGE, message, ctx);
             writeRandomMessage(message, MessagingFrameTupleAppender.MARKER_MESSAGE, DEFAULT_FRAME_SIZE + 1);
             ISerializerDeserializer<?>[] serdes = new ISerializerDeserializer<?>[] {
                     Integer64SerializerDeserializer.INSTANCE, DoubleSerializerDeserializer.INSTANCE,
                     BooleanSerializerDeserializer.INSTANCE, new UTF8StringSerializerDeserializer() };
             RecordDescriptor rDesc = new RecordDescriptor(serdes);
             TestPartitionWriterFactory partitionWriterFactory = new TestPartitionWriterFactory();
-            IFrameWriter partitioner = connector.createPartitioner(ctx, rDesc, partitionWriterFactory, CURRENT_PRODUCER,
-                    NUMBER_OF_CONSUMERS, NUMBER_OF_CONSUMERS);
+            IFrameWriter partitioner = connector.createPartitioner(ctx, rDesc, partitionWriterFactory,
+                    CURRENT_PRODUCER, NUMBER_OF_CONSUMERS, NUMBER_OF_CONSUMERS);
             partitioner.open();
             FrameTupleAccessor fta = new FrameTupleAccessor(rDesc);
             List<TestFrameWriter> recipients = new ArrayList<>();
@@ -230,7 +230,7 @@
             IHyracksTaskContext ctx = TestUtils.create(DEFAULT_FRAME_SIZE);
             VSizeFrame message = new VSizeFrame(ctx);
             VSizeFrame tempBuffer = new VSizeFrame(ctx);
-            TaskUtils.putInSharedMap(HyracksConstants.KEY_MESSAGE, message, ctx);
+            TaskUtil.putInSharedMap(HyracksConstants.KEY_MESSAGE, message, ctx);
             message.getBuffer().clear();
             writeRandomMessage(message, MessagingFrameTupleAppender.MARKER_MESSAGE, DEFAULT_FRAME_SIZE);
             ISerializerDeserializer<?>[] serdes = new ISerializerDeserializer<?>[] {
@@ -288,7 +288,7 @@
             IHyracksTaskContext ctx = TestUtils.create(DEFAULT_FRAME_SIZE);
             VSizeFrame message = new VSizeFrame(ctx);
             VSizeFrame tempBuffer = new VSizeFrame(ctx);
-            TaskUtils.putInSharedMap(HyracksConstants.KEY_MESSAGE, message, ctx);
+            TaskUtil.putInSharedMap(HyracksConstants.KEY_MESSAGE, message, ctx);
             message.getBuffer().clear();
             message.getBuffer().put(MessagingFrameTupleAppender.ACK_REQ_FEED_MESSAGE);
             message.getBuffer().flip();
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/LogMarkerTest.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/LogMarkerTest.java
index 00d725a..edd1848 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/LogMarkerTest.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/LogMarkerTest.java
@@ -18,8 +18,10 @@
  */
 package org.apache.asterix.test.dataflow;
 
+import java.util.Arrays;
 import java.util.Collection;
 import java.util.Collections;
+import java.util.List;
 
 import org.apache.asterix.app.bootstrap.TestNodeController;
 import org.apache.asterix.app.data.gen.TestTupleCounterFrameWriter;
@@ -31,7 +33,9 @@
 import org.apache.asterix.common.transactions.ILogRecord;
 import org.apache.asterix.common.transactions.ITransactionContext;
 import org.apache.asterix.external.util.DataflowUtils;
+import org.apache.asterix.file.StorageComponentProvider;
 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.entities.InternalDatasetDetails.PartitioningStrategy;
 import org.apache.asterix.om.types.ARecordType;
@@ -50,8 +54,8 @@
 import org.apache.hyracks.dataflow.common.comm.io.FrameTupleAppender;
 import org.apache.hyracks.dataflow.common.data.accessors.ITupleReference;
 import org.apache.hyracks.dataflow.common.io.MessagingFrameTupleAppender;
-import org.apache.hyracks.dataflow.common.util.TaskUtils;
-import org.apache.hyracks.storage.am.lsm.btree.dataflow.LSMBTreeDataflowHelper;
+import org.apache.hyracks.dataflow.common.utils.TaskUtil;
+import org.apache.hyracks.storage.am.common.api.IIndexDataflowHelper;
 import org.apache.hyracks.storage.am.lsm.btree.impls.LSMBTree;
 import org.apache.hyracks.storage.am.lsm.common.impls.NoMergePolicyFactory;
 import org.junit.After;
@@ -71,7 +75,8 @@
     private static final GenerationFunction[] META_GEN_FUNCTION = null;
     private static final boolean[] UNIQUE_META_FIELDS = null;
     private static final int[] KEY_INDEXES = { 0 };
-    private static final int[] KEY_INDICATORS = { 0 };
+    private static final int[] KEY_INDICATORS = { Index.RECORD_INDICATOR };
+    private static final List<Integer> KEY_INDICATORS_LIST = Arrays.asList(new Integer[] { Index.RECORD_INDICATOR });
     private static final int NUM_OF_RECORDS = 100000;
     private static final int SNAPSHOT_SIZE = 1000;
     private static final int DATASET_ID = 101;
@@ -97,18 +102,20 @@
         try {
             TestNodeController nc = new TestNodeController(null, false);
             nc.init();
+            StorageComponentProvider storageManager = new StorageComponentProvider();
             Dataset dataset = new Dataset(DATAVERSE_NAME, DATASET_NAME, DATAVERSE_NAME, DATA_TYPE_NAME,
                     NODE_GROUP_NAME, null, null, new InternalDatasetDetails(null, PartitioningStrategy.HASH,
                             Collections.emptyList(), null, null, null, false, null, false),
                     null, DatasetType.INTERNAL, DATASET_ID, 0);
             try {
                 nc.createPrimaryIndex(dataset, KEY_TYPES, RECORD_TYPE, META_TYPE, new NoMergePolicyFactory(), null,
-                        null);
+                        null, storageManager, KEY_INDEXES, KEY_INDICATORS_LIST);
                 IHyracksTaskContext ctx = nc.createTestContext(true);
                 nc.newJobId();
                 ITransactionContext txnCtx = nc.getTransactionManager().getTransactionContext(nc.getTxnJobId(), true);
                 LSMInsertDeleteOperatorNodePushable insertOp = nc.getInsertPipeline(ctx, dataset, KEY_TYPES,
-                        RECORD_TYPE, META_TYPE, new NoMergePolicyFactory(), null, null);
+                        RECORD_TYPE, META_TYPE, new NoMergePolicyFactory(), null, null, KEY_INDEXES,
+                        KEY_INDICATORS_LIST, storageManager);
                 insertOp.open();
                 TupleGenerator tupleGenerator = new TupleGenerator(RECORD_TYPE, META_TYPE, KEY_INDEXES, KEY_INDICATORS,
                         RECORD_GEN_FUNCTION, UNIQUE_RECORD_FIELDS, META_GEN_FUNCTION, UNIQUE_META_FIELDS);
@@ -123,7 +130,7 @@
                         marker.getBuffer().putLong(markerId);
                         marker.getBuffer().flip();
                         markerId++;
-                        TaskUtils.putInSharedMap(HyracksConstants.KEY_MESSAGE, marker, ctx);
+                        TaskUtil.putInSharedMap(HyracksConstants.KEY_MESSAGE, marker, ctx);
                         tupleAppender.flush(insertOp);
                     }
                     ITupleReference tuple = tupleGenerator.next();
@@ -134,8 +141,9 @@
                 }
                 insertOp.close();
                 nc.getTransactionManager().completedTransaction(txnCtx, new DatasetId(-1), -1, true);
-                LSMBTreeDataflowHelper dataflowHelper = nc.getPrimaryIndexDataflowHelper(dataset, KEY_TYPES,
-                        RECORD_TYPE, META_TYPE, new NoMergePolicyFactory(), null, null);
+                IIndexDataflowHelper dataflowHelper = nc.getPrimaryIndexDataflowHelper(dataset, KEY_TYPES,
+                        RECORD_TYPE, META_TYPE, new NoMergePolicyFactory(), null, null, storageManager, KEY_INDEXES,
+                        KEY_INDICATORS_LIST);
                 dataflowHelper.open();
                 LSMBTree btree = (LSMBTree) dataflowHelper.getIndexInstance();
                 long lsn = btree.getMostRecentMarkerLSN();
@@ -157,7 +165,8 @@
                 TestTupleCounterFrameWriter countOp = create(nc.getSearchOutputDesc(KEY_TYPES, RECORD_TYPE, META_TYPE),
                         Collections.emptyList(), Collections.emptyList(), false);
                 IPushRuntime emptyTupleOp = nc.getFullScanPipeline(countOp, ctx, dataset, KEY_TYPES, RECORD_TYPE,
-                        META_TYPE, new NoMergePolicyFactory(), null, null);
+                        META_TYPE, new NoMergePolicyFactory(), null, null, KEY_INDEXES, KEY_INDICATORS_LIST,
+                        storageManager);
                 emptyTupleOp.open();
                 emptyTupleOp.close();
                 Assert.assertEquals(NUM_OF_RECORDS, countOp.getCount());
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dml/DmlTest.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dml/DmlTest.java
index d5501c3..1158611 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dml/DmlTest.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dml/DmlTest.java
@@ -30,6 +30,7 @@
 import org.apache.asterix.app.translator.DefaultStatementExecutorFactory;
 import org.apache.asterix.common.exceptions.AsterixException;
 import org.apache.asterix.compiler.provider.AqlCompilationProvider;
+import org.apache.asterix.file.StorageComponentProvider;
 import org.apache.asterix.test.aql.TestExecutor;
 import org.apache.asterix.test.base.AsterixTestHelper;
 import org.junit.Test;
@@ -61,7 +62,7 @@
         Reader loadReader = new BufferedReader(
                 new InputStreamReader(new FileInputStream(LOAD_FOR_ENLIST_FILE), "UTF-8"));
         AsterixJavaClient asterixLoad = new AsterixJavaClient(integrationUtil.getHyracksClientConnection(), loadReader,
-                ERR, new AqlCompilationProvider(), new DefaultStatementExecutorFactory());
+                ERR, new AqlCompilationProvider(), new DefaultStatementExecutorFactory(), new StorageComponentProvider());
         try {
             asterixLoad.compile(true, false, false, false, false, true, false);
         } catch (AsterixException e) {
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/logging/CheckpointingTest.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/logging/CheckpointingTest.java
index 9fbf850..10e8658 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/logging/CheckpointingTest.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/logging/CheckpointingTest.java
@@ -19,13 +19,15 @@
 package org.apache.asterix.test.logging;
 
 import java.io.File;
+import java.util.Arrays;
 import java.util.Collections;
+import java.util.List;
 
 import org.apache.asterix.app.bootstrap.TestNodeController;
 import org.apache.asterix.app.data.gen.TupleGenerator;
 import org.apache.asterix.app.data.gen.TupleGenerator.GenerationFunction;
-import org.apache.asterix.common.config.TransactionProperties;
 import org.apache.asterix.common.config.DatasetConfig.DatasetType;
+import org.apache.asterix.common.config.TransactionProperties;
 import org.apache.asterix.common.configuration.AsterixConfiguration;
 import org.apache.asterix.common.configuration.Property;
 import org.apache.asterix.common.dataflow.LSMInsertDeleteOperatorNodePushable;
@@ -34,7 +36,9 @@
 import org.apache.asterix.common.transactions.IRecoveryManager;
 import org.apache.asterix.common.transactions.ITransactionContext;
 import org.apache.asterix.external.util.DataflowUtils;
+import org.apache.asterix.file.StorageComponentProvider;
 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.entities.InternalDatasetDetails.PartitioningStrategy;
 import org.apache.asterix.om.types.ARecordType;
@@ -71,7 +75,8 @@
     private static final GenerationFunction[] META_GEN_FUNCTION = null;
     private static final boolean[] UNIQUE_META_FIELDS = null;
     private static final int[] KEY_INDEXES = { 0 };
-    private static final int[] KEY_INDICATORS = { 0 };
+    private static final int[] KEY_INDICATOR = { Index.RECORD_INDICATOR };
+    private static final List<Integer> KEY_INDICATOR_LIST = Arrays.asList(new Integer[] { Index.RECORD_INDICATOR });
     private static final int DATASET_ID = 101;
     private static final String DATAVERSE_NAME = "TestDV";
     private static final String DATASET_NAME = "TestDS";
@@ -105,6 +110,7 @@
     public void testDeleteOldLogFiles() {
         try {
             TestNodeController nc = new TestNodeController(new File(TEST_CONFIG_FILE_PATH).getAbsolutePath(), false);
+            StorageComponentProvider storageManager = new StorageComponentProvider();
             nc.init();
             Dataset dataset = new Dataset(DATAVERSE_NAME, DATASET_NAME, DATAVERSE_NAME, DATA_TYPE_NAME,
                     NODE_GROUP_NAME, null, null, new InternalDatasetDetails(null, PartitioningStrategy.HASH,
@@ -112,15 +118,16 @@
                     null, DatasetType.INTERNAL, DATASET_ID, 0);
             try {
                 nc.createPrimaryIndex(dataset, KEY_TYPES, RECORD_TYPE, META_TYPE, new NoMergePolicyFactory(), null,
-                        null);
+                        null, storageManager, KEY_INDEXES, KEY_INDICATOR_LIST);
                 IHyracksTaskContext ctx = nc.createTestContext(false);
                 nc.newJobId();
                 ITransactionContext txnCtx = nc.getTransactionManager().getTransactionContext(nc.getTxnJobId(), true);
                 // Prepare insert operation
                 LSMInsertDeleteOperatorNodePushable insertOp = nc.getInsertPipeline(ctx, dataset, KEY_TYPES,
-                        RECORD_TYPE, META_TYPE, new NoMergePolicyFactory(), null, null);
+                        RECORD_TYPE, META_TYPE, new NoMergePolicyFactory(), null, null, KEY_INDEXES, KEY_INDICATOR_LIST,
+                        storageManager);
                 insertOp.open();
-                TupleGenerator tupleGenerator = new TupleGenerator(RECORD_TYPE, META_TYPE, KEY_INDEXES, KEY_INDICATORS,
+                TupleGenerator tupleGenerator = new TupleGenerator(RECORD_TYPE, META_TYPE, KEY_INDEXES, KEY_INDICATOR,
                         RECORD_GEN_FUNCTION, UNIQUE_RECORD_FIELDS, META_GEN_FUNCTION, UNIQUE_META_FIELDS);
                 VSizeFrame frame = new VSizeFrame(ctx);
                 FrameTupleAppender tupleAppender = new FrameTupleAppender(frame);
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/optimizer/OptimizerTest.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/optimizer/OptimizerTest.java
index 8db2bc0..01fe46a 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/optimizer/OptimizerTest.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/optimizer/OptimizerTest.java
@@ -32,12 +32,14 @@
 import org.apache.asterix.api.java.AsterixJavaClient;
 import org.apache.asterix.app.translator.DefaultStatementExecutorFactory;
 import org.apache.asterix.common.config.GlobalConfig;
+import org.apache.asterix.common.context.IStorageComponentProvider;
 import org.apache.asterix.common.exceptions.AsterixException;
 import org.apache.asterix.compiler.provider.AqlCompilationProvider;
 import org.apache.asterix.compiler.provider.ILangCompilationProvider;
 import org.apache.asterix.compiler.provider.SqlppCompilationProvider;
 import org.apache.asterix.external.util.ExternalDataConstants;
 import org.apache.asterix.external.util.IdentitiyResolverFactory;
+import org.apache.asterix.file.StorageComponentProvider;
 import org.apache.asterix.test.base.AsterixTestHelper;
 import org.apache.asterix.test.common.TestHelper;
 import org.apache.asterix.test.runtime.HDFSCluster;
@@ -76,6 +78,7 @@
     private static final ILangCompilationProvider sqlppCompilationProvider = new SqlppCompilationProvider();
     protected static ILangCompilationProvider extensionLangCompilationProvider = null;
     protected static IStatementExecutorFactory statementExecutorFactory = new DefaultStatementExecutorFactory();
+    protected static IStorageComponentProvider storageComponentProvider = new StorageComponentProvider();
 
     protected static AsterixHyracksIntegrationUtil integrationUtil = new AsterixHyracksIntegrationUtil();
 
@@ -124,7 +127,7 @@
 
     @Parameters(name = "OptimizerTest {index}: {0}")
     public static Collection<Object[]> tests() {
-        Collection<Object[]> testArgs = new ArrayList<Object[]>();
+        Collection<Object[]> testArgs = new ArrayList<>();
         if (only.isEmpty()) {
             suiteBuildPerFile(new File(PATH_QUERIES), testArgs, "");
         } else {
@@ -178,7 +181,8 @@
                 provider = extensionLangCompilationProvider;
             }
             IHyracksClientConnection hcc = integrationUtil.getHyracksClientConnection();
-            AsterixJavaClient asterix = new AsterixJavaClient(hcc, query, plan, provider, statementExecutorFactory);
+            AsterixJavaClient asterix = new AsterixJavaClient(hcc, query, plan, provider, statementExecutorFactory,
+                    storageComponentProvider);
             try {
                 asterix.compile(true, false, false, true, true, false, false);
             } catch (AsterixException e) {
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/runtime/ExecutionTestUtil.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/runtime/ExecutionTestUtil.java
index f488244..90eb441 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/runtime/ExecutionTestUtil.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/runtime/ExecutionTestUtil.java
@@ -31,7 +31,7 @@
 import org.apache.asterix.common.library.ILibraryManager;
 import org.apache.asterix.external.util.ExternalDataConstants;
 import org.apache.asterix.external.util.IdentitiyResolverFactory;
-import org.apache.asterix.runtime.util.AppContextInfo;
+import org.apache.asterix.runtime.utils.AppContextInfo;
 import org.apache.asterix.testframework.xml.TestGroup;
 import org.apache.asterix.testframework.xml.TestSuite;
 import org.apache.hyracks.control.nc.NodeControllerService;
diff --git a/asterixdb/asterix-app/src/test/resources/metadata/testsuite.xml b/asterixdb/asterix-app/src/test/resources/metadata/testsuite.xml
index a3864e4..0b93b96 100644
--- a/asterixdb/asterix-app/src/test/resources/metadata/testsuite.xml
+++ b/asterixdb/asterix-app/src/test/resources/metadata/testsuite.xml
@@ -417,13 +417,13 @@
     <test-case FilePath="exception">
       <compilation-unit name="issue_255_create_dataset_error_1">
         <output-dir compare="Text">none</output-dir>
-        <expected-error>Type not found for partitioning key [name]</expected-error>
+        <expected-error>Field "name" is not found</expected-error>
       </compilation-unit>
     </test-case>
     <test-case FilePath="exception">
       <compilation-unit name="issue_255_create_dataset_error_2">
         <output-dir compare="Text">none</output-dir>
-        <expected-error>Error: The partitioning key [open-type] cannot be of type record.</expected-error>
+        <expected-error>Field of type RECORD cannot be used as a primary key field</expected-error>
       </compilation-unit>
     </test-case>
     <!-- Feed datasets are not supported anymore
@@ -437,13 +437,13 @@
     <test-case FilePath="exception">
       <compilation-unit name="issue_266_create_dataset_error_1">
         <output-dir compare="Text">none</output-dir>
-        <expected-error>Type not found for partitioning key [point]</expected-error>
+        <expected-error>Field "point" is not found</expected-error>
       </compilation-unit>
     </test-case>
     <test-case FilePath="exception">
       <compilation-unit name="issue_266_create_dataset_error_2">
         <output-dir compare="Text">none</output-dir>
-        <expected-error>Error: The partitioning key [id] cannot be nullable</expected-error>
+        <expected-error>The primary key field "id" cannot be nullable</expected-error>
       </compilation-unit>
     </test-case>
     <test-case FilePath="exception">
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/testsuite.xml b/asterixdb/asterix-app/src/test/resources/runtimets/testsuite.xml
index ab55c89..5be8639 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/testsuite.xml
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/testsuite.xml
@@ -3317,7 +3317,7 @@
     <test-case FilePath="misc">
       <compilation-unit name="partition-by-nonexistent-field"> <!-- Seriously?? 3 expected errors -->
         <output-dir compare="Text">partition-by-nonexistent-field</output-dir>
-        <expected-error>Type not found for partitioning key [id]</expected-error>
+        <expected-error>Field "id" is not found</expected-error>
         <expected-error>Cannot find dataset</expected-error>
         <expected-error>Could not find dataset</expected-error>
       </compilation-unit>
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_sqlpp.xml b/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_sqlpp.xml
index e5c5bdd..a76ecbc 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_sqlpp.xml
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_sqlpp.xml
@@ -3119,7 +3119,7 @@
     <test-case FilePath="misc">
       <compilation-unit name="partition-by-nonexistent-field">
         <output-dir compare="Text">partition-by-nonexistent-field</output-dir>
-        <expected-error>Type not found for partitioning key [id]</expected-error>
+        <expected-error>Field "id" is not found</expected-error>
         <expected-error>Cannot find dataset testds in dataverse test</expected-error>
         <expected-error>Cannot find dataset testds in dataverse test nor an alias with name testds!</expected-error>
       </compilation-unit>
diff --git a/asterixdb/asterix-common/pom.xml b/asterixdb/asterix-common/pom.xml
index 277f94a..731ac14 100644
--- a/asterixdb/asterix-common/pom.xml
+++ b/asterixdb/asterix-common/pom.xml
@@ -24,7 +24,6 @@
     <version>0.9.1-SNAPSHOT</version>
   </parent>
   <artifactId>asterix-common</artifactId>
-
   <licenses>
     <license>
       <name>Apache License, Version 2.0</name>
@@ -33,11 +32,9 @@
       <comments>A business-friendly OSS license</comments>
     </license>
   </licenses>
-
   <properties>
     <appendedResourcesDirectory>${basedir}/../src/main/appended-resources</appendedResourcesDirectory>
   </properties>
-
   <build>
     <plugins>
       <plugin>
@@ -173,7 +170,6 @@
       </plugin>
     </plugins>
   </build>
-
   <dependencies>
     <dependency>
       <groupId>commons-io</groupId>
@@ -201,6 +197,10 @@
     </dependency>
     <dependency>
       <groupId>org.apache.hyracks</groupId>
+      <artifactId>algebricks-data</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hyracks</groupId>
       <artifactId>hyracks-util</artifactId>
     </dependency>
     <dependency>
@@ -316,5 +316,4 @@
       <artifactId>jackson-annotations</artifactId>
     </dependency>
   </dependencies>
-
-</project>
+</project>
\ No newline at end of file
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/api/IAppRuntimeContext.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/api/IAppRuntimeContext.java
index 7bc9421..548d714 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/api/IAppRuntimeContext.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/api/IAppRuntimeContext.java
@@ -22,6 +22,8 @@
 import java.rmi.RemoteException;
 import java.util.concurrent.Executor;
 
+import org.apache.asterix.common.config.IPropertiesProvider;
+import org.apache.asterix.common.context.IStorageComponentProvider;
 import org.apache.asterix.common.exceptions.ACIDException;
 import org.apache.asterix.common.exceptions.AsterixException;
 import org.apache.asterix.common.library.ILibraryManager;
@@ -40,58 +42,58 @@
 import org.apache.hyracks.storage.common.file.ILocalResourceRepository;
 import org.apache.hyracks.storage.common.file.IResourceIdFactory;
 
-public interface IAppRuntimeContext {
+public interface IAppRuntimeContext extends IPropertiesProvider {
 
-    public IIOManager getIOManager();
+    IIOManager getIOManager();
 
-    public Executor getThreadExecutor();
+    Executor getThreadExecutor();
 
-    public ITransactionSubsystem getTransactionSubsystem();
+    ITransactionSubsystem getTransactionSubsystem();
 
-    public boolean isShuttingdown();
+    boolean isShuttingdown();
 
-    public ILSMIOOperationScheduler getLSMIOScheduler();
+    ILSMIOOperationScheduler getLSMIOScheduler();
 
-    public ILSMMergePolicyFactory getMetadataMergePolicyFactory();
+    ILSMMergePolicyFactory getMetadataMergePolicyFactory();
 
-    public IBufferCache getBufferCache();
+    IBufferCache getBufferCache();
 
-    public IFileMapProvider getFileMapManager();
+    IFileMapProvider getFileMapManager();
 
-    public ILocalResourceRepository getLocalResourceRepository();
+    ILocalResourceRepository getLocalResourceRepository();
 
-    public IDatasetLifecycleManager getDatasetLifecycleManager();
+    IDatasetLifecycleManager getDatasetLifecycleManager();
 
-    public IResourceIdFactory getResourceIdFactory();
+    IResourceIdFactory getResourceIdFactory();
 
-    public ILSMOperationTracker getLSMBTreeOperationTracker(int datasetID);
+    ILSMOperationTracker getLSMBTreeOperationTracker(int datasetID);
 
-    public void initialize(boolean initialRun) throws IOException, ACIDException, AsterixException;
+    void initialize(boolean initialRun) throws IOException, ACIDException, AsterixException;
 
-    public void setShuttingdown(boolean b);
+    void setShuttingdown(boolean b);
 
-    public void deinitialize() throws HyracksDataException;
+    void deinitialize() throws HyracksDataException;
 
-    public double getBloomFilterFalsePositiveRate();
+    double getBloomFilterFalsePositiveRate();
 
-    public Object getActiveManager();
+    Object getActiveManager();
 
-    public IRemoteRecoveryManager getRemoteRecoveryManager();
+    IRemoteRecoveryManager getRemoteRecoveryManager();
 
-    public IReplicaResourcesManager getReplicaResourcesManager();
+    IReplicaResourcesManager getReplicaResourcesManager();
 
-    public IReplicationManager getReplicationManager();
+    IReplicationManager getReplicationManager();
 
-    public IReplicationChannel getReplicationChannel();
+    IReplicationChannel getReplicationChannel();
 
-    public ILibraryManager getLibraryManager();
+    ILibraryManager getLibraryManager();
 
     /**
      * Exports the metadata node to the metadata RMI port.
      *
      * @throws RemoteException
      */
-    public void exportMetadataNodeStub() throws RemoteException;
+    void exportMetadataNodeStub() throws RemoteException;
 
     /**
      * Initializes the metadata node and bootstraps the metadata.
@@ -99,12 +101,17 @@
      * @param newUniverse
      * @throws Exception
      */
-    public void initializeMetadata(boolean newUniverse) throws Exception;
+    void initializeMetadata(boolean newUniverse) throws Exception;
 
     /**
      * Unexports the metadata node from the RMI registry
      *
      * @throws RemoteException
      */
-    public void unexportMetadataNodeStub() throws RemoteException;
+    void unexportMetadataNodeStub() throws RemoteException;
+
+    /**
+     * @return instance of {@link org.apache.asterix.common.context.IStorageComponentProvider}
+     */
+    IStorageComponentProvider getStorageComponentProvider();
 }
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/cluster/IGlobalRecoveryMaanger.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/cluster/IGlobalRecoveryManager.java
similarity index 94%
rename from asterixdb/asterix-common/src/main/java/org/apache/asterix/common/cluster/IGlobalRecoveryMaanger.java
rename to asterixdb/asterix-common/src/main/java/org/apache/asterix/common/cluster/IGlobalRecoveryManager.java
index 48b1e73..b54bb39 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/cluster/IGlobalRecoveryMaanger.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/cluster/IGlobalRecoveryManager.java
@@ -20,7 +20,7 @@
 
 import org.apache.asterix.common.api.IClusterEventsSubscriber;
 
-public interface IGlobalRecoveryMaanger extends IClusterEventsSubscriber {
+public interface IGlobalRecoveryManager extends IClusterEventsSubscriber {
 
     /**
      * Starts the global recovery process if the cluster state changed to ACTIVE.
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/DatasetConfig.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/DatasetConfig.java
index 49ffa28..3c9acf4 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/DatasetConfig.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/DatasetConfig.java
@@ -42,16 +42,38 @@
         LENGTH_PARTITIONED_NGRAM_INVIX
     }
 
-    public enum ExternalDatasetTransactionState {
-        COMMIT, // The committed state <- nothing is required->
-        BEGIN, // The state after starting the refresh transaction <- will either abort moving to committed state or move to ready to commit->
-        READY_TO_COMMIT // The transaction is ready to commit <- can only move forward to committed state->
-    };
+    public enum TransactionState {
+        /**
+         * The committed state <- nothing is required->
+         */
+        COMMIT,
+        /**
+         * The state after starting the refresh transaction
+         * <- will either abort moving to committed state or move to ready to commit->
+         */
+        BEGIN,
+        /**
+         * The transaction is ready to commit <- can only move forward to committed state->
+         */
+        READY_TO_COMMIT
+    }
 
     public enum ExternalFilePendingOp {
-        PENDING_NO_OP, // the stored file is part of a committed transaction nothing is required
-        PENDING_ADD_OP, // the stored file is part of an ongoing transaction (will be added if transaction succeed)
-        PENDING_DROP_OP, // the stored file is part of an ongoing transaction (will be dropped if transaction succeed)
-        PENDING_APPEND_OP // the stored file is part of an ongoing transaction (will be updated if transaction succeed)
-    };
+        /**
+         * the stored file is part of a committed transaction nothing is required
+         */
+        NO_OP,
+        /**
+         * the stored file is part of an ongoing transaction (will be added if transaction succeed)
+         */
+        ADD_OP,
+        /**
+         * the stored file is part of an ongoing transaction (will be dropped if transaction succeed)
+         */
+        DROP_OP,
+        /**
+         * the stored file is part of an ongoing transaction (will be updated if transaction succeed)
+         */
+        APPEND_OP
+    }
 }
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/context/IStorageComponentProvider.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/context/IStorageComponentProvider.java
new file mode 100644
index 0000000..d454349
--- /dev/null
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/context/IStorageComponentProvider.java
@@ -0,0 +1,74 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.common.context;
+
+import org.apache.hyracks.algebricks.data.IBinaryComparatorFactoryProvider;
+import org.apache.hyracks.algebricks.data.ITypeTraitProvider;
+import org.apache.hyracks.storage.am.common.api.IIndexLifecycleManagerProvider;
+import org.apache.hyracks.storage.am.common.api.IMetadataPageManagerFactory;
+import org.apache.hyracks.storage.am.common.api.IPrimitiveValueProviderFactory;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationSchedulerProvider;
+import org.apache.hyracks.storage.common.IStorageManager;
+
+/**
+ * Responsible for storage components
+ */
+public interface IStorageComponentProvider {
+
+    /**
+     * @return {@link org.apache.asterix.common.context.ITransactionSubsystemProvider} instance
+     */
+    ITransactionSubsystemProvider getTransactionSubsystemProvider();
+
+    /**
+     * @return {@link org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationSchedulerProvider} instance
+     */
+    ILSMIOOperationSchedulerProvider getIoOperationSchedulerProvider();
+
+    /**
+     * @return the application's root
+     *         {@link org.apache.hyracks.storage.am.common.api.IIndexLifecycleManagerProvider} instance
+     */
+    IIndexLifecycleManagerProvider getIndexLifecycleManagerProvider();
+
+    /**
+     * @return {@link org.apache.hyracks.storage.common.IStorageManager} instance
+     */
+    IStorageManager getStorageManager();
+
+    /**
+     * @return {@link org.apache.hyracks.storage.am.common.api.IMetadataPageManagerFactory} instance
+     */
+    IMetadataPageManagerFactory getMetadataPageManagerFactory();
+
+    /**
+     * @return {@link org.apache.hyracks.storage.am.common.api.IPrimitiveValueProviderFactory} instance
+     */
+    IPrimitiveValueProviderFactory getPrimitiveValueProviderFactory();
+
+    /**
+     * @return {@link org.apache.hyracks.algebricks.data.IBinaryComparatorFactoryProvider} instance
+     */
+    IBinaryComparatorFactoryProvider getComparatorFactoryProvider();
+
+    /**
+     * @return {@link org.apache.hyracks.algebricks.data.ITypeTraitProvider} instance
+     */
+    ITypeTraitProvider getTypeTraitProvider();
+}
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/context/ITransactionSubsystemProvider.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/context/ITransactionSubsystemProvider.java
index 586fb4d..1dd1845 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/context/ITransactionSubsystemProvider.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/context/ITransactionSubsystemProvider.java
@@ -24,6 +24,10 @@
 import org.apache.asterix.common.transactions.ITransactionSubsystem;
 import org.apache.hyracks.api.context.IHyracksTaskContext;
 
+/**
+ * Provides the system's implementation of {@link ITransactionSubsystem}
+ */
+@FunctionalInterface
 public interface ITransactionSubsystemProvider extends Serializable {
-    public ITransactionSubsystem getTransactionSubsystem(IHyracksTaskContext ctx);
+    ITransactionSubsystem getTransactionSubsystem(IHyracksTaskContext ctx);
 }
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/context/TransactionSubsystemProvider.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/context/TransactionSubsystemProvider.java
index 706c303..cfbad43 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/context/TransactionSubsystemProvider.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/context/TransactionSubsystemProvider.java
@@ -30,6 +30,10 @@
  */
 public class TransactionSubsystemProvider implements ITransactionSubsystemProvider {
     private static final long serialVersionUID = 1L;
+    public static final TransactionSubsystemProvider INSTANCE = new TransactionSubsystemProvider();
+
+    private TransactionSubsystemProvider() {
+    }
 
     @Override
     public ITransactionSubsystem getTransactionSubsystem(IHyracksTaskContext ctx) {
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/dataflow/IApplicationContextInfo.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/dataflow/IApplicationContextInfo.java
index b9d547f..eb944a2 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/dataflow/IApplicationContextInfo.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/dataflow/IApplicationContextInfo.java
@@ -18,15 +18,20 @@
  */
 package org.apache.asterix.common.dataflow;
 
-import org.apache.asterix.common.cluster.IGlobalRecoveryMaanger;
+import org.apache.asterix.common.cluster.IGlobalRecoveryManager;
 import org.apache.asterix.common.library.ILibraryManager;
 import org.apache.hyracks.api.application.ICCApplicationContext;
 import org.apache.hyracks.storage.am.common.api.IIndexLifecycleManagerProvider;
-import org.apache.hyracks.storage.common.IStorageManagerInterface;
+import org.apache.hyracks.storage.common.IStorageManager;
 
 /**
- * Provides methods for obtaining the IIndexLifecycleManagerProvider, IStorageManagerInterface and
- * ICCApplicationContext implementation.
+ * Provides methods for obtaining
+ * {@link org.apache.hyracks.storage.am.common.api.IIndexLifecycleManagerProvider},
+ * {@link org.apache.hyracks.storage.common.IStorageManager},
+ * {@link org.apache.hyracks.api.application.ICCApplicationContext},
+ * {@link org.apache.asterix.common.cluster.IGlobalRecoveryManager},
+ * and {@link org.apache.asterix.common.library.ILibraryManager}
+ * at the cluster controller side.
  */
 public interface IApplicationContextInfo {
 
@@ -38,26 +43,23 @@
     public IIndexLifecycleManagerProvider getIndexLifecycleManagerProvider();
 
     /**
-     * Returns an instance of the implementation for IStorageManagerInterface.
-     *
-     * @return IStorageManagerInterface implementation instance
+     * @return an instance which implements {@link org.apache.hyracks.storage.common.IStorageManager}
      */
-    public IStorageManagerInterface getStorageManagerInterface();
+    public IStorageManager getStorageManager();
 
     /**
-     * Returns an instance of the implementation for ICCApplicationContext.
-     *
-     * @return ICCApplicationContext implementation instance
+     * @return an instance which implements {@link org.apache.hyracks.api.application.ICCApplicationContext}
      */
     public ICCApplicationContext getCCApplicationContext();
 
     /**
-     * @return the global recovery manager.
+     * @return the global recovery manager which implements
+     *         {@link org.apache.asterix.common.cluster.IGlobalRecoveryManager}
      */
-    public IGlobalRecoveryMaanger getGlobalRecoveryManager();
+    public IGlobalRecoveryManager getGlobalRecoveryManager();
 
     /**
-     * @return the library manager (at CC side).
+     * @return the library manager which implements {@link org.apache.asterix.common.library.ILibraryManager}
      */
     public ILibraryManager getLibraryManager();
 }
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/dataflow/LSMIndexUtil.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/dataflow/LSMIndexUtil.java
index 27bb14f..cd40179 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/dataflow/LSMIndexUtil.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/dataflow/LSMIndexUtil.java
@@ -21,8 +21,6 @@
 import org.apache.asterix.common.ioopcallbacks.AbstractLSMIOOperationCallback;
 import org.apache.asterix.common.transactions.ILogManager;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
-import org.apache.hyracks.storage.am.common.api.IMetadataPageManagerFactory;
-import org.apache.hyracks.storage.am.common.freepage.AppendOnlyLinkedMetadataPageManagerFactory;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponent;
 import org.apache.hyracks.storage.am.lsm.common.impls.AbstractLSMIndex;
 
@@ -49,8 +47,4 @@
                 .getIOOperationCallback();
         return ioOpCallback.getComponentFileLSNOffset(lsmComponent, componentFilePath);
     }
-
-    public static IMetadataPageManagerFactory getMetadataPageManagerFactory() {
-        return new AppendOnlyLinkedMetadataPageManagerFactory();
-    }
 }
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/dataflow/LSMInsertDeleteOperatorNodePushable.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/dataflow/LSMInsertDeleteOperatorNodePushable.java
index 20912ef..ac878ec 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/dataflow/LSMInsertDeleteOperatorNodePushable.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/dataflow/LSMInsertDeleteOperatorNodePushable.java
@@ -33,7 +33,7 @@
 import org.apache.hyracks.dataflow.common.comm.io.FrameTupleAppender;
 import org.apache.hyracks.dataflow.common.comm.util.FrameUtils;
 import org.apache.hyracks.dataflow.common.data.accessors.FrameTupleReference;
-import org.apache.hyracks.dataflow.common.util.TaskUtils;
+import org.apache.hyracks.dataflow.common.utils.TaskUtil;
 import org.apache.hyracks.storage.am.common.api.ITupleFilterFactory;
 import org.apache.hyracks.storage.am.common.dataflow.IIndexOperatorDescriptor;
 import org.apache.hyracks.storage.am.common.impls.NoOpOperationCallback;
@@ -82,7 +82,7 @@
         try {
             if (isPrimary && ctx.getSharedObject() != null) {
                 PrimaryIndexLogMarkerCallback callback = new PrimaryIndexLogMarkerCallback(lsmIndex);
-                TaskUtils.putInSharedMap(ILogMarkerCallback.KEY_MARKER_CALLBACK, callback, ctx);
+                TaskUtil.putInSharedMap(ILogMarkerCallback.KEY_MARKER_CALLBACK, callback, ctx);
             }
             writer.open();
             modCallback = opDesc.getModificationOpCallbackFactory().createModificationOperationCallback(
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/dataflow/LSMInvertedIndexInsertDeleteOperatorDescriptor.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/dataflow/LSMInvertedIndexInsertDeleteOperatorDescriptor.java
index 4a4502a..2ff0617 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/dataflow/LSMInvertedIndexInsertDeleteOperatorDescriptor.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/dataflow/LSMInvertedIndexInsertDeleteOperatorDescriptor.java
@@ -28,14 +28,14 @@
 import org.apache.hyracks.api.job.IOperatorDescriptorRegistry;
 import org.apache.hyracks.dataflow.std.file.IFileSplitProvider;
 import org.apache.hyracks.storage.am.common.api.IIndexLifecycleManagerProvider;
-import org.apache.hyracks.storage.am.common.api.IMetadataPageManagerFactory;
 import org.apache.hyracks.storage.am.common.api.IModificationOperationCallbackFactory;
+import org.apache.hyracks.storage.am.common.api.IPageManagerFactory;
 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.ophelpers.IndexOperation;
 import org.apache.hyracks.storage.am.lsm.invertedindex.dataflow.LSMInvertedIndexInsertUpdateDeleteOperator;
 import org.apache.hyracks.storage.am.lsm.invertedindex.tokenizers.IBinaryTokenizerFactory;
-import org.apache.hyracks.storage.common.IStorageManagerInterface;
+import org.apache.hyracks.storage.common.IStorageManager;
 
 public class LSMInvertedIndexInsertDeleteOperatorDescriptor extends LSMInvertedIndexInsertUpdateDeleteOperator {
 
@@ -43,27 +43,27 @@
 
     private final String indexName;
 
-    public LSMInvertedIndexInsertDeleteOperatorDescriptor(IOperatorDescriptorRegistry spec,
-            RecordDescriptor recDesc, IStorageManagerInterface storageManager, IFileSplitProvider fileSplitProvider,
+    public LSMInvertedIndexInsertDeleteOperatorDescriptor(IOperatorDescriptorRegistry spec, RecordDescriptor recDesc,
+            IStorageManager storageManager, IFileSplitProvider fileSplitProvider,
             IIndexLifecycleManagerProvider lifecycleManagerProvider, ITypeTraits[] tokenTypeTraits,
             IBinaryComparatorFactory[] tokenComparatorFactories, ITypeTraits[] invListsTypeTraits,
             IBinaryComparatorFactory[] invListComparatorFactories, IBinaryTokenizerFactory tokenizerFactory,
             int[] fieldPermutation, IndexOperation op, IIndexDataflowHelperFactory dataflowHelperFactory,
             ITupleFilterFactory tupleFilterFactory,
             IModificationOperationCallbackFactory modificationOpCallbackFactory, String indexName,
-            IMetadataPageManagerFactory metadataPageManagerFactory) {
+            IPageManagerFactory pageManagerFactory) {
         super(spec, recDesc, storageManager, fileSplitProvider, lifecycleManagerProvider, tokenTypeTraits,
                 tokenComparatorFactories, invListsTypeTraits, invListComparatorFactories, tokenizerFactory,
                 fieldPermutation, op, dataflowHelperFactory, tupleFilterFactory, modificationOpCallbackFactory,
-                metadataPageManagerFactory);
+                pageManagerFactory);
         this.indexName = indexName;
     }
 
     @Override
     public IOperatorNodePushable createPushRuntime(IHyracksTaskContext ctx,
             IRecordDescriptorProvider recordDescProvider, int partition, int nPartitions) throws HyracksDataException {
-        return new LSMInsertDeleteOperatorNodePushable(this, ctx, partition, fieldPermutation,
-                recordDescProvider, op, false);
+        return new LSMInsertDeleteOperatorNodePushable(this, ctx, partition, fieldPermutation, recordDescProvider, op,
+                false);
     }
 
     public String getIndexName() {
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/dataflow/LSMTreeInsertDeleteOperatorDescriptor.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/dataflow/LSMTreeInsertDeleteOperatorDescriptor.java
index 0f83dd3..b9e7c23 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/dataflow/LSMTreeInsertDeleteOperatorDescriptor.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/dataflow/LSMTreeInsertDeleteOperatorDescriptor.java
@@ -29,14 +29,14 @@
 import org.apache.hyracks.api.job.IOperatorDescriptorRegistry;
 import org.apache.hyracks.dataflow.std.file.IFileSplitProvider;
 import org.apache.hyracks.storage.am.common.api.IIndexLifecycleManagerProvider;
-import org.apache.hyracks.storage.am.common.api.IMetadataPageManagerFactory;
 import org.apache.hyracks.storage.am.common.api.IModificationOperationCallbackFactory;
+import org.apache.hyracks.storage.am.common.api.IPageManagerFactory;
 import org.apache.hyracks.storage.am.common.api.ISearchOperationCallbackFactory;
 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.ophelpers.IndexOperation;
 import org.apache.hyracks.storage.am.lsm.common.dataflow.LSMTreeIndexInsertUpdateDeleteOperatorDescriptor;
-import org.apache.hyracks.storage.common.IStorageManagerInterface;
+import org.apache.hyracks.storage.common.IStorageManager;
 
 public class LSMTreeInsertDeleteOperatorDescriptor extends LSMTreeIndexInsertUpdateDeleteOperatorDescriptor {
 
@@ -48,19 +48,18 @@
     private final String indexName;
 
     public LSMTreeInsertDeleteOperatorDescriptor(IOperatorDescriptorRegistry spec, RecordDescriptor recDesc,
-            IStorageManagerInterface storageManager, IIndexLifecycleManagerProvider lifecycleManagerProvider,
+            IStorageManager storageManager, IIndexLifecycleManagerProvider lifecycleManagerProvider,
             IFileSplitProvider fileSplitProvider, ITypeTraits[] typeTraits,
             IBinaryComparatorFactory[] comparatorFactories, int[] bloomFilterKeyFields, int[] fieldPermutation,
             IndexOperation op, IIndexDataflowHelperFactory dataflowHelperFactory,
             ITupleFilterFactory tupleFilterFactory, boolean isPrimary, String indexName,
             IMissingWriterFactory nullWriterFactory,
             IModificationOperationCallbackFactory modificationOpCallbackProvider,
-            ISearchOperationCallbackFactory searchOpCallbackProvider,
-            IMetadataPageManagerFactory metadataPageManagerFactory) {
+            ISearchOperationCallbackFactory searchOpCallbackProvider, IPageManagerFactory pageManagerFactory) {
         super(spec, recDesc, storageManager, lifecycleManagerProvider, fileSplitProvider, typeTraits,
                 comparatorFactories, bloomFilterKeyFields, fieldPermutation, op, dataflowHelperFactory,
                 tupleFilterFactory, nullWriterFactory, modificationOpCallbackProvider, searchOpCallbackProvider,
-                metadataPageManagerFactory);
+                pageManagerFactory);
         this.isPrimary = isPrimary;
         this.indexName = indexName;
     }
@@ -68,8 +67,8 @@
     @Override
     public IOperatorNodePushable createPushRuntime(IHyracksTaskContext ctx,
             IRecordDescriptorProvider recordDescProvider, int partition, int nPartitions) throws HyracksDataException {
-        return new LSMInsertDeleteOperatorNodePushable(this, ctx, partition, fieldPermutation,
-                recordDescProvider, op, isPrimary);
+        return new LSMInsertDeleteOperatorNodePushable(this, ctx, partition, fieldPermutation, recordDescProvider, op,
+                isPrimary);
     }
 
     public boolean isPrimary() {
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/exceptions/ErrorCode.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/exceptions/ErrorCode.java
index fad00a7..22c080a 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/exceptions/ErrorCode.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/exceptions/ErrorCode.java
@@ -64,6 +64,19 @@
     public static final int COMPILATION_INVALID_PARAMETER_NUMBER = 1008;
     public static final int COMPILATION_INVALID_RETURNING_EXPRESSION = 1009;
     public static final int COMPILATION_FULLTEXT_PHRASE_FOUND = 1010;
+    public static final int COMPILATION_UNKNOWN_DATASET_TYPE = 1011;
+    public static final int COMPILATION_UNKNOWN_INDEX_TYPE = 1012;
+    public static final int COMPILATION_ILLEGAL_INDEX_NUM_OF_FIELD = 1013;
+    public static final int COMPILATION_FIELD_NOT_FOUND = 1014;
+    public static final int COMPILATION_ILLEGAL_INDEX_FOR_DATASET_WITH_COMPOSITE_PRIMARY_INDEX = 1015;
+    public static final int COMPILATION_INDEX_TYPE_NOT_SUPPORTED_FOR_DATASET_TYPE = 1016;
+    public static final int COMPILATION_FILTER_CANNOT_BE_NULLABLE = 1017;
+    public static final int COMPILATION_ILLEGAL_FILTER_TYPE = 1018;
+    public static final int COMPILATION_CANNOT_AUTOGENERATE_COMPOSITE_PRIMARY_KEY = 1019;
+    public static final int COMPILATION_ILLEGAL_AUTOGENERATED_TYPE = 1020;
+    public static final int COMPILATION_PRIMARY_KEY_CANNOT_BE_NULLABLE = 1021;
+    public static final int COMPILATION_ILLEGAL_PRIMARY_KEY_TYPE = 1022;
+    public static final int COMPILATION_CANT_DROP_ACTIVE_DATASET = 1023;
 
     // Feed errors
     public static final int DATAFLOW_ILLEGAL_STATE = 3001;
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/ioopcallbacks/LSMBTreeWithBuddyIOOperationCallbackFactory.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/ioopcallbacks/LSMBTreeWithBuddyIOOperationCallbackFactory.java
index 7d33340..848de29 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/ioopcallbacks/LSMBTreeWithBuddyIOOperationCallbackFactory.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/ioopcallbacks/LSMBTreeWithBuddyIOOperationCallbackFactory.java
@@ -25,7 +25,8 @@
 
     private static final long serialVersionUID = 1L;
 
-    public static LSMBTreeWithBuddyIOOperationCallbackFactory INSTANCE = new LSMBTreeWithBuddyIOOperationCallbackFactory();
+    public static final LSMBTreeWithBuddyIOOperationCallbackFactory INSTANCE =
+            new LSMBTreeWithBuddyIOOperationCallbackFactory();
 
     private LSMBTreeWithBuddyIOOperationCallbackFactory() {
     }
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/ioopcallbacks/LSMInvertedIndexIOOperationCallbackFactory.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/ioopcallbacks/LSMInvertedIndexIOOperationCallbackFactory.java
index 3b71027..8951cb4 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/ioopcallbacks/LSMInvertedIndexIOOperationCallbackFactory.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/ioopcallbacks/LSMInvertedIndexIOOperationCallbackFactory.java
@@ -26,7 +26,8 @@
 
     private static final long serialVersionUID = 1L;
 
-    public static LSMInvertedIndexIOOperationCallbackFactory INSTANCE = new LSMInvertedIndexIOOperationCallbackFactory();
+    public static final LSMInvertedIndexIOOperationCallbackFactory INSTANCE =
+            new LSMInvertedIndexIOOperationCallbackFactory();
 
     private LSMInvertedIndexIOOperationCallbackFactory() {
     }
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/ioopcallbacks/LSMRTreeIOOperationCallbackFactory.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/ioopcallbacks/LSMRTreeIOOperationCallbackFactory.java
index 6ae833e..954c6e1 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/ioopcallbacks/LSMRTreeIOOperationCallbackFactory.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/ioopcallbacks/LSMRTreeIOOperationCallbackFactory.java
@@ -26,7 +26,7 @@
 
     private static final long serialVersionUID = 1L;
 
-    public static LSMRTreeIOOperationCallbackFactory INSTANCE = new LSMRTreeIOOperationCallbackFactory();
+    public static final LSMRTreeIOOperationCallbackFactory INSTANCE = new LSMRTreeIOOperationCallbackFactory();
 
     private LSMRTreeIOOperationCallbackFactory() {
     }
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/transactions/ITransactionManager.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/transactions/ITransactionManager.java
index af056ae..ebad94d 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/transactions/ITransactionManager.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/transactions/ITransactionManager.java
@@ -122,7 +122,7 @@
      * @see ITransactionSubsystem
      * @return TransactionProvider
      */
-    public ITransactionSubsystem getTransactionProvider();
+    public ITransactionSubsystem getTransactionSubsystem();
 
     /**
      * @return The current max job id.
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/transactions/ITransactionSubsystem.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/transactions/ITransactionSubsystem.java
index b3a3eba..7bd55e8 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/transactions/ITransactionSubsystem.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/transactions/ITransactionSubsystem.java
@@ -18,19 +18,29 @@
  */
 package org.apache.asterix.common.transactions;
 
+import org.apache.asterix.common.config.TransactionProperties;
+
+/**
+ * A transaction subsystem is responsible of maintaining Atomicity, Consistency, Isolation, and Durability
+ * on a record level.
+ */
 public interface ITransactionSubsystem {
 
-    public ILogManager getLogManager();
+    ILogManager getLogManager();
 
-    public ILockManager getLockManager();
+    ILockManager getLockManager();
 
-    public ITransactionManager getTransactionManager();
+    ITransactionManager getTransactionManager();
 
-    public IRecoveryManager getRecoveryManager();
+    IRecoveryManager getRecoveryManager();
 
-    public IAppRuntimeContextProvider getAsterixAppRuntimeContextProvider();
+    IAppRuntimeContextProvider getAsterixAppRuntimeContextProvider();
 
-    public String getId();
+    String getId();
 
-    public ICheckpointManager getCheckpointManager();
+    ICheckpointManager getCheckpointManager();
+
+    TransactionProperties getTransactionProperties();
+
+    void incrementEntityCommitCount();
 }
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/transactions/Resource.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/transactions/Resource.java
index 0032a70..e53ca7f 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/transactions/Resource.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/transactions/Resource.java
@@ -21,15 +21,20 @@
 import java.io.Serializable;
 import java.util.List;
 
+import org.apache.hyracks.api.application.INCApplicationContext;
 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.IIOManager;
 import org.apache.hyracks.api.io.IODeviceHandle;
+import org.apache.hyracks.storage.am.common.api.IMetadataPageManagerFactory;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallbackFactory;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndex;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMOperationTrackerFactory;
 import org.apache.hyracks.storage.common.file.LocalResource;
 
 /**
+ * TODO(amoudi): Change this class and its subclasses to use json serialization instead of Java serialization
  * The base resource that will be written to disk. it will go in the serializable resource
  * member in {@link LocalResource}
  */
@@ -41,14 +46,22 @@
     protected final ITypeTraits[] filterTypeTraits;
     protected final IBinaryComparatorFactory[] filterCmpFactories;
     protected final int[] filterFields;
+    protected final ILSMOperationTrackerFactory opTrackerProvider;
+    protected final ILSMIOOperationCallbackFactory ioOpCallbackFactory;
+    protected final IMetadataPageManagerFactory metadataPageManagerFactory;
 
     public Resource(int datasetId, int partition, ITypeTraits[] filterTypeTraits,
-            IBinaryComparatorFactory[] filterCmpFactories, int[] filterFields) {
+            IBinaryComparatorFactory[] filterCmpFactories, int[] filterFields,
+            ILSMOperationTrackerFactory opTrackerProvider, ILSMIOOperationCallbackFactory ioOpCallbackFactory,
+            IMetadataPageManagerFactory metadataPageManagerFactory) {
         this.datasetId = datasetId;
         this.partition = partition;
         this.filterTypeTraits = filterTypeTraits;
         this.filterCmpFactories = filterCmpFactories;
         this.filterFields = filterFields;
+        this.opTrackerProvider = opTrackerProvider;
+        this.ioOpCallbackFactory = ioOpCallbackFactory;
+        this.metadataPageManagerFactory = metadataPageManagerFactory;
     }
 
     public int partition() {
@@ -59,8 +72,8 @@
         return datasetId;
     }
 
-    public abstract ILSMIndex createIndexInstance(IAppRuntimeContextProvider runtimeContextProvider,
-            LocalResource resource) throws HyracksDataException;
+    public abstract ILSMIndex createIndexInstance(INCApplicationContext appCtx, LocalResource resource)
+            throws HyracksDataException;
 
     public static int getIoDeviceNum(IIOManager ioManager, IODeviceHandle deviceHandle) {
         List<IODeviceHandle> ioDevices = ioManager.getIODevices();
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/transactions/ResourceFactory.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/transactions/ResourceFactory.java
index f7f7a71..891edd8 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/transactions/ResourceFactory.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/transactions/ResourceFactory.java
@@ -20,6 +20,9 @@
 
 import org.apache.hyracks.api.dataflow.value.IBinaryComparatorFactory;
 import org.apache.hyracks.api.dataflow.value.ITypeTraits;
+import org.apache.hyracks.storage.am.common.api.IMetadataPageManagerFactory;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallbackFactory;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMOperationTrackerFactory;
 
 public abstract class ResourceFactory implements IResourceFactory {
 
@@ -28,12 +31,20 @@
     protected final ITypeTraits[] filterTypeTraits;
     protected final IBinaryComparatorFactory[] filterCmpFactories;
     protected final int[] filterFields;
+    protected final ILSMOperationTrackerFactory opTrackerProvider;
+    protected final ILSMIOOperationCallbackFactory ioOpCallbackFactory;
+    protected final IMetadataPageManagerFactory metadataPageManagerFactory;
 
     public ResourceFactory(int datasetId, ITypeTraits[] filterTypeTraits,
-            IBinaryComparatorFactory[] filterCmpFactories, int[] filterFields) {
+            IBinaryComparatorFactory[] filterCmpFactories, int[] filterFields,
+            ILSMOperationTrackerFactory opTrackerProvider, ILSMIOOperationCallbackFactory ioOpCallbackFactory,
+            IMetadataPageManagerFactory metadataPageManagerFactory) {
         this.datasetId = datasetId;
         this.filterTypeTraits = filterTypeTraits;
         this.filterCmpFactories = filterCmpFactories;
         this.filterFields = filterFields;
+        this.opTrackerProvider = opTrackerProvider;
+        this.ioOpCallbackFactory = ioOpCallbackFactory;
+        this.metadataPageManagerFactory = metadataPageManagerFactory;
     }
 }
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/utils/ANameSchema.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/utils/ANameSchema.java
deleted file mode 100644
index 2ce3cca..0000000
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/utils/ANameSchema.java
+++ /dev/null
@@ -1,29 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.asterix.common.utils;
-
-/*
- * Author: Guangqiang Li
- * Created on Nov 30, 2009
- */
-public class ANameSchema {
-    public static final int LOCALNAME_PROPERTY = 0;
-    public static final int URI = 1;
-    public static final int NODE_TYPE = 2;
-}
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/common/Job.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/utils/Job.java
similarity index 96%
rename from asterixdb/asterix-app/src/main/java/org/apache/asterix/api/common/Job.java
rename to asterixdb/asterix-common/src/main/java/org/apache/asterix/common/utils/Job.java
index 8eebdd0..88de295 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/common/Job.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/utils/Job.java
@@ -16,7 +16,7 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.asterix.api.common;
+package org.apache.asterix.common.utils;
 
 import org.apache.hyracks.api.job.JobSpecification;
 
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/util/JobUtils.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/utils/JobUtils.java
similarity index 92%
rename from asterixdb/asterix-app/src/main/java/org/apache/asterix/util/JobUtils.java
rename to asterixdb/asterix-common/src/main/java/org/apache/asterix/common/utils/JobUtils.java
index fb50b0c..9fb6a31 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/util/JobUtils.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/utils/JobUtils.java
@@ -17,15 +17,19 @@
  * under the License.
  */
 
-package org.apache.asterix.util;
+package org.apache.asterix.common.utils;
 
-import org.apache.asterix.api.common.Job;
 import org.apache.hyracks.api.client.IHyracksClientConnection;
 import org.apache.hyracks.api.job.JobId;
 import org.apache.hyracks.api.job.JobSpecification;
 
 public class JobUtils {
 
+    public enum ProgressState {
+        NO_PROGRESS,
+        ADDED_PENDINGOP_RECORD_TO_METADATA
+    }
+
     public static JobId runJob(IHyracksClientConnection hcc, JobSpecification spec, boolean waitForCompletion)
             throws Exception {
         JobId[] jobIds = executeJobArray(hcc, new Job[] { new Job(spec) }, waitForCompletion);
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/utils/StoragePathUtil.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/utils/StoragePathUtil.java
index 34af5c3..9f2e3e7 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/utils/StoragePathUtil.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/utils/StoragePathUtil.java
@@ -39,6 +39,9 @@
     public static final String DATASET_INDEX_NAME_SEPARATOR = "_idx_";
     public static final String ADAPTER_INSTANCE_PREFIX = "adapter_";
 
+    private StoragePathUtil() {
+    }
+
     public static Pair<IFileSplitProvider, AlgebricksPartitionConstraint> splitProviderAndPartitionConstraints(
             FileSplit[] splits) {
         IFileSplitProvider splitProvider = new ConstantFileSplitProvider(splits);
@@ -67,7 +70,7 @@
     }
 
     private static String prepareFullIndexName(String datasetName, String idxName) {
-        return (datasetName + DATASET_INDEX_NAME_SEPARATOR + idxName);
+        return datasetName + DATASET_INDEX_NAME_SEPARATOR + idxName;
     }
 
     public static int getPartitionNumFromName(String name) {
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/utils/TransactionUtil.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/utils/TransactionUtil.java
index 2878d5a..4ba35ae 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/utils/TransactionUtil.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/utils/TransactionUtil.java
@@ -29,6 +29,11 @@
 
 public class TransactionUtil {
 
+    public static final boolean PROFILE_MODE = false;
+
+    private TransactionUtil() {
+    }
+
     public static void formJobTerminateLogRecord(ITransactionContext txnCtx, LogRecord logRecord, boolean isCommit) {
         logRecord.setTxnCtx(txnCtx);
         TransactionUtil.formJobTerminateLogRecord(logRecord, txnCtx.getJobId().getId(), isCommit);
diff --git a/asterixdb/asterix-common/src/main/resources/asx_errormsg/en.properties b/asterixdb/asterix-common/src/main/resources/asx_errormsg/en.properties
index f0e98c9..f1e9836 100644
--- a/asterixdb/asterix-common/src/main/resources/asx_errormsg/en.properties
+++ b/asterixdb/asterix-common/src/main/resources/asx_errormsg/en.properties
@@ -49,6 +49,19 @@
 1007 = Invalid expression: function %1$s expects its %2$s input parameter to be a %3$s expression, but the actual expression is %4$s
 1008 = Invalid parameter number: function %1$s cannot take %2$s parameters
 1010 = Phrase search in Full-text is not yet supported. Only one keyword per expression is permitted
+1011 = Unknown dataset type %1$s
+1012 = Unknown index type %1$s
+1013 = Cannot use %1$s fields as a key for the %2$s index. The index can only support keys of size %3$s
+1014 = Field \"%1$s\" is not found
+1015 = Index of type %1$s is not supported for dataset \"%2$s\" since it has composite primary keys
+1016 = Index of type %1$s is not supported for dataset of type %2$s
+1017 = The filter field \"%1$s\" cannot be an optional field
+1018 = Field of type %1$s cannot be used as a filter field
+1019 = Cannot autogenerate a composite primary key
+1020 = Cannot autogenerate a primary key for primary key of type %1$s. Autogenerated primary keys must be of type %2$s
+1021 = The primary key field \"%1$s\" cannot be nullable
+1022 = Field of type %1$s cannot be used as a primary key field
+1023 = Can't drop dataset %1$s since it is connected to active entity: %2$s
 
 # Feed Errors
 3001 = Illegal state.
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/api/IExternalDataSourceFactory.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/api/IExternalDataSourceFactory.java
index b35d9c6..e2274b9 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/api/IExternalDataSourceFactory.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/api/IExternalDataSourceFactory.java
@@ -25,8 +25,8 @@
 import java.util.Set;
 
 import org.apache.asterix.common.exceptions.AsterixException;
-import org.apache.asterix.runtime.util.AppContextInfo;
-import org.apache.asterix.runtime.util.ClusterStateManager;
+import org.apache.asterix.runtime.utils.AppContextInfo;
+import org.apache.asterix.runtime.utils.ClusterStateManager;
 import org.apache.hyracks.algebricks.common.constraints.AlgebricksAbsolutePartitionConstraint;
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/dataflow/FeedRecordDataFlowController.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/dataflow/FeedRecordDataFlowController.java
index 859a011..2e687ba 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/dataflow/FeedRecordDataFlowController.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/dataflow/FeedRecordDataFlowController.java
@@ -39,7 +39,7 @@
 import org.apache.hyracks.api.util.HyracksConstants;
 import org.apache.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
 import org.apache.hyracks.dataflow.common.io.MessagingFrameTupleAppender;
-import org.apache.hyracks.dataflow.common.util.TaskUtils;
+import org.apache.hyracks.dataflow.common.utils.TaskUtil;
 import org.apache.log4j.Logger;
 
 public class FeedRecordDataFlowController<T> extends AbstractFeedDataFlowController {
@@ -144,7 +144,7 @@
         ExecutorService executorService = sendMarker ? Executors.newSingleThreadExecutor() : null;
         if (sendMarker && dataflowMarker == null) {
             dataflowMarker = new DataflowMarker(recordReader.getProgressReporter(),
-                    TaskUtils.<VSizeFrame> get(HyracksConstants.KEY_MESSAGE, ctx));
+                    TaskUtil.<VSizeFrame> get(HyracksConstants.KEY_MESSAGE, ctx));
             dataflowMarkerResult = executorService.submit(dataflowMarker);
         }
     }
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/dataflow/FeedTupleForwarder.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/dataflow/FeedTupleForwarder.java
index b8d0532..d31e074 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/dataflow/FeedTupleForwarder.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/dataflow/FeedTupleForwarder.java
@@ -33,7 +33,7 @@
 import org.apache.hyracks.dataflow.common.comm.io.FrameTupleAppender;
 import org.apache.hyracks.dataflow.common.comm.util.FrameUtils;
 import org.apache.hyracks.dataflow.common.io.MessagingFrameTupleAppender;
-import org.apache.hyracks.dataflow.common.util.TaskUtils;
+import org.apache.hyracks.dataflow.common.utils.TaskUtil;
 
 public class FeedTupleForwarder implements ITupleForwarder {
 
@@ -59,7 +59,7 @@
             this.writer = writer;
             this.appender = new FrameTupleAppender(frame);
             // Set null feed message
-            VSizeFrame message = TaskUtils.<VSizeFrame> get(HyracksConstants.KEY_MESSAGE, ctx);
+            VSizeFrame message = TaskUtil.<VSizeFrame> get(HyracksConstants.KEY_MESSAGE, ctx);
             // a null message
             message.getBuffer().put(MessagingFrameTupleAppender.NULL_FEED_MESSAGE);
             message.getBuffer().flip();
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/management/FeedEventsListener.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/management/FeedEventsListener.java
index 64f0b8d..fa48b58 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/management/FeedEventsListener.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/management/FeedEventsListener.java
@@ -45,7 +45,7 @@
 import org.apache.asterix.external.operators.FeedIntakeOperatorDescriptor;
 import org.apache.asterix.external.operators.FeedMetaOperatorDescriptor;
 import org.apache.asterix.external.util.FeedUtils.JobType;
-import org.apache.asterix.runtime.util.AppContextInfo;
+import org.apache.asterix.runtime.utils.AppContextInfo;
 import org.apache.hyracks.algebricks.common.utils.Pair;
 import org.apache.hyracks.algebricks.runtime.base.IPushRuntimeFactory;
 import org.apache.hyracks.algebricks.runtime.operators.meta.AlgebricksMetaOperatorDescriptor;
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/runtime/IngestionRuntime.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/runtime/IngestionRuntime.java
index 34237c4..d32a604 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/runtime/IngestionRuntime.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/runtime/IngestionRuntime.java
@@ -29,7 +29,7 @@
 import org.apache.hyracks.api.context.IHyracksTaskContext;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.api.util.HyracksConstants;
-import org.apache.hyracks.dataflow.common.util.TaskUtils;
+import org.apache.hyracks.dataflow.common.utils.TaskUtil;
 
 public class IngestionRuntime extends SubscribableRuntime {
 
@@ -50,9 +50,9 @@
         dWriter.subscribe(collector);
         subscribers.add(collectionRuntime);
         if (numSubscribers == 0) {
-            TaskUtils.putInSharedMap(HyracksConstants.KEY_MESSAGE, new VSizeFrame(ctx), ctx);
-            TaskUtils.putInSharedMap(HyracksConstants.KEY_MESSAGE,
-                    TaskUtils.<VSizeFrame> get(HyracksConstants.KEY_MESSAGE, ctx), collectionRuntime.getCtx());
+            TaskUtil.putInSharedMap(HyracksConstants.KEY_MESSAGE, new VSizeFrame(ctx), ctx);
+            TaskUtil.putInSharedMap(HyracksConstants.KEY_MESSAGE,
+                    TaskUtil.<VSizeFrame> get(HyracksConstants.KEY_MESSAGE, ctx), collectionRuntime.getCtx());
             start();
         }
         numSubscribers++;
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/indexing/ExternalFile.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/indexing/ExternalFile.java
index 7acb1f8..2aac497 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/indexing/ExternalFile.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/indexing/ExternalFile.java
@@ -48,7 +48,7 @@
         this.fileName = "";
         this.lastModefiedTime = new Date();
         this.size = 0;
-        this.pendingOp = ExternalFilePendingOp.PENDING_NO_OP;
+        this.pendingOp = ExternalFilePendingOp.NO_OP;
     }
 
     public ExternalFile(String dataverseName, String datasetName, int fileNumber, String fileName,
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
index 3ac8da8..152fa8b 100644
--- 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
@@ -37,12 +37,13 @@
 
 @SuppressWarnings("rawtypes")
 public class FilesIndexDescription {
-    public final static int FILE_INDEX_TUPLE_SIZE = 2;
-    public final static int FILE_KEY_INDEX = 0;
-    public final static int FILE_KEY_SIZE = 1;
-    public final static int FILE_PAYLOAD_INDEX = 1;
-    public final static String[] payloadFieldNames = { "FileName", "FileSize", "FileModDate" };
-    public final static IAType[] payloadFieldTypes = { BuiltinType.ASTRING, BuiltinType.AINT64, BuiltinType.ADATETIME };
+    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;
@@ -58,7 +59,7 @@
     public final RecordDescriptor FILE_BUDDY_BTREE_RECORD_DESCRIPTOR;
     public final ISerializerDeserializer[] EXTERNAL_FILE_BUDDY_BTREE_FIELDS = new ISerializerDeserializer[1];
     public final ISerializerDeserializer[] EXTERNAL_FILE_TUPLE_FIELDS = new ISerializerDeserializer[FILE_INDEX_TUPLE_SIZE];
-    public final IBinaryComparatorFactory[] FILES_INDEX_COMP_FACTORIES = new IBinaryComparatorFactory[] {
+    public static final IBinaryComparatorFactory[] FILES_INDEX_COMP_FACTORIES = new IBinaryComparatorFactory[] {
             BinaryComparatorFactoryProvider.INSTANCE.getBinaryComparatorFactory(BuiltinType.AINT32, true) };
 
     public FilesIndexDescription() {
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/indexing/IndexingConstants.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/indexing/IndexingConstants.java
index a269144..86f5aa5 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/indexing/IndexingConstants.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/indexing/IndexingConstants.java
@@ -20,6 +20,7 @@
 
 import java.util.ArrayList;
 import java.util.Arrays;
+import java.util.Collections;
 import java.util.List;
 import java.util.Map;
 
@@ -81,7 +82,7 @@
     public static final int RECORD_OFFSET_FIELD_INDEX = 1;
     public static final int ROW_NUMBER_FIELD_INDEX = 2;
 
-    public static final ArrayList<List<String>> RecordIDFields = new ArrayList<List<String>>();
+    public static final List<List<String>> RECORD_ID_FIELDS;
 
     static {
 
@@ -101,25 +102,32 @@
         rowNumberEvalFactory = new TupleFieldEvaluatorFactory(3);
 
         // Add field names
-        RecordIDFields.add(new ArrayList<String>(Arrays.asList("FileNumber")));
-        RecordIDFields.add(new ArrayList<String>(Arrays.asList("RecordOffset")));
-        RecordIDFields.add(new ArrayList<String>(Arrays.asList("RowNumber")));
+        List<List<String>> ridFields = new ArrayList<>();
+        ridFields.add(new ArrayList<>(Arrays.asList("FileNumber")));
+        ridFields.add(new ArrayList<>(Arrays.asList("RecordOffset")));
+        ridFields.add(new ArrayList<>(Arrays.asList("RowNumber")));
+        RECORD_ID_FIELDS = Collections.unmodifiableList(ridFields);
+    }
+
+    private IndexingConstants() {
     }
 
     // This function returns the size of the RID for the passed file input format
     public static int getRIDSize(String fileInputFormat) {
-        if (fileInputFormat.equals(INPUT_FORMAT_RC) || fileInputFormat.equals(INPUT_FORMAT_RC_FULLY_QUALIFIED))
+        if (fileInputFormat.equals(INPUT_FORMAT_RC) || fileInputFormat.equals(INPUT_FORMAT_RC_FULLY_QUALIFIED)) {
             return 3;
-        else
+        } else {
             return 2;
+        }
     }
 
     // This function returns the size of the RID for the passed file input format
     public static IBinaryComparatorFactory[] getComparatorFactories(String fileInputFormat) {
-        if (fileInputFormat.equals(INPUT_FORMAT_RC) || fileInputFormat.equals(INPUT_FORMAT_RC_FULLY_QUALIFIED))
+        if (fileInputFormat.equals(INPUT_FORMAT_RC) || fileInputFormat.equals(INPUT_FORMAT_RC_FULLY_QUALIFIED)) {
             return rCFileRIDComparatorFactories;
-        else
+        } else {
             return txtSeqFileRIDComparatorFactories;
+        }
     }
 
     public static IAType getFieldType(int fieldNumber) throws AsterixException {
@@ -197,9 +205,14 @@
 
     public static List<List<String>> getRIDKeys(Map<String, String> properties) {
         String fileInputFormat = properties.get(KEY_INPUT_FORMAT);
-        if (fileInputFormat.equals(INPUT_FORMAT_RC) || fileInputFormat.equals(INPUT_FORMAT_RC_FULLY_QUALIFIED))
-            return RecordIDFields;
-        else
-            return RecordIDFields.subList(0, ROW_NUMBER_FIELD_INDEX);
+        if (fileInputFormat.equals(INPUT_FORMAT_RC) || fileInputFormat.equals(INPUT_FORMAT_RC_FULLY_QUALIFIED)) {
+            return RECORD_ID_FIELDS;
+        } else {
+            return RECORD_ID_FIELDS.subList(0, ROW_NUMBER_FIELD_INDEX);
+        }
+    }
+
+    public static String getFilesIndexName(String datasetName) {
+        return datasetName.concat(IndexingConstants.EXTERNAL_FILE_INDEX_NAME_SUFFIX);
     }
 }
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/RecordWithMetadataAndPK.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/RecordWithMetadataAndPK.java
index c2f74e4..ebf1d17 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/RecordWithMetadataAndPK.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/RecordWithMetadataAndPK.java
@@ -40,7 +40,7 @@
 import org.apache.asterix.om.types.ATypeTag;
 import org.apache.asterix.om.types.BuiltinType;
 import org.apache.asterix.om.types.IAType;
-import org.apache.asterix.om.util.NonTaggedFormatUtil;
+import org.apache.asterix.om.utils.NonTaggedFormatUtil;
 import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/converter/DCPMessageToRecordConverter.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/converter/DCPMessageToRecordConverter.java
index 38e5fa3..f948c1c 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/converter/DCPMessageToRecordConverter.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/converter/DCPMessageToRecordConverter.java
@@ -30,9 +30,9 @@
 import org.apache.asterix.external.input.record.CharArrayRecord;
 import org.apache.asterix.external.input.record.RecordWithMetadataAndPK;
 import org.apache.asterix.external.util.ExternalDataConstants;
-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.om.utils.RecordUtil;
 
 import com.couchbase.client.core.message.dcp.DCPRequest;
 import com.couchbase.client.core.message.dcp.MutationMessage;
@@ -57,7 +57,7 @@
     public DCPMessageToRecordConverter() {
         this.value = new CharArrayRecord();
         this.recordWithMetadata = new RecordWithMetadataAndPK<>(value, CB_META_TYPES,
-                ARecordType.FULLY_OPEN_RECORD_TYPE, PK_INDICATOR, PK_INDEXES, PK_TYPES);
+                RecordUtil.FULLY_OPEN_RECORD_TYPE, PK_INDICATOR, PK_INDEXES, PK_TYPES);
     }
 
     @Override
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/stream/factory/SocketServerInputStreamFactory.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/stream/factory/SocketServerInputStreamFactory.java
index aafee8e..5337be1 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/stream/factory/SocketServerInputStreamFactory.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/stream/factory/SocketServerInputStreamFactory.java
@@ -32,7 +32,7 @@
 import org.apache.asterix.external.api.IInputStreamFactory;
 import org.apache.asterix.external.input.stream.SocketServerInputStream;
 import org.apache.asterix.external.util.ExternalDataConstants;
-import org.apache.asterix.runtime.util.RuntimeUtils;
+import org.apache.asterix.runtime.utils.RuntimeUtils;
 import org.apache.commons.lang3.StringUtils;
 import org.apache.hyracks.algebricks.common.constraints.AlgebricksAbsolutePartitionConstraint;
 import org.apache.hyracks.algebricks.common.utils.Pair;
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/library/ExternalFunction.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/library/ExternalFunction.java
index 1b89b0c..c80aebc 100755
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/library/ExternalFunction.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/library/ExternalFunction.java
@@ -32,7 +32,7 @@
 import org.apache.asterix.om.types.ATypeTag;
 import org.apache.asterix.om.types.EnumDeserializer;
 import org.apache.asterix.om.types.hierachy.ATypeHierarchy;
-import org.apache.asterix.runtime.util.AppContextInfo;
+import org.apache.asterix.runtime.utils.AppContextInfo;
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
 import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
 import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/library/java/JObjectUtil.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/library/java/JObjectUtil.java
index 95e566b..a107421 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/library/java/JObjectUtil.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/library/java/JObjectUtil.java
@@ -56,8 +56,8 @@
 import org.apache.asterix.om.types.BuiltinType;
 import org.apache.asterix.om.types.EnumDeserializer;
 import org.apache.asterix.om.types.IAType;
-import org.apache.asterix.om.util.NonTaggedFormatUtil;
 import org.apache.asterix.om.util.container.IObjectPool;
+import org.apache.asterix.om.utils.NonTaggedFormatUtil;
 
 public class JObjectUtil {
 
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
index 8080f81..d9471d1 100644
--- 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
@@ -32,9 +32,6 @@
 import org.apache.hyracks.dataflow.std.base.AbstractOperatorNodePushable;
 import org.apache.hyracks.dataflow.std.base.AbstractSingleActivityOperatorDescriptor;
 import org.apache.hyracks.storage.am.common.dataflow.IIndexDataflowHelperFactory;
-import org.apache.hyracks.storage.am.lsm.btree.dataflow.ExternalBTreeDataflowHelperFactory;
-import org.apache.hyracks.storage.am.lsm.btree.dataflow.ExternalBTreeWithBuddyDataflowHelperFactory;
-import org.apache.hyracks.storage.am.lsm.rtree.dataflow.ExternalRTreeDataflowHelperFactory;
 
 // This is an operator that takes a single file index and an array of secondary indexes
 // it is intended to be used for
@@ -45,27 +42,20 @@
         extends AbstractSingleActivityOperatorDescriptor {
 
     private static final long serialVersionUID = 1L;
-    private ExternalBTreeDataflowHelperFactory filesIndexDataflowHelperFactory;
+    private IIndexDataflowHelperFactory filesIndexDataflowHelperFactory;
     private IndexInfoOperatorDescriptor fileIndexInfo;
-    private List<ExternalBTreeWithBuddyDataflowHelperFactory> bTreeIndexesDataflowHelperFactories;
-    private List<IndexInfoOperatorDescriptor> bTreeIndexesInfos;
-    private List<ExternalRTreeDataflowHelperFactory> rTreeIndexesDataflowHelperFactories;
-    private List<IndexInfoOperatorDescriptor> rTreeIndexesInfos;
+    private List<IIndexDataflowHelperFactory> treeIndexesDataflowHelperFactories;
+    private List<IndexInfoOperatorDescriptor> treeIndexesInfos;
 
     public AbstractExternalDatasetIndexesOperatorDescriptor(IOperatorDescriptorRegistry spec,
-            ExternalBTreeDataflowHelperFactory filesIndexDataflowHelperFactory,
-            IndexInfoOperatorDescriptor fileIndexesInfo,
-            List<ExternalBTreeWithBuddyDataflowHelperFactory> bTreeIndexesDataflowHelperFactories,
-            List<IndexInfoOperatorDescriptor> bTreeIndexesInfos,
-            List<ExternalRTreeDataflowHelperFactory> rTreeIndexesDataflowHelperFactories,
-            List<IndexInfoOperatorDescriptor> rTreeIndexesInfos) {
+            IIndexDataflowHelperFactory filesIndexDataflowHelperFactory, IndexInfoOperatorDescriptor fileIndexesInfo,
+            List<IIndexDataflowHelperFactory> treeIndexesDataflowHelperFactories,
+            List<IndexInfoOperatorDescriptor> indexesInfos) {
         super(spec, 0, 0);
         this.filesIndexDataflowHelperFactory = filesIndexDataflowHelperFactory;
         this.fileIndexInfo = fileIndexesInfo;
-        this.bTreeIndexesDataflowHelperFactories = bTreeIndexesDataflowHelperFactories;
-        this.bTreeIndexesInfos = bTreeIndexesInfos;
-        this.rTreeIndexesDataflowHelperFactories = rTreeIndexesDataflowHelperFactories;
-        this.rTreeIndexesInfos = rTreeIndexesInfos;
+        this.treeIndexesDataflowHelperFactories = treeIndexesDataflowHelperFactories;
+        this.treeIndexesInfos = indexesInfos;
     }
 
     // opening and closing the index is done inside these methods since we don't always need open indexes
@@ -87,13 +77,8 @@
                         performOpOnIndex(filesIndexDataflowHelperFactory, ctx, fileIndexInfo, partition);
                     }
                     // perform operation on btrees
-                    for (int i = 0; i < bTreeIndexesDataflowHelperFactories.size(); i++) {
-                        performOpOnIndex(bTreeIndexesDataflowHelperFactories.get(i), ctx, bTreeIndexesInfos.get(i),
-                                partition);
-                    }
-                    // perform operation on rtrees
-                    for (int i = 0; i < rTreeIndexesDataflowHelperFactories.size(); i++) {
-                        performOpOnIndex(rTreeIndexesDataflowHelperFactories.get(i), ctx, rTreeIndexesInfos.get(i),
+                    for (int i = 0; i < treeIndexesDataflowHelperFactories.size(); i++) {
+                        performOpOnIndex(treeIndexesDataflowHelperFactories.get(i), ctx, treeIndexesInfos.get(i),
                                 partition);
                     }
                 } catch (Exception e) {
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
index 5f4e159..ac715a0 100644
--- 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
@@ -30,27 +30,27 @@
 import org.apache.hyracks.dataflow.std.file.IFileSplitProvider;
 import org.apache.hyracks.storage.am.btree.dataflow.BTreeSearchOperatorDescriptor;
 import org.apache.hyracks.storage.am.common.api.IIndexLifecycleManagerProvider;
-import org.apache.hyracks.storage.am.common.api.IMetadataPageManagerFactory;
+import org.apache.hyracks.storage.am.common.api.IPageManagerFactory;
 import org.apache.hyracks.storage.am.common.api.ISearchOperationCallbackFactory;
 import org.apache.hyracks.storage.am.common.dataflow.IIndexDataflowHelperFactory;
-import org.apache.hyracks.storage.common.IStorageManagerInterface;
+import org.apache.hyracks.storage.common.IStorageManager;
 
 public class ExternalBTreeSearchOperatorDescriptor extends BTreeSearchOperatorDescriptor {
 
     private static final long serialVersionUID = 1L;
 
     public ExternalBTreeSearchOperatorDescriptor(IOperatorDescriptorRegistry spec, RecordDescriptor recDesc,
-            IStorageManagerInterface storageManager, IIndexLifecycleManagerProvider lifecycleManagerProvider,
+            IStorageManager storageManager, IIndexLifecycleManagerProvider lifecycleManagerProvider,
             IFileSplitProvider fileSplitProvider, ITypeTraits[] typeTraits,
             IBinaryComparatorFactory[] comparatorFactories, int[] bloomFilterKeyFields, int[] lowKeyFields,
             int[] highKeyFields, boolean lowKeyInclusive, boolean highKeyInclusive,
             IIndexDataflowHelperFactory dataflowHelperFactory, boolean retainInput, boolean retainMissing,
             IMissingWriterFactory missingWriterFactory, ISearchOperationCallbackFactory searchOpCallbackProvider,
-            IMetadataPageManagerFactory metadataPageManagerFactory) {
+            IPageManagerFactory pageManagerFactory) {
         super(spec, recDesc, storageManager, lifecycleManagerProvider, fileSplitProvider, typeTraits,
                 comparatorFactories, bloomFilterKeyFields, lowKeyFields, highKeyFields, lowKeyInclusive,
                 highKeyInclusive, dataflowHelperFactory, retainInput, retainMissing, missingWriterFactory,
-                searchOpCallbackProvider, null, null, metadataPageManagerFactory);
+                searchOpCallbackProvider, null, null, pageManagerFactory);
     }
 
     @Override
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
index 8321074..3ce0da8 100644
--- 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
@@ -25,24 +25,18 @@
 import org.apache.hyracks.api.job.IOperatorDescriptorRegistry;
 import org.apache.hyracks.storage.am.common.dataflow.IIndexDataflowHelperFactory;
 import org.apache.hyracks.storage.am.common.util.IndexFileNameUtil;
-import org.apache.hyracks.storage.am.lsm.btree.dataflow.ExternalBTreeDataflowHelperFactory;
-import org.apache.hyracks.storage.am.lsm.btree.dataflow.ExternalBTreeWithBuddyDataflowHelperFactory;
 import org.apache.hyracks.storage.am.lsm.common.impls.AbortRecoverLSMIndexFileManager;
-import org.apache.hyracks.storage.am.lsm.rtree.dataflow.ExternalRTreeDataflowHelperFactory;
 
 public class ExternalDatasetIndexesAbortOperatorDescriptor extends AbstractExternalDatasetIndexesOperatorDescriptor {
 
     private static final long serialVersionUID = 1L;
 
     public ExternalDatasetIndexesAbortOperatorDescriptor(IOperatorDescriptorRegistry spec,
-            ExternalBTreeDataflowHelperFactory filesIndexDataflowHelperFactory,
-            IndexInfoOperatorDescriptor fileIndexesInfo,
-            List<ExternalBTreeWithBuddyDataflowHelperFactory> bTreeIndexesDataflowHelperFactories,
-            List<IndexInfoOperatorDescriptor> bTreeIndexesInfos,
-            List<ExternalRTreeDataflowHelperFactory> rTreeIndexesDataflowHelperFactories,
-            List<IndexInfoOperatorDescriptor> rTreeIndexesInfos) {
-        super(spec, filesIndexDataflowHelperFactory, fileIndexesInfo, bTreeIndexesDataflowHelperFactories,
-                bTreeIndexesInfos, rTreeIndexesDataflowHelperFactories, rTreeIndexesInfos);
+            IIndexDataflowHelperFactory filesIndexDataflowHelperFactory, IndexInfoOperatorDescriptor fileIndexesInfo,
+            List<IIndexDataflowHelperFactory> indexesDataflowHelperFactories,
+            List<IndexInfoOperatorDescriptor> indexesInfos) {
+        super(spec, filesIndexDataflowHelperFactory, fileIndexesInfo, indexesDataflowHelperFactories,
+                indexesInfos);
     }
 
     @Override
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
index fb6e0f9..2ddb646 100644
--- 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
@@ -29,39 +29,32 @@
 import org.apache.hyracks.storage.am.common.api.IndexException;
 import org.apache.hyracks.storage.am.common.dataflow.IIndexDataflowHelperFactory;
 import org.apache.hyracks.storage.am.common.util.IndexFileNameUtil;
-import org.apache.hyracks.storage.am.lsm.btree.dataflow.ExternalBTreeDataflowHelperFactory;
-import org.apache.hyracks.storage.am.lsm.btree.dataflow.ExternalBTreeWithBuddyDataflowHelperFactory;
 import org.apache.hyracks.storage.am.lsm.common.api.ITwoPCIndex;
-import org.apache.hyracks.storage.am.lsm.rtree.dataflow.ExternalRTreeDataflowHelperFactory;
 import org.apache.log4j.Logger;
 
 public class ExternalDatasetIndexesCommitOperatorDescriptor extends AbstractExternalDatasetIndexesOperatorDescriptor {
-    private static final Logger LOGGER = Logger
-            .getLogger(ExternalDatasetIndexesCommitOperatorDescriptor.class.getName());
+    private static final long serialVersionUID = 1L;
+    private static final Logger LOGGER =
+            Logger.getLogger(ExternalDatasetIndexesCommitOperatorDescriptor.class.getName());
 
     public ExternalDatasetIndexesCommitOperatorDescriptor(IOperatorDescriptorRegistry spec,
-            ExternalBTreeDataflowHelperFactory filesIndexDataflowHelperFactory,
-            IndexInfoOperatorDescriptor fileIndexesInfo,
-            List<ExternalBTreeWithBuddyDataflowHelperFactory> bTreeIndexesDataflowHelperFactories,
-            List<IndexInfoOperatorDescriptor> bTreeIndexesInfos,
-            List<ExternalRTreeDataflowHelperFactory> rTreeIndexesDataflowHelperFactories,
-            List<IndexInfoOperatorDescriptor> rTreeIndexesInfos) {
-        super(spec, filesIndexDataflowHelperFactory, fileIndexesInfo, bTreeIndexesDataflowHelperFactories,
-                bTreeIndexesInfos, rTreeIndexesDataflowHelperFactories, rTreeIndexesInfos);
+            IIndexDataflowHelperFactory filesIndexDataflowHelperFactory, IndexInfoOperatorDescriptor fileIndexesInfo,
+            List<IIndexDataflowHelperFactory> indexesDataflowHelperFactories,
+            List<IndexInfoOperatorDescriptor> indexesInfos) {
+        super(spec, filesIndexDataflowHelperFactory, fileIndexesInfo, indexesDataflowHelperFactories,
+                indexesInfos);
     }
 
-    private static final long serialVersionUID = 1L;
-
     @Override
     protected void performOpOnIndex(IIndexDataflowHelperFactory indexDataflowHelperFactory, IHyracksTaskContext ctx,
             IndexInfoOperatorDescriptor fileIndexInfo, int partition) {
         try {
-            FileReference resourecePath = IndexFileNameUtil.getIndexAbsoluteFileRef(fileIndexInfo, partition,
-                    ctx.getIOManager());
+            FileReference resourecePath =
+                    IndexFileNameUtil.getIndexAbsoluteFileRef(fileIndexInfo, partition, ctx.getIOManager());
             LOGGER.warn("performing the operation on " + resourecePath.getFile().getAbsolutePath());
             // Get DataflowHelper
-            IIndexDataflowHelper indexHelper = indexDataflowHelperFactory.createIndexDataflowHelper(fileIndexInfo, ctx,
-                    partition);
+            IIndexDataflowHelper indexHelper =
+                    indexDataflowHelperFactory.createIndexDataflowHelper(fileIndexInfo, ctx, partition);
             // Get index
             IIndex index = indexHelper.getIndexInstance();
             // commit transaction
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
index 0312eac..f74f4e9 100644
--- 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
@@ -25,24 +25,18 @@
 import org.apache.hyracks.api.job.IOperatorDescriptorRegistry;
 import org.apache.hyracks.storage.am.common.dataflow.IIndexDataflowHelperFactory;
 import org.apache.hyracks.storage.am.common.util.IndexFileNameUtil;
-import org.apache.hyracks.storage.am.lsm.btree.dataflow.ExternalBTreeDataflowHelperFactory;
-import org.apache.hyracks.storage.am.lsm.btree.dataflow.ExternalBTreeWithBuddyDataflowHelperFactory;
 import org.apache.hyracks.storage.am.lsm.common.impls.AbortRecoverLSMIndexFileManager;
-import org.apache.hyracks.storage.am.lsm.rtree.dataflow.ExternalRTreeDataflowHelperFactory;
 
 public class ExternalDatasetIndexesRecoverOperatorDescriptor extends AbstractExternalDatasetIndexesOperatorDescriptor {
 
     private static final long serialVersionUID = 1L;
 
     public ExternalDatasetIndexesRecoverOperatorDescriptor(IOperatorDescriptorRegistry spec,
-            ExternalBTreeDataflowHelperFactory filesIndexDataflowHelperFactory,
-            IndexInfoOperatorDescriptor fileIndexesInfo,
-            List<ExternalBTreeWithBuddyDataflowHelperFactory> bTreeIndexesDataflowHelperFactories,
-            List<IndexInfoOperatorDescriptor> bTreeIndexesInfos,
-            List<ExternalRTreeDataflowHelperFactory> rTreeIndexesDataflowHelperFactories,
-            List<IndexInfoOperatorDescriptor> rTreeIndexesInfos) {
-        super(spec, filesIndexDataflowHelperFactory, fileIndexesInfo, bTreeIndexesDataflowHelperFactories,
-                bTreeIndexesInfos, rTreeIndexesDataflowHelperFactories, rTreeIndexesInfos);
+            IIndexDataflowHelperFactory filesIndexDataflowHelperFactory, IndexInfoOperatorDescriptor fileIndexesInfo,
+            List<IIndexDataflowHelperFactory> indexesDataflowHelperFactories,
+            List<IndexInfoOperatorDescriptor> indexesInfos) {
+        super(spec, filesIndexDataflowHelperFactory, fileIndexesInfo, indexesDataflowHelperFactories,
+                indexesInfos);
     }
 
     @Override
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/operators/ExternalFilesIndexOperatorDescriptor.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/operators/ExternalFilesIndexOperatorDescriptor.java
index ec82ede..2bb986d 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/operators/ExternalFilesIndexOperatorDescriptor.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/operators/ExternalFilesIndexOperatorDescriptor.java
@@ -21,7 +21,6 @@
 import java.io.IOException;
 import java.util.List;
 
-import org.apache.asterix.common.exceptions.AsterixException;
 import org.apache.asterix.external.indexing.ExternalFile;
 import org.apache.asterix.external.indexing.FileIndexTupleTranslator;
 import org.apache.asterix.external.indexing.FilesIndexDescription;
@@ -46,7 +45,7 @@
 import org.apache.hyracks.storage.am.common.impls.NoOpOperationCallbackFactory;
 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.IStorageManagerInterface;
+import org.apache.hyracks.storage.common.IStorageManager;
 import org.apache.hyracks.storage.common.file.ILocalResourceFactoryProvider;
 
 /**
@@ -63,13 +62,13 @@
     private List<ExternalFile> files;
 
     public ExternalFilesIndexOperatorDescriptor(IOperatorDescriptorRegistry spec,
-            IStorageManagerInterface storageManager, IIndexLifecycleManagerProvider lifecycleManagerProvider,
+            IStorageManager storageManager, IIndexLifecycleManagerProvider lifecycleManagerProvider,
             IFileSplitProvider fileSplitProvider, IIndexDataflowHelperFactory dataflowHelperFactory,
             ILocalResourceFactoryProvider localResourceFactoryProvider, List<ExternalFile> files,
             boolean createNewIndex, IMetadataPageManagerFactory metadataPageManagerFactory) {
         super(spec, 0, 0, null, storageManager, lifecycleManagerProvider, fileSplitProvider,
                 new FilesIndexDescription().EXTERNAL_FILE_INDEX_TYPE_TRAITS,
-                new FilesIndexDescription().FILES_INDEX_COMP_FACTORIES, FilesIndexDescription.BLOOM_FILTER_FIELDS,
+                FilesIndexDescription.FILES_INDEX_COMP_FACTORIES, FilesIndexDescription.BLOOM_FILTER_FIELDS,
                 dataflowHelperFactory, null, false, false, null, localResourceFactoryProvider,
                 NoOpOperationCallbackFactory.INSTANCE, NoOpOperationCallbackFactory.INSTANCE,
                 metadataPageManagerFactory);
@@ -121,11 +120,11 @@
                         // The files must be ordered according to their numbers
                         for (ExternalFile file : files) {
                             switch (file.getPendingOp()) {
-                                case PENDING_ADD_OP:
-                                case PENDING_APPEND_OP:
+                                case ADD_OP:
+                                case APPEND_OP:
                                     bulkLoader.add(filesTupleTranslator.getTupleFromFile(file));
                                     break;
-                                case PENDING_DROP_OP:
+                                case DROP_OP:
                                     bulkLoader.delete(filesTupleTranslator.getTupleFromFile(file));
                                     break;
                             }
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
index 59352a0..a994c1f 100644
--- 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
@@ -32,7 +32,7 @@
 import org.apache.hyracks.storage.am.common.dataflow.AbstractTreeIndexOperatorDescriptor;
 import org.apache.hyracks.storage.am.common.dataflow.IIndexDataflowHelperFactory;
 import org.apache.hyracks.storage.am.common.impls.NoOpOperationCallbackFactory;
-import org.apache.hyracks.storage.common.IStorageManagerInterface;
+import org.apache.hyracks.storage.common.IStorageManager;
 import org.apache.hyracks.storage.common.file.NoOpLocalResourceFactoryProvider;
 
 public class ExternalIndexBulkModifyOperatorDescriptor extends AbstractTreeIndexOperatorDescriptor {
@@ -44,7 +44,7 @@
     private final long numElementsHint;
 
     public ExternalIndexBulkModifyOperatorDescriptor(IOperatorDescriptorRegistry spec,
-            IStorageManagerInterface storageManager, IIndexLifecycleManagerProvider lifecycleManagerProvider,
+            IStorageManager storageManager, IIndexLifecycleManagerProvider lifecycleManagerProvider,
             IFileSplitProvider fileSplitProvider, ITypeTraits[] typeTraits,
             IBinaryComparatorFactory[] comparatorFactories, int[] bloomFilterKeyFields,
             IIndexDataflowHelperFactory dataflowHelperFactory,
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
index 24e9cbe..d4718a9 100644
--- 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
@@ -34,12 +34,12 @@
 import org.apache.hyracks.dataflow.std.base.AbstractUnaryInputUnaryOutputOperatorNodePushable;
 import org.apache.hyracks.dataflow.std.file.IFileSplitProvider;
 import org.apache.hyracks.storage.am.common.api.IIndexLifecycleManagerProvider;
-import org.apache.hyracks.storage.am.common.api.IMetadataPageManagerFactory;
+import org.apache.hyracks.storage.am.common.api.IPageManagerFactory;
 import org.apache.hyracks.storage.am.common.api.ISearchOperationCallbackFactory;
 import org.apache.hyracks.storage.am.common.dataflow.AbstractTreeIndexOperatorDescriptor;
+import org.apache.hyracks.storage.am.common.dataflow.IIndexDataflowHelperFactory;
 import org.apache.hyracks.storage.am.lsm.btree.dataflow.ExternalBTreeDataflowHelper;
-import org.apache.hyracks.storage.am.lsm.btree.dataflow.ExternalBTreeDataflowHelperFactory;
-import org.apache.hyracks.storage.common.IStorageManagerInterface;
+import org.apache.hyracks.storage.common.IStorageManager;
 
 /*
  * This operator is intended for using record ids to access data in external sources
@@ -49,17 +49,17 @@
     private final LookupAdapterFactory<?> adapterFactory;
 
     public ExternalLookupOperatorDescriptor(IOperatorDescriptorRegistry spec, LookupAdapterFactory<?> adapterFactory,
-            RecordDescriptor outRecDesc, ExternalBTreeDataflowHelperFactory externalFilesIndexDataFlowHelperFactory,
+            RecordDescriptor outRecDesc, IIndexDataflowHelperFactory externalFilesIndexDataFlowHelperFactory,
             boolean propagateInput, IIndexLifecycleManagerProvider lcManagerProvider,
-            IStorageManagerInterface storageManager, IFileSplitProvider fileSplitProvider, int datasetId,
-            double bloomFilterFalsePositiveRate, ISearchOperationCallbackFactory searchOpCallbackFactory,
+            IStorageManager storageManager, IFileSplitProvider fileSplitProvider,
+            ISearchOperationCallbackFactory searchOpCallbackFactory,
             boolean retainMissing, IMissingWriterFactory missingWriterFactory,
-            IMetadataPageManagerFactory metadataPageManagerFactory) {
+            IPageManagerFactory pageManagerFactory) {
         super(spec, 1, 1, outRecDesc, storageManager, lcManagerProvider, fileSplitProvider,
                 new FilesIndexDescription().EXTERNAL_FILE_INDEX_TYPE_TRAITS,
-                new FilesIndexDescription().FILES_INDEX_COMP_FACTORIES, FilesIndexDescription.BLOOM_FILTER_FIELDS,
+                FilesIndexDescription.FILES_INDEX_COMP_FACTORIES, FilesIndexDescription.BLOOM_FILTER_FIELDS,
                 externalFilesIndexDataFlowHelperFactory, null, propagateInput, retainMissing, missingWriterFactory,
-                null, searchOpCallbackFactory, null, metadataPageManagerFactory);
+                null, searchOpCallbackFactory, null, pageManagerFactory);
         this.adapterFactory = adapterFactory;
     }
 
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
index 1a5eeef..22960e4 100644
--- 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
@@ -29,26 +29,26 @@
 import org.apache.hyracks.api.job.IOperatorDescriptorRegistry;
 import org.apache.hyracks.dataflow.std.file.IFileSplitProvider;
 import org.apache.hyracks.storage.am.common.api.IIndexLifecycleManagerProvider;
-import org.apache.hyracks.storage.am.common.api.IMetadataPageManagerFactory;
+import org.apache.hyracks.storage.am.common.api.IPageManagerFactory;
 import org.apache.hyracks.storage.am.common.api.ISearchOperationCallbackFactory;
-import org.apache.hyracks.storage.am.lsm.rtree.dataflow.ExternalRTreeDataflowHelperFactory;
+import org.apache.hyracks.storage.am.common.dataflow.IIndexDataflowHelperFactory;
 import org.apache.hyracks.storage.am.rtree.dataflow.RTreeSearchOperatorDescriptor;
-import org.apache.hyracks.storage.common.IStorageManagerInterface;
+import org.apache.hyracks.storage.common.IStorageManager;
 
 public class ExternalRTreeSearchOperatorDescriptor extends RTreeSearchOperatorDescriptor {
 
     private static final long serialVersionUID = 1L;
 
     public ExternalRTreeSearchOperatorDescriptor(IOperatorDescriptorRegistry spec, RecordDescriptor recDesc,
-            IStorageManagerInterface storageManager, IIndexLifecycleManagerProvider lifecycleManagerProvider,
+            IStorageManager storageManager, IIndexLifecycleManagerProvider lifecycleManagerProvider,
             IFileSplitProvider fileSplitProvider, ITypeTraits[] typeTraits,
             IBinaryComparatorFactory[] comparatorFactories, int[] keyFields,
-            ExternalRTreeDataflowHelperFactory dataflowHelperFactory, boolean retainInput, boolean retainMissing,
+            IIndexDataflowHelperFactory dataflowHelperFactory, boolean retainInput, boolean retainMissing,
             IMissingWriterFactory missingWriterFactory, ISearchOperationCallbackFactory searchOpCallbackFactory,
-            IMetadataPageManagerFactory metadataPageManagerFactory) {
+            IPageManagerFactory pageManagerFactory) {
         super(spec, recDesc, storageManager, lifecycleManagerProvider, fileSplitProvider, typeTraits,
                 comparatorFactories, keyFields, dataflowHelperFactory, retainInput, retainMissing, missingWriterFactory,
-                searchOpCallbackFactory, null, null, metadataPageManagerFactory);
+                searchOpCallbackFactory, null, null, pageManagerFactory);
     }
 
     @Override
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/operators/ExternalDataScanOperatorDescriptor.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/operators/ExternalScanOperatorDescriptor.java
similarity index 88%
rename from asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/operators/ExternalDataScanOperatorDescriptor.java
rename to asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/operators/ExternalScanOperatorDescriptor.java
index aed8bb9..96a6848 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/operators/ExternalDataScanOperatorDescriptor.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/operators/ExternalScanOperatorDescriptor.java
@@ -33,13 +33,13 @@
  * A single activity operator that provides the functionality of scanning data using an
  * instance of the configured adapter.
  */
-public class ExternalDataScanOperatorDescriptor extends AbstractSingleActivityOperatorDescriptor {
+public class ExternalScanOperatorDescriptor extends AbstractSingleActivityOperatorDescriptor {
 
     private static final long serialVersionUID = 1L;
 
     private IAdapterFactory adapterFactory;
 
-    public ExternalDataScanOperatorDescriptor(JobSpecification spec, RecordDescriptor rDesc,
+    public ExternalScanOperatorDescriptor(JobSpecification spec, RecordDescriptor rDesc,
             IAdapterFactory dataSourceAdapterFactory) {
         super(spec, 0, 1);
         recordDescriptors[0] = rDesc;
@@ -49,7 +49,7 @@
     @Override
     public IOperatorNodePushable createPushRuntime(final IHyracksTaskContext ctx,
             IRecordDescriptorProvider recordDescProvider, final int partition, final int nPartitions)
-                    throws HyracksDataException {
+            throws HyracksDataException {
 
         return new AbstractUnaryOutputSourceOperatorNodePushable() {
 
@@ -60,9 +60,9 @@
                     writer.open();
                     adapter = adapterFactory.createAdapter(ctx, partition);
                     adapter.start(partition, writer);
-                } catch (Throwable th) {
+                } catch (Exception e) {
                     writer.fail();
-                    throw new HyracksDataException(th);
+                    throw new HyracksDataException(e);
                 } finally {
                     writer.close();
                 }
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/operators/FeedMetaComputeNodePushable.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/operators/FeedMetaComputeNodePushable.java
index 66a6c42..ed017a3 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/operators/FeedMetaComputeNodePushable.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/operators/FeedMetaComputeNodePushable.java
@@ -41,7 +41,7 @@
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.api.util.HyracksConstants;
 import org.apache.hyracks.dataflow.common.comm.io.FrameTupleAccessor;
-import org.apache.hyracks.dataflow.common.util.TaskUtils;
+import org.apache.hyracks.dataflow.common.utils.TaskUtil;
 import org.apache.hyracks.dataflow.std.base.AbstractUnaryInputUnaryOutputOperatorNodePushable;
 
 /*
@@ -107,7 +107,7 @@
         this.feedManager = (ActiveManager) ((IAppRuntimeContext) ctx.getJobletContext().getApplicationContext()
                 .getApplicationObject()).getActiveManager();
         this.message = new VSizeFrame(ctx);
-        TaskUtils.putInSharedMap(HyracksConstants.KEY_MESSAGE, message, ctx);
+        TaskUtil.putInSharedMap(HyracksConstants.KEY_MESSAGE, message, ctx);
         this.opDesc = feedMetaOperatorDescriptor;
         this.recordDescProvider = recordDescProvider;
     }
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/operators/FeedMetaStoreNodePushable.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/operators/FeedMetaStoreNodePushable.java
index af54245..48276d5 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/operators/FeedMetaStoreNodePushable.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/operators/FeedMetaStoreNodePushable.java
@@ -41,7 +41,7 @@
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.api.util.HyracksConstants;
 import org.apache.hyracks.dataflow.common.comm.io.FrameTupleAccessor;
-import org.apache.hyracks.dataflow.common.util.TaskUtils;
+import org.apache.hyracks.dataflow.common.utils.TaskUtil;
 import org.apache.hyracks.dataflow.std.base.AbstractUnaryInputUnaryOutputOperatorNodePushable;
 
 public class FeedMetaStoreNodePushable extends AbstractUnaryInputUnaryOutputOperatorNodePushable {
@@ -101,7 +101,7 @@
                 .getApplicationObject()).getActiveManager();
         this.targetId = targetId;
         this.message = new VSizeFrame(ctx);
-        TaskUtils.putInSharedMap(HyracksConstants.KEY_MESSAGE, message, ctx);
+        TaskUtil.putInSharedMap(HyracksConstants.KEY_MESSAGE, message, ctx);
         this.recordDescProvider = recordDescProvider;
         this.opDesc = feedMetaOperatorDescriptor;
     }
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/operators/IndexInfoOperatorDescriptor.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/operators/IndexInfoOperatorDescriptor.java
index 7aa4b25..050cbcf 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/operators/IndexInfoOperatorDescriptor.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/operators/IndexInfoOperatorDescriptor.java
@@ -33,7 +33,7 @@
 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.IIndexOperatorDescriptor;
-import org.apache.hyracks.storage.common.IStorageManagerInterface;
+import org.apache.hyracks.storage.common.IStorageManager;
 import org.apache.hyracks.storage.common.file.ILocalResourceFactoryProvider;
 
 /*
@@ -43,9 +43,9 @@
 
     private static final long serialVersionUID = 1L;
     private final IFileSplitProvider fileSplitProvider;
-    private final IStorageManagerInterface storageManager;
+    private final IStorageManager storageManager;
     private final IIndexLifecycleManagerProvider lifecycleManagerProvider;
-    public IndexInfoOperatorDescriptor(IFileSplitProvider fileSplitProvider,IStorageManagerInterface storageManager,
+    public IndexInfoOperatorDescriptor(IFileSplitProvider fileSplitProvider,IStorageManager storageManager,
             IIndexLifecycleManagerProvider lifecycleManagerProvider){
         this.fileSplitProvider = fileSplitProvider;
         this.lifecycleManagerProvider = lifecycleManagerProvider;
@@ -69,7 +69,7 @@
     }
 
     @Override
-    public IStorageManagerInterface getStorageManager() {
+    public IStorageManager getStorageManager() {
         return storageManager;
     }
 
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/parser/DelimitedDataParser.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/parser/DelimitedDataParser.java
index b14a2c4..0df1412 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/parser/DelimitedDataParser.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/parser/DelimitedDataParser.java
@@ -34,7 +34,7 @@
 import org.apache.asterix.om.base.AMutableString;
 import org.apache.asterix.om.types.ARecordType;
 import org.apache.asterix.om.types.ATypeTag;
-import org.apache.asterix.om.util.NonTaggedFormatUtil;
+import org.apache.asterix.om.utils.NonTaggedFormatUtil;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
 import org.apache.hyracks.dataflow.common.data.parsers.IValueParser;
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/parser/HiveRecordParser.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/parser/HiveRecordParser.java
index fdcd4ff..5e6e288 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/parser/HiveRecordParser.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/parser/HiveRecordParser.java
@@ -38,7 +38,7 @@
 import org.apache.asterix.om.types.AUnionType;
 import org.apache.asterix.om.types.AUnorderedListType;
 import org.apache.asterix.om.types.IAType;
-import org.apache.asterix.om.util.NonTaggedFormatUtil;
+import org.apache.asterix.om.utils.NonTaggedFormatUtil;
 import org.apache.hadoop.hive.serde.Constants;
 import org.apache.hadoop.hive.serde2.SerDe;
 import org.apache.hadoop.hive.serde2.objectinspector.ListObjectInspector;
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/util/FeedUtils.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/util/FeedUtils.java
index 2ca9f82..5ec0399 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/util/FeedUtils.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/util/FeedUtils.java
@@ -28,7 +28,7 @@
 import org.apache.asterix.common.config.ClusterProperties;
 import org.apache.asterix.common.exceptions.AsterixException;
 import org.apache.asterix.common.utils.StoragePathUtil;
-import org.apache.asterix.runtime.util.ClusterStateManager;
+import org.apache.asterix.runtime.utils.ClusterStateManager;
 import org.apache.hyracks.algebricks.common.constraints.AlgebricksAbsolutePartitionConstraint;
 import org.apache.hyracks.algebricks.common.constraints.AlgebricksPartitionConstraint;
 import org.apache.hyracks.algebricks.common.constraints.AlgebricksPartitionConstraint.PartitionConstraintType;
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 9556054..53a46ab 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
@@ -31,13 +31,13 @@
 import org.apache.asterix.external.indexing.IndexingScheduler;
 import org.apache.asterix.external.indexing.RecordId.RecordIdType;
 import org.apache.asterix.external.input.stream.HDFSInputStream;
-import org.apache.asterix.runtime.util.AppContextInfo;
-import org.apache.asterix.runtime.util.ClusterStateManager;
 import org.apache.hadoop.fs.BlockLocation;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.asterix.hivecompat.io.RCFileInputFormat;
+import org.apache.asterix.runtime.utils.AppContextInfo;
+import org.apache.asterix.runtime.utils.ClusterStateManager;
 import org.apache.hadoop.mapred.FileSplit;
 import org.apache.hadoop.mapred.InputSplit;
 import org.apache.hadoop.mapred.JobConf;
@@ -102,7 +102,7 @@
                 // file was deleted at some point, skip to next file
                 continue;
             }
-            if (file.getPendingOp() == ExternalFilePendingOp.PENDING_ADD_OP
+            if (file.getPendingOp() == ExternalFilePendingOp.ADD_OP
                     && fileStatus.getModificationTime() == file.getLastModefiedTime().getTime()) {
                 // Get its information from HDFS name node
                 BlockLocation[] fileBlocks = fs.getFileBlockLocations(fileStatus, 0, file.getSize());
@@ -117,7 +117,7 @@
                         orderedExternalFiles.add(file);
                     }
                 }
-            } else if (file.getPendingOp() == ExternalFilePendingOp.PENDING_NO_OP
+            } else if (file.getPendingOp() == ExternalFilePendingOp.NO_OP
                     && fileStatus.getModificationTime() == file.getLastModefiedTime().getTime()) {
                 long oldSize = 0L;
                 long newSize = file.getSize();
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/util/NodeResolver.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/util/NodeResolver.java
index 222d49d..9970f27 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/util/NodeResolver.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/util/NodeResolver.java
@@ -30,7 +30,7 @@
 import org.apache.asterix.common.exceptions.AsterixException;
 import org.apache.asterix.common.exceptions.ErrorCode;
 import org.apache.asterix.external.api.INodeResolver;
-import org.apache.asterix.runtime.util.RuntimeUtils;
+import org.apache.asterix.runtime.utils.RuntimeUtils;
 
 /**
  * Resolves a value (DNS/IP Address) or a (Node Controller Id) to the id of a Node Controller running at the location.
diff --git a/asterixdb/asterix-external-data/src/test/java/org/apache/asterix/external/input/record/reader/kv/KVTestReaderFactory.java b/asterixdb/asterix-external-data/src/test/java/org/apache/asterix/external/input/record/reader/kv/KVTestReaderFactory.java
index 6841e6b..98105b2 100644
--- a/asterixdb/asterix-external-data/src/test/java/org/apache/asterix/external/input/record/reader/kv/KVTestReaderFactory.java
+++ b/asterixdb/asterix-external-data/src/test/java/org/apache/asterix/external/input/record/reader/kv/KVTestReaderFactory.java
@@ -22,7 +22,7 @@
 
 import org.apache.asterix.external.api.IRecordReader;
 import org.apache.asterix.external.api.IRecordReaderFactory;
-import org.apache.asterix.runtime.util.ClusterStateManager;
+import org.apache.asterix.runtime.utils.ClusterStateManager;
 import org.apache.hyracks.algebricks.common.constraints.AlgebricksAbsolutePartitionConstraint;
 import org.apache.hyracks.api.context.IHyracksTaskContext;
 
diff --git a/asterixdb/asterix-external-data/src/test/java/org/apache/asterix/external/input/stream/TwitterFirehoseStreamFactory.java b/asterixdb/asterix-external-data/src/test/java/org/apache/asterix/external/input/stream/TwitterFirehoseStreamFactory.java
index 77b8df1..c417aea 100644
--- a/asterixdb/asterix-external-data/src/test/java/org/apache/asterix/external/input/stream/TwitterFirehoseStreamFactory.java
+++ b/asterixdb/asterix-external-data/src/test/java/org/apache/asterix/external/input/stream/TwitterFirehoseStreamFactory.java
@@ -26,7 +26,7 @@
 import org.apache.asterix.external.api.AsterixInputStream;
 import org.apache.asterix.external.api.IInputStreamFactory;
 import org.apache.asterix.external.input.stream.TwitterFirehoseInputStream;
-import org.apache.asterix.runtime.util.ClusterStateManager;
+import org.apache.asterix.runtime.utils.ClusterStateManager;
 import org.apache.hyracks.algebricks.common.constraints.AlgebricksAbsolutePartitionConstraint;
 import org.apache.hyracks.api.context.IHyracksTaskContext;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
diff --git a/asterixdb/asterix-external-data/src/test/java/org/apache/asterix/external/library/ClassAdParser.java b/asterixdb/asterix-external-data/src/test/java/org/apache/asterix/external/library/ClassAdParser.java
index 40523ab..93b01f1 100644
--- a/asterixdb/asterix-external-data/src/test/java/org/apache/asterix/external/library/ClassAdParser.java
+++ b/asterixdb/asterix-external-data/src/test/java/org/apache/asterix/external/library/ClassAdParser.java
@@ -70,9 +70,9 @@
 import org.apache.asterix.om.types.AUnorderedListType;
 import org.apache.asterix.om.types.IAType;
 import org.apache.asterix.om.types.hierachy.ATypeHierarchy;
-import org.apache.asterix.om.util.NonTaggedFormatUtil;
 import org.apache.asterix.om.util.container.IObjectPool;
 import org.apache.asterix.om.util.container.ListObjectPool;
+import org.apache.asterix.om.utils.NonTaggedFormatUtil;
 import org.apache.commons.lang3.mutable.MutableBoolean;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.data.std.api.IMutableValueStorage;
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/base/Statement.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/base/Statement.java
index b12195f..06fbf33 100644
--- a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/base/Statement.java
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/base/Statement.java
@@ -26,7 +26,7 @@
     public static final List<Byte> KINDS = Collections.unmodifiableList(Kind.range(Kind.DATASET_DECL, Kind.RUN));
 
     /**
-     * get a byte representing the statement kind
+     * get a byte representing the statement kind.
      * Note: bytes 0x00 - 0x7f are reserved for core asterix statements
      * Use negative bytes for extension statements
      *
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/util/LangRecordParseUtil.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/util/LangRecordParseUtil.java
index cf57dfe..8907a72 100644
--- a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/util/LangRecordParseUtil.java
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/util/LangRecordParseUtil.java
@@ -44,8 +44,8 @@
 import org.apache.asterix.om.base.AString;
 import org.apache.asterix.om.base.IACursor;
 import org.apache.asterix.om.base.IAObject;
-import org.apache.asterix.om.types.ARecordType;
 import org.apache.asterix.om.types.BuiltinType;
+import org.apache.asterix.om.utils.RecordUtil;
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
 import org.apache.hyracks.algebricks.common.utils.Pair;
 import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
@@ -96,7 +96,7 @@
         ArrayBackedValueStorage fieldName = new ArrayBackedValueStorage();
         ArrayBackedValueStorage fieldValue = new ArrayBackedValueStorage();
         RecordBuilder recordBuilder = new RecordBuilder();
-        recordBuilder.reset(ARecordType.FULLY_OPEN_RECORD_TYPE);
+        recordBuilder.reset(RecordUtil.FULLY_OPEN_RECORD_TYPE);
         recordBuilder.init();
         List<FieldBinding> fbList = recordValue.getFbList();
         HashSet<String> fieldNames = new HashSet<>();
diff --git a/asterixdb/asterix-metadata/pom.xml b/asterixdb/asterix-metadata/pom.xml
index c73b1a4..a122189 100644
--- a/asterixdb/asterix-metadata/pom.xml
+++ b/asterixdb/asterix-metadata/pom.xml
@@ -171,5 +171,21 @@
       <version>2.0.2-beta</version>
       <scope>test</scope>
     </dependency>
+    <dependency>
+      <groupId>com.fasterxml.jackson.core</groupId>
+      <artifactId>jackson-core</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>com.fasterxml.jackson.core</groupId>
+      <artifactId>jackson-databind</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hadoop</groupId>
+      <artifactId>hadoop-hdfs</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hadoop</groupId>
+      <artifactId>hadoop-common</artifactId>
+    </dependency>
   </dependencies>
-</project>
+</project>
\ No newline at end of file
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/MetadataNode.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/MetadataNode.java
index 588f847..8ecc0ed 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/MetadataNode.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/MetadataNode.java
@@ -101,7 +101,7 @@
 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.dataflow.common.util.TupleUtils;
+import org.apache.hyracks.dataflow.common.utils.TupleUtils;
 import org.apache.hyracks.storage.am.btree.impls.RangePredicate;
 import org.apache.hyracks.storage.am.common.api.IIndex;
 import org.apache.hyracks.storage.am.common.api.IIndexAccessor;
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/MetadataTransactionContext.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/MetadataTransactionContext.java
index a8c3e44..87c1c473 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/MetadataTransactionContext.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/MetadataTransactionContext.java
@@ -26,7 +26,6 @@
 import org.apache.asterix.external.dataset.adapter.AdapterIdentifier;
 import org.apache.asterix.external.feed.api.IFeed;
 import org.apache.asterix.external.feed.api.IFeed.FeedType;
-import org.apache.asterix.metadata.api.IMetadataEntity;
 import org.apache.asterix.metadata.entities.CompactionPolicy;
 import org.apache.asterix.metadata.entities.Dataset;
 import org.apache.asterix.metadata.entities.DatasourceAdapter;
@@ -38,6 +37,7 @@
 import org.apache.asterix.metadata.entities.Index;
 import org.apache.asterix.metadata.entities.Library;
 import org.apache.asterix.metadata.entities.NodeGroup;
+import org.apache.asterix.metadata.utils.MetadataUtil;
 
 /**
  * Used to implement serializable transactions against the MetadataCache.
@@ -68,7 +68,7 @@
     // The APIs in this class make sure that these two caches are kept in sync.
     protected MetadataCache droppedCache = new MetadataCache();
 
-    protected ArrayList<MetadataLogicalOperation> opLog = new ArrayList<MetadataLogicalOperation>();
+    protected ArrayList<MetadataLogicalOperation> opLog = new ArrayList<>();
     private final JobId jobId;
 
     public MetadataTransactionContext(JobId jobId) {
@@ -122,20 +122,20 @@
 
     public void dropDataset(String dataverseName, String datasetName) {
         Dataset dataset = new Dataset(dataverseName, datasetName, null, null, null, null, null, null, null, null, -1,
-                IMetadataEntity.PENDING_NO_OP);
+                MetadataUtil.PENDING_NO_OP);
         droppedCache.addDatasetIfNotExists(dataset);
         logAndApply(new MetadataLogicalOperation(dataset, false));
     }
 
     public void dropIndex(String dataverseName, String datasetName, String indexName) {
         Index index = new Index(dataverseName, datasetName, indexName, null, null, null, null, false, false,
-                IMetadataEntity.PENDING_NO_OP);
+                MetadataUtil.PENDING_NO_OP);
         droppedCache.addIndexIfNotExists(index);
         logAndApply(new MetadataLogicalOperation(index, false));
     }
 
     public void dropDataverse(String dataverseName) {
-        Dataverse dataverse = new Dataverse(dataverseName, null, IMetadataEntity.PENDING_NO_OP);
+        Dataverse dataverse = new Dataverse(dataverseName, null, MetadataUtil.PENDING_NO_OP);
         droppedCache.addDataverseIfNotExists(dataverse);
         logAndApply(new MetadataLogicalOperation(dataverse, false));
     }
@@ -236,8 +236,8 @@
 
     public void dropFeed(String dataverseName, String feedName, IFeed.FeedType feedType) {
         Feed feed = null;
-        feed = new Feed(dataverseName, feedName, null, feedType, (feedType == FeedType.PRIMARY) ? feedName : null, null,
-                null);
+        feed = new Feed(dataverseName, feedName, null, feedType, (feedType == FeedType.PRIMARY) ? feedName : null,
+                null, null);
         droppedCache.addFeedIfNotExists(feed);
         logAndApply(new MetadataLogicalOperation(feed, false));
     }
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/api/IIndexDataflowHelperFactoryProvider.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/api/IIndexDataflowHelperFactoryProvider.java
new file mode 100644
index 0000000..679d6eb0
--- /dev/null
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/api/IIndexDataflowHelperFactoryProvider.java
@@ -0,0 +1,65 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.metadata.api;
+
+import java.util.Map;
+
+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.om.types.ARecordType;
+import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.hyracks.api.dataflow.value.IBinaryComparatorFactory;
+import org.apache.hyracks.api.dataflow.value.ITypeTraits;
+import org.apache.hyracks.storage.am.common.dataflow.IIndexDataflowHelperFactory;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMMergePolicyFactory;
+
+@FunctionalInterface
+public interface IIndexDataflowHelperFactoryProvider {
+
+    /**
+     * Get the index dataflow helper factory
+     *
+     * @param mdProvider
+     *            the system's metadata provider
+     * @param dataset
+     *            the index dataset
+     * @param index
+     *            the index
+     * @param recordType
+     *            the dataset's record type
+     * @param metaType
+     *            the detaset's meta type
+     * @param mergePolicyFactory
+     *            the index's merge policy factory
+     * @param mergePolicyProperties
+     *            the index's merge policy properties
+     * @param filterTypeTraits
+     *            the dataset's filter type traits
+     * @param filterCmpFactories
+     *            the dataset's filter comparator factories
+     * @return the index dataflow helper factory
+     * @throws AlgebricksException
+     *             if the dataflow helper factory couldn't be created for the index
+     */
+    IIndexDataflowHelperFactory getIndexDataflowHelperFactory(MetadataProvider mdProvider, Dataset dataset, Index index,
+            ARecordType recordType, ARecordType metaType, ILSMMergePolicyFactory mergePolicyFactory,
+            Map<String, String> mergePolicyProperties, ITypeTraits[] filterTypeTraits,
+            IBinaryComparatorFactory[] filterCmpFactories) throws AlgebricksException;
+}
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/api/IMetadataEntity.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/api/IMetadataEntity.java
index 1759c0f..482cc3b 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/api/IMetadataEntity.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/api/IMetadataEntity.java
@@ -23,12 +23,8 @@
 import org.apache.asterix.metadata.MetadataCache;
 
 public interface IMetadataEntity<T> extends Serializable {
-    public static final int PENDING_NO_OP = 0;
-    public static final int PENDING_ADD_OP = 1;
-    public static final int PENDING_DROP_OP = 2;
 
     public T addToCache(MetadataCache cache);
 
     public T dropFromCache(MetadataCache cache);
-
 }
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/api/IMetadataExtension.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/api/IMetadataExtension.java
index 9b55151..5626a83 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/api/IMetadataExtension.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/api/IMetadataExtension.java
@@ -25,6 +25,7 @@
 import org.apache.asterix.common.api.IExtension;
 import org.apache.asterix.common.exceptions.ACIDException;
 import org.apache.asterix.metadata.entitytupletranslators.MetadataTupleTranslatorProvider;
+import org.apache.hyracks.api.application.INCApplicationContext;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 
 /**
@@ -56,6 +57,6 @@
      * @throws RemoteException
      * @throws ACIDException
      */
-    void initializeMetadata() throws HyracksDataException, RemoteException, ACIDException;
+    void initializeMetadata(INCApplicationContext appCtx) throws HyracksDataException, RemoteException, ACIDException;
 
 }
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/cluster/IGlobalRecoveryMaanger.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/api/INCExtensionManager.java
similarity index 71%
copy from asterixdb/asterix-common/src/main/java/org/apache/asterix/common/cluster/IGlobalRecoveryMaanger.java
copy to asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/api/INCExtensionManager.java
index 48b1e73..62c4273 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/cluster/IGlobalRecoveryMaanger.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/api/INCExtensionManager.java
@@ -16,14 +16,15 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.asterix.common.cluster;
+package org.apache.asterix.metadata.api;
 
-import org.apache.asterix.common.api.IClusterEventsSubscriber;
+import org.apache.asterix.metadata.entitytupletranslators.MetadataTupleTranslatorProvider;
 
-public interface IGlobalRecoveryMaanger extends IClusterEventsSubscriber {
+@FunctionalInterface
+public interface INCExtensionManager {
 
     /**
-     * Starts the global recovery process if the cluster state changed to ACTIVE.
+     * @return the metadata tuple translator provider
      */
-    public void startGlobalRecovery();
+    MetadataTupleTranslatorProvider getMetadataTupleTranslatorProvider();
 }
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/bootstrap/MetadataBootstrap.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/bootstrap/MetadataBootstrap.java
index b64d2ad..15f0a35 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/bootstrap/MetadataBootstrap.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/bootstrap/MetadataBootstrap.java
@@ -29,14 +29,11 @@
 import org.apache.asterix.common.api.IAppRuntimeContext;
 import org.apache.asterix.common.api.IDatasetLifecycleManager;
 import org.apache.asterix.common.cluster.ClusterPartition;
-import org.apache.asterix.common.config.MetadataProperties;
 import org.apache.asterix.common.config.ClusterProperties;
 import org.apache.asterix.common.config.DatasetConfig.DatasetType;
 import org.apache.asterix.common.config.GlobalConfig;
-import org.apache.asterix.common.config.IPropertiesProvider;
-import org.apache.asterix.common.context.BaseOperationTracker;
+import org.apache.asterix.common.config.MetadataProperties;
 import org.apache.asterix.common.context.CorrelatedPrefixMergePolicyFactory;
-import org.apache.asterix.common.dataflow.LSMIndexUtil;
 import org.apache.asterix.common.exceptions.ACIDException;
 import org.apache.asterix.common.ioopcallbacks.LSMBTreeIOOperationCallbackFactory;
 import org.apache.asterix.common.transactions.Resource;
@@ -50,7 +47,6 @@
 import org.apache.asterix.metadata.MetadataException;
 import org.apache.asterix.metadata.MetadataManager;
 import org.apache.asterix.metadata.MetadataTransactionContext;
-import org.apache.asterix.metadata.api.IMetadataEntity;
 import org.apache.asterix.metadata.api.IMetadataIndex;
 import org.apache.asterix.metadata.entities.BuiltinTypeMap;
 import org.apache.asterix.metadata.entities.CompactionPolicy;
@@ -67,8 +63,11 @@
 import org.apache.asterix.metadata.entities.NodeGroup;
 import org.apache.asterix.metadata.feeds.BuiltinFeedPolicies;
 import org.apache.asterix.metadata.utils.MetadataConstants;
+import org.apache.asterix.metadata.utils.MetadataUtil;
 import org.apache.asterix.om.types.IAType;
 import org.apache.asterix.runtime.formats.NonTaggedDataFormat;
+import org.apache.asterix.transaction.management.opcallbacks.PrimaryIndexOperationTrackerFactory;
+import org.apache.asterix.transaction.management.opcallbacks.SecondaryIndexOperationTrackerFactory;
 import org.apache.asterix.transaction.management.resource.LSMBTreeLocalResourceMetadata;
 import org.apache.asterix.transaction.management.resource.PersistentLocalResourceFactoryProvider;
 import org.apache.asterix.transaction.management.service.transaction.TransactionManagementConstants.LockManagerConstants.LockMode;
@@ -80,9 +79,10 @@
 import org.apache.hyracks.api.io.IIOManager;
 import org.apache.hyracks.storage.am.common.api.ITreeIndexFrame;
 import org.apache.hyracks.storage.am.lsm.btree.impls.LSMBTree;
-import org.apache.hyracks.storage.am.lsm.btree.util.LSMBTreeUtils;
+import org.apache.hyracks.storage.am.lsm.btree.utils.LSMBTreeUtil;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallbackFactory;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMMergePolicyFactory;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMOperationTracker;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMOperationTrackerFactory;
 import org.apache.hyracks.storage.am.lsm.common.api.IVirtualBufferCache;
 import org.apache.hyracks.storage.am.lsm.common.impls.ConstantMergePolicyFactory;
 import org.apache.hyracks.storage.am.lsm.common.impls.NoMergePolicyFactory;
@@ -125,8 +125,6 @@
                     MetadataPrimaryIndexes.LIBRARY_DATASET, MetadataPrimaryIndexes.COMPACTION_POLICY_DATASET,
                     MetadataPrimaryIndexes.EXTERNAL_FILE_DATASET };
 
-    private static IPropertiesProvider propertiesProvider;
-
     private MetadataBootstrap() {
     }
 
@@ -141,14 +139,12 @@
      * @throws MetadataException
      * @throws Exception
      */
-    public static void startUniverse(IPropertiesProvider asterixPropertiesProvider,
-            INCApplicationContext ncApplicationContext, boolean isNewUniverse)
+    public static void startUniverse(INCApplicationContext ncApplicationContext, boolean isNewUniverse)
             throws RemoteException, ACIDException, MetadataException {
         MetadataBootstrap.setNewUniverse(isNewUniverse);
         runtimeContext = (IAppRuntimeContext) ncApplicationContext.getApplicationObject();
-        propertiesProvider = asterixPropertiesProvider;
 
-        MetadataProperties metadataProperties = propertiesProvider.getMetadataProperties();
+        MetadataProperties metadataProperties = runtimeContext.getMetadataProperties();
         metadataNodeName = metadataProperties.getMetadataNodeName();
         nodeNames = metadataProperties.getNodeNames();
         dataLifecycleManager = runtimeContext.getDatasetLifecycleManager();
@@ -164,7 +160,7 @@
             MetadataManager.INSTANCE.lock(mdTxnCtx, LockMode.X);
 
             for (int i = 0; i < PRIMARY_INDEXES.length; i++) {
-                enlistMetadataDataset(PRIMARY_INDEXES[i]);
+                enlistMetadataDataset(ncApplicationContext, PRIMARY_INDEXES[i]);
             }
             if (LOGGER.isLoggable(Level.INFO)) {
                 LOGGER.info(
@@ -204,7 +200,7 @@
     private static void insertInitialDataverses(MetadataTransactionContext mdTxnCtx) throws MetadataException {
         String dataFormat = NonTaggedDataFormat.NON_TAGGED_DATA_FORMAT;
         MetadataManager.INSTANCE.addDataverse(mdTxnCtx,
-                new Dataverse(MetadataConstants.METADATA_DATAVERSE_NAME, dataFormat, IMetadataEntity.PENDING_NO_OP));
+                new Dataverse(MetadataConstants.METADATA_DATAVERSE_NAME, dataFormat, MetadataUtil.PENDING_NO_OP));
         MetadataManager.INSTANCE.addDataverse(mdTxnCtx, MetadataBuiltinEntities.DEFAULT_DATAVERSE);
     }
 
@@ -227,8 +223,7 @@
                             indexes[i].getDataverseName(), indexes[i].getPayloadRecordType().getTypeName(),
                             indexes[i].getNodeGroupName(), GlobalConfig.DEFAULT_COMPACTION_POLICY_NAME,
                             GlobalConfig.DEFAULT_COMPACTION_POLICY_PROPERTIES, id, new HashMap<String, String>(),
-                            DatasetType.INTERNAL, indexes[i].getDatasetId().getId(),
-                            IMetadataEntity.PENDING_NO_OP));
+                            DatasetType.INTERNAL, indexes[i].getDatasetId().getId(), MetadataUtil.PENDING_NO_OP));
         }
         if (LOGGER.isLoggable(Level.INFO)) {
             LOGGER.info("Finished inserting initial datasets.");
@@ -246,12 +241,10 @@
         types.addAll(BuiltinTypeMap.getAllBuiltinTypes());
         getMetadataTypes(types);
         for (int i = 0; i < types.size(); i++) {
-            MetadataManager.INSTANCE.addDatatype(mdTxnCtx,
-                    new Datatype(MetadataConstants.METADATA_DATAVERSE_NAME, types.get(i).getTypeName(), types.get(i),
-                            false));
+            MetadataManager.INSTANCE.addDatatype(mdTxnCtx, new Datatype(MetadataConstants.METADATA_DATAVERSE_NAME,
+                    types.get(i).getTypeName(), types.get(i), false));
         }
-        MetadataManager.INSTANCE.addDatatype(mdTxnCtx,
-                MetadataBuiltinEntities.ANY_OBJECT_DATATYPE);
+        MetadataManager.INSTANCE.addDatatype(mdTxnCtx, MetadataBuiltinEntities.ANY_OBJECT_DATATYPE);
         if (LOGGER.isLoggable(Level.INFO)) {
             LOGGER.info("Finished inserting initial datatypes.");
         }
@@ -274,8 +267,7 @@
         MetadataManager.INSTANCE.addNodegroup(mdTxnCtx, defaultGroup);
     }
 
-    private static void insertInitialAdapters(MetadataTransactionContext mdTxnCtx)
-            throws MetadataException {
+    private static void insertInitialAdapters(MetadataTransactionContext mdTxnCtx) throws MetadataException {
         String[] builtInAdapterClassNames = new String[] { GenericAdapterFactory.class.getName() };
         DatasourceAdapter adapter;
         for (String adapterClassName : builtInAdapterClassNames) {
@@ -306,8 +298,7 @@
         }
     }
 
-    private static DatasourceAdapter getAdapter(String adapterFactoryClassName)
-            throws MetadataException {
+    private static DatasourceAdapter getAdapter(String adapterFactoryClassName) throws MetadataException {
         try {
             String adapterName = ((IAdapterFactory) (Class.forName(adapterFactoryClassName).newInstance())).getAlias();
             return new DatasourceAdapter(new AdapterIdentifier(MetadataConstants.METADATA_DATAVERSE_NAME, adapterName),
@@ -336,8 +327,9 @@
      * @param index
      * @throws HyracksDataException
      */
-    public static void enlistMetadataDataset(IMetadataIndex index) throws HyracksDataException {
-        ClusterPartition metadataPartition = propertiesProvider.getMetadataProperties().getMetadataPartition();
+    public static void enlistMetadataDataset(INCApplicationContext appCtx, IMetadataIndex index)
+            throws HyracksDataException {
+        ClusterPartition metadataPartition = runtimeContext.getMetadataProperties().getMetadataPartition();
         int metadataDeviceId = metadataPartition.getIODeviceNum();
         String metadataPartitionPath = StoragePathUtil.prepareStoragePartitionPath(
                 ClusterProperties.INSTANCE.getStorageDirectoryName(), metadataPartition.getPartitionId());
@@ -353,29 +345,33 @@
         int[] bloomFilterKeyFields = index.getBloomFilterKeyFields();
         LSMBTree lsmBtree;
         long resourceID;
-        ILSMOperationTracker opTracker =
-                index.isPrimaryIndex() ? runtimeContext.getLSMBTreeOperationTracker(index.getDatasetId().getId())
-                        : new BaseOperationTracker(index.getDatasetId().getId(),
-                                dataLifecycleManager.getDatasetInfo(index.getDatasetId().getId()));
+        // opTrackerProvider and ioOpCallbackFactory should both be acquired through IStorageManager
+        // We are unable to do this since IStorageManager needs a dataset to determine the appropriate
+        // objects
+        ILSMOperationTrackerFactory opTrackerProvider =
+                index.isPrimaryIndex() ? new PrimaryIndexOperationTrackerFactory(index.getDatasetId().getId())
+                        : new SecondaryIndexOperationTrackerFactory(index.getDatasetId().getId());
+        ILSMIOOperationCallbackFactory ioOpCallbackFactory = LSMBTreeIOOperationCallbackFactory.INSTANCE;
         if (isNewUniverse()) {
-            lsmBtree = LSMBTreeUtils.createLSMTree(ioManager, virtualBufferCaches, file, bufferCache, fileMapProvider,
-                    typeTraits,
-                    comparatorFactories, bloomFilterKeyFields, runtimeContext.getBloomFilterFalsePositiveRate(),
-                    runtimeContext.getMetadataMergePolicyFactory()
-                            .createMergePolicy(GlobalConfig.DEFAULT_COMPACTION_POLICY_PROPERTIES, dataLifecycleManager),
-                    opTracker, runtimeContext.getLSMIOScheduler(),
-                    LSMBTreeIOOperationCallbackFactory.INSTANCE.createIOOperationCallback(), index.isPrimaryIndex(),
-                    null, null, null, null, true, LSMIndexUtil.getMetadataPageManagerFactory());
+            // TODO(amoudi) Creating the index should be done through the same code path as other indexes
+            // This is to be done by having a metadata dataset associated with each index
+            lsmBtree = LSMBTreeUtil.createLSMTree(ioManager, virtualBufferCaches, file, bufferCache, fileMapProvider,
+                    typeTraits, comparatorFactories, bloomFilterKeyFields,
+                    runtimeContext.getBloomFilterFalsePositiveRate(),
+                    runtimeContext.getMetadataMergePolicyFactory().createMergePolicy(
+                            GlobalConfig.DEFAULT_COMPACTION_POLICY_PROPERTIES, dataLifecycleManager),
+                    opTrackerProvider.getOperationTracker(appCtx), runtimeContext.getLSMIOScheduler(),
+                    ioOpCallbackFactory.createIOOperationCallback(), index.isPrimaryIndex(), null, null, null, null,
+                    true, runtimeContext.getStorageComponentProvider().getMetadataPageManagerFactory());
             lsmBtree.create();
             resourceID = index.getResourceID();
-            Resource localResourceMetadata = new LSMBTreeLocalResourceMetadata(typeTraits,
-                    comparatorFactories, bloomFilterKeyFields, index.isPrimaryIndex(), index.getDatasetId().getId(),
-                    metadataPartition.getPartitionId(),
-                    runtimeContext.getMetadataMergePolicyFactory(), GlobalConfig.DEFAULT_COMPACTION_POLICY_PROPERTIES,
-                    null, null, null, null);
-            ILocalResourceFactoryProvider localResourceFactoryProvider =
-                    new PersistentLocalResourceFactoryProvider(partition -> localResourceMetadata,
-                            LocalResource.LSMBTreeResource);
+            Resource localResourceMetadata = new LSMBTreeLocalResourceMetadata(typeTraits, comparatorFactories,
+                    bloomFilterKeyFields, index.isPrimaryIndex(), index.getDatasetId().getId(),
+                    metadataPartition.getPartitionId(), runtimeContext.getMetadataMergePolicyFactory(),
+                    GlobalConfig.DEFAULT_COMPACTION_POLICY_PROPERTIES, null, null, null, null, opTrackerProvider,
+                    ioOpCallbackFactory, runtimeContext.getStorageComponentProvider().getMetadataPageManagerFactory());
+            ILocalResourceFactoryProvider localResourceFactoryProvider = new PersistentLocalResourceFactoryProvider(
+                    partition -> localResourceMetadata, LocalResource.LSMBTreeResource);
             ILocalResourceFactory localResourceFactory = localResourceFactoryProvider.getLocalResourceFactory();
             localResourceRepository.insert(localResourceFactory.createLocalResource(resourceID, resourceName,
                     ITreeIndexFrame.Constants.VERSION, metadataPartition.getPartitionId()));
@@ -384,7 +380,7 @@
             final LocalResource resource = localResourceRepository.get(file.getRelativePath());
             if (resource == null) {
                 throw new HyracksDataException("Could not find required metadata indexes. Please delete "
-                        + propertiesProvider.getMetadataProperties().getTransactionLogDirs()
+                        + runtimeContext.getMetadataProperties().getTransactionLogDirs()
                                 .get(runtimeContext.getTransactionSubsystem().getId())
                         + " to intialize as a new instance. (WARNING: all data will be lost.)");
             }
@@ -394,15 +390,15 @@
             }
             lsmBtree = (LSMBTree) dataLifecycleManager.get(file.getRelativePath());
             if (lsmBtree == null) {
-                lsmBtree = LSMBTreeUtils.createLSMTree(ioManager, virtualBufferCaches, file, bufferCache,
-                        fileMapProvider,
-                        typeTraits, comparatorFactories, bloomFilterKeyFields,
+                lsmBtree = LSMBTreeUtil.createLSMTree(ioManager, virtualBufferCaches, file, bufferCache,
+                        fileMapProvider, typeTraits, comparatorFactories, bloomFilterKeyFields,
                         runtimeContext.getBloomFilterFalsePositiveRate(),
                         runtimeContext.getMetadataMergePolicyFactory().createMergePolicy(
                                 GlobalConfig.DEFAULT_COMPACTION_POLICY_PROPERTIES, dataLifecycleManager),
-                        opTracker, runtimeContext.getLSMIOScheduler(),
-                        LSMBTreeIOOperationCallbackFactory.INSTANCE.createIOOperationCallback(), index.isPrimaryIndex(),
-                        null, null, null, null, true, LSMIndexUtil.getMetadataPageManagerFactory());
+                        opTrackerProvider.getOperationTracker(appCtx), runtimeContext.getLSMIOScheduler(),
+                        LSMBTreeIOOperationCallbackFactory.INSTANCE.createIOOperationCallback(),
+                        index.isPrimaryIndex(), null, null, null, null, true,
+                        runtimeContext.getStorageComponentProvider().getMetadataPageManagerFactory());
                 dataLifecycleManager.register(file.getRelativePath(), lsmBtree);
             }
         }
@@ -458,7 +454,7 @@
 
     private static void recoverDataverse(MetadataTransactionContext mdTxnCtx, Dataverse dataverse)
             throws MetadataException {
-        if (dataverse.getPendingOp() != IMetadataEntity.PENDING_NO_OP) {
+        if (dataverse.getPendingOp() != MetadataUtil.PENDING_NO_OP) {
             // drop pending dataverse
             MetadataManager.INSTANCE.dropDataverse(mdTxnCtx, dataverse.getDataverseName());
             if (LOGGER.isLoggable(Level.INFO)) {
@@ -474,7 +470,7 @@
     }
 
     private static void recoverDataset(MetadataTransactionContext mdTxnCtx, Dataset dataset) throws MetadataException {
-        if (dataset.getPendingOp() != IMetadataEntity.PENDING_NO_OP) {
+        if (dataset.getPendingOp() != MetadataUtil.PENDING_NO_OP) {
             // drop pending dataset
             MetadataManager.INSTANCE.dropDataset(mdTxnCtx, dataset.getDataverseName(), dataset.getDatasetName());
             if (LOGGER.isLoggable(Level.INFO)) {
@@ -482,36 +478,31 @@
                         "Dropped a pending dataset: " + dataset.getDataverseName() + "." + dataset.getDatasetName());
             }
         } else {
-            List<Index> indexes =
-                    MetadataManager.INSTANCE.getDatasetIndexes(mdTxnCtx, dataset.getDataverseName(),
-                            dataset.getDatasetName());
+            List<Index> indexes = MetadataManager.INSTANCE.getDatasetIndexes(mdTxnCtx, dataset.getDataverseName(),
+                    dataset.getDatasetName());
             for (Index index : indexes) {
-                if (index.getPendingOp() != IMetadataEntity.PENDING_NO_OP) {
+                if (index.getPendingOp() != MetadataUtil.PENDING_NO_OP) {
                     // drop pending index
                     MetadataManager.INSTANCE.dropIndex(mdTxnCtx, dataset.getDataverseName(), dataset.getDatasetName(),
                             index.getIndexName());
                     if (LOGGER.isLoggable(Level.INFO)) {
                         LOGGER.info("Dropped a pending index: " + dataset.getDataverseName() + "."
-                                + dataset.getDatasetName()
-                                + "." + index.getIndexName());
+                                + dataset.getDatasetName() + "." + index.getIndexName());
                     }
                 }
             }
         }
         if (dataset.getDatasetType() == DatasetType.EXTERNAL) {
             // if the dataset has no indexes, delete all its files
-            List<Index> indexes =
-                    MetadataManager.INSTANCE.getDatasetIndexes(mdTxnCtx, dataset.getDataverseName(),
-                            dataset.getDatasetName());
+            List<Index> indexes = MetadataManager.INSTANCE.getDatasetIndexes(mdTxnCtx, dataset.getDataverseName(),
+                    dataset.getDatasetName());
             if (indexes.isEmpty()) {
-                List<ExternalFile> files =
-                        MetadataManager.INSTANCE.getDatasetExternalFiles(mdTxnCtx, dataset);
+                List<ExternalFile> files = MetadataManager.INSTANCE.getDatasetExternalFiles(mdTxnCtx, dataset);
                 for (ExternalFile file : files) {
                     MetadataManager.INSTANCE.dropExternalFile(mdTxnCtx, file);
                     if (LOGGER.isLoggable(Level.INFO)) {
                         LOGGER.info("Dropped an external file: " + dataset.getDataverseName() + "."
-                                + dataset.getDatasetName()
-                                + "." + file.getFileNumber());
+                                + dataset.getDatasetName() + "." + file.getFileNumber());
                     }
                 }
             }
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/bootstrap/MetadataBuiltinEntities.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/bootstrap/MetadataBuiltinEntities.java
index edbbc4a..d796fed 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/bootstrap/MetadataBuiltinEntities.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/bootstrap/MetadataBuiltinEntities.java
@@ -18,26 +18,25 @@
  */
 package org.apache.asterix.metadata.bootstrap;
 
-import org.apache.asterix.metadata.api.IMetadataEntity;
 import org.apache.asterix.metadata.entities.Datatype;
 import org.apache.asterix.metadata.entities.Dataverse;
 import org.apache.asterix.metadata.utils.MetadataConstants;
+import org.apache.asterix.metadata.utils.MetadataUtil;
 import org.apache.asterix.om.types.ARecordType;
 import org.apache.asterix.om.types.IAType;
+import org.apache.asterix.om.utils.RecordUtil;
 import org.apache.asterix.runtime.formats.NonTaggedDataFormat;
 
 public class MetadataBuiltinEntities {
     //--------------------------------------- Dataverses ----------------------------------------//
     public static final String DEFAULT_DATAVERSE_NAME = "Default";
     public static final Dataverse DEFAULT_DATAVERSE =
-            new Dataverse(DEFAULT_DATAVERSE_NAME, NonTaggedDataFormat.class.getName(),
-                    IMetadataEntity.PENDING_NO_OP);
+            new Dataverse(DEFAULT_DATAVERSE_NAME, NonTaggedDataFormat.class.getName(), MetadataUtil.PENDING_NO_OP);
     //--------------------------------------- Datatypes -----------------------------------------//
     public static final ARecordType ANY_OBJECT_RECORD_TYPE =
             new ARecordType("AnyObject", new String[0], new IAType[0], true);
-    public static final Datatype ANY_OBJECT_DATATYPE =
-            new Datatype(MetadataConstants.METADATA_DATAVERSE_NAME, ANY_OBJECT_RECORD_TYPE.getTypeName(),
-                    ARecordType.FULLY_OPEN_RECORD_TYPE, false);
+    public static final Datatype ANY_OBJECT_DATATYPE = new Datatype(MetadataConstants.METADATA_DATAVERSE_NAME,
+            ANY_OBJECT_RECORD_TYPE.getTypeName(), RecordUtil.FULLY_OPEN_RECORD_TYPE, false);
 
     private MetadataBuiltinEntities() {
     }
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/cluster/ClusterManager.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/cluster/ClusterManager.java
index fa826de..5bfe876 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/cluster/ClusterManager.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/cluster/ClusterManager.java
@@ -47,7 +47,7 @@
 import org.apache.asterix.event.util.PatternCreator;
 import org.apache.asterix.installer.schema.conf.Configuration;
 import org.apache.asterix.metadata.api.IClusterManager;
-import org.apache.asterix.runtime.util.AppContextInfo;
+import org.apache.asterix.runtime.utils.AppContextInfo;
 
 public class ClusterManager implements IClusterManager {
 
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/dataset/hints/DatasetHints.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/dataset/hints/DatasetHints.java
index 0a8e9b1..90c8b1e 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/dataset/hints/DatasetHints.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/dataset/hints/DatasetHints.java
@@ -21,7 +21,7 @@
 import java.util.HashSet;
 import java.util.Set;
 
-import org.apache.asterix.runtime.util.AppContextInfo;
+import org.apache.asterix.runtime.utils.AppContextInfo;
 import org.apache.hyracks.algebricks.common.utils.Pair;
 
 /**
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/BTreeDataflowHelperFactoryProvider.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/BTreeDataflowHelperFactoryProvider.java
new file mode 100644
index 0000000..0560bd0
--- /dev/null
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/BTreeDataflowHelperFactoryProvider.java
@@ -0,0 +1,93 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.metadata.declared;
+
+import java.util.Map;
+
+import org.apache.asterix.common.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.IndexingConstants;
+import org.apache.asterix.metadata.api.IIndexDataflowHelperFactoryProvider;
+import org.apache.asterix.metadata.entities.Dataset;
+import org.apache.asterix.metadata.entities.Index;
+import org.apache.asterix.metadata.utils.ExternalDatasetsRegistry;
+import org.apache.asterix.metadata.utils.IndexUtil;
+import org.apache.asterix.om.types.ARecordType;
+import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.hyracks.api.dataflow.value.IBinaryComparatorFactory;
+import org.apache.hyracks.api.dataflow.value.ITypeTraits;
+import org.apache.hyracks.storage.am.common.dataflow.IIndexDataflowHelperFactory;
+import org.apache.hyracks.storage.am.lsm.btree.dataflow.ExternalBTreeDataflowHelperFactory;
+import org.apache.hyracks.storage.am.lsm.btree.dataflow.ExternalBTreeWithBuddyDataflowHelperFactory;
+import org.apache.hyracks.storage.am.lsm.btree.dataflow.LSMBTreeDataflowHelperFactory;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMMergePolicyFactory;
+
+public class BTreeDataflowHelperFactoryProvider implements IIndexDataflowHelperFactoryProvider {
+
+    public static final BTreeDataflowHelperFactoryProvider INSTANCE = new BTreeDataflowHelperFactoryProvider();
+
+    private BTreeDataflowHelperFactoryProvider() {
+    }
+
+    public static String externalFileIndexName(Dataset dataset) {
+        return dataset.getDatasetName().concat(IndexingConstants.EXTERNAL_FILE_INDEX_NAME_SUFFIX);
+    }
+
+    @Override
+    public IIndexDataflowHelperFactory getIndexDataflowHelperFactory(MetadataProvider mdProvider, Dataset dataset,
+            Index index, ARecordType recordType, ARecordType metaType, ILSMMergePolicyFactory mergePolicyFactory,
+            Map<String, String> mergePolicyProperties, ITypeTraits[] filterTypeTraits,
+            IBinaryComparatorFactory[] filterCmpFactories) throws AlgebricksException {
+        int[] filterFields = IndexUtil.getFilterFields(dataset, index, filterTypeTraits);
+        int[] btreeFields = IndexUtil.getBtreeFieldsIfFiltered(dataset, index);
+        IStorageComponentProvider storageComponentProvider = mdProvider.getStorageComponentProvider();
+        switch (dataset.getDatasetType()) {
+            case EXTERNAL:
+                return index.getIndexName().equals(externalFileIndexName(dataset))
+                        ? new ExternalBTreeDataflowHelperFactory(mergePolicyFactory, mergePolicyProperties,
+                                dataset.getIndexOperationTrackerFactory(index),
+                                storageComponentProvider.getIoOperationSchedulerProvider(),
+                                dataset.getIoOperationCallbackFactory(index),
+                                mdProvider.getStorageProperties().getBloomFilterFalsePositiveRate(),
+                                ExternalDatasetsRegistry.INSTANCE.getAndLockDatasetVersion(dataset, mdProvider),
+                                !dataset.getDatasetDetails().isTemp())
+                        : new ExternalBTreeWithBuddyDataflowHelperFactory(mergePolicyFactory, mergePolicyProperties,
+                                dataset.getIndexOperationTrackerFactory(index),
+                                storageComponentProvider.getIoOperationSchedulerProvider(),
+                                dataset.getIoOperationCallbackFactory(index),
+                                mdProvider.getStorageProperties().getBloomFilterFalsePositiveRate(),
+                                new int[] { index.getKeyFieldNames().size() },
+                                ExternalDatasetsRegistry.INSTANCE.getAndLockDatasetVersion(dataset, mdProvider),
+                                !dataset.getDatasetDetails().isTemp());
+            case INTERNAL:
+                return new LSMBTreeDataflowHelperFactory(new AsterixVirtualBufferCacheProvider(dataset.getDatasetId()),
+                        mergePolicyFactory, mergePolicyProperties, dataset.getIndexOperationTrackerFactory(index),
+                        storageComponentProvider.getIoOperationSchedulerProvider(),
+                        dataset.getIoOperationCallbackFactory(index),
+                        mdProvider.getStorageProperties().getBloomFilterFalsePositiveRate(), index.isPrimaryIndex(),
+                        filterTypeTraits, filterCmpFactories, btreeFields, filterFields,
+                        !dataset.getDatasetDetails().isTemp());
+            default:
+                throw new CompilationException(ErrorCode.COMPILATION_UNKNOWN_DATASET_TYPE,
+                        dataset.getDatasetType().toString());
+        }
+    }
+}
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/DatasetDataSource.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/DatasetDataSource.java
index 973651f..68c7e22 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/DatasetDataSource.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/DatasetDataSource.java
@@ -28,7 +28,7 @@
 import org.apache.asterix.metadata.entities.ExternalDatasetDetails;
 import org.apache.asterix.metadata.entities.Index;
 import org.apache.asterix.metadata.entities.InternalDatasetDetails;
-import org.apache.asterix.metadata.utils.KeyFieldTypeUtils;
+import org.apache.asterix.metadata.utils.KeyFieldTypeUtil;
 import org.apache.asterix.om.types.ARecordType;
 import org.apache.asterix.om.types.IAType;
 import org.apache.asterix.runtime.formats.NonTaggedDataFormat;
@@ -73,7 +73,7 @@
         ARecordType recordType = (ARecordType) itemType;
         ARecordType metaRecordType = (ARecordType) metaItemType;
         List<IAType> partitioningKeyTypes =
-                KeyFieldTypeUtils.getPartitioningKeyTypes(internalDatasetDetails, recordType, metaRecordType);
+                KeyFieldTypeUtil.getPartitioningKeyTypes(internalDatasetDetails, recordType, metaRecordType);
         int n = partitioningKeyTypes.size();
         schemaTypes = metaItemType == null ? new IAType[n + 1] : new IAType[n + 2];
         for (int keyIndex = 0; keyIndex < n; ++keyIndex) {
@@ -106,7 +106,7 @@
 
                 ExternalDatasetDetails edd = (ExternalDatasetDetails) externalDataset.getDatasetDetails();
                 IAdapterFactory adapterFactory = metadataProvider.getConfiguredAdapterFactory(externalDataset,
-                        edd.getAdapter(), edd.getProperties(), (ARecordType) itemType, false, null, null);
+                        edd.getAdapter(), edd.getProperties(), (ARecordType) itemType, null);
                 return metadataProvider.buildExternalDatasetDataScannerRuntime(jobSpec, itemType, adapterFactory,
                         NonTaggedDataFormat.INSTANCE);
             case INTERNAL:
@@ -118,9 +118,9 @@
 
                 int[] minFilterFieldIndexes = createFilterIndexes(minFilterVars, opSchema);
                 int[] maxFilterFieldIndexes = createFilterIndexes(maxFilterVars, opSchema);
-                return metadataProvider.buildBtreeRuntime(jobSpec, scanVariables, opSchema, typeEnv, context, true,
+                return metadataProvider.buildBtreeRuntime(jobSpec, opSchema, typeEnv, context, true,
                         false, ((DatasetDataSource) dataSource).getDataset(), primaryIndex.getIndexName(), null, null,
-                        true, true, implConfig, minFilterFieldIndexes, maxFilterFieldIndexes);
+                        true, true, minFilterFieldIndexes, maxFilterFieldIndexes);
             default:
                 throw new AlgebricksException("Unknown datasource type");
         }
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/FeedDataSource.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/FeedDataSource.java
index 703f8c8..0d3d06d 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/FeedDataSource.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/FeedDataSource.java
@@ -33,7 +33,7 @@
 import org.apache.asterix.om.types.ARecordType;
 import org.apache.asterix.om.types.IAType;
 import org.apache.asterix.runtime.formats.NonTaggedDataFormat;
-import org.apache.asterix.runtime.util.ClusterStateManager;
+import org.apache.asterix.runtime.utils.ClusterStateManager;
 import org.apache.hyracks.algebricks.common.constraints.AlgebricksAbsolutePartitionConstraint;
 import org.apache.hyracks.algebricks.common.constraints.AlgebricksPartitionConstraint;
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/LoadableDataSource.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/LoadableDataSource.java
index 5b5844c..cdf699d 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/LoadableDataSource.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/LoadableDataSource.java
@@ -27,7 +27,7 @@
 import org.apache.asterix.external.api.IAdapterFactory;
 import org.apache.asterix.metadata.entities.Dataset;
 import org.apache.asterix.metadata.entities.InternalDatasetDetails;
-import org.apache.asterix.metadata.utils.DatasetUtils;
+import org.apache.asterix.metadata.utils.DatasetUtil;
 import org.apache.asterix.om.types.ARecordType;
 import org.apache.asterix.om.types.ATypeTag;
 import org.apache.asterix.om.types.IAType;
@@ -60,14 +60,14 @@
         this.targetDataset = targetDataset;
         this.adapter = adapter;
         this.adapterProperties = properties;
-        partitioningKeys = DatasetUtils.getPartitioningKeys(targetDataset);
+        partitioningKeys = DatasetUtil.getPartitioningKeys(targetDataset);
         ARecordType recType = (ARecordType) itemType;
         isPKAutoGenerated = ((InternalDatasetDetails) targetDataset.getDatasetDetails()).isAutogenerated();
         if (isPKAutoGenerated) {
             // Since the key is auto-generated, we need to use another
             // record type (possibly nested) which has all fields except the PK
             try {
-                recType = getStrippedPKType(new LinkedList<String>(partitioningKeys.get(0)), recType);
+                recType = getStrippedPKType(new LinkedList<>(partitioningKeys.get(0)), recType);
             } catch (AsterixException e) {
                 throw new AlgebricksException(e);
             }
@@ -81,7 +81,7 @@
         List<IAType> fieldTypes = new LinkedList<>();
         int j = 0;
         for (int i = 0; i < recType.getFieldNames().length; i++) {
-            IAType fieldType = null;
+            IAType fieldType;
             if (partitioningKeys.get(0).equals(recType.getFieldNames()[j])) {
                 if (recType.getFieldTypes()[j].getTypeTag() == ATypeTag.RECORD) {
                     if (j != 0) {
@@ -132,15 +132,10 @@
             List<LogicalVariable> minFilterVars, List<LogicalVariable> maxFilterVars, IOperatorSchema opSchema,
             IVariableTypeEnvironment typeEnv, JobGenContext context, JobSpecification jobSpec, Object implConfig)
             throws AlgebricksException {
-        // This is a load into dataset operation
         LoadableDataSource alds = (LoadableDataSource) dataSource;
-        List<List<String>> aldsPartitioningKeys = alds.getPartitioningKeys();
-        boolean isAldsPKAutoGenerated = ((InternalDatasetDetails) alds.getTargetDataset().getDatasetDetails())
-                .isAutogenerated();
         ARecordType itemType = (ARecordType) alds.getLoadedType();
         IAdapterFactory adapterFactory = metadataProvider.getConfiguredAdapterFactory(alds.getTargetDataset(),
-                alds.getAdapter(), alds.getAdapterProperties(), itemType, isAldsPKAutoGenerated, aldsPartitioningKeys,
-                null);
+                alds.getAdapter(), alds.getAdapterProperties(), itemType, null);
         RecordDescriptor rDesc = JobGenHelper.mkRecordDescriptor(typeEnv, opSchema, context);
         return metadataProvider.buildLoadableDatasetScan(jobSpec, adapterFactory, rDesc);
     }
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/MetadataManagerUtil.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/MetadataManagerUtil.java
index 536a688..f1a90c7 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/MetadataManagerUtil.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/MetadataManagerUtil.java
@@ -35,7 +35,7 @@
 import org.apache.asterix.metadata.utils.MetadataConstants;
 import org.apache.asterix.om.types.ARecordType;
 import org.apache.asterix.om.types.IAType;
-import org.apache.asterix.runtime.util.ClusterStateManager;
+import org.apache.asterix.runtime.utils.ClusterStateManager;
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
 import org.apache.hyracks.algebricks.core.algebra.properties.DefaultNodeGroupDomain;
 import org.apache.hyracks.algebricks.core.algebra.properties.INodeDomain;
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 7238db9..1beaed0 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
@@ -30,24 +30,17 @@
 import org.apache.asterix.common.config.DatasetConfig.IndexType;
 import org.apache.asterix.common.config.GlobalConfig;
 import org.apache.asterix.common.config.StorageProperties;
-import org.apache.asterix.common.context.AsterixVirtualBufferCacheProvider;
+import org.apache.asterix.common.context.IStorageComponentProvider;
 import org.apache.asterix.common.context.ITransactionSubsystemProvider;
-import org.apache.asterix.common.context.TransactionSubsystemProvider;
 import org.apache.asterix.common.dataflow.IApplicationContextInfo;
-import org.apache.asterix.common.dataflow.LSMIndexUtil;
 import org.apache.asterix.common.dataflow.LSMInvertedIndexInsertDeleteOperatorDescriptor;
 import org.apache.asterix.common.dataflow.LSMTreeInsertDeleteOperatorDescriptor;
 import org.apache.asterix.common.exceptions.AsterixException;
 import org.apache.asterix.common.exceptions.CompilationException;
-import org.apache.asterix.common.ioopcallbacks.LSMBTreeIOOperationCallbackFactory;
-import org.apache.asterix.common.ioopcallbacks.LSMBTreeWithBuddyIOOperationCallbackFactory;
-import org.apache.asterix.common.ioopcallbacks.LSMInvertedIndexIOOperationCallbackFactory;
-import org.apache.asterix.common.ioopcallbacks.LSMRTreeIOOperationCallbackFactory;
 import org.apache.asterix.common.library.ILibraryManager;
 import org.apache.asterix.common.transactions.IRecoveryManager.ResourceType;
 import org.apache.asterix.common.transactions.JobId;
 import org.apache.asterix.common.utils.StoragePathUtil;
-import org.apache.asterix.dataflow.data.nontagged.valueproviders.PrimitiveValueProviderFactory;
 import org.apache.asterix.external.adapter.factory.LookupAdapterFactory;
 import org.apache.asterix.external.api.IAdapterFactory;
 import org.apache.asterix.external.api.IDataSourceAdapter;
@@ -55,9 +48,9 @@
 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.ExternalDataScanOperatorDescriptor;
 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;
 import org.apache.asterix.external.util.ExternalDataConstants;
@@ -79,29 +72,26 @@
 import org.apache.asterix.metadata.entities.Index;
 import org.apache.asterix.metadata.entities.InternalDatasetDetails;
 import org.apache.asterix.metadata.feeds.FeedMetadataUtil;
-import org.apache.asterix.metadata.utils.DatasetUtils;
-import org.apache.asterix.metadata.utils.ExternalDatasetsRegistry;
+import org.apache.asterix.metadata.utils.DatasetUtil;
 import org.apache.asterix.metadata.utils.MetadataConstants;
 import org.apache.asterix.metadata.utils.SplitsAndConstraintsUtil;
 import org.apache.asterix.om.functions.BuiltinFunctions;
 import org.apache.asterix.om.types.ARecordType;
 import org.apache.asterix.om.types.ATypeTag;
 import org.apache.asterix.om.types.IAType;
-import org.apache.asterix.om.util.NonTaggedFormatUtil;
+import org.apache.asterix.om.utils.NonTaggedFormatUtil;
 import org.apache.asterix.runtime.base.AsterixTupleFilterFactory;
 import org.apache.asterix.runtime.formats.FormatUtils;
 import org.apache.asterix.runtime.job.listener.JobEventListenerFactory;
 import org.apache.asterix.runtime.operators.LSMInvertedIndexUpsertOperatorDescriptor;
 import org.apache.asterix.runtime.operators.LSMTreeUpsertOperatorDescriptor;
-import org.apache.asterix.runtime.util.AppContextInfo;
-import org.apache.asterix.runtime.util.ClusterStateManager;
-import org.apache.asterix.runtime.util.RuntimeComponentsProvider;
+import org.apache.asterix.runtime.utils.AppContextInfo;
+import org.apache.asterix.runtime.utils.ClusterStateManager;
+import org.apache.asterix.runtime.utils.RuntimeComponentsProvider;
 import org.apache.asterix.transaction.management.opcallbacks.LockThenSearchOperationCallbackFactory;
 import org.apache.asterix.transaction.management.opcallbacks.PrimaryIndexInstantSearchOperationCallbackFactory;
 import org.apache.asterix.transaction.management.opcallbacks.PrimaryIndexModificationOperationCallbackFactory;
-import org.apache.asterix.transaction.management.opcallbacks.PrimaryIndexOperationTrackerProvider;
 import org.apache.asterix.transaction.management.opcallbacks.SecondaryIndexModificationOperationCallbackFactory;
-import org.apache.asterix.transaction.management.opcallbacks.SecondaryIndexOperationTrackerProvider;
 import org.apache.asterix.transaction.management.opcallbacks.SecondaryIndexSearchOperationCallbackFactory;
 import org.apache.asterix.transaction.management.opcallbacks.TempDatasetPrimaryIndexModificationOperationCallbackFactory;
 import org.apache.asterix.transaction.management.opcallbacks.TempDatasetSecondaryIndexModificationOperationCallbackFactory;
@@ -148,6 +138,7 @@
 import org.apache.hyracks.dataflow.std.file.IFileSplitProvider;
 import org.apache.hyracks.dataflow.std.result.ResultWriterOperatorDescriptor;
 import org.apache.hyracks.storage.am.btree.dataflow.BTreeSearchOperatorDescriptor;
+import org.apache.hyracks.storage.am.common.api.IMetadataPageManagerFactory;
 import org.apache.hyracks.storage.am.common.api.IModificationOperationCallbackFactory;
 import org.apache.hyracks.storage.am.common.api.IPrimitiveValueProviderFactory;
 import org.apache.hyracks.storage.am.common.api.ISearchOperationCallbackFactory;
@@ -155,22 +146,18 @@
 import org.apache.hyracks.storage.am.common.dataflow.TreeIndexBulkLoadOperatorDescriptor;
 import org.apache.hyracks.storage.am.common.impls.NoOpOperationCallbackFactory;
 import org.apache.hyracks.storage.am.common.ophelpers.IndexOperation;
-import org.apache.hyracks.storage.am.lsm.btree.dataflow.ExternalBTreeDataflowHelperFactory;
-import org.apache.hyracks.storage.am.lsm.btree.dataflow.ExternalBTreeWithBuddyDataflowHelperFactory;
-import org.apache.hyracks.storage.am.lsm.btree.dataflow.LSMBTreeDataflowHelperFactory;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMMergePolicyFactory;
 import org.apache.hyracks.storage.am.lsm.invertedindex.dataflow.BinaryTokenizerOperatorDescriptor;
 import org.apache.hyracks.storage.am.lsm.invertedindex.dataflow.LSMInvertedIndexBulkLoadOperatorDescriptor;
-import org.apache.hyracks.storage.am.lsm.invertedindex.dataflow.LSMInvertedIndexDataflowHelperFactory;
-import org.apache.hyracks.storage.am.lsm.invertedindex.dataflow.PartitionedLSMInvertedIndexDataflowHelperFactory;
 import org.apache.hyracks.storage.am.lsm.invertedindex.tokenizers.IBinaryTokenizerFactory;
-import org.apache.hyracks.storage.am.lsm.rtree.dataflow.ExternalRTreeDataflowHelperFactory;
-import org.apache.hyracks.storage.am.lsm.rtree.dataflow.LSMRTreeWithAntiMatterTuplesDataflowHelperFactory;
 import org.apache.hyracks.storage.am.rtree.dataflow.RTreeSearchOperatorDescriptor;
-import org.apache.hyracks.storage.am.rtree.frames.RTreePolicyType;
 
 public class MetadataProvider implements IMetadataProvider<DataSourceId, String> {
 
+    private final IStorageComponentProvider storaegComponentProvider;
+    private final ITransactionSubsystemProvider txnSubsystemProvider;
+    private final IMetadataPageManagerFactory metadataPageManagerFactory;
+    private final IPrimitiveValueProviderFactory primitiveValueProviderFactory;
     private final StorageProperties storageProperties;
     private final ILibraryManager libraryManager;
     private final Dataverse defaultDataverse;
@@ -188,10 +175,14 @@
     private boolean isTemporaryDatasetWriteJob = true;
     private boolean blockingOperatorDisabled = false;
 
-    public MetadataProvider(Dataverse defaultDataverse) {
+    public MetadataProvider(Dataverse defaultDataverse, IStorageComponentProvider componentProvider) {
         this.defaultDataverse = defaultDataverse;
-        this.storageProperties = AppContextInfo.INSTANCE.getStorageProperties();
-        this.libraryManager = AppContextInfo.INSTANCE.getLibraryManager();
+        this.storaegComponentProvider = componentProvider;
+        storageProperties = AppContextInfo.INSTANCE.getStorageProperties();
+        libraryManager = AppContextInfo.INSTANCE.getLibraryManager();
+        txnSubsystemProvider = componentProvider.getTransactionSubsystemProvider();
+        metadataPageManagerFactory = componentProvider.getMetadataPageManagerFactory();
+        primitiveValueProviderFactory = componentProvider.getPrimitiveValueProviderFactory();
     }
 
     public String getPropertyValue(String propertyName) {
@@ -408,8 +399,8 @@
     protected Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> buildLoadableDatasetScan(
             JobSpecification jobSpec, IAdapterFactory adapterFactory, RecordDescriptor rDesc)
             throws AlgebricksException {
-        ExternalDataScanOperatorDescriptor dataScanner = new ExternalDataScanOperatorDescriptor(jobSpec, rDesc,
-                adapterFactory);
+        ExternalScanOperatorDescriptor dataScanner =
+                new ExternalScanOperatorDescriptor(jobSpec, rDesc, adapterFactory);
         AlgebricksPartitionConstraint constraint;
         try {
             constraint = adapterFactory.getPartitionConstraint();
@@ -437,8 +428,8 @@
     public Triple<IOperatorDescriptor, AlgebricksPartitionConstraint, IAdapterFactory> buildFeedIntakeRuntime(
             JobSpecification jobSpec, Feed primaryFeed, FeedPolicyAccessor policyAccessor) throws Exception {
         Triple<IAdapterFactory, RecordDescriptor, IDataSourceAdapter.AdapterType> factoryOutput;
-        factoryOutput = FeedMetadataUtil.getPrimaryFeedFactoryAndOutput(primaryFeed, policyAccessor, mdTxnCtx,
-                libraryManager);
+        factoryOutput =
+                FeedMetadataUtil.getPrimaryFeedFactoryAndOutput(primaryFeed, policyAccessor, mdTxnCtx, libraryManager);
         ARecordType recordType = FeedMetadataUtil.getOutputType(primaryFeed, primaryFeed.getAdapterConfiguration(),
                 ExternalDataConstants.KEY_TYPE_NAME);
         IAdapterFactory adapterFactory = factoryOutput.first;
@@ -454,6 +445,8 @@
                 feedIngestor = new FeedIntakeOperatorDescriptor(jobSpec, primaryFeed, libraryName,
                         adapterFactory.getClass().getName(), recordType, policyAccessor, factoryOutput.second);
                 break;
+            default:
+                break;
         }
 
         AlgebricksPartitionConstraint partitionConstraint = adapterFactory.getPartitionConstraint();
@@ -461,10 +454,10 @@
     }
 
     public Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> buildBtreeRuntime(JobSpecification jobSpec,
-            List<LogicalVariable> outputVars, IOperatorSchema opSchema, IVariableTypeEnvironment typeEnv,
-            JobGenContext context, boolean retainInput, boolean retainMissing, Dataset dataset, String indexName,
-            int[] lowKeyFields, int[] highKeyFields, boolean lowKeyInclusive, boolean highKeyInclusive,
-            Object implConfig, int[] minFilterFieldIndexes, int[] maxFilterFieldIndexes) throws AlgebricksException {
+            IOperatorSchema opSchema, IVariableTypeEnvironment typeEnv, JobGenContext context, boolean retainInput,
+            boolean retainMissing, Dataset dataset, String indexName, int[] lowKeyFields, int[] highKeyFields,
+            boolean lowKeyInclusive, boolean highKeyInclusive, int[] minFilterFieldIndexes,
+            int[] maxFilterFieldIndexes) throws AlgebricksException {
         boolean isSecondary = true;
         int numSecondaryKeys = 0;
         try {
@@ -474,43 +467,38 @@
             if (primaryIndex != null && (dataset.getDatasetType() != DatasetType.EXTERNAL)) {
                 isSecondary = !indexName.equals(primaryIndex.getIndexName());
             }
-            int numPrimaryKeys = DatasetUtils.getPartitioningKeys(dataset).size();
+            Index theIndex = isSecondary ? MetadataManager.INSTANCE.getIndex(mdTxnCtx, dataset.getDataverseName(),
+                    dataset.getDatasetName(), indexName) : primaryIndex;
+            int numPrimaryKeys = DatasetUtil.getPartitioningKeys(dataset).size();
             RecordDescriptor outputRecDesc = JobGenHelper.mkRecordDescriptor(typeEnv, opSchema, context);
             int[] bloomFilterKeyFields;
             ITypeTraits[] typeTraits;
             IBinaryComparatorFactory[] comparatorFactories;
 
-            ARecordType itemType = (ARecordType) this.findType(dataset.getItemTypeDataverseName(),
-                    dataset.getItemTypeName());
+            ARecordType itemType =
+                    (ARecordType) this.findType(dataset.getItemTypeDataverseName(), dataset.getItemTypeName());
             ARecordType metaType = null;
             List<Integer> primaryKeyIndicators = null;
             if (dataset.hasMetaPart()) {
-                metaType = (ARecordType) findType(dataset.getMetaItemTypeDataverseName(),
-                        dataset.getMetaItemTypeName());
+                metaType =
+                        (ARecordType) findType(dataset.getMetaItemTypeDataverseName(), dataset.getMetaItemTypeName());
                 primaryKeyIndicators = ((InternalDatasetDetails) dataset.getDatasetDetails()).getKeySourceIndicator();
             }
 
-            ITypeTraits[] filterTypeTraits = DatasetUtils.computeFilterTypeTraits(dataset, itemType);
-            IBinaryComparatorFactory[] filterCmpFactories = DatasetUtils.computeFilterBinaryComparatorFactories(dataset,
-                    itemType, context.getBinaryComparatorFactoryProvider());
-            int[] filterFields = null;
-            int[] btreeFields = null;
-
+            ITypeTraits[] filterTypeTraits = DatasetUtil.computeFilterTypeTraits(dataset, itemType);
+            int[] filterFields;
+            int[] btreeFields;
             if (isSecondary) {
-                Index secondaryIndex = MetadataManager.INSTANCE.getIndex(mdTxnCtx, dataset.getDataverseName(),
-                        dataset.getDatasetName(), indexName);
-                numSecondaryKeys = secondaryIndex.getKeyFieldNames().size();
+                numSecondaryKeys = theIndex.getKeyFieldNames().size();
                 bloomFilterKeyFields = new int[numSecondaryKeys];
                 for (int i = 0; i < numSecondaryKeys; i++) {
                     bloomFilterKeyFields[i] = i;
                 }
                 Pair<IBinaryComparatorFactory[], ITypeTraits[]> comparatorFactoriesAndTypeTraits =
-                        getComparatorFactoriesAndTypeTraitsOfSecondaryBTreeIndex(
-                                secondaryIndex.getKeyFieldNames(), secondaryIndex.getKeyFieldTypes(),
-                                DatasetUtils.getPartitioningKeys(dataset), itemType, dataset.getDatasetType(),
-                                dataset.hasMetaPart(), primaryKeyIndicators,
-                                secondaryIndex.getKeyFieldSourceIndicators(),
-                                metaType);
+                        getComparatorFactoriesAndTypeTraitsOfSecondaryBTreeIndex(theIndex.getKeyFieldNames(),
+                                theIndex.getKeyFieldTypes(), DatasetUtil.getPartitioningKeys(dataset), itemType,
+                                dataset.getDatasetType(), dataset.hasMetaPart(), primaryKeyIndicators,
+                                theIndex.getKeyFieldSourceIndicators(), metaType);
                 comparatorFactories = comparatorFactoriesAndTypeTraits.first;
                 typeTraits = comparatorFactoriesAndTypeTraits.second;
                 if (filterTypeTraits != null) {
@@ -528,77 +516,58 @@
                     bloomFilterKeyFields[i] = i;
                 }
                 // get meta item type
-                ARecordType metaItemType = DatasetUtils.getMetaType(this, dataset);
-                typeTraits = DatasetUtils.computeTupleTypeTraits(dataset, itemType, metaItemType);
-                comparatorFactories = DatasetUtils.computeKeysBinaryComparatorFactories(dataset, itemType, metaItemType,
+                ARecordType metaItemType = DatasetUtil.getMetaType(this, dataset);
+                typeTraits = DatasetUtil.computeTupleTypeTraits(dataset, itemType, metaItemType);
+                comparatorFactories = DatasetUtil.computeKeysBinaryComparatorFactories(dataset, itemType, metaItemType,
                         context.getBinaryComparatorFactoryProvider());
-                filterFields = DatasetUtils.createFilterFields(dataset);
-                btreeFields = DatasetUtils.createBTreeFieldsWhenThereisAFilter(dataset);
             }
 
             IApplicationContextInfo appContext = (IApplicationContextInfo) context.getAppContext();
             Pair<IFileSplitProvider, AlgebricksPartitionConstraint> spPc;
-            spPc = splitProviderAndPartitionConstraintsForDataset(dataset.getDataverseName(), dataset.getDatasetName(),
-                    indexName, temp);
+            spPc = getSplitProviderAndConstraints(dataset.getDataverseName(), dataset.getDatasetName(), indexName,
+                    temp);
 
             ISearchOperationCallbackFactory searchCallbackFactory;
             if (isSecondary) {
                 searchCallbackFactory = temp ? NoOpOperationCallbackFactory.INSTANCE
                         : new SecondaryIndexSearchOperationCallbackFactory();
             } else {
-                JobId jobId = ((JobEventListenerFactory) jobSpec.getJobletEventListenerFactory()).getJobId();
                 int datasetId = dataset.getDatasetId();
                 int[] primaryKeyFields = new int[numPrimaryKeys];
                 for (int i = 0; i < numPrimaryKeys; i++) {
                     primaryKeyFields[i] = i;
                 }
 
-                ITransactionSubsystemProvider txnSubsystemProvider = new TransactionSubsystemProvider();
-
                 /**
                  * Due to the read-committed isolation level,
                  * we may acquire very short duration lock(i.e., instant lock) for readers.
                  */
                 searchCallbackFactory = temp ? NoOpOperationCallbackFactory.INSTANCE
-                        : new PrimaryIndexInstantSearchOperationCallbackFactory(jobId, datasetId, primaryKeyFields,
-                                txnSubsystemProvider, ResourceType.LSM_BTREE);
+                        : new PrimaryIndexInstantSearchOperationCallbackFactory(
+                                ((JobEventListenerFactory) jobSpec.getJobletEventListenerFactory()).getJobId(),
+                                datasetId, primaryKeyFields, txnSubsystemProvider, ResourceType.LSM_BTREE);
             }
-            Pair<ILSMMergePolicyFactory, Map<String, String>> compactionInfo = DatasetUtils
-                    .getMergePolicyFactory(dataset, mdTxnCtx);
+            Pair<ILSMMergePolicyFactory, Map<String, String>> compactionInfo =
+                    DatasetUtil.getMergePolicyFactory(dataset, mdTxnCtx);
             RuntimeComponentsProvider rtcProvider = RuntimeComponentsProvider.RUNTIME_PROVIDER;
             BTreeSearchOperatorDescriptor btreeSearchOp;
             if (dataset.getDatasetType() == DatasetType.INTERNAL) {
                 btreeSearchOp = new BTreeSearchOperatorDescriptor(jobSpec, outputRecDesc,
-                        appContext.getStorageManagerInterface(), appContext.getIndexLifecycleManagerProvider(),
-                        spPc.first, typeTraits, comparatorFactories, bloomFilterKeyFields, lowKeyFields, highKeyFields,
+                        appContext.getStorageManager(), appContext.getIndexLifecycleManagerProvider(), spPc.first,
+                        typeTraits, comparatorFactories, bloomFilterKeyFields, lowKeyFields, highKeyFields,
                         lowKeyInclusive, highKeyInclusive,
-                        new LSMBTreeDataflowHelperFactory(new AsterixVirtualBufferCacheProvider(dataset.getDatasetId()),
-                                compactionInfo.first, compactionInfo.second,
-                                isSecondary ? new SecondaryIndexOperationTrackerProvider(dataset.getDatasetId())
-                                        : new PrimaryIndexOperationTrackerProvider(dataset.getDatasetId()),
-                                rtcProvider, LSMBTreeIOOperationCallbackFactory.INSTANCE,
-                                storageProperties.getBloomFilterFalsePositiveRate(), !isSecondary, filterTypeTraits,
-                                filterCmpFactories, btreeFields, filterFields, !temp),
+                        dataset.getIndexDataflowHelperFactory(this, theIndex, itemType, metaType, compactionInfo.first,
+                                compactionInfo.second),
                         retainInput, retainMissing, context.getMissingWriterFactory(), searchCallbackFactory,
-                        minFilterFieldIndexes, maxFilterFieldIndexes, LSMIndexUtil
-                                .getMetadataPageManagerFactory());
+                        minFilterFieldIndexes, maxFilterFieldIndexes, metadataPageManagerFactory);
             } else {
-                // External dataset <- use the btree with buddy btree->
-                // Be Careful of Key Start Index ?
-                int[] buddyBreeFields = new int[] { numSecondaryKeys };
-                ExternalBTreeWithBuddyDataflowHelperFactory indexDataflowHelperFactory =
-                        new ExternalBTreeWithBuddyDataflowHelperFactory(
-                                compactionInfo.first, compactionInfo.second,
-                                new SecondaryIndexOperationTrackerProvider(dataset.getDatasetId()),
-                                RuntimeComponentsProvider.RUNTIME_PROVIDER,
-                                LSMBTreeWithBuddyIOOperationCallbackFactory.INSTANCE,
-                                getStorageProperties().getBloomFilterFalsePositiveRate(), buddyBreeFields,
-                                ExternalDatasetsRegistry.INSTANCE.getAndLockDatasetVersion(dataset, this), !temp);
+                IIndexDataflowHelperFactory indexDataflowHelperFactory = dataset.getIndexDataflowHelperFactory(this,
+                        theIndex, itemType, metaType, compactionInfo.first, compactionInfo.second);
                 btreeSearchOp = new ExternalBTreeSearchOperatorDescriptor(jobSpec, outputRecDesc, rtcProvider,
                         rtcProvider, spPc.first, typeTraits, comparatorFactories, bloomFilterKeyFields, lowKeyFields,
                         highKeyFields, lowKeyInclusive, highKeyInclusive, indexDataflowHelperFactory, retainInput,
-                        retainMissing, context.getMissingWriterFactory(), searchCallbackFactory, LSMIndexUtil
-                                .getMetadataPageManagerFactory());
+                        retainMissing, context.getMissingWriterFactory(), searchCallbackFactory,
+                        metadataPageManagerFactory);
             }
             return new Pair<>(btreeSearchOp, spPc.second);
         } catch (MetadataException me) {
@@ -611,8 +580,9 @@
             JobGenContext context, boolean retainInput, boolean retainMissing, Dataset dataset, String indexName,
             int[] keyFields, int[] minFilterFieldIndexes, int[] maxFilterFieldIndexes) throws AlgebricksException {
         try {
-            ARecordType recType = (ARecordType) findType(dataset.getItemTypeDataverseName(), dataset.getItemTypeName());
-            int numPrimaryKeys = DatasetUtils.getPartitioningKeys(dataset).size();
+            ARecordType recType =
+                    (ARecordType) findType(dataset.getItemTypeDataverseName(), dataset.getItemTypeName());
+            int numPrimaryKeys = DatasetUtil.getPartitioningKeys(dataset).size();
 
             boolean temp = dataset.getDatasetDetails().isTemp();
             Index secondaryIndex = MetadataManager.INSTANCE.getIndex(mdTxnCtx, dataset.getDataverseName(),
@@ -629,19 +599,18 @@
                         "Cannot use " + numSecondaryKeys + " fields as a key for the R-tree index. "
                                 + "There can be only one field as a key for the R-tree index.");
             }
-            Pair<IAType, Boolean> keyTypePair = Index.getNonNullableOpenFieldType(secondaryKeyTypes.get(0),
-                    secondaryKeyFields.get(0), recType);
+            Pair<IAType, Boolean> keyTypePair =
+                    Index.getNonNullableOpenFieldType(secondaryKeyTypes.get(0), secondaryKeyFields.get(0), recType);
             IAType keyType = keyTypePair.first;
             if (keyType == null) {
                 throw new AlgebricksException("Could not find field " + secondaryKeyFields.get(0) + " in the schema.");
             }
             int numDimensions = NonTaggedFormatUtil.getNumDimensions(keyType.getTypeTag());
-            boolean isPointMBR = keyType.getTypeTag() == ATypeTag.POINT || keyType.getTypeTag() == ATypeTag.POINT3D;
             int numNestedSecondaryKeyFields = numDimensions * 2;
             IPrimitiveValueProviderFactory[] valueProviderFactories =
                     new IPrimitiveValueProviderFactory[numNestedSecondaryKeyFields];
             for (int i = 0; i < numNestedSecondaryKeyFields; i++) {
-                valueProviderFactories[i] = PrimitiveValueProviderFactory.INSTANCE;
+                valueProviderFactories[i] = primitiveValueProviderFactory;
             }
 
             RecordDescriptor outputRecDesc = JobGenHelper.mkRecordDescriptor(typeEnv, opSchema, context);
@@ -655,27 +624,24 @@
             ITypeTraits[] typeTraits = JobGenHelper.variablesToTypeTraits(outputVars, keysStartIndex,
                     numNestedSecondaryKeyFields + numPrimaryKeys, typeEnv, context);
             IApplicationContextInfo appContext = (IApplicationContextInfo) context.getAppContext();
-            Pair<IFileSplitProvider, AlgebricksPartitionConstraint> spPc =
-                    splitProviderAndPartitionConstraintsForDataset(dataset.getDataverseName(),
-                            dataset.getDatasetName(), indexName, temp);
+            Pair<IFileSplitProvider, AlgebricksPartitionConstraint> spPc = getSplitProviderAndConstraints(
+                    dataset.getDataverseName(), dataset.getDatasetName(), indexName, temp);
             ARecordType metaType = null;
             if (dataset.hasMetaPart()) {
-                metaType = (ARecordType) findType(dataset.getMetaItemTypeDataverseName(),
-                        dataset.getMetaItemTypeName());
+                metaType =
+                        (ARecordType) findType(dataset.getMetaItemTypeDataverseName(), dataset.getMetaItemTypeName());
             }
 
-            IBinaryComparatorFactory[] primaryComparatorFactories = DatasetUtils.computeKeysBinaryComparatorFactories(
+            IBinaryComparatorFactory[] primaryComparatorFactories = DatasetUtil.computeKeysBinaryComparatorFactories(
                     dataset, recType, metaType, context.getBinaryComparatorFactoryProvider());
             int[] btreeFields = new int[primaryComparatorFactories.length];
             for (int i = 0; i < btreeFields.length; i++) {
                 btreeFields[i] = i + numNestedSecondaryKeyFields;
             }
 
-            ITypeTraits[] filterTypeTraits = DatasetUtils.computeFilterTypeTraits(dataset, recType);
-            IBinaryComparatorFactory[] filterCmpFactories = DatasetUtils.computeFilterBinaryComparatorFactories(dataset,
-                    recType, context.getBinaryComparatorFactoryProvider());
-            int[] filterFields = null;
-            int[] rtreeFields = null;
+            ITypeTraits[] filterTypeTraits = DatasetUtil.computeFilterTypeTraits(dataset, recType);
+            int[] filterFields;
+            int[] rtreeFields;
             if (filterTypeTraits != null) {
                 filterFields = new int[1];
                 filterFields[0] = numNestedSecondaryKeyFields + numPrimaryKeys;
@@ -684,46 +650,26 @@
                     rtreeFields[i] = i;
                 }
             }
-
-            IAType nestedKeyType = NonTaggedFormatUtil.getNestedSpatialType(keyType.getTypeTag());
-            Pair<ILSMMergePolicyFactory, Map<String, String>> compactionInfo = DatasetUtils
-                    .getMergePolicyFactory(dataset, mdTxnCtx);
-            ISearchOperationCallbackFactory searchCallbackFactory = temp ? NoOpOperationCallbackFactory.INSTANCE
-                    : new SecondaryIndexSearchOperationCallbackFactory();
-
+            Pair<ILSMMergePolicyFactory, Map<String, String>> compactionInfo =
+                    DatasetUtil.getMergePolicyFactory(dataset, mdTxnCtx);
+            ISearchOperationCallbackFactory searchCallbackFactory =
+                    temp ? NoOpOperationCallbackFactory.INSTANCE : new SecondaryIndexSearchOperationCallbackFactory();
             RTreeSearchOperatorDescriptor rtreeSearchOp;
+            IIndexDataflowHelperFactory indexDataflowHelperFactory = dataset.getIndexDataflowHelperFactory(this,
+                    secondaryIndex, recType, metaType, compactionInfo.first, compactionInfo.second);
             if (dataset.getDatasetType() == DatasetType.INTERNAL) {
-                IBinaryComparatorFactory[] deletedKeyBTreeCompFactories = getMergedComparatorFactories(
-                        comparatorFactories, primaryComparatorFactories);
-                IIndexDataflowHelperFactory idff = new LSMRTreeWithAntiMatterTuplesDataflowHelperFactory(
-                        valueProviderFactories, RTreePolicyType.RTREE, deletedKeyBTreeCompFactories,
-                        new AsterixVirtualBufferCacheProvider(dataset.getDatasetId()), compactionInfo.first,
-                        compactionInfo.second, new SecondaryIndexOperationTrackerProvider(dataset.getDatasetId()),
-                        RuntimeComponentsProvider.RUNTIME_PROVIDER, LSMRTreeIOOperationCallbackFactory.INSTANCE,
-                        MetadataProvider.proposeLinearizer(nestedKeyType.getTypeTag(), comparatorFactories.length),
-                        rtreeFields, filterTypeTraits, filterCmpFactories, filterFields, !temp, isPointMBR);
                 rtreeSearchOp = new RTreeSearchOperatorDescriptor(jobSpec, outputRecDesc,
-                        appContext.getStorageManagerInterface(), appContext.getIndexLifecycleManagerProvider(),
-                        spPc.first, typeTraits, comparatorFactories, keyFields, idff, retainInput, retainMissing,
-                        context.getMissingWriterFactory(), searchCallbackFactory, minFilterFieldIndexes,
-                        maxFilterFieldIndexes, LSMIndexUtil.getMetadataPageManagerFactory());
+                        appContext.getStorageManager(), appContext.getIndexLifecycleManagerProvider(), spPc.first,
+                        typeTraits, comparatorFactories, keyFields, indexDataflowHelperFactory, retainInput,
+                        retainMissing, context.getMissingWriterFactory(), searchCallbackFactory, minFilterFieldIndexes,
+                        maxFilterFieldIndexes, metadataPageManagerFactory);
             } else {
-                // External Dataset
-                ExternalRTreeDataflowHelperFactory indexDataflowHelperFactory = new ExternalRTreeDataflowHelperFactory(
-                        valueProviderFactories, RTreePolicyType.RTREE,
-                        IndexingConstants.getBuddyBtreeComparatorFactories(), compactionInfo.first,
-                        compactionInfo.second, new SecondaryIndexOperationTrackerProvider(dataset.getDatasetId()),
-                        RuntimeComponentsProvider.RUNTIME_PROVIDER, LSMRTreeIOOperationCallbackFactory.INSTANCE,
-                        proposeLinearizer(nestedKeyType.getTypeTag(), comparatorFactories.length),
-                        getStorageProperties().getBloomFilterFalsePositiveRate(),
-                        new int[] { numNestedSecondaryKeyFields },
-                        ExternalDatasetsRegistry.INSTANCE.getAndLockDatasetVersion(dataset, this), !temp, isPointMBR);
                 // Create the operator
                 rtreeSearchOp = new ExternalRTreeSearchOperatorDescriptor(jobSpec, outputRecDesc,
-                        appContext.getStorageManagerInterface(), appContext.getIndexLifecycleManagerProvider(),
-                        spPc.first, typeTraits, comparatorFactories, keyFields, indexDataflowHelperFactory, retainInput,
-                        retainMissing, context.getMissingWriterFactory(), searchCallbackFactory, LSMIndexUtil
-                                .getMetadataPageManagerFactory());
+                        appContext.getStorageManager(), appContext.getIndexLifecycleManagerProvider(), spPc.first,
+                        typeTraits, comparatorFactories, keyFields, indexDataflowHelperFactory, retainInput,
+                        retainMissing, context.getMissingWriterFactory(), searchCallbackFactory,
+                        metadataPageManagerFactory);
             }
 
             return new Pair<>(rtreeSearchOp, spPc.second);
@@ -741,8 +687,8 @@
         File outFile = new File(fs.getPath());
         String nodeId = fs.getNodeName();
 
-        SinkWriterRuntimeFactory runtime = new SinkWriterRuntimeFactory(printColumns, printerFactories, outFile,
-                getWriterFactory(), inputDesc);
+        SinkWriterRuntimeFactory runtime =
+                new SinkWriterRuntimeFactory(printColumns, printerFactories, outFile, getWriterFactory(), inputDesc);
         AlgebricksPartitionConstraint apc = new AlgebricksAbsolutePartitionConstraint(new String[] { nodeId });
         return new Pair<>(runtime, apc);
     }
@@ -776,7 +722,7 @@
 
         Dataset dataset = MetadataManagerUtil.findExistingDataset(mdTxnCtx, dataverseName, datasetName);
         int numKeys = keys.size();
-        int numFilterFields = DatasetUtils.getFilterField(dataset) == null ? 0 : 1;
+        int numFilterFields = DatasetUtil.getFilterField(dataset) == null ? 0 : 1;
 
         // move key fields to front
         int[] fieldPermutation = new int[numKeys + 1 + numFilterFields];
@@ -797,56 +743,40 @@
         try {
             boolean temp = dataset.getDatasetDetails().isTemp();
             isTemporaryDatasetWriteJob = isTemporaryDatasetWriteJob && temp;
-
             Index primaryIndex = MetadataManager.INSTANCE.getIndex(mdTxnCtx, dataset.getDataverseName(),
                     dataset.getDatasetName(), dataset.getDatasetName());
             String indexName = primaryIndex.getIndexName();
-
-            ARecordType metaType = null;
-            if (dataset.hasMetaPart()) {
-                metaType = (ARecordType) findType(dataset.getMetaItemTypeDataverseName(),
-                        dataset.getMetaItemTypeName());
-            }
-
+            ARecordType metaType = dataset.hasMetaPart()
+                    ? (ARecordType) findType(dataset.getMetaItemTypeDataverseName(), dataset.getMetaItemTypeName())
+                    : null;
             String itemTypeName = dataset.getItemTypeName();
             ARecordType itemType = (ARecordType) MetadataManager.INSTANCE
                     .getDatatype(mdTxnCtx, dataset.getItemTypeDataverseName(), itemTypeName).getDatatype();
-            ITypeTraits[] typeTraits = DatasetUtils.computeTupleTypeTraits(dataset, itemType, null);
-            IBinaryComparatorFactory[] comparatorFactories = DatasetUtils.computeKeysBinaryComparatorFactories(dataset,
+            ITypeTraits[] typeTraits = DatasetUtil.computeTupleTypeTraits(dataset, itemType, null);
+            IBinaryComparatorFactory[] comparatorFactories = DatasetUtil.computeKeysBinaryComparatorFactories(dataset,
                     itemType, metaType, context.getBinaryComparatorFactoryProvider());
 
             Pair<IFileSplitProvider, AlgebricksPartitionConstraint> splitsAndConstraint =
-                    splitProviderAndPartitionConstraintsForDataset(dataSource.getId().getDataverseName(),
-                            datasetName, indexName, temp);
+                    getSplitProviderAndConstraints(dataSource.getId().getDataverseName(), datasetName, indexName,
+                            temp);
             IApplicationContextInfo appContext = (IApplicationContextInfo) context.getAppContext();
-
             long numElementsHint = getCardinalityPerPartitionHint(dataset);
 
-            ITypeTraits[] filterTypeTraits = DatasetUtils.computeFilterTypeTraits(dataset, itemType);
-            IBinaryComparatorFactory[] filterCmpFactories = DatasetUtils.computeFilterBinaryComparatorFactories(dataset,
-                    itemType, context.getBinaryComparatorFactoryProvider());
-            int[] filterFields = DatasetUtils.createFilterFields(dataset);
-            int[] btreeFields = DatasetUtils.createBTreeFieldsWhenThereisAFilter(dataset);
-
             // TODO
             // figure out the right behavior of the bulkload and then give the
             // right callback
             // (ex. what's the expected behavior when there is an error during
             // bulkload?)
-            Pair<ILSMMergePolicyFactory, Map<String, String>> compactionInfo = DatasetUtils
-                    .getMergePolicyFactory(dataset, mdTxnCtx);
-            TreeIndexBulkLoadOperatorDescriptor btreeBulkLoad = new TreeIndexBulkLoadOperatorDescriptor(spec, null,
-                    appContext.getStorageManagerInterface(), appContext.getIndexLifecycleManagerProvider(),
-                    splitsAndConstraint.first, typeTraits, comparatorFactories, bloomFilterKeyFields, fieldPermutation,
-                    GlobalConfig.DEFAULT_TREE_FILL_FACTOR, false, numElementsHint, true,
-                    new LSMBTreeDataflowHelperFactory(new AsterixVirtualBufferCacheProvider(dataset.getDatasetId()),
-                            compactionInfo.first, compactionInfo.second,
-                            new PrimaryIndexOperationTrackerProvider(dataset.getDatasetId()),
-                            RuntimeComponentsProvider.RUNTIME_PROVIDER,
-                            LSMBTreeIOOperationCallbackFactory.INSTANCE,
-                            storageProperties.getBloomFilterFalsePositiveRate(), true, filterTypeTraits,
-                            filterCmpFactories, btreeFields, filterFields, !temp), LSMIndexUtil
-                                    .getMetadataPageManagerFactory());
+            Pair<ILSMMergePolicyFactory, Map<String, String>> compactionInfo =
+                    DatasetUtil.getMergePolicyFactory(dataset, mdTxnCtx);
+            TreeIndexBulkLoadOperatorDescriptor btreeBulkLoad =
+                    new TreeIndexBulkLoadOperatorDescriptor(spec, null, appContext.getStorageManager(),
+                            appContext.getIndexLifecycleManagerProvider(), splitsAndConstraint.first, typeTraits,
+                            comparatorFactories, bloomFilterKeyFields, fieldPermutation,
+                            GlobalConfig.DEFAULT_TREE_FILL_FACTOR, false,
+                            numElementsHint, true, dataset.getIndexDataflowHelperFactory(this, primaryIndex, itemType,
+                                    metaType, compactionInfo.first, compactionInfo.second),
+                            metadataPageManagerFactory);
             return new Pair<>(btreeBulkLoad, splitsAndConstraint.second);
         } catch (MetadataException me) {
             throw new AlgebricksException(me);
@@ -957,10 +887,9 @@
      * @param dataset
      * @return Number of elements that will be used to create a bloom filter per
      *         dataset per partition
-     * @throws MetadataException
      * @throws AlgebricksException
      */
-    public long getCardinalityPerPartitionHint(Dataset dataset) throws MetadataException, AlgebricksException {
+    public long getCardinalityPerPartitionHint(Dataset dataset) throws AlgebricksException {
         String numElementsHintString = dataset.getHints().get(DatasetCardinalityHint.NAME);
         long numElementsHint;
         if (numElementsHintString == null) {
@@ -969,34 +898,32 @@
             numElementsHint = Long.parseLong(numElementsHintString);
         }
         int numPartitions = 0;
-        List<String> nodeGroup = MetadataManager.INSTANCE.getNodegroup(mdTxnCtx, dataset.getNodeGroupName())
-                .getNodeNames();
+        List<String> nodeGroup =
+                MetadataManager.INSTANCE.getNodegroup(mdTxnCtx, dataset.getNodeGroupName()).getNodeNames();
         for (String nd : nodeGroup) {
             numPartitions += ClusterStateManager.INSTANCE.getNodePartitionsCount(nd);
         }
-        numElementsHint = numElementsHint / numPartitions;
-        return numElementsHint;
+        return numElementsHint / numPartitions;
     }
 
     protected IAdapterFactory getConfiguredAdapterFactory(Dataset dataset, String adapterName,
-            Map<String, String> configuration, ARecordType itemType, boolean isPKAutoGenerated,
-            List<List<String>> primaryKeys, ARecordType metaType) throws AlgebricksException {
+            Map<String, String> configuration, ARecordType itemType, ARecordType metaType) throws AlgebricksException {
         try {
             configuration.put(ExternalDataConstants.KEY_DATAVERSE, dataset.getDataverseName());
             IAdapterFactory adapterFactory = AdapterFactoryProvider.getAdapterFactory(libraryManager, adapterName,
                     configuration, itemType, metaType);
 
             // check to see if dataset is indexed
-            Index filesIndex = MetadataManager.INSTANCE.getIndex(mdTxnCtx, dataset.getDataverseName(),
-                    dataset.getDatasetName(),
-                    dataset.getDatasetName().concat(IndexingConstants.EXTERNAL_FILE_INDEX_NAME_SUFFIX));
+            Index filesIndex =
+                    MetadataManager.INSTANCE.getIndex(mdTxnCtx, dataset.getDataverseName(), dataset.getDatasetName(),
+                            dataset.getDatasetName().concat(IndexingConstants.EXTERNAL_FILE_INDEX_NAME_SUFFIX));
 
             if (filesIndex != null && filesIndex.getPendingOp() == 0) {
                 // get files
                 List<ExternalFile> files = MetadataManager.INSTANCE.getDatasetExternalFiles(mdTxnCtx, dataset);
                 Iterator<ExternalFile> iterator = files.iterator();
                 while (iterator.hasNext()) {
-                    if (iterator.next().getPendingOp() != ExternalFilePendingOp.PENDING_NO_OP) {
+                    if (iterator.next().getPendingOp() != ExternalFilePendingOp.NO_OP) {
                         iterator.remove();
                     }
                 }
@@ -1018,27 +945,27 @@
                 numKeyFields / 2);
     }
 
-    public Pair<IFileSplitProvider, AlgebricksPartitionConstraint> splitProviderAndPartitionConstraintsForDataset(
-            String dataverseName, String datasetName, String targetIdxName, boolean temp) throws AlgebricksException {
+    public Pair<IFileSplitProvider, AlgebricksPartitionConstraint> getSplitProviderAndConstraints(String dataverseName,
+            String datasetName, String targetIdxName, boolean temp) throws AlgebricksException {
         FileSplit[] splits = splitsForDataset(mdTxnCtx, dataverseName, datasetName, targetIdxName, temp);
         return StoragePathUtil.splitProviderAndPartitionConstraints(splits);
     }
 
-    public Pair<IFileSplitProvider, AlgebricksPartitionConstraint> splitProviderAndPartitionConstraintsForDataverse(
-            String dataverse) {
-        return SplitsAndConstraintsUtil.splitProviderAndPartitionConstraintsForDataverse(dataverse);
+    public Pair<IFileSplitProvider, AlgebricksPartitionConstraint> splitAndConstraints(String dataverse) {
+        return SplitsAndConstraintsUtil.getDataverseSplitProviderAndConstraints(dataverse);
     }
 
     public FileSplit[] splitsForDataset(MetadataTransactionContext mdTxnCtx, String dataverseName, String datasetName,
             String targetIdxName, boolean temp) throws AlgebricksException {
-        return SplitsAndConstraintsUtil.splitsForDataset(mdTxnCtx, dataverseName, datasetName, targetIdxName, temp);
+        return SplitsAndConstraintsUtil.getDatasetSplits(mdTxnCtx, dataverseName, datasetName, targetIdxName, temp);
     }
 
     public DatasourceAdapter getAdapter(MetadataTransactionContext mdTxnCtx, String dataverseName, String adapterName)
             throws MetadataException {
         DatasourceAdapter adapter;
         // search in default namespace (built-in adapter)
-        adapter = MetadataManager.INSTANCE.getAdapter(mdTxnCtx, MetadataConstants.METADATA_DATAVERSE_NAME, adapterName);
+        adapter =
+                MetadataManager.INSTANCE.getAdapter(mdTxnCtx, MetadataConstants.METADATA_DATAVERSE_NAME, adapterName);
 
         // search in dataverse (user-defined adapter)
         if (adapter == null) {
@@ -1052,61 +979,61 @@
     }
 
     public Pair<IFileSplitProvider, AlgebricksPartitionConstraint> splitProviderAndPartitionConstraintsForFilesIndex(
-            String dataverseName, String datasetName, String targetIdxName, boolean create) throws AlgebricksException {
-        return SplitsAndConstraintsUtil.splitProviderAndPartitionConstraintsForFilesIndex(mdTxnCtx, dataverseName,
-                datasetName, targetIdxName, create);
+            String dataverseName, String datasetName, String targetIdxName, boolean create)
+            throws AlgebricksException {
+        return SplitsAndConstraintsUtil.getFilesIndexSplitProviderAndConstraints(mdTxnCtx, dataverseName, datasetName,
+                targetIdxName, create);
     }
 
     public Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> buildExternalDataLookupRuntime(
-            JobSpecification jobSpec, Dataset dataset, Index secondaryIndex, int[] ridIndexes, boolean retainInput,
-            IVariableTypeEnvironment typeEnv, List<LogicalVariable> outputVars, IOperatorSchema opSchema,
-            JobGenContext context, MetadataProvider metadataProvider, boolean retainMissing)
-            throws AlgebricksException {
+            JobSpecification jobSpec, Dataset dataset, int[] ridIndexes, boolean retainInput,
+            IVariableTypeEnvironment typeEnv, IOperatorSchema opSchema, JobGenContext context,
+            MetadataProvider metadataProvider, boolean retainMissing) throws AlgebricksException {
         try {
             // Get data type
-            IAType itemType;
-            itemType = MetadataManager.INSTANCE.getDatatype(metadataProvider.getMetadataTxnContext(),
-                    dataset.getDataverseName(), dataset.getItemTypeName()).getDatatype();
-
-            // Create the adapter factory <- right now there is only one. if there are more in the future, we can create
-            // a map->
+            ARecordType itemType =
+                    (ARecordType) MetadataManager.INSTANCE.getDatatype(metadataProvider.getMetadataTxnContext(),
+                            dataset.getDataverseName(), dataset.getItemTypeName()).getDatatype();
+            ARecordType metaType = null;
+            if (dataset.hasMetaPart()) {
+                metaType =
+                        (ARecordType) MetadataManager.INSTANCE
+                                .getDatatype(metadataProvider.getMetadataTxnContext(),
+                                        dataset.getMetaItemTypeDataverseName(), dataset.getMetaItemTypeName())
+                                .getDatatype();
+            }
             ExternalDatasetDetails datasetDetails = (ExternalDatasetDetails) dataset.getDatasetDetails();
-            LookupAdapterFactory<?> adapterFactory = AdapterFactoryProvider.getLookupAdapterFactory(libraryManager,
-                    datasetDetails.getProperties(), (ARecordType) itemType, ridIndexes, retainInput, retainMissing,
-                    context.getMissingWriterFactory());
+            LookupAdapterFactory<?> adapterFactory =
+                    AdapterFactoryProvider.getLookupAdapterFactory(libraryManager, datasetDetails.getProperties(),
+                            itemType, ridIndexes, retainInput, retainMissing, context.getMissingWriterFactory());
 
             Pair<ILSMMergePolicyFactory, Map<String, String>> compactionInfo;
             try {
-                compactionInfo = DatasetUtils.getMergePolicyFactory(dataset, metadataProvider.getMetadataTxnContext());
+                compactionInfo = DatasetUtil.getMergePolicyFactory(dataset, metadataProvider.getMetadataTxnContext());
             } catch (MetadataException e) {
                 throw new AlgebricksException(" Unabel to create merge policy factory for external dataset", e);
             }
 
             boolean temp = datasetDetails.isTemp();
+            String fileIndexName = BTreeDataflowHelperFactoryProvider.externalFileIndexName(dataset);
+            Index fileIndex = MetadataManager.INSTANCE.getIndex(mdTxnCtx, dataset.getDataverseName(),
+                    dataset.getDatasetName(), fileIndexName);
             // Create the file index data flow helper
-            ExternalBTreeDataflowHelperFactory indexDataflowHelperFactory = new ExternalBTreeDataflowHelperFactory(
-                    compactionInfo.first, compactionInfo.second,
-                    new SecondaryIndexOperationTrackerProvider(dataset.getDatasetId()),
-                    RuntimeComponentsProvider.RUNTIME_PROVIDER, LSMBTreeIOOperationCallbackFactory.INSTANCE,
-                    metadataProvider.getStorageProperties().getBloomFilterFalsePositiveRate(),
-                    ExternalDatasetsRegistry.INSTANCE.getAndLockDatasetVersion(dataset, metadataProvider), !temp);
-
+            IIndexDataflowHelperFactory indexDataflowHelperFactory = dataset.getIndexDataflowHelperFactory(this,
+                    fileIndex, itemType, metaType, compactionInfo.first, compactionInfo.second);
             // Create the out record descriptor, appContext and fileSplitProvider for the files index
             RecordDescriptor outRecDesc = JobGenHelper.mkRecordDescriptor(typeEnv, opSchema, context);
             IApplicationContextInfo appContext = (IApplicationContextInfo) context.getAppContext();
             Pair<IFileSplitProvider, AlgebricksPartitionConstraint> spPc;
             spPc = metadataProvider.splitProviderAndPartitionConstraintsForFilesIndex(dataset.getDataverseName(),
-                    dataset.getDatasetName(),
-                    dataset.getDatasetName().concat(IndexingConstants.EXTERNAL_FILE_INDEX_NAME_SUFFIX), false);
-            ISearchOperationCallbackFactory searchOpCallbackFactory = temp ? NoOpOperationCallbackFactory.INSTANCE
-                    : new SecondaryIndexSearchOperationCallbackFactory();
+                    dataset.getDatasetName(), fileIndexName, false);
+            ISearchOperationCallbackFactory searchOpCallbackFactory =
+                    temp ? NoOpOperationCallbackFactory.INSTANCE : new SecondaryIndexSearchOperationCallbackFactory();
             // Create the operator
             ExternalLookupOperatorDescriptor op = new ExternalLookupOperatorDescriptor(jobSpec, adapterFactory,
                     outRecDesc, indexDataflowHelperFactory, retainInput, appContext.getIndexLifecycleManagerProvider(),
-                    appContext.getStorageManagerInterface(), spPc.first, dataset.getDatasetId(),
-                    metadataProvider.getStorageProperties().getBloomFilterFalsePositiveRate(), searchOpCallbackFactory,
-                    retainMissing, context.getMissingWriterFactory(), LSMIndexUtil
-                            .getMetadataPageManagerFactory());
+                    appContext.getStorageManager(), spPc.first, searchOpCallbackFactory, retainMissing,
+                    context.getMissingWriterFactory(), metadataPageManagerFactory);
             return new Pair<>(op, spPc.second);
         } catch (Exception e) {
             throw new AlgebricksException(e);
@@ -1129,7 +1056,7 @@
         isTemporaryDatasetWriteJob = isTemporaryDatasetWriteJob && temp;
 
         int numKeys = primaryKeys.size();
-        int numFilterFields = DatasetUtils.getFilterField(dataset) == null ? 0 : 1;
+        int numFilterFields = DatasetUtil.getFilterField(dataset) == null ? 0 : 1;
         int numOfAdditionalFields = additionalNonFilterFields == null ? 0 : additionalNonFilterFields.size();
         // Move key fields to front. [keys, record, filters]
         int[] fieldPermutation = new int[numKeys + 1 + numFilterFields + numOfAdditionalFields];
@@ -1156,7 +1083,6 @@
                 fieldPermutation[i++] = idx;
             }
         }
-
         try {
             Index primaryIndex = MetadataManager.INSTANCE.getIndex(mdTxnCtx, dataset.getDataverseName(),
                     dataset.getDatasetName(), dataset.getDatasetName());
@@ -1166,15 +1092,14 @@
             String itemTypeDataverseName = dataset.getItemTypeDataverseName();
             ARecordType itemType = (ARecordType) MetadataManager.INSTANCE
                     .getDatatype(mdTxnCtx, itemTypeDataverseName, itemTypeName).getDatatype();
-            ARecordType metaItemType = DatasetUtils.getMetaType(this, dataset);
-            ITypeTraits[] typeTraits = DatasetUtils.computeTupleTypeTraits(dataset, itemType, metaItemType);
+            ARecordType metaItemType = DatasetUtil.getMetaType(this, dataset);
+            ITypeTraits[] typeTraits = DatasetUtil.computeTupleTypeTraits(dataset, itemType, metaItemType);
             IApplicationContextInfo appContext = (IApplicationContextInfo) context.getAppContext();
-            IBinaryComparatorFactory[] comparatorFactories = DatasetUtils.computeKeysBinaryComparatorFactories(dataset,
+            IBinaryComparatorFactory[] comparatorFactories = DatasetUtil.computeKeysBinaryComparatorFactories(dataset,
                     itemType, metaItemType, context.getBinaryComparatorFactoryProvider());
             Pair<IFileSplitProvider, AlgebricksPartitionConstraint> splitsAndConstraint =
-                    splitProviderAndPartitionConstraintsForDataset(dataSource.getId().getDataverseName(), datasetName,
-                            indexName, temp);
-
+                    getSplitProviderAndConstraints(dataSource.getId().getDataverseName(), datasetName, indexName,
+                            temp);
             // prepare callback
             JobId jobId = ((JobEventListenerFactory) spec.getJobletEventListenerFactory()).getJobId();
             int datasetId = dataset.getDatasetId();
@@ -1183,13 +1108,6 @@
                 primaryKeyFields[i] = i;
             }
 
-            ITypeTraits[] filterTypeTraits = DatasetUtils.computeFilterTypeTraits(dataset, itemType);
-            IBinaryComparatorFactory[] filterCmpFactories = DatasetUtils.computeFilterBinaryComparatorFactories(dataset,
-                    itemType, context.getBinaryComparatorFactoryProvider());
-            int[] filterFields = DatasetUtils.createFilterFields(dataset);
-            int[] btreeFields = DatasetUtils.createBTreeFieldsWhenThereisAFilter(dataset);
-
-            TransactionSubsystemProvider txnSubsystemProvider = new TransactionSubsystemProvider();
             IModificationOperationCallbackFactory modificationCallbackFactory = temp
                     ? new TempDatasetPrimaryIndexModificationOperationCallbackFactory(jobId, datasetId,
                             primaryKeyFields, txnSubsystemProvider, IndexOperation.UPSERT, ResourceType.LSM_BTREE)
@@ -1199,18 +1117,14 @@
             LockThenSearchOperationCallbackFactory searchCallbackFactory = new LockThenSearchOperationCallbackFactory(
                     jobId, datasetId, primaryKeyFields, txnSubsystemProvider, ResourceType.LSM_BTREE);
 
-            Pair<ILSMMergePolicyFactory, Map<String, String>> compactionInfo = DatasetUtils
-                    .getMergePolicyFactory(dataset, mdTxnCtx);
-            IIndexDataflowHelperFactory idfh = new LSMBTreeDataflowHelperFactory(
-                    new AsterixVirtualBufferCacheProvider(datasetId), compactionInfo.first, compactionInfo.second,
-                    new PrimaryIndexOperationTrackerProvider(dataset.getDatasetId()),
-                    RuntimeComponentsProvider.RUNTIME_PROVIDER, LSMBTreeIOOperationCallbackFactory.INSTANCE,
-                    storageProperties.getBloomFilterFalsePositiveRate(), true, filterTypeTraits, filterCmpFactories,
-                    btreeFields, filterFields, !temp);
+            Pair<ILSMMergePolicyFactory, Map<String, String>> compactionInfo =
+                    DatasetUtil.getMergePolicyFactory(dataset, mdTxnCtx);
+            IIndexDataflowHelperFactory idfh = dataset.getIndexDataflowHelperFactory(this, primaryIndex, itemType,
+                    metaItemType, compactionInfo.first, compactionInfo.second);
             LSMTreeUpsertOperatorDescriptor op;
 
-            ITypeTraits[] outputTypeTraits = new ITypeTraits[recordDesc.getFieldCount()
-                    + (dataset.hasMetaPart() ? 2 : 1) + numFilterFields];
+            ITypeTraits[] outputTypeTraits =
+                    new ITypeTraits[recordDesc.getFieldCount() + (dataset.hasMetaPart() ? 2 : 1) + numFilterFields];
             ISerializerDeserializer[] outputSerDes = new ISerializerDeserializer[recordDesc.getFieldCount()
                     + (dataset.hasMetaPart() ? 2 : 1) + numFilterFields];
 
@@ -1220,23 +1134,23 @@
             f++;
             // add the previous meta second
             if (dataset.hasMetaPart()) {
-                outputSerDes[f] = FormatUtils.getDefaultFormat().getSerdeProvider().getSerializerDeserializer(
-                        metaItemType);
+                outputSerDes[f] =
+                        FormatUtils.getDefaultFormat().getSerdeProvider().getSerializerDeserializer(metaItemType);
                 outputTypeTraits[f] = FormatUtils.getDefaultFormat().getTypeTraitProvider().getTypeTrait(metaItemType);
                 f++;
             }
             // add the previous filter third
             int fieldIdx = -1;
             if (numFilterFields > 0) {
-                String filterField = DatasetUtils.getFilterField(dataset).get(0);
+                String filterField = DatasetUtil.getFilterField(dataset).get(0);
                 for (i = 0; i < itemType.getFieldNames().length; i++) {
                     if (itemType.getFieldNames()[i].equals(filterField)) {
                         break;
                     }
                 }
                 fieldIdx = i;
-                outputTypeTraits[f] = FormatUtils.getDefaultFormat().getTypeTraitProvider().getTypeTrait(itemType
-                        .getFieldTypes()[fieldIdx]);
+                outputTypeTraits[f] = FormatUtils.getDefaultFormat().getTypeTraitProvider()
+                        .getTypeTrait(itemType.getFieldTypes()[fieldIdx]);
                 outputSerDes[f] = FormatUtils.getDefaultFormat().getSerdeProvider()
                         .getSerializerDeserializer(itemType.getFieldTypes()[fieldIdx]);
                 f++;
@@ -1247,11 +1161,11 @@
             }
 
             RecordDescriptor outputRecordDesc = new RecordDescriptor(outputSerDes, outputTypeTraits);
-            op = new LSMTreeUpsertOperatorDescriptor(spec, outputRecordDesc,
-                    appContext.getStorageManagerInterface(), appContext.getIndexLifecycleManagerProvider(),
-                    splitsAndConstraint.first, typeTraits, comparatorFactories, bloomFilterKeyFields, fieldPermutation,
-                    idfh, null, true, indexName, context.getMissingWriterFactory(), modificationCallbackFactory,
-                    searchCallbackFactory, null, LSMIndexUtil.getMetadataPageManagerFactory());
+            op = new LSMTreeUpsertOperatorDescriptor(spec, outputRecordDesc, appContext.getStorageManager(),
+                    appContext.getIndexLifecycleManagerProvider(), splitsAndConstraint.first, typeTraits,
+                    comparatorFactories, bloomFilterKeyFields, fieldPermutation, idfh, null, true, indexName,
+                    context.getMissingWriterFactory(), modificationCallbackFactory, searchCallbackFactory, null,
+                    metadataPageManagerFactory);
             op.setType(itemType);
             op.setFilterIndex(fieldIdx);
             return new Pair<>(op, splitsAndConstraint.second);
@@ -1271,8 +1185,8 @@
         ISerializerDeserializer<?> payloadSerde = format.getSerdeProvider().getSerializerDeserializer(itemType);
         RecordDescriptor scannerDesc = new RecordDescriptor(new ISerializerDeserializer[] { payloadSerde });
 
-        ExternalDataScanOperatorDescriptor dataScanner = new ExternalDataScanOperatorDescriptor(jobSpec, scannerDesc,
-                adapterFactory);
+        ExternalScanOperatorDescriptor dataScanner =
+                new ExternalScanOperatorDescriptor(jobSpec, scannerDesc, adapterFactory);
 
         AlgebricksPartitionConstraint constraint;
         try {
@@ -1298,12 +1212,12 @@
 
         int i = 0;
         for (; i < sidxKeyFieldCount; ++i) {
-            Pair<IAType, Boolean> keyPairType = Index.getNonNullableOpenFieldType(sidxKeyFieldTypes.get(i),
-                    sidxKeyFieldNames.get(i),
-                    (hasMeta && secondaryIndexIndicators.get(i).intValue() == 1) ? metaType : recType);
+            Pair<IAType, Boolean> keyPairType =
+                    Index.getNonNullableOpenFieldType(sidxKeyFieldTypes.get(i), sidxKeyFieldNames.get(i),
+                            (hasMeta && secondaryIndexIndicators.get(i).intValue() == 1) ? metaType : recType);
             IAType keyType = keyPairType.first;
-            comparatorFactories[i] = BinaryComparatorFactoryProvider.INSTANCE.getBinaryComparatorFactory(keyType,
-                    true);
+            comparatorFactories[i] =
+                    BinaryComparatorFactoryProvider.INSTANCE.getBinaryComparatorFactory(keyType, true);
             typeTraits[i] = TypeTraitProvider.INSTANCE.getTypeTrait(keyType);
         }
 
@@ -1325,8 +1239,8 @@
             } catch (AsterixException e) {
                 throw new AlgebricksException(e);
             }
-            comparatorFactories[i] = BinaryComparatorFactoryProvider.INSTANCE.getBinaryComparatorFactory(keyType,
-                    true);
+            comparatorFactories[i] =
+                    BinaryComparatorFactoryProvider.INSTANCE.getBinaryComparatorFactory(keyType, true);
             typeTraits[i] = TypeTraitProvider.INSTANCE.getTypeTrait(keyType);
         }
 
@@ -1340,13 +1254,13 @@
             List<LogicalVariable> additionalNonFilteringFields) throws AlgebricksException {
 
         String datasetName = dataSource.getId().getDatasourceName();
-        Dataset dataset = MetadataManagerUtil.findExistingDataset(mdTxnCtx, dataSource.getId().getDataverseName(),
-                datasetName);
+        Dataset dataset =
+                MetadataManagerUtil.findExistingDataset(mdTxnCtx, dataSource.getId().getDataverseName(), datasetName);
         boolean temp = dataset.getDatasetDetails().isTemp();
         isTemporaryDatasetWriteJob = isTemporaryDatasetWriteJob && temp;
 
         int numKeys = keys.size();
-        int numFilterFields = DatasetUtils.getFilterField(dataset) == null ? 0 : 1;
+        int numFilterFields = DatasetUtil.getFilterField(dataset) == null ? 0 : 1;
         // Move key fields to front.
         int[] fieldPermutation = new int[numKeys + 1 + numFilterFields
                 + (additionalNonFilteringFields == null ? 0 : additionalNonFilteringFields.size())];
@@ -1375,60 +1289,51 @@
                     dataset.getDatasetName(), dataset.getDatasetName());
             String indexName = primaryIndex.getIndexName();
             ARecordType itemType = (ARecordType) MetadataManager.INSTANCE
-                    .getDatatype(mdTxnCtx, dataset.getItemTypeDataverseName(), dataset.getItemTypeName()).getDatatype();
-            ARecordType metaItemType = DatasetUtils.getMetaType(this, dataset);
-            ITypeTraits[] typeTraits = DatasetUtils.computeTupleTypeTraits(dataset, itemType, metaItemType);
+                    .getDatatype(mdTxnCtx, dataset.getItemTypeDataverseName(), dataset.getItemTypeName())
+                    .getDatatype();
+            ARecordType metaItemType = DatasetUtil.getMetaType(this, dataset);
+            ITypeTraits[] typeTraits = DatasetUtil.computeTupleTypeTraits(dataset, itemType, metaItemType);
 
             IApplicationContextInfo appContext = (IApplicationContextInfo) context.getAppContext();
-            IBinaryComparatorFactory[] comparatorFactories = DatasetUtils.computeKeysBinaryComparatorFactories(dataset,
+            IBinaryComparatorFactory[] comparatorFactories = DatasetUtil.computeKeysBinaryComparatorFactories(dataset,
                     itemType, metaItemType, context.getBinaryComparatorFactoryProvider());
             Pair<IFileSplitProvider, AlgebricksPartitionConstraint> splitsAndConstraint =
-                    splitProviderAndPartitionConstraintsForDataset(dataSource.getId().getDataverseName(), datasetName,
-                            indexName, temp);
+                    getSplitProviderAndConstraints(dataSource.getId().getDataverseName(), datasetName, indexName,
+                            temp);
 
             // prepare callback
-            JobId jobId = ((JobEventListenerFactory) spec.getJobletEventListenerFactory()).getJobId();
             int datasetId = dataset.getDatasetId();
             int[] primaryKeyFields = new int[numKeys];
             for (i = 0; i < numKeys; i++) {
                 primaryKeyFields[i] = i;
             }
-
-            ITypeTraits[] filterTypeTraits = DatasetUtils.computeFilterTypeTraits(dataset, itemType);
-            IBinaryComparatorFactory[] filterCmpFactories = DatasetUtils.computeFilterBinaryComparatorFactories(dataset,
-                    itemType, context.getBinaryComparatorFactoryProvider());
-            int[] filterFields = DatasetUtils.createFilterFields(dataset);
-            int[] btreeFields = DatasetUtils.createBTreeFieldsWhenThereisAFilter(dataset);
-
-            TransactionSubsystemProvider txnSubsystemProvider = new TransactionSubsystemProvider();
             IModificationOperationCallbackFactory modificationCallbackFactory = temp
-                    ? new TempDatasetPrimaryIndexModificationOperationCallbackFactory(jobId, datasetId,
+                    ? new TempDatasetPrimaryIndexModificationOperationCallbackFactory(
+                            ((JobEventListenerFactory) spec.getJobletEventListenerFactory()).getJobId(), datasetId,
                             primaryKeyFields, txnSubsystemProvider, indexOp, ResourceType.LSM_BTREE)
-                    : new PrimaryIndexModificationOperationCallbackFactory(jobId, datasetId, primaryKeyFields,
-                            txnSubsystemProvider, indexOp, ResourceType.LSM_BTREE, dataset.hasMetaPart());
+                    : new PrimaryIndexModificationOperationCallbackFactory(
+                            ((JobEventListenerFactory) spec.getJobletEventListenerFactory()).getJobId(), datasetId,
+                            primaryKeyFields, txnSubsystemProvider, indexOp, ResourceType.LSM_BTREE,
+                            dataset.hasMetaPart());
 
-            Pair<ILSMMergePolicyFactory, Map<String, String>> compactionInfo = DatasetUtils
-                    .getMergePolicyFactory(dataset, mdTxnCtx);
-            IIndexDataflowHelperFactory idfh = new LSMBTreeDataflowHelperFactory(
-                    new AsterixVirtualBufferCacheProvider(datasetId), compactionInfo.first, compactionInfo.second,
-                    new PrimaryIndexOperationTrackerProvider(dataset.getDatasetId()),
-                    RuntimeComponentsProvider.RUNTIME_PROVIDER, LSMBTreeIOOperationCallbackFactory.INSTANCE,
-                    storageProperties.getBloomFilterFalsePositiveRate(), true, filterTypeTraits, filterCmpFactories,
-                    btreeFields, filterFields, !temp);
+            Pair<ILSMMergePolicyFactory, Map<String, String>> compactionInfo =
+                    DatasetUtil.getMergePolicyFactory(dataset, mdTxnCtx);
+            IIndexDataflowHelperFactory idfh = dataset.getIndexDataflowHelperFactory(this, primaryIndex, itemType,
+                    metaItemType, compactionInfo.first, compactionInfo.second);
             IOperatorDescriptor op;
             if (bulkload) {
                 long numElementsHint = getCardinalityPerPartitionHint(dataset);
-                op = new TreeIndexBulkLoadOperatorDescriptor(spec, recordDesc, appContext.getStorageManagerInterface(),
+                op = new TreeIndexBulkLoadOperatorDescriptor(spec, recordDesc, appContext.getStorageManager(),
                         appContext.getIndexLifecycleManagerProvider(), splitsAndConstraint.first, typeTraits,
                         comparatorFactories, bloomFilterKeyFields, fieldPermutation,
-                        GlobalConfig.DEFAULT_TREE_FILL_FACTOR, true, numElementsHint, true, idfh, LSMIndexUtil
-                                .getMetadataPageManagerFactory());
+                        GlobalConfig.DEFAULT_TREE_FILL_FACTOR, true, numElementsHint, true, idfh,
+                        metadataPageManagerFactory);
             } else {
-                op = new LSMTreeInsertDeleteOperatorDescriptor(spec, recordDesc,
-                        appContext.getStorageManagerInterface(), appContext.getIndexLifecycleManagerProvider(),
-                        splitsAndConstraint.first, typeTraits, comparatorFactories, bloomFilterKeyFields,
-                        fieldPermutation, indexOp, idfh, null, true, indexName, null, modificationCallbackFactory,
-                        NoOpOperationCallbackFactory.INSTANCE, LSMIndexUtil.getMetadataPageManagerFactory());
+                op = new LSMTreeInsertDeleteOperatorDescriptor(spec, recordDesc, appContext.getStorageManager(),
+                        appContext.getIndexLifecycleManagerProvider(), splitsAndConstraint.first, typeTraits,
+                        comparatorFactories, bloomFilterKeyFields, fieldPermutation, indexOp, idfh, null, true,
+                        indexName, null, modificationCallbackFactory, NoOpOperationCallbackFactory.INSTANCE,
+                        metadataPageManagerFactory);
             }
             return new Pair<>(op, splitsAndConstraint.second);
         } catch (MetadataException me) {
@@ -1496,7 +1401,7 @@
         isTemporaryDatasetWriteJob = isTemporaryDatasetWriteJob && temp;
 
         int numKeys = primaryKeys.size() + secondaryKeys.size();
-        int numFilterFields = DatasetUtils.getFilterField(dataset) == null ? 0 : 1;
+        int numFilterFields = DatasetUtil.getFilterField(dataset) == null ? 0 : 1;
 
         // generate field permutations
         int[] fieldPermutation = new int[numKeys + numFilterFields];
@@ -1546,22 +1451,23 @@
         }
 
         String itemTypeName = dataset.getItemTypeName();
-        IAType itemType;
+        ARecordType itemType;
         try {
-            itemType = MetadataManager.INSTANCE.getDatatype(mdTxnCtx, dataset.getItemTypeDataverseName(), itemTypeName)
-                    .getDatatype();
+            itemType = (ARecordType) MetadataManager.INSTANCE
+                    .getDatatype(mdTxnCtx, dataset.getItemTypeDataverseName(), itemTypeName).getDatatype();
             validateRecordType(itemType);
-            ARecordType recType = (ARecordType) itemType;
+            ARecordType metaType = dataset.hasMetaPart()
+                    ? (ARecordType) MetadataManager.INSTANCE.getDatatype(mdTxnCtx,
+                            dataset.getMetaItemTypeDataverseName(), dataset.getMetaItemTypeName()).getDatatype()
+                    : null;
 
             // Index parameters.
             Index secondaryIndex = MetadataManager.INSTANCE.getIndex(mdTxnCtx, dataset.getDataverseName(),
                     dataset.getDatasetName(), indexName);
 
-            ITypeTraits[] filterTypeTraits = DatasetUtils.computeFilterTypeTraits(dataset, recType);
-            IBinaryComparatorFactory[] filterCmpFactories = DatasetUtils.computeFilterBinaryComparatorFactories(dataset,
-                    recType, context.getBinaryComparatorFactoryProvider());
-            int[] filterFields = null;
-            int[] btreeFields = null;
+            ITypeTraits[] filterTypeTraits = DatasetUtil.computeFilterTypeTraits(dataset, itemType);
+            int[] filterFields;
+            int[] btreeFields;
             if (filterTypeTraits != null) {
                 filterFields = new int[1];
                 filterFields[0] = numKeys;
@@ -1577,73 +1483,60 @@
             IBinaryComparatorFactory[] comparatorFactories = new IBinaryComparatorFactory[numKeys];
             for (i = 0; i < secondaryKeys.size(); ++i) {
                 Pair<IAType, Boolean> keyPairType = Index.getNonNullableOpenFieldType(secondaryKeyTypes.get(i),
-                        secondaryKeyNames.get(i), recType);
+                        secondaryKeyNames.get(i), itemType);
                 IAType keyType = keyPairType.first;
-                comparatorFactories[i] = BinaryComparatorFactoryProvider.INSTANCE.getBinaryComparatorFactory(keyType,
-                        true);
+                comparatorFactories[i] =
+                        BinaryComparatorFactoryProvider.INSTANCE.getBinaryComparatorFactory(keyType, true);
                 typeTraits[i] = TypeTraitProvider.INSTANCE.getTypeTrait(keyType);
             }
-            List<List<String>> partitioningKeys = DatasetUtils.getPartitioningKeys(dataset);
+            List<List<String>> partitioningKeys = DatasetUtil.getPartitioningKeys(dataset);
             for (List<String> partitioningKey : partitioningKeys) {
-                IAType keyType = recType.getSubFieldType(partitioningKey);
-                comparatorFactories[i] = BinaryComparatorFactoryProvider.INSTANCE.getBinaryComparatorFactory(keyType,
-                        true);
+                IAType keyType = itemType.getSubFieldType(partitioningKey);
+                comparatorFactories[i] =
+                        BinaryComparatorFactoryProvider.INSTANCE.getBinaryComparatorFactory(keyType, true);
                 typeTraits[i] = TypeTraitProvider.INSTANCE.getTypeTrait(keyType);
                 ++i;
             }
 
             IApplicationContextInfo appContext = (IApplicationContextInfo) context.getAppContext();
             Pair<IFileSplitProvider, AlgebricksPartitionConstraint> splitsAndConstraint =
-                    splitProviderAndPartitionConstraintsForDataset(dataverseName, datasetName, indexName, temp);
+                    getSplitProviderAndConstraints(dataverseName, datasetName, indexName, temp);
 
             // prepare callback
             JobId jobId = ((JobEventListenerFactory) spec.getJobletEventListenerFactory()).getJobId();
             int datasetId = dataset.getDatasetId();
-            TransactionSubsystemProvider txnSubsystemProvider = new TransactionSubsystemProvider();
             IModificationOperationCallbackFactory modificationCallbackFactory = temp
                     ? new TempDatasetSecondaryIndexModificationOperationCallbackFactory(jobId, datasetId,
-                            modificationCallbackPrimaryKeyFields, txnSubsystemProvider, indexOp, ResourceType.LSM_BTREE)
+                            modificationCallbackPrimaryKeyFields, txnSubsystemProvider, indexOp,
+                            ResourceType.LSM_BTREE)
                     : new SecondaryIndexModificationOperationCallbackFactory(jobId, datasetId,
-                            modificationCallbackPrimaryKeyFields, txnSubsystemProvider, indexOp, ResourceType.LSM_BTREE,
-                            dataset.hasMetaPart());
+                            modificationCallbackPrimaryKeyFields, txnSubsystemProvider, indexOp,
+                            ResourceType.LSM_BTREE, dataset.hasMetaPart());
 
-            Pair<ILSMMergePolicyFactory, Map<String, String>> compactionInfo = DatasetUtils
-                    .getMergePolicyFactory(dataset, mdTxnCtx);
-            IIndexDataflowHelperFactory idfh = new LSMBTreeDataflowHelperFactory(
-                    new AsterixVirtualBufferCacheProvider(datasetId), compactionInfo.first, compactionInfo.second,
-                    new SecondaryIndexOperationTrackerProvider(dataset.getDatasetId()),
-                    RuntimeComponentsProvider.RUNTIME_PROVIDER, LSMBTreeIOOperationCallbackFactory.INSTANCE,
-                    storageProperties.getBloomFilterFalsePositiveRate(), false, filterTypeTraits, filterCmpFactories,
-                    btreeFields, filterFields, !temp);
+            Pair<ILSMMergePolicyFactory, Map<String, String>> compactionInfo =
+                    DatasetUtil.getMergePolicyFactory(dataset, mdTxnCtx);
+            IIndexDataflowHelperFactory idfh = dataset.getIndexDataflowHelperFactory(this, secondaryIndex, itemType,
+                    metaType, compactionInfo.first, compactionInfo.second);
             IOperatorDescriptor op;
             if (bulkload) {
                 long numElementsHint = getCardinalityPerPartitionHint(dataset);
-                op = new TreeIndexBulkLoadOperatorDescriptor(spec, recordDesc, appContext.getStorageManagerInterface(),
+                op = new TreeIndexBulkLoadOperatorDescriptor(spec, recordDesc, appContext.getStorageManager(),
                         appContext.getIndexLifecycleManagerProvider(), splitsAndConstraint.first, typeTraits,
                         comparatorFactories, bloomFilterKeyFields, fieldPermutation,
-                        GlobalConfig.DEFAULT_TREE_FILL_FACTOR, false, numElementsHint, false, idfh, LSMIndexUtil
-                                .getMetadataPageManagerFactory());
+                        GlobalConfig.DEFAULT_TREE_FILL_FACTOR, false, numElementsHint, false, idfh,
+                        metadataPageManagerFactory);
             } else if (indexOp == IndexOperation.UPSERT) {
-                op = new LSMTreeUpsertOperatorDescriptor(spec, recordDesc,
-                        appContext.getStorageManagerInterface(), appContext.getIndexLifecycleManagerProvider(),
-                        splitsAndConstraint.first, typeTraits, comparatorFactories, bloomFilterKeyFields,
-                        fieldPermutation, idfh, filterFactory, false, indexName, null, modificationCallbackFactory,
-                        NoOpOperationCallbackFactory.INSTANCE, prevFieldPermutation, LSMIndexUtil
-                                .getMetadataPageManagerFactory());
+                op = new LSMTreeUpsertOperatorDescriptor(spec, recordDesc, appContext.getStorageManager(),
+                        appContext.getIndexLifecycleManagerProvider(), splitsAndConstraint.first, typeTraits,
+                        comparatorFactories, bloomFilterKeyFields, fieldPermutation, idfh, filterFactory, false,
+                        indexName, null, modificationCallbackFactory, NoOpOperationCallbackFactory.INSTANCE,
+                        prevFieldPermutation, metadataPageManagerFactory);
             } else {
-                op = new LSMTreeInsertDeleteOperatorDescriptor(spec, recordDesc,
-                        appContext.getStorageManagerInterface(), appContext.getIndexLifecycleManagerProvider(),
-                        splitsAndConstraint.first, typeTraits, comparatorFactories, bloomFilterKeyFields,
-                        fieldPermutation, indexOp,
-                        new LSMBTreeDataflowHelperFactory(new AsterixVirtualBufferCacheProvider(datasetId),
-                                compactionInfo.first, compactionInfo.second,
-                                new SecondaryIndexOperationTrackerProvider(dataset.getDatasetId()),
-                                RuntimeComponentsProvider.RUNTIME_PROVIDER,
-                                LSMBTreeIOOperationCallbackFactory.INSTANCE,
-                                storageProperties.getBloomFilterFalsePositiveRate(), false, filterTypeTraits,
-                                filterCmpFactories, btreeFields, filterFields, !temp),
-                        filterFactory, false, indexName, null, modificationCallbackFactory,
-                        NoOpOperationCallbackFactory.INSTANCE, LSMIndexUtil.getMetadataPageManagerFactory());
+                op = new LSMTreeInsertDeleteOperatorDescriptor(spec, recordDesc, appContext.getStorageManager(),
+                        appContext.getIndexLifecycleManagerProvider(), splitsAndConstraint.first, typeTraits,
+                        comparatorFactories, bloomFilterKeyFields, fieldPermutation, indexOp, idfh, filterFactory,
+                        false, indexName, null, modificationCallbackFactory, NoOpOperationCallbackFactory.INSTANCE,
+                        metadataPageManagerFactory);
             }
             return new Pair<>(op, splitsAndConstraint.second);
         } catch (Exception e) {
@@ -1672,11 +1565,9 @@
                     dataset.getDatasetName(), indexName);
             List<List<String>> secondaryKeyExprs = secondaryIndex.getKeyFieldNames();
             List<IAType> secondaryKeyTypes = secondaryIndex.getKeyFieldTypes();
-            Pair<IAType, Boolean> keyPairType = Index.getNonNullableOpenFieldType(secondaryKeyTypes.get(0),
-                    secondaryKeyExprs.get(0), recType);
+            Pair<IAType, Boolean> keyPairType =
+                    Index.getNonNullableOpenFieldType(secondaryKeyTypes.get(0), secondaryKeyExprs.get(0), recType);
             IAType spatialType = keyPairType.first;
-            boolean isPointMBR = spatialType.getTypeTag() == ATypeTag.POINT
-                    || spatialType.getTypeTag() == ATypeTag.POINT3D;
             int dimension = NonTaggedFormatUtil.getNumDimensions(spatialType.getTypeTag());
             int numSecondaryKeys = dimension * 2;
             int numPrimaryKeys = primaryKeys.size();
@@ -1684,7 +1575,7 @@
             ITypeTraits[] typeTraits = new ITypeTraits[numKeys];
             IBinaryComparatorFactory[] comparatorFactories = new IBinaryComparatorFactory[numSecondaryKeys];
 
-            int numFilterFields = DatasetUtils.getFilterField(dataset) == null ? 0 : 1;
+            int numFilterFields = DatasetUtil.getFilterField(dataset) == null ? 0 : 1;
             int[] fieldPermutation = new int[numKeys + numFilterFields];
             int[] modificationCallbackPrimaryKeyFields = new int[primaryKeys.size()];
             int i = 0;
@@ -1735,33 +1626,31 @@
             IPrimitiveValueProviderFactory[] valueProviderFactories =
                     new IPrimitiveValueProviderFactory[numSecondaryKeys];
             for (i = 0; i < numSecondaryKeys; i++) {
-                comparatorFactories[i] = BinaryComparatorFactoryProvider.INSTANCE
-                        .getBinaryComparatorFactory(nestedKeyType, true);
+                comparatorFactories[i] =
+                        BinaryComparatorFactoryProvider.INSTANCE.getBinaryComparatorFactory(nestedKeyType, true);
                 typeTraits[i] = TypeTraitProvider.INSTANCE.getTypeTrait(nestedKeyType);
-                valueProviderFactories[i] = PrimitiveValueProviderFactory.INSTANCE;
+                valueProviderFactories[i] = primitiveValueProviderFactory;
             }
-            List<List<String>> partitioningKeys = DatasetUtils.getPartitioningKeys(dataset);
+            List<List<String>> partitioningKeys = DatasetUtil.getPartitioningKeys(dataset);
             for (List<String> partitioningKey : partitioningKeys) {
                 IAType keyType = recType.getSubFieldType(partitioningKey);
                 typeTraits[i] = TypeTraitProvider.INSTANCE.getTypeTrait(keyType);
                 ++i;
             }
-            ARecordType metaItemType = DatasetUtils.getMetaType(this, dataset);
-            IBinaryComparatorFactory[] primaryComparatorFactories = DatasetUtils.computeKeysBinaryComparatorFactories(
+            ARecordType metaItemType = DatasetUtil.getMetaType(this, dataset);
+            IBinaryComparatorFactory[] primaryComparatorFactories = DatasetUtil.computeKeysBinaryComparatorFactories(
                     dataset, recType, metaItemType, context.getBinaryComparatorFactoryProvider());
             IApplicationContextInfo appContext = (IApplicationContextInfo) context.getAppContext();
             Pair<IFileSplitProvider, AlgebricksPartitionConstraint> splitsAndConstraint =
-                    splitProviderAndPartitionConstraintsForDataset(dataverseName, datasetName, indexName, temp);
+                    getSplitProviderAndConstraints(dataverseName, datasetName, indexName, temp);
             int[] btreeFields = new int[primaryComparatorFactories.length];
             for (int k = 0; k < btreeFields.length; k++) {
                 btreeFields[k] = k + numSecondaryKeys;
             }
 
-            ITypeTraits[] filterTypeTraits = DatasetUtils.computeFilterTypeTraits(dataset, recType);
-            IBinaryComparatorFactory[] filterCmpFactories = DatasetUtils.computeFilterBinaryComparatorFactories(dataset,
-                    recType, context.getBinaryComparatorFactoryProvider());
-            int[] filterFields = null;
-            int[] rtreeFields = null;
+            ITypeTraits[] filterTypeTraits = DatasetUtil.computeFilterTypeTraits(dataset, recType);
+            int[] filterFields;
+            int[] rtreeFields;
             if (filterTypeTraits != null) {
                 filterFields = new int[1];
                 filterFields[0] = numSecondaryKeys + numPrimaryKeys;
@@ -1774,47 +1663,38 @@
             // prepare callback
             JobId jobId = ((JobEventListenerFactory) spec.getJobletEventListenerFactory()).getJobId();
             int datasetId = dataset.getDatasetId();
-            TransactionSubsystemProvider txnSubsystemProvider = new TransactionSubsystemProvider();
             IModificationOperationCallbackFactory modificationCallbackFactory = temp
                     ? new TempDatasetSecondaryIndexModificationOperationCallbackFactory(jobId, datasetId,
-                            modificationCallbackPrimaryKeyFields, txnSubsystemProvider, indexOp, ResourceType.LSM_RTREE)
+                            modificationCallbackPrimaryKeyFields, txnSubsystemProvider, indexOp,
+                            ResourceType.LSM_RTREE)
                     : new SecondaryIndexModificationOperationCallbackFactory(jobId, datasetId,
-                            modificationCallbackPrimaryKeyFields, txnSubsystemProvider, indexOp, ResourceType.LSM_RTREE,
-                            dataset.hasMetaPart());
+                            modificationCallbackPrimaryKeyFields, txnSubsystemProvider, indexOp,
+                            ResourceType.LSM_RTREE, dataset.hasMetaPart());
 
-            Pair<ILSMMergePolicyFactory, Map<String, String>> compactionInfo = DatasetUtils
-                    .getMergePolicyFactory(dataset, mdTxnCtx);
-
-            IBinaryComparatorFactory[] deletedKeyBTreeCompFactories = getMergedComparatorFactories(comparatorFactories,
-                    primaryComparatorFactories);
-            IIndexDataflowHelperFactory idff = new LSMRTreeWithAntiMatterTuplesDataflowHelperFactory(
-                    valueProviderFactories, RTreePolicyType.RTREE, deletedKeyBTreeCompFactories,
-                    new AsterixVirtualBufferCacheProvider(dataset.getDatasetId()), compactionInfo.first,
-                    compactionInfo.second, new SecondaryIndexOperationTrackerProvider(dataset.getDatasetId()),
-                    RuntimeComponentsProvider.RUNTIME_PROVIDER, LSMRTreeIOOperationCallbackFactory.INSTANCE,
-                    MetadataProvider.proposeLinearizer(nestedKeyType.getTypeTag(), comparatorFactories.length),
-                    rtreeFields, filterTypeTraits, filterCmpFactories, filterFields, !temp, isPointMBR);
+            Pair<ILSMMergePolicyFactory, Map<String, String>> compactionInfo =
+                    DatasetUtil.getMergePolicyFactory(dataset, mdTxnCtx);
+            IIndexDataflowHelperFactory indexDataflowHelperFactory = dataset.getIndexDataflowHelperFactory(this,
+                    secondaryIndex, recType, metaItemType, compactionInfo.first, compactionInfo.second);
             IOperatorDescriptor op;
             if (bulkload) {
                 long numElementsHint = getCardinalityPerPartitionHint(dataset);
-                op = new TreeIndexBulkLoadOperatorDescriptor(spec, recordDesc, appContext.getStorageManagerInterface(),
+                op = new TreeIndexBulkLoadOperatorDescriptor(spec, recordDesc, appContext.getStorageManager(),
                         appContext.getIndexLifecycleManagerProvider(), splitsAndConstraint.first, typeTraits,
                         primaryComparatorFactories, btreeFields, fieldPermutation,
-                        GlobalConfig.DEFAULT_TREE_FILL_FACTOR, false, numElementsHint, false, idff, LSMIndexUtil
-                                .getMetadataPageManagerFactory());
+                        GlobalConfig.DEFAULT_TREE_FILL_FACTOR, false, numElementsHint, false,
+                        indexDataflowHelperFactory, metadataPageManagerFactory);
             } else if (indexOp == IndexOperation.UPSERT) {
-                op = new LSMTreeUpsertOperatorDescriptor(spec, recordDesc,
-                        appContext.getStorageManagerInterface(), appContext.getIndexLifecycleManagerProvider(),
-                        splitsAndConstraint.first, typeTraits, comparatorFactories, null, fieldPermutation, idff,
-                        filterFactory, false, indexName, null, modificationCallbackFactory,
-                        NoOpOperationCallbackFactory.INSTANCE, prevFieldPermutation, LSMIndexUtil
-                                .getMetadataPageManagerFactory());
+                op = new LSMTreeUpsertOperatorDescriptor(spec, recordDesc, appContext.getStorageManager(),
+                        appContext.getIndexLifecycleManagerProvider(), splitsAndConstraint.first, typeTraits,
+                        comparatorFactories, null, fieldPermutation, indexDataflowHelperFactory, filterFactory, false,
+                        indexName, null, modificationCallbackFactory, NoOpOperationCallbackFactory.INSTANCE,
+                        prevFieldPermutation, metadataPageManagerFactory);
             } else {
-                op = new LSMTreeInsertDeleteOperatorDescriptor(spec, recordDesc,
-                        appContext.getStorageManagerInterface(), appContext.getIndexLifecycleManagerProvider(),
-                        splitsAndConstraint.first, typeTraits, comparatorFactories, null, fieldPermutation, indexOp,
-                        idff, filterFactory, false, indexName, null, modificationCallbackFactory,
-                        NoOpOperationCallbackFactory.INSTANCE, LSMIndexUtil.getMetadataPageManagerFactory());
+                op = new LSMTreeInsertDeleteOperatorDescriptor(spec, recordDesc, appContext.getStorageManager(),
+                        appContext.getIndexLifecycleManagerProvider(), splitsAndConstraint.first, typeTraits,
+                        comparatorFactories, null, fieldPermutation, indexOp, indexDataflowHelperFactory,
+                        filterFactory, false, indexName, null, modificationCallbackFactory,
+                        NoOpOperationCallbackFactory.INSTANCE, metadataPageManagerFactory);
             }
             return new Pair<>(op, splitsAndConstraint.second);
         } catch (MetadataException e) {
@@ -1844,12 +1724,9 @@
         }
         // The size of secondaryKeys can be two if it receives input from its
         // TokenizeOperator- [token, number of token]
-        if (secondaryKeys.size() > 1 && !isPartitioned) {
-            throw new AlgebricksException("Cannot create composite inverted index on multiple fields.");
-        } else if (secondaryKeys.size() > 2 && isPartitioned) {
+        if ((secondaryKeys.size() > 1 && !isPartitioned) || (secondaryKeys.size() > 2 && isPartitioned)) {
             throw new AlgebricksException("Cannot create composite inverted index on multiple fields.");
         }
-
         Dataset dataset = MetadataManagerUtil.findExistingDataset(mdTxnCtx, dataverseName, datasetName);
         boolean temp = dataset.getDatasetDetails().isTemp();
         isTemporaryDatasetWriteJob = isTemporaryDatasetWriteJob && temp;
@@ -1858,7 +1735,7 @@
         // One token (+ optional partitioning field) + primary keys: [token,
         // number of token, PK]
         int numKeys = primaryKeys.size() + secondaryKeys.size();
-        int numFilterFields = DatasetUtils.getFilterField(dataset) == null ? 0 : 1;
+        int numFilterFields = DatasetUtil.getFilterField(dataset) == null ? 0 : 1;
 
         // generate field permutations
         int[] fieldPermutation = new int[numKeys + numFilterFields];
@@ -1935,19 +1812,19 @@
                 numTokenFields = secondaryKeys.size() + 1;
             }
 
-            ARecordType metaItemType = DatasetUtils.getMetaType(this, dataset);
+            ARecordType metaItemType = DatasetUtil.getMetaType(this, dataset);
             ITypeTraits[] tokenTypeTraits = new ITypeTraits[numTokenFields];
             ITypeTraits[] invListsTypeTraits = new ITypeTraits[primaryKeys.size()];
             IBinaryComparatorFactory[] tokenComparatorFactories = new IBinaryComparatorFactory[numTokenFields];
-            IBinaryComparatorFactory[] invListComparatorFactories = DatasetUtils.computeKeysBinaryComparatorFactories(
+            IBinaryComparatorFactory[] invListComparatorFactories = DatasetUtil.computeKeysBinaryComparatorFactories(
                     dataset, recType, metaItemType, context.getBinaryComparatorFactoryProvider());
 
             IAType secondaryKeyType;
-            Pair<IAType, Boolean> keyPairType = Index.getNonNullableOpenFieldType(secondaryKeyTypes.get(0),
-                    secondaryKeyExprs.get(0), recType);
+            Pair<IAType, Boolean> keyPairType =
+                    Index.getNonNullableOpenFieldType(secondaryKeyTypes.get(0), secondaryKeyExprs.get(0), recType);
             secondaryKeyType = keyPairType.first;
 
-            List<List<String>> partitioningKeys = DatasetUtils.getPartitioningKeys(dataset);
+            List<List<String>> partitioningKeys = DatasetUtil.getPartitioningKeys(dataset);
 
             i = 0;
             for (List<String> partitioningKey : partitioningKeys) {
@@ -1967,14 +1844,12 @@
             IBinaryTokenizerFactory tokenizerFactory = NonTaggedFormatUtil.getBinaryTokenizerFactory(
                     secondaryKeyType.getTypeTag(), indexType, secondaryIndex.getGramLength());
 
-            ITypeTraits[] filterTypeTraits = DatasetUtils.computeFilterTypeTraits(dataset, recType);
-            IBinaryComparatorFactory[] filterCmpFactories = DatasetUtils.computeFilterBinaryComparatorFactories(dataset,
-                    recType, context.getBinaryComparatorFactoryProvider());
+            ITypeTraits[] filterTypeTraits = DatasetUtil.computeFilterTypeTraits(dataset, recType);
 
-            int[] filterFields = null;
-            int[] invertedIndexFields = null;
-            int[] filterFieldsForNonBulkLoadOps = null;
-            int[] invertedIndexFieldsForNonBulkLoadOps = null;
+            int[] filterFields;
+            int[] invertedIndexFields;
+            int[] filterFieldsForNonBulkLoadOps;
+            int[] invertedIndexFieldsForNonBulkLoadOps;
             if (filterTypeTraits != null) {
                 filterFields = new int[1];
                 filterFields[0] = numTokenFields + primaryKeys.size();
@@ -1982,7 +1857,6 @@
                 for (int k = 0; k < invertedIndexFields.length; k++) {
                     invertedIndexFields[k] = k;
                 }
-
                 filterFieldsForNonBulkLoadOps = new int[numFilterFields];
                 //for non-bulk-loads, there is only <SK,PK,F> in the incoming tuples
                 filterFieldsForNonBulkLoadOps[0] = numKeys;
@@ -1994,12 +1868,11 @@
 
             IApplicationContextInfo appContext = (IApplicationContextInfo) context.getAppContext();
             Pair<IFileSplitProvider, AlgebricksPartitionConstraint> splitsAndConstraint =
-                    splitProviderAndPartitionConstraintsForDataset(dataverseName, datasetName, indexName, temp);
+                    getSplitProviderAndConstraints(dataverseName, datasetName, indexName, temp);
 
             // prepare callback
             JobId jobId = ((JobEventListenerFactory) spec.getJobletEventListenerFactory()).getJobId();
             int datasetId = dataset.getDatasetId();
-            TransactionSubsystemProvider txnSubsystemProvider = new TransactionSubsystemProvider();
             IModificationOperationCallbackFactory modificationCallbackFactory = temp
                     ? new TempDatasetSecondaryIndexModificationOperationCallbackFactory(jobId, datasetId,
                             modificationCallbackPrimaryKeyFields, txnSubsystemProvider, indexOp,
@@ -2007,51 +1880,31 @@
                     : new SecondaryIndexModificationOperationCallbackFactory(jobId, datasetId,
                             modificationCallbackPrimaryKeyFields, txnSubsystemProvider, indexOp,
                             ResourceType.LSM_INVERTED_INDEX, dataset.hasMetaPart());
-
-            Pair<ILSMMergePolicyFactory, Map<String, String>> compactionInfo = DatasetUtils
-                    .getMergePolicyFactory(dataset, mdTxnCtx);
-            IIndexDataflowHelperFactory indexDataFlowFactory;
-            if (!isPartitioned) {
-                indexDataFlowFactory = new LSMInvertedIndexDataflowHelperFactory(
-                        new AsterixVirtualBufferCacheProvider(datasetId), compactionInfo.first, compactionInfo.second,
-                        new SecondaryIndexOperationTrackerProvider(dataset.getDatasetId()),
-                        RuntimeComponentsProvider.RUNTIME_PROVIDER,
-                        LSMInvertedIndexIOOperationCallbackFactory.INSTANCE,
-                        storageProperties.getBloomFilterFalsePositiveRate(), invertedIndexFields, filterTypeTraits,
-                        filterCmpFactories, filterFields, filterFieldsForNonBulkLoadOps,
-                        invertedIndexFieldsForNonBulkLoadOps, !temp);
-            } else {
-                indexDataFlowFactory = new PartitionedLSMInvertedIndexDataflowHelperFactory(
-                        new AsterixVirtualBufferCacheProvider(dataset.getDatasetId()), compactionInfo.first,
-                        compactionInfo.second, new SecondaryIndexOperationTrackerProvider(dataset.getDatasetId()),
-                        RuntimeComponentsProvider.RUNTIME_PROVIDER,
-                        LSMInvertedIndexIOOperationCallbackFactory.INSTANCE,
-                        storageProperties.getBloomFilterFalsePositiveRate(), invertedIndexFields, filterTypeTraits,
-                        filterCmpFactories, filterFields, filterFieldsForNonBulkLoadOps,
-                        invertedIndexFieldsForNonBulkLoadOps, !temp);
-            }
+            Pair<ILSMMergePolicyFactory, Map<String, String>> compactionInfo =
+                    DatasetUtil.getMergePolicyFactory(dataset, mdTxnCtx);
+            IIndexDataflowHelperFactory indexDataFlowFactory = dataset.getIndexDataflowHelperFactory(this,
+                    secondaryIndex, recType, metaItemType, compactionInfo.first, compactionInfo.second);
             IOperatorDescriptor op;
             if (bulkload) {
                 long numElementsHint = getCardinalityPerPartitionHint(dataset);
                 op = new LSMInvertedIndexBulkLoadOperatorDescriptor(spec, recordDesc, fieldPermutation, false,
-                        numElementsHint, false, appContext.getStorageManagerInterface(), splitsAndConstraint.first,
+                        numElementsHint, false, appContext.getStorageManager(), splitsAndConstraint.first,
                         appContext.getIndexLifecycleManagerProvider(), tokenTypeTraits, tokenComparatorFactories,
                         invListsTypeTraits, invListComparatorFactories, tokenizerFactory, indexDataFlowFactory,
-                        LSMIndexUtil.getMetadataPageManagerFactory());
+                        metadataPageManagerFactory);
             } else if (indexOp == IndexOperation.UPSERT) {
-                op = new LSMInvertedIndexUpsertOperatorDescriptor(spec, recordDesc,
-                        appContext.getStorageManagerInterface(), splitsAndConstraint.first,
-                        appContext.getIndexLifecycleManagerProvider(), tokenTypeTraits, tokenComparatorFactories,
-                        invListsTypeTraits, invListComparatorFactories, tokenizerFactory, fieldPermutation,
-                        indexDataFlowFactory, filterFactory, modificationCallbackFactory, indexName,
-                        prevFieldPermutation, LSMIndexUtil.getMetadataPageManagerFactory());
+                op = new LSMInvertedIndexUpsertOperatorDescriptor(spec, recordDesc, appContext.getStorageManager(),
+                        splitsAndConstraint.first, appContext.getIndexLifecycleManagerProvider(), tokenTypeTraits,
+                        tokenComparatorFactories, invListsTypeTraits, invListComparatorFactories, tokenizerFactory,
+                        fieldPermutation, indexDataFlowFactory, filterFactory, modificationCallbackFactory, indexName,
+                        prevFieldPermutation, metadataPageManagerFactory);
             } else {
                 op = new LSMInvertedIndexInsertDeleteOperatorDescriptor(spec, recordDesc,
-                        appContext.getStorageManagerInterface(), splitsAndConstraint.first,
+                        appContext.getStorageManager(), splitsAndConstraint.first,
                         appContext.getIndexLifecycleManagerProvider(), tokenTypeTraits, tokenComparatorFactories,
                         invListsTypeTraits, invListComparatorFactories, tokenizerFactory, fieldPermutation, indexOp,
-                        indexDataFlowFactory, filterFactory, modificationCallbackFactory, indexName, LSMIndexUtil
-                                .getMetadataPageManagerFactory());
+                        indexDataFlowFactory, filterFactory, modificationCallbackFactory, indexName,
+                        metadataPageManagerFactory);
             }
             return new Pair<>(op, splitsAndConstraint.second);
         } catch (Exception e) {
@@ -2177,7 +2030,7 @@
             Pair<IAType, Boolean> keyPairType = Index.getNonNullableOpenFieldType(secondaryKeyTypeEntries.get(0),
                     secondaryKeyExprs.get(0), recType);
             secondaryKeyType = keyPairType.first;
-            List<List<String>> partitioningKeys = DatasetUtils.getPartitioningKeys(dataset);
+            List<List<String>> partitioningKeys = DatasetUtil.getPartitioningKeys(dataset);
             i = 0;
             for (List<String> partitioningKey : partitioningKeys) {
                 IAType keyType = recType.getSubFieldType(partitioningKey);
@@ -2196,7 +2049,7 @@
                     secondaryKeyType.getTypeTag(), indexType, secondaryIndex.getGramLength());
 
             Pair<IFileSplitProvider, AlgebricksPartitionConstraint> splitsAndConstraint =
-                    splitProviderAndPartitionConstraintsForDataset(dataverseName, datasetName, indexName,
+                    getSplitProviderAndConstraints(dataverseName, datasetName, indexName,
                             dataset.getDatasetDetails().isTemp());
 
             // Generate Output Record format
@@ -2244,21 +2097,6 @@
         }
     }
 
-    private IBinaryComparatorFactory[] getMergedComparatorFactories(IBinaryComparatorFactory[] comparatorFactories,
-            IBinaryComparatorFactory[] primaryComparatorFactories) {
-        IBinaryComparatorFactory[] btreeCompFactories;
-        int btreeCompFactoriesCount = comparatorFactories.length + primaryComparatorFactories.length;
-        btreeCompFactories = new IBinaryComparatorFactory[btreeCompFactoriesCount];
-        int i = 0;
-        for (; i < comparatorFactories.length; i++) {
-            btreeCompFactories[i] = comparatorFactories[i];
-        }
-        for (int j = 0; i < btreeCompFactoriesCount; i++, j++) {
-            btreeCompFactories[i] = primaryComparatorFactories[j];
-        }
-        return btreeCompFactories;
-    }
-
     private AsterixTupleFilterFactory createTupleFilterFactory(IOperatorSchema[] inputSchemas,
             IVariableTypeEnvironment typeEnv, ILogicalExpression filterExpr, JobGenContext context)
             throws AlgebricksException {
@@ -2267,8 +2105,8 @@
             return null;
         }
         IExpressionRuntimeProvider expressionRuntimeProvider = context.getExpressionRuntimeProvider();
-        IScalarEvaluatorFactory filterEvalFactory = expressionRuntimeProvider.createEvaluatorFactory(filterExpr,
-                typeEnv, inputSchemas, context);
+        IScalarEvaluatorFactory filterEvalFactory =
+                expressionRuntimeProvider.createEvaluatorFactory(filterExpr, typeEnv, inputSchemas, context);
         return new AsterixTupleFilterFactory(filterEvalFactory, context.getBinaryBooleanInspectorFactory());
     }
 
@@ -2277,4 +2115,8 @@
             throw new AlgebricksException("Only record types can be indexed.");
         }
     }
+
+    public IStorageComponentProvider getStorageComponentProvider() {
+        return storaegComponentProvider;
+    }
 }
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 e684285..4ebf055 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,58 +19,116 @@
 
 package org.apache.asterix.metadata.entities;
 
+import java.util.HashMap;
+import java.util.List;
 import java.util.Map;
+import java.util.logging.Level;
+import java.util.logging.Logger;
 
+import org.apache.asterix.active.ActiveJobNotificationHandler;
+import org.apache.asterix.active.IActiveEntityEventsListener;
 import org.apache.asterix.common.config.DatasetConfig.DatasetType;
+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.common.ioopcallbacks.LSMBTreeIOOperationCallbackFactory;
+import org.apache.asterix.common.ioopcallbacks.LSMBTreeWithBuddyIOOperationCallbackFactory;
+import org.apache.asterix.common.ioopcallbacks.LSMInvertedIndexIOOperationCallbackFactory;
+import org.apache.asterix.common.ioopcallbacks.LSMRTreeIOOperationCallbackFactory;
+import org.apache.asterix.common.transactions.IRecoveryManager.ResourceType;
+import org.apache.asterix.common.transactions.JobId;
+import org.apache.asterix.common.utils.JobUtils;
+import org.apache.asterix.common.utils.JobUtils.ProgressState;
+import org.apache.asterix.external.feed.management.FeedConnectionId;
 import org.apache.asterix.metadata.IDatasetDetails;
 import org.apache.asterix.metadata.MetadataCache;
+import org.apache.asterix.metadata.MetadataManager;
+import org.apache.asterix.metadata.MetadataTransactionContext;
 import org.apache.asterix.metadata.api.IMetadataEntity;
+import org.apache.asterix.metadata.declared.BTreeDataflowHelperFactoryProvider;
+import org.apache.asterix.metadata.declared.MetadataProvider;
+import org.apache.asterix.metadata.utils.DatasetUtil;
+import org.apache.asterix.metadata.utils.ExternalDatasetsRegistry;
+import org.apache.asterix.metadata.utils.ExternalIndexingOperations;
+import org.apache.asterix.metadata.utils.IndexUtil;
+import org.apache.asterix.metadata.utils.InvertedIndexDataflowHelperFactoryProvider;
+import org.apache.asterix.metadata.utils.MetadataConstants;
+import org.apache.asterix.metadata.utils.MetadataUtil;
+import org.apache.asterix.metadata.utils.RTreeDataflowHelperFactoryProvider;
+import org.apache.asterix.om.types.ARecordType;
+import org.apache.asterix.om.utils.RecordUtil;
+import org.apache.asterix.transaction.management.opcallbacks.LockThenSearchOperationCallbackFactory;
+import org.apache.asterix.transaction.management.opcallbacks.PrimaryIndexInstantSearchOperationCallbackFactory;
+import org.apache.asterix.transaction.management.opcallbacks.PrimaryIndexModificationOperationCallbackFactory;
+import org.apache.asterix.transaction.management.opcallbacks.PrimaryIndexOperationTrackerFactory;
+import org.apache.asterix.transaction.management.opcallbacks.SecondaryIndexModificationOperationCallbackFactory;
+import org.apache.asterix.transaction.management.opcallbacks.SecondaryIndexOperationTrackerFactory;
+import org.apache.asterix.transaction.management.opcallbacks.SecondaryIndexSearchOperationCallbackFactory;
+import org.apache.asterix.transaction.management.opcallbacks.TempDatasetSecondaryIndexModificationOperationCallbackFactory;
+import org.apache.asterix.transaction.management.opcallbacks.UpsertOperationCallbackFactory;
+import org.apache.commons.lang3.mutable.MutableBoolean;
+import org.apache.commons.lang3.mutable.MutableObject;
+import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.hyracks.algebricks.common.utils.Pair;
 import org.apache.hyracks.algebricks.core.algebra.base.ILogicalOperator;
-import org.apache.hyracks.algebricks.core.algebra.operators.logical.ProjectOperator;
+import org.apache.hyracks.api.client.IHyracksClientConnection;
+import org.apache.hyracks.api.dataflow.value.IBinaryComparatorFactory;
+import org.apache.hyracks.api.dataflow.value.ITypeTraits;
+import org.apache.hyracks.api.job.JobSpecification;
+import org.apache.hyracks.storage.am.common.api.IModificationOperationCallbackFactory;
+import org.apache.hyracks.storage.am.common.api.ISearchOperationCallbackFactory;
+import org.apache.hyracks.storage.am.common.dataflow.IIndexDataflowHelperFactory;
+import org.apache.hyracks.storage.am.common.impls.NoOpOperationCallbackFactory;
+import org.apache.hyracks.storage.am.common.ophelpers.IndexOperation;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallbackFactory;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMMergePolicyFactory;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMOperationTrackerFactory;
+
+import com.fasterxml.jackson.core.JsonProcessingException;
+import com.fasterxml.jackson.databind.ObjectMapper;
 
 /**
  * Metadata describing a dataset.
  */
 public class Dataset implements IMetadataEntity<Dataset> {
 
-    /**
-     * Dataset related operations
+    /*
+     * Constants
      */
-    public static final byte OP_READ = 0x00;
-    public static final byte OP_INSERT = 0x01;
-    public static final byte OP_DELETE = 0x02;
-    public static final byte OP_UPSERT = 0x03;
-
     private static final long serialVersionUID = 1L;
+    private static final Logger LOGGER = Logger.getLogger(Dataset.class.getName());
+    //TODO: Remove Singletons
+    private static final BTreeDataflowHelperFactoryProvider bTreeDataflowHelperFactoryProvider =
+            BTreeDataflowHelperFactoryProvider.INSTANCE;
+    private static final RTreeDataflowHelperFactoryProvider rTreeDataflowHelperFactoryProvider =
+            RTreeDataflowHelperFactoryProvider.INSTANCE;
+    private static final InvertedIndexDataflowHelperFactoryProvider invertedIndexDataflowHelperFactoryProvider =
+            InvertedIndexDataflowHelperFactoryProvider.INSTANCE;
+    /*
+     * Members
+     */
+    private final int datasetId;
     private final String dataverseName;
-    // Enforced to be unique within a dataverse.
     private final String datasetName;
-    // Dataverse of ItemType for this dataset
-    private final String itemTypeDataverseName;
-    // Type of items stored in this dataset.
-    private final String itemTypeName;
+    private final String recordTypeDataverseName;
+    private final String recordTypeName;
     private final String nodeGroupName;
-    private final String compactionPolicy;
+    private final String compactionPolicyFactory;
+    private final Map<String, String> hints;
     private final Map<String, String> compactionPolicyProperties;
     private final DatasetType datasetType;
     private final IDatasetDetails datasetDetails;
-    // Hints related to cardinatlity of dataset, avg size of tuples etc.
-    private final Map<String, String> hints;
-    private final int datasetId;
-    // Type of pending operations with respect to atomic DDL operation
+    private final String metaTypeDataverseName;
+    private final String metaTypeName;
     private int pendingOp;
 
-    // Dataverse of Meta ItemType for this dataset.
-    private final String metaItemTypeDataverseName;
-    // Type of Meta items stored in this dataset.
-    private final String metaItemTypeName;
-
-    public Dataset(String dataverseName, String datasetName, String itemTypeDataverseName, String itemTypeName,
+    public Dataset(String dataverseName, String datasetName, String recordTypeDataverseName, String recordTypeName,
             String nodeGroupName, String compactionPolicy, Map<String, String> compactionPolicyProperties,
             IDatasetDetails datasetDetails, Map<String, String> hints, DatasetType datasetType, int datasetId,
             int pendingOp) {
-        this(dataverseName, datasetName, itemTypeDataverseName, itemTypeName, null, null, nodeGroupName,
-                compactionPolicy, compactionPolicyProperties, datasetDetails, hints, datasetType, datasetId, pendingOp);
+        this(dataverseName, datasetName, recordTypeDataverseName, recordTypeName, /*metaTypeDataverseName*/null,
+                /*metaTypeName*/null, nodeGroupName, compactionPolicy, compactionPolicyProperties, datasetDetails,
+                hints, datasetType, datasetId, pendingOp);
     }
 
     public Dataset(String dataverseName, String datasetName, String itemTypeDataverseName, String itemTypeName,
@@ -79,12 +137,12 @@
             DatasetType datasetType, int datasetId, int pendingOp) {
         this.dataverseName = dataverseName;
         this.datasetName = datasetName;
-        this.itemTypeName = itemTypeName;
-        this.itemTypeDataverseName = itemTypeDataverseName;
-        this.metaItemTypeDataverseName = metaItemTypeDataverseName;
-        this.metaItemTypeName = metaItemTypeName;
+        this.recordTypeName = itemTypeName;
+        this.recordTypeDataverseName = itemTypeDataverseName;
+        this.metaTypeDataverseName = metaItemTypeDataverseName;
+        this.metaTypeName = metaItemTypeName;
         this.nodeGroupName = nodeGroupName;
-        this.compactionPolicy = compactionPolicy;
+        this.compactionPolicyFactory = compactionPolicy;
         this.compactionPolicyProperties = compactionPolicyProperties;
         this.datasetType = datasetType;
         this.datasetDetails = datasetDetails;
@@ -94,10 +152,10 @@
     }
 
     public Dataset(Dataset dataset) {
-        this(dataset.dataverseName, dataset.datasetName, dataset.itemTypeDataverseName, dataset.itemTypeName,
-                dataset.metaItemTypeDataverseName, dataset.metaItemTypeName, dataset.nodeGroupName,
-                dataset.compactionPolicy, dataset.compactionPolicyProperties, dataset.datasetDetails, dataset.hints,
-                dataset.datasetType, dataset.datasetId, dataset.pendingOp);
+        this(dataset.dataverseName, dataset.datasetName, dataset.recordTypeDataverseName, dataset.recordTypeName,
+                dataset.metaTypeDataverseName, dataset.metaTypeName, dataset.nodeGroupName,
+                dataset.compactionPolicyFactory, dataset.compactionPolicyProperties, dataset.datasetDetails,
+                dataset.hints, dataset.datasetType, dataset.datasetId, dataset.pendingOp);
     }
 
     public String getDataverseName() {
@@ -109,11 +167,11 @@
     }
 
     public String getItemTypeName() {
-        return itemTypeName;
+        return recordTypeName;
     }
 
     public String getItemTypeDataverseName() {
-        return itemTypeDataverseName;
+        return recordTypeDataverseName;
     }
 
     public String getNodeGroupName() {
@@ -121,7 +179,7 @@
     }
 
     public String getCompactionPolicy() {
-        return compactionPolicy;
+        return compactionPolicyFactory;
     }
 
     public Map<String, String> getCompactionPolicyProperties() {
@@ -149,15 +207,15 @@
     }
 
     public String getMetaItemTypeDataverseName() {
-        return metaItemTypeDataverseName;
+        return metaTypeDataverseName;
     }
 
     public String getMetaItemTypeName() {
-        return metaItemTypeName;
+        return metaTypeName;
     }
 
     public boolean hasMetaPart() {
-        return metaItemTypeDataverseName != null && metaItemTypeName != null;
+        return metaTypeDataverseName != null && metaTypeName != null;
     }
 
     public void setPendingOp(int pendingOp) {
@@ -192,12 +250,327 @@
         return true;
     }
 
-    public boolean allow(ILogicalOperator topOp, byte operation) {
+    public boolean allow(ILogicalOperator topOp, byte operation) {//NOSONAR: this method is meant to be extended
         return !hasMetaPart();
     }
 
+    /**
+     * Drop this dataset
+     *
+     * @param metadataProvider
+     *            metadata provider that can be used to get metadata info and runtimes
+     * @param mdTxnCtx
+     *            the transaction context
+     * @param jobsToExecute
+     *            a list of jobs to be executed as part of the drop operation
+     * @param bActiveTxn
+     *            whether the metadata transaction is ongoing
+     * @param progress
+     *            a mutable progress state used for error handling during the drop operation
+     * @param hcc
+     *            a client connection to hyracks master for job execution
+     * @throws Exception
+     *             if an error occur during the drop process or if the dataset can't be dropped for any reason
+     */
+    public void drop(MetadataProvider metadataProvider, MutableObject<MetadataTransactionContext> mdTxnCtx,
+            List<JobSpecification> jobsToExecute, MutableBoolean bActiveTxn, MutableObject<ProgressState> progress,
+            IHyracksClientConnection hcc) throws Exception {
+        Map<FeedConnectionId, Pair<JobSpecification, Boolean>> disconnectJobList = new HashMap<>();
+        if (getDatasetType() == DatasetType.INTERNAL) {
+            // prepare job spec(s) that would disconnect any active feeds involving the dataset.
+            IActiveEntityEventsListener[] activeListeners = ActiveJobNotificationHandler.INSTANCE.getEventListeners();
+            for (IActiveEntityEventsListener listener : activeListeners) {
+                if (listener.isEntityUsingDataset(dataverseName, datasetName)) {
+                    throw new CompilationException(ErrorCode.COMPILATION_CANT_DROP_ACTIVE_DATASET,
+                            RecordUtil.toFullyQualifiedName(dataverseName, datasetName),
+                            listener.getEntityId().toString());
+                }
+            }
+            // #. 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 (indexes.get(j).isSecondaryIndex()) {
+                    jobsToExecute.add(IndexUtil.dropJob(indexes.get(j), metadataProvider, this));
+                }
+            }
+            Index primaryIndex =
+                    MetadataManager.INSTANCE.getIndex(mdTxnCtx.getValue(), dataverseName, datasetName, datasetName);
+            jobsToExecute.add(DatasetUtil.createDropDatasetJobSpec(this, primaryIndex, metadataProvider));
+            // #. mark the existing dataset as PendingDropOp
+            MetadataManager.INSTANCE.dropDataset(mdTxnCtx.getValue(), dataverseName, datasetName);
+            MetadataManager.INSTANCE.addDataset(mdTxnCtx.getValue(),
+                    new Dataset(dataverseName, datasetName, getItemTypeDataverseName(), getItemTypeName(),
+                            getMetaItemTypeDataverseName(), getMetaItemTypeName(), 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);
+
+            // # disconnect the feeds
+            for (Pair<JobSpecification, Boolean> p : disconnectJobList.values()) {
+                JobUtils.runJob(hcc, p.first, true);
+            }
+
+            // #. run the jobs
+            for (JobSpecification jobSpec : jobsToExecute) {
+                JobUtils.runJob(hcc, jobSpec, true);
+            }
+
+            mdTxnCtx.setValue(MetadataManager.INSTANCE.beginTransaction());
+            bActiveTxn.setValue(true);
+            metadataProvider.setMetadataTxnContext(mdTxnCtx.getValue());
+        } else {
+            // 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.dropJob(indexes.get(j), metadataProvider, this));
+                } else {
+                    jobsToExecute.add(DatasetUtil.buildDropFilesIndexJobSpec(metadataProvider, this));
+                }
+            }
+
+            // #. mark the existing dataset as PendingDropOp
+            MetadataManager.INSTANCE.dropDataset(mdTxnCtx.getValue(), dataverseName, datasetName);
+            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.
+        MetadataManager.INSTANCE.dropDataset(mdTxnCtx.getValue(), dataverseName, datasetName);
+        // Drop the associated nodegroup
+        String nodegroup = getNodeGroupName();
+        if (!nodegroup.equalsIgnoreCase(MetadataConstants.METADATA_DEFAULT_NODEGROUP_NAME)) {
+            MetadataManager.INSTANCE.dropNodegroup(mdTxnCtx.getValue(), dataverseName + ":" + datasetName);
+        }
+    }
+
+    /**
+     * Create the index dataflow helper factory for a particular index on the dataset
+     *
+     * @param mdProvider
+     *            metadata provider to get metadata information, components, and runtimes
+     * @param index
+     *            the index to get the dataflow helper factory for
+     * @param recordType
+     *            the record type for the dataset
+     * @param metaType
+     *            the meta type for the dataset
+     * @param mergePolicyFactory
+     *            the merge policy factory of the dataset
+     * @param mergePolicyProperties
+     *            the merge policy properties for the dataset
+     * @return indexDataflowHelperFactory
+     *         an instance of {@link org.apache.hyracks.storage.am.common.dataflow.IIndexDataflowHelperFactory}
+     * @throws AlgebricksException
+     *             if dataflow helper factory could not be created
+     */
+    public IIndexDataflowHelperFactory getIndexDataflowHelperFactory(MetadataProvider mdProvider, Index index,
+            ARecordType recordType, ARecordType metaType, ILSMMergePolicyFactory mergePolicyFactory,
+            Map<String, String> mergePolicyProperties) throws AlgebricksException {
+        ITypeTraits[] filterTypeTraits = DatasetUtil.computeFilterTypeTraits(this, recordType);
+        IBinaryComparatorFactory[] filterCmpFactories = DatasetUtil.computeFilterBinaryComparatorFactories(this,
+                recordType, mdProvider.getStorageComponentProvider().getComparatorFactoryProvider());
+        switch (index.getIndexType()) {
+            case BTREE:
+                return bTreeDataflowHelperFactoryProvider.getIndexDataflowHelperFactory(mdProvider, this, index,
+                        recordType, metaType, mergePolicyFactory, mergePolicyProperties, filterTypeTraits,
+                        filterCmpFactories);
+            case RTREE:
+                return rTreeDataflowHelperFactoryProvider.getIndexDataflowHelperFactory(mdProvider, this, index,
+                        recordType, metaType, mergePolicyFactory, mergePolicyProperties, filterTypeTraits,
+                        filterCmpFactories);
+            case LENGTH_PARTITIONED_NGRAM_INVIX:
+            case LENGTH_PARTITIONED_WORD_INVIX:
+            case SINGLE_PARTITION_NGRAM_INVIX:
+            case SINGLE_PARTITION_WORD_INVIX:
+                return invertedIndexDataflowHelperFactoryProvider.getIndexDataflowHelperFactory(mdProvider, this,
+                        index, recordType, metaType, mergePolicyFactory, mergePolicyProperties, filterTypeTraits,
+                        filterCmpFactories);
+            default:
+                throw new CompilationException(ErrorCode.COMPILATION_UNKNOWN_INDEX_TYPE,
+                        index.getIndexType().toString());
+        }
+    }
+
+    /**
+     * Get the IO Operation callback factory for the index which belongs to this dataset
+     *
+     * @param index
+     *            the index
+     * @return ioOperationCallbackFactory
+     *         an instance of {@link org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallbackFactory}
+     *         to be used with IO operations
+     * @throws AlgebricksException
+     *             if the factory could not be created for the index/dataset combination
+     */
+    public ILSMIOOperationCallbackFactory getIoOperationCallbackFactory(Index index) throws AlgebricksException {
+        switch (index.getIndexType()) {
+            case BTREE:
+                return getDatasetType() == DatasetType.EXTERNAL
+                        && !index.getIndexName().equals(BTreeDataflowHelperFactoryProvider.externalFileIndexName(this))
+                                ? LSMBTreeWithBuddyIOOperationCallbackFactory.INSTANCE
+                                : LSMBTreeIOOperationCallbackFactory.INSTANCE;
+            case RTREE:
+                return LSMRTreeIOOperationCallbackFactory.INSTANCE;
+            case LENGTH_PARTITIONED_NGRAM_INVIX:
+            case LENGTH_PARTITIONED_WORD_INVIX:
+            case SINGLE_PARTITION_NGRAM_INVIX:
+            case SINGLE_PARTITION_WORD_INVIX:
+                return LSMInvertedIndexIOOperationCallbackFactory.INSTANCE;
+            default:
+                throw new CompilationException(ErrorCode.COMPILATION_UNKNOWN_INDEX_TYPE,
+                        index.getIndexType().toString());
+        }
+    }
+
+    /**
+     * get the IndexOperationTrackerFactory for a particular index on the dataset
+     *
+     * @param index
+     *            the index
+     * @return an instance of {@link org.apache.hyracks.storage.am.lsm.common.api.ILSMOperationTrackerFactory}
+     */
+    public ILSMOperationTrackerFactory getIndexOperationTrackerFactory(Index index) {
+        return index.isPrimaryIndex() ? new PrimaryIndexOperationTrackerFactory(getDatasetId())
+                : new SecondaryIndexOperationTrackerFactory(getDatasetId());
+    }
+
+    /**
+     * Get search callback factory for this dataset with the passed index and operation
+     *
+     * @param index
+     *            the index
+     * @param jobId
+     *            the job id being compiled
+     * @param op
+     *            the operation this search is part of
+     * @param primaryKeyFields
+     *            the primary key fields indexes for locking purposes
+     * @return
+     *         an instance of {@link org.apache.hyracks.storage.am.common.api.ISearchOperationCallbackFactory}
+     * @throws AlgebricksException
+     *             if the callback factory could not be created
+     */
+    public ISearchOperationCallbackFactory getSearchCallbackFactory(IStorageComponentProvider storageComponentProvider,
+            Index index, JobId jobId, IndexOperation op, int[] primaryKeyFields) throws AlgebricksException {
+        if (getDatasetDetails().isTemp()) {
+            return NoOpOperationCallbackFactory.INSTANCE;
+        } else if (index.isPrimaryIndex()) {
+            /**
+             * Due to the read-committed isolation level,
+             * we may acquire very short duration lock(i.e., instant lock) for readers.
+             */
+            return (op == IndexOperation.UPSERT)
+                    ? new LockThenSearchOperationCallbackFactory(jobId, getDatasetId(), primaryKeyFields,
+                            storageComponentProvider.getTransactionSubsystemProvider(), ResourceType.LSM_BTREE)
+                    : new PrimaryIndexInstantSearchOperationCallbackFactory(jobId, getDatasetId(), primaryKeyFields,
+                            storageComponentProvider.getTransactionSubsystemProvider(), ResourceType.LSM_BTREE);
+        }
+        return new SecondaryIndexSearchOperationCallbackFactory();
+    }
+
+    /**
+     * Get the modification callback factory associated with this dataset, the passed index, and operation.
+     *
+     * @param index
+     *            the index
+     * @param jobId
+     *            the job id of the job being compiled
+     * @param op
+     *            the operation performed for this callback
+     * @param primaryKeyFields
+     *            the indexes of the primary keys (used for lock operations)
+     * @return
+     *         an instance of {@link org.apache.hyracks.storage.am.common.api.IModificationOperationCallbackFactory}
+     * @throws AlgebricksException
+     *             If the callback factory could not be created
+     */
+    public IModificationOperationCallbackFactory getModificationCallbackFactory(
+            IStorageComponentProvider componentProvider, Index index, JobId jobId, IndexOperation op,
+            int[] primaryKeyFields) throws AlgebricksException {
+        if (getDatasetDetails().isTemp()) {
+            return new TempDatasetSecondaryIndexModificationOperationCallbackFactory(jobId, getDatasetId(),
+                    primaryKeyFields, componentProvider.getTransactionSubsystemProvider(), op, index.resourceType());
+        } else if (index.isPrimaryIndex()) {
+            return op == IndexOperation.UPSERT
+                    ? new UpsertOperationCallbackFactory(jobId, getDatasetId(), primaryKeyFields,
+                            componentProvider.getTransactionSubsystemProvider(), op, index.resourceType(),
+                            hasMetaPart())
+                    : op == IndexOperation.DELETE || op == IndexOperation.INSERT
+                            ? new PrimaryIndexModificationOperationCallbackFactory(jobId, getDatasetId(),
+                                    primaryKeyFields, componentProvider.getTransactionSubsystemProvider(), op,
+                                    index.resourceType(), hasMetaPart())
+                            : NoOpOperationCallbackFactory.INSTANCE;
+        } else {
+            return op == IndexOperation.DELETE || op == IndexOperation.INSERT || op == IndexOperation.UPSERT
+                    ? new SecondaryIndexModificationOperationCallbackFactory(jobId, getDatasetId(), primaryKeyFields,
+                            componentProvider.getTransactionSubsystemProvider(), op, index.resourceType(),
+                            hasMetaPart())
+                    : NoOpOperationCallbackFactory.INSTANCE;
+        }
+    }
+
     @Override
     public String toString() {
-        return dataverseName + "." + datasetName;
+        ObjectMapper mapper = new ObjectMapper();
+        try {
+            return mapper.writeValueAsString(toMap());
+        } catch (JsonProcessingException e) {
+            LOGGER.log(Level.WARNING, "Unable to convert map to json String", e);
+            return dataverseName + "." + datasetName;
+        }
+    }
+
+    public Map<String, Object> toMap() {
+        Map<String, Object> tree = new HashMap<>();
+        tree.put("datasetId", Integer.toString(datasetId));
+        tree.put("dataverseName", dataverseName);
+        tree.put("datasetName", datasetName);
+        tree.put("recordTypeDataverseName", recordTypeDataverseName);
+        tree.put("recordTypeName", recordTypeName);
+        tree.put("nodeGroupName", nodeGroupName);
+        tree.put("compactionPolicyFactory", compactionPolicyFactory);
+        tree.put("hints", hints);
+        tree.put("compactionPolicyProperties", compactionPolicyProperties);
+        tree.put("datasetType", datasetType.name());
+        tree.put("datasetDetails", datasetDetails.toString());
+        tree.put("metaTypeDataverseName", metaTypeDataverseName);
+        tree.put("metaTypeName", metaTypeName);
+        tree.put("pendingOp", MetadataUtil.pendingOpToString(pendingOp));
+        return tree;
+    }
+
+    @Override
+    public int hashCode() {
+        final int prime = 31;
+        int result = 1;
+        result = prime * result + ((datasetName == null) ? 0 : datasetName.hashCode());
+        result = prime * result + ((dataverseName == null) ? 0 : dataverseName.hashCode());
+        return result;
     }
 }
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entities/ExternalDatasetDetails.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entities/ExternalDatasetDetails.java
index cdc982f..b9b4cd9 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entities/ExternalDatasetDetails.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entities/ExternalDatasetDetails.java
@@ -21,17 +21,20 @@
 
 import java.io.DataOutput;
 import java.util.Date;
+import java.util.HashMap;
 import java.util.Map;
+import java.util.logging.Level;
+import java.util.logging.Logger;
 
 import org.apache.asterix.builders.IARecordBuilder;
 import org.apache.asterix.builders.OrderedListBuilder;
 import org.apache.asterix.builders.RecordBuilder;
 import org.apache.asterix.common.config.DatasetConfig.DatasetType;
-import org.apache.asterix.common.config.DatasetConfig.ExternalDatasetTransactionState;
+import org.apache.asterix.common.config.DatasetConfig.TransactionState;
 import org.apache.asterix.formats.nontagged.SerializerDeserializerProvider;
 import org.apache.asterix.metadata.IDatasetDetails;
 import org.apache.asterix.metadata.bootstrap.MetadataRecordTypes;
-import org.apache.asterix.metadata.utils.DatasetUtils;
+import org.apache.asterix.metadata.utils.DatasetUtil;
 import org.apache.asterix.om.base.ADateTime;
 import org.apache.asterix.om.base.AInt32;
 import org.apache.asterix.om.base.AMutableString;
@@ -42,17 +45,21 @@
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
 
+import com.fasterxml.jackson.core.JsonProcessingException;
+import com.fasterxml.jackson.databind.ObjectMapper;
+
 public class ExternalDatasetDetails implements IDatasetDetails {
 
     private static final long serialVersionUID = 1L;
+    private static final Logger LOGGER = Logger.getLogger(ExternalDatasetDetails.class.getName());
     private final String adapter;
     private final Map<String, String> properties;
     private final long addToCacheTime;
     private Date lastRefreshTime;
-    private ExternalDatasetTransactionState state;
+    private TransactionState state;
 
     public ExternalDatasetDetails(String adapter, Map<String, String> properties, Date lastRefreshTime,
-            ExternalDatasetTransactionState state) {
+            TransactionState state) {
         this.properties = properties;
         this.adapter = adapter;
         this.addToCacheTime = System.currentTimeMillis();
@@ -103,7 +110,7 @@
             String name = property.getKey();
             String value = property.getValue();
             itemValue.reset();
-            DatasetUtils.writePropertyTypeRecord(name, value, itemValue.getDataOutput(),
+            DatasetUtil.writePropertyTypeRecord(name, value, itemValue.getDataOutput(),
                     MetadataRecordTypes.DATASOURCE_ADAPTER_PROPERTIES_RECORDTYPE);
             listBuilder.addItem(itemValue);
         }
@@ -143,11 +150,31 @@
         this.lastRefreshTime = timestamp;
     }
 
-    public ExternalDatasetTransactionState getState() {
+    public TransactionState getState() {
         return state;
     }
 
-    public void setState(ExternalDatasetTransactionState state) {
+    public void setState(TransactionState state) {
         this.state = state;
     }
+
+    @Override
+    public String toString() {
+        ObjectMapper mapper = new ObjectMapper();
+        try {
+            return mapper.writeValueAsString(toMap());
+        } catch (JsonProcessingException e) {
+            LOGGER.log(Level.WARNING, "Unable to convert map to json String", e);
+            return getClass().getSimpleName();
+        }
+    }
+
+    private Map<String, Object> toMap() {
+        Map<String, Object> map = new HashMap<>();
+        map.put("adapter", adapter);
+        map.put("properties", properties);
+        map.put("lastRefreshTime", lastRefreshTime.toString());
+        map.put("state", state.name());
+        return map;
+    }
 }
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entities/Index.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entities/Index.java
index f33a2b6..df47c70 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entities/Index.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entities/Index.java
@@ -22,13 +22,16 @@
 import java.util.List;
 
 import org.apache.asterix.common.config.DatasetConfig.IndexType;
-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.common.transactions.IRecoveryManager.ResourceType;
 import org.apache.asterix.metadata.MetadataCache;
 import org.apache.asterix.metadata.api.IMetadataEntity;
 import org.apache.asterix.om.types.ARecordType;
 import org.apache.asterix.om.types.AUnionType;
 import org.apache.asterix.om.types.IAType;
-import org.apache.asterix.om.util.NonTaggedFormatUtil;
+import org.apache.asterix.om.utils.NonTaggedFormatUtil;
+import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
 import org.apache.hyracks.algebricks.common.utils.Pair;
 
 /**
@@ -131,7 +134,7 @@
         return !isPrimaryIndex();
     }
 
-    public static Pair<IAType, Boolean> getNonNullableType(IAType keyType) throws AsterixException {
+    public static Pair<IAType, Boolean> getNonNullableType(IAType keyType) {
         boolean nullable = false;
         IAType actualKeyType = keyType;
         if (NonTaggedFormatUtil.isOptional(keyType)) {
@@ -142,7 +145,7 @@
     }
 
     public static Pair<IAType, Boolean> getNonNullableOpenFieldType(IAType fieldType, List<String> fieldName,
-            ARecordType recType) throws AsterixException {
+            ARecordType recType) throws AlgebricksException {
         Pair<IAType, Boolean> keyPairType = null;
         IAType subType = recType;
         for (int i = 0; i < fieldName.size(); i++) {
@@ -159,12 +162,12 @@
     }
 
     public static Pair<IAType, Boolean> getNonNullableKeyFieldType(List<String> expr, ARecordType recType)
-            throws AsterixException {
+            throws AlgebricksException {
         IAType keyType = Index.keyFieldType(expr, recType);
         return getNonNullableType(keyType);
     }
 
-    private static IAType keyFieldType(List<String> expr, ARecordType recType) throws AsterixException {
+    private static IAType keyFieldType(List<String> expr, ARecordType recType) throws AlgebricksException {
         IAType fieldType = recType;
         fieldType = recType.getSubFieldType(expr);
         return fieldType;
@@ -251,4 +254,20 @@
         }
         return false;
     }
+
+    public byte resourceType() throws CompilationException {
+        switch (indexType) {
+            case BTREE:
+                return ResourceType.LSM_BTREE;
+            case RTREE:
+                return ResourceType.LSM_RTREE;
+            case LENGTH_PARTITIONED_NGRAM_INVIX:
+            case LENGTH_PARTITIONED_WORD_INVIX:
+            case SINGLE_PARTITION_NGRAM_INVIX:
+            case SINGLE_PARTITION_WORD_INVIX:
+                return ResourceType.LSM_INVERTED_INDEX;
+            default:
+                throw new CompilationException(ErrorCode.COMPILATION_UNKNOWN_INDEX_TYPE, indexType);
+        }
+    }
 }
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/DatasetTupleTranslator.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/DatasetTupleTranslator.java
index dbe4a67..138cad7 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/DatasetTupleTranslator.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/DatasetTupleTranslator.java
@@ -37,7 +37,7 @@
 import org.apache.asterix.builders.RecordBuilder;
 import org.apache.asterix.builders.UnorderedListBuilder;
 import org.apache.asterix.common.config.DatasetConfig.DatasetType;
-import org.apache.asterix.common.config.DatasetConfig.ExternalDatasetTransactionState;
+import org.apache.asterix.common.config.DatasetConfig.TransactionState;
 import org.apache.asterix.formats.nontagged.SerializerDeserializerProvider;
 import org.apache.asterix.metadata.IDatasetDetails;
 import org.apache.asterix.metadata.MetadataException;
@@ -48,7 +48,7 @@
 import org.apache.asterix.metadata.entities.InternalDatasetDetails;
 import org.apache.asterix.metadata.entities.InternalDatasetDetails.FileStructure;
 import org.apache.asterix.metadata.entities.InternalDatasetDetails.PartitioningStrategy;
-import org.apache.asterix.metadata.utils.DatasetUtils;
+import org.apache.asterix.metadata.utils.DatasetUtil;
 import org.apache.asterix.om.base.ABoolean;
 import org.apache.asterix.om.base.ADateTime;
 import org.apache.asterix.om.base.AInt32;
@@ -233,7 +233,7 @@
                         .getValueByPos(MetadataRecordTypes.EXTERNAL_DETAILS_ARECORD_LAST_REFRESH_TIME_FIELD_INDEX)))
                                 .getChrononTime());
                 // State
-                ExternalDatasetTransactionState state = ExternalDatasetTransactionState
+                TransactionState state = TransactionState
                         .values()[((AInt32) datasetDetailsRecord.getValueByPos(
                                 MetadataRecordTypes.EXTERNAL_DETAILS_ARECORD_TRANSACTION_STATE_FIELD_INDEX))
                                         .getIntegerValue()];
@@ -326,7 +326,7 @@
                 String name = property.getKey();
                 String value = property.getValue();
                 itemValue.reset();
-                DatasetUtils.writePropertyTypeRecord(name, value, itemValue.getDataOutput(),
+                DatasetUtil.writePropertyTypeRecord(name, value, itemValue.getDataOutput(),
                         MetadataRecordTypes.COMPACTION_POLICY_PROPERTIES_RECORDTYPE);
                 listBuilder.addItem(itemValue);
             }
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/DatatypeTupleTranslator.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/DatatypeTupleTranslator.java
index fc7ef60..b81ec29 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/DatatypeTupleTranslator.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/DatatypeTupleTranslator.java
@@ -51,7 +51,7 @@
 import org.apache.asterix.om.types.AbstractCollectionType;
 import org.apache.asterix.om.types.AbstractComplexType;
 import org.apache.asterix.om.types.IAType;
-import org.apache.asterix.om.util.NonTaggedFormatUtil;
+import org.apache.asterix.om.utils.NonTaggedFormatUtil;
 import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/IndexTupleTranslator.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/IndexTupleTranslator.java
index 70d18b4..6446e67 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/IndexTupleTranslator.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/IndexTupleTranslator.java
@@ -29,7 +29,6 @@
 
 import org.apache.asterix.builders.OrderedListBuilder;
 import org.apache.asterix.common.config.DatasetConfig.IndexType;
-import org.apache.asterix.common.exceptions.AsterixException;
 import org.apache.asterix.common.transactions.JobId;
 import org.apache.asterix.formats.nontagged.SerializerDeserializerProvider;
 import org.apache.asterix.metadata.MetadataException;
@@ -39,7 +38,7 @@
 import org.apache.asterix.metadata.entities.BuiltinTypeMap;
 import org.apache.asterix.metadata.entities.Dataset;
 import org.apache.asterix.metadata.entities.Index;
-import org.apache.asterix.metadata.utils.KeyFieldTypeUtils;
+import org.apache.asterix.metadata.utils.KeyFieldTypeUtil;
 import org.apache.asterix.om.base.ABoolean;
 import org.apache.asterix.om.base.ACollectionCursor;
 import org.apache.asterix.om.base.AInt32;
@@ -53,6 +52,7 @@
 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.algebricks.common.exceptions.AlgebricksException;
 import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
 import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
 import org.apache.hyracks.dataflow.common.data.accessors.ITupleReference;
@@ -61,6 +61,7 @@
  * Translates an Index metadata entity to an ITupleReference and vice versa.
  */
 public class IndexTupleTranslator extends AbstractTupleTranslator<Index> {
+    private static final long serialVersionUID = 1L;
     // Field indexes of serialized Index in a tuple.
     // First key field.
     public static final int INDEX_DATAVERSENAME_TUPLE_FIELD_INDEX = 0;
@@ -82,18 +83,16 @@
     private transient AOrderedListType int8List = new AOrderedListType(BuiltinType.AINT8, null);
     private transient ArrayBackedValueStorage nameValue = new ArrayBackedValueStorage();
     private transient ArrayBackedValueStorage itemValue = new ArrayBackedValueStorage();
-    private transient List<List<String>> searchKey;
-    private transient List<IAType> searchKeyType;
     private transient AMutableInt8 aInt8 = new AMutableInt8((byte) 0);
     @SuppressWarnings("unchecked")
-    private ISerializerDeserializer<AInt32> intSerde = SerializerDeserializerProvider.INSTANCE
-            .getSerializerDeserializer(BuiltinType.AINT32);
+    private ISerializerDeserializer<AInt32> intSerde =
+            SerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.AINT32);
     @SuppressWarnings("unchecked")
-    private ISerializerDeserializer<AInt8> int8Serde = SerializerDeserializerProvider.INSTANCE
-            .getSerializerDeserializer(BuiltinType.AINT8);
+    private ISerializerDeserializer<AInt8> int8Serde =
+            SerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.AINT8);
     @SuppressWarnings("unchecked")
-    private ISerializerDeserializer<ARecord> recordSerde = SerializerDeserializerProvider.INSTANCE
-            .getSerializerDeserializer(MetadataRecordTypes.INDEX_RECORDTYPE);
+    private ISerializerDeserializer<ARecord> recordSerde =
+            SerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(MetadataRecordTypes.INDEX_RECORDTYPE);
     private final MetadataNode metadataNode;
     private final JobId jobId;
 
@@ -120,14 +119,15 @@
         IndexType indexStructure = IndexType
                 .valueOf(((AString) rec.getValueByPos(MetadataRecordTypes.INDEX_ARECORD_INDEXSTRUCTURE_FIELD_INDEX))
                         .getStringValue());
-        IACursor fieldNameCursor = ((AOrderedList) rec
-                .getValueByPos(MetadataRecordTypes.INDEX_ARECORD_SEARCHKEY_FIELD_INDEX)).getCursor();
-        List<List<String>> searchKey = new ArrayList<List<String>>();
+        IACursor fieldNameCursor =
+                ((AOrderedList) rec.getValueByPos(MetadataRecordTypes.INDEX_ARECORD_SEARCHKEY_FIELD_INDEX))
+                        .getCursor();
+        List<List<String>> searchKey = new ArrayList<>();
         AOrderedList fieldNameList;
         while (fieldNameCursor.next()) {
             fieldNameList = (AOrderedList) fieldNameCursor.get();
             IACursor nestedFieldNameCursor = (fieldNameList.getCursor());
-            List<String> nestedFieldName = new ArrayList<String>();
+            List<String> nestedFieldName = new ArrayList<>();
             while (nestedFieldNameCursor.next()) {
                 nestedFieldName.add(((AString) nestedFieldNameCursor.get()).getStringValue());
             }
@@ -138,7 +138,7 @@
         if (indexKeyTypeFieldPos > 0) {
             fieldTypeCursor = ((AOrderedList) rec.getValueByPos(indexKeyTypeFieldPos)).getCursor();
         }
-        List<IAType> searchKeyType = new ArrayList<IAType>(searchKey.size());
+        List<IAType> searchKeyType = new ArrayList<>(searchKey.size());
         while (fieldTypeCursor.next()) {
             String typeName = ((AString) fieldTypeCursor.get()).getStringValue();
             IAType fieldType = BuiltinTypeMap.getTypeFromTypeName(metadataNode, jobId, dvName, typeName, false);
@@ -150,8 +150,8 @@
         if (isEnforcedFieldPos > 0) {
             isEnforcingKeys = ((ABoolean) rec.getValueByPos(isEnforcedFieldPos)).getBoolean();
         }
-        Boolean isPrimaryIndex = ((ABoolean) rec.getValueByPos(MetadataRecordTypes.INDEX_ARECORD_ISPRIMARY_FIELD_INDEX))
-                .getBoolean();
+        Boolean isPrimaryIndex =
+                ((ABoolean) rec.getValueByPos(MetadataRecordTypes.INDEX_ARECORD_ISPRIMARY_FIELD_INDEX)).getBoolean();
         int pendingOp = ((AInt32) rec.getValueByPos(MetadataRecordTypes.INDEX_ARECORD_PENDINGOP_FIELD_INDEX))
                 .getIntegerValue();
         // Check if there is a gram length as well.
@@ -180,8 +180,8 @@
             Dataset dSet = metadataNode.getDataset(jobId, dvName, dsName);
             String datatypeName = dSet.getItemTypeName();
             String datatypeDataverseName = dSet.getItemTypeDataverseName();
-            ARecordType recordDt = (ARecordType) metadataNode.getDatatype(jobId, datatypeDataverseName, datatypeName)
-                    .getDatatype();
+            ARecordType recordDt =
+                    (ARecordType) metadataNode.getDatatype(jobId, datatypeDataverseName, datatypeName).getDatatype();
             String metatypeName = dSet.getMetaItemTypeName();
             String metatypeDataverseName = dSet.getMetaItemTypeDataverseName();
             ARecordType metaDt = null;
@@ -190,8 +190,8 @@
                         .getDatatype();
             }
             try {
-                searchKeyType = KeyFieldTypeUtils.getKeyTypes(recordDt, metaDt, searchKey, keyFieldSourceIndicator);
-            } catch (AsterixException e) {
+                searchKeyType = KeyFieldTypeUtil.getKeyTypes(recordDt, metaDt, searchKey, keyFieldSourceIndicator);
+            } catch (AlgebricksException e) {
                 throw new MetadataException(e);
             }
         }
@@ -242,8 +242,8 @@
         // write field 4
         primaryKeyListBuilder.reset((AOrderedListType) MetadataRecordTypes.INDEX_RECORDTYPE
                 .getFieldTypes()[MetadataRecordTypes.INDEX_ARECORD_SEARCHKEY_FIELD_INDEX]);
-        this.searchKey = instance.getKeyFieldNames();
-        for (List<String> field : this.searchKey) {
+        List<List<String>> searchKey = instance.getKeyFieldNames();
+        for (List<String> field : searchKey) {
             listBuilder.reset(stringList);
             for (String subField : field) {
                 itemValue.reset();
@@ -293,14 +293,13 @@
             // write optional field 9
             OrderedListBuilder typeListBuilder = new OrderedListBuilder();
             typeListBuilder.reset(new AOrderedListType(BuiltinType.ANY, null));
-            ArrayBackedValueStorage nameValue = new ArrayBackedValueStorage();
             nameValue.reset();
             aString.setValue(INDEX_SEARCHKEY_TYPE_FIELD_NAME);
 
             stringSerde.serialize(aString, nameValue.getDataOutput());
 
-            this.searchKeyType = instance.getKeyFieldTypes();
-            for (IAType type : this.searchKeyType) {
+            List<IAType> searchKeyType = instance.getKeyFieldTypes();
+            for (IAType type : searchKeyType) {
                 itemValue.reset();
                 aString.setValue(type.getTypeName());
                 stringSerde.serialize(aString, itemValue.getDataOutput());
@@ -334,7 +333,6 @@
         }
         if (needSerialization) {
             listBuilder.reset(int8List);
-            ArrayBackedValueStorage nameValue = new ArrayBackedValueStorage();
             nameValue.reset();
             aString.setValue(INDEX_SEARCHKEY_SOURCE_INDICATOR_FIELD_NAME);
             stringSerde.serialize(aString, nameValue.getDataOutput());
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/external/FeedOperations.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/feeds/FeedOperations.java
similarity index 93%
rename from asterixdb/asterix-app/src/main/java/org/apache/asterix/app/external/FeedOperations.java
rename to asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/feeds/FeedOperations.java
index 34746d3..00c9010 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/external/FeedOperations.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/feeds/FeedOperations.java
@@ -16,7 +16,7 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.asterix.app.external;
+package org.apache.asterix.metadata.feeds;
 
 import java.util.Collection;
 import java.util.List;
@@ -39,10 +39,10 @@
 import org.apache.asterix.external.util.FeedConstants;
 import org.apache.asterix.external.util.FeedUtils;
 import org.apache.asterix.external.util.FeedUtils.FeedRuntimeType;
-import org.apache.asterix.file.JobSpecificationUtils;
 import org.apache.asterix.metadata.declared.MetadataProvider;
 import org.apache.asterix.metadata.entities.Feed;
-import org.apache.asterix.runtime.util.ClusterStateManager;
+import org.apache.asterix.runtime.utils.ClusterStateManager;
+import org.apache.asterix.runtime.utils.RuntimeUtils;
 import org.apache.hyracks.algebricks.common.constraints.AlgebricksAbsolutePartitionConstraint;
 import org.apache.hyracks.algebricks.common.constraints.AlgebricksPartitionConstraint;
 import org.apache.hyracks.algebricks.common.constraints.AlgebricksPartitionConstraintHelper;
@@ -62,6 +62,9 @@
  */
 public class FeedOperations {
 
+    private FeedOperations() {
+    }
+
     /**
      * Builds the job spec for ingesting a (primary) feed from its external source via the feed adaptor.
      *
@@ -72,9 +75,9 @@
      */
     public static Pair<JobSpecification, IAdapterFactory> buildFeedIntakeJobSpec(Feed primaryFeed,
             MetadataProvider metadataProvider, FeedPolicyAccessor policyAccessor) throws Exception {
-        JobSpecification spec = JobSpecificationUtils.createJobSpecification();
+        JobSpecification spec = RuntimeUtils.createJobSpecification();
         spec.setFrameSize(FeedConstants.JobConstants.DEFAULT_FRAME_SIZE);
-        IAdapterFactory adapterFactory = null;
+        IAdapterFactory adapterFactory;
         IOperatorDescriptor feedIngestor;
         AlgebricksPartitionConstraint ingesterPc;
         Triple<IOperatorDescriptor, AlgebricksPartitionConstraint, IAdapterFactory> t =
@@ -94,10 +97,10 @@
      * Builds the job spec for sending message to an active feed to disconnect it from the
      * its source.
      */
-    public static Pair<JobSpecification, Boolean> buildDisconnectFeedJobSpec(MetadataProvider metadataProvider,
-            FeedConnectionId connectionId) throws AsterixException, AlgebricksException {
+    public static Pair<JobSpecification, Boolean> buildDisconnectFeedJobSpec(FeedConnectionId connectionId)
+            throws AlgebricksException {
 
-        JobSpecification spec = JobSpecificationUtils.createJobSpecification();
+        JobSpecification spec = RuntimeUtils.createJobSpecification();
         IOperatorDescriptor feedMessenger;
         AlgebricksPartitionConstraint messengerPc;
         List<String> locations = null;
@@ -158,8 +161,8 @@
         return buildSendFeedMessageRuntime(jobSpec, feedConenctionId, feedMessage, locations);
     }
 
-    public static JobSpecification buildRemoveFeedStorageJob(Feed feed) throws Exception {
-        JobSpecification spec = JobSpecificationUtils.createJobSpecification();
+    public static JobSpecification buildRemoveFeedStorageJob(Feed feed) throws AsterixException {
+        JobSpecification spec = RuntimeUtils.createJobSpecification();
         AlgebricksAbsolutePartitionConstraint allCluster = ClusterStateManager.INSTANCE.getClusterLocations();
         Set<String> nodes = new TreeSet<>();
         for (String node : allCluster.getLocations()) {
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/functions/MetadataBuiltinFunctions.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/functions/MetadataBuiltinFunctions.java
index 185232e..137e625 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/functions/MetadataBuiltinFunctions.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/functions/MetadataBuiltinFunctions.java
@@ -25,7 +25,7 @@
 import org.apache.asterix.om.types.ATypeTag;
 import org.apache.asterix.om.types.BuiltinType;
 import org.apache.asterix.om.types.IAType;
-import org.apache.asterix.om.util.ConstantExpressionUtil;
+import org.apache.asterix.om.utils.ConstantExpressionUtil;
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
 import org.apache.hyracks.algebricks.common.utils.Pair;
 import org.apache.hyracks.algebricks.core.algebra.base.ILogicalExpression;
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/DatasetUtil.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/DatasetUtil.java
new file mode 100644
index 0000000..80792b5
--- /dev/null
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/DatasetUtil.java
@@ -0,0 +1,528 @@
+/*
+ * 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.DataOutput;
+import java.io.File;
+import java.rmi.RemoteException;
+import java.util.List;
+import java.util.Map;
+import java.util.logging.Logger;
+
+import org.apache.asterix.builders.IARecordBuilder;
+import org.apache.asterix.builders.RecordBuilder;
+import org.apache.asterix.common.config.DatasetConfig.DatasetType;
+import org.apache.asterix.common.context.CorrelatedPrefixMergePolicyFactory;
+import org.apache.asterix.common.context.IStorageComponentProvider;
+import org.apache.asterix.common.exceptions.ACIDException;
+import org.apache.asterix.common.exceptions.AsterixException;
+import org.apache.asterix.common.transactions.IResourceFactory;
+import org.apache.asterix.external.indexing.IndexingConstants;
+import org.apache.asterix.formats.base.IDataFormat;
+import org.apache.asterix.formats.nontagged.SerializerDeserializerProvider;
+import org.apache.asterix.formats.nontagged.TypeTraitProvider;
+import org.apache.asterix.metadata.MetadataException;
+import org.apache.asterix.metadata.MetadataManager;
+import org.apache.asterix.metadata.MetadataTransactionContext;
+import org.apache.asterix.metadata.declared.BTreeDataflowHelperFactoryProvider;
+import org.apache.asterix.metadata.declared.MetadataProvider;
+import org.apache.asterix.metadata.entities.CompactionPolicy;
+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.entities.InternalDatasetDetails;
+import org.apache.asterix.om.base.AMutableString;
+import org.apache.asterix.om.base.AString;
+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.RuntimeComponentsProvider;
+import org.apache.asterix.runtime.utils.RuntimeUtils;
+import org.apache.asterix.transaction.management.resource.LSMBTreeLocalResourceMetadataFactory;
+import org.apache.asterix.transaction.management.resource.PersistentLocalResourceFactoryProvider;
+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.data.IBinaryComparatorFactoryProvider;
+import org.apache.hyracks.algebricks.data.IBinaryHashFunctionFactoryProvider;
+import org.apache.hyracks.api.dataflow.value.IBinaryComparatorFactory;
+import org.apache.hyracks.api.dataflow.value.IBinaryHashFunctionFactory;
+import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
+import org.apache.hyracks.api.dataflow.value.ITypeTraits;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.io.FileSplit;
+import org.apache.hyracks.api.job.JobSpecification;
+import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
+import org.apache.hyracks.dataflow.std.file.IFileSplitProvider;
+import org.apache.hyracks.storage.am.common.dataflow.IIndexDataflowHelperFactory;
+import org.apache.hyracks.storage.am.common.dataflow.IndexDropOperatorDescriptor;
+import org.apache.hyracks.storage.am.common.dataflow.TreeIndexCreateOperatorDescriptor;
+import org.apache.hyracks.storage.am.common.impls.NoOpOperationCallbackFactory;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMMergePolicyFactory;
+import org.apache.hyracks.storage.am.lsm.common.dataflow.LSMTreeIndexCompactOperatorDescriptor;
+import org.apache.hyracks.storage.common.file.ILocalResourceFactoryProvider;
+import org.apache.hyracks.storage.common.file.LocalResource;
+
+public class DatasetUtil {
+    private static final Logger LOGGER = Logger.getLogger(DatasetUtil.class.getName());
+    /*
+     * Dataset related operations
+     */
+    public static final byte OP_READ = 0x00;
+    public static final byte OP_INSERT = 0x01;
+    public static final byte OP_DELETE = 0x02;
+    public static final byte OP_UPSERT = 0x03;
+
+    private DatasetUtil() {
+    }
+
+    public static IBinaryComparatorFactory[] computeKeysBinaryComparatorFactories(Dataset dataset,
+            ARecordType itemType, ARecordType metaItemType, IBinaryComparatorFactoryProvider comparatorFactoryProvider)
+            throws AlgebricksException {
+        List<List<String>> partitioningKeys = getPartitioningKeys(dataset);
+        IBinaryComparatorFactory[] bcfs = new IBinaryComparatorFactory[partitioningKeys.size()];
+        if (dataset.getDatasetType() == DatasetType.EXTERNAL) {
+            // Get comparators for RID fields.
+            for (int i = 0; i < partitioningKeys.size(); i++) {
+                try {
+                    bcfs[i] = IndexingConstants.getComparatorFactory(i);
+                } catch (AsterixException e) {
+                    throw new AlgebricksException(e);
+                }
+            }
+        } else {
+            InternalDatasetDetails dsd = (InternalDatasetDetails) dataset.getDatasetDetails();
+            for (int i = 0; i < partitioningKeys.size(); i++) {
+                IAType keyType = (dataset.hasMetaPart() && dsd.getKeySourceIndicator().get(i).intValue() == 1)
+                        ? metaItemType.getSubFieldType(partitioningKeys.get(i))
+                        : itemType.getSubFieldType(partitioningKeys.get(i));
+                bcfs[i] = comparatorFactoryProvider.getBinaryComparatorFactory(keyType, true);
+            }
+        }
+        return bcfs;
+    }
+
+    public static int[] createBloomFilterKeyFields(Dataset dataset) throws AlgebricksException {
+        if (dataset.getDatasetType() == DatasetType.EXTERNAL) {
+            throw new AlgebricksException("not implemented");
+        }
+        List<List<String>> partitioningKeys = getPartitioningKeys(dataset);
+        int[] bloomFilterKeyFields = new int[partitioningKeys.size()];
+        for (int i = 0; i < partitioningKeys.size(); ++i) {
+            bloomFilterKeyFields[i] = i;
+        }
+        return bloomFilterKeyFields;
+    }
+
+    public static IBinaryHashFunctionFactory[] computeKeysBinaryHashFunFactories(Dataset dataset, ARecordType itemType,
+            IBinaryHashFunctionFactoryProvider hashFunProvider) throws AlgebricksException {
+        if (dataset.getDatasetType() == DatasetType.EXTERNAL) {
+            throw new AlgebricksException("not implemented");
+        }
+        List<List<String>> partitioningKeys = getPartitioningKeys(dataset);
+        IBinaryHashFunctionFactory[] bhffs = new IBinaryHashFunctionFactory[partitioningKeys.size()];
+        for (int i = 0; i < partitioningKeys.size(); i++) {
+            IAType keyType = itemType.getSubFieldType(partitioningKeys.get(i));
+            bhffs[i] = hashFunProvider.getBinaryHashFunctionFactory(keyType);
+        }
+        return bhffs;
+    }
+
+    public static ITypeTraits[] computeTupleTypeTraits(Dataset dataset, ARecordType itemType, ARecordType metaItemType)
+            throws AlgebricksException {
+        if (dataset.getDatasetType() == DatasetType.EXTERNAL) {
+            throw new AlgebricksException("not implemented");
+        }
+        List<List<String>> partitioningKeys = DatasetUtil.getPartitioningKeys(dataset);
+        int numKeys = partitioningKeys.size();
+        ITypeTraits[] typeTraits;
+        if (metaItemType != null) {
+            typeTraits = new ITypeTraits[numKeys + 2];
+            List<Integer> indicator = ((InternalDatasetDetails) dataset.getDatasetDetails()).getKeySourceIndicator();
+            typeTraits[numKeys + 1] = TypeTraitProvider.INSTANCE.getTypeTrait(metaItemType);
+            for (int i = 0; i < numKeys; i++) {
+                IAType keyType;
+                if (indicator.get(i) == 0) {
+                    keyType = itemType.getSubFieldType(partitioningKeys.get(i));
+                } else {
+                    keyType = metaItemType.getSubFieldType(partitioningKeys.get(i));
+                }
+                typeTraits[i] = TypeTraitProvider.INSTANCE.getTypeTrait(keyType);
+            }
+        } else {
+            typeTraits = new ITypeTraits[numKeys + 1];
+            for (int i = 0; i < numKeys; i++) {
+                IAType keyType;
+                keyType = itemType.getSubFieldType(partitioningKeys.get(i));
+                typeTraits[i] = TypeTraitProvider.INSTANCE.getTypeTrait(keyType);
+            }
+        }
+        typeTraits[numKeys] = TypeTraitProvider.INSTANCE.getTypeTrait(itemType);
+        return typeTraits;
+    }
+
+    public static List<List<String>> getPartitioningKeys(Dataset dataset) {
+        if (dataset.getDatasetType() == DatasetType.EXTERNAL) {
+            return IndexingConstants
+                    .getRIDKeys(((ExternalDatasetDetails) dataset.getDatasetDetails()).getProperties());
+        }
+        return ((InternalDatasetDetails) dataset.getDatasetDetails()).getPartitioningKey();
+    }
+
+    public static List<String> getFilterField(Dataset dataset) {
+        return ((InternalDatasetDetails) dataset.getDatasetDetails()).getFilterField();
+    }
+
+    public static IBinaryComparatorFactory[] computeFilterBinaryComparatorFactories(Dataset dataset,
+            ARecordType itemType, IBinaryComparatorFactoryProvider comparatorFactoryProvider)
+            throws AlgebricksException {
+        if (dataset.getDatasetType() == DatasetType.EXTERNAL) {
+            return null;
+        }
+        List<String> filterField = getFilterField(dataset);
+        if (filterField == null) {
+            return null;
+        }
+        IBinaryComparatorFactory[] bcfs = new IBinaryComparatorFactory[1];
+        IAType type = itemType.getSubFieldType(filterField);
+        bcfs[0] = comparatorFactoryProvider.getBinaryComparatorFactory(type, true);
+        return bcfs;
+    }
+
+    public static ITypeTraits[] computeFilterTypeTraits(Dataset dataset, ARecordType itemType)
+            throws AlgebricksException {
+        if (dataset.getDatasetType() == DatasetType.EXTERNAL) {
+            return null;
+        }
+        List<String> filterField = getFilterField(dataset);
+        if (filterField == null) {
+            return null;
+        }
+        ITypeTraits[] typeTraits = new ITypeTraits[1];
+        IAType type = itemType.getSubFieldType(filterField);
+        typeTraits[0] = TypeTraitProvider.INSTANCE.getTypeTrait(type);
+        return typeTraits;
+    }
+
+    public static int[] createFilterFields(Dataset dataset) throws AlgebricksException {
+        if (dataset.getDatasetType() == DatasetType.EXTERNAL) {
+            return null;
+        }
+
+        List<String> filterField = getFilterField(dataset);
+        if (filterField == null) {
+            return null;
+        }
+        List<List<String>> partitioningKeys = DatasetUtil.getPartitioningKeys(dataset);
+        int numKeys = partitioningKeys.size();
+
+        int[] filterFields = new int[1];
+        filterFields[0] = numKeys + 1;
+        return filterFields;
+    }
+
+    public static int[] createBTreeFieldsWhenThereisAFilter(Dataset dataset) throws AlgebricksException {
+        if (dataset.getDatasetType() == DatasetType.EXTERNAL) {
+            return null;
+        }
+
+        List<String> filterField = getFilterField(dataset);
+        if (filterField == null) {
+            return null;
+        }
+
+        List<List<String>> partitioningKeys = getPartitioningKeys(dataset);
+        int valueFields = dataset.hasMetaPart() ? 2 : 1;
+        int[] btreeFields = new int[partitioningKeys.size() + valueFields];
+        for (int i = 0; i < btreeFields.length; ++i) {
+            btreeFields[i] = i;
+        }
+        return btreeFields;
+    }
+
+    public static int getPositionOfPartitioningKeyField(Dataset dataset, String fieldExpr) {
+        List<List<String>> partitioningKeys = DatasetUtil.getPartitioningKeys(dataset);
+        for (int i = 0; i < partitioningKeys.size(); i++) {
+            if ((partitioningKeys.get(i).size() == 1) && partitioningKeys.get(i).get(0).equals(fieldExpr)) {
+                return i;
+            }
+        }
+        return -1;
+    }
+
+    public static Pair<ILSMMergePolicyFactory, Map<String, String>> getMergePolicyFactory(Dataset dataset,
+            MetadataTransactionContext mdTxnCtx) throws AlgebricksException, MetadataException {
+        String policyName = dataset.getCompactionPolicy();
+        CompactionPolicy compactionPolicy = MetadataManager.INSTANCE.getCompactionPolicy(mdTxnCtx,
+                MetadataConstants.METADATA_DATAVERSE_NAME, policyName);
+        String compactionPolicyFactoryClassName = compactionPolicy.getClassName();
+        ILSMMergePolicyFactory mergePolicyFactory;
+        try {
+            mergePolicyFactory =
+                    (ILSMMergePolicyFactory) Class.forName(compactionPolicyFactoryClassName).newInstance();
+            if (mergePolicyFactory.getName().compareTo("correlated-prefix") == 0) {
+                ((CorrelatedPrefixMergePolicyFactory) mergePolicyFactory).setDatasetID(dataset.getDatasetId());
+            }
+        } catch (InstantiationException | IllegalAccessException | ClassNotFoundException e) {
+            throw new AlgebricksException(e);
+        }
+        Map<String, String> properties = dataset.getCompactionPolicyProperties();
+        return new Pair<>(mergePolicyFactory, properties);
+    }
+
+    @SuppressWarnings("unchecked")
+    public static void writePropertyTypeRecord(String name, String value, DataOutput out, ARecordType recordType)
+            throws HyracksDataException {
+        IARecordBuilder propertyRecordBuilder = new RecordBuilder();
+        ArrayBackedValueStorage fieldValue = new ArrayBackedValueStorage();
+        propertyRecordBuilder.reset(recordType);
+        AMutableString aString = new AMutableString("");
+        ISerializerDeserializer<AString> stringSerde =
+                SerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.ASTRING);
+
+        // write field 0
+        fieldValue.reset();
+        aString.setValue(name);
+        stringSerde.serialize(aString, fieldValue.getDataOutput());
+        propertyRecordBuilder.addField(0, fieldValue);
+
+        // write field 1
+        fieldValue.reset();
+        aString.setValue(value);
+        stringSerde.serialize(aString, fieldValue.getDataOutput());
+        propertyRecordBuilder.addField(1, fieldValue);
+
+        propertyRecordBuilder.write(out, true);
+    }
+
+    public static ARecordType getMetaType(MetadataProvider metadataProvider, Dataset dataset)
+            throws AlgebricksException {
+        if (dataset.hasMetaPart()) {
+            return (ARecordType) metadataProvider.findType(dataset.getMetaItemTypeDataverseName(),
+                    dataset.getMetaItemTypeName());
+        }
+        return null;
+    }
+
+    public static JobSpecification createDropDatasetJobSpec(Dataset dataset, Index primaryIndex,
+            MetadataProvider metadataProvider)
+            throws AlgebricksException, HyracksDataException, RemoteException, ACIDException {
+        String datasetPath = dataset.getDataverseName() + File.separator + dataset.getDatasetName();
+        LOGGER.info("DROP DATASETPATH: " + datasetPath);
+        if (dataset.getDatasetType() == DatasetType.EXTERNAL) {
+            return RuntimeUtils.createJobSpecification();
+        }
+        boolean temp = dataset.getDatasetDetails().isTemp();
+        ARecordType itemType =
+                (ARecordType) metadataProvider.findType(dataset.getItemTypeDataverseName(), dataset.getItemTypeName());
+        ARecordType metaType = DatasetUtil.getMetaType(metadataProvider, dataset);
+        JobSpecification specPrimary = RuntimeUtils.createJobSpecification();
+        Pair<IFileSplitProvider, AlgebricksPartitionConstraint> splitsAndConstraint =
+                metadataProvider.getSplitProviderAndConstraints(dataset.getDataverseName(), dataset.getDatasetName(),
+                        dataset.getDatasetName(), temp);
+        Pair<ILSMMergePolicyFactory, Map<String, String>> compactionInfo =
+                DatasetUtil.getMergePolicyFactory(dataset, metadataProvider.getMetadataTxnContext());
+        IIndexDataflowHelperFactory indexDataflowHelperFactory = dataset.getIndexDataflowHelperFactory(
+                metadataProvider, primaryIndex, itemType, metaType, compactionInfo.first, compactionInfo.second);
+        IStorageComponentProvider storageComponentProvider = metadataProvider.getStorageComponentProvider();
+        IndexDropOperatorDescriptor primaryBtreeDrop =
+                new IndexDropOperatorDescriptor(specPrimary, storageComponentProvider.getStorageManager(),
+                        storageComponentProvider.getIndexLifecycleManagerProvider(), splitsAndConstraint.first,
+                        indexDataflowHelperFactory, storageComponentProvider.getMetadataPageManagerFactory());
+        AlgebricksPartitionConstraintHelper.setPartitionConstraintInJobSpec(specPrimary, primaryBtreeDrop,
+                splitsAndConstraint.second);
+        specPrimary.addRoot(primaryBtreeDrop);
+        return specPrimary;
+    }
+
+    public static JobSpecification buildDropFilesIndexJobSpec(MetadataProvider metadataProvider, Dataset dataset)
+            throws AlgebricksException {
+        String indexName = IndexingConstants.getFilesIndexName(dataset.getDatasetName());
+        JobSpecification spec = RuntimeUtils.createJobSpecification();
+        IStorageComponentProvider storageComponentProvider = metadataProvider.getStorageComponentProvider();
+        Pair<IFileSplitProvider, AlgebricksPartitionConstraint> splitsAndConstraint =
+                metadataProvider.splitProviderAndPartitionConstraintsForFilesIndex(dataset.getDataverseName(),
+                        dataset.getDatasetName(), indexName, true);
+        Pair<ILSMMergePolicyFactory, Map<String, String>> compactionInfo =
+                DatasetUtil.getMergePolicyFactory(dataset, metadataProvider.getMetadataTxnContext());
+        String fileIndexName = BTreeDataflowHelperFactoryProvider.externalFileIndexName(dataset);
+        Index fileIndex = MetadataManager.INSTANCE.getIndex(metadataProvider.getMetadataTxnContext(),
+                dataset.getDataverseName(), dataset.getDatasetName(), fileIndexName);
+        IIndexDataflowHelperFactory dataflowHelperFactory = dataset.getIndexDataflowHelperFactory(metadataProvider,
+                fileIndex, null, null, compactionInfo.first, compactionInfo.second);
+        IndexDropOperatorDescriptor btreeDrop =
+                new IndexDropOperatorDescriptor(spec, storageComponentProvider.getStorageManager(),
+                        storageComponentProvider.getIndexLifecycleManagerProvider(), splitsAndConstraint.first,
+                        dataflowHelperFactory, storageComponentProvider.getMetadataPageManagerFactory());
+        AlgebricksPartitionConstraintHelper.setPartitionConstraintInJobSpec(spec, btreeDrop,
+                splitsAndConstraint.second);
+        spec.addRoot(btreeDrop);
+        return spec;
+    }
+
+    public static JobSpecification dropDatasetJobSpec(Dataset dataset, Index primaryIndex,
+            MetadataProvider metadataProvider)
+            throws AlgebricksException, HyracksDataException, RemoteException, ACIDException {
+        String datasetPath = dataset.getDataverseName() + File.separator + dataset.getDatasetName();
+        LOGGER.info("DROP DATASETPATH: " + datasetPath);
+        if (dataset.getDatasetType() == DatasetType.EXTERNAL) {
+            return RuntimeUtils.createJobSpecification();
+        }
+
+        boolean temp = dataset.getDatasetDetails().isTemp();
+        ARecordType itemType =
+                (ARecordType) metadataProvider.findType(dataset.getItemTypeDataverseName(), dataset.getItemTypeName());
+        ARecordType metaType = DatasetUtil.getMetaType(metadataProvider, dataset);
+        JobSpecification specPrimary = RuntimeUtils.createJobSpecification();
+        Pair<IFileSplitProvider, AlgebricksPartitionConstraint> splitsAndConstraint =
+                metadataProvider.getSplitProviderAndConstraints(dataset.getDataverseName(), dataset.getDatasetName(),
+                        dataset.getDatasetName(), temp);
+        Pair<ILSMMergePolicyFactory, Map<String, String>> compactionInfo =
+                DatasetUtil.getMergePolicyFactory(dataset, metadataProvider.getMetadataTxnContext());
+
+        IIndexDataflowHelperFactory indexDataflowHelperFactory = dataset.getIndexDataflowHelperFactory(
+                metadataProvider, primaryIndex, itemType, metaType, compactionInfo.first, compactionInfo.second);
+        IStorageComponentProvider storageComponentProvider = metadataProvider.getStorageComponentProvider();
+        IndexDropOperatorDescriptor primaryBtreeDrop =
+                new IndexDropOperatorDescriptor(specPrimary, storageComponentProvider.getStorageManager(),
+                        storageComponentProvider.getIndexLifecycleManagerProvider(), splitsAndConstraint.first,
+                        indexDataflowHelperFactory, storageComponentProvider.getMetadataPageManagerFactory());
+        AlgebricksPartitionConstraintHelper.setPartitionConstraintInJobSpec(specPrimary, primaryBtreeDrop,
+                splitsAndConstraint.second);
+
+        specPrimary.addRoot(primaryBtreeDrop);
+
+        return specPrimary;
+    }
+
+    public static JobSpecification createDatasetJobSpec(Dataverse dataverse, String datasetName,
+            MetadataProvider metadataProvider) throws AsterixException, AlgebricksException {
+        IStorageComponentProvider storageComponentProvider = metadataProvider.getStorageComponentProvider();
+        String dataverseName = dataverse.getDataverseName();
+        IDataFormat format;
+        try {
+            format = (IDataFormat) Class.forName(dataverse.getDataFormat()).newInstance();
+        } catch (Exception e) {
+            throw new AsterixException(e);
+        }
+        Dataset dataset = metadataProvider.findDataset(dataverseName, datasetName);
+        if (dataset == null) {
+            throw new AsterixException("Could not find dataset " + datasetName + " in dataverse " + dataverseName);
+        }
+        Index index = MetadataManager.INSTANCE.getIndex(metadataProvider.getMetadataTxnContext(), dataverseName,
+                datasetName, datasetName);
+        boolean temp = dataset.getDatasetDetails().isTemp();
+        ARecordType itemType =
+                (ARecordType) metadataProvider.findType(dataset.getItemTypeDataverseName(), dataset.getItemTypeName());
+        // get meta item type
+        ARecordType metaItemType = null;
+        if (dataset.hasMetaPart()) {
+            metaItemType = (ARecordType) metadataProvider.findType(dataset.getMetaItemTypeDataverseName(),
+                    dataset.getMetaItemTypeName());
+        }
+        JobSpecification spec = RuntimeUtils.createJobSpecification();
+        IBinaryComparatorFactory[] comparatorFactories = DatasetUtil.computeKeysBinaryComparatorFactories(dataset,
+                itemType, metaItemType, format.getBinaryComparatorFactoryProvider());
+        ITypeTraits[] typeTraits = DatasetUtil.computeTupleTypeTraits(dataset, itemType, metaItemType);
+        int[] bloomFilterKeyFields = DatasetUtil.createBloomFilterKeyFields(dataset);
+
+        ITypeTraits[] filterTypeTraits = DatasetUtil.computeFilterTypeTraits(dataset, itemType);
+        IBinaryComparatorFactory[] filterCmpFactories = DatasetUtil.computeFilterBinaryComparatorFactories(dataset,
+                itemType, format.getBinaryComparatorFactoryProvider());
+        int[] filterFields = DatasetUtil.createFilterFields(dataset);
+        int[] btreeFields = DatasetUtil.createBTreeFieldsWhenThereisAFilter(dataset);
+
+        Pair<IFileSplitProvider, AlgebricksPartitionConstraint> splitsAndConstraint =
+                metadataProvider.getSplitProviderAndConstraints(dataverseName, datasetName, datasetName, temp);
+        FileSplit[] fs = splitsAndConstraint.first.getFileSplits();
+        StringBuilder sb = new StringBuilder();
+        for (int i = 0; i < fs.length; i++) {
+            sb.append(fs[i] + " ");
+        }
+        LOGGER.info("CREATING File Splits: " + sb.toString());
+
+        Pair<ILSMMergePolicyFactory, Map<String, String>> compactionInfo =
+                DatasetUtil.getMergePolicyFactory(dataset, metadataProvider.getMetadataTxnContext());
+        //prepare a LocalResourceMetadata which will be stored in NC's local resource repository
+        IResourceFactory localResourceMetadata = new LSMBTreeLocalResourceMetadataFactory(typeTraits,
+                comparatorFactories, bloomFilterKeyFields, true, dataset.getDatasetId(), compactionInfo.first,
+                compactionInfo.second, filterTypeTraits, filterCmpFactories, btreeFields, filterFields,
+                dataset.getIndexOperationTrackerFactory(index), dataset.getIoOperationCallbackFactory(index),
+                storageComponentProvider.getMetadataPageManagerFactory());
+        ILocalResourceFactoryProvider localResourceFactoryProvider =
+                new PersistentLocalResourceFactoryProvider(localResourceMetadata, LocalResource.LSMBTreeResource);
+        IIndexDataflowHelperFactory dataflowHelperFactory = dataset.getIndexDataflowHelperFactory(metadataProvider,
+                index, itemType, metaItemType, compactionInfo.first, compactionInfo.second);
+        TreeIndexCreateOperatorDescriptor indexCreateOp = new TreeIndexCreateOperatorDescriptor(spec,
+                RuntimeComponentsProvider.RUNTIME_PROVIDER, RuntimeComponentsProvider.RUNTIME_PROVIDER,
+                splitsAndConstraint.first, typeTraits, comparatorFactories, bloomFilterKeyFields,
+                dataflowHelperFactory, localResourceFactoryProvider, NoOpOperationCallbackFactory.INSTANCE,
+                storageComponentProvider.getMetadataPageManagerFactory());
+        AlgebricksPartitionConstraintHelper.setPartitionConstraintInJobSpec(spec, indexCreateOp,
+                splitsAndConstraint.second);
+        spec.addRoot(indexCreateOp);
+        return spec;
+    }
+
+    public static JobSpecification compactDatasetJobSpec(Dataverse dataverse, String datasetName,
+            MetadataProvider metadataProvider) throws AsterixException, AlgebricksException {
+        String dataverseName = dataverse.getDataverseName();
+        IDataFormat format;
+        try {
+            format = (IDataFormat) Class.forName(dataverse.getDataFormat()).newInstance();
+        } catch (Exception e) {
+            throw new AsterixException(e);
+        }
+        Dataset dataset = metadataProvider.findDataset(dataverseName, datasetName);
+        if (dataset == null) {
+            throw new AsterixException("Could not find dataset " + datasetName + " in dataverse " + dataverseName);
+        }
+        boolean temp = dataset.getDatasetDetails().isTemp();
+        ARecordType itemType =
+                (ARecordType) metadataProvider.findType(dataset.getItemTypeDataverseName(), dataset.getItemTypeName());
+        ARecordType metaItemType = DatasetUtil.getMetaType(metadataProvider, dataset);
+        JobSpecification spec = RuntimeUtils.createJobSpecification();
+        IBinaryComparatorFactory[] comparatorFactories = DatasetUtil.computeKeysBinaryComparatorFactories(dataset,
+                itemType, metaItemType, format.getBinaryComparatorFactoryProvider());
+        ITypeTraits[] typeTraits = DatasetUtil.computeTupleTypeTraits(dataset, itemType, metaItemType);
+        int[] blooFilterKeyFields = DatasetUtil.createBloomFilterKeyFields(dataset);
+        Pair<IFileSplitProvider, AlgebricksPartitionConstraint> splitsAndConstraint =
+                metadataProvider.getSplitProviderAndConstraints(dataverseName, datasetName, datasetName, temp);
+        Pair<ILSMMergePolicyFactory, Map<String, String>> compactionInfo =
+                DatasetUtil.getMergePolicyFactory(dataset, metadataProvider.getMetadataTxnContext());
+        Index index = MetadataManager.INSTANCE.getIndex(metadataProvider.getMetadataTxnContext(),
+                dataset.getDataverseName(), datasetName, datasetName);
+        IIndexDataflowHelperFactory dataflowHelperFactory = dataset.getIndexDataflowHelperFactory(metadataProvider,
+                index, itemType, metaItemType, compactionInfo.first, compactionInfo.second);
+        LSMTreeIndexCompactOperatorDescriptor compactOp = new LSMTreeIndexCompactOperatorDescriptor(spec,
+                RuntimeComponentsProvider.RUNTIME_PROVIDER, RuntimeComponentsProvider.RUNTIME_PROVIDER,
+                splitsAndConstraint.first, typeTraits, comparatorFactories, blooFilterKeyFields, dataflowHelperFactory,
+                NoOpOperationCallbackFactory.INSTANCE,
+                metadataProvider.getStorageComponentProvider().getMetadataPageManagerFactory());
+        AlgebricksPartitionConstraintHelper.setPartitionConstraintInJobSpec(spec, compactOp,
+                splitsAndConstraint.second);
+
+        AlgebricksPartitionConstraintHelper.setPartitionConstraintInJobSpec(spec, compactOp,
+                splitsAndConstraint.second);
+        spec.addRoot(compactOp);
+        return spec;
+    }
+}
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/DatasetUtils.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/DatasetUtils.java
deleted file mode 100644
index 18baab1..0000000
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/DatasetUtils.java
+++ /dev/null
@@ -1,285 +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.DataOutput;
-import java.util.List;
-import java.util.Map;
-
-import org.apache.asterix.builders.IARecordBuilder;
-import org.apache.asterix.builders.RecordBuilder;
-import org.apache.asterix.common.config.DatasetConfig.DatasetType;
-import org.apache.asterix.common.context.CorrelatedPrefixMergePolicyFactory;
-import org.apache.asterix.common.exceptions.AsterixException;
-import org.apache.asterix.external.indexing.IndexingConstants;
-import org.apache.asterix.formats.nontagged.SerializerDeserializerProvider;
-import org.apache.asterix.formats.nontagged.TypeTraitProvider;
-import org.apache.asterix.metadata.MetadataException;
-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.CompactionPolicy;
-import org.apache.asterix.metadata.entities.Dataset;
-import org.apache.asterix.metadata.entities.ExternalDatasetDetails;
-import org.apache.asterix.metadata.entities.InternalDatasetDetails;
-import org.apache.asterix.om.base.AMutableString;
-import org.apache.asterix.om.base.AString;
-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.algebricks.common.exceptions.AlgebricksException;
-import org.apache.hyracks.algebricks.common.utils.Pair;
-import org.apache.hyracks.algebricks.data.IBinaryComparatorFactoryProvider;
-import org.apache.hyracks.algebricks.data.IBinaryHashFunctionFactoryProvider;
-import org.apache.hyracks.api.dataflow.value.IBinaryComparatorFactory;
-import org.apache.hyracks.api.dataflow.value.IBinaryHashFunctionFactory;
-import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
-import org.apache.hyracks.api.dataflow.value.ITypeTraits;
-import org.apache.hyracks.api.exceptions.HyracksDataException;
-import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMMergePolicyFactory;
-
-public class DatasetUtils {
-    public static IBinaryComparatorFactory[] computeKeysBinaryComparatorFactories(Dataset dataset, ARecordType itemType,
-            ARecordType metaItemType, IBinaryComparatorFactoryProvider comparatorFactoryProvider)
-            throws AlgebricksException {
-        List<List<String>> partitioningKeys = getPartitioningKeys(dataset);
-        IBinaryComparatorFactory[] bcfs = new IBinaryComparatorFactory[partitioningKeys.size()];
-        if (dataset.getDatasetType() == DatasetType.EXTERNAL) {
-            // Get comparators for RID fields.
-            for (int i = 0; i < partitioningKeys.size(); i++) {
-                try {
-                    bcfs[i] = IndexingConstants.getComparatorFactory(i);
-                } catch (AsterixException e) {
-                    throw new AlgebricksException(e);
-                }
-            }
-        } else {
-            InternalDatasetDetails dsd = (InternalDatasetDetails) dataset.getDatasetDetails();
-            for (int i = 0; i < partitioningKeys.size(); i++) {
-                IAType keyType = (dataset.hasMetaPart() && dsd.getKeySourceIndicator().get(i).intValue() == 1)
-                        ? metaItemType.getSubFieldType(partitioningKeys.get(i))
-                        : itemType.getSubFieldType(partitioningKeys.get(i));
-                bcfs[i] = comparatorFactoryProvider.getBinaryComparatorFactory(keyType, true);
-            }
-        }
-        return bcfs;
-    }
-
-    public static int[] createBloomFilterKeyFields(Dataset dataset) throws AlgebricksException {
-        if (dataset.getDatasetType() == DatasetType.EXTERNAL) {
-            throw new AlgebricksException("not implemented");
-        }
-        List<List<String>> partitioningKeys = getPartitioningKeys(dataset);
-        int[] bloomFilterKeyFields = new int[partitioningKeys.size()];
-        for (int i = 0; i < partitioningKeys.size(); ++i) {
-            bloomFilterKeyFields[i] = i;
-        }
-        return bloomFilterKeyFields;
-    }
-
-    public static IBinaryHashFunctionFactory[] computeKeysBinaryHashFunFactories(Dataset dataset, ARecordType itemType,
-            IBinaryHashFunctionFactoryProvider hashFunProvider) throws AlgebricksException {
-        if (dataset.getDatasetType() == DatasetType.EXTERNAL) {
-            throw new AlgebricksException("not implemented");
-        }
-        List<List<String>> partitioningKeys = getPartitioningKeys(dataset);
-        IBinaryHashFunctionFactory[] bhffs = new IBinaryHashFunctionFactory[partitioningKeys.size()];
-        for (int i = 0; i < partitioningKeys.size(); i++) {
-            IAType keyType = itemType.getSubFieldType(partitioningKeys.get(i));
-            bhffs[i] = hashFunProvider.getBinaryHashFunctionFactory(keyType);
-        }
-        return bhffs;
-    }
-
-    public static ITypeTraits[] computeTupleTypeTraits(Dataset dataset, ARecordType itemType, ARecordType metaItemType)
-            throws AlgebricksException {
-        if (dataset.getDatasetType() == DatasetType.EXTERNAL) {
-            throw new AlgebricksException("not implemented");
-        }
-        List<List<String>> partitioningKeys = DatasetUtils.getPartitioningKeys(dataset);
-        int numKeys = partitioningKeys.size();
-        ITypeTraits[] typeTraits;
-        if (metaItemType != null) {
-            typeTraits = new ITypeTraits[numKeys + 2];
-            List<Integer> indicator = ((InternalDatasetDetails) dataset.getDatasetDetails()).getKeySourceIndicator();
-            typeTraits[numKeys + 1] = TypeTraitProvider.INSTANCE.getTypeTrait(metaItemType);
-            for (int i = 0; i < numKeys; i++) {
-                IAType keyType;
-                if (indicator.get(i) == 0) {
-                    keyType = itemType.getSubFieldType(partitioningKeys.get(i));
-                } else {
-                    keyType = metaItemType.getSubFieldType(partitioningKeys.get(i));
-                }
-                typeTraits[i] = TypeTraitProvider.INSTANCE.getTypeTrait(keyType);
-            }
-        } else {
-            typeTraits = new ITypeTraits[numKeys + 1];
-            for (int i = 0; i < numKeys; i++) {
-                IAType keyType;
-                keyType = itemType.getSubFieldType(partitioningKeys.get(i));
-                typeTraits[i] = TypeTraitProvider.INSTANCE.getTypeTrait(keyType);
-            }
-        }
-        typeTraits[numKeys] = TypeTraitProvider.INSTANCE.getTypeTrait(itemType);
-        return typeTraits;
-    }
-
-    public static List<List<String>> getPartitioningKeys(Dataset dataset) {
-        if (dataset.getDatasetType() == DatasetType.EXTERNAL) {
-            return IndexingConstants.getRIDKeys(((ExternalDatasetDetails) dataset.getDatasetDetails()).getProperties());
-        }
-        return ((InternalDatasetDetails) dataset.getDatasetDetails()).getPartitioningKey();
-    }
-
-    public static List<String> getFilterField(Dataset dataset) {
-        return (((InternalDatasetDetails) dataset.getDatasetDetails())).getFilterField();
-    }
-
-    public static IBinaryComparatorFactory[] computeFilterBinaryComparatorFactories(Dataset dataset,
-            ARecordType itemType, IBinaryComparatorFactoryProvider comparatorFactoryProvider)
-            throws AlgebricksException {
-        if (dataset.getDatasetType() == DatasetType.EXTERNAL) {
-            return null;
-        }
-        List<String> filterField = getFilterField(dataset);
-        if (filterField == null) {
-            return null;
-        }
-        IBinaryComparatorFactory[] bcfs = new IBinaryComparatorFactory[1];
-        IAType type = itemType.getSubFieldType(filterField);
-        bcfs[0] = comparatorFactoryProvider.getBinaryComparatorFactory(type, true);
-        return bcfs;
-    }
-
-    public static ITypeTraits[] computeFilterTypeTraits(Dataset dataset, ARecordType itemType)
-            throws AlgebricksException {
-        if (dataset.getDatasetType() == DatasetType.EXTERNAL) {
-            return null;
-        }
-        List<String> filterField = getFilterField(dataset);
-        if (filterField == null) {
-            return null;
-        }
-        ITypeTraits[] typeTraits = new ITypeTraits[1];
-        IAType type = itemType.getSubFieldType(filterField);
-        typeTraits[0] = TypeTraitProvider.INSTANCE.getTypeTrait(type);
-        return typeTraits;
-    }
-
-    public static int[] createFilterFields(Dataset dataset) throws AlgebricksException {
-        if (dataset.getDatasetType() == DatasetType.EXTERNAL) {
-            return null;
-        }
-
-        List<String> filterField = getFilterField(dataset);
-        if (filterField == null) {
-            return null;
-        }
-        List<List<String>> partitioningKeys = DatasetUtils.getPartitioningKeys(dataset);
-        int numKeys = partitioningKeys.size();
-
-        int[] filterFields = new int[1];
-        filterFields[0] = numKeys + 1;
-        return filterFields;
-    }
-
-    public static int[] createBTreeFieldsWhenThereisAFilter(Dataset dataset) throws AlgebricksException {
-        if (dataset.getDatasetType() == DatasetType.EXTERNAL) {
-            return null;
-        }
-
-        List<String> filterField = getFilterField(dataset);
-        if (filterField == null) {
-            return null;
-        }
-
-        List<List<String>> partitioningKeys = getPartitioningKeys(dataset);
-        int valueFields = dataset.hasMetaPart() ? 2 : 1;
-        int[] btreeFields = new int[partitioningKeys.size() + valueFields];
-        for (int i = 0; i < btreeFields.length; ++i) {
-            btreeFields[i] = i;
-        }
-        return btreeFields;
-    }
-
-    public static int getPositionOfPartitioningKeyField(Dataset dataset, String fieldExpr) {
-        List<List<String>> partitioningKeys = DatasetUtils.getPartitioningKeys(dataset);
-        for (int i = 0; i < partitioningKeys.size(); i++) {
-            if ((partitioningKeys.get(i).size() == 1) && partitioningKeys.get(i).get(0).equals(fieldExpr)) {
-                return i;
-            }
-        }
-        return -1;
-    }
-
-    public static Pair<ILSMMergePolicyFactory, Map<String, String>> getMergePolicyFactory(Dataset dataset,
-            MetadataTransactionContext mdTxnCtx) throws AlgebricksException, MetadataException {
-        String policyName = dataset.getCompactionPolicy();
-        CompactionPolicy compactionPolicy = MetadataManager.INSTANCE.getCompactionPolicy(mdTxnCtx,
-                MetadataConstants.METADATA_DATAVERSE_NAME, policyName);
-        String compactionPolicyFactoryClassName = compactionPolicy.getClassName();
-        ILSMMergePolicyFactory mergePolicyFactory;
-        try {
-            mergePolicyFactory = (ILSMMergePolicyFactory) Class.forName(compactionPolicyFactoryClassName).newInstance();
-            if (mergePolicyFactory.getName().compareTo("correlated-prefix") == 0) {
-                ((CorrelatedPrefixMergePolicyFactory) mergePolicyFactory).setDatasetID(dataset.getDatasetId());
-            }
-        } catch (InstantiationException | IllegalAccessException | ClassNotFoundException e) {
-            throw new AlgebricksException(e);
-        }
-        Map<String, String> properties = dataset.getCompactionPolicyProperties();
-        return new Pair<ILSMMergePolicyFactory, Map<String, String>>(mergePolicyFactory, properties);
-    }
-
-    @SuppressWarnings("unchecked")
-    public static void writePropertyTypeRecord(String name, String value, DataOutput out, ARecordType recordType)
-            throws HyracksDataException {
-        IARecordBuilder propertyRecordBuilder = new RecordBuilder();
-        ArrayBackedValueStorage fieldValue = new ArrayBackedValueStorage();
-        propertyRecordBuilder.reset(recordType);
-        AMutableString aString = new AMutableString("");
-        ISerializerDeserializer<AString> stringSerde =
-                SerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.ASTRING);
-
-        // write field 0
-        fieldValue.reset();
-        aString.setValue(name);
-        stringSerde.serialize(aString, fieldValue.getDataOutput());
-        propertyRecordBuilder.addField(0, fieldValue);
-
-        // write field 1
-        fieldValue.reset();
-        aString.setValue(value);
-        stringSerde.serialize(aString, fieldValue.getDataOutput());
-        propertyRecordBuilder.addField(1, fieldValue);
-
-        propertyRecordBuilder.write(out, true);
-    }
-
-    public static ARecordType getMetaType(MetadataProvider metadataProvider, Dataset dataset)
-            throws AlgebricksException {
-        if (dataset.hasMetaPart()) {
-            return (ARecordType) metadataProvider.findType(dataset.getMetaItemTypeDataverseName(),
-                    dataset.getMetaItemTypeName());
-        }
-        return null;
-    }
-}
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/ExternalDatasetsRegistry.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/ExternalDatasetsRegistry.java
index 17a3ebe..2e35ed4 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/ExternalDatasetsRegistry.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/ExternalDatasetsRegistry.java
@@ -34,11 +34,11 @@
  * @author alamouda
  */
 public class ExternalDatasetsRegistry {
-    public static ExternalDatasetsRegistry INSTANCE = new ExternalDatasetsRegistry();
-    private ConcurrentHashMap<String, ExternalDatasetAccessManager> globalRegister;
+    public static final ExternalDatasetsRegistry INSTANCE = new ExternalDatasetsRegistry();
+    private final ConcurrentHashMap<String, ExternalDatasetAccessManager> globalRegister;
 
     private ExternalDatasetsRegistry() {
-        globalRegister = new ConcurrentHashMap<String, ExternalDatasetAccessManager>();
+        globalRegister = new ConcurrentHashMap<>();
     }
 
     /**
@@ -59,12 +59,12 @@
 
     public int getAndLockDatasetVersion(Dataset dataset, MetadataProvider metadataProvider) {
 
-        Map<String, Integer> locks = null;
+        Map<String, Integer> locks;
         String lockKey = dataset.getDataverseName() + "." + dataset.getDatasetName();
         // check first if the lock was aquired already
         locks = metadataProvider.getLocks();
         if (locks == null) {
-            locks = new HashMap<String, Integer>();
+            locks = new HashMap<>();
             metadataProvider.setLocks(locks);
         } else {
             // if dataset was accessed already by this job, return the registered version
@@ -130,7 +130,10 @@
             // if dataset was accessed already by this job, return the registered version
             Set<Entry<String, Integer>> aquiredLocks = locks.entrySet();
             for (Entry<String, Integer> entry : aquiredLocks) {
-                globalRegister.get(entry.getKey()).queryEnd(entry.getValue());
+                ExternalDatasetAccessManager accessManager = globalRegister.get(entry.getKey());
+                if (accessManager != null) {
+                    accessManager.queryEnd(entry.getValue());
+                }
             }
             locks.clear();
         }
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
new file mode 100644
index 0000000..249f035
--- /dev/null
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/ExternalIndexingOperations.java
@@ -0,0 +1,621 @@
+/*
+ * 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 java.util.logging.Level;
+import java.util.logging.Logger;
+
+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.transactions.IResourceFactory;
+import org.apache.asterix.external.api.IAdapterFactory;
+import org.apache.asterix.external.indexing.ExternalFile;
+import org.apache.asterix.external.indexing.FilesIndexDescription;
+import org.apache.asterix.external.indexing.IndexingConstants;
+import org.apache.asterix.external.operators.ExternalDatasetIndexesAbortOperatorDescriptor;
+import org.apache.asterix.external.operators.ExternalDatasetIndexesCommitOperatorDescriptor;
+import org.apache.asterix.external.operators.ExternalDatasetIndexesRecoverOperatorDescriptor;
+import org.apache.asterix.external.operators.ExternalFilesIndexOperatorDescriptor;
+import org.apache.asterix.external.operators.ExternalScanOperatorDescriptor;
+import org.apache.asterix.external.operators.IndexInfoOperatorDescriptor;
+import org.apache.asterix.external.provider.AdapterFactoryProvider;
+import org.apache.asterix.external.util.ExternalDataConstants;
+import org.apache.asterix.metadata.MetadataManager;
+import org.apache.asterix.metadata.declared.BTreeDataflowHelperFactoryProvider;
+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.RuntimeComponentsProvider;
+import org.apache.asterix.runtime.utils.RuntimeUtils;
+import org.apache.asterix.transaction.management.resource.ExternalBTreeLocalResourceMetadataFactory;
+import org.apache.asterix.transaction.management.resource.PersistentLocalResourceFactoryProvider;
+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.job.JobSpecification;
+import org.apache.hyracks.dataflow.std.file.IFileSplitProvider;
+import org.apache.hyracks.storage.am.common.dataflow.IIndexDataflowHelperFactory;
+import org.apache.hyracks.storage.am.common.dataflow.IndexDropOperatorDescriptor;
+import org.apache.hyracks.storage.am.common.impls.NoOpOperationCallbackFactory;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMMergePolicyFactory;
+import org.apache.hyracks.storage.am.lsm.common.dataflow.LSMTreeIndexCompactOperatorDescriptor;
+import org.apache.hyracks.storage.common.file.LocalResource;
+
+public class ExternalIndexingOperations {
+    private static final Logger LOGGER = Logger.getLogger(ExternalIndexingOperations.class.getName());
+    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.log(Level.WARNING, "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 buildFilesIndexReplicationJobSpec(Dataset dataset,
+            List<ExternalFile> externalFilesSnapshot, MetadataProvider metadataProvider, boolean createIndex)
+            throws AlgebricksException {
+        IStorageComponentProvider storageComponentProvider = metadataProvider.getStorageComponentProvider();
+        JobSpecification spec = RuntimeUtils.createJobSpecification();
+        Pair<ILSMMergePolicyFactory, Map<String, String>> compactionInfo =
+                DatasetUtil.getMergePolicyFactory(dataset, metadataProvider.getMetadataTxnContext());
+        ILSMMergePolicyFactory mergePolicyFactory = compactionInfo.first;
+        Map<String, String> mergePolicyFactoryProperties = compactionInfo.second;
+        Pair<IFileSplitProvider, AlgebricksPartitionConstraint> secondarySplitsAndConstraint =
+                metadataProvider.splitProviderAndPartitionConstraintsForFilesIndex(dataset.getDataverseName(),
+                        dataset.getDatasetName(), IndexingConstants.getFilesIndexName(dataset.getDatasetName()), true);
+        IFileSplitProvider secondaryFileSplitProvider = secondarySplitsAndConstraint.first;
+        FilesIndexDescription filesIndexDescription = new FilesIndexDescription();
+        String fileIndexName = BTreeDataflowHelperFactoryProvider.externalFileIndexName(dataset);
+        Index fileIndex = MetadataManager.INSTANCE.getIndex(metadataProvider.getMetadataTxnContext(),
+                dataset.getDataverseName(), dataset.getDatasetName(), fileIndexName);
+        IResourceFactory localResourceMetadata = new ExternalBTreeLocalResourceMetadataFactory(
+                filesIndexDescription.EXTERNAL_FILE_INDEX_TYPE_TRAITS,
+                FilesIndexDescription.FILES_INDEX_COMP_FACTORIES, new int[] { 0 }, false, dataset.getDatasetId(),
+                mergePolicyFactory, mergePolicyFactoryProperties, dataset.getIndexOperationTrackerFactory(fileIndex),
+                dataset.getIoOperationCallbackFactory(fileIndex),
+                storageComponentProvider.getMetadataPageManagerFactory());
+        PersistentLocalResourceFactoryProvider localResourceFactoryProvider =
+                new PersistentLocalResourceFactoryProvider(localResourceMetadata, LocalResource.ExternalBTreeResource);
+        IIndexDataflowHelperFactory dataflowHelperFactory = dataset.getIndexDataflowHelperFactory(metadataProvider,
+                fileIndex, null, null, mergePolicyFactory, mergePolicyFactoryProperties);
+        ExternalFilesIndexOperatorDescriptor externalFilesOp =
+                new ExternalFilesIndexOperatorDescriptor(spec, storageComponentProvider.getStorageManager(),
+                        storageComponentProvider.getIndexLifecycleManagerProvider(), secondaryFileSplitProvider,
+                        dataflowHelperFactory, localResourceFactoryProvider, externalFilesSnapshot, createIndex,
+                        storageComponentProvider.getMetadataPageManagerFactory());
+        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
+     * @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) throws HyracksDataException, AlgebricksException {
+        ExternalDatasetDetails externalDatasetDetails = (ExternalDatasetDetails) dataset.getDatasetDetails();
+        Map<String, String> configuration = externalDatasetDetails.getProperties();
+        IAdapterFactory adapterFactory =
+                AdapterFactoryProvider.getIndexingAdapterFactory(metadataProvider.getLibraryManager(),
+                        externalDatasetDetails.getAdapter(), configuration, (ARecordType) itemType, files, true, null);
+        return new Pair<>(new ExternalScanOperatorDescriptor(jobSpec, indexerDesc, adapterFactory),
+                adapterFactory.getPartitionConstraint());
+    }
+
+    public static Pair<ExternalScanOperatorDescriptor, AlgebricksPartitionConstraint> createExternalIndexingOp(
+            JobSpecification spec, MetadataProvider metadataProvider, Dataset dataset, ARecordType itemType,
+            RecordDescriptor indexerDesc, List<ExternalFile> files) throws HyracksDataException, AlgebricksException {
+        return getIndexingOperator(metadataProvider, spec, itemType, dataset,
+                files == null ? MetadataManager.INSTANCE
+                        .getDatasetExternalFiles(metadataProvider.getMetadataTxnContext(), dataset) : files,
+                indexerDesc);
+    }
+
+    /**
+     * 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();
+        Pair<IFileSplitProvider, AlgebricksPartitionConstraint> splitsAndConstraint =
+                metadataProvider.splitProviderAndPartitionConstraintsForFilesIndex(dataset.getDataverseName(),
+                        dataset.getDatasetName(), indexName, true);
+        Pair<ILSMMergePolicyFactory, Map<String, String>> compactionInfo =
+                DatasetUtil.getMergePolicyFactory(dataset, metadataProvider.getMetadataTxnContext());
+        String fileIndexName = BTreeDataflowHelperFactoryProvider.externalFileIndexName(dataset);
+        Index fileIndex = MetadataManager.INSTANCE.getIndex(metadataProvider.getMetadataTxnContext(),
+                dataset.getDataverseName(), dataset.getDatasetName(), fileIndexName);
+        IIndexDataflowHelperFactory dataflowHelperFactory = dataset.getIndexDataflowHelperFactory(metadataProvider,
+                fileIndex, null, null, compactionInfo.first, compactionInfo.second);
+        IndexDropOperatorDescriptor btreeDrop = new IndexDropOperatorDescriptor(spec,
+                metadataProvider.getStorageComponentProvider().getStorageManager(),
+                metadataProvider.getStorageComponentProvider().getIndexLifecycleManagerProvider(),
+                splitsAndConstraint.first, dataflowHelperFactory,
+                metadataProvider.getStorageComponentProvider().getMetadataPageManagerFactory());
+        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 buildFilesIndexReplicationJobSpec(ds, files, metadataProvider, false);
+    }
+
+    public static JobSpecification buildIndexUpdateOp(Dataset ds, Index index, List<ExternalFile> metadataFiles,
+            List<ExternalFile> addedFiles, List<ExternalFile> appendedFiles, MetadataProvider metadataProvider)
+            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, null, null, null, null, metadataProvider, files);
+    }
+
+    public static JobSpecification buildCommitJob(Dataset ds, List<Index> indexes, MetadataProvider metadataProvider)
+            throws AlgebricksException {
+        JobSpecification spec = RuntimeUtils.createJobSpecification();
+        Pair<ILSMMergePolicyFactory, Map<String, String>> compactionInfo =
+                DatasetUtil.getMergePolicyFactory(ds, metadataProvider.getMetadataTxnContext());
+        boolean temp = ds.getDatasetDetails().isTemp();
+        ILSMMergePolicyFactory mergePolicyFactory = compactionInfo.first;
+        Map<String, String> mergePolicyFactoryProperties = compactionInfo.second;
+        Pair<IFileSplitProvider, AlgebricksPartitionConstraint> filesIndexSplitsAndConstraint =
+                metadataProvider.getSplitProviderAndConstraints(ds.getDataverseName(), ds.getDatasetName(),
+                        IndexingConstants.getFilesIndexName(ds.getDatasetName()), temp);
+        IFileSplitProvider filesIndexSplitProvider = filesIndexSplitsAndConstraint.first;
+        String fileIndexName = BTreeDataflowHelperFactoryProvider.externalFileIndexName(ds);
+        Index fileIndex = MetadataManager.INSTANCE.getIndex(metadataProvider.getMetadataTxnContext(),
+                ds.getDataverseName(), ds.getDatasetName(), fileIndexName);
+        IIndexDataflowHelperFactory filesIndexDataflowHelperFactory = ds.getIndexDataflowHelperFactory(
+                metadataProvider, fileIndex, null, null, mergePolicyFactory, mergePolicyFactoryProperties);
+        IndexInfoOperatorDescriptor filesIndexInfo = new IndexInfoOperatorDescriptor(filesIndexSplitProvider,
+                RuntimeComponentsProvider.RUNTIME_PROVIDER, RuntimeComponentsProvider.RUNTIME_PROVIDER);
+
+        ArrayList<IIndexDataflowHelperFactory> treeDataflowHelperFactories = new ArrayList<>();
+        ArrayList<IndexInfoOperatorDescriptor> treeInfos = new ArrayList<>();
+        for (Index index : indexes) {
+            if (isValidIndexName(index.getDatasetName(), index.getIndexName())) {
+                Pair<IFileSplitProvider, AlgebricksPartitionConstraint> indexSplitsAndConstraint =
+                        metadataProvider.getSplitProviderAndConstraints(ds.getDataverseName(), ds.getDatasetName(),
+                                index.getIndexName(), temp);
+                IIndexDataflowHelperFactory indexDataflowHelperFactory = ds.getIndexDataflowHelperFactory(
+                        metadataProvider, index, null, null, mergePolicyFactory, mergePolicyFactoryProperties);
+                treeDataflowHelperFactories.add(indexDataflowHelperFactory);
+                treeInfos.add(new IndexInfoOperatorDescriptor(indexSplitsAndConstraint.first,
+                        RuntimeComponentsProvider.RUNTIME_PROVIDER, RuntimeComponentsProvider.RUNTIME_PROVIDER));
+            }
+        }
+
+        ExternalDatasetIndexesCommitOperatorDescriptor op = new ExternalDatasetIndexesCommitOperatorDescriptor(spec,
+                filesIndexDataflowHelperFactory, filesIndexInfo, treeDataflowHelperFactories, treeInfos);
+
+        spec.addRoot(op);
+        AlgebricksPartitionConstraintHelper.setPartitionConstraintInJobSpec(spec, op,
+                filesIndexSplitsAndConstraint.second);
+        spec.setConnectorPolicyAssignmentPolicy(new ConnectorPolicyAssignmentPolicy());
+        return spec;
+    }
+
+    public static JobSpecification buildAbortOp(Dataset ds, List<Index> indexes, MetadataProvider metadataProvider)
+            throws AlgebricksException {
+        JobSpecification spec = RuntimeUtils.createJobSpecification();
+        Pair<ILSMMergePolicyFactory, Map<String, String>> compactionInfo =
+                DatasetUtil.getMergePolicyFactory(ds, metadataProvider.getMetadataTxnContext());
+        ILSMMergePolicyFactory mergePolicyFactory = compactionInfo.first;
+        Map<String, String> mergePolicyFactoryProperties = compactionInfo.second;
+
+        boolean temp = ds.getDatasetDetails().isTemp();
+        Pair<IFileSplitProvider, AlgebricksPartitionConstraint> filesIndexSplitsAndConstraint =
+                metadataProvider.getSplitProviderAndConstraints(ds.getDataverseName(), ds.getDatasetName(),
+                        IndexingConstants.getFilesIndexName(ds.getDatasetName()), temp);
+        IFileSplitProvider filesIndexSplitProvider = filesIndexSplitsAndConstraint.first;
+        String fileIndexName = BTreeDataflowHelperFactoryProvider.externalFileIndexName(ds);
+        Index fileIndex = MetadataManager.INSTANCE.getIndex(metadataProvider.getMetadataTxnContext(),
+                ds.getDataverseName(), ds.getDatasetName(), fileIndexName);
+        IIndexDataflowHelperFactory filesIndexDataflowHelperFactory = ds.getIndexDataflowHelperFactory(
+                metadataProvider, fileIndex, null, null, mergePolicyFactory, mergePolicyFactoryProperties);
+        IndexInfoOperatorDescriptor filesIndexInfo = new IndexInfoOperatorDescriptor(filesIndexSplitProvider,
+                RuntimeComponentsProvider.RUNTIME_PROVIDER, RuntimeComponentsProvider.RUNTIME_PROVIDER);
+
+        ArrayList<IIndexDataflowHelperFactory> treeDataflowHelperFactories = new ArrayList<>();
+        ArrayList<IndexInfoOperatorDescriptor> treeInfos = new ArrayList<>();
+        for (Index index : indexes) {
+            if (isValidIndexName(index.getDatasetName(), index.getIndexName())) {
+                Pair<IFileSplitProvider, AlgebricksPartitionConstraint> indexSplitsAndConstraint =
+                        metadataProvider.getSplitProviderAndConstraints(ds.getDataverseName(), ds.getDatasetName(),
+                                index.getIndexName(), temp);
+                IIndexDataflowHelperFactory indexDataflowHelperFactory = ds.getIndexDataflowHelperFactory(
+                        metadataProvider, index, null, null, mergePolicyFactory, mergePolicyFactoryProperties);
+                treeDataflowHelperFactories.add(indexDataflowHelperFactory);
+                treeInfos.add(new IndexInfoOperatorDescriptor(indexSplitsAndConstraint.first,
+                        RuntimeComponentsProvider.RUNTIME_PROVIDER, RuntimeComponentsProvider.RUNTIME_PROVIDER));
+            }
+        }
+
+        ExternalDatasetIndexesAbortOperatorDescriptor op = new ExternalDatasetIndexesAbortOperatorDescriptor(spec,
+                filesIndexDataflowHelperFactory, filesIndexInfo, treeDataflowHelperFactories, treeInfos);
+
+        spec.addRoot(op);
+        AlgebricksPartitionConstraintHelper.setPartitionConstraintInJobSpec(spec, op,
+                filesIndexSplitsAndConstraint.second);
+        spec.setConnectorPolicyAssignmentPolicy(new ConnectorPolicyAssignmentPolicy());
+        return spec;
+
+    }
+
+    public static JobSpecification buildRecoverOp(Dataset ds, List<Index> indexes, MetadataProvider metadataProvider)
+            throws AlgebricksException {
+        JobSpecification spec = RuntimeUtils.createJobSpecification();
+        Pair<ILSMMergePolicyFactory, Map<String, String>> compactionInfo =
+                DatasetUtil.getMergePolicyFactory(ds, metadataProvider.getMetadataTxnContext());
+        ILSMMergePolicyFactory mergePolicyFactory = compactionInfo.first;
+        Map<String, String> mergePolicyFactoryProperties = compactionInfo.second;
+        boolean temp = ds.getDatasetDetails().isTemp();
+
+        Pair<IFileSplitProvider, AlgebricksPartitionConstraint> filesIndexSplitsAndConstraint =
+                metadataProvider.getSplitProviderAndConstraints(ds.getDataverseName(), ds.getDatasetName(),
+                        IndexingConstants.getFilesIndexName(ds.getDatasetName()), temp);
+        IFileSplitProvider filesIndexSplitProvider = filesIndexSplitsAndConstraint.first;
+        String fileIndexName = BTreeDataflowHelperFactoryProvider.externalFileIndexName(ds);
+        Index fileIndex = MetadataManager.INSTANCE.getIndex(metadataProvider.getMetadataTxnContext(),
+                ds.getDataverseName(), ds.getDatasetName(), fileIndexName);
+        IIndexDataflowHelperFactory filesIndexDataflowHelperFactory = ds.getIndexDataflowHelperFactory(
+                metadataProvider, fileIndex, null, null, mergePolicyFactory, mergePolicyFactoryProperties);
+        IndexInfoOperatorDescriptor filesIndexInfo = new IndexInfoOperatorDescriptor(filesIndexSplitProvider,
+                RuntimeComponentsProvider.RUNTIME_PROVIDER, RuntimeComponentsProvider.RUNTIME_PROVIDER);
+
+        ArrayList<IIndexDataflowHelperFactory> treeDataflowHelperFactories = new ArrayList<>();
+        ArrayList<IndexInfoOperatorDescriptor> treeInfos = new ArrayList<>();
+        for (Index index : indexes) {
+            if (isValidIndexName(index.getDatasetName(), index.getIndexName())) {
+                Pair<IFileSplitProvider, AlgebricksPartitionConstraint> indexSplitsAndConstraint =
+                        metadataProvider.getSplitProviderAndConstraints(ds.getDataverseName(), ds.getDatasetName(),
+                                index.getIndexName(), temp);
+                IIndexDataflowHelperFactory indexDataflowHelperFactory = ds.getIndexDataflowHelperFactory(
+                        metadataProvider, index, null, null, mergePolicyFactory, mergePolicyFactoryProperties);
+                treeDataflowHelperFactories.add(indexDataflowHelperFactory);
+                treeInfos.add(new IndexInfoOperatorDescriptor(indexSplitsAndConstraint.first,
+                        RuntimeComponentsProvider.RUNTIME_PROVIDER, RuntimeComponentsProvider.RUNTIME_PROVIDER));
+            }
+        }
+
+        ExternalDatasetIndexesRecoverOperatorDescriptor op = new ExternalDatasetIndexesRecoverOperatorDescriptor(spec,
+                filesIndexDataflowHelperFactory, filesIndexInfo, treeDataflowHelperFactories, treeInfos);
+
+        spec.addRoot(op);
+        AlgebricksPartitionConstraintHelper.setPartitionConstraintInJobSpec(spec, op,
+                filesIndexSplitsAndConstraint.second);
+        spec.setConnectorPolicyAssignmentPolicy(new ConnectorPolicyAssignmentPolicy());
+        return spec;
+    }
+
+    public static JobSpecification compactFilesIndexJobSpec(Dataset dataset, MetadataProvider metadataProvider,
+            IStorageComponentProvider storageComponentProvider) throws AlgebricksException {
+        JobSpecification spec = RuntimeUtils.createJobSpecification();
+        Pair<ILSMMergePolicyFactory, Map<String, String>> compactionInfo =
+                DatasetUtil.getMergePolicyFactory(dataset, metadataProvider.getMetadataTxnContext());
+        ILSMMergePolicyFactory mergePolicyFactory = compactionInfo.first;
+        Map<String, String> mergePolicyFactoryProperties = compactionInfo.second;
+        Pair<IFileSplitProvider, AlgebricksPartitionConstraint> secondarySplitsAndConstraint =
+                metadataProvider.splitProviderAndPartitionConstraintsForFilesIndex(dataset.getDataverseName(),
+                        dataset.getDatasetName(), IndexingConstants.getFilesIndexName(dataset.getDatasetName()), true);
+        IFileSplitProvider secondaryFileSplitProvider = secondarySplitsAndConstraint.first;
+
+        String fileIndexName = BTreeDataflowHelperFactoryProvider.externalFileIndexName(dataset);
+        Index fileIndex = MetadataManager.INSTANCE.getIndex(metadataProvider.getMetadataTxnContext(),
+                dataset.getDataverseName(), dataset.getDatasetName(), fileIndexName);
+        IIndexDataflowHelperFactory dataflowHelperFactory = dataset.getIndexDataflowHelperFactory(metadataProvider,
+                fileIndex, null, null, mergePolicyFactory, mergePolicyFactoryProperties);
+        FilesIndexDescription filesIndexDescription = new FilesIndexDescription();
+        LSMTreeIndexCompactOperatorDescriptor compactOp = new LSMTreeIndexCompactOperatorDescriptor(spec,
+                RuntimeComponentsProvider.RUNTIME_PROVIDER, RuntimeComponentsProvider.RUNTIME_PROVIDER,
+                secondaryFileSplitProvider, filesIndexDescription.EXTERNAL_FILE_INDEX_TYPE_TRAITS,
+                FilesIndexDescription.FILES_INDEX_COMP_FACTORIES, new int[] { 0 }, dataflowHelperFactory,
+                NoOpOperationCallbackFactory.INSTANCE, storageComponentProvider.getMetadataPageManagerFactory());
+        spec.addRoot(compactOp);
+        AlgebricksPartitionConstraintHelper.setPartitionConstraintInJobSpec(spec, compactOp,
+                secondarySplitsAndConstraint.second);
+        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/IndexUtil.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/IndexUtil.java
new file mode 100644
index 0000000..c6e0a6b
--- /dev/null
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/IndexUtil.java
@@ -0,0 +1,188 @@
+/*
+ * 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.util.List;
+import java.util.Map;
+
+import org.apache.asterix.common.config.OptimizationConfUtil;
+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.ExternalFile;
+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.om.types.ARecordType;
+import org.apache.asterix.runtime.utils.RuntimeUtils;
+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.rewriter.base.PhysicalOptimizationConfig;
+import org.apache.hyracks.api.dataflow.value.ITypeTraits;
+import org.apache.hyracks.api.job.JobSpecification;
+import org.apache.hyracks.dataflow.std.file.IFileSplitProvider;
+import org.apache.hyracks.storage.am.common.dataflow.IIndexDataflowHelperFactory;
+import org.apache.hyracks.storage.am.common.dataflow.IndexDropOperatorDescriptor;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMMergePolicyFactory;
+
+public class IndexUtil {
+
+    //TODO: replace this null with an empty array. currently, this breaks many tests
+    private static final int[] empty = null;
+    private static final PhysicalOptimizationConfig physicalOptimizationConfig =
+            OptimizationConfUtil.getPhysicalOptimizationConfig();
+
+    private IndexUtil() {
+    }
+
+    public static int[] getFilterFields(Dataset dataset, Index index, ITypeTraits[] filterTypeTraits)
+            throws AlgebricksException {
+        if (index.isPrimaryIndex()) {
+            return DatasetUtil.createFilterFields(dataset);
+        }
+        return secondaryFilterFields(dataset, index, filterTypeTraits);
+    }
+
+    public static int[] getBtreeFieldsIfFiltered(Dataset dataset, Index index) throws AlgebricksException {
+        if (index.isPrimaryIndex()) {
+            return DatasetUtil.createBTreeFieldsWhenThereisAFilter(dataset);
+        }
+        int numPrimaryKeys = DatasetUtil.getPartitioningKeys(dataset).size();
+        int numSecondaryKeys = index.getKeyFieldNames().size();
+        int[] btreeFields = new int[numSecondaryKeys + numPrimaryKeys];
+        for (int k = 0; k < btreeFields.length; k++) {
+            btreeFields[k] = k;
+        }
+        return btreeFields;
+    }
+
+    private static int[] secondaryFilterFields(Dataset dataset, Index index, ITypeTraits[] filterTypeTraits)
+            throws CompilationException {
+        if (filterTypeTraits == null) {
+            return empty;
+        }
+        int numPrimaryKeys = DatasetUtil.getPartitioningKeys(dataset).size();
+        int numSecondaryKeys = index.getKeyFieldNames().size();
+        switch (index.getIndexType()) {
+            case BTREE:
+                return new int[] { numPrimaryKeys + numSecondaryKeys };
+            case RTREE:
+            case LENGTH_PARTITIONED_NGRAM_INVIX:
+            case LENGTH_PARTITIONED_WORD_INVIX:
+            case SINGLE_PARTITION_NGRAM_INVIX:
+            case SINGLE_PARTITION_WORD_INVIX:
+                break;
+            default:
+                throw new CompilationException(ErrorCode.COMPILATION_UNKNOWN_INDEX_TYPE,
+                        index.getIndexType().toString());
+        }
+        return empty;
+    }
+
+    public static JobSpecification dropJob(Index index, MetadataProvider metadataProvider, Dataset dataset)
+            throws AlgebricksException {
+        JobSpecification spec = RuntimeUtils.createJobSpecification();
+        IStorageComponentProvider storageComponentProvider = metadataProvider.getStorageComponentProvider();
+        boolean temp = dataset.getDatasetDetails().isTemp();
+        Pair<IFileSplitProvider, AlgebricksPartitionConstraint> splitsAndConstraint =
+                metadataProvider.getSplitProviderAndConstraints(index.getDataverseName(), index.getDatasetName(),
+                        index.getIndexName(), temp);
+        Pair<ILSMMergePolicyFactory, Map<String, String>> compactionInfo =
+                DatasetUtil.getMergePolicyFactory(dataset, metadataProvider.getMetadataTxnContext());
+        ARecordType recordType =
+                (ARecordType) metadataProvider.findType(dataset.getItemTypeDataverseName(), dataset.getItemTypeName());
+        ARecordType metaType = DatasetUtil.getMetaType(metadataProvider, dataset);
+        IIndexDataflowHelperFactory dataflowHelperFactory = dataset.getIndexDataflowHelperFactory(metadataProvider,
+                index, recordType, metaType, compactionInfo.first, compactionInfo.second);
+        IndexDropOperatorDescriptor btreeDrop =
+                new IndexDropOperatorDescriptor(spec, storageComponentProvider.getStorageManager(),
+                        storageComponentProvider.getIndexLifecycleManagerProvider(), splitsAndConstraint.first,
+                        dataflowHelperFactory, storageComponentProvider.getMetadataPageManagerFactory());
+        AlgebricksPartitionConstraintHelper.setPartitionConstraintInJobSpec(spec, btreeDrop,
+                splitsAndConstraint.second);
+        spec.addRoot(btreeDrop);
+        return spec;
+    }
+
+    public static JobSpecification buildSecondaryIndexCreationJobSpec(Dataset dataset, Index index,
+            ARecordType recType, ARecordType metaType, ARecordType enforcedType, ARecordType enforcedMetaType,
+            MetadataProvider metadataProvider) throws AlgebricksException {
+        SecondaryIndexOperationsHelper secondaryIndexHelper =
+                SecondaryIndexOperationsHelper.createIndexOperationsHelper(dataset, index, metadataProvider,
+                        physicalOptimizationConfig, recType, metaType, enforcedType, enforcedMetaType);
+        return secondaryIndexHelper.buildCreationJobSpec();
+    }
+
+    public static JobSpecification buildSecondaryIndexLoadingJobSpec(Dataset dataset, Index index, ARecordType recType,
+            ARecordType metaType, ARecordType enforcedType, ARecordType enforcedMetaType,
+            MetadataProvider metadataProvider) throws AlgebricksException {
+        SecondaryIndexOperationsHelper secondaryIndexHelper =
+                SecondaryIndexOperationsHelper.createIndexOperationsHelper(dataset, index, metadataProvider,
+                        physicalOptimizationConfig, recType, metaType, enforcedType, enforcedMetaType);
+        return secondaryIndexHelper.buildLoadingJobSpec();
+    }
+
+    public static JobSpecification buildSecondaryIndexLoadingJobSpec(Dataset dataset, Index index, ARecordType recType,
+            ARecordType metaType, ARecordType enforcedType, ARecordType enforcedMetaType,
+            MetadataProvider metadataProvider, List<ExternalFile> files) throws AlgebricksException {
+        SecondaryIndexOperationsHelper secondaryIndexHelper =
+                SecondaryIndexOperationsHelper.createIndexOperationsHelper(dataset, index, metadataProvider,
+                        physicalOptimizationConfig, recType, metaType, enforcedType, enforcedMetaType);
+        secondaryIndexHelper.setExternalFiles(files);
+        return secondaryIndexHelper.buildLoadingJobSpec();
+    }
+
+    public static JobSpecification buildDropSecondaryIndexJobSpec(Index index, MetadataProvider metadataProvider,
+            Dataset dataset) throws AlgebricksException {
+        JobSpecification spec = RuntimeUtils.createJobSpecification();
+        boolean temp = dataset.getDatasetDetails().isTemp();
+        IStorageComponentProvider storageComponentProvider = metadataProvider.getStorageComponentProvider();
+        Pair<IFileSplitProvider, AlgebricksPartitionConstraint> splitsAndConstraint =
+                metadataProvider.getSplitProviderAndConstraints(index.getDataverseName(), index.getDatasetName(),
+                        index.getIndexName(), temp);
+        Pair<ILSMMergePolicyFactory, Map<String, String>> compactionInfo =
+                DatasetUtil.getMergePolicyFactory(dataset, metadataProvider.getMetadataTxnContext());
+        ARecordType recordType =
+                (ARecordType) metadataProvider.findType(dataset.getItemTypeDataverseName(), dataset.getItemTypeName());
+        ARecordType metaType = DatasetUtil.getMetaType(metadataProvider, dataset);
+        IIndexDataflowHelperFactory dataflowHelperFactory = dataset.getIndexDataflowHelperFactory(metadataProvider,
+                index, recordType, metaType, compactionInfo.first, compactionInfo.second);
+        // The index drop operation should be persistent regardless of temp datasets or permanent dataset.
+        IndexDropOperatorDescriptor btreeDrop =
+                new IndexDropOperatorDescriptor(spec, storageComponentProvider.getStorageManager(),
+                        storageComponentProvider.getIndexLifecycleManagerProvider(), splitsAndConstraint.first,
+                        dataflowHelperFactory, storageComponentProvider.getMetadataPageManagerFactory());
+        AlgebricksPartitionConstraintHelper.setPartitionConstraintInJobSpec(spec, btreeDrop,
+                splitsAndConstraint.second);
+        spec.addRoot(btreeDrop);
+
+        return spec;
+    }
+
+    public static JobSpecification buildSecondaryIndexCompactJobSpec(Dataset dataset, Index index, ARecordType recType,
+            ARecordType metaType, ARecordType enforcedType, ARecordType enforcedMetaType,
+            MetadataProvider metadataProvider) throws AlgebricksException {
+        SecondaryIndexOperationsHelper secondaryIndexHelper =
+                SecondaryIndexOperationsHelper.createIndexOperationsHelper(dataset, index, metadataProvider,
+                        physicalOptimizationConfig, recType, metaType, enforcedType, enforcedMetaType);
+        return secondaryIndexHelper.buildCompactJobSpec();
+    }
+}
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/InvertedIndexDataflowHelperFactoryProvider.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/InvertedIndexDataflowHelperFactoryProvider.java
new file mode 100644
index 0000000..15aae94
--- /dev/null
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/InvertedIndexDataflowHelperFactoryProvider.java
@@ -0,0 +1,117 @@
+/*
+ * 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.util.List;
+import java.util.Map;
+
+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.common.exceptions.AsterixException;
+import org.apache.asterix.common.exceptions.CompilationException;
+import org.apache.asterix.common.exceptions.ErrorCode;
+import org.apache.asterix.metadata.api.IIndexDataflowHelperFactoryProvider;
+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.om.types.ARecordType;
+import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.hyracks.api.dataflow.value.IBinaryComparatorFactory;
+import org.apache.hyracks.api.dataflow.value.ITypeTraits;
+import org.apache.hyracks.storage.am.common.dataflow.IIndexDataflowHelperFactory;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMMergePolicyFactory;
+import org.apache.hyracks.storage.am.lsm.invertedindex.dataflow.LSMInvertedIndexDataflowHelperFactory;
+import org.apache.hyracks.storage.am.lsm.invertedindex.dataflow.PartitionedLSMInvertedIndexDataflowHelperFactory;
+
+public class InvertedIndexDataflowHelperFactoryProvider implements IIndexDataflowHelperFactoryProvider {
+    public static final InvertedIndexDataflowHelperFactoryProvider INSTANCE =
+            new InvertedIndexDataflowHelperFactoryProvider();
+
+    private InvertedIndexDataflowHelperFactoryProvider() {
+    }
+
+    @Override
+    public IIndexDataflowHelperFactory getIndexDataflowHelperFactory(MetadataProvider mdProvider, Dataset dataset,
+            Index index, ARecordType recordType, ARecordType metaType, ILSMMergePolicyFactory mergePolicyFactory,
+            Map<String, String> mergePolicyProperties, ITypeTraits[] filterTypeTraits,
+            IBinaryComparatorFactory[] filterCmpFactories) throws AlgebricksException {
+        // Get basic info
+        List<List<String>> primaryKeys = DatasetUtil.getPartitioningKeys(dataset);
+        List<List<String>> secondaryKeys = index.getKeyFieldNames();
+        List<String> filterFieldName = DatasetUtil.getFilterField(dataset);
+        int numPrimaryKeys = primaryKeys.size();
+        int numSecondaryKeys = secondaryKeys.size();
+        // Validate
+        if (dataset.getDatasetType() != DatasetType.INTERNAL) {
+            throw new CompilationException(ErrorCode.COMPILATION_INDEX_TYPE_NOT_SUPPORTED_FOR_DATASET_TYPE,
+                    index.getIndexType().name(), dataset.getDatasetType());
+        }
+        if (numPrimaryKeys > 1) {
+            throw new AsterixException("Cannot create inverted index on dataset with composite primary key.");
+        }
+        if (numSecondaryKeys > 1) {
+            throw new AsterixException("Cannot create composite inverted index on multiple fields.");
+        }
+        boolean isPartitioned = index.getIndexType() == IndexType.LENGTH_PARTITIONED_WORD_INVIX
+                || index.getIndexType() == IndexType.LENGTH_PARTITIONED_NGRAM_INVIX;
+        int numTokenKeyPairFields = (!isPartitioned) ? 1 + numPrimaryKeys : 2 + numPrimaryKeys;
+        int[] invertedIndexFields = null;
+        int[] secondaryFilterFieldsForNonBulkLoadOps = null;
+        int[] invertedIndexFieldsForNonBulkLoadOps = null;
+        int[] secondaryFilterFields = null;
+        if (filterFieldName != null) {
+            invertedIndexFields = new int[numTokenKeyPairFields];
+            for (int i = 0; i < invertedIndexFields.length; i++) {
+                invertedIndexFields[i] = i;
+            }
+            secondaryFilterFieldsForNonBulkLoadOps = new int[filterFieldName.size()];
+            secondaryFilterFieldsForNonBulkLoadOps[0] = numSecondaryKeys + numPrimaryKeys;
+            invertedIndexFieldsForNonBulkLoadOps = new int[numSecondaryKeys + numPrimaryKeys];
+            for (int i = 0; i < invertedIndexFieldsForNonBulkLoadOps.length; i++) {
+                invertedIndexFieldsForNonBulkLoadOps[i] = i;
+            }
+            secondaryFilterFields = new int[filterFieldName.size()];
+            secondaryFilterFields[0] = numTokenKeyPairFields - numPrimaryKeys + numPrimaryKeys;
+        }
+        IStorageComponentProvider storageComponentProvider = mdProvider.getStorageComponentProvider();
+        if (!isPartitioned) {
+            return new LSMInvertedIndexDataflowHelperFactory(
+                    new AsterixVirtualBufferCacheProvider(dataset.getDatasetId()), mergePolicyFactory,
+                    mergePolicyProperties, dataset.getIndexOperationTrackerFactory(index),
+                    storageComponentProvider.getIoOperationSchedulerProvider(),
+                    dataset.getIoOperationCallbackFactory(index),
+                    mdProvider.getStorageProperties().getBloomFilterFalsePositiveRate(), invertedIndexFields,
+                    filterTypeTraits, filterCmpFactories, secondaryFilterFields,
+                    secondaryFilterFieldsForNonBulkLoadOps, invertedIndexFieldsForNonBulkLoadOps,
+                    !dataset.getDatasetDetails().isTemp());
+        } else {
+            return new PartitionedLSMInvertedIndexDataflowHelperFactory(
+                    new AsterixVirtualBufferCacheProvider(dataset.getDatasetId()), mergePolicyFactory,
+                    mergePolicyProperties, dataset.getIndexOperationTrackerFactory(index),
+                    storageComponentProvider.getIoOperationSchedulerProvider(),
+                    dataset.getIoOperationCallbackFactory(index),
+                    mdProvider.getStorageProperties().getBloomFilterFalsePositiveRate(), invertedIndexFields,
+                    filterTypeTraits, filterCmpFactories, secondaryFilterFields,
+                    secondaryFilterFieldsForNonBulkLoadOps, invertedIndexFieldsForNonBulkLoadOps,
+                    !dataset.getDatasetDetails().isTemp());
+        }
+    }
+}
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/KeyFieldTypeUtils.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/KeyFieldTypeUtil.java
similarity index 85%
rename from asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/KeyFieldTypeUtils.java
rename to asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/KeyFieldTypeUtil.java
index 108cd33..e074241 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/KeyFieldTypeUtils.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/KeyFieldTypeUtil.java
@@ -22,16 +22,21 @@
 import java.util.List;
 
 import org.apache.asterix.common.config.DatasetConfig.DatasetType;
-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.metadata.entities.Dataset;
 import org.apache.asterix.metadata.entities.Index;
 import org.apache.asterix.metadata.entities.InternalDatasetDetails;
 import org.apache.asterix.om.types.ARecordType;
 import org.apache.asterix.om.types.IAType;
-import org.apache.asterix.om.util.NonTaggedFormatUtil;
+import org.apache.asterix.om.utils.NonTaggedFormatUtil;
+import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
 import org.apache.hyracks.algebricks.common.utils.Pair;
 
-public class KeyFieldTypeUtils {
+public class KeyFieldTypeUtil {
+
+    private KeyFieldTypeUtil() {
+    }
 
     /**
      * Get the types of primary key (partitioning key) fields
@@ -43,10 +48,10 @@
      * @param metaRecordType
      *            the auxiliary meta record type.
      * @return a list of IATypes, one for each corresponding primary key field.
-     * @throws AsterixException
+     * @throws AlgebricksException
      */
     public static List<IAType> getPartitoningKeyTypes(Dataset dataset, ARecordType recordType,
-            ARecordType metaRecordType) throws AsterixException {
+            ARecordType metaRecordType) throws AlgebricksException {
         if (dataset.getDatasetType() != DatasetType.INTERNAL) {
             return null;
         }
@@ -64,10 +69,10 @@
      * @param metaRecordType
      *            the auxiliary meta record type.
      * @return a list of IATypes, one for each corresponding primary key field.
-     * @throws AsterixException
+     * @throws AlgebricksException
      */
     public static List<IAType> getPartitioningKeyTypes(InternalDatasetDetails datasetDetails, ARecordType recordType,
-            ARecordType metaRecordType) throws AsterixException {
+            ARecordType metaRecordType) throws AlgebricksException {
         List<Integer> keySourceIndicators = datasetDetails.getKeySourceIndicator();
         List<List<String>> partitioningKeys = datasetDetails.getPartitioningKey();
         return getKeyTypes(recordType, metaRecordType, partitioningKeys, keySourceIndicators);
@@ -83,12 +88,13 @@
      * @param keys,
      *            the list of key fields.
      * @param keySourceIndicators,
-     *            a list of integers to indicate that each key field is from the main record or the auxiliary meta record.
+     *            a list of integers to indicate that each key field is from the main record or the auxiliary meta
+     *            record.
      * @return a list of IATypes, one for each corresponding index key field.
-     * @throws AsterixException
+     * @throws AlgebricksException
      */
     public static List<IAType> getKeyTypes(ARecordType recordType, ARecordType metaRecordType, List<List<String>> keys,
-            List<Integer> keySourceIndicators) throws AsterixException {
+            List<Integer> keySourceIndicators) throws AlgebricksException {
         List<IAType> keyTypes = new ArrayList<>();
         int index = 0;
         for (List<String> partitioningKey : keys) {
@@ -109,10 +115,10 @@
      * @param metaRecordType
      *            the auxiliary meta record type.
      * @return a list of IATypes, one for each corresponding index key field.
-     * @throws AsterixException
+     * @throws AlgebricksException
      */
     public static List<IAType> getBTreeIndexKeyTypes(Index index, ARecordType recordType, ARecordType metaRecordType)
-            throws AsterixException {
+            throws AlgebricksException {
         List<Integer> keySourceIndicators = index.getKeyFieldSourceIndicators();
         List<IAType> indexKeyTypes = new ArrayList<>();
         for (int i = 0; i < index.getKeyFieldNames().size(); i++) {
@@ -133,10 +139,10 @@
      * @param metaRecordType
      *            the auxiliary meta record type.
      * @return a list of IATypes, one for each corresponding index key field.
-     * @throws AsterixException
+     * @throws AlgebricksException
      */
     public static List<IAType> getRTreeIndexKeyTypes(Index index, ARecordType recordType, ARecordType metaRecordType)
-            throws AsterixException {
+            throws AlgebricksException {
         List<Integer> keySourceIndicators = index.getKeyFieldSourceIndicators();
         List<IAType> indexKeyTypes = new ArrayList<>();
         ARecordType targetRecType = chooseSource(keySourceIndicators, 0, recordType, metaRecordType);
@@ -144,7 +150,7 @@
                 index.getKeyFieldNames().get(0), targetRecType);
         IAType keyType = keyPairType.first;
         IAType nestedKeyType = NonTaggedFormatUtil.getNestedSpatialType(keyType.getTypeTag());
-        int numKeys = KeyFieldTypeUtils.getNumSecondaryKeys(index, targetRecType, metaRecordType);
+        int numKeys = KeyFieldTypeUtil.getNumSecondaryKeys(index, targetRecType, metaRecordType);
         for (int i = 0; i < numKeys; i++) {
             indexKeyTypes.add(nestedKeyType);
         }
@@ -161,30 +167,26 @@
      * @param metaRecordType
      *            the auxiliary meta record type.
      * @return the number of secondary index keys.
-     * @throws AsterixException
+     * @throws AlgebricksException
      */
     public static int getNumSecondaryKeys(Index index, ARecordType recordType, ARecordType metaRecordType)
-            throws AsterixException {
+            throws AlgebricksException {
         List<Integer> keySourceIndicators = index.getKeyFieldSourceIndicators();
         switch (index.getIndexType()) {
             case BTREE:
             case SINGLE_PARTITION_WORD_INVIX:
             case SINGLE_PARTITION_NGRAM_INVIX:
             case LENGTH_PARTITIONED_WORD_INVIX:
-            case LENGTH_PARTITIONED_NGRAM_INVIX: {
+            case LENGTH_PARTITIONED_NGRAM_INVIX:
                 return index.getKeyFieldNames().size();
-            }
-            case RTREE: {
+            case RTREE:
                 Pair<IAType, Boolean> keyPairType = Index.getNonNullableOpenFieldType(index.getKeyFieldTypes().get(0),
                         index.getKeyFieldNames().get(0),
                         chooseSource(keySourceIndicators, 0, recordType, metaRecordType));
                 IAType keyType = keyPairType.first;
-                int numDimensions = NonTaggedFormatUtil.getNumDimensions(keyType.getTypeTag());
-                return numDimensions * 2;
-            }
-            default: {
-                throw new AsterixException("Unknown index kind: " + index.getIndexType());
-            }
+                return NonTaggedFormatUtil.getNumDimensions(keyType.getTypeTag()) * 2;
+            default:
+                throw new CompilationException(ErrorCode.COMPILATION_UNKNOWN_INDEX_TYPE, index.getIndexType());
         }
     }
 
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/MetadataLockManager.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/MetadataLockManager.java
index f16919d..a9c721a 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/MetadataLockManager.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/MetadataLockManager.java
@@ -27,7 +27,7 @@
 
 public class MetadataLockManager {
 
-    public static MetadataLockManager INSTANCE = new MetadataLockManager();
+    public static final MetadataLockManager INSTANCE = new MetadataLockManager();
     private final ConcurrentHashMap<String, ReentrantReadWriteLock> dataversesLocks;
     private final ConcurrentHashMap<String, DatasetLock> datasetsLocks;
     private final ConcurrentHashMap<String, ReentrantReadWriteLock> functionsLocks;
@@ -38,14 +38,14 @@
     private final ConcurrentHashMap<String, ReentrantReadWriteLock> dataTypeLocks;
 
     private MetadataLockManager() {
-        dataversesLocks = new ConcurrentHashMap<String, ReentrantReadWriteLock>();
-        datasetsLocks = new ConcurrentHashMap<String, DatasetLock>();
-        functionsLocks = new ConcurrentHashMap<String, ReentrantReadWriteLock>();
-        nodeGroupsLocks = new ConcurrentHashMap<String, ReentrantReadWriteLock>();
-        feedsLocks = new ConcurrentHashMap<String, ReentrantReadWriteLock>();
-        feedPolicyLocks = new ConcurrentHashMap<String, ReentrantReadWriteLock>();
-        compactionPolicyLocks = new ConcurrentHashMap<String, ReentrantReadWriteLock>();
-        dataTypeLocks = new ConcurrentHashMap<String, ReentrantReadWriteLock>();
+        dataversesLocks = new ConcurrentHashMap<>();
+        datasetsLocks = new ConcurrentHashMap<>();
+        functionsLocks = new ConcurrentHashMap<>();
+        nodeGroupsLocks = new ConcurrentHashMap<>();
+        feedsLocks = new ConcurrentHashMap<>();
+        feedPolicyLocks = new ConcurrentHashMap<>();
+        compactionPolicyLocks = new ConcurrentHashMap<>();
+        dataTypeLocks = new ConcurrentHashMap<>();
     }
 
     public void acquireDataverseReadLock(String dataverseName) {
@@ -408,8 +408,8 @@
         releaseDataverseReadLock(dataverseName);
     }
 
-    public void insertDeleteUpsertBegin(String dataverseName, String datasetFullyQualifiedName, List<String> dataverses,
-            List<String> datasets) {
+    public void insertDeleteUpsertBegin(String dataverseName, String datasetFullyQualifiedName,
+            List<String> dataverses, List<String> datasets) {
         dataverses.add(dataverseName);
         datasets.add(datasetFullyQualifiedName);
         Collections.sort(dataverses);
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/MetadataUtil.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/MetadataUtil.java
new file mode 100644
index 0000000..3133aba
--- /dev/null
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/MetadataUtil.java
@@ -0,0 +1,41 @@
+/*
+ * 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;
+
+public class MetadataUtil {
+    public static final int PENDING_NO_OP = 0;
+    public static final int PENDING_ADD_OP = 1;
+    public static final int PENDING_DROP_OP = 2;
+
+    private MetadataUtil() {
+    }
+
+    public static String pendingOpToString(int pendingOp) {
+        switch (pendingOp) {
+            case PENDING_NO_OP:
+                return "Pending No Operation";
+            case PENDING_ADD_OP:
+                return "Pending Add Operation";
+            case PENDING_DROP_OP:
+                return "Pending Drop Operation";
+            default:
+                return "Unknown Pending Operation";
+        }
+    }
+}
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/RTreeDataflowHelperFactoryProvider.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/RTreeDataflowHelperFactoryProvider.java
new file mode 100644
index 0000000..8859b9d
--- /dev/null
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/RTreeDataflowHelperFactoryProvider.java
@@ -0,0 +1,171 @@
+/*
+ * 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.util.List;
+import java.util.Map;
+
+import org.apache.asterix.common.config.DatasetConfig.DatasetType;
+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.IndexingConstants;
+import org.apache.asterix.metadata.api.IIndexDataflowHelperFactoryProvider;
+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.om.types.ARecordType;
+import org.apache.asterix.om.types.ATypeTag;
+import org.apache.asterix.om.types.IAType;
+import org.apache.asterix.om.utils.NonTaggedFormatUtil;
+import org.apache.commons.lang.StringUtils;
+import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.hyracks.api.dataflow.value.IBinaryComparatorFactory;
+import org.apache.hyracks.api.dataflow.value.ITypeTraits;
+import org.apache.hyracks.storage.am.common.api.IPrimitiveValueProviderFactory;
+import org.apache.hyracks.storage.am.common.dataflow.IIndexDataflowHelperFactory;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMMergePolicyFactory;
+import org.apache.hyracks.storage.am.lsm.rtree.dataflow.ExternalRTreeDataflowHelperFactory;
+import org.apache.hyracks.storage.am.lsm.rtree.dataflow.LSMRTreeWithAntiMatterTuplesDataflowHelperFactory;
+import org.apache.hyracks.storage.am.rtree.frames.RTreePolicyType;
+
+public class RTreeDataflowHelperFactoryProvider implements IIndexDataflowHelperFactoryProvider {
+
+    public static final RTreeDataflowHelperFactoryProvider INSTANCE = new RTreeDataflowHelperFactoryProvider();
+
+    private RTreeDataflowHelperFactoryProvider() {
+    }
+
+    protected RTreePolicyType rTreePolicyType() {
+        return RTreePolicyType.RTREE;
+    }
+
+    @Override
+    public IIndexDataflowHelperFactory getIndexDataflowHelperFactory(MetadataProvider mdProvider, Dataset dataset,
+            Index index, ARecordType recordType, ARecordType metaType, ILSMMergePolicyFactory mergePolicyFactory,
+            Map<String, String> mergePolicyProperties, ITypeTraits[] filterTypeTraits,
+            IBinaryComparatorFactory[] filterCmpFactories) throws AlgebricksException {
+        if (index.getKeyFieldNames().size() != 1) {
+            throw new CompilationException(ErrorCode.COMPILATION_ILLEGAL_INDEX_NUM_OF_FIELD,
+                    index.getKeyFieldNames().size(), index.getIndexType(), 1);
+        }
+        IAType spatialType = Index.getNonNullableOpenFieldType(index.getKeyFieldTypes().get(0),
+                index.getKeyFieldNames().get(0), recordType).first;
+        if (spatialType == null) {
+            throw new CompilationException(ErrorCode.COMPILATION_FIELD_NOT_FOUND,
+                    StringUtils.join(index.getKeyFieldNames().get(0), '.'));
+        }
+        List<List<String>> primaryKeyFields = DatasetUtil.getPartitioningKeys(dataset);
+        int numPrimaryKeys = primaryKeyFields.size();
+        ITypeTraits[] primaryTypeTraits = null;
+        IBinaryComparatorFactory[] primaryComparatorFactories = null;
+        IStorageComponentProvider storageComponentProvider = mdProvider.getStorageComponentProvider();
+        if (dataset.getDatasetType() == DatasetType.INTERNAL) {
+            primaryTypeTraits = new ITypeTraits[numPrimaryKeys + 1 + (dataset.hasMetaPart() ? 1 : 0)];
+            primaryComparatorFactories = new IBinaryComparatorFactory[numPrimaryKeys];
+            List<Integer> indicators = null;
+            if (dataset.hasMetaPart()) {
+                indicators = ((InternalDatasetDetails) dataset.getDatasetDetails()).getKeySourceIndicator();
+            }
+            for (int i = 0; i < numPrimaryKeys; i++) {
+                IAType keyType = (indicators == null || indicators.get(i) == 0)
+                        ? recordType.getSubFieldType(primaryKeyFields.get(i))
+                        : metaType.getSubFieldType(primaryKeyFields.get(i));
+                primaryComparatorFactories[i] = storageComponentProvider.getComparatorFactoryProvider()
+                        .getBinaryComparatorFactory(keyType, true);
+                primaryTypeTraits[i] = storageComponentProvider.getTypeTraitProvider().getTypeTrait(keyType);
+            }
+            primaryTypeTraits[numPrimaryKeys] =
+                    storageComponentProvider.getTypeTraitProvider().getTypeTrait(recordType);
+            if (dataset.hasMetaPart()) {
+                primaryTypeTraits[numPrimaryKeys + 1] =
+                        storageComponentProvider.getTypeTraitProvider().getTypeTrait(recordType);
+            }
+        }
+        boolean isPointMBR =
+                spatialType.getTypeTag() == ATypeTag.POINT || spatialType.getTypeTag() == ATypeTag.POINT3D;
+        int numDimensions = NonTaggedFormatUtil.getNumDimensions(spatialType.getTypeTag());
+        int numNestedSecondaryKeyFields = numDimensions * 2;
+        IBinaryComparatorFactory[] secondaryComparatorFactories =
+                new IBinaryComparatorFactory[numNestedSecondaryKeyFields];
+        IPrimitiveValueProviderFactory[] valueProviderFactories =
+                new IPrimitiveValueProviderFactory[numNestedSecondaryKeyFields];
+        ITypeTraits[] secondaryTypeTraits = new ITypeTraits[numNestedSecondaryKeyFields + numPrimaryKeys];
+        IAType nestedKeyType = NonTaggedFormatUtil.getNestedSpatialType(spatialType.getTypeTag());
+        ATypeTag keyType = nestedKeyType.getTypeTag();
+        for (int i = 0; i < numNestedSecondaryKeyFields; i++) {
+            secondaryComparatorFactories[i] = storageComponentProvider.getComparatorFactoryProvider()
+                    .getBinaryComparatorFactory(nestedKeyType, true);
+            secondaryTypeTraits[i] = storageComponentProvider.getTypeTraitProvider().getTypeTrait(nestedKeyType);
+            valueProviderFactories[i] = storageComponentProvider.getPrimitiveValueProviderFactory();
+
+        }
+        for (int i = 0; i < numPrimaryKeys; i++) {
+            secondaryTypeTraits[numNestedSecondaryKeyFields + i] = (dataset.getDatasetType() == DatasetType.INTERNAL)
+                    ? primaryTypeTraits[i] : IndexingConstants.getTypeTraits(i);
+        }
+        int[] rtreeFields = null;
+        if (filterTypeTraits != null && filterTypeTraits.length > 0) {
+            rtreeFields = new int[numNestedSecondaryKeyFields + numPrimaryKeys];
+            for (int i = 0; i < rtreeFields.length; i++) {
+                rtreeFields[i] = i;
+            }
+        }
+        if (dataset.getDatasetType() == DatasetType.INTERNAL) {
+            int[] secondaryFilterFields = (filterTypeTraits != null && filterTypeTraits.length > 0)
+                    ? new int[] { numNestedSecondaryKeyFields + numPrimaryKeys } : null;
+            IBinaryComparatorFactory[] btreeCompFactories = getComparatorFactoriesForDeletedKeyBTree(
+                    secondaryTypeTraits, primaryComparatorFactories, secondaryComparatorFactories);
+            return new LSMRTreeWithAntiMatterTuplesDataflowHelperFactory(valueProviderFactories, rTreePolicyType(),
+                    btreeCompFactories, new AsterixVirtualBufferCacheProvider(dataset.getDatasetId()),
+                    mergePolicyFactory, mergePolicyProperties, dataset.getIndexOperationTrackerFactory(index),
+                    storageComponentProvider.getIoOperationSchedulerProvider(),
+                    dataset.getIoOperationCallbackFactory(index),
+                    MetadataProvider.proposeLinearizer(keyType, secondaryComparatorFactories.length), rtreeFields,
+                    filterTypeTraits, filterCmpFactories, secondaryFilterFields, !dataset.getDatasetDetails().isTemp(),
+                    isPointMBR);
+        } else {
+            return new ExternalRTreeDataflowHelperFactory(valueProviderFactories, rTreePolicyType(),
+                    ExternalIndexingOperations.getBuddyBtreeComparatorFactories(), mergePolicyFactory,
+                    mergePolicyProperties, dataset.getIndexOperationTrackerFactory(index),
+                    storageComponentProvider.getIoOperationSchedulerProvider(),
+                    dataset.getIoOperationCallbackFactory(index),
+                    MetadataProvider.proposeLinearizer(keyType, secondaryComparatorFactories.length),
+                    mdProvider.getStorageProperties().getBloomFilterFalsePositiveRate(),
+                    new int[] { numNestedSecondaryKeyFields },
+                    ExternalDatasetsRegistry.INSTANCE.getDatasetVersion(dataset), true, isPointMBR);
+        }
+    }
+
+    private static IBinaryComparatorFactory[] getComparatorFactoriesForDeletedKeyBTree(
+            ITypeTraits[] secondaryTypeTraits, IBinaryComparatorFactory[] primaryComparatorFactories,
+            IBinaryComparatorFactory[] secondaryComparatorFactories) {
+        IBinaryComparatorFactory[] btreeCompFactories = new IBinaryComparatorFactory[secondaryTypeTraits.length];
+        int i = 0;
+        for (; i < secondaryComparatorFactories.length; i++) {
+            btreeCompFactories[i] = secondaryComparatorFactories[i];
+        }
+        for (int j = 0; i < secondaryTypeTraits.length; i++, j++) {
+            btreeCompFactories[i] = primaryComparatorFactories[j];
+        }
+        return btreeCompFactories;
+    }
+}
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/file/SecondaryBTreeOperationsHelper.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryBTreeOperationsHelper.java
similarity index 64%
rename from asterixdb/asterix-app/src/main/java/org/apache/asterix/file/SecondaryBTreeOperationsHelper.java
rename to asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryBTreeOperationsHelper.java
index 3a9a72b..63368c7 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/file/SecondaryBTreeOperationsHelper.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryBTreeOperationsHelper.java
@@ -16,29 +16,22 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.asterix.file;
+package org.apache.asterix.metadata.utils;
 
 import java.util.List;
 
-import org.apache.asterix.common.config.StorageProperties;
 import org.apache.asterix.common.config.DatasetConfig.DatasetType;
-import org.apache.asterix.common.config.DatasetConfig.IndexType;
 import org.apache.asterix.common.config.GlobalConfig;
 import org.apache.asterix.common.config.IPropertiesProvider;
-import org.apache.asterix.common.context.AsterixVirtualBufferCacheProvider;
-import org.apache.asterix.common.dataflow.LSMIndexUtil;
-import org.apache.asterix.common.exceptions.AsterixException;
-import org.apache.asterix.common.ioopcallbacks.LSMBTreeIOOperationCallbackFactory;
-import org.apache.asterix.common.ioopcallbacks.LSMBTreeWithBuddyIOOperationCallbackFactory;
+import org.apache.asterix.common.context.IStorageComponentProvider;
 import org.apache.asterix.external.indexing.IndexingConstants;
-import org.apache.asterix.external.operators.ExternalDataScanOperatorDescriptor;
+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;
-import org.apache.asterix.metadata.utils.ExternalDatasetsRegistry;
 import org.apache.asterix.om.types.ARecordType;
 import org.apache.asterix.om.types.IAType;
-import org.apache.asterix.runtime.util.RuntimeComponentsProvider;
-import org.apache.asterix.transaction.management.opcallbacks.SecondaryIndexOperationTrackerProvider;
+import org.apache.asterix.runtime.utils.RuntimeUtils;
 import org.apache.asterix.transaction.management.resource.ExternalBTreeWithBuddyLocalResourceMetadataFactory;
 import org.apache.asterix.transaction.management.resource.LSMBTreeLocalResourceMetadataFactory;
 import org.apache.asterix.transaction.management.resource.PersistentLocalResourceFactoryProvider;
@@ -68,63 +61,58 @@
 import org.apache.hyracks.storage.am.common.dataflow.IIndexDataflowHelperFactory;
 import org.apache.hyracks.storage.am.common.dataflow.TreeIndexBulkLoadOperatorDescriptor;
 import org.apache.hyracks.storage.am.common.dataflow.TreeIndexCreateOperatorDescriptor;
-import org.apache.hyracks.storage.am.common.impls.NoOpOperationCallbackFactory;
-import org.apache.hyracks.storage.am.lsm.btree.dataflow.ExternalBTreeWithBuddyDataflowHelperFactory;
-import org.apache.hyracks.storage.am.lsm.btree.dataflow.LSMBTreeDataflowHelperFactory;
+import org.apache.hyracks.storage.am.common.ophelpers.IndexOperation;
 import org.apache.hyracks.storage.am.lsm.common.dataflow.LSMTreeIndexCompactOperatorDescriptor;
 import org.apache.hyracks.storage.common.file.ILocalResourceFactoryProvider;
 import org.apache.hyracks.storage.common.file.LocalResource;
 
 public class SecondaryBTreeOperationsHelper extends SecondaryIndexOperationsHelper {
 
-    protected SecondaryBTreeOperationsHelper(PhysicalOptimizationConfig physOptConf,
-            IPropertiesProvider propertiesProvider) {
-        super(physOptConf, propertiesProvider);
+    protected SecondaryBTreeOperationsHelper(Dataset dataset, Index index, PhysicalOptimizationConfig physOptConf,
+            IPropertiesProvider propertiesProvider, MetadataProvider metadataProvider, ARecordType recType,
+            ARecordType metaType, ARecordType enforcedType, ARecordType enforcedMetaType) {
+        super(dataset, index, physOptConf, propertiesProvider, metadataProvider, recType, metaType, enforcedType,
+                enforcedMetaType);
     }
 
     @Override
-    public JobSpecification buildCreationJobSpec() throws AsterixException, AlgebricksException {
-        JobSpecification spec = JobSpecificationUtils.createJobSpecification();
-
-        StorageProperties storageProperties = propertiesProvider.getStorageProperties();
+    public JobSpecification buildCreationJobSpec() throws AlgebricksException {
+        JobSpecification spec = RuntimeUtils.createJobSpecification();
         ILocalResourceFactoryProvider localResourceFactoryProvider;
-        IIndexDataflowHelperFactory indexDataflowHelperFactory;
+        IIndexDataflowHelperFactory indexDataflowHelperFactory = dataset.getIndexDataflowHelperFactory(
+                metadataProvider, index, itemType, metaType, mergePolicyFactory, mergePolicyFactoryProperties);
+        IStorageComponentProvider storageComponentProvider = metadataProvider.getStorageComponentProvider();
         if (dataset.getDatasetType() == DatasetType.INTERNAL) {
             //prepare a LocalResourceMetadata which will be stored in NC's local resource repository
             LSMBTreeLocalResourceMetadataFactory localResourceMetadata = new LSMBTreeLocalResourceMetadataFactory(
-                    secondaryTypeTraits,
-                    secondaryComparatorFactories, secondaryBloomFilterKeyFields, false, dataset.getDatasetId(),
-                    mergePolicyFactory, mergePolicyFactoryProperties, filterTypeTraits, filterCmpFactories,
-                    secondaryBTreeFields, secondaryFilterFields);
+                    secondaryTypeTraits, secondaryComparatorFactories, secondaryBloomFilterKeyFields, false,
+                    dataset.getDatasetId(), mergePolicyFactory, mergePolicyFactoryProperties, filterTypeTraits,
+                    filterCmpFactories, secondaryBTreeFields, secondaryFilterFields,
+                    dataset.getIndexOperationTrackerFactory(index), dataset.getIoOperationCallbackFactory(index),
+                    storageComponentProvider.getMetadataPageManagerFactory());
             localResourceFactoryProvider =
                     new PersistentLocalResourceFactoryProvider(localResourceMetadata, LocalResource.LSMBTreeResource);
-            indexDataflowHelperFactory = new LSMBTreeDataflowHelperFactory(
-                    new AsterixVirtualBufferCacheProvider(dataset.getDatasetId()), mergePolicyFactory,
-                    mergePolicyFactoryProperties, new SecondaryIndexOperationTrackerProvider(dataset.getDatasetId()),
-                    RuntimeComponentsProvider.RUNTIME_PROVIDER, LSMBTreeIOOperationCallbackFactory.INSTANCE,
-                    storageProperties.getBloomFilterFalsePositiveRate(), false, filterTypeTraits, filterCmpFactories,
-                    secondaryBTreeFields, secondaryFilterFields, !dataset.getDatasetDetails().isTemp());
         } else {
             // External dataset local resource and dataflow helper
-            int[] buddyBreeFields = new int[] { numSecondaryKeys };
+            int[] buddyBreeFields = new int[] { index.getKeyFieldNames().size() };
             ExternalBTreeWithBuddyLocalResourceMetadataFactory localResourceMetadata =
-                    new ExternalBTreeWithBuddyLocalResourceMetadataFactory(
-                    dataset.getDatasetId(), secondaryComparatorFactories, secondaryTypeTraits, mergePolicyFactory,
-                    mergePolicyFactoryProperties, buddyBreeFields);
+                    new ExternalBTreeWithBuddyLocalResourceMetadataFactory(dataset.getDatasetId(),
+                            secondaryComparatorFactories, secondaryTypeTraits, mergePolicyFactory,
+                            mergePolicyFactoryProperties, buddyBreeFields,
+                            dataset.getIndexOperationTrackerFactory(index),
+                            dataset.getIoOperationCallbackFactory(index),
+                            storageComponentProvider.getMetadataPageManagerFactory());
             localResourceFactoryProvider = new PersistentLocalResourceFactoryProvider(localResourceMetadata,
                     LocalResource.ExternalBTreeWithBuddyResource);
-            indexDataflowHelperFactory = new ExternalBTreeWithBuddyDataflowHelperFactory(mergePolicyFactory,
-                    mergePolicyFactoryProperties, new SecondaryIndexOperationTrackerProvider(dataset.getDatasetId()),
-                    RuntimeComponentsProvider.RUNTIME_PROVIDER,
-                    LSMBTreeWithBuddyIOOperationCallbackFactory.INSTANCE,
-                    storageProperties.getBloomFilterFalsePositiveRate(), buddyBreeFields,
-                    ExternalDatasetsRegistry.INSTANCE.getDatasetVersion(dataset), true);
         }
-        TreeIndexCreateOperatorDescriptor secondaryIndexCreateOp = new TreeIndexCreateOperatorDescriptor(spec,
-                RuntimeComponentsProvider.RUNTIME_PROVIDER, RuntimeComponentsProvider.RUNTIME_PROVIDER,
-                secondaryFileSplitProvider, secondaryTypeTraits, secondaryComparatorFactories,
-                secondaryBloomFilterKeyFields, indexDataflowHelperFactory, localResourceFactoryProvider,
-                NoOpOperationCallbackFactory.INSTANCE, LSMIndexUtil.getMetadataPageManagerFactory());
+        TreeIndexCreateOperatorDescriptor secondaryIndexCreateOp =
+                new TreeIndexCreateOperatorDescriptor(spec, storageComponentProvider.getStorageManager(),
+                        storageComponentProvider.getIndexLifecycleManagerProvider(), secondaryFileSplitProvider,
+                        secondaryTypeTraits, secondaryComparatorFactories, secondaryBloomFilterKeyFields,
+                        indexDataflowHelperFactory, localResourceFactoryProvider,
+                        dataset.getModificationCallbackFactory(storageComponentProvider, index, null,
+                                IndexOperation.CREATE, null),
+                        storageComponentProvider.getMetadataPageManagerFactory());
         AlgebricksPartitionConstraintHelper.setPartitionConstraintInJobSpec(spec, secondaryIndexCreateOp,
                 secondaryPartitionConstraint);
         spec.addRoot(secondaryIndexCreateOp);
@@ -133,18 +121,21 @@
     }
 
     @Override
-    public JobSpecification buildLoadingJobSpec() throws AsterixException, AlgebricksException {
-        JobSpecification spec = JobSpecificationUtils.createJobSpecification();
-
-        int[] fieldPermutation = createFieldPermutationForBulkLoadOp(numSecondaryKeys);
+    public JobSpecification buildLoadingJobSpec() throws AlgebricksException {
+        JobSpecification spec = RuntimeUtils.createJobSpecification();
+        boolean isEnforcingKeyTypes = index.isEnforcingKeyFileds();
+        int[] fieldPermutation = createFieldPermutationForBulkLoadOp(index.getKeyFieldNames().size());
+        IIndexDataflowHelperFactory dataflowHelperFactory = dataset.getIndexDataflowHelperFactory(metadataProvider,
+                index, itemType, metaType, mergePolicyFactory, mergePolicyFactoryProperties);
         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
+             * 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
-            ExternalDataScanOperatorDescriptor primaryScanOp = createExternalIndexingOp(spec);
+            ExternalScanOperatorDescriptor primaryScanOp = createExternalIndexingOp(spec);
 
             // Assign op.
             AbstractOperatorDescriptor sourceOp = primaryScanOp;
@@ -153,27 +144,18 @@
                 spec.connect(new OneToOneConnectorDescriptor(spec), primaryScanOp, 0, sourceOp, 0);
             }
             AlgebricksMetaOperatorDescriptor asterixAssignOp =
-                    createExternalAssignOp(spec, numSecondaryKeys, secondaryRecDesc);
+                    createExternalAssignOp(spec, index.getKeyFieldNames().size(), secondaryRecDesc);
 
             // If any of the secondary fields are nullable, then add a select op that filters nulls.
             AlgebricksMetaOperatorDescriptor selectOp = null;
             if (anySecondaryKeyIsNullable || isEnforcingKeyTypes) {
-                selectOp = createFilterNullsSelectOp(spec, numSecondaryKeys, secondaryRecDesc);
+                selectOp = createFilterNullsSelectOp(spec, index.getKeyFieldNames().size(), secondaryRecDesc);
             }
 
             // Sort by secondary keys.
             ExternalSortOperatorDescriptor sortOp = createSortOp(spec, secondaryComparatorFactories, secondaryRecDesc);
-
-            StorageProperties storageProperties = propertiesProvider.getStorageProperties();
             // Create secondary BTree bulk load op.
             AbstractTreeIndexOperatorDescriptor secondaryBulkLoadOp;
-            ExternalBTreeWithBuddyDataflowHelperFactory dataflowHelperFactory =
-                    new ExternalBTreeWithBuddyDataflowHelperFactory(mergePolicyFactory, mergePolicyFactoryProperties,
-                            new SecondaryIndexOperationTrackerProvider(dataset.getDatasetId()),
-                            RuntimeComponentsProvider.RUNTIME_PROVIDER,
-                            LSMBTreeWithBuddyIOOperationCallbackFactory.INSTANCE,
-                            storageProperties.getBloomFilterFalsePositiveRate(), new int[] { numSecondaryKeys },
-                            ExternalDatasetsRegistry.INSTANCE.getDatasetVersion(dataset), true);
             IOperatorDescriptor root;
             if (externalFiles != null) {
                 // Transaction load
@@ -215,29 +197,19 @@
                 spec.connect(new OneToOneConnectorDescriptor(spec), primaryScanOp, 0, sourceOp, 0);
             }
             AlgebricksMetaOperatorDescriptor asterixAssignOp =
-                    createAssignOp(spec, sourceOp, numSecondaryKeys, secondaryRecDesc);
+                    createAssignOp(spec, index.getKeyFieldNames().size(), secondaryRecDesc);
 
             // If any of the secondary fields are nullable, then add a select op that filters nulls.
             AlgebricksMetaOperatorDescriptor selectOp = null;
             if (anySecondaryKeyIsNullable || isEnforcingKeyTypes) {
-                selectOp = createFilterNullsSelectOp(spec, numSecondaryKeys, secondaryRecDesc);
+                selectOp = createFilterNullsSelectOp(spec, index.getKeyFieldNames().size(), secondaryRecDesc);
             }
 
             // Sort by secondary keys.
             ExternalSortOperatorDescriptor sortOp = createSortOp(spec, secondaryComparatorFactories, secondaryRecDesc);
-
-            StorageProperties storageProperties = propertiesProvider.getStorageProperties();
-            boolean temp = dataset.getDatasetDetails().isTemp();
             // Create secondary BTree bulk load op.
             TreeIndexBulkLoadOperatorDescriptor secondaryBulkLoadOp = createTreeIndexBulkLoadOp(spec, fieldPermutation,
-                    new LSMBTreeDataflowHelperFactory(new AsterixVirtualBufferCacheProvider(dataset.getDatasetId()),
-                            mergePolicyFactory, mergePolicyFactoryProperties,
-                            new SecondaryIndexOperationTrackerProvider(dataset.getDatasetId()),
-                            RuntimeComponentsProvider.RUNTIME_PROVIDER,
-                            LSMBTreeIOOperationCallbackFactory.INSTANCE,
-                            storageProperties.getBloomFilterFalsePositiveRate(), false, filterTypeTraits,
-                            filterCmpFactories, secondaryBTreeFields, secondaryFilterFields, !temp),
-                    GlobalConfig.DEFAULT_TREE_FILL_FACTOR);
+                    dataflowHelperFactory, GlobalConfig.DEFAULT_TREE_FILL_FACTOR);
 
             AlgebricksMetaOperatorDescriptor metaOp = new AlgebricksMetaOperatorDescriptor(spec, 1, 0,
                     new IPushRuntimeFactory[] { new SinkRuntimeFactory() },
@@ -261,43 +233,24 @@
 
     @Override
     protected int getNumSecondaryKeys() {
-        return numSecondaryKeys;
+        return index.getKeyFieldNames().size();
     }
 
     @Override
-    public JobSpecification buildCompactJobSpec() throws AsterixException, AlgebricksException {
-        JobSpecification spec = JobSpecificationUtils.createJobSpecification();
-
-        StorageProperties storageProperties = propertiesProvider.getStorageProperties();
-        boolean temp = dataset.getDatasetDetails().isTemp();
+    public JobSpecification buildCompactJobSpec() throws AlgebricksException {
+        JobSpecification spec = RuntimeUtils.createJobSpecification();
         LSMTreeIndexCompactOperatorDescriptor compactOp;
-        if (dataset.getDatasetType() == DatasetType.INTERNAL) {
-            compactOp = new LSMTreeIndexCompactOperatorDescriptor(spec,
-                    RuntimeComponentsProvider.RUNTIME_PROVIDER,
-                    RuntimeComponentsProvider.RUNTIME_PROVIDER, secondaryFileSplitProvider, secondaryTypeTraits,
-                    secondaryComparatorFactories, secondaryBloomFilterKeyFields,
-                    new LSMBTreeDataflowHelperFactory(new AsterixVirtualBufferCacheProvider(dataset.getDatasetId()),
-                            mergePolicyFactory, mergePolicyFactoryProperties,
-                            new SecondaryIndexOperationTrackerProvider(dataset.getDatasetId()),
-                            RuntimeComponentsProvider.RUNTIME_PROVIDER,
-                            LSMBTreeIOOperationCallbackFactory.INSTANCE,
-                            storageProperties.getBloomFilterFalsePositiveRate(), false, filterTypeTraits,
-                            filterCmpFactories, secondaryBTreeFields, secondaryFilterFields, !temp),
-                    NoOpOperationCallbackFactory.INSTANCE, LSMIndexUtil.getMetadataPageManagerFactory());
-        } else {
-            // External dataset
-            compactOp = new LSMTreeIndexCompactOperatorDescriptor(spec,
-                    RuntimeComponentsProvider.RUNTIME_PROVIDER,
-                    RuntimeComponentsProvider.RUNTIME_PROVIDER, secondaryFileSplitProvider, secondaryTypeTraits,
-                    secondaryComparatorFactories, secondaryBloomFilterKeyFields,
-                    new ExternalBTreeWithBuddyDataflowHelperFactory(mergePolicyFactory, mergePolicyFactoryProperties,
-                            new SecondaryIndexOperationTrackerProvider(dataset.getDatasetId()),
-                            RuntimeComponentsProvider.RUNTIME_PROVIDER,
-                            LSMBTreeWithBuddyIOOperationCallbackFactory.INSTANCE,
-                            storageProperties.getBloomFilterFalsePositiveRate(), new int[] { numSecondaryKeys },
-                            ExternalDatasetsRegistry.INSTANCE.getDatasetVersion(dataset), true),
-                    NoOpOperationCallbackFactory.INSTANCE, LSMIndexUtil.getMetadataPageManagerFactory());
-        }
+        IIndexDataflowHelperFactory dataflowHelperFactory = dataset.getIndexDataflowHelperFactory(metadataProvider,
+                index, itemType, metaType, mergePolicyFactory, mergePolicyFactoryProperties);
+        IStorageComponentProvider storageComponentProvider = metadataProvider.getStorageComponentProvider();
+        compactOp =
+                new LSMTreeIndexCompactOperatorDescriptor(spec, storageComponentProvider.getStorageManager(),
+                        storageComponentProvider.getIndexLifecycleManagerProvider(), secondaryFileSplitProvider,
+                        secondaryTypeTraits, secondaryComparatorFactories, secondaryBloomFilterKeyFields,
+                        dataflowHelperFactory,
+                        dataset.getModificationCallbackFactory(storageComponentProvider, index, null,
+                                IndexOperation.FULL_MERGE, null),
+                        storageComponentProvider.getMetadataPageManagerFactory());
         AlgebricksPartitionConstraintHelper.setPartitionConstraintInJobSpec(spec, compactOp,
                 secondaryPartitionConstraint);
         spec.addRoot(compactOp);
@@ -307,9 +260,8 @@
 
     @Override
     @SuppressWarnings("rawtypes")
-    protected void setSecondaryRecDescAndComparators(IndexType indexType, List<List<String>> secondaryKeyFields,
-            List<IAType> secondaryKeyTypes, int gramLength, MetadataProvider metadataProvider)
-            throws AlgebricksException, AsterixException {
+    protected void setSecondaryRecDescAndComparators() throws AlgebricksException {
+        int numSecondaryKeys = index.getKeyFieldNames().size();
         secondaryFieldAccessEvalFactories = new IScalarEvaluatorFactory[numSecondaryKeys + numFilterFields];
         secondaryComparatorFactories = new IBinaryComparatorFactory[numSecondaryKeys + numPrimaryKeys];
         secondaryBloomFilterKeyFields = new int[numSecondaryKeys];
@@ -326,9 +278,11 @@
                 metadataProvider.getFormat().getBinaryComparatorFactoryProvider();
         // Record column is 0 for external datasets, numPrimaryKeys for internal ones
         int recordColumn = dataset.getDatasetType() == DatasetType.INTERNAL ? numPrimaryKeys : 0;
+        boolean isEnforcingKeyTypes = index.isEnforcingKeyFileds();
         for (int i = 0; i < numSecondaryKeys; i++) {
             ARecordType sourceType;
             int sourceColumn;
+            List<Integer> keySourceIndicators = index.getKeyFieldSourceIndicators();
             if (keySourceIndicators == null || keySourceIndicators.get(i) == 0) {
                 sourceType = itemType;
                 sourceColumn = recordColumn;
@@ -337,9 +291,10 @@
                 sourceColumn = recordColumn + 1;
             }
             secondaryFieldAccessEvalFactories[i] = metadataProvider.getFormat().getFieldAccessEvaluatorFactory(
-                    isEnforcingKeyTypes ? enforcedItemType : sourceType, secondaryKeyFields.get(i), sourceColumn);
-            Pair<IAType, Boolean> keyTypePair =
-                    Index.getNonNullableOpenFieldType(secondaryKeyTypes.get(i), secondaryKeyFields.get(i), sourceType);
+                    isEnforcingKeyTypes ? enforcedItemType : sourceType, index.getKeyFieldNames().get(i),
+                    sourceColumn);
+            Pair<IAType, Boolean> keyTypePair = Index.getNonNullableOpenFieldType(index.getKeyFieldTypes().get(i),
+                    index.getKeyFieldNames().get(i), sourceType);
             IAType keyType = keyTypePair.first;
             anySecondaryKeyIsNullable = anySecondaryKeyIsNullable || keyTypePair.second;
             ISerializerDeserializer keySerde = serdeProvider.getSerializerDeserializer(keyType);
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/file/SecondaryIndexOperationsHelper.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryIndexOperationsHelper.java
similarity index 73%
rename from asterixdb/asterix-app/src/main/java/org/apache/asterix/file/SecondaryIndexOperationsHelper.java
rename to asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryIndexOperationsHelper.java
index c362e2e..f7e569c 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/file/SecondaryIndexOperationsHelper.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryIndexOperationsHelper.java
@@ -17,42 +17,35 @@
  * under the License.
  */
 
-package org.apache.asterix.file;
+package org.apache.asterix.metadata.utils;
 
 import java.io.DataOutput;
 import java.util.List;
 import java.util.Map;
 
-import org.apache.asterix.app.external.ExternalIndexingOperations;
-import org.apache.asterix.common.config.StorageProperties;
 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.IPropertiesProvider;
-import org.apache.asterix.common.context.AsterixVirtualBufferCacheProvider;
 import org.apache.asterix.common.context.ITransactionSubsystemProvider;
 import org.apache.asterix.common.context.TransactionSubsystemProvider;
-import org.apache.asterix.common.dataflow.LSMIndexUtil;
 import org.apache.asterix.common.exceptions.AsterixException;
-import org.apache.asterix.common.ioopcallbacks.LSMBTreeIOOperationCallbackFactory;
-import org.apache.asterix.metadata.declared.MetadataProvider;
-import org.apache.asterix.runtime.util.AppContextInfo;
-import org.apache.asterix.runtime.util.RuntimeComponentsProvider;
+import org.apache.asterix.common.exceptions.CompilationException;
+import org.apache.asterix.common.exceptions.ErrorCode;
 import org.apache.asterix.common.transactions.IRecoveryManager.ResourceType;
 import org.apache.asterix.common.transactions.JobId;
 import org.apache.asterix.external.indexing.ExternalFile;
 import org.apache.asterix.external.indexing.IndexingConstants;
-import org.apache.asterix.external.operators.ExternalDataScanOperatorDescriptor;
 import org.apache.asterix.external.operators.ExternalIndexBulkModifyOperatorDescriptor;
+import org.apache.asterix.external.operators.ExternalScanOperatorDescriptor;
 import org.apache.asterix.formats.nontagged.BinaryBooleanInspector;
 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.metadata.MetadataException;
+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.Index;
 import org.apache.asterix.metadata.entities.InternalDatasetDetails;
-import org.apache.asterix.metadata.utils.DatasetUtils;
 import org.apache.asterix.om.types.ARecordType;
 import org.apache.asterix.om.types.IAType;
 import org.apache.asterix.runtime.evaluators.functions.AndDescriptor;
@@ -60,8 +53,9 @@
 import org.apache.asterix.runtime.evaluators.functions.IsUnknownDescriptor;
 import org.apache.asterix.runtime.evaluators.functions.NotDescriptor;
 import org.apache.asterix.runtime.job.listener.JobEventListenerFactory;
+import org.apache.asterix.runtime.utils.AppContextInfo;
+import org.apache.asterix.runtime.utils.RuntimeComponentsProvider;
 import org.apache.asterix.transaction.management.opcallbacks.PrimaryIndexInstantSearchOperationCallbackFactory;
-import org.apache.asterix.transaction.management.opcallbacks.PrimaryIndexOperationTrackerProvider;
 import org.apache.asterix.transaction.management.service.transaction.JobIdFactory;
 import org.apache.hyracks.algebricks.common.constraints.AlgebricksPartitionConstraint;
 import org.apache.hyracks.algebricks.common.constraints.AlgebricksPartitionConstraintHelper;
@@ -93,7 +87,6 @@
 import org.apache.hyracks.storage.am.common.dataflow.IIndexDataflowHelperFactory;
 import org.apache.hyracks.storage.am.common.dataflow.TreeIndexBulkLoadOperatorDescriptor;
 import org.apache.hyracks.storage.am.common.impls.NoOpOperationCallbackFactory;
-import org.apache.hyracks.storage.am.lsm.btree.dataflow.LSMBTreeDataflowHelperFactory;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMMergePolicyFactory;
 
 @SuppressWarnings("rawtypes")
@@ -102,26 +95,20 @@
 // not just for creation.
 public abstract class SecondaryIndexOperationsHelper {
     protected final PhysicalOptimizationConfig physOptConf;
-
-    protected int numPrimaryKeys;
-    protected int numSecondaryKeys;
-    protected MetadataProvider metadataProvider;
-    protected String dataverseName;
-    protected String datasetName;
-    protected Dataset dataset;
-    protected ARecordType itemType;
-    protected ARecordType metaType;
-    protected List<Integer> keySourceIndicators;
+    protected final MetadataProvider metadataProvider;
+    protected final Dataset dataset;
+    protected final Index index;
+    protected final ARecordType itemType;
+    protected final ARecordType metaType;
+    protected final ARecordType enforcedItemType;
+    protected final ARecordType enforcedMetaType;
     protected ISerializerDeserializer metaSerde;
     protected ISerializerDeserializer payloadSerde;
     protected IFileSplitProvider primaryFileSplitProvider;
     protected AlgebricksPartitionConstraint primaryPartitionConstraint;
     protected IFileSplitProvider secondaryFileSplitProvider;
     protected AlgebricksPartitionConstraint secondaryPartitionConstraint;
-    protected String secondaryIndexName;
     protected boolean anySecondaryKeyIsNullable = false;
-    protected boolean isEnforcingKeyTypes = false;
-
     protected long numElementsHint;
     protected IBinaryComparatorFactory[] primaryComparatorFactories;
     protected int[] primaryBloomFilterKeyFields;
@@ -131,13 +118,10 @@
     protected int[] secondaryBloomFilterKeyFields;
     protected RecordDescriptor secondaryRecDesc;
     protected IScalarEvaluatorFactory[] secondaryFieldAccessEvalFactories;
-
     protected IPropertiesProvider propertiesProvider;
     protected ILSMMergePolicyFactory mergePolicyFactory;
     protected Map<String, String> mergePolicyFactoryProperties;
     protected RecordDescriptor enforcedRecDesc;
-    protected ARecordType enforcedItemType;
-    protected ARecordType enforcedMetaType;
     protected int numFilterFields;
     protected List<String> filterFieldName;
     protected ITypeTraits[] filterTypeTraits;
@@ -147,106 +131,90 @@
     protected int[] primaryBTreeFields;
     protected int[] secondaryBTreeFields;
     protected List<ExternalFile> externalFiles;
+    protected int numPrimaryKeys;
 
     // Prevent public construction. Should be created via createIndexCreator().
-    protected SecondaryIndexOperationsHelper(PhysicalOptimizationConfig physOptConf,
-            IPropertiesProvider propertiesProvider) {
+    protected SecondaryIndexOperationsHelper(Dataset dataset, Index index, PhysicalOptimizationConfig physOptConf,
+            IPropertiesProvider propertiesProvider, MetadataProvider metadataProvider, ARecordType recType,
+            ARecordType metaType, ARecordType enforcedType, ARecordType enforcedMetaType) {
+        this.dataset = dataset;
+        this.index = index;
         this.physOptConf = physOptConf;
         this.propertiesProvider = propertiesProvider;
+        this.metadataProvider = metadataProvider;
+        this.itemType = recType;
+        this.metaType = metaType;
+        this.enforcedItemType = enforcedType;
+        this.enforcedMetaType = enforcedMetaType;
     }
 
-    public static SecondaryIndexOperationsHelper createIndexOperationsHelper(IndexType indexType, String dataverseName,
-            String datasetName, String indexName, List<List<String>> secondaryKeyFields, List<IAType> secondaryKeyTypes,
-            boolean isEnforced, int gramLength, MetadataProvider metadataProvider,
-            PhysicalOptimizationConfig physOptConf, ARecordType recType, ARecordType metaType,
-            List<Integer> keySourceIndicators, ARecordType enforcedType) throws AsterixException, AlgebricksException {
+    public static SecondaryIndexOperationsHelper createIndexOperationsHelper(Dataset dataset, Index index,
+            MetadataProvider metadataProvider, PhysicalOptimizationConfig physOptConf, ARecordType recType,
+            ARecordType metaType, ARecordType enforcedType, ARecordType enforcedMetaType) throws AlgebricksException {
         IPropertiesProvider asterixPropertiesProvider = AppContextInfo.INSTANCE;
-        SecondaryIndexOperationsHelper indexOperationsHelper = null;
-        switch (indexType) {
-            case BTREE: {
-                indexOperationsHelper = new SecondaryBTreeOperationsHelper(physOptConf, asterixPropertiesProvider);
+        SecondaryIndexOperationsHelper indexOperationsHelper;
+        switch (index.getIndexType()) {
+            case BTREE:
+                indexOperationsHelper =
+                        new SecondaryBTreeOperationsHelper(dataset, index, physOptConf, asterixPropertiesProvider,
+                                metadataProvider, recType, metaType, enforcedType, enforcedMetaType);
                 break;
-            }
-            case RTREE: {
-                indexOperationsHelper = new SecondaryRTreeOperationsHelper(physOptConf, asterixPropertiesProvider);
+            case RTREE:
+                indexOperationsHelper =
+                        new SecondaryRTreeOperationsHelper(dataset, index, physOptConf, asterixPropertiesProvider,
+                                metadataProvider, recType, metaType, enforcedType, enforcedMetaType);
                 break;
-            }
             case SINGLE_PARTITION_WORD_INVIX:
             case SINGLE_PARTITION_NGRAM_INVIX:
             case LENGTH_PARTITIONED_WORD_INVIX:
-            case LENGTH_PARTITIONED_NGRAM_INVIX: {
-                indexOperationsHelper =
-                        new SecondaryInvertedIndexOperationsHelper(physOptConf, asterixPropertiesProvider);
+            case LENGTH_PARTITIONED_NGRAM_INVIX:
+                indexOperationsHelper = new SecondaryInvertedIndexOperationsHelper(dataset, index, physOptConf,
+                        asterixPropertiesProvider, metadataProvider, recType, metaType, enforcedType,
+                        enforcedMetaType);
                 break;
-            }
-            default: {
-                throw new AsterixException("Unknown Index Type: " + indexType);
-            }
+            default:
+                throw new CompilationException(ErrorCode.COMPILATION_UNKNOWN_INDEX_TYPE, index.getIndexType());
         }
-        indexOperationsHelper.init(indexType, dataverseName, datasetName, indexName, secondaryKeyFields,
-                secondaryKeyTypes, isEnforced, gramLength, metadataProvider, recType, metaType, keySourceIndicators,
-                enforcedType);
+        indexOperationsHelper.init();
         return indexOperationsHelper;
     }
 
-    public abstract JobSpecification buildCreationJobSpec() throws AsterixException, AlgebricksException;
+    public abstract JobSpecification buildCreationJobSpec() throws AlgebricksException;
 
-    public abstract JobSpecification buildLoadingJobSpec() throws AsterixException, AlgebricksException;
+    public abstract JobSpecification buildLoadingJobSpec() throws AlgebricksException;
 
-    public abstract JobSpecification buildCompactJobSpec() throws AsterixException, AlgebricksException;
+    public abstract JobSpecification buildCompactJobSpec() throws AlgebricksException;
 
-    protected void init(IndexType indexType, String dvn, String dsn, String in, List<List<String>> secondaryKeyFields,
-            List<IAType> secondaryKeyTypes, boolean isEnforced, int gramLength, MetadataProvider metadataProvider,
-            ARecordType aRecType, ARecordType metaType, List<Integer> keySourceIndicators, ARecordType enforcedType)
-            throws AsterixException, AlgebricksException {
-        this.metadataProvider = metadataProvider;
-        dataverseName = dvn == null ? metadataProvider.getDefaultDataverseName() : dvn;
-        datasetName = dsn;
-        secondaryIndexName = in;
-        isEnforcingKeyTypes = isEnforced;
-        dataset = metadataProvider.findDataset(dataverseName, datasetName);
-        if (dataset == null) {
-            throw new AsterixException("Unknown dataset " + datasetName);
-        }
-        boolean temp = dataset.getDatasetDetails().isTemp();
-        itemType = aRecType;
-        this.metaType = metaType;
-        this.keySourceIndicators = keySourceIndicators;
-        enforcedItemType = enforcedType;
+    protected void init() throws AlgebricksException {
         payloadSerde = SerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(itemType);
-        metaSerde = metaType == null ? null
-                : SerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(metaType);
-        numSecondaryKeys = secondaryKeyFields.size();
-        Pair<IFileSplitProvider, AlgebricksPartitionConstraint> secondarySplitsAndConstraint = metadataProvider
-                .splitProviderAndPartitionConstraintsForDataset(dataverseName, datasetName, secondaryIndexName, temp);
+        metaSerde =
+                metaType == null ? null : SerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(metaType);
+        Pair<IFileSplitProvider, AlgebricksPartitionConstraint> secondarySplitsAndConstraint =
+                metadataProvider.getSplitProviderAndConstraints(index.getDataverseName(), index.getDatasetName(),
+                        index.getIndexName(), dataset.getDatasetDetails().isTemp());
         secondaryFileSplitProvider = secondarySplitsAndConstraint.first;
         secondaryPartitionConstraint = secondarySplitsAndConstraint.second;
-
-        if (dataset.getDatasetType() == DatasetType.EXTERNAL) {
-            numPrimaryKeys = ExternalIndexingOperations.getRIDSize(dataset);
-        } else {
-            filterFieldName = DatasetUtils.getFilterField(dataset);
+        numPrimaryKeys = DatasetUtil.getPartitioningKeys(dataset).size();
+        if (dataset.getDatasetType() == DatasetType.INTERNAL) {
+            filterFieldName = DatasetUtil.getFilterField(dataset);
             if (filterFieldName != null) {
                 numFilterFields = 1;
             } else {
                 numFilterFields = 0;
             }
-
-            numPrimaryKeys = DatasetUtils.getPartitioningKeys(dataset).size();
-            Pair<IFileSplitProvider, AlgebricksPartitionConstraint> primarySplitsAndConstraint = metadataProvider
-                    .splitProviderAndPartitionConstraintsForDataset(dataverseName, datasetName, datasetName, temp);
+            Pair<IFileSplitProvider, AlgebricksPartitionConstraint> primarySplitsAndConstraint =
+                    metadataProvider.getSplitProviderAndConstraints(dataset.getDataverseName(),
+                            dataset.getDatasetName(), dataset.getDatasetName(), dataset.getDatasetDetails().isTemp());
             primaryFileSplitProvider = primarySplitsAndConstraint.first;
             primaryPartitionConstraint = primarySplitsAndConstraint.second;
             setPrimaryRecDescAndComparators();
         }
-        setSecondaryRecDescAndComparators(indexType, secondaryKeyFields, secondaryKeyTypes, gramLength,
-                metadataProvider);
+        setSecondaryRecDescAndComparators();
         numElementsHint = metadataProvider.getCardinalityPerPartitionHint(dataset);
         Pair<ILSMMergePolicyFactory, Map<String, String>> compactionInfo =
-                DatasetUtils.getMergePolicyFactory(dataset, metadataProvider.getMetadataTxnContext());
+                DatasetUtil.getMergePolicyFactory(dataset, metadataProvider.getMetadataTxnContext());
         mergePolicyFactory = compactionInfo.first;
         mergePolicyFactoryProperties = compactionInfo.second;
-
         if (numFilterFields > 0) {
             setFilterTypeTraitsAndComparators();
         }
@@ -258,7 +226,7 @@
         secondaryFilterFields = new int[numFilterFields];
         primaryFilterFields = new int[numFilterFields];
         primaryBTreeFields = new int[numPrimaryKeys + 1];
-        secondaryBTreeFields = new int[numSecondaryKeys + numPrimaryKeys];
+        secondaryBTreeFields = new int[index.getKeyFieldNames().size() + numPrimaryKeys];
         for (int i = 0; i < primaryBTreeFields.length; i++) {
             primaryBTreeFields[i] = i;
         }
@@ -276,8 +244,7 @@
     protected abstract int getNumSecondaryKeys();
 
     protected void setPrimaryRecDescAndComparators() throws AlgebricksException {
-        List<List<String>> partitioningKeys = DatasetUtils.getPartitioningKeys(dataset);
-        int numPrimaryKeys = partitioningKeys.size();
+        List<List<String>> partitioningKeys = DatasetUtil.getPartitioningKeys(dataset);
         ISerializerDeserializer[] primaryRecFields =
                 new ISerializerDeserializer[numPrimaryKeys + 1 + (dataset.hasMetaPart() ? 1 : 0)];
         ITypeTraits[] primaryTypeTraits = new ITypeTraits[numPrimaryKeys + 1 + (dataset.hasMetaPart() ? 1 : 0)];
@@ -307,12 +274,9 @@
         primaryRecDesc = new RecordDescriptor(primaryRecFields, primaryTypeTraits);
     }
 
-    protected abstract void setSecondaryRecDescAndComparators(IndexType indexType,
-            List<List<String>> secondaryKeyFields, List<IAType> secondaryKeyTypes, int gramLength,
-            MetadataProvider metadataProvider) throws AlgebricksException, AsterixException;
+    protected abstract void setSecondaryRecDescAndComparators() throws AlgebricksException;
 
-    protected AbstractOperatorDescriptor createDummyKeyProviderOp(JobSpecification spec)
-            throws AsterixException, AlgebricksException {
+    protected AbstractOperatorDescriptor createDummyKeyProviderOp(JobSpecification spec) throws AlgebricksException {
         // Build dummy tuple containing one field with a dummy value inside.
         ArrayTupleBuilder tb = new ArrayTupleBuilder(1);
         DataOutput dos = tb.getDataOutput();
@@ -334,44 +298,41 @@
         return keyProviderOp;
     }
 
-    protected BTreeSearchOperatorDescriptor createPrimaryIndexScanOp(JobSpecification spec) throws AlgebricksException {
+    protected BTreeSearchOperatorDescriptor createPrimaryIndexScanOp(JobSpecification spec)
+            throws AlgebricksException {
         // -Infinity
         int[] lowKeyFields = null;
         // +Infinity
         int[] highKeyFields = null;
-        ITransactionSubsystemProvider txnSubsystemProvider = new TransactionSubsystemProvider();
+        ITransactionSubsystemProvider txnSubsystemProvider = TransactionSubsystemProvider.INSTANCE;
         JobId jobId = JobIdFactory.generateJobId();
         metadataProvider.setJobId(jobId);
         boolean isWriteTransaction = metadataProvider.isWriteTransaction();
         IJobletEventListenerFactory jobEventListenerFactory = new JobEventListenerFactory(jobId, isWriteTransaction);
         spec.setJobletEventListenerFactory(jobEventListenerFactory);
+        Index primaryIndex = MetadataManager.INSTANCE.getIndex(metadataProvider.getMetadataTxnContext(),
+                dataset.getDataverseName(), dataset.getDatasetName(), dataset.getDatasetName());
 
         boolean temp = dataset.getDatasetDetails().isTemp();
         ISearchOperationCallbackFactory searchCallbackFactory = temp ? NoOpOperationCallbackFactory.INSTANCE
                 : new PrimaryIndexInstantSearchOperationCallbackFactory(jobId, dataset.getDatasetId(),
                         primaryBloomFilterKeyFields, txnSubsystemProvider, ResourceType.LSM_BTREE);
-        StorageProperties storageProperties = propertiesProvider.getStorageProperties();
         BTreeSearchOperatorDescriptor primarySearchOp = new BTreeSearchOperatorDescriptor(spec, primaryRecDesc,
                 RuntimeComponentsProvider.RUNTIME_PROVIDER, RuntimeComponentsProvider.RUNTIME_PROVIDER,
                 primaryFileSplitProvider, primaryRecDesc.getTypeTraits(), primaryComparatorFactories,
                 primaryBloomFilterKeyFields, lowKeyFields, highKeyFields, true, true,
-                new LSMBTreeDataflowHelperFactory(new AsterixVirtualBufferCacheProvider(dataset.getDatasetId()),
-                        mergePolicyFactory, mergePolicyFactoryProperties,
-                        new PrimaryIndexOperationTrackerProvider(dataset.getDatasetId()),
-                        RuntimeComponentsProvider.RUNTIME_PROVIDER, LSMBTreeIOOperationCallbackFactory.INSTANCE,
-                        storageProperties.getBloomFilterFalsePositiveRate(), true, filterTypeTraits, filterCmpFactories,
-                        primaryBTreeFields, primaryFilterFields, !temp),
-                false, false, null, searchCallbackFactory, null, null, LSMIndexUtil
-                        .getMetadataPageManagerFactory());
+                dataset.getIndexDataflowHelperFactory(metadataProvider, primaryIndex, itemType, metaType,
+                        mergePolicyFactory, mergePolicyFactoryProperties),
+                false, false, null, searchCallbackFactory, null, null,
+                metadataProvider.getStorageComponentProvider().getMetadataPageManagerFactory());
 
         AlgebricksPartitionConstraintHelper.setPartitionConstraintInJobSpec(spec, primarySearchOp,
                 primaryPartitionConstraint);
         return primarySearchOp;
     }
 
-    protected AlgebricksMetaOperatorDescriptor createAssignOp(JobSpecification spec,
-            AbstractOperatorDescriptor primaryScanOp, int numSecondaryKeyFields, RecordDescriptor secondaryRecDesc)
-            throws AlgebricksException {
+    protected AlgebricksMetaOperatorDescriptor createAssignOp(JobSpecification spec, int numSecondaryKeyFields,
+            RecordDescriptor secondaryRecDesc) throws AlgebricksException {
         int[] outColumns = new int[numSecondaryKeyFields + numFilterFields];
         int[] projectionList = new int[numSecondaryKeyFields + numPrimaryKeys + numFilterFields];
         for (int i = 0; i < numSecondaryKeyFields + numFilterFields; i++) {
@@ -385,7 +346,7 @@
             projectionList[projCount++] = i;
         }
         if (numFilterFields > 0) {
-            projectionList[projCount++] = numPrimaryKeys + numSecondaryKeyFields;
+            projectionList[projCount] = numPrimaryKeys + numSecondaryKeyFields;
         }
 
         IScalarEvaluatorFactory[] sefs = new IScalarEvaluatorFactory[secondaryFieldAccessEvalFactories.length];
@@ -426,10 +387,8 @@
         IScalarEvaluatorFactory[] sefs = new IScalarEvaluatorFactory[1];
         sefs[0] = castFuncDesc.createEvaluatorFactory(castEvalFact);
         AssignRuntimeFactory castAssign = new AssignRuntimeFactory(outColumns, sefs, projectionList);
-        AlgebricksMetaOperatorDescriptor castRecAssignOp = new AlgebricksMetaOperatorDescriptor(spec, 1, 1,
-                new IPushRuntimeFactory[] { castAssign }, new RecordDescriptor[] { enforcedRecDesc });
-
-        return castRecAssignOp;
+        return new AlgebricksMetaOperatorDescriptor(spec, 1, 1, new IPushRuntimeFactory[] { castAssign },
+                new RecordDescriptor[] { enforcedRecDesc });
     }
 
     protected ExternalSortOperatorDescriptor createSortOp(JobSpecification spec,
@@ -446,13 +405,13 @@
 
     protected TreeIndexBulkLoadOperatorDescriptor createTreeIndexBulkLoadOp(JobSpecification spec,
             int[] fieldPermutation, IIndexDataflowHelperFactory dataflowHelperFactory, float fillFactor)
-            throws MetadataException, AlgebricksException {
+            throws AlgebricksException {
         TreeIndexBulkLoadOperatorDescriptor treeIndexBulkLoadOp = new TreeIndexBulkLoadOperatorDescriptor(spec,
                 secondaryRecDesc, RuntimeComponentsProvider.RUNTIME_PROVIDER,
                 RuntimeComponentsProvider.RUNTIME_PROVIDER, secondaryFileSplitProvider,
                 secondaryRecDesc.getTypeTraits(), secondaryComparatorFactories, secondaryBloomFilterKeyFields,
-                fieldPermutation, fillFactor, false, numElementsHint, false, dataflowHelperFactory, LSMIndexUtil
-                        .getMetadataPageManagerFactory());
+                fieldPermutation, fillFactor, false, numElementsHint, false, dataflowHelperFactory,
+                metadataProvider.getStorageComponentProvider().getMetadataPageManagerFactory());
         AlgebricksPartitionConstraintHelper.setPartitionConstraintInJobSpec(spec, treeIndexBulkLoadOp,
                 secondaryPartitionConstraint);
         return treeIndexBulkLoadOp;
@@ -472,7 +431,7 @@
                     notDesc.createEvaluatorFactory(new IScalarEvaluatorFactory[] { isUnknownEvalFactory });
             andArgsEvalFactories[i] = notEvalFactory;
         }
-        IScalarEvaluatorFactory selectCond = null;
+        IScalarEvaluatorFactory selectCond;
         if (numSecondaryKeyFields > 1) {
             // Create conjunctive condition where all secondary index keys must
             // satisfy 'is not null'.
@@ -481,8 +440,8 @@
         } else {
             selectCond = andArgsEvalFactories[0];
         }
-        StreamSelectRuntimeFactory select = new StreamSelectRuntimeFactory(selectCond, null,
-                BinaryBooleanInspector.FACTORY, false, -1, null);
+        StreamSelectRuntimeFactory select =
+                new StreamSelectRuntimeFactory(selectCond, null, BinaryBooleanInspector.FACTORY, false, -1, null);
         AlgebricksMetaOperatorDescriptor asterixSelectOp = new AlgebricksMetaOperatorDescriptor(spec, 1, 1,
                 new IPushRuntimeFactory[] { select }, new RecordDescriptor[] { secondaryRecDesc });
         AlgebricksPartitionConstraintHelper.setPartitionConstraintInJobSpec(spec, asterixSelectOp,
@@ -491,8 +450,8 @@
     }
 
     // This method creates a source indexing operator for external data
-    protected ExternalDataScanOperatorDescriptor createExternalIndexingOp(JobSpecification spec)
-            throws AlgebricksException, AsterixException {
+    protected ExternalScanOperatorDescriptor createExternalIndexingOp(JobSpecification spec)
+            throws AlgebricksException {
         // A record + primary keys
         ISerializerDeserializer[] serdes = new ISerializerDeserializer[1 + numPrimaryKeys];
         ITypeTraits[] typeTraits = new ITypeTraits[1 + numPrimaryKeys];
@@ -508,7 +467,7 @@
         RecordDescriptor indexerDesc = new RecordDescriptor(serdes, typeTraits);
 
         // Create the operator and its partition constraits
-        Pair<ExternalDataScanOperatorDescriptor, AlgebricksPartitionConstraint> indexingOpAndConstraints;
+        Pair<ExternalScanOperatorDescriptor, AlgebricksPartitionConstraint> indexingOpAndConstraints;
         try {
             indexingOpAndConstraints = ExternalIndexingOperations.createExternalIndexingOp(spec, metadataProvider,
                     dataset, itemType, indexerDesc, externalFiles);
@@ -542,34 +501,33 @@
         }
 
         AssignRuntimeFactory assign = new AssignRuntimeFactory(outColumns, sefs, projectionList);
-        AlgebricksMetaOperatorDescriptor asterixAssignOp = new AlgebricksMetaOperatorDescriptor(spec, 1, 1,
-                new IPushRuntimeFactory[] { assign }, new RecordDescriptor[] { secondaryRecDesc });
-        return asterixAssignOp;
+        return new AlgebricksMetaOperatorDescriptor(spec, 1, 1, new IPushRuntimeFactory[] { assign },
+                new RecordDescriptor[] { secondaryRecDesc });
     }
 
     protected ExternalIndexBulkModifyOperatorDescriptor createExternalIndexBulkModifyOp(JobSpecification spec,
             int[] fieldPermutation, IIndexDataflowHelperFactory dataflowHelperFactory, float fillFactor)
-            throws MetadataException, AlgebricksException {
+            throws AlgebricksException {
         // create a list of file ids
         int numOfDeletedFiles = 0;
         for (ExternalFile file : externalFiles) {
-            if (file.getPendingOp() == ExternalFilePendingOp.PENDING_DROP_OP) {
+            if (file.getPendingOp() == ExternalFilePendingOp.DROP_OP) {
                 numOfDeletedFiles++;
             }
         }
         int[] deletedFiles = new int[numOfDeletedFiles];
         int i = 0;
         for (ExternalFile file : externalFiles) {
-            if (file.getPendingOp() == ExternalFilePendingOp.PENDING_DROP_OP) {
+            if (file.getPendingOp() == ExternalFilePendingOp.DROP_OP) {
                 deletedFiles[i] = file.getFileNumber();
             }
         }
         ExternalIndexBulkModifyOperatorDescriptor treeIndexBulkLoadOp = new ExternalIndexBulkModifyOperatorDescriptor(
-                spec, RuntimeComponentsProvider.RUNTIME_PROVIDER,
-                RuntimeComponentsProvider.RUNTIME_PROVIDER, secondaryFileSplitProvider, secondaryTypeTraits,
-                secondaryComparatorFactories, secondaryBloomFilterKeyFields, dataflowHelperFactory,
-                NoOpOperationCallbackFactory.INSTANCE, deletedFiles, fieldPermutation, fillFactor, numElementsHint,
-                LSMIndexUtil.getMetadataPageManagerFactory());
+                spec, RuntimeComponentsProvider.RUNTIME_PROVIDER, RuntimeComponentsProvider.RUNTIME_PROVIDER,
+                secondaryFileSplitProvider, secondaryTypeTraits, secondaryComparatorFactories,
+                secondaryBloomFilterKeyFields, dataflowHelperFactory, NoOpOperationCallbackFactory.INSTANCE,
+                deletedFiles, fieldPermutation, fillFactor, numElementsHint,
+                metadataProvider.getStorageComponentProvider().getMetadataPageManagerFactory());
         AlgebricksPartitionConstraintHelper.setPartitionConstraintInJobSpec(spec, treeIndexBulkLoadOp,
                 secondaryPartitionConstraint);
         return treeIndexBulkLoadOp;
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/file/SecondaryInvertedIndexOperationsHelper.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryInvertedIndexOperationsHelper.java
similarity index 75%
rename from asterixdb/asterix-app/src/main/java/org/apache/asterix/file/SecondaryInvertedIndexOperationsHelper.java
rename to asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryInvertedIndexOperationsHelper.java
index 3f5a9e9..b86004a 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/file/SecondaryInvertedIndexOperationsHelper.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryInvertedIndexOperationsHelper.java
@@ -16,25 +16,24 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.asterix.file;
+package org.apache.asterix.metadata.utils;
 
-import java.util.List;
-
-import org.apache.asterix.common.config.StorageProperties;
 import org.apache.asterix.common.config.DatasetConfig.IndexType;
 import org.apache.asterix.common.config.IPropertiesProvider;
-import org.apache.asterix.common.context.AsterixVirtualBufferCacheProvider;
-import org.apache.asterix.common.dataflow.LSMIndexUtil;
+import org.apache.asterix.common.context.IStorageComponentProvider;
 import org.apache.asterix.common.exceptions.AsterixException;
-import org.apache.asterix.common.ioopcallbacks.LSMInvertedIndexIOOperationCallbackFactory;
+import org.apache.asterix.common.exceptions.CompilationException;
+import org.apache.asterix.common.exceptions.ErrorCode;
 import org.apache.asterix.common.transactions.IResourceFactory;
 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.om.types.ARecordType;
 import org.apache.asterix.om.types.IAType;
-import org.apache.asterix.om.util.NonTaggedFormatUtil;
+import org.apache.asterix.om.utils.NonTaggedFormatUtil;
+import org.apache.asterix.om.utils.RecordUtil;
 import org.apache.asterix.runtime.formats.FormatUtils;
-import org.apache.asterix.runtime.util.RuntimeComponentsProvider;
-import org.apache.asterix.transaction.management.opcallbacks.SecondaryIndexOperationTrackerProvider;
+import org.apache.asterix.runtime.utils.RuntimeUtils;
 import org.apache.asterix.transaction.management.resource.LSMInvertedIndexLocalResourceMetadataFactory;
 import org.apache.asterix.transaction.management.resource.PersistentLocalResourceFactoryProvider;
 import org.apache.hyracks.algebricks.common.constraints.AlgebricksPartitionConstraintHelper;
@@ -61,13 +60,11 @@
 import org.apache.hyracks.dataflow.std.sort.ExternalSortOperatorDescriptor;
 import org.apache.hyracks.storage.am.btree.dataflow.BTreeSearchOperatorDescriptor;
 import org.apache.hyracks.storage.am.common.dataflow.IIndexDataflowHelperFactory;
-import org.apache.hyracks.storage.am.common.impls.NoOpOperationCallbackFactory;
+import org.apache.hyracks.storage.am.common.ophelpers.IndexOperation;
 import org.apache.hyracks.storage.am.lsm.invertedindex.dataflow.BinaryTokenizerOperatorDescriptor;
 import org.apache.hyracks.storage.am.lsm.invertedindex.dataflow.LSMInvertedIndexBulkLoadOperatorDescriptor;
 import org.apache.hyracks.storage.am.lsm.invertedindex.dataflow.LSMInvertedIndexCompactOperator;
 import org.apache.hyracks.storage.am.lsm.invertedindex.dataflow.LSMInvertedIndexCreateOperatorDescriptor;
-import org.apache.hyracks.storage.am.lsm.invertedindex.dataflow.LSMInvertedIndexDataflowHelperFactory;
-import org.apache.hyracks.storage.am.lsm.invertedindex.dataflow.PartitionedLSMInvertedIndexDataflowHelperFactory;
 import org.apache.hyracks.storage.am.lsm.invertedindex.tokenizers.IBinaryTokenizerFactory;
 import org.apache.hyracks.storage.common.file.ILocalResourceFactoryProvider;
 import org.apache.hyracks.storage.common.file.LocalResource;
@@ -88,22 +85,29 @@
     private int[] invertedIndexFieldsForNonBulkLoadOps;
     private int[] secondaryFilterFieldsForNonBulkLoadOps;
 
-    protected SecondaryInvertedIndexOperationsHelper(PhysicalOptimizationConfig physOptConf,
-            IPropertiesProvider propertiesProvider) {
-        super(physOptConf, propertiesProvider);
+    protected SecondaryInvertedIndexOperationsHelper(Dataset dataset, Index index,
+            PhysicalOptimizationConfig physOptConf, IPropertiesProvider propertiesProvider,
+            MetadataProvider metadataProvider, ARecordType recType, ARecordType metaType, ARecordType enforcedType,
+            ARecordType enforcedMetaType) {
+        super(dataset, index, physOptConf, propertiesProvider, metadataProvider, recType, metaType, enforcedType,
+                enforcedMetaType);
     }
 
     @Override
     @SuppressWarnings("rawtypes")
-    protected void setSecondaryRecDescAndComparators(IndexType indexType, List<List<String>> secondaryKeyFields,
-            List<IAType> secondaryKeyTypes, int gramLength, MetadataProvider metadata)
-            throws AlgebricksException, AsterixException {
+    protected void setSecondaryRecDescAndComparators() throws AlgebricksException {
+        int numSecondaryKeys = index.getKeyFieldNames().size();
+        IndexType indexType = index.getIndexType();
+        boolean isEnforcingKeyTypes = index.isEnforcingKeyFileds();
         // Sanity checks.
         if (numPrimaryKeys > 1) {
-            throw new AsterixException("Cannot create inverted index on dataset with composite primary key.");
+            throw new CompilationException(
+                    ErrorCode.COMPILATION_ILLEGAL_INDEX_FOR_DATASET_WITH_COMPOSITE_PRIMARY_INDEX, indexType,
+                    RecordUtil.toFullyQualifiedName(dataset.getDataverseName(), dataset.getDatasetName()));
         }
         if (numSecondaryKeys > 1) {
-            throw new AsterixException("Cannot create composite inverted index on multiple fields.");
+            throw new CompilationException(ErrorCode.COMPILATION_ILLEGAL_INDEX_NUM_OF_FIELD, numSecondaryKeys,
+                    indexType, 1);
         }
         if (indexType == IndexType.LENGTH_PARTITIONED_WORD_INVIX
                 || indexType == IndexType.LENGTH_PARTITIONED_NGRAM_INVIX) {
@@ -116,16 +120,18 @@
         secondaryFieldAccessEvalFactories = new IScalarEvaluatorFactory[numSecondaryKeys + numFilterFields];
         ISerializerDeserializer[] secondaryRecFields =
                 new ISerializerDeserializer[numPrimaryKeys + numSecondaryKeys + numFilterFields];
-        ISerializerDeserializer[] enforcedRecFields = new ISerializerDeserializer[1 + numPrimaryKeys + numFilterFields];
+        ISerializerDeserializer[] enforcedRecFields =
+                new ISerializerDeserializer[1 + numPrimaryKeys + numFilterFields];
         secondaryTypeTraits = new ITypeTraits[numSecondaryKeys + numPrimaryKeys];
         ITypeTraits[] enforcedTypeTraits = new ITypeTraits[1 + numPrimaryKeys];
         ISerializerDeserializerProvider serdeProvider = FormatUtils.getDefaultFormat().getSerdeProvider();
         ITypeTraitProvider typeTraitProvider = FormatUtils.getDefaultFormat().getTypeTraitProvider();
         if (numSecondaryKeys > 0) {
             secondaryFieldAccessEvalFactories[0] = FormatUtils.getDefaultFormat().getFieldAccessEvaluatorFactory(
-                    isEnforcingKeyTypes ? enforcedItemType : itemType, secondaryKeyFields.get(0), numPrimaryKeys);
-            Pair<IAType, Boolean> keyTypePair =
-                    Index.getNonNullableOpenFieldType(secondaryKeyTypes.get(0), secondaryKeyFields.get(0), itemType);
+                    isEnforcingKeyTypes ? enforcedItemType : itemType, index.getKeyFieldNames().get(0),
+                    numPrimaryKeys);
+            Pair<IAType, Boolean> keyTypePair = Index.getNonNullableOpenFieldType(index.getKeyFieldTypes().get(0),
+                    index.getKeyFieldNames().get(0), itemType);
             secondaryKeyType = keyTypePair.first;
             anySecondaryKeyIsNullable = anySecondaryKeyIsNullable || keyTypePair.second;
             ISerializerDeserializer keySerde = serdeProvider.getSerializerDeserializer(secondaryKeyType);
@@ -155,8 +161,8 @@
         // Set tokenizer factory.
         // TODO: We might want to expose the hashing option at the AQL level,
         // and add the choice to the index metadata.
-        tokenizerFactory =
-                NonTaggedFormatUtil.getBinaryTokenizerFactory(secondaryKeyType.getTypeTag(), indexType, gramLength);
+        tokenizerFactory = NonTaggedFormatUtil.getBinaryTokenizerFactory(secondaryKeyType.getTypeTag(), indexType,
+                index.getGramLength());
         // Type traits for inverted-list elements. Inverted lists contain
         // primary keys.
         invListsTypeTraits = new ITypeTraits[numPrimaryKeys];
@@ -205,7 +211,6 @@
                 invertedIndexFieldsForNonBulkLoadOps[i] = i;
             }
         }
-
     }
 
     @Override
@@ -214,25 +219,29 @@
     }
 
     @Override
-    public JobSpecification buildCreationJobSpec() throws AsterixException, AlgebricksException {
-        JobSpecification spec = JobSpecificationUtils.createJobSpecification();
-
+    public JobSpecification buildCreationJobSpec() throws AlgebricksException {
+        JobSpecification spec = RuntimeUtils.createJobSpecification();
+        IStorageComponentProvider storageComponentProvider = metadataProvider.getStorageComponentProvider();
         //prepare a LocalResourceMetadata which will be stored in NC's local resource repository
         IResourceFactory localResourceMetadata = new LSMInvertedIndexLocalResourceMetadataFactory(invListsTypeTraits,
                 primaryComparatorFactories, tokenTypeTraits, tokenComparatorFactories, tokenizerFactory, isPartitioned,
                 dataset.getDatasetId(), mergePolicyFactory, mergePolicyFactoryProperties, filterTypeTraits,
                 filterCmpFactories, invertedIndexFields, secondaryFilterFields, secondaryFilterFieldsForNonBulkLoadOps,
-                invertedIndexFieldsForNonBulkLoadOps);
+                invertedIndexFieldsForNonBulkLoadOps, dataset.getIndexOperationTrackerFactory(index),
+                dataset.getIoOperationCallbackFactory(index),
+                storageComponentProvider.getMetadataPageManagerFactory());
         ILocalResourceFactoryProvider localResourceFactoryProvider = new PersistentLocalResourceFactoryProvider(
                 localResourceMetadata, LocalResource.LSMInvertedIndexResource);
 
         IIndexDataflowHelperFactory dataflowHelperFactory = createDataflowHelperFactory();
         LSMInvertedIndexCreateOperatorDescriptor invIndexCreateOp =
-                new LSMInvertedIndexCreateOperatorDescriptor(spec, RuntimeComponentsProvider.RUNTIME_PROVIDER,
-                        secondaryFileSplitProvider, RuntimeComponentsProvider.RUNTIME_PROVIDER, tokenTypeTraits,
-                        tokenComparatorFactories, invListsTypeTraits, primaryComparatorFactories, tokenizerFactory,
-                        dataflowHelperFactory, localResourceFactoryProvider, NoOpOperationCallbackFactory.INSTANCE,
-                        LSMIndexUtil.getMetadataPageManagerFactory());
+                new LSMInvertedIndexCreateOperatorDescriptor(spec, storageComponentProvider.getStorageManager(),
+                        secondaryFileSplitProvider, storageComponentProvider.getIndexLifecycleManagerProvider(),
+                        tokenTypeTraits, tokenComparatorFactories, invListsTypeTraits, primaryComparatorFactories,
+                        tokenizerFactory, dataflowHelperFactory, localResourceFactoryProvider,
+                        dataset.getModificationCallbackFactory(storageComponentProvider, index, null,
+                                IndexOperation.CREATE, null),
+                        storageComponentProvider.getMetadataPageManagerFactory());
         AlgebricksPartitionConstraintHelper.setPartitionConstraintInJobSpec(spec, invIndexCreateOp,
                 secondaryPartitionConstraint);
         spec.addRoot(invIndexCreateOp);
@@ -241,8 +250,8 @@
     }
 
     @Override
-    public JobSpecification buildLoadingJobSpec() throws AsterixException, AlgebricksException {
-        JobSpecification spec = JobSpecificationUtils.createJobSpecification();
+    public JobSpecification buildLoadingJobSpec() throws AlgebricksException {
+        JobSpecification spec = RuntimeUtils.createJobSpecification();
 
         // Create dummy key provider for feeding the primary index scan.
         AbstractOperatorDescriptor keyProviderOp = createDummyKeyProviderOp(spec);
@@ -251,12 +260,13 @@
         BTreeSearchOperatorDescriptor primaryScanOp = createPrimaryIndexScanOp(spec);
 
         AbstractOperatorDescriptor sourceOp = primaryScanOp;
+        boolean isEnforcingKeyTypes = index.isEnforcingKeyFileds();
+        int numSecondaryKeys = index.getKeyFieldNames().size();
         if (isEnforcingKeyTypes && !enforcedItemType.equals(itemType)) {
             sourceOp = createCastOp(spec, dataset.getDatasetType());
             spec.connect(new OneToOneConnectorDescriptor(spec), primaryScanOp, 0, sourceOp, 0);
         }
-        AlgebricksMetaOperatorDescriptor asterixAssignOp =
-                createAssignOp(spec, sourceOp, numSecondaryKeys, secondaryRecDesc);
+        AlgebricksMetaOperatorDescriptor asterixAssignOp = createAssignOp(spec, numSecondaryKeys, secondaryRecDesc);
 
         // If any of the secondary fields are nullable, then add a select op
         // that filters nulls.
@@ -296,12 +306,13 @@
 
     private AbstractOperatorDescriptor createTokenizerOp(JobSpecification spec) throws AlgebricksException {
         int docField = 0;
+        int numSecondaryKeys = index.getKeyFieldNames().size();
         int[] primaryKeyFields = new int[numPrimaryKeys + numFilterFields];
         for (int i = 0; i < primaryKeyFields.length; i++) {
             primaryKeyFields[i] = numSecondaryKeys + i;
         }
-        BinaryTokenizerOperatorDescriptor tokenizerOp = new BinaryTokenizerOperatorDescriptor(spec, tokenKeyPairRecDesc,
-                tokenizerFactory, docField, primaryKeyFields, isPartitioned, false);
+        BinaryTokenizerOperatorDescriptor tokenizerOp = new BinaryTokenizerOperatorDescriptor(spec,
+                tokenKeyPairRecDesc, tokenizerFactory, docField, primaryKeyFields, isPartitioned, false);
         AlgebricksPartitionConstraintHelper.setPartitionConstraintInJobSpec(spec, tokenizerOp,
                 primaryPartitionConstraint);
         return tokenizerOp;
@@ -321,58 +332,43 @@
         return sortOp;
     }
 
-    private LSMInvertedIndexBulkLoadOperatorDescriptor createInvertedIndexBulkLoadOp(JobSpecification spec) {
+    private LSMInvertedIndexBulkLoadOperatorDescriptor createInvertedIndexBulkLoadOp(JobSpecification spec)
+            throws AlgebricksException {
         int[] fieldPermutation = new int[numTokenKeyPairFields + numFilterFields];
         for (int i = 0; i < fieldPermutation.length; i++) {
             fieldPermutation[i] = i;
         }
         IIndexDataflowHelperFactory dataflowHelperFactory = createDataflowHelperFactory();
+        IStorageComponentProvider storageComponentProvider = metadataProvider.getStorageComponentProvider();
         LSMInvertedIndexBulkLoadOperatorDescriptor invIndexBulkLoadOp = new LSMInvertedIndexBulkLoadOperatorDescriptor(
                 spec, secondaryRecDesc, fieldPermutation, false, numElementsHint, false,
-                RuntimeComponentsProvider.RUNTIME_PROVIDER, secondaryFileSplitProvider,
-                RuntimeComponentsProvider.RUNTIME_PROVIDER, tokenTypeTraits, tokenComparatorFactories,
+                storageComponentProvider.getStorageManager(), secondaryFileSplitProvider,
+                storageComponentProvider.getIndexLifecycleManagerProvider(), tokenTypeTraits, tokenComparatorFactories,
                 invListsTypeTraits, primaryComparatorFactories, tokenizerFactory, dataflowHelperFactory,
-                LSMIndexUtil.getMetadataPageManagerFactory());
+                storageComponentProvider.getMetadataPageManagerFactory());
         AlgebricksPartitionConstraintHelper.setPartitionConstraintInJobSpec(spec, invIndexBulkLoadOp,
                 secondaryPartitionConstraint);
         return invIndexBulkLoadOp;
     }
 
-    private IIndexDataflowHelperFactory createDataflowHelperFactory() {
-        StorageProperties storageProperties = propertiesProvider.getStorageProperties();
-        boolean temp = dataset.getDatasetDetails().isTemp();
-        if (!isPartitioned) {
-            return new LSMInvertedIndexDataflowHelperFactory(
-                    new AsterixVirtualBufferCacheProvider(dataset.getDatasetId()), mergePolicyFactory,
-                    mergePolicyFactoryProperties, new SecondaryIndexOperationTrackerProvider(dataset.getDatasetId()),
-                    RuntimeComponentsProvider.RUNTIME_PROVIDER,
-                    LSMInvertedIndexIOOperationCallbackFactory.INSTANCE,
-                    storageProperties.getBloomFilterFalsePositiveRate(), invertedIndexFields, filterTypeTraits,
-                    filterCmpFactories, secondaryFilterFields, secondaryFilterFieldsForNonBulkLoadOps,
-                    invertedIndexFieldsForNonBulkLoadOps, !temp);
-        } else {
-            return new PartitionedLSMInvertedIndexDataflowHelperFactory(
-                    new AsterixVirtualBufferCacheProvider(dataset.getDatasetId()), mergePolicyFactory,
-                    mergePolicyFactoryProperties, new SecondaryIndexOperationTrackerProvider(dataset.getDatasetId()),
-                    RuntimeComponentsProvider.RUNTIME_PROVIDER,
-                    LSMInvertedIndexIOOperationCallbackFactory.INSTANCE,
-                    storageProperties.getBloomFilterFalsePositiveRate(), invertedIndexFields, filterTypeTraits,
-                    filterCmpFactories, secondaryFilterFields, secondaryFilterFieldsForNonBulkLoadOps,
-                    invertedIndexFieldsForNonBulkLoadOps, !temp);
-        }
+    private IIndexDataflowHelperFactory createDataflowHelperFactory() throws AlgebricksException {
+        return dataset.getIndexDataflowHelperFactory(metadataProvider, index, itemType, metaType, mergePolicyFactory,
+                mergePolicyFactoryProperties);
     }
 
     @Override
     public JobSpecification buildCompactJobSpec() throws AsterixException, AlgebricksException {
-        JobSpecification spec = JobSpecificationUtils.createJobSpecification();
-
+        JobSpecification spec = RuntimeUtils.createJobSpecification();
         IIndexDataflowHelperFactory dataflowHelperFactory = createDataflowHelperFactory();
+        IStorageComponentProvider storageComponentProvider = metadataProvider.getStorageComponentProvider();
         LSMInvertedIndexCompactOperator compactOp =
-                new LSMInvertedIndexCompactOperator(spec, RuntimeComponentsProvider.RUNTIME_PROVIDER,
-                        secondaryFileSplitProvider, RuntimeComponentsProvider.RUNTIME_PROVIDER, tokenTypeTraits,
-                        tokenComparatorFactories, invListsTypeTraits, primaryComparatorFactories, tokenizerFactory,
-                        dataflowHelperFactory, NoOpOperationCallbackFactory.INSTANCE, LSMIndexUtil
-                                .getMetadataPageManagerFactory());
+                new LSMInvertedIndexCompactOperator(spec, storageComponentProvider.getStorageManager(),
+                        secondaryFileSplitProvider, storageComponentProvider.getIndexLifecycleManagerProvider(),
+                        tokenTypeTraits, tokenComparatorFactories, invListsTypeTraits, primaryComparatorFactories,
+                        tokenizerFactory, dataflowHelperFactory,
+                        dataset.getModificationCallbackFactory(storageComponentProvider, index, null,
+                                IndexOperation.FULL_MERGE, null),
+                        storageComponentProvider.getMetadataPageManagerFactory());
         AlgebricksPartitionConstraintHelper.setPartitionConstraintInJobSpec(spec, compactOp,
                 secondaryPartitionConstraint);
 
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/file/SecondaryRTreeOperationsHelper.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryRTreeOperationsHelper.java
similarity index 67%
rename from asterixdb/asterix-app/src/main/java/org/apache/asterix/file/SecondaryRTreeOperationsHelper.java
rename to asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryRTreeOperationsHelper.java
index c96ba4c..460b635 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/file/SecondaryRTreeOperationsHelper.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryRTreeOperationsHelper.java
@@ -16,35 +16,29 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.asterix.file;
+package org.apache.asterix.metadata.utils;
 
 import java.util.List;
 
-import org.apache.asterix.app.external.ExternalIndexingOperations;
-import org.apache.asterix.common.config.StorageProperties;
 import org.apache.asterix.common.config.DatasetConfig.DatasetType;
-import org.apache.asterix.common.config.DatasetConfig.IndexType;
 import org.apache.asterix.common.config.GlobalConfig;
 import org.apache.asterix.common.config.IPropertiesProvider;
-import org.apache.asterix.common.context.AsterixVirtualBufferCacheProvider;
-import org.apache.asterix.common.dataflow.LSMIndexUtil;
+import org.apache.asterix.common.context.IStorageComponentProvider;
 import org.apache.asterix.common.exceptions.AsterixException;
-import org.apache.asterix.common.ioopcallbacks.LSMRTreeIOOperationCallbackFactory;
 import org.apache.asterix.common.transactions.IResourceFactory;
-import org.apache.asterix.dataflow.data.nontagged.valueproviders.PrimitiveValueProviderFactory;
 import org.apache.asterix.external.indexing.IndexingConstants;
-import org.apache.asterix.external.operators.ExternalDataScanOperatorDescriptor;
+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;
 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.utils.ExternalDatasetsRegistry;
+import org.apache.asterix.om.types.ARecordType;
 import org.apache.asterix.om.types.ATypeTag;
 import org.apache.asterix.om.types.IAType;
-import org.apache.asterix.om.util.NonTaggedFormatUtil;
-import org.apache.asterix.runtime.util.RuntimeComponentsProvider;
-import org.apache.asterix.transaction.management.opcallbacks.SecondaryIndexOperationTrackerProvider;
+import org.apache.asterix.om.utils.NonTaggedFormatUtil;
+import org.apache.asterix.runtime.utils.RuntimeUtils;
 import org.apache.asterix.transaction.management.resource.ExternalRTreeLocalResourceMetadataFactory;
 import org.apache.asterix.transaction.management.resource.LSMRTreeLocalResourceMetadataFactory;
 import org.apache.asterix.transaction.management.resource.PersistentLocalResourceFactoryProvider;
@@ -71,10 +65,8 @@
 import org.apache.hyracks.storage.am.common.dataflow.IIndexDataflowHelperFactory;
 import org.apache.hyracks.storage.am.common.dataflow.TreeIndexBulkLoadOperatorDescriptor;
 import org.apache.hyracks.storage.am.common.dataflow.TreeIndexCreateOperatorDescriptor;
-import org.apache.hyracks.storage.am.common.impls.NoOpOperationCallbackFactory;
+import org.apache.hyracks.storage.am.common.ophelpers.IndexOperation;
 import org.apache.hyracks.storage.am.lsm.common.dataflow.LSMTreeIndexCompactOperatorDescriptor;
-import org.apache.hyracks.storage.am.lsm.rtree.dataflow.ExternalRTreeDataflowHelperFactory;
-import org.apache.hyracks.storage.am.lsm.rtree.dataflow.LSMRTreeWithAntiMatterTuplesDataflowHelperFactory;
 import org.apache.hyracks.storage.am.rtree.frames.RTreePolicyType;
 import org.apache.hyracks.storage.common.file.ILocalResourceFactoryProvider;
 import org.apache.hyracks.storage.common.file.LocalResource;
@@ -90,38 +82,32 @@
     protected boolean isPointMBR;
     protected RecordDescriptor secondaryRecDescForPointMBR = null;
 
-    protected SecondaryRTreeOperationsHelper(PhysicalOptimizationConfig physOptConf,
-            IPropertiesProvider propertiesProvider) {
-        super(physOptConf, propertiesProvider);
+    protected SecondaryRTreeOperationsHelper(Dataset dataset, Index index, PhysicalOptimizationConfig physOptConf,
+            IPropertiesProvider propertiesProvider, MetadataProvider metadataProvider, ARecordType recType,
+            ARecordType metaType, ARecordType enforcedType, ARecordType enforcedMetaType) {
+        super(dataset, index, physOptConf, propertiesProvider, metadataProvider, recType, metaType, enforcedType,
+                enforcedMetaType);
     }
 
     @Override
-    public JobSpecification buildCreationJobSpec() throws AsterixException, AlgebricksException {
-        JobSpecification spec = JobSpecificationUtils.createJobSpecification();
-
-        StorageProperties storageProperties = propertiesProvider.getStorageProperties();
-        boolean temp = dataset.getDatasetDetails().isTemp();
-        IIndexDataflowHelperFactory indexDataflowHelperFactory;
+    public JobSpecification buildCreationJobSpec() throws AlgebricksException {
+        JobSpecification spec = RuntimeUtils.createJobSpecification();
+        IIndexDataflowHelperFactory indexDataflowHelperFactory = dataset.getIndexDataflowHelperFactory(
+                metadataProvider, index, itemType, metaType, mergePolicyFactory, mergePolicyFactoryProperties);
+        IStorageComponentProvider storageComponentProvider = metadataProvider.getStorageComponentProvider();
         ILocalResourceFactoryProvider localResourceFactoryProvider;
         if (dataset.getDatasetType() == DatasetType.INTERNAL) {
-
             IBinaryComparatorFactory[] btreeCompFactories = getComparatorFactoriesForDeletedKeyBTree();
-
             //prepare a LocalResourceMetadata which will be stored in NC's local resource repository
             IResourceFactory localResourceMetadata = new LSMRTreeLocalResourceMetadataFactory(secondaryTypeTraits,
                     secondaryComparatorFactories, btreeCompFactories, valueProviderFactories, RTreePolicyType.RTREE,
                     MetadataProvider.proposeLinearizer(keyType, secondaryComparatorFactories.length),
                     dataset.getDatasetId(), mergePolicyFactory, mergePolicyFactoryProperties, filterTypeTraits,
-                    filterCmpFactories, rtreeFields, primaryKeyFields, secondaryFilterFields, isPointMBR);
+                    filterCmpFactories, rtreeFields, primaryKeyFields, secondaryFilterFields, isPointMBR,
+                    dataset.getIndexOperationTrackerFactory(index), dataset.getIoOperationCallbackFactory(index),
+                    storageComponentProvider.getMetadataPageManagerFactory());
             localResourceFactoryProvider =
                     new PersistentLocalResourceFactoryProvider(localResourceMetadata, LocalResource.LSMRTreeResource);
-            indexDataflowHelperFactory = new LSMRTreeWithAntiMatterTuplesDataflowHelperFactory(valueProviderFactories,
-                    RTreePolicyType.RTREE, btreeCompFactories,
-                    new AsterixVirtualBufferCacheProvider(dataset.getDatasetId()), mergePolicyFactory,
-                    mergePolicyFactoryProperties, new SecondaryIndexOperationTrackerProvider(dataset.getDatasetId()),
-                    RuntimeComponentsProvider.RUNTIME_PROVIDER, LSMRTreeIOOperationCallbackFactory.INSTANCE,
-                    MetadataProvider.proposeLinearizer(keyType, secondaryComparatorFactories.length), rtreeFields,
-                    filterTypeTraits, filterCmpFactories, secondaryFilterFields, !temp, isPointMBR);
         } else {
             // External dataset
             // Prepare a LocalResourceMetadata which will be stored in NC's local resource repository
@@ -130,25 +116,21 @@
                     valueProviderFactories, RTreePolicyType.RTREE,
                     MetadataProvider.proposeLinearizer(keyType, secondaryComparatorFactories.length),
                     dataset.getDatasetId(), mergePolicyFactory, mergePolicyFactoryProperties, primaryKeyFields,
-                    isPointMBR);
+                    isPointMBR, dataset.getIndexOperationTrackerFactory(index),
+                    dataset.getIoOperationCallbackFactory(index),
+                    storageComponentProvider.getMetadataPageManagerFactory());
             localResourceFactoryProvider = new PersistentLocalResourceFactoryProvider(localResourceMetadata,
                     LocalResource.ExternalRTreeResource);
-
-            indexDataflowHelperFactory = new ExternalRTreeDataflowHelperFactory(valueProviderFactories,
-                    RTreePolicyType.RTREE, ExternalIndexingOperations.getBuddyBtreeComparatorFactories(),
-                    mergePolicyFactory, mergePolicyFactoryProperties,
-                    new SecondaryIndexOperationTrackerProvider(dataset.getDatasetId()),
-                    RuntimeComponentsProvider.RUNTIME_PROVIDER, LSMRTreeIOOperationCallbackFactory.INSTANCE,
-                    MetadataProvider.proposeLinearizer(keyType, secondaryComparatorFactories.length),
-                    storageProperties.getBloomFilterFalsePositiveRate(), new int[] { numNestedSecondaryKeyFields },
-                    ExternalDatasetsRegistry.INSTANCE.getDatasetVersion(dataset), true, isPointMBR);
         }
 
-        TreeIndexCreateOperatorDescriptor secondaryIndexCreateOp = new TreeIndexCreateOperatorDescriptor(spec,
-                RuntimeComponentsProvider.RUNTIME_PROVIDER, RuntimeComponentsProvider.RUNTIME_PROVIDER,
-                secondaryFileSplitProvider, secondaryTypeTraits, secondaryComparatorFactories, null,
-                indexDataflowHelperFactory, localResourceFactoryProvider, NoOpOperationCallbackFactory.INSTANCE,
-                LSMIndexUtil.getMetadataPageManagerFactory());
+        TreeIndexCreateOperatorDescriptor secondaryIndexCreateOp =
+                new TreeIndexCreateOperatorDescriptor(spec, storageComponentProvider.getStorageManager(),
+                        storageComponentProvider.getIndexLifecycleManagerProvider(), secondaryFileSplitProvider,
+                        secondaryTypeTraits, secondaryComparatorFactories, null, indexDataflowHelperFactory,
+                        localResourceFactoryProvider,
+                        dataset.getModificationCallbackFactory(storageComponentProvider, index, null,
+                                IndexOperation.CREATE, null),
+                        storageComponentProvider.getMetadataPageManagerFactory());
         AlgebricksPartitionConstraintHelper.setPartitionConstraintInJobSpec(spec, secondaryIndexCreateOp,
                 secondaryPartitionConstraint);
         spec.addRoot(secondaryIndexCreateOp);
@@ -157,7 +139,7 @@
     }
 
     private IBinaryComparatorFactory[] getComparatorFactoriesForDeletedKeyBTree() {
-        IBinaryComparatorFactory[] btreeCompFactories = new IBinaryComparatorFactory[secondaryTypeTraits.length];;
+        IBinaryComparatorFactory[] btreeCompFactories = new IBinaryComparatorFactory[secondaryTypeTraits.length];
         int i = 0;
         for (; i < secondaryComparatorFactories.length; i++) {
             btreeCompFactories[i] = secondaryComparatorFactories[i];
@@ -174,16 +156,16 @@
     }
 
     @Override
-    protected void setSecondaryRecDescAndComparators(IndexType indexType, List<List<String>> secondaryKeyFields,
-            List<IAType> secondaryKeyTypes, int gramLength, MetadataProvider metadata)
-            throws AlgebricksException, AsterixException {
+    protected void setSecondaryRecDescAndComparators() throws AlgebricksException {
+        List<List<String>> secondaryKeyFields = index.getKeyFieldNames();
         int numSecondaryKeys = secondaryKeyFields.size();
+        boolean isEnforcingKeyTypes = index.isEnforcingKeyFileds();
         if (numSecondaryKeys != 1) {
-            throw new AsterixException("Cannot use " + numSecondaryKeys
-                    + " fields as a key for the R-tree index. There can be only one field as a key for the R-tree index.");
+            throw new AsterixException("Cannot use " + numSecondaryKeys + " fields as a key for the R-tree index. "
+                    + "There can be only one field as a key for the R-tree index.");
         }
-        Pair<IAType, Boolean> spatialTypePair =
-                Index.getNonNullableOpenFieldType(secondaryKeyTypes.get(0), secondaryKeyFields.get(0), itemType);
+        Pair<IAType, Boolean> spatialTypePair = Index.getNonNullableOpenFieldType(index.getKeyFieldTypes().get(0),
+                secondaryKeyFields.get(0), itemType);
         IAType spatialType = spatialTypePair.first;
         anySecondaryKeyIsNullable = spatialTypePair.second;
         if (spatialType == null) {
@@ -194,13 +176,14 @@
         numNestedSecondaryKeyFields = numDimensions * 2;
         int recordColumn = dataset.getDatasetType() == DatasetType.INTERNAL ? numPrimaryKeys : 0;
         secondaryFieldAccessEvalFactories =
-                metadata.getFormat().createMBRFactory(isEnforcingKeyTypes ? enforcedItemType : itemType,
+                metadataProvider.getFormat().createMBRFactory(isEnforcingKeyTypes ? enforcedItemType : itemType,
                         secondaryKeyFields.get(0), recordColumn, numDimensions, filterFieldName);
         secondaryComparatorFactories = new IBinaryComparatorFactory[numNestedSecondaryKeyFields];
         valueProviderFactories = new IPrimitiveValueProviderFactory[numNestedSecondaryKeyFields];
         ISerializerDeserializer[] secondaryRecFields =
                 new ISerializerDeserializer[numPrimaryKeys + numNestedSecondaryKeyFields + numFilterFields];
-        ISerializerDeserializer[] enforcedRecFields = new ISerializerDeserializer[1 + numPrimaryKeys + numFilterFields];
+        ISerializerDeserializer[] enforcedRecFields =
+                new ISerializerDeserializer[1 + numPrimaryKeys + numFilterFields];
         secondaryTypeTraits = new ITypeTraits[numNestedSecondaryKeyFields + numPrimaryKeys];
         ITypeTraits[] enforcedTypeTraits = new ITypeTraits[1 + numPrimaryKeys];
         IAType nestedKeyType = NonTaggedFormatUtil.getNestedSpatialType(spatialType.getTypeTag());
@@ -212,7 +195,8 @@
             secondaryComparatorFactories[i] =
                     BinaryComparatorFactoryProvider.INSTANCE.getBinaryComparatorFactory(nestedKeyType, true);
             secondaryTypeTraits[i] = TypeTraitProvider.INSTANCE.getTypeTrait(nestedKeyType);
-            valueProviderFactories[i] = PrimitiveValueProviderFactory.INSTANCE;
+            valueProviderFactories[i] =
+                    metadataProvider.getStorageComponentProvider().getPrimitiveValueProviderFactory();
 
         }
         // Add serializers and comparators for primary index fields.
@@ -269,24 +253,29 @@
     public JobSpecification buildLoadingJobSpec() throws AsterixException, AlgebricksException {
         /***************************************************
          * [ About PointMBR Optimization ]
-         * Instead of storing a MBR(4 doubles) for a point(2 doubles) in RTree leaf node, PointMBR concept is introduced.
+         * Instead of storing a MBR(4 doubles) for a point(2 doubles) in RTree leaf node,
+         * PointMBR concept is introduced.
          * PointMBR is a way to store a point as 2 doubles in RTree leaf node.
          * This reduces RTree index size roughly in half.
          * In order to fully benefit from the PointMBR concept, besides RTree,
          * external sort operator during bulk-loading (from either data loading or index creation)
          * must deal with point as 2 doubles instead of 4 doubles. Otherwise, external sort will suffer from twice as
-         * many doubles as it actually requires. For this purpose, PointMBR specific optimization logic is added as follows:
+         * many doubles as it actually requires. For this purpose,
+         * PointMBR specific optimization logic is added as follows:
          * 1) CreateMBR function in assign operator generates 2 doubles, instead of 4 doubles.
          * 2) External sort operator sorts points represented with 2 doubles.
-         * 3) Bulk-loading in RTree takes 4 doubles by reading 2 doubles twice and then, do the same work as non-point MBR cases.
+         * 3) Bulk-loading in RTree takes 4 doubles by reading 2 doubles twice and then,
+         * do the same work as non-point MBR cases.
          ***************************************************/
-        JobSpecification spec = JobSpecificationUtils.createJobSpecification();
-        boolean temp = dataset.getDatasetDetails().isTemp();
+        JobSpecification spec = RuntimeUtils.createJobSpecification();
         int[] fieldPermutation = createFieldPermutationForBulkLoadOp(numNestedSecondaryKeyFields);
         int numNestedSecondaryKeFieldsConsideringPointMBR =
                 isPointMBR ? numNestedSecondaryKeyFields / 2 : numNestedSecondaryKeyFields;
         RecordDescriptor secondaryRecDescConsideringPointMBR =
                 isPointMBR ? secondaryRecDescForPointMBR : secondaryRecDesc;
+        boolean isEnforcingKeyTypes = index.isEnforcingKeyFileds();
+        IIndexDataflowHelperFactory indexDataflowHelperFactory = dataset.getIndexDataflowHelperFactory(
+                metadataProvider, index, itemType, metaType, mergePolicyFactory, mergePolicyFactoryProperties);
         if (dataset.getDatasetType() == DatasetType.INTERNAL) {
             // Create dummy key provider for feeding the primary index scan.
             AbstractOperatorDescriptor keyProviderOp = createDummyKeyProviderOp(spec);
@@ -300,7 +289,7 @@
                 sourceOp = createCastOp(spec, dataset.getDatasetType());
                 spec.connect(new OneToOneConnectorDescriptor(spec), primaryScanOp, 0, sourceOp, 0);
             }
-            AlgebricksMetaOperatorDescriptor asterixAssignOp = createAssignOp(spec, sourceOp,
+            AlgebricksMetaOperatorDescriptor asterixAssignOp = createAssignOp(spec,
                     numNestedSecondaryKeFieldsConsideringPointMBR, secondaryRecDescConsideringPointMBR);
 
             // If any of the secondary fields are nullable, then add a select op that filters nulls.
@@ -311,26 +300,13 @@
             }
 
             // Sort by secondary keys.
-            ExternalSortOperatorDescriptor sortOp =
-                    createSortOp(spec,
-                            new IBinaryComparatorFactory[] { MetadataProvider.proposeLinearizer(keyType,
-                                    secondaryComparatorFactories.length) },
-                            isPointMBR ? secondaryRecDescForPointMBR : secondaryRecDesc);
-
-            StorageProperties storageProperties = propertiesProvider.getStorageProperties();
-
-            IBinaryComparatorFactory[] btreeCompFactories = getComparatorFactoriesForDeletedKeyBTree();
-            IIndexDataflowHelperFactory idff = new LSMRTreeWithAntiMatterTuplesDataflowHelperFactory(
-                    valueProviderFactories, RTreePolicyType.RTREE, btreeCompFactories,
-                    new AsterixVirtualBufferCacheProvider(dataset.getDatasetId()), mergePolicyFactory,
-                    mergePolicyFactoryProperties, new SecondaryIndexOperationTrackerProvider(dataset.getDatasetId()),
-                    RuntimeComponentsProvider.RUNTIME_PROVIDER, LSMRTreeIOOperationCallbackFactory.INSTANCE,
-                    MetadataProvider.proposeLinearizer(keyType, secondaryComparatorFactories.length), rtreeFields,
-                    filterTypeTraits, filterCmpFactories, secondaryFilterFields, !temp, isPointMBR);;
-
+            ExternalSortOperatorDescriptor sortOp = createSortOp(spec,
+                    new IBinaryComparatorFactory[] {
+                            MetadataProvider.proposeLinearizer(keyType, secondaryComparatorFactories.length) },
+                    isPointMBR ? secondaryRecDescForPointMBR : secondaryRecDesc);
             // Create secondary RTree bulk load op.
-            TreeIndexBulkLoadOperatorDescriptor secondaryBulkLoadOp =
-                    createTreeIndexBulkLoadOp(spec, fieldPermutation, idff, GlobalConfig.DEFAULT_TREE_FILL_FACTOR);
+            TreeIndexBulkLoadOperatorDescriptor secondaryBulkLoadOp = createTreeIndexBulkLoadOp(spec, fieldPermutation,
+                    indexDataflowHelperFactory, GlobalConfig.DEFAULT_TREE_FILL_FACTOR);
             AlgebricksMetaOperatorDescriptor metaOp = new AlgebricksMetaOperatorDescriptor(spec, 1, 0,
                     new IPushRuntimeFactory[] { new SinkRuntimeFactory() }, new RecordDescriptor[] {});
             // Connect the operators.
@@ -349,11 +325,12 @@
         } else {
             // External dataset
             /*
-             * In case of external data, this method is used to build loading jobs for both initial load on index creation
+             * 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
-            ExternalDataScanOperatorDescriptor primaryScanOp = createExternalIndexingOp(spec);
+            ExternalScanOperatorDescriptor primaryScanOp = createExternalIndexingOp(spec);
             AbstractOperatorDescriptor sourceOp = primaryScanOp;
             if (isEnforcingKeyTypes && !enforcedItemType.equals(itemType)) {
                 sourceOp = createCastOp(spec, dataset.getDatasetType());
@@ -371,32 +348,21 @@
             }
 
             // Sort by secondary keys.
-            ExternalSortOperatorDescriptor sortOp =
-                    createSortOp(spec,
-                            new IBinaryComparatorFactory[] { MetadataProvider.proposeLinearizer(keyType,
-                                    secondaryComparatorFactories.length) },
-                            isPointMBR ? secondaryRecDescForPointMBR : secondaryRecDesc);
-            StorageProperties storageProperties = propertiesProvider.getStorageProperties();
-
-            // Create the dataflow helper factory
-            ExternalRTreeDataflowHelperFactory dataflowHelperFactory = new ExternalRTreeDataflowHelperFactory(
-                    valueProviderFactories, RTreePolicyType.RTREE, primaryComparatorFactories, mergePolicyFactory,
-                    mergePolicyFactoryProperties, new SecondaryIndexOperationTrackerProvider(dataset.getDatasetId()),
-                    RuntimeComponentsProvider.RUNTIME_PROVIDER, LSMRTreeIOOperationCallbackFactory.INSTANCE,
-                    MetadataProvider.proposeLinearizer(keyType, secondaryComparatorFactories.length),
-                    storageProperties.getBloomFilterFalsePositiveRate(), new int[] { numNestedSecondaryKeyFields },
-                    ExternalDatasetsRegistry.INSTANCE.getDatasetVersion(dataset), true, isPointMBR);
+            ExternalSortOperatorDescriptor sortOp = createSortOp(spec,
+                    new IBinaryComparatorFactory[] {
+                            MetadataProvider.proposeLinearizer(keyType, secondaryComparatorFactories.length) },
+                    isPointMBR ? secondaryRecDescForPointMBR : secondaryRecDesc);
             // Create secondary RTree bulk load op.
             IOperatorDescriptor root;
             AbstractTreeIndexOperatorDescriptor secondaryBulkLoadOp;
             if (externalFiles != null) {
                 // Transaction load
-                secondaryBulkLoadOp = createExternalIndexBulkModifyOp(spec, fieldPermutation, dataflowHelperFactory,
-                        GlobalConfig.DEFAULT_TREE_FILL_FACTOR);
+                secondaryBulkLoadOp = createExternalIndexBulkModifyOp(spec, fieldPermutation,
+                        indexDataflowHelperFactory, GlobalConfig.DEFAULT_TREE_FILL_FACTOR);
                 root = secondaryBulkLoadOp;
             } else {
                 // Initial load
-                secondaryBulkLoadOp = createTreeIndexBulkLoadOp(spec, fieldPermutation, dataflowHelperFactory,
+                secondaryBulkLoadOp = createTreeIndexBulkLoadOp(spec, fieldPermutation, indexDataflowHelperFactory,
                         GlobalConfig.DEFAULT_TREE_FILL_FACTOR);
                 AlgebricksMetaOperatorDescriptor metaOp = new AlgebricksMetaOperatorDescriptor(spec, 1, 0,
                         new IPushRuntimeFactory[] { new SinkRuntimeFactory() },
@@ -467,42 +433,17 @@
 
     @Override
     public JobSpecification buildCompactJobSpec() throws AsterixException, AlgebricksException {
-        JobSpecification spec = JobSpecificationUtils.createJobSpecification();
-
-        StorageProperties storageProperties = propertiesProvider.getStorageProperties();
-        boolean temp = dataset.getDatasetDetails().isTemp();
-        LSMTreeIndexCompactOperatorDescriptor compactOp;
-        if (dataset.getDatasetType() == DatasetType.INTERNAL) {
-            IBinaryComparatorFactory[] btreeCompFactories = getComparatorFactoriesForDeletedKeyBTree();;
-            IIndexDataflowHelperFactory idff = new LSMRTreeWithAntiMatterTuplesDataflowHelperFactory(
-                    valueProviderFactories, RTreePolicyType.RTREE, btreeCompFactories,
-                    new AsterixVirtualBufferCacheProvider(dataset.getDatasetId()), mergePolicyFactory,
-                    mergePolicyFactoryProperties, new SecondaryIndexOperationTrackerProvider(dataset.getDatasetId()),
-                    RuntimeComponentsProvider.RUNTIME_PROVIDER, LSMRTreeIOOperationCallbackFactory.INSTANCE,
-                    MetadataProvider.proposeLinearizer(keyType, secondaryComparatorFactories.length), rtreeFields,
-                    filterTypeTraits, filterCmpFactories, secondaryFilterFields, !temp, isPointMBR);
-            compactOp =
-                    new LSMTreeIndexCompactOperatorDescriptor(spec, RuntimeComponentsProvider.RUNTIME_PROVIDER,
-                            RuntimeComponentsProvider.RUNTIME_PROVIDER, secondaryFileSplitProvider,
-                            secondaryTypeTraits, secondaryComparatorFactories, secondaryBloomFilterKeyFields, idff,
-                            NoOpOperationCallbackFactory.INSTANCE, LSMIndexUtil.getMetadataPageManagerFactory());
-        } else {
-            // External dataset
-            compactOp =
-                    new LSMTreeIndexCompactOperatorDescriptor(spec, RuntimeComponentsProvider.RUNTIME_PROVIDER,
-                            RuntimeComponentsProvider.RUNTIME_PROVIDER, secondaryFileSplitProvider,
-                            secondaryTypeTraits, secondaryComparatorFactories, secondaryBloomFilterKeyFields,
-                            new ExternalRTreeDataflowHelperFactory(valueProviderFactories, RTreePolicyType.RTREE,
-                                    primaryComparatorFactories, mergePolicyFactory, mergePolicyFactoryProperties,
-                                    new SecondaryIndexOperationTrackerProvider(dataset.getDatasetId()),
-                                    RuntimeComponentsProvider.RUNTIME_PROVIDER,
-                                    LSMRTreeIOOperationCallbackFactory.INSTANCE,
-                                    MetadataProvider.proposeLinearizer(keyType, secondaryComparatorFactories.length),
-                                    storageProperties.getBloomFilterFalsePositiveRate(),
-                                    new int[] { numNestedSecondaryKeyFields },
-                                    ExternalDatasetsRegistry.INSTANCE.getDatasetVersion(dataset), true, isPointMBR),
-                            NoOpOperationCallbackFactory.INSTANCE, LSMIndexUtil.getMetadataPageManagerFactory());
-        }
+        JobSpecification spec = RuntimeUtils.createJobSpecification();
+        IIndexDataflowHelperFactory indexDataflowHelperFactory = dataset.getIndexDataflowHelperFactory(
+                metadataProvider, index, itemType, metaType, mergePolicyFactory, mergePolicyFactoryProperties);
+        LSMTreeIndexCompactOperatorDescriptor compactOp = new LSMTreeIndexCompactOperatorDescriptor(spec,
+                metadataProvider.getStorageComponentProvider().getStorageManager(),
+                metadataProvider.getStorageComponentProvider().getIndexLifecycleManagerProvider(),
+                secondaryFileSplitProvider, secondaryTypeTraits, secondaryComparatorFactories,
+                secondaryBloomFilterKeyFields, indexDataflowHelperFactory,
+                dataset.getModificationCallbackFactory(metadataProvider.getStorageComponentProvider(), index, null,
+                        IndexOperation.FULL_MERGE, null),
+                metadataProvider.getStorageComponentProvider().getMetadataPageManagerFactory());
 
         AlgebricksPartitionConstraintHelper.setPartitionConstraintInJobSpec(spec, compactOp,
                 secondaryPartitionConstraint);
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SplitsAndConstraintsUtil.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SplitsAndConstraintsUtil.java
index 24d185f..35e7acb 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SplitsAndConstraintsUtil.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SplitsAndConstraintsUtil.java
@@ -29,7 +29,7 @@
 import org.apache.asterix.metadata.MetadataManager;
 import org.apache.asterix.metadata.MetadataTransactionContext;
 import org.apache.asterix.metadata.entities.Dataset;
-import org.apache.asterix.runtime.util.ClusterStateManager;
+import org.apache.asterix.runtime.utils.ClusterStateManager;
 import org.apache.hyracks.algebricks.common.constraints.AlgebricksPartitionConstraint;
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
 import org.apache.hyracks.algebricks.common.utils.Pair;
@@ -41,7 +41,7 @@
     private SplitsAndConstraintsUtil() {
     }
 
-    private static FileSplit[] splitsForDataverse(String dataverseName) {
+    private static FileSplit[] getDataverseSplits(String dataverseName) {
         File relPathFile = new File(dataverseName);
         List<FileSplit> splits = new ArrayList<>();
         // get all partitions
@@ -55,7 +55,7 @@
         return splits.toArray(new FileSplit[] {});
     }
 
-    public static FileSplit[] splitsForDataset(MetadataTransactionContext mdTxnCtx, String dataverseName,
+    public static FileSplit[] getDatasetSplits(MetadataTransactionContext mdTxnCtx, String dataverseName,
             String datasetName, String targetIdxName, boolean temp) throws AlgebricksException {
         try {
             File relPathFile =
@@ -92,7 +92,7 @@
         }
     }
 
-    private static FileSplit[] splitsForFilesIndex(MetadataTransactionContext mdTxnCtx, String dataverseName,
+    private static FileSplit[] getFilesIndexSplits(MetadataTransactionContext mdTxnCtx, String dataverseName,
             String datasetName, String targetIdxName, boolean create) throws AlgebricksException {
         try {
             File relPathFile =
@@ -131,16 +131,16 @@
         }
     }
 
-    public static Pair<IFileSplitProvider, AlgebricksPartitionConstraint>
-            splitProviderAndPartitionConstraintsForDataverse(String dataverse) {
-        FileSplit[] splits = splitsForDataverse(dataverse);
+    public static Pair<IFileSplitProvider, AlgebricksPartitionConstraint> getDataverseSplitProviderAndConstraints(
+            String dataverse) {
+        FileSplit[] splits = getDataverseSplits(dataverse);
         return StoragePathUtil.splitProviderAndPartitionConstraints(splits);
     }
 
-    public static Pair<IFileSplitProvider, AlgebricksPartitionConstraint>
-            splitProviderAndPartitionConstraintsForFilesIndex(MetadataTransactionContext mdTxnCtx, String dataverseName,
-                    String datasetName, String targetIdxName, boolean create) throws AlgebricksException {
-        FileSplit[] splits = splitsForFilesIndex(mdTxnCtx, dataverseName, datasetName, targetIdxName, create);
+    public static Pair<IFileSplitProvider, AlgebricksPartitionConstraint> getFilesIndexSplitProviderAndConstraints(
+            MetadataTransactionContext mdTxnCtx, String dataverseName, String datasetName, String targetIdxName,
+            boolean create) throws AlgebricksException {
+        FileSplit[] splits = getFilesIndexSplits(mdTxnCtx, dataverseName, datasetName, targetIdxName, create);
         return StoragePathUtil.splitProviderAndPartitionConstraints(splits);
     }
 
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/TypeUtil.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/TypeUtil.java
new file mode 100644
index 0000000..d629940
--- /dev/null
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/TypeUtil.java
@@ -0,0 +1,156 @@
+/*
+ * 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.util.ArrayDeque;
+import java.util.Deque;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.asterix.metadata.entities.Index;
+import org.apache.asterix.om.types.ARecordType;
+import org.apache.asterix.om.types.ATypeTag;
+import org.apache.asterix.om.types.AUnionType;
+import org.apache.asterix.om.types.IAType;
+import org.apache.asterix.om.types.hierachy.ATypeHierarchy;
+import org.apache.commons.lang3.ArrayUtils;
+import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.hyracks.algebricks.common.utils.Pair;
+
+/**
+ * Provider utility methods for data types
+ */
+public class TypeUtil {
+
+    private TypeUtil() {
+    }
+
+    /**
+     * Merges typed index fields with specified recordType, allowing indexed fields to be optional.
+     * I.e. the type { "personId":int32, "name": string, "address" : { "street": string } } with typed indexes
+     * on age:int32, address.state:string will be merged into type { "personId":int32, "name": string,
+     * "age": int32? "address" : { "street": string, "state": string? } } Used by open indexes to enforce
+     * the type of an indexed record
+     */
+    public static Pair<ARecordType, ARecordType> createEnforcedType(ARecordType recordType, ARecordType metaType,
+            List<Index> indexes) throws AlgebricksException {
+        ARecordType enforcedRecordType = recordType;
+        ARecordType enforcedMetaType = metaType;
+        for (Index index : indexes) {
+            if (!index.isSecondaryIndex() || !index.isEnforcingKeyFileds()) {
+                continue;
+            }
+            if (index.hasMetaFields()) {
+                throw new AlgebricksException("Indexing an open field is only supported on the record part");
+            }
+            for (int i = 0; i < index.getKeyFieldNames().size(); i++) {
+                Deque<Pair<ARecordType, String>> nestedTypeStack = new ArrayDeque<>();
+                List<String> splits = index.getKeyFieldNames().get(i);
+                ARecordType nestedFieldType = enforcedRecordType;
+                boolean openRecords = false;
+                String bridgeName = nestedFieldType.getTypeName();
+                int j;
+                // Build the stack for the enforced type
+                for (j = 1; j < splits.size(); j++) {
+                    nestedTypeStack.push(new Pair<>(nestedFieldType, splits.get(j - 1)));
+                    bridgeName = nestedFieldType.getTypeName();
+                    nestedFieldType = (ARecordType) enforcedRecordType.getSubFieldType(splits.subList(0, j));
+                    if (nestedFieldType == null) {
+                        openRecords = true;
+                        break;
+                    }
+                }
+                if (openRecords) {
+                    // create the smallest record
+                    enforcedRecordType = new ARecordType(splits.get(splits.size() - 2),
+                            new String[] { splits.get(splits.size() - 1) },
+                            new IAType[] { AUnionType.createUnknownableType(index.getKeyFieldTypes().get(i)) }, true);
+                    // create the open part of the nested field
+                    for (int k = splits.size() - 3; k > (j - 2); k--) {
+                        enforcedRecordType = new ARecordType(splits.get(k), new String[] { splits.get(k + 1) },
+                                new IAType[] { AUnionType.createUnknownableType(enforcedRecordType) }, true);
+                    }
+                    // Bridge the gap
+                    Pair<ARecordType, String> gapPair = nestedTypeStack.pop();
+                    ARecordType parent = gapPair.first;
+
+                    IAType[] parentFieldTypes = ArrayUtils.addAll(parent.getFieldTypes().clone(),
+                            new IAType[] { AUnionType.createUnknownableType(enforcedRecordType) });
+                    enforcedRecordType = new ARecordType(bridgeName,
+                            ArrayUtils.addAll(parent.getFieldNames(), enforcedRecordType.getTypeName()),
+                            parentFieldTypes, true);
+                } else {
+                    //Schema is closed all the way to the field
+                    //enforced fields are either null or strongly typed
+                    Map<String, IAType> recordNameTypesMap = TypeUtil.createRecordNameTypeMap(nestedFieldType);
+                    // if a an enforced field already exists and the type is correct
+                    IAType enforcedFieldType = recordNameTypesMap.get(splits.get(splits.size() - 1));
+                    if (enforcedFieldType != null && enforcedFieldType.getTypeTag() == ATypeTag.UNION
+                            && ((AUnionType) enforcedFieldType).isUnknownableType()) {
+                        enforcedFieldType = ((AUnionType) enforcedFieldType).getActualType();
+                    }
+                    if (enforcedFieldType != null && !ATypeHierarchy.canPromote(enforcedFieldType.getTypeTag(),
+                            index.getKeyFieldTypes().get(i).getTypeTag())) {
+                        throw new AlgebricksException("Cannot enforce field " + index.getKeyFieldNames().get(i)
+                                + " to have type " + index.getKeyFieldTypes().get(i));
+                    }
+                    if (enforcedFieldType == null) {
+                        recordNameTypesMap.put(splits.get(splits.size() - 1),
+                                AUnionType.createUnknownableType(index.getKeyFieldTypes().get(i)));
+                    }
+                    enforcedRecordType = new ARecordType(nestedFieldType.getTypeName(),
+                            recordNameTypesMap.keySet().toArray(new String[recordNameTypesMap.size()]),
+                            recordNameTypesMap.values().toArray(new IAType[recordNameTypesMap.size()]),
+                            nestedFieldType.isOpen());
+                }
+
+                // Create the enforced type for the nested fields in the schema, from the ground up
+                if (!nestedTypeStack.isEmpty()) {
+                    while (!nestedTypeStack.isEmpty()) {
+                        Pair<ARecordType, String> nestedTypePair = nestedTypeStack.pop();
+                        ARecordType nestedRecType = nestedTypePair.first;
+                        IAType[] nestedRecTypeFieldTypes = nestedRecType.getFieldTypes().clone();
+                        nestedRecTypeFieldTypes[nestedRecType.getFieldIndex(nestedTypePair.second)] =
+                                enforcedRecordType;
+                        enforcedRecordType = new ARecordType(nestedRecType.getTypeName() + "_enforced",
+                                nestedRecType.getFieldNames(), nestedRecTypeFieldTypes, nestedRecType.isOpen());
+                    }
+                }
+            }
+        }
+        return new Pair<>(enforcedRecordType, enforcedMetaType);
+    }
+
+    /**
+     * Creates a map from name to type for fields in the passed type
+     *
+     * @param recordType
+     *            the type to be mapped
+     * @return a map mapping all fields to their types
+     */
+    public static Map<String, IAType> createRecordNameTypeMap(ARecordType recordType) {
+        LinkedHashMap<String, IAType> recordNameTypesMap = new LinkedHashMap<>();
+        for (int j = 0; j < recordType.getFieldNames().length; j++) {
+            recordNameTypesMap.put(recordType.getFieldNames()[j], recordType.getFieldTypes()[j]);
+        }
+        return recordNameTypesMap;
+    }
+
+}
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/builders/AbstractListBuilder.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/builders/AbstractListBuilder.java
index b3cb416..f64206e 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/builders/AbstractListBuilder.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/builders/AbstractListBuilder.java
@@ -25,7 +25,7 @@
 import org.apache.asterix.dataflow.data.nontagged.serde.SerializerDeserializerUtil;
 import org.apache.asterix.om.types.ATypeTag;
 import org.apache.asterix.om.types.AbstractCollectionType;
-import org.apache.asterix.om.util.NonTaggedFormatUtil;
+import org.apache.asterix.om.utils.NonTaggedFormatUtil;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.data.std.api.IValueReference;
 import org.apache.hyracks.data.std.util.GrowableArray;
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/builders/RecordBuilder.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/builders/RecordBuilder.java
index 44181d5..797901e 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/builders/RecordBuilder.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/builders/RecordBuilder.java
@@ -29,7 +29,7 @@
 import org.apache.asterix.om.types.ARecordType;
 import org.apache.asterix.om.types.ATypeTag;
 import org.apache.asterix.om.types.runtime.RuntimeRecordTypeInfo;
-import org.apache.asterix.om.util.NonTaggedFormatUtil;
+import org.apache.asterix.om.utils.NonTaggedFormatUtil;
 import org.apache.hyracks.api.dataflow.value.IBinaryComparator;
 import org.apache.hyracks.api.dataflow.value.IBinaryHashFunction;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/common/AOrderedListBinaryTokenizer.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/common/AOrderedListBinaryTokenizer.java
index ace692f..90af41b 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/common/AOrderedListBinaryTokenizer.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/common/AOrderedListBinaryTokenizer.java
@@ -22,7 +22,7 @@
 import org.apache.asterix.dataflow.data.nontagged.serde.AOrderedListSerializerDeserializer;
 import org.apache.asterix.om.types.ATypeTag;
 import org.apache.asterix.om.types.EnumDeserializer;
-import org.apache.asterix.om.util.NonTaggedFormatUtil;
+import org.apache.asterix.om.utils.NonTaggedFormatUtil;
 import org.apache.hyracks.storage.am.lsm.invertedindex.tokenizers.IBinaryTokenizer;
 import org.apache.hyracks.storage.am.lsm.invertedindex.tokenizers.IToken;
 import org.apache.hyracks.storage.am.lsm.invertedindex.tokenizers.ITokenFactory;
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/AOrderedListSerializerDeserializer.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/AOrderedListSerializerDeserializer.java
index 6be61d6..437958c 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/AOrderedListSerializerDeserializer.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/AOrderedListSerializerDeserializer.java
@@ -34,7 +34,7 @@
 import org.apache.asterix.om.types.EnumDeserializer;
 import org.apache.asterix.om.types.IAType;
 import org.apache.asterix.om.types.TypeTagUtil;
-import org.apache.asterix.om.util.NonTaggedFormatUtil;
+import org.apache.asterix.om.utils.NonTaggedFormatUtil;
 import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/ARecordSerializerDeserializer.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/ARecordSerializerDeserializer.java
index 1d3a420..75aa61d 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/ARecordSerializerDeserializer.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/ARecordSerializerDeserializer.java
@@ -39,7 +39,8 @@
 import org.apache.asterix.om.types.AUnionType;
 import org.apache.asterix.om.types.BuiltinType;
 import org.apache.asterix.om.types.IAType;
-import org.apache.asterix.om.util.NonTaggedFormatUtil;
+import org.apache.asterix.om.utils.NonTaggedFormatUtil;
+import org.apache.asterix.om.utils.RecordUtil;
 import org.apache.hyracks.algebricks.common.utils.Pair;
 import org.apache.hyracks.api.dataflow.value.IBinaryComparator;
 import org.apache.hyracks.api.dataflow.value.IBinaryHashFunction;
@@ -196,7 +197,7 @@
         ISerializerDeserializer<AString> stringSerde = SerializerDeserializerProvider.INSTANCE
                 .getSerializerDeserializer(BuiltinType.ASTRING);
         RecordBuilder confRecordBuilder = new RecordBuilder();
-        confRecordBuilder.reset(ARecordType.FULLY_OPEN_RECORD_TYPE);
+        confRecordBuilder.reset(RecordUtil.FULLY_OPEN_RECORD_TYPE);
         ArrayBackedValueStorage fieldNameBytes = new ArrayBackedValueStorage();
         ArrayBackedValueStorage fieldValueBytes = new ArrayBackedValueStorage();
         for (int i = 0; i < record.getType().getFieldNames().length; i++) {
@@ -218,7 +219,7 @@
         ISerializerDeserializer<AString> stringSerde = SerializerDeserializerProvider.INSTANCE
                 .getSerializerDeserializer(BuiltinType.ASTRING);
         RecordBuilder confRecordBuilder = new RecordBuilder();
-        confRecordBuilder.reset(ARecordType.FULLY_OPEN_RECORD_TYPE);
+        confRecordBuilder.reset(RecordUtil.FULLY_OPEN_RECORD_TYPE);
         ArrayBackedValueStorage fieldNameBytes = new ArrayBackedValueStorage();
         ArrayBackedValueStorage fieldValueBytes = new ArrayBackedValueStorage();
         for (int i = 0; i < record.size(); i++) {
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/AUnorderedListSerializerDeserializer.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/AUnorderedListSerializerDeserializer.java
index fad023f..02b662e 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/AUnorderedListSerializerDeserializer.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/AUnorderedListSerializerDeserializer.java
@@ -34,7 +34,7 @@
 import org.apache.asterix.om.types.EnumDeserializer;
 import org.apache.asterix.om.types.IAType;
 import org.apache.asterix.om.types.TypeTagUtil;
-import org.apache.asterix.om.util.NonTaggedFormatUtil;
+import org.apache.asterix.om.utils.NonTaggedFormatUtil;
 import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/base/ARecord.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/base/ARecord.java
index cda4d2b..147ef6d 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/base/ARecord.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/base/ARecord.java
@@ -21,13 +21,14 @@
 import com.fasterxml.jackson.databind.ObjectMapper;
 import org.apache.asterix.common.exceptions.AsterixException;
 import org.apache.asterix.om.types.ARecordType;
+import org.apache.asterix.om.utils.RecordUtil;
 import org.apache.asterix.om.visitors.IOMVisitor;
 import com.fasterxml.jackson.databind.node.ArrayNode;
 
 import com.fasterxml.jackson.databind.node.ObjectNode;
 
 public class ARecord implements IAObject {
-    public static final ARecord EMPTY_OPEN_RECORD = new ARecord(ARecordType.FULLY_OPEN_RECORD_TYPE, new IAObject[] {});
+    public static final ARecord EMPTY_OPEN_RECORD = new ARecord(RecordUtil.FULLY_OPEN_RECORD_TYPE, new IAObject[] {});
 
     protected ARecordType type;
     protected IAObject[] fields;
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/pointables/AListVisitablePointable.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/pointables/AListVisitablePointable.java
index 2f7d9af..ec9bf48 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/pointables/AListVisitablePointable.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/pointables/AListVisitablePointable.java
@@ -32,9 +32,9 @@
 import org.apache.asterix.om.types.AbstractCollectionType;
 import org.apache.asterix.om.types.EnumDeserializer;
 import org.apache.asterix.om.types.IAType;
-import org.apache.asterix.om.util.NonTaggedFormatUtil;
-import org.apache.asterix.om.util.ResettableByteArrayOutputStream;
 import org.apache.asterix.om.util.container.IObjectFactory;
+import org.apache.asterix.om.utils.NonTaggedFormatUtil;
+import org.apache.asterix.om.utils.ResettableByteArrayOutputStream;
 
 /**
  * This class interprets the binary data representation of a list, one can
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/pointables/ARecordVisitablePointable.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/pointables/ARecordVisitablePointable.java
index d3c18a9..c5a0ee5 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/pointables/ARecordVisitablePointable.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/pointables/ARecordVisitablePointable.java
@@ -33,9 +33,9 @@
 import org.apache.asterix.om.types.AUnionType;
 import org.apache.asterix.om.types.EnumDeserializer;
 import org.apache.asterix.om.types.IAType;
-import org.apache.asterix.om.util.NonTaggedFormatUtil;
-import org.apache.asterix.om.util.ResettableByteArrayOutputStream;
 import org.apache.asterix.om.util.container.IObjectFactory;
+import org.apache.asterix.om.utils.NonTaggedFormatUtil;
+import org.apache.asterix.om.utils.ResettableByteArrayOutputStream;
 import org.apache.hyracks.util.string.UTF8StringWriter;
 
 /**
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/pointables/cast/AListCaster.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/pointables/cast/AListCaster.java
index 7115827..94cfe75 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/pointables/cast/AListCaster.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/pointables/cast/AListCaster.java
@@ -35,7 +35,7 @@
 import org.apache.asterix.om.types.AbstractCollectionType;
 import org.apache.asterix.om.types.EnumDeserializer;
 import org.apache.asterix.om.types.IAType;
-import org.apache.asterix.om.util.ResettableByteArrayOutputStream;
+import org.apache.asterix.om.utils.ResettableByteArrayOutputStream;
 import org.apache.hyracks.algebricks.common.utils.Triple;
 
 /**
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/pointables/cast/ARecordCaster.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/pointables/cast/ARecordCaster.java
index e0deb5a..c2aa3d8 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/pointables/cast/ARecordCaster.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/pointables/cast/ARecordCaster.java
@@ -42,8 +42,8 @@
 import org.apache.asterix.om.types.EnumDeserializer;
 import org.apache.asterix.om.types.IAType;
 import org.apache.asterix.om.types.hierachy.ATypeHierarchy;
-import org.apache.asterix.om.util.NonTaggedFormatUtil;
-import org.apache.asterix.om.util.ResettableByteArrayOutputStream;
+import org.apache.asterix.om.utils.NonTaggedFormatUtil;
+import org.apache.asterix.om.utils.ResettableByteArrayOutputStream;
 import org.apache.hyracks.algebricks.common.utils.Pair;
 import org.apache.hyracks.algebricks.common.utils.Triple;
 import org.apache.hyracks.api.dataflow.value.IBinaryComparator;
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/pointables/nonvisitor/AListPointable.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/pointables/nonvisitor/AListPointable.java
index 336aaac..f2e7299 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/pointables/nonvisitor/AListPointable.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/pointables/nonvisitor/AListPointable.java
@@ -26,8 +26,8 @@
 import org.apache.asterix.om.types.AOrderedListType;
 import org.apache.asterix.om.types.ATypeTag;
 import org.apache.asterix.om.types.AbstractCollectionType;
-import org.apache.asterix.om.util.NonTaggedFormatUtil;
 import org.apache.asterix.om.util.container.IObjectFactory;
+import org.apache.asterix.om.utils.NonTaggedFormatUtil;
 import org.apache.hyracks.api.dataflow.value.ITypeTraits;
 import org.apache.hyracks.data.std.api.AbstractPointable;
 import org.apache.hyracks.data.std.api.IPointable;
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/pointables/nonvisitor/ARecordPointable.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/pointables/nonvisitor/ARecordPointable.java
index 08cde54..2fae304 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/pointables/nonvisitor/ARecordPointable.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/pointables/nonvisitor/ARecordPointable.java
@@ -28,8 +28,9 @@
 import org.apache.asterix.om.types.AUnionType;
 import org.apache.asterix.om.types.EnumDeserializer;
 import org.apache.asterix.om.types.IAType;
-import org.apache.asterix.om.util.NonTaggedFormatUtil;
 import org.apache.asterix.om.util.container.IObjectFactory;
+import org.apache.asterix.om.utils.NonTaggedFormatUtil;
+import org.apache.asterix.om.utils.RecordUtil;
 import org.apache.hyracks.api.dataflow.value.ITypeTraits;
 import org.apache.hyracks.data.std.api.AbstractPointable;
 import org.apache.hyracks.data.std.api.IPointable;
@@ -190,7 +191,7 @@
     }
 
     public int getNullBitmapSize(ARecordType recordType) {
-        return ARecordType.computeNullBitmapSize(recordType);
+        return RecordUtil.computeNullBitmapSize(recordType);
     }
 
     public boolean isClosedFieldNull(ARecordType recordType, int fieldId) {
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/ClosedRecordConstructorResultType.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/ClosedRecordConstructorResultType.java
index 0e351e2..b328cf4 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/ClosedRecordConstructorResultType.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/ClosedRecordConstructorResultType.java
@@ -28,7 +28,7 @@
 import org.apache.asterix.om.types.ATypeTag;
 import org.apache.asterix.om.types.AUnionType;
 import org.apache.asterix.om.types.IAType;
-import org.apache.asterix.om.util.ConstantExpressionUtil;
+import org.apache.asterix.om.utils.ConstantExpressionUtil;
 import org.apache.commons.lang3.mutable.Mutable;
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
 import org.apache.hyracks.algebricks.core.algebra.base.ILogicalExpression;
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/FieldAccessByIndexResultType.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/FieldAccessByIndexResultType.java
index 0c9d538..0e04480 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/FieldAccessByIndexResultType.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/FieldAccessByIndexResultType.java
@@ -24,7 +24,7 @@
 import org.apache.asterix.om.types.ATypeTag;
 import org.apache.asterix.om.types.BuiltinType;
 import org.apache.asterix.om.types.IAType;
-import org.apache.asterix.om.util.ConstantExpressionUtil;
+import org.apache.asterix.om.utils.ConstantExpressionUtil;
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
 import org.apache.hyracks.algebricks.core.algebra.base.ILogicalExpression;
 import org.apache.hyracks.algebricks.core.algebra.expressions.AbstractFunctionCallExpression;
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/FieldAccessByNameResultType.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/FieldAccessByNameResultType.java
index deb667c..ffa3029 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/FieldAccessByNameResultType.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/FieldAccessByNameResultType.java
@@ -24,7 +24,7 @@
 import org.apache.asterix.om.types.ATypeTag;
 import org.apache.asterix.om.types.BuiltinType;
 import org.apache.asterix.om.types.IAType;
-import org.apache.asterix.om.util.ConstantExpressionUtil;
+import org.apache.asterix.om.utils.ConstantExpressionUtil;
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
 import org.apache.hyracks.algebricks.core.algebra.base.ILogicalExpression;
 import org.apache.hyracks.algebricks.core.algebra.expressions.AbstractFunctionCallExpression;
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/OpenRecordConstructorResultType.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/OpenRecordConstructorResultType.java
index baee36a..1442dfe 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/OpenRecordConstructorResultType.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/OpenRecordConstructorResultType.java
@@ -32,7 +32,7 @@
 import org.apache.asterix.om.types.AUnionType;
 import org.apache.asterix.om.types.IAType;
 import org.apache.asterix.om.types.TypeHelper;
-import org.apache.asterix.om.util.ConstantExpressionUtil;
+import org.apache.asterix.om.utils.ConstantExpressionUtil;
 import org.apache.commons.lang3.mutable.Mutable;
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
 import org.apache.hyracks.algebricks.core.algebra.base.ILogicalExpression;
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/RecordAddFieldsTypeComputer.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/RecordAddFieldsTypeComputer.java
index 8c643dc..cb6d90b 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/RecordAddFieldsTypeComputer.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/RecordAddFieldsTypeComputer.java
@@ -36,7 +36,7 @@
 import org.apache.asterix.om.types.AUnionType;
 import org.apache.asterix.om.types.IAType;
 import org.apache.asterix.om.types.TypeHelper;
-import org.apache.asterix.om.util.ConstantExpressionUtil;
+import org.apache.asterix.om.utils.ConstantExpressionUtil;
 import org.apache.commons.lang3.mutable.Mutable;
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
 import org.apache.hyracks.algebricks.core.algebra.base.ILogicalExpression;
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/TypeComputeUtils.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/TypeComputeUtils.java
index 823f51e..9110c12 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/TypeComputeUtils.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/TypeComputeUtils.java
@@ -25,6 +25,7 @@
 import org.apache.asterix.om.types.AUnorderedListType;
 import org.apache.asterix.om.types.BuiltinType;
 import org.apache.asterix.om.types.IAType;
+import org.apache.asterix.om.utils.RecordUtil;
 import org.apache.commons.lang3.mutable.Mutable;
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
 import org.apache.hyracks.algebricks.core.algebra.base.ILogicalExpression;
@@ -197,7 +198,7 @@
                     return null;
                 }
             case ANY:
-                return ARecordType.FULLY_OPEN_RECORD_TYPE;
+                return RecordUtil.FULLY_OPEN_RECORD_TYPE;
             default:
                 return null;
         }
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/types/ARecordType.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/types/ARecordType.java
index 4ce1a6d..0196e94 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/types/ARecordType.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/types/ARecordType.java
@@ -26,13 +26,13 @@
 import java.util.Map;
 import java.util.Set;
 
-import com.fasterxml.jackson.databind.ObjectMapper;
 import org.apache.asterix.common.annotations.IRecordTypeAnnotation;
 import org.apache.asterix.common.exceptions.AsterixException;
 import org.apache.asterix.om.base.IAObject;
-import org.apache.asterix.om.util.NonTaggedFormatUtil;
 import org.apache.asterix.om.visitors.IOMVisitor;
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+
+import com.fasterxml.jackson.databind.ObjectMapper;
 import com.fasterxml.jackson.databind.node.ArrayNode;
 import com.fasterxml.jackson.databind.node.ObjectNode;
 
@@ -44,9 +44,6 @@
  */
 public class ARecordType extends AbstractComplexType {
 
-    public static final ARecordType FULLY_OPEN_RECORD_TYPE = new ARecordType("OpenRecord", new String[0], new IAType[0],
-            true);
-
     private static final long serialVersionUID = 1L;
     private final String[] fieldNames;
     private final IAType[] fieldTypes;
@@ -209,7 +206,7 @@
      * @return the type of the child
      * @throws AsterixException
      */
-    public IAType getSubFieldType(List<String> subFieldName) throws AsterixException {
+    public IAType getSubFieldType(List<String> subFieldName) throws AlgebricksException {
         IAType subRecordType = getFieldType(subFieldName.get(0));
         for (int i = 1; i < subFieldName.size(); i++) {
             if (subRecordType == null) {
@@ -327,7 +324,7 @@
     }
 
     @Override
-    public ObjectNode toJSON()  {
+    public ObjectNode toJSON() {
         ObjectMapper om = new ObjectMapper();
         ObjectNode type = om.createObjectNode();
         type.put("type", ARecordType.class.getName());
@@ -349,10 +346,6 @@
         return type;
     }
 
-    public static int computeNullBitmapSize(ARecordType rt) {
-        return NonTaggedFormatUtil.hasOptionalField(rt) ? (int) Math.ceil(rt.getFieldNames().length / 4.0) : 0;
-    }
-
     public List<IAType> getFieldTypes(List<List<String>> fields) throws AlgebricksException {
         List<IAType> typeList = new ArrayList<>();
         for (List<String> field : fields) {
@@ -370,14 +363,4 @@
         }
         return false;
     }
-
-    /**
-     * Create a fully open record type with the passed name
-     *
-     * @param name
-     * @return
-     */
-    public static ARecordType createOpenRecordType(String name) {
-        return new ARecordType(name, new String[0], new IAType[0], true);
-    }
 }
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/types/TypeTagUtil.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/types/TypeTagUtil.java
index b6c5712..1b0beab 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/types/TypeTagUtil.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/types/TypeTagUtil.java
@@ -19,6 +19,7 @@
 package org.apache.asterix.om.types;
 
 import org.apache.asterix.common.exceptions.AsterixException;
+import org.apache.asterix.om.utils.RecordUtil;
 
 public class TypeTagUtil {
 
@@ -81,7 +82,7 @@
             case UUID:
                 return BuiltinType.AUUID;
             case RECORD:
-                return ARecordType.FULLY_OPEN_RECORD_TYPE;
+                return RecordUtil.FULLY_OPEN_RECORD_TYPE;
             case UNORDEREDLIST:
                 return AUnorderedListType.FULLY_OPEN_UNORDEREDLIST_TYPE;
             case ORDEREDLIST:
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/util/ConstantExpressionUtil.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/utils/ConstantExpressionUtil.java
similarity index 99%
rename from asterixdb/asterix-om/src/main/java/org/apache/asterix/om/util/ConstantExpressionUtil.java
rename to asterixdb/asterix-om/src/main/java/org/apache/asterix/om/utils/ConstantExpressionUtil.java
index 406f356..28b3da2 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/util/ConstantExpressionUtil.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/utils/ConstantExpressionUtil.java
@@ -16,7 +16,7 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.asterix.om.util;
+package org.apache.asterix.om.utils;
 
 import org.apache.asterix.om.base.ABoolean;
 import org.apache.asterix.om.base.AInt32;
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/util/JSONDeserializerForTypes.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/utils/JSONDeserializerForTypes.java
similarity index 98%
rename from asterixdb/asterix-om/src/main/java/org/apache/asterix/om/util/JSONDeserializerForTypes.java
rename to asterixdb/asterix-om/src/main/java/org/apache/asterix/om/utils/JSONDeserializerForTypes.java
index 8e8993a..069e47b 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/util/JSONDeserializerForTypes.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/utils/JSONDeserializerForTypes.java
@@ -17,7 +17,7 @@
  * under the License.
  */
 
-package org.apache.asterix.om.util;
+package org.apache.asterix.om.utils;
 
 import java.lang.reflect.Field;
 import java.util.ArrayList;
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/util/NonTaggedFormatUtil.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/utils/NonTaggedFormatUtil.java
similarity index 99%
rename from asterixdb/asterix-om/src/main/java/org/apache/asterix/om/util/NonTaggedFormatUtil.java
rename to asterixdb/asterix-om/src/main/java/org/apache/asterix/om/utils/NonTaggedFormatUtil.java
index 0608b79..c524d28 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/util/NonTaggedFormatUtil.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/utils/NonTaggedFormatUtil.java
@@ -16,7 +16,7 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.asterix.om.util;
+package org.apache.asterix.om.utils;
 
 import org.apache.asterix.common.config.DatasetConfig.IndexType;
 import org.apache.asterix.common.exceptions.AsterixException;
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/utils/RecordUtil.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/utils/RecordUtil.java
new file mode 100644
index 0000000..031669a
--- /dev/null
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/utils/RecordUtil.java
@@ -0,0 +1,78 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.om.utils;
+
+import java.util.List;
+
+import org.apache.asterix.om.types.ARecordType;
+import org.apache.asterix.om.types.IAType;
+import org.apache.commons.lang3.StringUtils;
+
+public class RecordUtil {
+    /**
+     * A fully open record type which has the name OpenRecord
+     */
+    public static final ARecordType FULLY_OPEN_RECORD_TYPE =
+            new ARecordType("OpenRecord", new String[0], new IAType[0], true);
+
+    private RecordUtil() {
+    }
+
+    /**
+     * Create a fully open record type with the passed name
+     *
+     * @param name
+     * @return
+     */
+    public static ARecordType createOpenRecordType(String name) {
+        return new ARecordType(name, new String[0], new IAType[0], true);
+    }
+
+    /**
+     * A util method that takes a field name and return a String representation for error messages
+     *
+     * @param field
+     * @return
+     */
+    public static String toFullyQualifiedName(List<String> field) {
+        return StringUtils.join(field, ".");
+    }
+
+    /**
+     * A util method that takes String array and return a String representation for error messages
+     *
+     * @param field
+     * @return
+     */
+    public static String toFullyQualifiedName(String... names) {
+        return StringUtils.join(names, ".");
+    }
+
+    /**
+     * compute the null Bitmap size for the open fields
+     *
+     * @param recordType
+     *            the record type
+     * @return the size of the bitmap
+     */
+    public static int computeNullBitmapSize(ARecordType recordType) {
+        return NonTaggedFormatUtil.hasOptionalField(recordType)
+                ? (int) Math.ceil(recordType.getFieldNames().length / 4.0) : 0;
+    }
+}
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/util/ResettableByteArrayOutputStream.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/utils/ResettableByteArrayOutputStream.java
similarity index 96%
rename from asterixdb/asterix-om/src/main/java/org/apache/asterix/om/util/ResettableByteArrayOutputStream.java
rename to asterixdb/asterix-om/src/main/java/org/apache/asterix/om/utils/ResettableByteArrayOutputStream.java
index a40005f..c1b671b 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/util/ResettableByteArrayOutputStream.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/utils/ResettableByteArrayOutputStream.java
@@ -16,7 +16,7 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.asterix.om.util;
+package org.apache.asterix.om.utils;
 
 import org.apache.hyracks.data.std.util.ByteArrayAccessibleOutputStream;
 
diff --git a/asterixdb/asterix-om/src/test/java/org/apache/asterix/om/util/JSONDeserializerForTypesTest.java b/asterixdb/asterix-om/src/test/java/org/apache/asterix/om/util/JSONDeserializerForTypesTest.java
index a0a5c5f..32ec75a 100644
--- a/asterixdb/asterix-om/src/test/java/org/apache/asterix/om/util/JSONDeserializerForTypesTest.java
+++ b/asterixdb/asterix-om/src/test/java/org/apache/asterix/om/util/JSONDeserializerForTypesTest.java
@@ -28,6 +28,7 @@
 import org.apache.asterix.om.types.AUnorderedListType;
 import org.apache.asterix.om.types.BuiltinType;
 import org.apache.asterix.om.types.IAType;
+import org.apache.asterix.om.utils.JSONDeserializerForTypes;
 import org.junit.Assert;
 import org.junit.Test;
 
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/common/ListAccessor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/common/ListAccessor.java
index e456bc3..9783284 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/common/ListAccessor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/common/ListAccessor.java
@@ -27,7 +27,7 @@
 import org.apache.asterix.dataflow.data.nontagged.serde.AUnorderedListSerializerDeserializer;
 import org.apache.asterix.om.types.ATypeTag;
 import org.apache.asterix.om.types.EnumDeserializer;
-import org.apache.asterix.om.util.NonTaggedFormatUtil;
+import org.apache.asterix.om.utils.NonTaggedFormatUtil;
 import org.apache.asterix.runtime.exceptions.TypeMismatchException;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/AnyCollectionMemberDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/AnyCollectionMemberDescriptor.java
index fb35332..714fabe 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/AnyCollectionMemberDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/AnyCollectionMemberDescriptor.java
@@ -29,7 +29,7 @@
 import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
 import org.apache.asterix.om.types.ATypeTag;
 import org.apache.asterix.om.types.EnumDeserializer;
-import org.apache.asterix.om.util.NonTaggedFormatUtil;
+import org.apache.asterix.om.utils.NonTaggedFormatUtil;
 import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
 import org.apache.asterix.runtime.exceptions.TypeMismatchException;
 import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/GetItemDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/GetItemDescriptor.java
index 145b6d5..276c9ae 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/GetItemDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/GetItemDescriptor.java
@@ -29,7 +29,7 @@
 import org.apache.asterix.om.types.ATypeTag;
 import org.apache.asterix.om.types.EnumDeserializer;
 import org.apache.asterix.om.types.hierachy.ATypeHierarchy;
-import org.apache.asterix.om.util.NonTaggedFormatUtil;
+import org.apache.asterix.om.utils.NonTaggedFormatUtil;
 import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
 import org.apache.asterix.runtime.exceptions.TypeMismatchException;
 import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/FieldAccessByIndexEvalFactory.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/FieldAccessByIndexEvalFactory.java
index bec79e8..3321efc 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/FieldAccessByIndexEvalFactory.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/FieldAccessByIndexEvalFactory.java
@@ -29,7 +29,8 @@
 import org.apache.asterix.om.types.ATypeTag;
 import org.apache.asterix.om.types.AUnionType;
 import org.apache.asterix.om.types.IAType;
-import org.apache.asterix.om.util.NonTaggedFormatUtil;
+import org.apache.asterix.om.utils.NonTaggedFormatUtil;
+import org.apache.asterix.om.utils.RecordUtil;
 import org.apache.hyracks.algebricks.common.exceptions.NotImplementedException;
 import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
 import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
@@ -55,7 +56,7 @@
         this.recordEvalFactory = recordEvalFactory;
         this.fieldIndexEvalFactory = fieldIndexEvalFactory;
         this.recordType = recordType;
-        this.nullBitmapSize = ARecordType.computeNullBitmapSize(recordType);
+        this.nullBitmapSize = RecordUtil.computeNullBitmapSize(recordType);
     }
 
     @Override
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/FieldAccessByNameEvalFactory.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/FieldAccessByNameEvalFactory.java
index 4d13d09..87fa292 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/FieldAccessByNameEvalFactory.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/FieldAccessByNameEvalFactory.java
@@ -27,7 +27,7 @@
 import org.apache.asterix.om.functions.BuiltinFunctions;
 import org.apache.asterix.om.types.ATypeTag;
 import org.apache.asterix.om.types.EnumDeserializer;
-import org.apache.asterix.om.util.NonTaggedFormatUtil;
+import org.apache.asterix.om.utils.NonTaggedFormatUtil;
 import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
 import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
 import org.apache.hyracks.api.context.IHyracksTaskContext;
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/FieldAccessNestedEvalFactory.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/FieldAccessNestedEvalFactory.java
index d95dcfe..f13771e 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/FieldAccessNestedEvalFactory.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/FieldAccessNestedEvalFactory.java
@@ -36,7 +36,8 @@
 import org.apache.asterix.om.types.EnumDeserializer;
 import org.apache.asterix.om.types.IAType;
 import org.apache.asterix.om.types.runtime.RuntimeRecordTypeInfo;
-import org.apache.asterix.om.util.NonTaggedFormatUtil;
+import org.apache.asterix.om.utils.NonTaggedFormatUtil;
+import org.apache.asterix.om.utils.RecordUtil;
 import org.apache.asterix.runtime.exceptions.TypeMismatchException;
 import org.apache.asterix.runtime.exceptions.UnsupportedTypeException;
 import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
@@ -154,7 +155,7 @@
                         if (subFieldIndex == -1) {
                             break;
                         }
-                        nullBitmapSize = ARecordType.computeNullBitmapSize((ARecordType) subType);
+                        nullBitmapSize = RecordUtil.computeNullBitmapSize((ARecordType) subType);
                         subFieldOffset = ARecordSerializerDeserializer.getFieldOffsetById(serRecord, start,
                                 subFieldIndex, nullBitmapSize, ((ARecordType) subType).isOpen());
                         if (subFieldOffset == 0) {
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/GetRecordFieldValueEvalFactory.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/GetRecordFieldValueEvalFactory.java
index b58c19e..c162e3d 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/GetRecordFieldValueEvalFactory.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/GetRecordFieldValueEvalFactory.java
@@ -29,7 +29,8 @@
 import org.apache.asterix.om.types.ATypeTag;
 import org.apache.asterix.om.types.EnumDeserializer;
 import org.apache.asterix.om.types.runtime.RuntimeRecordTypeInfo;
-import org.apache.asterix.om.util.NonTaggedFormatUtil;
+import org.apache.asterix.om.utils.NonTaggedFormatUtil;
+import org.apache.asterix.om.utils.RecordUtil;
 import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
 import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
 import org.apache.hyracks.api.context.IHyracksTaskContext;
@@ -96,7 +97,7 @@
                     // Look at closed fields first.
                     int subFieldIndex = recTypeInfo.getFieldIndex(serFldName, serFldNameOffset + 1, serFldNameLen - 1);
                     if (subFieldIndex >= 0) {
-                        int nullBitmapSize = ARecordType.computeNullBitmapSize(recordType);
+                        int nullBitmapSize = RecordUtil.computeNullBitmapSize(recordType);
                         subFieldOffset = ARecordSerializerDeserializer.getFieldOffsetById(serRecord, serRecordOffset,
                                 subFieldIndex, nullBitmapSize, recordType.isOpen());
                         if (subFieldOffset == 0) {
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordPairsDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordPairsDescriptor.java
index 8ecdca7..7158d2d 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordPairsDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordPairsDescriptor.java
@@ -35,6 +35,7 @@
 import org.apache.asterix.om.types.AOrderedListType;
 import org.apache.asterix.om.types.ARecordType;
 import org.apache.asterix.om.types.ATypeTag;
+import org.apache.asterix.om.utils.RecordUtil;
 import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
 import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
 import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
@@ -79,7 +80,7 @@
                 final ArrayBackedValueStorage itemStorage = new ArrayBackedValueStorage();
                 final DataOutput itemOutput = itemStorage.getDataOutput();
                 final RecordBuilder recBuilder = new RecordBuilder();
-                recBuilder.reset(ARecordType.FULLY_OPEN_RECORD_TYPE);
+                recBuilder.reset(RecordUtil.FULLY_OPEN_RECORD_TYPE);
 
                 // For writing the resulting list of records.
                 final OrderedListBuilder listBuilder = new OrderedListBuilder();
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/formats/NonTaggedDataFormat.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/formats/NonTaggedDataFormat.java
index ad43a48..01aff1a 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/formats/NonTaggedDataFormat.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/formats/NonTaggedDataFormat.java
@@ -59,7 +59,8 @@
 import org.apache.asterix.om.types.ATypeTag;
 import org.apache.asterix.om.types.AUnionType;
 import org.apache.asterix.om.types.IAType;
-import org.apache.asterix.om.util.ConstantExpressionUtil;
+import org.apache.asterix.om.utils.ConstantExpressionUtil;
+import org.apache.asterix.om.utils.RecordUtil;
 import org.apache.asterix.runtime.evaluators.common.CreateMBREvalFactory;
 import org.apache.asterix.runtime.evaluators.common.FunctionManagerImpl;
 import org.apache.asterix.runtime.evaluators.functions.records.FieldAccessByIndexEvalFactory;
@@ -535,7 +536,7 @@
                         break;
                     }
                     case ANY:
-                        fd.setImmutableStates(ARecordType.FULLY_OPEN_RECORD_TYPE, listFieldPath);
+                        fd.setImmutableStates(RecordUtil.FULLY_OPEN_RECORD_TYPE, listFieldPath);
                         break;
                     default: {
                         throw new NotImplementedException("field-access-nested for data of type " + t);
@@ -553,7 +554,7 @@
                 if (typeTag.equals(ATypeTag.RECORD)) {
                     fd.setImmutableStates(t);
                 } else if (typeTag.equals(ATypeTag.ANY)) {
-                    fd.setImmutableStates(ARecordType.FULLY_OPEN_RECORD_TYPE);
+                    fd.setImmutableStates(RecordUtil.FULLY_OPEN_RECORD_TYPE);
                 } else {
                     throw new NotImplementedException("get-record-fields for data of type " + t);
                 }
@@ -569,7 +570,7 @@
                 if (typeTag.equals(ATypeTag.RECORD)) {
                     fd.setImmutableStates(t);
                 } else if (typeTag.equals(ATypeTag.ANY)) {
-                    fd.setImmutableStates(ARecordType.FULLY_OPEN_RECORD_TYPE);
+                    fd.setImmutableStates(RecordUtil.FULLY_OPEN_RECORD_TYPE);
                 } else {
                     throw new NotImplementedException("get-record-field-value for data of type " + t);
                 }
@@ -585,7 +586,7 @@
                 if (typeTag.equals(ATypeTag.RECORD)) {
                     fd.setImmutableStates(t);
                 } else if (typeTag.equals(ATypeTag.ANY)) {
-                    fd.setImmutableStates(ARecordType.FULLY_OPEN_RECORD_TYPE);
+                    fd.setImmutableStates(RecordUtil.FULLY_OPEN_RECORD_TYPE);
                 } else {
                     throw new NotImplementedException("record-fields with data of type " + t);
                 }
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/message/CompleteFailbackResponseMessage.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/message/CompleteFailbackResponseMessage.java
index cb56c39..07c366c 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/message/CompleteFailbackResponseMessage.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/message/CompleteFailbackResponseMessage.java
@@ -20,7 +20,7 @@
 
 import java.util.Set;
 
-import org.apache.asterix.runtime.util.ClusterStateManager;
+import org.apache.asterix.runtime.utils.ClusterStateManager;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.api.service.IControllerService;
 
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/message/PreparePartitionsFailbackResponseMessage.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/message/PreparePartitionsFailbackResponseMessage.java
index d87cd23..c655ecd 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/message/PreparePartitionsFailbackResponseMessage.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/message/PreparePartitionsFailbackResponseMessage.java
@@ -20,7 +20,7 @@
 
 import java.util.Set;
 
-import org.apache.asterix.runtime.util.ClusterStateManager;
+import org.apache.asterix.runtime.utils.ClusterStateManager;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.api.service.IControllerService;
 
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/message/ReportMaxResourceIdMessage.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/message/ReportMaxResourceIdMessage.java
index b1e6e69..dd7335a 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/message/ReportMaxResourceIdMessage.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/message/ReportMaxResourceIdMessage.java
@@ -27,7 +27,7 @@
 import org.apache.asterix.common.messaging.api.INCMessageBroker;
 import org.apache.asterix.common.metadata.MetadataIndexImmutableProperties;
 import org.apache.asterix.common.transactions.IResourceIdManager;
-import org.apache.asterix.runtime.util.AppContextInfo;
+import org.apache.asterix.runtime.utils.AppContextInfo;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.api.service.IControllerService;
 import org.apache.hyracks.control.nc.NodeControllerService;
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/message/ResourceIdRequestMessage.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/message/ResourceIdRequestMessage.java
index 9247e09..2fedcca 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/message/ResourceIdRequestMessage.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/message/ResourceIdRequestMessage.java
@@ -24,8 +24,8 @@
 import org.apache.asterix.common.messaging.api.IApplicationMessage;
 import org.apache.asterix.common.messaging.api.ICCMessageBroker;
 import org.apache.asterix.common.transactions.IResourceIdManager;
-import org.apache.asterix.runtime.util.AppContextInfo;
-import org.apache.asterix.runtime.util.ClusterStateManager;
+import org.apache.asterix.runtime.utils.AppContextInfo;
+import org.apache.asterix.runtime.utils.ClusterStateManager;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.api.service.IControllerService;
 
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/message/TakeoverMetadataNodeResponseMessage.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/message/TakeoverMetadataNodeResponseMessage.java
index d3c3502..d8b2136 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/message/TakeoverMetadataNodeResponseMessage.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/message/TakeoverMetadataNodeResponseMessage.java
@@ -19,7 +19,7 @@
 package org.apache.asterix.runtime.message;
 
 import org.apache.asterix.common.messaging.api.IApplicationMessage;
-import org.apache.asterix.runtime.util.ClusterStateManager;
+import org.apache.asterix.runtime.utils.ClusterStateManager;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.api.service.IControllerService;
 
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/message/TakeoverPartitionsResponseMessage.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/message/TakeoverPartitionsResponseMessage.java
index 3adc8e9..81492c2 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/message/TakeoverPartitionsResponseMessage.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/message/TakeoverPartitionsResponseMessage.java
@@ -19,7 +19,7 @@
 package org.apache.asterix.runtime.message;
 
 import org.apache.asterix.common.messaging.api.IApplicationMessage;
-import org.apache.asterix.runtime.util.ClusterStateManager;
+import org.apache.asterix.runtime.utils.ClusterStateManager;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.api.service.IControllerService;
 
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/LSMInvertedIndexUpsertOperatorDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/LSMInvertedIndexUpsertOperatorDescriptor.java
index 8e88b9c..f1547a8 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/LSMInvertedIndexUpsertOperatorDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/LSMInvertedIndexUpsertOperatorDescriptor.java
@@ -29,13 +29,13 @@
 import org.apache.hyracks.api.job.IOperatorDescriptorRegistry;
 import org.apache.hyracks.dataflow.std.file.IFileSplitProvider;
 import org.apache.hyracks.storage.am.common.api.IIndexLifecycleManagerProvider;
-import org.apache.hyracks.storage.am.common.api.IMetadataPageManagerFactory;
 import org.apache.hyracks.storage.am.common.api.IModificationOperationCallbackFactory;
+import org.apache.hyracks.storage.am.common.api.IPageManagerFactory;
 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.ophelpers.IndexOperation;
 import org.apache.hyracks.storage.am.lsm.invertedindex.tokenizers.IBinaryTokenizerFactory;
-import org.apache.hyracks.storage.common.IStorageManagerInterface;
+import org.apache.hyracks.storage.common.IStorageManager;
 
 public class LSMInvertedIndexUpsertOperatorDescriptor
         extends LSMInvertedIndexInsertDeleteOperatorDescriptor {
@@ -44,17 +44,17 @@
     private final int[] prevFieldPermutation;
 
     public LSMInvertedIndexUpsertOperatorDescriptor(IOperatorDescriptorRegistry spec, RecordDescriptor recDesc,
-            IStorageManagerInterface storageManager, IFileSplitProvider fileSplitProvider,
+            IStorageManager storageManager, IFileSplitProvider fileSplitProvider,
             IIndexLifecycleManagerProvider lifecycleManagerProvider, ITypeTraits[] tokenTypeTraits,
             IBinaryComparatorFactory[] tokenComparatorFactories, ITypeTraits[] invListsTypeTraits,
             IBinaryComparatorFactory[] invListComparatorFactories, IBinaryTokenizerFactory tokenizerFactory,
             int[] fieldPermutation, IIndexDataflowHelperFactory dataflowHelperFactory,
             ITupleFilterFactory tupleFilterFactory, IModificationOperationCallbackFactory modificationOpCallbackFactory,
-            String indexName, int[] prevFieldPermutation, IMetadataPageManagerFactory metadataPageManagerFactory) {
+            String indexName, int[] prevFieldPermutation, IPageManagerFactory pageManagerFactory) {
         super(spec, recDesc, storageManager, fileSplitProvider, lifecycleManagerProvider, tokenTypeTraits,
                 tokenComparatorFactories, invListsTypeTraits, invListComparatorFactories, tokenizerFactory,
                 fieldPermutation, IndexOperation.UPSERT, dataflowHelperFactory, tupleFilterFactory,
-                modificationOpCallbackFactory, indexName, metadataPageManagerFactory);
+                modificationOpCallbackFactory, indexName, pageManagerFactory);
         this.prevFieldPermutation = prevFieldPermutation;
     }
 
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/LSMPrimaryUpsertOperatorNodePushable.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/LSMPrimaryUpsertOperatorNodePushable.java
index 878f3eb..b2ffd6e 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/LSMPrimaryUpsertOperatorNodePushable.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/LSMPrimaryUpsertOperatorNodePushable.java
@@ -45,7 +45,7 @@
 import org.apache.hyracks.dataflow.common.comm.util.FrameUtils;
 import org.apache.hyracks.dataflow.common.data.accessors.FrameTupleReference;
 import org.apache.hyracks.dataflow.common.data.accessors.ITupleReference;
-import org.apache.hyracks.dataflow.common.util.TaskUtils;
+import org.apache.hyracks.dataflow.common.utils.TaskUtil;
 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.IIndexCursor;
@@ -126,7 +126,7 @@
         try {
             if (ctx.getSharedObject() != null) {
                 PrimaryIndexLogMarkerCallback callback = new PrimaryIndexLogMarkerCallback((AbstractLSMIndex) index);
-                TaskUtils.putInSharedMap(ILogMarkerCallback.KEY_MARKER_CALLBACK, callback, ctx);
+                TaskUtil.putInSharedMap(ILogMarkerCallback.KEY_MARKER_CALLBACK, callback, ctx);
             }
             missingTupleBuilder = new ArrayTupleBuilder(1);
             DataOutput out = missingTupleBuilder.getDataOutput();
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/LSMTreeUpsertOperatorDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/LSMTreeUpsertOperatorDescriptor.java
index 79a02a4..fe69a04 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/LSMTreeUpsertOperatorDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/LSMTreeUpsertOperatorDescriptor.java
@@ -31,13 +31,13 @@
 import org.apache.hyracks.api.job.IOperatorDescriptorRegistry;
 import org.apache.hyracks.dataflow.std.file.IFileSplitProvider;
 import org.apache.hyracks.storage.am.common.api.IIndexLifecycleManagerProvider;
-import org.apache.hyracks.storage.am.common.api.IMetadataPageManagerFactory;
 import org.apache.hyracks.storage.am.common.api.IModificationOperationCallbackFactory;
+import org.apache.hyracks.storage.am.common.api.IPageManagerFactory;
 import org.apache.hyracks.storage.am.common.api.ISearchOperationCallbackFactory;
 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.ophelpers.IndexOperation;
-import org.apache.hyracks.storage.common.IStorageManagerInterface;
+import org.apache.hyracks.storage.common.IStorageManager;
 
 public class LSMTreeUpsertOperatorDescriptor extends LSMTreeInsertDeleteOperatorDescriptor {
 
@@ -47,18 +47,18 @@
     private int filterIndex = -1;
 
     public LSMTreeUpsertOperatorDescriptor(IOperatorDescriptorRegistry spec, RecordDescriptor recDesc,
-            IStorageManagerInterface storageManager, IIndexLifecycleManagerProvider lifecycleManagerProvider,
+            IStorageManager storageManager, IIndexLifecycleManagerProvider lifecycleManagerProvider,
             IFileSplitProvider fileSplitProvider, ITypeTraits[] typeTraits,
             IBinaryComparatorFactory[] comparatorFactories, int[] bloomFilterKeyFields, int[] fieldPermutation,
             IIndexDataflowHelperFactory dataflowHelperFactory, ITupleFilterFactory tupleFilterFactory,
             boolean isPrimary, String indexName, IMissingWriterFactory missingWriterFactory,
             IModificationOperationCallbackFactory modificationOpCallbackProvider,
             ISearchOperationCallbackFactory searchOpCallbackProvider, int[] prevValuePermutation,
-            IMetadataPageManagerFactory metadataPageManagerFactory) {
+            IPageManagerFactory pageManagerFactory) {
         super(spec, recDesc, storageManager, lifecycleManagerProvider, fileSplitProvider, typeTraits,
                 comparatorFactories, bloomFilterKeyFields, fieldPermutation, IndexOperation.UPSERT,
                 dataflowHelperFactory, tupleFilterFactory, isPrimary, indexName, missingWriterFactory,
-                modificationOpCallbackProvider, searchOpCallbackProvider, metadataPageManagerFactory);
+                modificationOpCallbackProvider, searchOpCallbackProvider, pageManagerFactory);
         this.prevValuePermutation = prevValuePermutation;
     }
 
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/unnestingfunctions/std/SubsetCollectionDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/unnestingfunctions/std/SubsetCollectionDescriptor.java
index 1d4dd89..84a59ac 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/unnestingfunctions/std/SubsetCollectionDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/unnestingfunctions/std/SubsetCollectionDescriptor.java
@@ -31,7 +31,7 @@
 import org.apache.asterix.om.types.ATypeTag;
 import org.apache.asterix.om.types.EnumDeserializer;
 import org.apache.asterix.om.types.hierachy.ATypeHierarchy;
-import org.apache.asterix.om.util.NonTaggedFormatUtil;
+import org.apache.asterix.om.utils.NonTaggedFormatUtil;
 import org.apache.asterix.runtime.unnestingfunctions.base.AbstractUnnestingFunctionDynamicDescriptor;
 import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
 import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/util/AppContextInfo.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/utils/AppContextInfo.java
similarity index 89%
rename from asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/util/AppContextInfo.java
rename to asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/utils/AppContextInfo.java
index ce3c2ca..355f503 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/util/AppContextInfo.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/utils/AppContextInfo.java
@@ -16,25 +16,25 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.asterix.runtime.util;
+package org.apache.asterix.runtime.utils;
 
 import java.io.IOException;
 import java.util.function.Supplier;
 import java.util.logging.Logger;
 
-import org.apache.asterix.common.cluster.IGlobalRecoveryMaanger;
+import org.apache.asterix.common.cluster.IGlobalRecoveryManager;
 import org.apache.asterix.common.config.BuildProperties;
 import org.apache.asterix.common.config.CompilerProperties;
 import org.apache.asterix.common.config.ExtensionProperties;
 import org.apache.asterix.common.config.ExternalProperties;
 import org.apache.asterix.common.config.FeedProperties;
+import org.apache.asterix.common.config.IPropertiesProvider;
+import org.apache.asterix.common.config.MessagingProperties;
 import org.apache.asterix.common.config.MetadataProperties;
 import org.apache.asterix.common.config.PropertiesAccessor;
 import org.apache.asterix.common.config.ReplicationProperties;
 import org.apache.asterix.common.config.StorageProperties;
 import org.apache.asterix.common.config.TransactionProperties;
-import org.apache.asterix.common.config.IPropertiesProvider;
-import org.apache.asterix.common.config.MessagingProperties;
 import org.apache.asterix.common.dataflow.IApplicationContextInfo;
 import org.apache.asterix.common.exceptions.AsterixException;
 import org.apache.asterix.common.library.ILibraryManager;
@@ -43,7 +43,7 @@
 import org.apache.hyracks.api.application.ICCApplicationContext;
 import org.apache.hyracks.api.client.IHyracksClientConnection;
 import org.apache.hyracks.storage.am.common.api.IIndexLifecycleManagerProvider;
-import org.apache.hyracks.storage.common.IStorageManagerInterface;
+import org.apache.hyracks.storage.common.IStorageManager;
 
 /*
  * Acts as an holder class for IndexRegistryProvider, AsterixStorageManager
@@ -54,7 +54,7 @@
 
     public static final AppContextInfo INSTANCE = new AppContextInfo();
     private ICCApplicationContext appCtx;
-    private IGlobalRecoveryMaanger globalRecoveryMaanger;
+    private IGlobalRecoveryManager globalRecoveryManager;
     private ILibraryManager libraryManager;
     private IResourceIdManager resourceIdManager;
     private CompilerProperties compilerProperties;
@@ -76,10 +76,8 @@
     }
 
     public static synchronized void initialize(ICCApplicationContext ccAppCtx, IHyracksClientConnection hcc,
-                                               IGlobalRecoveryMaanger globalRecoveryMaanger,
-                                               ILibraryManager libraryManager,
-                                               IResourceIdManager resourceIdManager,
-                                               Supplier<IMetadataBootstrap> metadataBootstrapSupplier)
+            ILibraryManager libraryManager, IResourceIdManager resourceIdManager,
+            Supplier<IMetadataBootstrap> metadataBootstrapSupplier, IGlobalRecoveryManager globalRecoveryManager)
             throws AsterixException, IOException {
         if (INSTANCE.initialized) {
             throw new AsterixException(AppContextInfo.class.getSimpleName() + " has been initialized already");
@@ -87,7 +85,6 @@
         INSTANCE.initialized = true;
         INSTANCE.appCtx = ccAppCtx;
         INSTANCE.hcc = hcc;
-        INSTANCE.globalRecoveryMaanger = globalRecoveryMaanger;
         INSTANCE.libraryManager = libraryManager;
         INSTANCE.resourceIdManager = resourceIdManager;
         // Determine whether to use old-style asterix-configuration.xml or new-style configuration.
@@ -105,6 +102,7 @@
         INSTANCE.buildProperties = new BuildProperties(propertiesAccessor);
         INSTANCE.messagingProperties = new MessagingProperties(propertiesAccessor);
         INSTANCE.metadataBootstrapSupplier = metadataBootstrapSupplier;
+        INSTANCE.globalRecoveryManager = globalRecoveryManager;
 
         Logger.getLogger("org.apache.asterix").setLevel(INSTANCE.externalProperties.getLogLevel());
         Logger.getLogger("org.apache.hyracks").setLevel(INSTANCE.externalProperties.getLogLevel());
@@ -164,7 +162,7 @@
     }
 
     @Override
-    public IStorageManagerInterface getStorageManagerInterface() {
+    public IStorageManager getStorageManager() {
         return RuntimeComponentsProvider.RUNTIME_PROVIDER;
     }
 
@@ -174,8 +172,8 @@
     }
 
     @Override
-    public IGlobalRecoveryMaanger getGlobalRecoveryManager() {
-        return globalRecoveryMaanger;
+    public IGlobalRecoveryManager getGlobalRecoveryManager() {
+        return globalRecoveryManager;
     }
 
     @Override
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/util/ClusterStateManager.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/utils/ClusterStateManager.java
similarity index 99%
rename from asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/util/ClusterStateManager.java
rename to asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/utils/ClusterStateManager.java
index c79524a..8fffdb1 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/util/ClusterStateManager.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/utils/ClusterStateManager.java
@@ -16,7 +16,7 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.asterix.runtime.util;
+package org.apache.asterix.runtime.utils;
 
 import java.util.ArrayList;
 import java.util.HashMap;
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/util/RuntimeComponentsProvider.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/utils/RuntimeComponentsProvider.java
similarity index 95%
rename from asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/util/RuntimeComponentsProvider.java
rename to asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/utils/RuntimeComponentsProvider.java
index 5b4daa6..117fa9e 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/util/RuntimeComponentsProvider.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/utils/RuntimeComponentsProvider.java
@@ -16,7 +16,7 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.asterix.runtime.util;
+package org.apache.asterix.runtime.utils;
 
 import org.apache.asterix.common.api.IAppRuntimeContext;
 import org.apache.asterix.common.api.IDatasetLifecycleManager;
@@ -24,13 +24,13 @@
 import org.apache.hyracks.storage.am.common.api.IIndexLifecycleManagerProvider;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationScheduler;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationSchedulerProvider;
-import org.apache.hyracks.storage.common.IStorageManagerInterface;
+import org.apache.hyracks.storage.common.IStorageManager;
 import org.apache.hyracks.storage.common.buffercache.IBufferCache;
 import org.apache.hyracks.storage.common.file.IFileMapProvider;
 import org.apache.hyracks.storage.common.file.ILocalResourceRepository;
 import org.apache.hyracks.storage.common.file.IResourceIdFactory;
 
-public class RuntimeComponentsProvider implements IIndexLifecycleManagerProvider, IStorageManagerInterface,
+public class RuntimeComponentsProvider implements IIndexLifecycleManagerProvider, IStorageManager,
         ILSMIOOperationSchedulerProvider {
 
     private static final long serialVersionUID = 1L;
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/util/RuntimeUtils.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/utils/RuntimeUtils.java
similarity index 80%
rename from asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/util/RuntimeUtils.java
rename to asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/utils/RuntimeUtils.java
index 608def7..8e3f989 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/util/RuntimeUtils.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/utils/RuntimeUtils.java
@@ -16,7 +16,7 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.asterix.runtime.util;
+package org.apache.asterix.runtime.utils;
 
 import java.net.InetAddress;
 import java.util.ArrayList;
@@ -26,14 +26,12 @@
 import java.util.Map;
 import java.util.Set;
 
+import org.apache.asterix.common.config.CompilerProperties;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.job.JobSpecification;
 import org.apache.hyracks.control.cc.ClusterControllerService;
 import org.apache.hyracks.control.cc.cluster.INodeManager;
 
-/**
- * Utility class for obtaining information on the set of Hyracks NodeController
- * processes that are running on a given host.
- */
 public class RuntimeUtils {
 
     private RuntimeUtils() {
@@ -60,9 +58,15 @@
     }
 
     public static void getNodeControllerMap(Map<InetAddress, Set<String>> map) {
-        ClusterControllerService ccs = (ClusterControllerService) AppContextInfo.INSTANCE
-                .getCCApplicationContext().getControllerService();
+        ClusterControllerService ccs =
+                (ClusterControllerService) AppContextInfo.INSTANCE.getCCApplicationContext().getControllerService();
         INodeManager nodeManager = ccs.getNodeManager();
         map.putAll(nodeManager.getIpAddressNodeNameMap());
     }
+
+    public static JobSpecification createJobSpecification() {
+        CompilerProperties compilerProperties = AppContextInfo.INSTANCE.getCompilerProperties();
+        int frameSize = compilerProperties.getFrameSize();
+        return new JobSpecification(frameSize);
+    }
 }
diff --git a/asterixdb/asterix-tools/src/test/java/org/apache/asterix/tools/datagen/AdmDataGen.java b/asterixdb/asterix-tools/src/test/java/org/apache/asterix/tools/datagen/AdmDataGen.java
index 2918575..7dacce3 100644
--- a/asterixdb/asterix-tools/src/test/java/org/apache/asterix/tools/datagen/AdmDataGen.java
+++ b/asterixdb/asterix-tools/src/test/java/org/apache/asterix/tools/datagen/AdmDataGen.java
@@ -67,7 +67,7 @@
 import org.apache.asterix.om.types.BuiltinType;
 import org.apache.asterix.om.types.IAType;
 import org.apache.asterix.om.types.TypeSignature;
-import org.apache.asterix.om.util.NonTaggedFormatUtil;
+import org.apache.asterix.om.utils.NonTaggedFormatUtil;
 import org.apache.asterix.tools.translator.ADGenDmlTranslator;
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
 import org.apache.hyracks.algebricks.common.exceptions.NotImplementedException;
diff --git a/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/opcallbacks/PrimaryIndexOperationTrackerProvider.java b/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/opcallbacks/PrimaryIndexOperationTrackerFactory.java
similarity index 76%
rename from asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/opcallbacks/PrimaryIndexOperationTrackerProvider.java
rename to asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/opcallbacks/PrimaryIndexOperationTrackerFactory.java
index 03a862c..4e12f96 100644
--- a/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/opcallbacks/PrimaryIndexOperationTrackerProvider.java
+++ b/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/opcallbacks/PrimaryIndexOperationTrackerFactory.java
@@ -21,24 +21,24 @@
 
 import org.apache.asterix.common.api.IAppRuntimeContext;
 import org.apache.asterix.common.api.IDatasetLifecycleManager;
-import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.application.INCApplicationContext;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMOperationTracker;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMOperationTrackerProvider;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMOperationTrackerFactory;
 
-public class PrimaryIndexOperationTrackerProvider implements ILSMOperationTrackerProvider {
+public class PrimaryIndexOperationTrackerFactory implements ILSMOperationTrackerFactory {
 
     private static final long serialVersionUID = 1L;
 
     private final int datasetID;
 
-    public PrimaryIndexOperationTrackerProvider(int datasetID) {
+    public PrimaryIndexOperationTrackerFactory(int datasetID) {
         this.datasetID = datasetID;
     }
 
     @Override
-    public ILSMOperationTracker getOperationTracker(IHyracksTaskContext ctx) {
-        IDatasetLifecycleManager dslcManager = ((IAppRuntimeContext) ctx.getJobletContext()
-                .getApplicationContext().getApplicationObject()).getDatasetLifecycleManager();
+    public ILSMOperationTracker getOperationTracker(INCApplicationContext ctx) {
+        IDatasetLifecycleManager dslcManager = ((IAppRuntimeContext) ctx.getApplicationObject())
+                .getDatasetLifecycleManager();
         return dslcManager.getOperationTracker(datasetID);
     }
 
diff --git a/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/opcallbacks/SecondaryIndexOperationTrackerProvider.java b/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/opcallbacks/SecondaryIndexOperationTrackerFactory.java
similarity index 77%
rename from asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/opcallbacks/SecondaryIndexOperationTrackerProvider.java
rename to asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/opcallbacks/SecondaryIndexOperationTrackerFactory.java
index 6fe71d1..be98704 100644
--- a/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/opcallbacks/SecondaryIndexOperationTrackerProvider.java
+++ b/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/opcallbacks/SecondaryIndexOperationTrackerFactory.java
@@ -21,24 +21,24 @@
 import org.apache.asterix.common.api.IAppRuntimeContext;
 import org.apache.asterix.common.api.IDatasetLifecycleManager;
 import org.apache.asterix.common.context.BaseOperationTracker;
-import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.application.INCApplicationContext;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMOperationTracker;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMOperationTrackerProvider;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMOperationTrackerFactory;
 
-public class SecondaryIndexOperationTrackerProvider implements ILSMOperationTrackerProvider {
+public class SecondaryIndexOperationTrackerFactory implements ILSMOperationTrackerFactory {
 
     private static final long serialVersionUID = 1L;
 
     private final int datasetID;
 
-    public SecondaryIndexOperationTrackerProvider(int datasetID) {
+    public SecondaryIndexOperationTrackerFactory(int datasetID) {
         this.datasetID = datasetID;
     }
 
     @Override
-    public ILSMOperationTracker getOperationTracker(IHyracksTaskContext ctx) {
-        IDatasetLifecycleManager dslcManager = ((IAppRuntimeContext) ctx
-                .getJobletContext().getApplicationContext().getApplicationObject()).getDatasetLifecycleManager();
+    public ILSMOperationTracker getOperationTracker(INCApplicationContext ctx) {
+        IDatasetLifecycleManager dslcManager = ((IAppRuntimeContext) ctx.getApplicationObject())
+                .getDatasetLifecycleManager();
         return new BaseOperationTracker(datasetID, dslcManager.getDatasetInfo(datasetID));
     }
 
diff --git a/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/resource/ExternalBTreeLocalResourceMetadata.java b/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/resource/ExternalBTreeLocalResourceMetadata.java
index 6fd5298..2d6f3c9 100644
--- a/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/resource/ExternalBTreeLocalResourceMetadata.java
+++ b/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/resource/ExternalBTreeLocalResourceMetadata.java
@@ -20,19 +20,19 @@
 
 import java.util.Map;
 
-import org.apache.asterix.common.context.BaseOperationTracker;
-import org.apache.asterix.common.dataflow.LSMIndexUtil;
-import org.apache.asterix.common.ioopcallbacks.LSMBTreeIOOperationCallbackFactory;
-import org.apache.asterix.common.transactions.IAppRuntimeContextProvider;
+import org.apache.asterix.common.api.IAppRuntimeContext;
+import org.apache.hyracks.api.application.INCApplicationContext;
 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.storage.am.lsm.btree.impls.LSMBTree;
-import org.apache.hyracks.storage.am.lsm.btree.util.LSMBTreeUtils;
+import org.apache.hyracks.storage.am.common.api.IMetadataPageManagerFactory;
+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.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.common.file.LocalResource;
 
 public class ExternalBTreeLocalResourceMetadata extends LSMBTreeLocalResourceMetadata {
@@ -42,26 +42,28 @@
     public ExternalBTreeLocalResourceMetadata(ITypeTraits[] typeTraits, IBinaryComparatorFactory[] cmpFactories,
             int[] bloomFilterKeyFields, boolean isPrimary, int datasetID, int partition,
             ILSMMergePolicyFactory mergePolicyFactory,
-            Map<String, String> mergePolicyProperties) {
+            Map<String, String> mergePolicyProperties, ILSMOperationTrackerFactory opTrackerProvider,
+            ILSMIOOperationCallbackFactory ioOpCallbackFactory,
+            IMetadataPageManagerFactory metadataPageManagerFactory) {
         super(typeTraits, cmpFactories, bloomFilterKeyFields, isPrimary, datasetID, partition, mergePolicyFactory,
-                mergePolicyProperties, null, null, null, null);
+                mergePolicyProperties, null, null, null, null, opTrackerProvider, ioOpCallbackFactory,
+                metadataPageManagerFactory);
     }
 
     @Override
-    public ILSMIndex createIndexInstance(IAppRuntimeContextProvider runtimeContextProvider,
+    public ILSMIndex createIndexInstance(INCApplicationContext appCtx,
             LocalResource resource) throws HyracksDataException {
+        IAppRuntimeContext runtimeContextProvider = (IAppRuntimeContext) appCtx.getApplicationObject();
         IIOManager ioManager = runtimeContextProvider.getIOManager();
         FileReference file = ioManager.resolve(resource.getPath());
-        LSMBTree lsmBTree = LSMBTreeUtils.createExternalBTree(ioManager, file, runtimeContextProvider.getBufferCache(),
+        return LSMBTreeUtil.createExternalBTree(ioManager, file, runtimeContextProvider.getBufferCache(),
                 runtimeContextProvider.getFileMapManager(), typeTraits, cmpFactories, bloomFilterKeyFields,
                 runtimeContextProvider.getBloomFilterFalsePositiveRate(),
                 mergePolicyFactory.createMergePolicy(mergePolicyProperties,
                         runtimeContextProvider.getDatasetLifecycleManager()),
-                new BaseOperationTracker(datasetId(),
-                        runtimeContextProvider.getDatasetLifecycleManager().getDatasetInfo(datasetId())),
+                opTrackerProvider.getOperationTracker(appCtx),
                 runtimeContextProvider.getLSMIOScheduler(),
-                LSMBTreeIOOperationCallbackFactory.INSTANCE.createIOOperationCallback(), -1, true,
-                LSMIndexUtil.getMetadataPageManagerFactory());
-        return lsmBTree;
+                ioOpCallbackFactory.createIOOperationCallback(), -1, true,
+                metadataPageManagerFactory);
     }
 }
diff --git a/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/resource/ExternalBTreeLocalResourceMetadataFactory.java b/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/resource/ExternalBTreeLocalResourceMetadataFactory.java
index 61d19c3..ea25aab 100644
--- a/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/resource/ExternalBTreeLocalResourceMetadataFactory.java
+++ b/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/resource/ExternalBTreeLocalResourceMetadataFactory.java
@@ -23,7 +23,10 @@
 import org.apache.asterix.common.transactions.Resource;
 import org.apache.hyracks.api.dataflow.value.IBinaryComparatorFactory;
 import org.apache.hyracks.api.dataflow.value.ITypeTraits;
+import org.apache.hyracks.storage.am.common.api.IMetadataPageManagerFactory;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallbackFactory;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMMergePolicyFactory;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMOperationTrackerFactory;
 
 public class ExternalBTreeLocalResourceMetadataFactory extends LSMBTreeLocalResourceMetadataFactory {
 
@@ -32,14 +35,18 @@
     public ExternalBTreeLocalResourceMetadataFactory(ITypeTraits[] typeTraits, IBinaryComparatorFactory[] cmpFactories,
             int[] bloomFilterKeyFields, boolean isPrimary, int datasetID,
             ILSMMergePolicyFactory mergePolicyFactory,
-            Map<String, String> mergePolicyProperties) {
+            Map<String, String> mergePolicyProperties, ILSMOperationTrackerFactory opTrackerProvider,
+            ILSMIOOperationCallbackFactory ioOpCallbackFactory,
+            IMetadataPageManagerFactory metadataPageManagerFactory) {
         super(typeTraits, cmpFactories, bloomFilterKeyFields, isPrimary, datasetID, mergePolicyFactory,
-                mergePolicyProperties, null, null, null, null);
+                mergePolicyProperties, null, null, null, null, opTrackerProvider, ioOpCallbackFactory,
+                metadataPageManagerFactory);
     }
 
     @Override
     public Resource resource(int partition) {
         return new ExternalBTreeLocalResourceMetadata(filterTypeTraits, filterCmpFactories, bloomFilterKeyFields,
-                isPrimary, datasetId, partition, mergePolicyFactory, mergePolicyProperties);
+                isPrimary, datasetId, partition, mergePolicyFactory, mergePolicyProperties, opTrackerProvider,
+                ioOpCallbackFactory, metadataPageManagerFactory);
     }
 }
diff --git a/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/resource/ExternalBTreeWithBuddyLocalResourceMetadata.java b/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/resource/ExternalBTreeWithBuddyLocalResourceMetadata.java
index 1e029a9..e411f8b 100644
--- a/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/resource/ExternalBTreeWithBuddyLocalResourceMetadata.java
+++ b/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/resource/ExternalBTreeWithBuddyLocalResourceMetadata.java
@@ -20,19 +20,20 @@
 
 import java.util.Map;
 
-import org.apache.asterix.common.context.BaseOperationTracker;
-import org.apache.asterix.common.dataflow.LSMIndexUtil;
-import org.apache.asterix.common.ioopcallbacks.LSMBTreeWithBuddyIOOperationCallbackFactory;
-import org.apache.asterix.common.transactions.IAppRuntimeContextProvider;
+import org.apache.asterix.common.api.IAppRuntimeContext;
 import org.apache.asterix.common.transactions.Resource;
+import org.apache.hyracks.api.application.INCApplicationContext;
 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.storage.am.lsm.btree.util.LSMBTreeUtils;
+import org.apache.hyracks.storage.am.common.api.IMetadataPageManagerFactory;
+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.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.common.file.LocalResource;
 
 /**
@@ -51,8 +52,11 @@
     public ExternalBTreeWithBuddyLocalResourceMetadata(int datasetID, int partition,
             IBinaryComparatorFactory[] btreeCmpFactories,
             ITypeTraits[] typeTraits, ILSMMergePolicyFactory mergePolicyFactory,
-            Map<String, String> mergePolicyProperties, int[] buddyBtreeFields) {
-        super(datasetID, partition, null, null, null);
+            Map<String, String> mergePolicyProperties, int[] buddyBtreeFields,
+            ILSMOperationTrackerFactory opTrackerProvider, ILSMIOOperationCallbackFactory ioOpCallbackFactory,
+            IMetadataPageManagerFactory metadataPageManagerFactory) {
+        super(datasetID, partition, null, null, null, opTrackerProvider, ioOpCallbackFactory,
+                metadataPageManagerFactory);
         this.btreeCmpFactories = btreeCmpFactories;
         this.typeTraits = typeTraits;
         this.mergePolicyFactory = mergePolicyFactory;
@@ -61,19 +65,21 @@
     }
 
     @Override
-    public ILSMIndex createIndexInstance(IAppRuntimeContextProvider runtimeContextProvider,
-            LocalResource resource) throws HyracksDataException {
-        IIOManager ioManager = runtimeContextProvider.getIOManager();
+    public ILSMIndex createIndexInstance(INCApplicationContext appCtx, LocalResource resource)
+            throws HyracksDataException {
+        IAppRuntimeContext appRuntimeCtx = (IAppRuntimeContext) appCtx.getApplicationObject();
+        IIOManager ioManager = appCtx.getIoManager();
         FileReference file = ioManager.resolve(resource.getPath());
-        return LSMBTreeUtils.createExternalBTreeWithBuddy(ioManager, file, runtimeContextProvider.getBufferCache(),
-                runtimeContextProvider.getFileMapManager(), typeTraits, btreeCmpFactories,
-                runtimeContextProvider.getBloomFilterFalsePositiveRate(),
-                mergePolicyFactory.createMergePolicy(mergePolicyProperties,
-                        runtimeContextProvider.getDatasetLifecycleManager()),
-                new BaseOperationTracker(datasetId(),
-                        runtimeContextProvider.getDatasetLifecycleManager().getDatasetInfo(datasetId())),
-                runtimeContextProvider.getLSMIOScheduler(),
-                LSMBTreeWithBuddyIOOperationCallbackFactory.INSTANCE.createIOOperationCallback(), buddyBtreeFields, -1,
-                true, LSMIndexUtil.getMetadataPageManagerFactory());
+        return LSMBTreeUtil.createExternalBTreeWithBuddy(
+                ioManager, file, appRuntimeCtx.getBufferCache(),
+                appRuntimeCtx.getFileMapManager(), typeTraits, btreeCmpFactories,
+                appRuntimeCtx.getBloomFilterFalsePositiveRate(),
+                mergePolicyFactory.createMergePolicy(
+                        mergePolicyProperties,
+                        appRuntimeCtx.getDatasetLifecycleManager()),
+                opTrackerProvider.getOperationTracker(appCtx),
+                appRuntimeCtx.getLSMIOScheduler(),
+                ioOpCallbackFactory.createIOOperationCallback(), buddyBtreeFields, -1,
+                true, metadataPageManagerFactory);
     }
 }
diff --git a/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/resource/ExternalBTreeWithBuddyLocalResourceMetadataFactory.java b/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/resource/ExternalBTreeWithBuddyLocalResourceMetadataFactory.java
index 836bcf4..98cd6dc 100644
--- a/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/resource/ExternalBTreeWithBuddyLocalResourceMetadataFactory.java
+++ b/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/resource/ExternalBTreeWithBuddyLocalResourceMetadataFactory.java
@@ -24,7 +24,10 @@
 import org.apache.asterix.common.transactions.ResourceFactory;
 import org.apache.hyracks.api.dataflow.value.IBinaryComparatorFactory;
 import org.apache.hyracks.api.dataflow.value.ITypeTraits;
+import org.apache.hyracks.storage.am.common.api.IMetadataPageManagerFactory;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallbackFactory;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMMergePolicyFactory;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMOperationTrackerFactory;
 
 public class ExternalBTreeWithBuddyLocalResourceMetadataFactory extends ResourceFactory {
     private static final long serialVersionUID = 1L;
@@ -37,8 +40,11 @@
     public ExternalBTreeWithBuddyLocalResourceMetadataFactory(int datasetID,
             IBinaryComparatorFactory[] btreeCmpFactories,
             ITypeTraits[] typeTraits, ILSMMergePolicyFactory mergePolicyFactory,
-            Map<String, String> mergePolicyProperties, int[] buddyBtreeFields) {
-        super(datasetID, null, null, null);
+            Map<String, String> mergePolicyProperties, int[] buddyBtreeFields,
+            ILSMOperationTrackerFactory opTrackerProvider,
+            ILSMIOOperationCallbackFactory ioOpCallbackFactory,
+            IMetadataPageManagerFactory metadataPageManagerFactory) {
+        super(datasetID, null, null, null, opTrackerProvider, ioOpCallbackFactory, metadataPageManagerFactory);
         this.btreeCmpFactories = btreeCmpFactories;
         this.typeTraits = typeTraits;
         this.mergePolicyFactory = mergePolicyFactory;
@@ -49,6 +55,7 @@
     @Override
     public Resource resource(int partition) {
         return new ExternalBTreeWithBuddyLocalResourceMetadata(datasetId, partition, btreeCmpFactories, typeTraits,
-                mergePolicyFactory, mergePolicyProperties, buddyBtreeFields);
+                mergePolicyFactory, mergePolicyProperties, buddyBtreeFields, opTrackerProvider, ioOpCallbackFactory,
+                metadataPageManagerFactory);
     }
 }
diff --git a/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/resource/ExternalRTreeLocalResourceMetadata.java b/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/resource/ExternalRTreeLocalResourceMetadata.java
index 1667bbd..3de3a58 100644
--- a/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/resource/ExternalRTreeLocalResourceMetadata.java
+++ b/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/resource/ExternalRTreeLocalResourceMetadata.java
@@ -20,20 +20,21 @@
 
 import java.util.Map;
 
-import org.apache.asterix.common.context.BaseOperationTracker;
-import org.apache.asterix.common.dataflow.LSMIndexUtil;
-import org.apache.asterix.common.ioopcallbacks.LSMRTreeIOOperationCallbackFactory;
-import org.apache.asterix.common.transactions.IAppRuntimeContextProvider;
+import org.apache.asterix.common.api.IAppRuntimeContext;
+import org.apache.hyracks.api.application.INCApplicationContext;
 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.storage.am.common.api.IMetadataPageManagerFactory;
 import org.apache.hyracks.storage.am.common.api.IPrimitiveValueProviderFactory;
 import org.apache.hyracks.storage.am.common.api.TreeIndexException;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallbackFactory;
 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.rtree.utils.LSMRTreeUtils;
 import org.apache.hyracks.storage.am.rtree.frames.RTreePolicyType;
 import org.apache.hyracks.storage.common.file.LocalResource;
@@ -49,15 +50,18 @@
             IBinaryComparatorFactory[] btreeCmpFactories, IPrimitiveValueProviderFactory[] valueProviderFactories,
             RTreePolicyType rtreePolicyType, ILinearizeComparatorFactory linearizeCmpFactory, int datasetID,
             int partition, ILSMMergePolicyFactory mergePolicyFactory, Map<String, String> mergePolicyProperties,
-            int[] btreeFields, boolean isPointMBR) {
+            int[] btreeFields, boolean isPointMBR, ILSMOperationTrackerFactory opTrackerProvider,
+            ILSMIOOperationCallbackFactory ioOpCallbackFactory,
+            IMetadataPageManagerFactory metadataPageManagerFactory) {
         super(typeTraits, rtreeCmpFactories, btreeCmpFactories, valueProviderFactories, rtreePolicyType,
                 linearizeCmpFactory, datasetID, partition, mergePolicyFactory, mergePolicyProperties, null, null, null,
-                btreeFields, null, isPointMBR);
+                btreeFields, null, isPointMBR, opTrackerProvider, ioOpCallbackFactory, metadataPageManagerFactory);
     }
 
     @Override
-    public ILSMIndex createIndexInstance(IAppRuntimeContextProvider runtimeContextProvider,
+    public ILSMIndex createIndexInstance(INCApplicationContext appCtx,
             LocalResource resource) throws HyracksDataException {
+        IAppRuntimeContext runtimeContextProvider = (IAppRuntimeContext) appCtx.getApplicationObject();
         IIOManager ioManager = runtimeContextProvider.getIOManager();
         FileReference file = ioManager.resolve(resource.getPath());
         try {
@@ -66,11 +70,10 @@
                     valueProviderFactories, rtreePolicyType, runtimeContextProvider.getBloomFilterFalsePositiveRate(),
                     mergePolicyFactory.createMergePolicy(mergePolicyProperties,
                             runtimeContextProvider.getDatasetLifecycleManager()),
-                    new BaseOperationTracker(datasetId(),
-                            runtimeContextProvider.getDatasetLifecycleManager().getDatasetInfo(datasetId())),
+                    opTrackerProvider.getOperationTracker(appCtx),
                     runtimeContextProvider.getLSMIOScheduler(),
-                    LSMRTreeIOOperationCallbackFactory.INSTANCE.createIOOperationCallback(), linearizeCmpFactory,
-                    btreeFields, -1, true, isPointMBR, LSMIndexUtil.getMetadataPageManagerFactory());
+                    ioOpCallbackFactory.createIOOperationCallback(), linearizeCmpFactory,
+                    btreeFields, -1, true, isPointMBR, metadataPageManagerFactory);
         } catch (TreeIndexException e) {
             throw new HyracksDataException(e);
         }
diff --git a/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/resource/ExternalRTreeLocalResourceMetadataFactory.java b/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/resource/ExternalRTreeLocalResourceMetadataFactory.java
index 076fef1..e46729a 100644
--- a/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/resource/ExternalRTreeLocalResourceMetadataFactory.java
+++ b/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/resource/ExternalRTreeLocalResourceMetadataFactory.java
@@ -24,8 +24,11 @@
 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.storage.am.common.api.IMetadataPageManagerFactory;
 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.ILSMMergePolicyFactory;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMOperationTrackerFactory;
 import org.apache.hyracks.storage.am.rtree.frames.RTreePolicyType;
 
 public class ExternalRTreeLocalResourceMetadataFactory extends LSMRTreeLocalResourceMetadataFactory {
@@ -37,17 +40,20 @@
             IBinaryComparatorFactory[] btreeCmpFactories, IPrimitiveValueProviderFactory[] valueProviderFactories,
             RTreePolicyType rtreePolicyType, ILinearizeComparatorFactory linearizeCmpFactory, int datasetID,
             ILSMMergePolicyFactory mergePolicyFactory, Map<String, String> mergePolicyProperties,
-            int[] btreeFields, boolean isPointMBR) {
+            int[] btreeFields, boolean isPointMBR, ILSMOperationTrackerFactory opTrackerProvider,
+            ILSMIOOperationCallbackFactory ioOpCallbackFactory,
+            IMetadataPageManagerFactory metadataPageManagerFactory) {
         super(typeTraits, rtreeCmpFactories, btreeCmpFactories, valueProviderFactories, rtreePolicyType,
                 linearizeCmpFactory, datasetID, mergePolicyFactory, mergePolicyProperties, null, null, null,
-                btreeFields, null, isPointMBR);
+                btreeFields, null, isPointMBR, opTrackerProvider, ioOpCallbackFactory, metadataPageManagerFactory);
     }
 
     @Override
     public Resource resource(int partition) {
         return new ExternalRTreeLocalResourceMetadata(filterTypeTraits, rtreeCmpFactories, btreeCmpFactories,
                 valueProviderFactories, rtreePolicyType, linearizeCmpFactory, datasetId, partition, mergePolicyFactory,
-                mergePolicyProperties, btreeFields, isPointMBR);
+                mergePolicyProperties, btreeFields, isPointMBR, opTrackerProvider, ioOpCallbackFactory,
+                metadataPageManagerFactory);
     }
 
 }
diff --git a/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/resource/LSMBTreeLocalResourceMetadata.java b/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/resource/LSMBTreeLocalResourceMetadata.java
index 35263ce..0ecb396 100644
--- a/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/resource/LSMBTreeLocalResourceMetadata.java
+++ b/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/resource/LSMBTreeLocalResourceMetadata.java
@@ -20,21 +20,21 @@
 
 import java.util.Map;
 
+import org.apache.asterix.common.api.IAppRuntimeContext;
 import org.apache.asterix.common.api.IDatasetLifecycleManager;
-import org.apache.asterix.common.context.BaseOperationTracker;
-import org.apache.asterix.common.dataflow.LSMIndexUtil;
-import org.apache.asterix.common.ioopcallbacks.LSMBTreeIOOperationCallbackFactory;
-import org.apache.asterix.common.transactions.IAppRuntimeContextProvider;
 import org.apache.asterix.common.transactions.Resource;
+import org.apache.hyracks.api.application.INCApplicationContext;
 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.storage.am.lsm.btree.impls.LSMBTree;
-import org.apache.hyracks.storage.am.lsm.btree.util.LSMBTreeUtils;
+import org.apache.hyracks.storage.am.common.api.IMetadataPageManagerFactory;
+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.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.common.file.LocalResource;
 
 public class LSMBTreeLocalResourceMetadata extends Resource {
@@ -51,10 +51,13 @@
 
     public LSMBTreeLocalResourceMetadata(ITypeTraits[] typeTraits, IBinaryComparatorFactory[] cmpFactories,
             int[] bloomFilterKeyFields, boolean isPrimary, int datasetID, int partition,
-            ILSMMergePolicyFactory mergePolicyFactory,
-            Map<String, String> mergePolicyProperties, ITypeTraits[] filterTypeTraits,
-            IBinaryComparatorFactory[] filterCmpFactories, int[] btreeFields, int[] filterFields) {
-        super(datasetID, partition, filterTypeTraits, filterCmpFactories, filterFields);
+            ILSMMergePolicyFactory mergePolicyFactory, Map<String, String> mergePolicyProperties,
+            ITypeTraits[] filterTypeTraits, IBinaryComparatorFactory[] filterCmpFactories, int[] btreeFields,
+            int[] filterFields, ILSMOperationTrackerFactory opTrackerProvider,
+            ILSMIOOperationCallbackFactory ioOpCallbackFactory,
+            IMetadataPageManagerFactory metadataPageManagerFactory) {
+        super(datasetID, partition, filterTypeTraits, filterCmpFactories, filterFields, opTrackerProvider,
+                ioOpCallbackFactory, metadataPageManagerFactory);
         this.typeTraits = typeTraits;
         this.cmpFactories = cmpFactories;
         this.bloomFilterKeyFields = bloomFilterKeyFields;
@@ -72,24 +75,22 @@
     }
 
     @Override
-    public ILSMIndex createIndexInstance(IAppRuntimeContextProvider runtimeContextProvider,
-            LocalResource resource) throws HyracksDataException {
-        IIOManager ioManager = runtimeContextProvider.getIOManager();
+    public ILSMIndex createIndexInstance(INCApplicationContext appCtx, LocalResource resource)
+            throws HyracksDataException {
+        IAppRuntimeContext appRuntimeCtx = (IAppRuntimeContext) appCtx.getApplicationObject();
+        IIOManager ioManager = appRuntimeCtx.getIOManager();
         FileReference file = ioManager.resolve(resource.getPath());
         int ioDeviceNum = Resource.getIoDeviceNum(ioManager, file.getDeviceHandle());
-        final IDatasetLifecycleManager datasetLifecycleManager = runtimeContextProvider.getDatasetLifecycleManager();
-        LSMBTree lsmBTree = LSMBTreeUtils.createLSMTree(ioManager, datasetLifecycleManager.getVirtualBufferCaches(
-                datasetId(),
-                ioDeviceNum), file, runtimeContextProvider.getBufferCache(), runtimeContextProvider.getFileMapManager(),
-                typeTraits, cmpFactories, bloomFilterKeyFields, runtimeContextProvider
+        final IDatasetLifecycleManager datasetLifecycleManager = appRuntimeCtx.getDatasetLifecycleManager();
+        return LSMBTreeUtil.createLSMTree(ioManager, datasetLifecycleManager.getVirtualBufferCaches(
+                datasetId(), ioDeviceNum), file, appRuntimeCtx.getBufferCache(), appRuntimeCtx
+                        .getFileMapManager(),
+                typeTraits, cmpFactories, bloomFilterKeyFields, appRuntimeCtx
                         .getBloomFilterFalsePositiveRate(),
                 mergePolicyFactory.createMergePolicy(mergePolicyProperties, datasetLifecycleManager),
-                isPrimary ? runtimeContextProvider.getLSMBTreeOperationTracker(datasetId())
-                        : new BaseOperationTracker(datasetId(), datasetLifecycleManager.getDatasetInfo(datasetId())),
-                runtimeContextProvider.getLSMIOScheduler(),
-                LSMBTreeIOOperationCallbackFactory.INSTANCE.createIOOperationCallback(), isPrimary, filterTypeTraits,
-                filterCmpFactories, btreeFields, filterFields, true, LSMIndexUtil
-                        .getMetadataPageManagerFactory());
-        return lsmBTree;
+                opTrackerProvider.getOperationTracker(appCtx),
+                appRuntimeCtx.getLSMIOScheduler(),
+                ioOpCallbackFactory.createIOOperationCallback(), isPrimary, filterTypeTraits,
+                filterCmpFactories, btreeFields, filterFields, true, metadataPageManagerFactory);
     }
 }
diff --git a/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/resource/LSMBTreeLocalResourceMetadataFactory.java b/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/resource/LSMBTreeLocalResourceMetadataFactory.java
index 34d646d..3100d6c 100644
--- a/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/resource/LSMBTreeLocalResourceMetadataFactory.java
+++ b/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/resource/LSMBTreeLocalResourceMetadataFactory.java
@@ -24,7 +24,10 @@
 import org.apache.asterix.common.transactions.ResourceFactory;
 import org.apache.hyracks.api.dataflow.value.IBinaryComparatorFactory;
 import org.apache.hyracks.api.dataflow.value.ITypeTraits;
+import org.apache.hyracks.storage.am.common.api.IMetadataPageManagerFactory;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallbackFactory;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMMergePolicyFactory;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMOperationTrackerFactory;
 
 public class LSMBTreeLocalResourceMetadataFactory extends ResourceFactory {
 
@@ -41,8 +44,12 @@
             int[] bloomFilterKeyFields, boolean isPrimary, int datasetID,
             ILSMMergePolicyFactory mergePolicyFactory,
             Map<String, String> mergePolicyProperties, ITypeTraits[] filterTypeTraits,
-            IBinaryComparatorFactory[] filterCmpFactories, int[] btreeFields, int[] filterFields) {
-        super(datasetID, filterTypeTraits, filterCmpFactories, filterFields);
+            IBinaryComparatorFactory[] filterCmpFactories, int[] btreeFields, int[] filterFields,
+            ILSMOperationTrackerFactory opTrackerProvider,
+            ILSMIOOperationCallbackFactory ioOpCallbackFactory,
+            IMetadataPageManagerFactory metadataPageManagerFactory) {
+        super(datasetID, filterTypeTraits, filterCmpFactories, filterFields, opTrackerProvider, ioOpCallbackFactory,
+                metadataPageManagerFactory);
         this.typeTraits = typeTraits;
         this.cmpFactories = cmpFactories;
         this.bloomFilterKeyFields = bloomFilterKeyFields;
@@ -56,6 +63,6 @@
     public Resource resource(int partition) {
         return new LSMBTreeLocalResourceMetadata(typeTraits, cmpFactories, bloomFilterKeyFields, isPrimary, datasetId,
                 partition, mergePolicyFactory, mergePolicyProperties, typeTraits, filterCmpFactories, btreeFields,
-                filterFields);
+                filterFields, opTrackerProvider, ioOpCallbackFactory, metadataPageManagerFactory);
     }
 }
diff --git a/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/resource/LSMInvertedIndexLocalResourceMetadata.java b/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/resource/LSMInvertedIndexLocalResourceMetadata.java
index c80437f..097c922 100644
--- a/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/resource/LSMInvertedIndexLocalResourceMetadata.java
+++ b/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/resource/LSMInvertedIndexLocalResourceMetadata.java
@@ -21,19 +21,20 @@
 import java.util.List;
 import java.util.Map;
 
-import org.apache.asterix.common.context.BaseOperationTracker;
-import org.apache.asterix.common.dataflow.LSMIndexUtil;
-import org.apache.asterix.common.ioopcallbacks.LSMInvertedIndexIOOperationCallbackFactory;
-import org.apache.asterix.common.transactions.IAppRuntimeContextProvider;
+import org.apache.asterix.common.api.IAppRuntimeContext;
 import org.apache.asterix.common.transactions.Resource;
+import org.apache.hyracks.api.application.INCApplicationContext;
 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.storage.am.common.api.IMetadataPageManagerFactory;
 import org.apache.hyracks.storage.am.common.api.IndexException;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallbackFactory;
 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.IVirtualBufferCache;
 import org.apache.hyracks.storage.am.lsm.invertedindex.tokenizers.IBinaryTokenizerFactory;
 import org.apache.hyracks.storage.am.lsm.invertedindex.util.InvertedIndexUtils;
@@ -61,8 +62,11 @@
             boolean isPartitioned, int datasetID, int partition, ILSMMergePolicyFactory mergePolicyFactory,
             Map<String, String> mergePolicyProperties, ITypeTraits[] filterTypeTraits,
             IBinaryComparatorFactory[] filterCmpFactories, int[] invertedIndexFields, int[] filterFields,
-            int[] filterFieldsForNonBulkLoadOps, int[] invertedIndexFieldsForNonBulkLoadOps) {
-        super(datasetID, partition, filterTypeTraits, filterCmpFactories, filterFields);
+            int[] filterFieldsForNonBulkLoadOps, int[] invertedIndexFieldsForNonBulkLoadOps,
+            ILSMOperationTrackerFactory opTrackerProvider, ILSMIOOperationCallbackFactory ioOpCallbackFactory,
+            IMetadataPageManagerFactory metadataPageManagerFactory) {
+        super(datasetID, partition, filterTypeTraits, filterCmpFactories, filterFields, opTrackerProvider,
+                ioOpCallbackFactory, metadataPageManagerFactory);
         this.invListTypeTraits = invListTypeTraits;
         this.invListCmpFactories = invListCmpFactories;
         this.tokenTypeTraits = tokenTypeTraits;
@@ -77,13 +81,14 @@
     }
 
     @Override
-    public ILSMIndex createIndexInstance(IAppRuntimeContextProvider runtimeContextProvider,
-            LocalResource resource) throws HyracksDataException {
+    public ILSMIndex createIndexInstance(INCApplicationContext appCtx, LocalResource resource)
+            throws HyracksDataException {
+        IAppRuntimeContext runtimeContextProvider = (IAppRuntimeContext) appCtx.getApplicationObject();
         IIOManager ioManager = runtimeContextProvider.getIOManager();
         FileReference file = ioManager.resolve(resource.getPath());
         int ioDeviceNum = Resource.getIoDeviceNum(ioManager, file.getDeviceHandle());
-        List<IVirtualBufferCache> virtualBufferCaches = runtimeContextProvider.getDatasetLifecycleManager()
-                .getVirtualBufferCaches(datasetId(), ioDeviceNum);
+        List<IVirtualBufferCache> virtualBufferCaches =
+                runtimeContextProvider.getDatasetLifecycleManager().getVirtualBufferCaches(datasetId(), ioDeviceNum);
         try {
             if (isPartitioned) {
                 return InvertedIndexUtils.createPartitionedLSMInvertedIndex(ioManager, virtualBufferCaches,
@@ -92,13 +97,10 @@
                         file.getAbsolutePath(), runtimeContextProvider.getBloomFilterFalsePositiveRate(),
                         mergePolicyFactory.createMergePolicy(mergePolicyProperties,
                                 runtimeContextProvider.getDatasetLifecycleManager()),
-                        new BaseOperationTracker(datasetId(),
-                                runtimeContextProvider.getDatasetLifecycleManager().getDatasetInfo(datasetId())),
-                        runtimeContextProvider.getLSMIOScheduler(),
-                        LSMInvertedIndexIOOperationCallbackFactory.INSTANCE.createIOOperationCallback(),
-                        invertedIndexFields, filterTypeTraits, filterCmpFactories, filterFields,
-                        filterFieldsForNonBulkLoadOps, invertedIndexFieldsForNonBulkLoadOps, true, LSMIndexUtil
-                                .getMetadataPageManagerFactory());
+                        opTrackerProvider.getOperationTracker(appCtx), runtimeContextProvider.getLSMIOScheduler(),
+                        ioOpCallbackFactory.createIOOperationCallback(), invertedIndexFields, filterTypeTraits,
+                        filterCmpFactories, filterFields, filterFieldsForNonBulkLoadOps,
+                        invertedIndexFieldsForNonBulkLoadOps, true, metadataPageManagerFactory);
             } else {
                 return InvertedIndexUtils.createLSMInvertedIndex(ioManager, virtualBufferCaches,
                         runtimeContextProvider.getFileMapManager(), invListTypeTraits, invListCmpFactories,
@@ -106,13 +108,10 @@
                         file.getAbsolutePath(), runtimeContextProvider.getBloomFilterFalsePositiveRate(),
                         mergePolicyFactory.createMergePolicy(mergePolicyProperties,
                                 runtimeContextProvider.getDatasetLifecycleManager()),
-                        new BaseOperationTracker(datasetId(),
-                                runtimeContextProvider.getDatasetLifecycleManager().getDatasetInfo(datasetId())),
-                        runtimeContextProvider.getLSMIOScheduler(),
-                        LSMInvertedIndexIOOperationCallbackFactory.INSTANCE.createIOOperationCallback(),
-                        invertedIndexFields, filterTypeTraits, filterCmpFactories, filterFields,
-                        filterFieldsForNonBulkLoadOps, invertedIndexFieldsForNonBulkLoadOps, true, LSMIndexUtil
-                                .getMetadataPageManagerFactory());
+                        opTrackerProvider.getOperationTracker(appCtx), runtimeContextProvider.getLSMIOScheduler(),
+                        ioOpCallbackFactory.createIOOperationCallback(), invertedIndexFields, filterTypeTraits,
+                        filterCmpFactories, filterFields, filterFieldsForNonBulkLoadOps,
+                        invertedIndexFieldsForNonBulkLoadOps, true, metadataPageManagerFactory);
             }
         } catch (IndexException e) {
             throw new HyracksDataException(e);
diff --git a/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/resource/LSMInvertedIndexLocalResourceMetadataFactory.java b/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/resource/LSMInvertedIndexLocalResourceMetadataFactory.java
index a5a6c2a..2d55357 100644
--- a/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/resource/LSMInvertedIndexLocalResourceMetadataFactory.java
+++ b/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/resource/LSMInvertedIndexLocalResourceMetadataFactory.java
@@ -24,7 +24,10 @@
 import org.apache.asterix.common.transactions.ResourceFactory;
 import org.apache.hyracks.api.dataflow.value.IBinaryComparatorFactory;
 import org.apache.hyracks.api.dataflow.value.ITypeTraits;
+import org.apache.hyracks.storage.am.common.api.IMetadataPageManagerFactory;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallbackFactory;
 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.invertedindex.tokenizers.IBinaryTokenizerFactory;
 
 public class LSMInvertedIndexLocalResourceMetadataFactory extends ResourceFactory {
@@ -48,8 +51,12 @@
             boolean isPartitioned, int datasetID, ILSMMergePolicyFactory mergePolicyFactory,
             Map<String, String> mergePolicyProperties, ITypeTraits[] filterTypeTraits,
             IBinaryComparatorFactory[] filterCmpFactories, int[] invertedIndexFields, int[] filterFields,
-            int[] filterFieldsForNonBulkLoadOps, int[] invertedIndexFieldsForNonBulkLoadOps) {
-        super(datasetID, filterTypeTraits, filterCmpFactories, filterFields);
+            int[] filterFieldsForNonBulkLoadOps, int[] invertedIndexFieldsForNonBulkLoadOps,
+            ILSMOperationTrackerFactory opTrackerProvider,
+            ILSMIOOperationCallbackFactory ioOpCallbackFactory,
+            IMetadataPageManagerFactory metadataPageManagerFactory) {
+        super(datasetID, filterTypeTraits, filterCmpFactories, filterFields, opTrackerProvider, ioOpCallbackFactory,
+                metadataPageManagerFactory);
         this.invListTypeTraits = invListTypeTraits;
         this.invListCmpFactories = invListCmpFactories;
         this.tokenTypeTraits = tokenTypeTraits;
@@ -68,7 +75,8 @@
         return new LSMInvertedIndexLocalResourceMetadata(invListTypeTraits, invListCmpFactories, tokenTypeTraits,
                 tokenCmpFactories, tokenizerFactory, isPartitioned, datasetId, partition, mergePolicyFactory,
                 mergePolicyProperties, filterTypeTraits, filterCmpFactories, invertedIndexFields,
-                filterFields, filterFieldsForNonBulkLoadOps, invertedIndexFieldsForNonBulkLoadOps);
+                filterFields, filterFieldsForNonBulkLoadOps, invertedIndexFieldsForNonBulkLoadOps, opTrackerProvider,
+                ioOpCallbackFactory, metadataPageManagerFactory);
     }
 
 }
diff --git a/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/resource/LSMRTreeLocalResourceMetadata.java b/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/resource/LSMRTreeLocalResourceMetadata.java
index 2fc3022..c9c5d25 100644
--- a/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/resource/LSMRTreeLocalResourceMetadata.java
+++ b/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/resource/LSMRTreeLocalResourceMetadata.java
@@ -21,21 +21,22 @@
 import java.util.List;
 import java.util.Map;
 
-import org.apache.asterix.common.context.BaseOperationTracker;
-import org.apache.asterix.common.dataflow.LSMIndexUtil;
-import org.apache.asterix.common.ioopcallbacks.LSMRTreeIOOperationCallbackFactory;
-import org.apache.asterix.common.transactions.IAppRuntimeContextProvider;
+import org.apache.asterix.common.api.IAppRuntimeContext;
 import org.apache.asterix.common.transactions.Resource;
+import org.apache.hyracks.api.application.INCApplicationContext;
 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.storage.am.common.api.IMetadataPageManagerFactory;
 import org.apache.hyracks.storage.am.common.api.IPrimitiveValueProviderFactory;
 import org.apache.hyracks.storage.am.common.api.TreeIndexException;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallbackFactory;
 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.IVirtualBufferCache;
 import org.apache.hyracks.storage.am.lsm.rtree.utils.LSMRTreeUtils;
 import org.apache.hyracks.storage.am.rtree.frames.RTreePolicyType;
@@ -62,8 +63,11 @@
             RTreePolicyType rtreePolicyType, ILinearizeComparatorFactory linearizeCmpFactory, int datasetID,
             int partition, ILSMMergePolicyFactory mergePolicyFactory, Map<String, String> mergePolicyProperties,
             ITypeTraits[] filterTypeTraits, IBinaryComparatorFactory[] filterCmpFactories, int[] rtreeFields,
-            int[] btreeFields, int[] filterFields, boolean isPointMBR) {
-        super(datasetID, partition, filterTypeTraits, filterCmpFactories, filterFields);
+            int[] btreeFields, int[] filterFields, boolean isPointMBR, ILSMOperationTrackerFactory opTrackerProvider,
+            ILSMIOOperationCallbackFactory ioOpCallbackFactory,
+            IMetadataPageManagerFactory metadataPageManagerFactory) {
+        super(datasetID, partition, filterTypeTraits, filterCmpFactories, filterFields, opTrackerProvider,
+                ioOpCallbackFactory, metadataPageManagerFactory);
         this.typeTraits = typeTraits;
         this.rtreeCmpFactories = rtreeCmpFactories;
         this.btreeCmpFactories = btreeCmpFactories;
@@ -78,25 +82,23 @@
     }
 
     @Override
-    public ILSMIndex createIndexInstance(IAppRuntimeContextProvider runtimeContextProvider,
-            LocalResource resource) throws HyracksDataException {
+    public ILSMIndex createIndexInstance(INCApplicationContext appCtx, LocalResource resource)
+            throws HyracksDataException {
+        IAppRuntimeContext runtimeContextProvider = (IAppRuntimeContext) appCtx.getApplicationObject();
         IIOManager ioManager = runtimeContextProvider.getIOManager();
         FileReference file = ioManager.resolve(resource.getPath());
         int ioDeviceNum = Resource.getIoDeviceNum(ioManager, file.getDeviceHandle());
-        List<IVirtualBufferCache> virtualBufferCaches = runtimeContextProvider.getDatasetLifecycleManager()
-                .getVirtualBufferCaches(datasetId(), ioDeviceNum);
+        List<IVirtualBufferCache> virtualBufferCaches =
+                runtimeContextProvider.getDatasetLifecycleManager().getVirtualBufferCaches(datasetId(), ioDeviceNum);
         try {
             return LSMRTreeUtils.createLSMTreeWithAntiMatterTuples(ioManager, virtualBufferCaches, file,
                     runtimeContextProvider.getBufferCache(), runtimeContextProvider.getFileMapManager(), typeTraits,
                     rtreeCmpFactories, btreeCmpFactories, valueProviderFactories, rtreePolicyType,
                     mergePolicyFactory.createMergePolicy(mergePolicyProperties,
                             runtimeContextProvider.getDatasetLifecycleManager()),
-                    new BaseOperationTracker(datasetId(),
-                            runtimeContextProvider.getDatasetLifecycleManager().getDatasetInfo(datasetId())),
-                    runtimeContextProvider.getLSMIOScheduler(),
-                    LSMRTreeIOOperationCallbackFactory.INSTANCE.createIOOperationCallback(), linearizeCmpFactory,
-                    rtreeFields, filterTypeTraits, filterCmpFactories, filterFields, true, isPointMBR,
-                    LSMIndexUtil.getMetadataPageManagerFactory());
+                    opTrackerProvider.getOperationTracker(appCtx), runtimeContextProvider.getLSMIOScheduler(),
+                    ioOpCallbackFactory.createIOOperationCallback(), linearizeCmpFactory, rtreeFields,
+                    filterTypeTraits, filterCmpFactories, filterFields, true, isPointMBR, metadataPageManagerFactory);
         } catch (TreeIndexException e) {
             throw new HyracksDataException(e);
         }
diff --git a/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/resource/LSMRTreeLocalResourceMetadataFactory.java b/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/resource/LSMRTreeLocalResourceMetadataFactory.java
index 0ecf0aa..f05d28a 100644
--- a/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/resource/LSMRTreeLocalResourceMetadataFactory.java
+++ b/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/resource/LSMRTreeLocalResourceMetadataFactory.java
@@ -25,8 +25,11 @@
 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.storage.am.common.api.IMetadataPageManagerFactory;
 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.ILSMMergePolicyFactory;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMOperationTrackerFactory;
 import org.apache.hyracks.storage.am.rtree.frames.RTreePolicyType;
 
 public class LSMRTreeLocalResourceMetadataFactory extends ResourceFactory {
@@ -50,8 +53,11 @@
             RTreePolicyType rtreePolicyType, ILinearizeComparatorFactory linearizeCmpFactory, int datasetID,
             ILSMMergePolicyFactory mergePolicyFactory, Map<String, String> mergePolicyProperties,
             ITypeTraits[] filterTypeTraits, IBinaryComparatorFactory[] filterCmpFactories, int[] rtreeFields,
-            int[] btreeFields, int[] filterFields, boolean isPointMBR) {
-        super(datasetID, filterTypeTraits, filterCmpFactories, filterFields);
+            int[] btreeFields, int[] filterFields, boolean isPointMBR, ILSMOperationTrackerFactory opTrackerProvider,
+            ILSMIOOperationCallbackFactory ioOpCallbackFactory,
+            IMetadataPageManagerFactory metadataPageManagerFactory) {
+        super(datasetID, filterTypeTraits, filterCmpFactories, filterFields, opTrackerProvider, ioOpCallbackFactory,
+                metadataPageManagerFactory);
         this.typeTraits = typeTraits;
         this.rtreeCmpFactories = rtreeCmpFactories;
         this.btreeCmpFactories = btreeCmpFactories;
@@ -70,6 +76,6 @@
         return new LSMRTreeLocalResourceMetadata(typeTraits, rtreeCmpFactories, btreeCmpFactories,
                 valueProviderFactories, rtreePolicyType, linearizeCmpFactory, datasetId, partition, mergePolicyFactory,
                 mergePolicyProperties, filterTypeTraits, filterCmpFactories, rtreeFields, btreeFields, filterFields,
-                isPointMBR);
+                isPointMBR, opTrackerProvider, ioOpCallbackFactory, metadataPageManagerFactory);
     }
 }
diff --git a/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/resource/PersistentLocalResourceRepository.java b/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/resource/PersistentLocalResourceRepository.java
index 517292a..f7023b3 100644
--- a/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/resource/PersistentLocalResourceRepository.java
+++ b/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/resource/PersistentLocalResourceRepository.java
@@ -43,8 +43,8 @@
 import org.apache.asterix.common.cluster.ClusterPartition;
 import org.apache.asterix.common.config.MetadataProperties;
 import org.apache.asterix.common.exceptions.AsterixException;
-import org.apache.asterix.common.replication.ReplicationJob;
 import org.apache.asterix.common.replication.IReplicationManager;
+import org.apache.asterix.common.replication.ReplicationJob;
 import org.apache.asterix.common.utils.StorageConstants;
 import org.apache.asterix.common.utils.StoragePathUtil;
 import org.apache.commons.io.FileUtils;
@@ -72,8 +72,8 @@
     private static final String STORAGE_METADATA_FILE_NAME_PREFIX = "." + StorageConstants.METADATA_ROOT;
     private static final long STORAGE_LOCAL_RESOURCE_ID = -4321;
     private static final int MAX_CACHED_RESOURCES = 1000;
-    private static final FilenameFilter METADATA_FILES_FILTER = (File dir, String name) -> name
-            .equalsIgnoreCase(METADATA_FILE_NAME);
+    private static final FilenameFilter METADATA_FILES_FILTER =
+            (File dir, String name) -> name.equalsIgnoreCase(METADATA_FILE_NAME);
     // Finals
     private final IIOManager ioManager;
     private final String[] mountPoints;
@@ -164,8 +164,7 @@
             }
 
             LocalResource rootLocalResource = new LocalResource(STORAGE_LOCAL_RESOURCE_ID,
-                    storageMetadataFile.getRelativePath(), 0, ITreeIndexFrame.Constants.VERSION,
-                    storageRootDirPath);
+                    storageMetadataFile.getRelativePath(), 0, ITreeIndexFrame.Constants.VERSION, storageRootDirPath);
             insert(rootLocalResource);
             LOGGER.log(Level.INFO, "created the root-metadata-file: " + storageMetadataFile.getAbsolutePath());
         }
diff --git a/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/logging/LogBuffer.java b/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/logging/LogBuffer.java
index 1245674..424e800 100644
--- a/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/logging/LogBuffer.java
+++ b/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/logging/LogBuffer.java
@@ -32,20 +32,21 @@
 import org.apache.asterix.common.transactions.ILogBuffer;
 import org.apache.asterix.common.transactions.ILogRecord;
 import org.apache.asterix.common.transactions.ITransactionContext;
+import org.apache.asterix.common.transactions.ITransactionSubsystem;
 import org.apache.asterix.common.transactions.JobId;
 import org.apache.asterix.common.transactions.LogRecord;
 import org.apache.asterix.common.transactions.LogSource;
 import org.apache.asterix.common.transactions.LogType;
 import org.apache.asterix.common.transactions.MutableLong;
+import org.apache.asterix.common.utils.TransactionUtil;
 import org.apache.asterix.transaction.management.service.transaction.TransactionManagementConstants.LockManagerConstants.LockMode;
-import org.apache.asterix.transaction.management.service.transaction.TransactionSubsystem;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 
 public class LogBuffer implements ILogBuffer {
 
     public static final boolean IS_DEBUG_MODE = false;//true
     private static final Logger LOGGER = Logger.getLogger(LogBuffer.class.getName());
-    private final TransactionSubsystem txnSubsystem;
+    private final ITransactionSubsystem txnSubsystem;
     private final LogBufferTailReader logBufferTailReader;
     private final int logPageSize;
     private final MutableLong flushLSN;
@@ -64,7 +65,7 @@
     private final DatasetId reusableDsId;
     private final JobId reusableJobId;
 
-    public LogBuffer(TransactionSubsystem txnSubsystem, int logPageSize, MutableLong flushLSN) {
+    public LogBuffer(ITransactionSubsystem txnSubsystem, int logPageSize, MutableLong flushLSN) {
         this.txnSubsystem = txnSubsystem;
         this.logPageSize = logPageSize;
         this.flushLSN = flushLSN;
@@ -266,7 +267,7 @@
                             // since this operation consisted of delete and insert, we need to notify the optracker twice
                             txnCtx.notifyOptracker(false);
                         }
-                        if (TransactionSubsystem.IS_PROFILE_MODE) {
+                        if (TransactionUtil.PROFILE_MODE) {
                             txnSubsystem.incrementEntityCommitCount();
                         }
                     } else if (logRecord.getLogType() == LogType.JOB_COMMIT
diff --git a/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/logging/LogManager.java b/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/logging/LogManager.java
index 25096f6..613808f 100644
--- a/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/logging/LogManager.java
+++ b/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/logging/LogManager.java
@@ -45,18 +45,18 @@
 import org.apache.asterix.common.transactions.ILogRecord;
 import org.apache.asterix.common.transactions.ITransactionContext;
 import org.apache.asterix.common.transactions.ITransactionManager;
+import org.apache.asterix.common.transactions.ITransactionSubsystem;
 import org.apache.asterix.common.transactions.LogManagerProperties;
 import org.apache.asterix.common.transactions.LogType;
 import org.apache.asterix.common.transactions.MutableLong;
 import org.apache.asterix.common.transactions.TxnLogFile;
-import org.apache.asterix.transaction.management.service.transaction.TransactionSubsystem;
 import org.apache.hyracks.api.lifecycle.ILifeCycleComponent;
 
 public class LogManager implements ILogManager, ILifeCycleComponent {
 
     public static final boolean IS_DEBUG_MODE = false;// true
     private static final Logger LOGGER = Logger.getLogger(LogManager.class.getName());
-    private final TransactionSubsystem txnSubsystem;
+    private final ITransactionSubsystem txnSubsystem;
 
     private final LogManagerProperties logManagerProperties;
     protected final long logFileSize;
@@ -80,7 +80,7 @@
     private final FlushLogsLogger flushLogsLogger;
     private final HashMap<Long, Integer> txnLogFileId2ReaderCount = new HashMap<>();
 
-    public LogManager(TransactionSubsystem txnSubsystem) {
+    public LogManager(ITransactionSubsystem txnSubsystem) {
         this.txnSubsystem = txnSubsystem;
         logManagerProperties = new LogManagerProperties(this.txnSubsystem.getTransactionProperties(),
                 this.txnSubsystem.getId());
@@ -251,7 +251,7 @@
         return logManagerProperties;
     }
 
-    public TransactionSubsystem getTransactionSubsystem() {
+    public ITransactionSubsystem getTransactionSubsystem() {
         return txnSubsystem;
     }
 
diff --git a/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/logging/LogManagerWithReplication.java b/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/logging/LogManagerWithReplication.java
index c81225f..a248f77 100644
--- a/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/logging/LogManagerWithReplication.java
+++ b/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/logging/LogManagerWithReplication.java
@@ -23,15 +23,15 @@
 import org.apache.asterix.common.transactions.ILogRecord;
 import org.apache.asterix.common.transactions.ITransactionContext;
 import org.apache.asterix.common.transactions.ITransactionManager;
+import org.apache.asterix.common.transactions.ITransactionSubsystem;
 import org.apache.asterix.common.transactions.LogSource;
 import org.apache.asterix.common.transactions.LogType;
-import org.apache.asterix.transaction.management.service.transaction.TransactionSubsystem;
 
 public class LogManagerWithReplication extends LogManager {
 
     private IReplicationManager replicationManager;
 
-    public LogManagerWithReplication(TransactionSubsystem txnSubsystem) {
+    public LogManagerWithReplication(ITransactionSubsystem txnSubsystem) {
         super(txnSubsystem);
     }
 
diff --git a/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/transaction/AsterixRuntimeComponentsProvider.java b/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/transaction/AsterixRuntimeComponentsProvider.java
deleted file mode 100644
index 8c6e253..0000000
--- a/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/transaction/AsterixRuntimeComponentsProvider.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.transaction.management.service.transaction;
-
-import org.apache.asterix.common.api.IAppRuntimeContext;
-import org.apache.asterix.common.api.IDatasetLifecycleManager;
-import org.apache.hyracks.api.context.IHyracksTaskContext;
-import org.apache.hyracks.storage.am.common.api.IIndexLifecycleManagerProvider;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationScheduler;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationSchedulerProvider;
-import org.apache.hyracks.storage.common.IStorageManagerInterface;
-import org.apache.hyracks.storage.common.buffercache.IBufferCache;
-import org.apache.hyracks.storage.common.file.IFileMapProvider;
-import org.apache.hyracks.storage.common.file.ILocalResourceRepository;
-import org.apache.hyracks.storage.common.file.IResourceIdFactory;
-
-public class AsterixRuntimeComponentsProvider implements IIndexLifecycleManagerProvider, IStorageManagerInterface,
-        ILSMIOOperationSchedulerProvider {
-
-    private static final long serialVersionUID = 1L;
-
-    public static final AsterixRuntimeComponentsProvider RUNTIME_PROVIDER = new AsterixRuntimeComponentsProvider();
-
-    private AsterixRuntimeComponentsProvider() {
-    }
-
-    @Override
-    public ILSMIOOperationScheduler getIOScheduler(IHyracksTaskContext ctx) {
-        return ((IAppRuntimeContext) ctx.getJobletContext().getApplicationContext().getApplicationObject())
-                .getLSMIOScheduler();
-    }
-
-    @Override
-    public IBufferCache getBufferCache(IHyracksTaskContext ctx) {
-        return ((IAppRuntimeContext) ctx.getJobletContext().getApplicationContext().getApplicationObject())
-                .getBufferCache();
-    }
-
-    @Override
-    public IFileMapProvider getFileMapProvider(IHyracksTaskContext ctx) {
-        return ((IAppRuntimeContext) ctx.getJobletContext().getApplicationContext().getApplicationObject())
-                .getFileMapManager();
-    }
-
-    @Override
-    public ILocalResourceRepository getLocalResourceRepository(IHyracksTaskContext ctx) {
-        return ((IAppRuntimeContext) ctx.getJobletContext().getApplicationContext().getApplicationObject())
-                .getLocalResourceRepository();
-    }
-
-    @Override
-    public IDatasetLifecycleManager getLifecycleManager(IHyracksTaskContext ctx) {
-        return ((IAppRuntimeContext) ctx.getJobletContext().getApplicationContext().getApplicationObject())
-                .getDatasetLifecycleManager();
-    }
-
-    @Override
-    public IResourceIdFactory getResourceIdFactory(IHyracksTaskContext ctx) {
-        return ((IAppRuntimeContext) ctx.getJobletContext().getApplicationContext().getApplicationObject())
-                .getResourceIdFactory();
-    }
-
-}
diff --git a/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/transaction/TransactionContext.java b/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/transaction/TransactionContext.java
index 215eb14..ad03a25 100644
--- a/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/transaction/TransactionContext.java
+++ b/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/transaction/TransactionContext.java
@@ -98,7 +98,7 @@
     // creations.
     // also, the pool can throttle the number of concurrent active jobs at every
     // moment.
-    public TransactionContext(JobId jobId, TransactionSubsystem transactionSubsystem) throws ACIDException {
+    public TransactionContext(JobId jobId) throws ACIDException {
         this.jobId = jobId;
         firstLSN = new AtomicLong(-1);
         lastLSN = new AtomicLong(-1);
@@ -106,7 +106,7 @@
         isTimeout = false;
         isWriteTxn = new AtomicBoolean(false);
         isMetadataTxn = false;
-        indexMap = new HashMap<MutableLong, AbstractLSMIOOperationCallback>();
+        indexMap = new HashMap<>();
         primaryIndex = null;
         tempResourceIdForRegister = new MutableLong();
         logRecord = new LogRecord();
diff --git a/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/transaction/TransactionManager.java b/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/transaction/TransactionManager.java
index b08ecbb..3e79e1d 100644
--- a/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/transaction/TransactionManager.java
+++ b/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/transaction/TransactionManager.java
@@ -30,6 +30,7 @@
 import org.apache.asterix.common.transactions.DatasetId;
 import org.apache.asterix.common.transactions.ITransactionContext;
 import org.apache.asterix.common.transactions.ITransactionManager;
+import org.apache.asterix.common.transactions.ITransactionSubsystem;
 import org.apache.asterix.common.transactions.JobId;
 import org.apache.asterix.common.transactions.LogRecord;
 import org.apache.asterix.common.utils.TransactionUtil;
@@ -43,11 +44,11 @@
 
     public static final boolean IS_DEBUG_MODE = false;//true
     private static final Logger LOGGER = Logger.getLogger(TransactionManager.class.getName());
-    private final TransactionSubsystem txnSubsystem;
-    private Map<JobId, ITransactionContext> transactionContextRepository = new ConcurrentHashMap<JobId, ITransactionContext>();
+    private final ITransactionSubsystem txnSubsystem;
+    private Map<JobId, ITransactionContext> transactionContextRepository = new ConcurrentHashMap<>();
     private AtomicInteger maxJobId = new AtomicInteger(0);
 
-    public TransactionManager(TransactionSubsystem provider) {
+    public TransactionManager(ITransactionSubsystem provider) {
         this.txnSubsystem = provider;
     }
 
@@ -91,7 +92,7 @@
                 synchronized (this) {
                     txnCtx = transactionContextRepository.get(jobId);
                     if (txnCtx == null) {
-                        txnCtx = new TransactionContext(jobId, txnSubsystem);
+                        txnCtx = new TransactionContext(jobId);
                         transactionContextRepository.put(jobId, txnCtx);
                     }
                 }
@@ -103,7 +104,8 @@
     }
 
     @Override
-    public void commitTransaction(ITransactionContext txnCtx, DatasetId datasetId, int PKHashVal) throws ACIDException {
+    public void commitTransaction(ITransactionContext txnCtx, DatasetId datasetId, int PKHashVal)
+            throws ACIDException {
         //Only job-level commits call this method.
         try {
             if (txnCtx.isWriteTxn()) {
@@ -134,7 +136,7 @@
     }
 
     @Override
-    public TransactionSubsystem getTransactionProvider() {
+    public ITransactionSubsystem getTransactionSubsystem() {
         return txnSubsystem;
     }
 
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/expressions/LogicalExpressionJobGenToExpressionRuntimeProviderAdapter.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/expressions/ExpressionRuntimeProvider.java
similarity index 93%
rename from hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/expressions/LogicalExpressionJobGenToExpressionRuntimeProviderAdapter.java
rename to hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/expressions/ExpressionRuntimeProvider.java
index 5b07a7e..31726d2 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/expressions/LogicalExpressionJobGenToExpressionRuntimeProviderAdapter.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/expressions/ExpressionRuntimeProvider.java
@@ -28,10 +28,10 @@
 import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
 import org.apache.hyracks.algebricks.runtime.base.IUnnestingEvaluatorFactory;
 
-public class LogicalExpressionJobGenToExpressionRuntimeProviderAdapter implements IExpressionRuntimeProvider {
+public class ExpressionRuntimeProvider implements IExpressionRuntimeProvider {
     private final ILogicalExpressionJobGen lejg;
 
-    public LogicalExpressionJobGenToExpressionRuntimeProviderAdapter(ILogicalExpressionJobGen lejg) {
+    public ExpressionRuntimeProvider(ILogicalExpressionJobGen lejg) {
         this.lejg = lejg;
     }
 
diff --git a/hyracks-fullstack/algebricks/algebricks-data/src/main/java/org/apache/hyracks/algebricks/data/IBinaryComparatorFactoryProvider.java b/hyracks-fullstack/algebricks/algebricks-data/src/main/java/org/apache/hyracks/algebricks/data/IBinaryComparatorFactoryProvider.java
index bdeb018..0acb40a 100644
--- a/hyracks-fullstack/algebricks/algebricks-data/src/main/java/org/apache/hyracks/algebricks/data/IBinaryComparatorFactoryProvider.java
+++ b/hyracks-fullstack/algebricks/algebricks-data/src/main/java/org/apache/hyracks/algebricks/data/IBinaryComparatorFactoryProvider.java
@@ -21,7 +21,19 @@
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
 import org.apache.hyracks.api.dataflow.value.IBinaryComparatorFactory;
 
+/**
+ * Provides {@link org.apache.hyracks.api.dataflow.value.IBinaryComparatorFactory} for different types
+ */
+@FunctionalInterface
 public interface IBinaryComparatorFactoryProvider {
-    public IBinaryComparatorFactory getBinaryComparatorFactory(Object type, boolean ascending)
-            throws AlgebricksException;
+    /**
+     * @param type
+     *            the type of the binary data
+     * @param ascending
+     *            the order direction. true if ascending order is desired, false otherwise
+     * @return the appropriate {@link org.apache.hyracks.api.dataflow.value.IBinaryComparatorFactory} instance
+     * @throws AlgebricksException
+     *             if the comparator factory for the passed type could not be created
+     */
+    IBinaryComparatorFactory getBinaryComparatorFactory(Object type, boolean ascending) throws AlgebricksException;
 }
diff --git a/hyracks-fullstack/algebricks/algebricks-data/src/main/java/org/apache/hyracks/algebricks/data/ITypeTraitProvider.java b/hyracks-fullstack/algebricks/algebricks-data/src/main/java/org/apache/hyracks/algebricks/data/ITypeTraitProvider.java
index 3bac25d..1e502b3 100644
--- a/hyracks-fullstack/algebricks/algebricks-data/src/main/java/org/apache/hyracks/algebricks/data/ITypeTraitProvider.java
+++ b/hyracks-fullstack/algebricks/algebricks-data/src/main/java/org/apache/hyracks/algebricks/data/ITypeTraitProvider.java
@@ -20,6 +20,10 @@
 
 import org.apache.hyracks.api.dataflow.value.ITypeTraits;
 
+/**
+ * Provides {@link org.apache.hyracks.api.dataflow.value.ITypeTraits} for different data types
+ */
+@FunctionalInterface
 public interface ITypeTraitProvider {
-    public ITypeTraits getTypeTrait(Object type);
+    ITypeTraits getTypeTrait(Object type);
 }
diff --git a/hyracks-fullstack/algebricks/algebricks-examples/piglet-example/src/main/java/org/apache/hyracks/algebricks/examples/piglet/compiler/PigletCompiler.java b/hyracks-fullstack/algebricks/algebricks-examples/piglet-example/src/main/java/org/apache/hyracks/algebricks/examples/piglet/compiler/PigletCompiler.java
index 0ac08c3..c8b6e59 100644
--- a/hyracks-fullstack/algebricks/algebricks-examples/piglet-example/src/main/java/org/apache/hyracks/algebricks/examples/piglet/compiler/PigletCompiler.java
+++ b/hyracks-fullstack/algebricks/algebricks-examples/piglet-example/src/main/java/org/apache/hyracks/algebricks/examples/piglet/compiler/PigletCompiler.java
@@ -44,7 +44,7 @@
 import org.apache.hyracks.algebricks.core.algebra.expressions.ConstantExpression;
 import org.apache.hyracks.algebricks.core.algebra.expressions.IExpressionTypeComputer;
 import org.apache.hyracks.algebricks.core.algebra.expressions.IVariableTypeEnvironment;
-import org.apache.hyracks.algebricks.core.algebra.expressions.LogicalExpressionJobGenToExpressionRuntimeProviderAdapter;
+import org.apache.hyracks.algebricks.core.algebra.expressions.ExpressionRuntimeProvider;
 import org.apache.hyracks.algebricks.core.algebra.expressions.ScalarFunctionCallExpression;
 import org.apache.hyracks.algebricks.core.algebra.expressions.VariableReferenceExpression;
 import org.apache.hyracks.algebricks.core.algebra.functions.AlgebricksBuiltinFunctions;
@@ -153,7 +153,7 @@
         });
         builder.setPrinterProvider(PigletPrinterFactoryProvider.INSTANCE);
         builder.setExpressionRuntimeProvider(
-                new LogicalExpressionJobGenToExpressionRuntimeProviderAdapter(new PigletExpressionJobGen()));
+                new ExpressionRuntimeProvider(new PigletExpressionJobGen()));
         builder.setExpressionTypeComputer(new IExpressionTypeComputer() {
             @Override
             public Object getType(ILogicalExpression expr, IMetadataProvider<?, ?> metadataProvider,
diff --git a/hyracks-fullstack/hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/io/MessagingFrameTupleAppender.java b/hyracks-fullstack/hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/io/MessagingFrameTupleAppender.java
index 9463982..8f005d8 100644
--- a/hyracks-fullstack/hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/io/MessagingFrameTupleAppender.java
+++ b/hyracks-fullstack/hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/io/MessagingFrameTupleAppender.java
@@ -28,7 +28,7 @@
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.api.util.HyracksConstants;
 import org.apache.hyracks.dataflow.common.comm.io.FrameTupleAppender;
-import org.apache.hyracks.dataflow.common.util.TaskUtils;
+import org.apache.hyracks.dataflow.common.utils.TaskUtil;
 import org.apache.hyracks.util.IntSerDeUtils;
 
 /**
@@ -112,7 +112,7 @@
     @Override
     public void write(IFrameWriter outWriter, boolean clearFrame) throws HyracksDataException {
         if (!initialized) {
-            message = TaskUtils.<VSizeFrame> get(HyracksConstants.KEY_MESSAGE, ctx);
+            message = TaskUtil.<VSizeFrame> get(HyracksConstants.KEY_MESSAGE, ctx);
             initialized = true;
         }
         // If message fits, we append it, otherwise, we append a null message, then send a message only
diff --git a/hyracks-fullstack/hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/util/FrameDebugUtils.java b/hyracks-fullstack/hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/utils/FrameDebugUtils.java
similarity index 99%
rename from hyracks-fullstack/hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/util/FrameDebugUtils.java
rename to hyracks-fullstack/hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/utils/FrameDebugUtils.java
index aa27c42..66e7ae0 100644
--- a/hyracks-fullstack/hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/util/FrameDebugUtils.java
+++ b/hyracks-fullstack/hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/utils/FrameDebugUtils.java
@@ -16,7 +16,7 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.hyracks.dataflow.common.util;
+package org.apache.hyracks.dataflow.common.utils;
 
 import java.io.DataInputStream;
 import java.io.IOException;
diff --git a/hyracks-fullstack/hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/util/SerdeUtils.java b/hyracks-fullstack/hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/utils/SerdeUtils.java
similarity index 98%
rename from hyracks-fullstack/hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/util/SerdeUtils.java
rename to hyracks-fullstack/hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/utils/SerdeUtils.java
index 99c438e..81f06da 100644
--- a/hyracks-fullstack/hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/util/SerdeUtils.java
+++ b/hyracks-fullstack/hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/utils/SerdeUtils.java
@@ -17,7 +17,7 @@
  * under the License.
  */
 
-package org.apache.hyracks.dataflow.common.util;
+package org.apache.hyracks.dataflow.common.utils;
 
 import org.apache.hyracks.api.dataflow.value.IBinaryComparator;
 import org.apache.hyracks.api.dataflow.value.IBinaryComparatorFactory;
diff --git a/hyracks-fullstack/hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/util/TaskUtils.java b/hyracks-fullstack/hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/utils/TaskUtil.java
similarity index 89%
rename from hyracks-fullstack/hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/util/TaskUtils.java
rename to hyracks-fullstack/hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/utils/TaskUtil.java
index 4f27d79..4a2021f 100644
--- a/hyracks-fullstack/hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/util/TaskUtils.java
+++ b/hyracks-fullstack/hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/utils/TaskUtil.java
@@ -16,7 +16,7 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.hyracks.dataflow.common.util;
+package org.apache.hyracks.dataflow.common.utils;
 
 import java.util.HashMap;
 import java.util.Map;
@@ -26,8 +26,8 @@
 /**
  * A Utility class for facilitating common operations used with a hyracks task
  */
-public class TaskUtils {
-    private TaskUtils() {
+public class TaskUtil {
+    private TaskUtil() {
     }
 
     /**
@@ -59,7 +59,7 @@
      * @param object
      */
     public static void putInSharedMap(String key, Object object, IHyracksTaskContext ctx) {
-        TaskUtils.getSharedMap(ctx, true).put(key, object);
+        TaskUtil.getSharedMap(ctx, true).put(key, object);
     }
 
     /**
@@ -71,7 +71,7 @@
      */
     @SuppressWarnings("unchecked")
     public static <T> T get(String key, IHyracksTaskContext ctx) {
-        Map<String, Object> sharedMap = TaskUtils.getSharedMap(ctx, false);
+        Map<String, Object> sharedMap = TaskUtil.getSharedMap(ctx, false);
         return sharedMap == null ? null : (T) sharedMap.get(key);
     }
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/util/TupleUtils.java b/hyracks-fullstack/hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/utils/TupleUtils.java
similarity index 98%
rename from hyracks-fullstack/hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/util/TupleUtils.java
rename to hyracks-fullstack/hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/utils/TupleUtils.java
index fcc6274..500cee7 100644
--- a/hyracks-fullstack/hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/util/TupleUtils.java
+++ b/hyracks-fullstack/hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/utils/TupleUtils.java
@@ -17,7 +17,7 @@
  * under the License.
  */
 
-package org.apache.hyracks.dataflow.common.util;
+package org.apache.hyracks.dataflow.common.utils;
 
 import java.io.ByteArrayInputStream;
 import java.io.DataInput;
diff --git a/hyracks-fullstack/hyracks/hyracks-examples/btree-example/btreeclient/src/main/java/org/apache/hyracks/examples/btree/client/InsertPipelineExample.java b/hyracks-fullstack/hyracks/hyracks-examples/btree-example/btreeclient/src/main/java/org/apache/hyracks/examples/btree/client/InsertPipelineExample.java
index 91aa263..0772d83 100644
--- a/hyracks-fullstack/hyracks/hyracks-examples/btree-example/btreeclient/src/main/java/org/apache/hyracks/examples/btree/client/InsertPipelineExample.java
+++ b/hyracks-fullstack/hyracks/hyracks-examples/btree-example/btreeclient/src/main/java/org/apache/hyracks/examples/btree/client/InsertPipelineExample.java
@@ -43,7 +43,7 @@
 import org.apache.hyracks.dataflow.std.misc.NullSinkOperatorDescriptor;
 import org.apache.hyracks.examples.btree.helper.DataGenOperatorDescriptor;
 import org.apache.hyracks.examples.btree.helper.IndexLifecycleManagerProvider;
-import org.apache.hyracks.examples.btree.helper.StorageManagerInterface;
+import org.apache.hyracks.examples.btree.helper.BTreeHelperStorageManager;
 import org.apache.hyracks.storage.am.btree.dataflow.BTreeDataflowHelperFactory;
 import org.apache.hyracks.storage.am.common.api.IIndexLifecycleManagerProvider;
 import org.apache.hyracks.storage.am.common.dataflow.IIndexDataflowHelperFactory;
@@ -51,7 +51,7 @@
 import org.apache.hyracks.storage.am.common.freepage.LinkedMetadataPageManagerFactory;
 import org.apache.hyracks.storage.am.common.impls.NoOpOperationCallbackFactory;
 import org.apache.hyracks.storage.am.common.ophelpers.IndexOperation;
-import org.apache.hyracks.storage.common.IStorageManagerInterface;
+import org.apache.hyracks.storage.common.IStorageManager;
 import org.kohsuke.args4j.CmdLineParser;
 import org.kohsuke.args4j.Option;
 
@@ -128,7 +128,7 @@
         PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, dataGen, splitNCs[0]);
 
         IIndexLifecycleManagerProvider lcManagerProvider = IndexLifecycleManagerProvider.INSTANCE;
-        IStorageManagerInterface storageManager = StorageManagerInterface.INSTANCE;
+        IStorageManager storageManager = BTreeHelperStorageManager.INSTANCE;
 
         // prepare insertion into primary index
         // tuples to be put into B-Tree shall have 4 fields
diff --git a/hyracks-fullstack/hyracks/hyracks-examples/btree-example/btreeclient/src/main/java/org/apache/hyracks/examples/btree/client/PrimaryIndexBulkLoadExample.java b/hyracks-fullstack/hyracks/hyracks-examples/btree-example/btreeclient/src/main/java/org/apache/hyracks/examples/btree/client/PrimaryIndexBulkLoadExample.java
index b8216e3..550af4c 100644
--- a/hyracks-fullstack/hyracks/hyracks-examples/btree-example/btreeclient/src/main/java/org/apache/hyracks/examples/btree/client/PrimaryIndexBulkLoadExample.java
+++ b/hyracks-fullstack/hyracks/hyracks-examples/btree-example/btreeclient/src/main/java/org/apache/hyracks/examples/btree/client/PrimaryIndexBulkLoadExample.java
@@ -43,13 +43,13 @@
 import org.apache.hyracks.dataflow.std.sort.ExternalSortOperatorDescriptor;
 import org.apache.hyracks.examples.btree.helper.DataGenOperatorDescriptor;
 import org.apache.hyracks.examples.btree.helper.IndexLifecycleManagerProvider;
-import org.apache.hyracks.examples.btree.helper.StorageManagerInterface;
+import org.apache.hyracks.examples.btree.helper.BTreeHelperStorageManager;
 import org.apache.hyracks.storage.am.btree.dataflow.BTreeDataflowHelperFactory;
 import org.apache.hyracks.storage.am.common.api.IIndexLifecycleManagerProvider;
 import org.apache.hyracks.storage.am.common.dataflow.IIndexDataflowHelperFactory;
 import org.apache.hyracks.storage.am.common.dataflow.TreeIndexBulkLoadOperatorDescriptor;
 import org.apache.hyracks.storage.am.common.freepage.LinkedMetadataPageManagerFactory;
-import org.apache.hyracks.storage.common.IStorageManagerInterface;
+import org.apache.hyracks.storage.common.IStorageManager;
 import org.kohsuke.args4j.CmdLineParser;
 import org.kohsuke.args4j.Option;
 
@@ -145,7 +145,7 @@
 
         // create providers for B-Tree
         IIndexLifecycleManagerProvider lcManagerProvider = IndexLifecycleManagerProvider.INSTANCE;
-        IStorageManagerInterface storageManager = StorageManagerInterface.INSTANCE;
+        IStorageManager storageManager = BTreeHelperStorageManager.INSTANCE;
 
         // the B-Tree expects its keyfields to be at the front of its input
         // tuple
diff --git a/hyracks-fullstack/hyracks/hyracks-examples/btree-example/btreeclient/src/main/java/org/apache/hyracks/examples/btree/client/PrimaryIndexSearchExample.java b/hyracks-fullstack/hyracks/hyracks-examples/btree-example/btreeclient/src/main/java/org/apache/hyracks/examples/btree/client/PrimaryIndexSearchExample.java
index 2d1d515..f15648e 100644
--- a/hyracks-fullstack/hyracks/hyracks-examples/btree-example/btreeclient/src/main/java/org/apache/hyracks/examples/btree/client/PrimaryIndexSearchExample.java
+++ b/hyracks-fullstack/hyracks/hyracks-examples/btree-example/btreeclient/src/main/java/org/apache/hyracks/examples/btree/client/PrimaryIndexSearchExample.java
@@ -40,14 +40,14 @@
 import org.apache.hyracks.dataflow.std.misc.ConstantTupleSourceOperatorDescriptor;
 import org.apache.hyracks.dataflow.std.misc.PrinterOperatorDescriptor;
 import org.apache.hyracks.examples.btree.helper.IndexLifecycleManagerProvider;
-import org.apache.hyracks.examples.btree.helper.StorageManagerInterface;
+import org.apache.hyracks.examples.btree.helper.BTreeHelperStorageManager;
 import org.apache.hyracks.storage.am.btree.dataflow.BTreeDataflowHelperFactory;
 import org.apache.hyracks.storage.am.btree.dataflow.BTreeSearchOperatorDescriptor;
 import org.apache.hyracks.storage.am.common.api.IIndexLifecycleManagerProvider;
 import org.apache.hyracks.storage.am.common.dataflow.IIndexDataflowHelperFactory;
 import org.apache.hyracks.storage.am.common.freepage.LinkedMetadataPageManagerFactory;
 import org.apache.hyracks.storage.am.common.impls.NoOpOperationCallbackFactory;
-import org.apache.hyracks.storage.common.IStorageManagerInterface;
+import org.apache.hyracks.storage.common.IStorageManager;
 import org.kohsuke.args4j.CmdLineParser;
 import org.kohsuke.args4j.Option;
 
@@ -107,7 +107,7 @@
 
         // create roviders for B-Tree
         IIndexLifecycleManagerProvider lcManagerProvider = IndexLifecycleManagerProvider.INSTANCE;
-        IStorageManagerInterface storageManager = StorageManagerInterface.INSTANCE;
+        IStorageManager storageManager = BTreeHelperStorageManager.INSTANCE;
 
         // schema of tuples coming out of primary index
         RecordDescriptor recDesc = new RecordDescriptor(new ISerializerDeserializer[] {
diff --git a/hyracks-fullstack/hyracks/hyracks-examples/btree-example/btreeclient/src/main/java/org/apache/hyracks/examples/btree/client/SecondaryIndexBulkLoadExample.java b/hyracks-fullstack/hyracks/hyracks-examples/btree-example/btreeclient/src/main/java/org/apache/hyracks/examples/btree/client/SecondaryIndexBulkLoadExample.java
index c1ccc10..003e353 100644
--- a/hyracks-fullstack/hyracks/hyracks-examples/btree-example/btreeclient/src/main/java/org/apache/hyracks/examples/btree/client/SecondaryIndexBulkLoadExample.java
+++ b/hyracks-fullstack/hyracks/hyracks-examples/btree-example/btreeclient/src/main/java/org/apache/hyracks/examples/btree/client/SecondaryIndexBulkLoadExample.java
@@ -36,7 +36,7 @@
 import org.apache.hyracks.dataflow.std.misc.NullSinkOperatorDescriptor;
 import org.apache.hyracks.dataflow.std.sort.ExternalSortOperatorDescriptor;
 import org.apache.hyracks.examples.btree.helper.IndexLifecycleManagerProvider;
-import org.apache.hyracks.examples.btree.helper.StorageManagerInterface;
+import org.apache.hyracks.examples.btree.helper.BTreeHelperStorageManager;
 import org.apache.hyracks.storage.am.btree.dataflow.BTreeDataflowHelperFactory;
 import org.apache.hyracks.storage.am.common.api.IIndexLifecycleManagerProvider;
 import org.apache.hyracks.storage.am.common.dataflow.IIndexDataflowHelperFactory;
@@ -44,7 +44,7 @@
 import org.apache.hyracks.storage.am.common.dataflow.TreeIndexDiskOrderScanOperatorDescriptor;
 import org.apache.hyracks.storage.am.common.freepage.LinkedMetadataPageManagerFactory;
 import org.apache.hyracks.storage.am.common.impls.NoOpOperationCallbackFactory;
-import org.apache.hyracks.storage.common.IStorageManagerInterface;
+import org.apache.hyracks.storage.common.IStorageManager;
 import org.kohsuke.args4j.CmdLineParser;
 import org.kohsuke.args4j.Option;
 
@@ -98,7 +98,7 @@
         String[] splitNCs = options.ncs.split(",");
 
         IIndexLifecycleManagerProvider lcManagerProvider = IndexLifecycleManagerProvider.INSTANCE;
-        IStorageManagerInterface storageManager = StorageManagerInterface.INSTANCE;
+        IStorageManager storageManager = BTreeHelperStorageManager.INSTANCE;
 
         // schema of tuples that we are retrieving from the primary index
         RecordDescriptor recDesc = new RecordDescriptor(new ISerializerDeserializer[] {
diff --git a/hyracks-fullstack/hyracks/hyracks-examples/btree-example/btreeclient/src/main/java/org/apache/hyracks/examples/btree/client/SecondaryIndexSearchExample.java b/hyracks-fullstack/hyracks/hyracks-examples/btree-example/btreeclient/src/main/java/org/apache/hyracks/examples/btree/client/SecondaryIndexSearchExample.java
index a4b366a..94152d1 100644
--- a/hyracks-fullstack/hyracks/hyracks-examples/btree-example/btreeclient/src/main/java/org/apache/hyracks/examples/btree/client/SecondaryIndexSearchExample.java
+++ b/hyracks-fullstack/hyracks/hyracks-examples/btree-example/btreeclient/src/main/java/org/apache/hyracks/examples/btree/client/SecondaryIndexSearchExample.java
@@ -40,14 +40,14 @@
 import org.apache.hyracks.dataflow.std.misc.ConstantTupleSourceOperatorDescriptor;
 import org.apache.hyracks.dataflow.std.misc.PrinterOperatorDescriptor;
 import org.apache.hyracks.examples.btree.helper.IndexLifecycleManagerProvider;
-import org.apache.hyracks.examples.btree.helper.StorageManagerInterface;
+import org.apache.hyracks.examples.btree.helper.BTreeHelperStorageManager;
 import org.apache.hyracks.storage.am.btree.dataflow.BTreeDataflowHelperFactory;
 import org.apache.hyracks.storage.am.btree.dataflow.BTreeSearchOperatorDescriptor;
 import org.apache.hyracks.storage.am.common.api.IIndexLifecycleManagerProvider;
 import org.apache.hyracks.storage.am.common.dataflow.IIndexDataflowHelperFactory;
 import org.apache.hyracks.storage.am.common.freepage.LinkedMetadataPageManagerFactory;
 import org.apache.hyracks.storage.am.common.impls.NoOpOperationCallbackFactory;
-import org.apache.hyracks.storage.common.IStorageManagerInterface;
+import org.apache.hyracks.storage.common.IStorageManager;
 import org.kohsuke.args4j.CmdLineParser;
 import org.kohsuke.args4j.Option;
 
@@ -98,7 +98,7 @@
         String[] splitNCs = options.ncs.split(",");
 
         IIndexLifecycleManagerProvider lcManagerProvider = IndexLifecycleManagerProvider.INSTANCE;
-        IStorageManagerInterface storageManager = StorageManagerInterface.INSTANCE;
+        IStorageManager storageManager = BTreeHelperStorageManager.INSTANCE;
 
         // schema of tuples coming out of secondary index
         RecordDescriptor secondaryRecDesc = new RecordDescriptor(new ISerializerDeserializer[] {
diff --git a/hyracks-fullstack/hyracks/hyracks-examples/btree-example/btreehelper/src/main/java/org/apache/hyracks/examples/btree/helper/StorageManagerInterface.java b/hyracks-fullstack/hyracks/hyracks-examples/btree-example/btreehelper/src/main/java/org/apache/hyracks/examples/btree/helper/BTreeHelperStorageManager.java
similarity index 87%
rename from hyracks-fullstack/hyracks/hyracks-examples/btree-example/btreehelper/src/main/java/org/apache/hyracks/examples/btree/helper/StorageManagerInterface.java
rename to hyracks-fullstack/hyracks/hyracks-examples/btree-example/btreehelper/src/main/java/org/apache/hyracks/examples/btree/helper/BTreeHelperStorageManager.java
index 7996bb0..8f05266 100644
--- a/hyracks-fullstack/hyracks/hyracks-examples/btree-example/btreehelper/src/main/java/org/apache/hyracks/examples/btree/helper/StorageManagerInterface.java
+++ b/hyracks-fullstack/hyracks/hyracks-examples/btree-example/btreehelper/src/main/java/org/apache/hyracks/examples/btree/helper/BTreeHelperStorageManager.java
@@ -20,18 +20,18 @@
 package org.apache.hyracks.examples.btree.helper;
 
 import org.apache.hyracks.api.context.IHyracksTaskContext;
-import org.apache.hyracks.storage.common.IStorageManagerInterface;
+import org.apache.hyracks.storage.common.IStorageManager;
 import org.apache.hyracks.storage.common.buffercache.IBufferCache;
 import org.apache.hyracks.storage.common.file.IFileMapProvider;
 import org.apache.hyracks.storage.common.file.ILocalResourceRepository;
 import org.apache.hyracks.storage.common.file.ResourceIdFactory;
 
-public class StorageManagerInterface implements IStorageManagerInterface {
+public class BTreeHelperStorageManager implements IStorageManager {
     private static final long serialVersionUID = 1L;
 
-    public static final StorageManagerInterface INSTANCE = new StorageManagerInterface();
+    public static final BTreeHelperStorageManager INSTANCE = new BTreeHelperStorageManager();
 
-    private StorageManagerInterface() {
+    private BTreeHelperStorageManager() {
     }
 
     @Override
diff --git a/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/am/btree/AbstractBTreeOperatorTest.java b/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/am/btree/AbstractBTreeOperatorTest.java
index 1bec90c..94252b3 100644
--- a/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/am/btree/AbstractBTreeOperatorTest.java
+++ b/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/am/btree/AbstractBTreeOperatorTest.java
@@ -56,11 +56,11 @@
 import org.apache.hyracks.storage.am.common.freepage.LinkedMetadataPageManagerFactory;
 import org.apache.hyracks.storage.am.common.impls.NoOpOperationCallbackFactory;
 import org.apache.hyracks.storage.am.common.ophelpers.IndexOperation;
-import org.apache.hyracks.storage.common.IStorageManagerInterface;
+import org.apache.hyracks.storage.common.IStorageManager;
 import org.apache.hyracks.storage.common.file.TransientLocalResourceFactoryProvider;
 import org.apache.hyracks.test.support.TestIndexLifecycleManagerProvider;
 import org.apache.hyracks.test.support.TestStorageManagerComponentHolder;
-import org.apache.hyracks.test.support.TestStorageManagerInterface;
+import org.apache.hyracks.test.support.TestStorageManager;
 import org.apache.hyracks.tests.am.common.ITreeIndexOperatorTestHelper;
 import org.apache.hyracks.tests.integration.AbstractIntegrationTest;
 import org.junit.After;
@@ -71,7 +71,7 @@
         TestStorageManagerComponentHolder.init(8192, 20, 20);
     }
 
-    protected final IStorageManagerInterface storageManager = new TestStorageManagerInterface();
+    protected final IStorageManager storageManager = new TestStorageManager();
     protected final IIndexLifecycleManagerProvider lcManagerProvider = new TestIndexLifecycleManagerProvider();
     protected IIndexDataflowHelperFactory dataflowHelperFactory;
 
diff --git a/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/am/lsm/btree/LSMBTreeOperatorTestHelper.java b/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/am/lsm/btree/LSMBTreeOperatorTestHelper.java
index 80f4d34..936bd13 100644
--- a/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/am/lsm/btree/LSMBTreeOperatorTestHelper.java
+++ b/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/am/lsm/btree/LSMBTreeOperatorTestHelper.java
@@ -28,7 +28,7 @@
 import org.apache.hyracks.storage.am.lsm.common.impls.ConstantMergePolicyFactory;
 import org.apache.hyracks.storage.am.lsm.common.impls.NoOpIOOperationCallback;
 import org.apache.hyracks.storage.am.lsm.common.impls.SynchronousSchedulerProvider;
-import org.apache.hyracks.storage.am.lsm.common.impls.ThreadCountingOperationTrackerProvider;
+import org.apache.hyracks.storage.am.lsm.common.impls.ThreadCountingOperationTrackerFactory;
 import org.apache.hyracks.tests.am.common.LSMTreeOperatorTestHelper;
 
 public class LSMBTreeOperatorTestHelper extends LSMTreeOperatorTestHelper {
@@ -45,7 +45,7 @@
 
     public IIndexDataflowHelperFactory createDataFlowHelperFactory() {
         return new LSMBTreeDataflowHelperFactory(virtualBufferCacheProvider, new ConstantMergePolicyFactory(),
-                MERGE_POLICY_PROPERTIES, ThreadCountingOperationTrackerProvider.INSTANCE,
+                MERGE_POLICY_PROPERTIES, ThreadCountingOperationTrackerFactory.INSTANCE,
                 SynchronousSchedulerProvider.INSTANCE, NoOpIOOperationCallback.INSTANCE,
                 DEFAULT_BLOOM_FILTER_FALSE_POSITIVE_RATE, true, null, null, null, null, true);
     }
diff --git a/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/am/lsm/rtree/LSMRTreeOperatorTestHelper.java b/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/am/lsm/rtree/LSMRTreeOperatorTestHelper.java
index c1f2cf4..f4c80c1 100644
--- a/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/am/lsm/rtree/LSMRTreeOperatorTestHelper.java
+++ b/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/am/lsm/rtree/LSMRTreeOperatorTestHelper.java
@@ -30,7 +30,7 @@
 import org.apache.hyracks.storage.am.lsm.common.impls.ConstantMergePolicyFactory;
 import org.apache.hyracks.storage.am.lsm.common.impls.NoOpIOOperationCallback;
 import org.apache.hyracks.storage.am.lsm.common.impls.SynchronousSchedulerProvider;
-import org.apache.hyracks.storage.am.lsm.common.impls.ThreadCountingOperationTrackerProvider;
+import org.apache.hyracks.storage.am.lsm.common.impls.ThreadCountingOperationTrackerFactory;
 import org.apache.hyracks.storage.am.lsm.rtree.dataflow.LSMRTreeDataflowHelperFactory;
 import org.apache.hyracks.storage.am.rtree.frames.RTreePolicyType;
 import org.apache.hyracks.tests.am.common.LSMTreeOperatorTestHelper;
@@ -53,7 +53,7 @@
             int[] btreeFields) {
         return new LSMRTreeDataflowHelperFactory(valueProviderFactories, rtreePolicyType, btreeComparatorFactories,
                 virtualBufferCacheProvider, new ConstantMergePolicyFactory(), MERGE_POLICY_PROPERTIES,
-                ThreadCountingOperationTrackerProvider.INSTANCE, SynchronousSchedulerProvider.INSTANCE,
+                ThreadCountingOperationTrackerFactory.INSTANCE, SynchronousSchedulerProvider.INSTANCE,
                 NoOpIOOperationCallback.INSTANCE, linearizerCmpFactory, DEFAULT_BLOOM_FILTER_FALSE_POSITIVE_RATE, null,
                 btreeFields, null, null, null, true, false);
     }
diff --git a/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/am/lsm/rtree/LSMRTreeWithAntiMatterTuplesOperatorTestHelper.java b/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/am/lsm/rtree/LSMRTreeWithAntiMatterTuplesOperatorTestHelper.java
index 403e6ba..a2a6971 100644
--- a/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/am/lsm/rtree/LSMRTreeWithAntiMatterTuplesOperatorTestHelper.java
+++ b/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/am/lsm/rtree/LSMRTreeWithAntiMatterTuplesOperatorTestHelper.java
@@ -30,7 +30,7 @@
 import org.apache.hyracks.storage.am.lsm.common.impls.ConstantMergePolicyFactory;
 import org.apache.hyracks.storage.am.lsm.common.impls.NoOpIOOperationCallback;
 import org.apache.hyracks.storage.am.lsm.common.impls.SynchronousSchedulerProvider;
-import org.apache.hyracks.storage.am.lsm.common.impls.ThreadCountingOperationTrackerProvider;
+import org.apache.hyracks.storage.am.lsm.common.impls.ThreadCountingOperationTrackerFactory;
 import org.apache.hyracks.storage.am.lsm.rtree.dataflow.LSMRTreeWithAntiMatterTuplesDataflowHelperFactory;
 import org.apache.hyracks.storage.am.rtree.frames.RTreePolicyType;
 import org.apache.hyracks.tests.am.common.LSMTreeOperatorTestHelper;
@@ -52,7 +52,7 @@
             IBinaryComparatorFactory[] btreeComparatorFactories, ILinearizeComparatorFactory linearizerCmpFactory) {
         return new LSMRTreeWithAntiMatterTuplesDataflowHelperFactory(valueProviderFactories, rtreePolicyType,
                 btreeComparatorFactories, virtualBufferCacheProvider, new ConstantMergePolicyFactory(),
-                MERGE_POLICY_PROPERTIES, ThreadCountingOperationTrackerProvider.INSTANCE,
+                MERGE_POLICY_PROPERTIES, ThreadCountingOperationTrackerFactory.INSTANCE,
                 SynchronousSchedulerProvider.INSTANCE, NoOpIOOperationCallback.INSTANCE, linearizerCmpFactory, null,
                 null, null, null, true, false);
     }
diff --git a/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/am/rtree/AbstractRTreeOperatorTest.java b/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/am/rtree/AbstractRTreeOperatorTest.java
index 6d90d07..6e4214f 100644
--- a/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/am/rtree/AbstractRTreeOperatorTest.java
+++ b/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/am/rtree/AbstractRTreeOperatorTest.java
@@ -66,11 +66,11 @@
 import org.apache.hyracks.storage.am.lsm.rtree.utils.LSMRTreeUtils;
 import org.apache.hyracks.storage.am.rtree.frames.RTreePolicyType;
 import org.apache.hyracks.storage.am.rtree.util.RTreeUtils;
-import org.apache.hyracks.storage.common.IStorageManagerInterface;
+import org.apache.hyracks.storage.common.IStorageManager;
 import org.apache.hyracks.storage.common.file.TransientLocalResourceFactoryProvider;
 import org.apache.hyracks.test.support.TestIndexLifecycleManagerProvider;
 import org.apache.hyracks.test.support.TestStorageManagerComponentHolder;
-import org.apache.hyracks.test.support.TestStorageManagerInterface;
+import org.apache.hyracks.test.support.TestStorageManager;
 import org.apache.hyracks.tests.am.common.ITreeIndexOperatorTestHelper;
 import org.apache.hyracks.tests.integration.AbstractIntegrationTest;
 import org.junit.After;
@@ -89,7 +89,7 @@
 
     protected RTreeType rTreeType;
 
-    protected final IStorageManagerInterface storageManager = new TestStorageManagerInterface();
+    protected final IStorageManager storageManager = new TestStorageManager();
     protected final IIndexLifecycleManagerProvider lcManagerProvider = new TestIndexLifecycleManagerProvider();
     protected final IPageManagerFactory pageManagerFactory = new LinkedMetadataPageManagerFactory();
     protected IIndexDataflowHelperFactory rtreeDataflowHelperFactory;
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/dataflow/BTreeSearchOperatorDescriptor.java b/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/dataflow/BTreeSearchOperatorDescriptor.java
index 1c815bf..acb008d 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/dataflow/BTreeSearchOperatorDescriptor.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/dataflow/BTreeSearchOperatorDescriptor.java
@@ -35,7 +35,7 @@
 import org.apache.hyracks.storage.am.common.dataflow.AbstractTreeIndexOperatorDescriptor;
 import org.apache.hyracks.storage.am.common.dataflow.IIndexDataflowHelperFactory;
 import org.apache.hyracks.storage.am.common.impls.NoOpOperationCallbackFactory;
-import org.apache.hyracks.storage.common.IStorageManagerInterface;
+import org.apache.hyracks.storage.common.IStorageManager;
 import org.apache.hyracks.storage.common.file.NoOpLocalResourceFactoryProvider;
 
 public class BTreeSearchOperatorDescriptor extends AbstractTreeIndexOperatorDescriptor {
@@ -50,7 +50,7 @@
     private final int[] maxFilterFieldIndexes;
 
     public BTreeSearchOperatorDescriptor(IOperatorDescriptorRegistry spec, RecordDescriptor recDesc,
-            IStorageManagerInterface storageManager, IIndexLifecycleManagerProvider lifecycleManagerProvider,
+            IStorageManager storageManager, IIndexLifecycleManagerProvider lifecycleManagerProvider,
             IFileSplitProvider fileSplitProvider, ITypeTraits[] typeTraits,
             IBinaryComparatorFactory[] comparatorFactories, int[] bloomFilterKeyFields, int[] lowKeyFields,
             int[] highKeyFields, boolean lowKeyInclusive, boolean highKeyInclusive,
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/dataflow/BTreeUpdateSearchOperatorDescriptor.java b/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/dataflow/BTreeUpdateSearchOperatorDescriptor.java
index 00cf356..665e027 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/dataflow/BTreeUpdateSearchOperatorDescriptor.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/dataflow/BTreeUpdateSearchOperatorDescriptor.java
@@ -33,7 +33,7 @@
 import org.apache.hyracks.storage.am.common.api.ISearchOperationCallbackFactory;
 import org.apache.hyracks.storage.am.common.api.ITupleUpdaterFactory;
 import org.apache.hyracks.storage.am.common.dataflow.IIndexDataflowHelperFactory;
-import org.apache.hyracks.storage.common.IStorageManagerInterface;
+import org.apache.hyracks.storage.common.IStorageManager;
 
 public class BTreeUpdateSearchOperatorDescriptor extends BTreeSearchOperatorDescriptor {
 
@@ -42,7 +42,7 @@
     private final ITupleUpdaterFactory tupleUpdaterFactory;
 
     public BTreeUpdateSearchOperatorDescriptor(IOperatorDescriptorRegistry spec, RecordDescriptor recDesc,
-            IStorageManagerInterface storageManager, IIndexLifecycleManagerProvider lifecycleManagerProvider,
+            IStorageManager storageManager, IIndexLifecycleManagerProvider lifecycleManagerProvider,
             IFileSplitProvider fileSplitProvider, ITypeTraits[] typeTraits,
             IBinaryComparatorFactory[] comparatorFactories, int[] bloomFilterKeyFields, int[] lowKeyFields,
             int[] highKeyFields, boolean lowKeyInclusive, boolean highKeyInclusive,
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/impls/BTree.java b/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/impls/BTree.java
index 5fdd943..653689a 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/impls/BTree.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/impls/BTree.java
@@ -31,7 +31,7 @@
 import org.apache.hyracks.api.io.FileReference;
 import org.apache.hyracks.data.std.primitive.IntegerPointable;
 import org.apache.hyracks.dataflow.common.data.accessors.ITupleReference;
-import org.apache.hyracks.dataflow.common.util.TupleUtils;
+import org.apache.hyracks.dataflow.common.utils.TupleUtils;
 import org.apache.hyracks.storage.am.btree.api.IBTreeFrame;
 import org.apache.hyracks.storage.am.btree.api.IBTreeInteriorFrame;
 import org.apache.hyracks.storage.am.btree.api.IBTreeLeafFrame;
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/impls/BTreeOpContext.java b/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/impls/BTreeOpContext.java
index 3c5d0e5..dff6743 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/impls/BTreeOpContext.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/impls/BTreeOpContext.java
@@ -27,7 +27,7 @@
 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.dataflow.common.util.TupleUtils;
+import org.apache.hyracks.dataflow.common.utils.TupleUtils;
 import org.apache.hyracks.storage.am.btree.api.IBTreeInteriorFrame;
 import org.apache.hyracks.storage.am.btree.api.IBTreeLeafFrame;
 import org.apache.hyracks.storage.am.btree.api.ITupleAcceptor;
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/impls/BTreeRangeSearchCursor.java b/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/impls/BTreeRangeSearchCursor.java
index 3301e37..b4fdd9e 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/impls/BTreeRangeSearchCursor.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/impls/BTreeRangeSearchCursor.java
@@ -23,7 +23,7 @@
 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.dataflow.common.util.TupleUtils;
+import org.apache.hyracks.dataflow.common.utils.TupleUtils;
 import org.apache.hyracks.storage.am.btree.api.IBTreeLeafFrame;
 import org.apache.hyracks.storage.am.common.api.ICursorInitialState;
 import org.apache.hyracks.storage.am.common.api.IIndexAccessor;
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/api/IIndexLifecycleManagerProvider.java b/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/api/IIndexLifecycleManagerProvider.java
index 5378610..673319f 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/api/IIndexLifecycleManagerProvider.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/api/IIndexLifecycleManagerProvider.java
@@ -22,6 +22,10 @@
 
 import org.apache.hyracks.api.context.IHyracksTaskContext;
 
+/**
+ * Provides an {@link org.apache.hyracks.storage.am.common.api.IResourceLifecycleManager<IIndex>} instance
+ */
+@FunctionalInterface
 public interface IIndexLifecycleManagerProvider extends Serializable {
-    public IResourceLifecycleManager<IIndex> getLifecycleManager(IHyracksTaskContext ctx);
+    IResourceLifecycleManager<IIndex> getLifecycleManager(IHyracksTaskContext ctx);
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/api/IMetadataPageManagerFactory.java b/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/api/IMetadataPageManagerFactory.java
index 78a80e7..0c4167c 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/api/IMetadataPageManagerFactory.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/api/IMetadataPageManagerFactory.java
@@ -20,8 +20,12 @@
 
 import org.apache.hyracks.storage.common.buffercache.IBufferCache;
 
+/**
+ * A factory class for {@link org.apache.hyracks.storage.am.common.api.IMetadataPageManager}
+ */
 @FunctionalInterface
 public interface IMetadataPageManagerFactory extends IPageManagerFactory {
+
     @Override
     IMetadataPageManager createPageManager(IBufferCache bufferCache);
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/api/IPrimitiveValueProvider.java b/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/api/IPrimitiveValueProvider.java
index d9ab69f..754bb41 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/api/IPrimitiveValueProvider.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/api/IPrimitiveValueProvider.java
@@ -19,6 +19,17 @@
 
 package org.apache.hyracks.storage.am.common.api;
 
+/**
+ * Provides double values for RTree operations from encoded bytes
+ */
+@FunctionalInterface
 public interface IPrimitiveValueProvider {
-    public double getValue(byte[] bytes, int offset);
+    /**
+     * @param bytes
+     *            the source byte array
+     * @param offset
+     *            the offset of the value
+     * @return the decoded double value
+     */
+    double getValue(byte[] bytes, int offset);
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/api/IPrimitiveValueProviderFactory.java b/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/api/IPrimitiveValueProviderFactory.java
index ceb5f71..942517c 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/api/IPrimitiveValueProviderFactory.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/api/IPrimitiveValueProviderFactory.java
@@ -21,6 +21,13 @@
 
 import java.io.Serializable;
 
+/**
+ * Provides {@link org.apache.hyracks.storage.am.common.api.IPrimitiveValueProvider} for R-Tree operations
+ */
+@FunctionalInterface
 public interface IPrimitiveValueProviderFactory extends Serializable {
-    public IPrimitiveValueProvider createPrimitiveValueProvider();
+    /**
+     * @return {@link org.apache.hyracks.storage.am.common.api.IPrimitiveValueProvider} instance
+     */
+    IPrimitiveValueProvider createPrimitiveValueProvider();
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/dataflow/AbstractIndexOperatorDescriptor.java b/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/dataflow/AbstractIndexOperatorDescriptor.java
index d304883..00123f7 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/dataflow/AbstractIndexOperatorDescriptor.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/dataflow/AbstractIndexOperatorDescriptor.java
@@ -29,7 +29,7 @@
 import org.apache.hyracks.storage.am.common.api.IPageManagerFactory;
 import org.apache.hyracks.storage.am.common.api.ISearchOperationCallbackFactory;
 import org.apache.hyracks.storage.am.common.api.ITupleFilterFactory;
-import org.apache.hyracks.storage.common.IStorageManagerInterface;
+import org.apache.hyracks.storage.common.IStorageManager;
 import org.apache.hyracks.storage.common.file.ILocalResourceFactoryProvider;
 
 public abstract class AbstractIndexOperatorDescriptor extends AbstractSingleActivityOperatorDescriptor implements
@@ -38,7 +38,7 @@
     private static final long serialVersionUID = 1L;
 
     protected final IFileSplitProvider fileSplitProvider;
-    protected final IStorageManagerInterface storageManager;
+    protected final IStorageManager storageManager;
     protected final IIndexLifecycleManagerProvider lifecycleManagerProvider;
     protected final IIndexDataflowHelperFactory dataflowHelperFactory;
     protected final ITupleFilterFactory tupleFilterFactory;
@@ -51,7 +51,7 @@
     protected final ILocalResourceFactoryProvider localResourceFactoryProvider;
 
     public AbstractIndexOperatorDescriptor(IOperatorDescriptorRegistry spec, int inputArity, int outputArity,
-            RecordDescriptor recDesc, IStorageManagerInterface storageManager,
+            RecordDescriptor recDesc, IStorageManager storageManager,
             IIndexLifecycleManagerProvider lifecycleManagerProvider, IFileSplitProvider fileSplitProvider,
             IIndexDataflowHelperFactory dataflowHelperFactory, ITupleFilterFactory tupleFilterFactory,
             boolean retainInput, boolean retainNull, IMissingWriterFactory nullWriterFactory,
@@ -83,7 +83,7 @@
     }
 
     @Override
-    public IStorageManagerInterface getStorageManager() {
+    public IStorageManager getStorageManager() {
         return storageManager;
     }
 
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/dataflow/AbstractTreeIndexOperatorDescriptor.java b/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/dataflow/AbstractTreeIndexOperatorDescriptor.java
index 9455d9e..99b0854 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/dataflow/AbstractTreeIndexOperatorDescriptor.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/dataflow/AbstractTreeIndexOperatorDescriptor.java
@@ -30,7 +30,7 @@
 import org.apache.hyracks.storage.am.common.api.IPageManagerFactory;
 import org.apache.hyracks.storage.am.common.api.ISearchOperationCallbackFactory;
 import org.apache.hyracks.storage.am.common.api.ITupleFilterFactory;
-import org.apache.hyracks.storage.common.IStorageManagerInterface;
+import org.apache.hyracks.storage.common.IStorageManager;
 import org.apache.hyracks.storage.common.file.ILocalResourceFactoryProvider;
 
 public abstract class AbstractTreeIndexOperatorDescriptor extends AbstractIndexOperatorDescriptor {
@@ -42,17 +42,18 @@
     protected final int[] bloomFilterKeyFields;
 
     public AbstractTreeIndexOperatorDescriptor(IOperatorDescriptorRegistry spec, int inputArity, int outputArity,
-            RecordDescriptor recDesc, IStorageManagerInterface storageManager,
+            RecordDescriptor recDesc, IStorageManager storageManager,
             IIndexLifecycleManagerProvider lifecycleManagerProvider, IFileSplitProvider fileSplitProvider,
             ITypeTraits[] typeTraits, IBinaryComparatorFactory[] comparatorFactories, int[] bloomFilterKeyFields,
             IIndexDataflowHelperFactory dataflowHelperFactory, ITupleFilterFactory tupleFilterFactory,
-            boolean retainInput, boolean retainNull, IMissingWriterFactory nullWriterFactory,
+            boolean retainInput, boolean retainNull, IMissingWriterFactory missingWriterFactory,
             ILocalResourceFactoryProvider localResourceFactoryProvider,
             ISearchOperationCallbackFactory searchOpCallbackFactory,
             IModificationOperationCallbackFactory modificationOpCallbackFactory,
             IPageManagerFactory pageManagerFactory) {
-        super(spec, inputArity, outputArity, recDesc, storageManager, lifecycleManagerProvider, fileSplitProvider,
-                dataflowHelperFactory, tupleFilterFactory, retainInput, retainNull, nullWriterFactory,
+        super(spec, inputArity, outputArity, recDesc, storageManager, lifecycleManagerProvider,
+                fileSplitProvider,
+                dataflowHelperFactory, tupleFilterFactory, retainInput, retainNull, missingWriterFactory,
                 localResourceFactoryProvider, searchOpCallbackFactory, modificationOpCallbackFactory,
                 pageManagerFactory);
         this.typeTraits = typeTraits;
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/dataflow/IIndexOperatorDescriptor.java b/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/dataflow/IIndexOperatorDescriptor.java
index f78c843..388dc85 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/dataflow/IIndexOperatorDescriptor.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/dataflow/IIndexOperatorDescriptor.java
@@ -28,13 +28,13 @@
 import org.apache.hyracks.storage.am.common.api.IPageManagerFactory;
 import org.apache.hyracks.storage.am.common.api.ISearchOperationCallbackFactory;
 import org.apache.hyracks.storage.am.common.api.ITupleFilterFactory;
-import org.apache.hyracks.storage.common.IStorageManagerInterface;
+import org.apache.hyracks.storage.common.IStorageManager;
 import org.apache.hyracks.storage.common.file.ILocalResourceFactoryProvider;
 
 public interface IIndexOperatorDescriptor extends IActivity {
     IFileSplitProvider getFileSplitProvider();
 
-    IStorageManagerInterface getStorageManager();
+    IStorageManager getStorageManager();
 
     IIndexLifecycleManagerProvider getLifecycleManagerProvider();
 
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/dataflow/IndexDropOperatorDescriptor.java b/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/dataflow/IndexDropOperatorDescriptor.java
index 9d3be6f..9ddd61a 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/dataflow/IndexDropOperatorDescriptor.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/dataflow/IndexDropOperatorDescriptor.java
@@ -31,14 +31,14 @@
 import org.apache.hyracks.storage.am.common.api.IIndexLifecycleManagerProvider;
 import org.apache.hyracks.storage.am.common.api.IPageManagerFactory;
 import org.apache.hyracks.storage.am.common.impls.NoOpOperationCallbackFactory;
-import org.apache.hyracks.storage.common.IStorageManagerInterface;
+import org.apache.hyracks.storage.common.IStorageManager;
 import org.apache.hyracks.storage.common.file.NoOpLocalResourceFactoryProvider;
 
 public class IndexDropOperatorDescriptor extends AbstractTreeIndexOperatorDescriptor {
 
     private static final long serialVersionUID = 1L;
 
-    public IndexDropOperatorDescriptor(IOperatorDescriptorRegistry spec, IStorageManagerInterface storageManager,
+    public IndexDropOperatorDescriptor(IOperatorDescriptorRegistry spec, IStorageManager storageManager,
             IIndexLifecycleManagerProvider lifecycleManagerProvider, IFileSplitProvider fileSplitProvider,
             IIndexDataflowHelperFactory dataflowHelperFactory, IPageManagerFactory pageManagerFactory) {
         // TODO: providing the type traits below is a hack to allow:
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/dataflow/TreeIndexBulkLoadOperatorDescriptor.java b/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/dataflow/TreeIndexBulkLoadOperatorDescriptor.java
index cd26fb0..ddb5df7 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/dataflow/TreeIndexBulkLoadOperatorDescriptor.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/dataflow/TreeIndexBulkLoadOperatorDescriptor.java
@@ -31,7 +31,7 @@
 import org.apache.hyracks.storage.am.common.api.IIndexLifecycleManagerProvider;
 import org.apache.hyracks.storage.am.common.api.IPageManagerFactory;
 import org.apache.hyracks.storage.am.common.impls.NoOpOperationCallbackFactory;
-import org.apache.hyracks.storage.common.IStorageManagerInterface;
+import org.apache.hyracks.storage.common.IStorageManager;
 import org.apache.hyracks.storage.common.file.NoOpLocalResourceFactoryProvider;
 
 public class TreeIndexBulkLoadOperatorDescriptor extends AbstractTreeIndexOperatorDescriptor {
@@ -45,7 +45,7 @@
     private final boolean checkIfEmptyIndex;
 
     public TreeIndexBulkLoadOperatorDescriptor(IOperatorDescriptorRegistry spec, RecordDescriptor recDesc,
-            IStorageManagerInterface storageManager, IIndexLifecycleManagerProvider lifecycleManagerProvider,
+            IStorageManager storageManager, IIndexLifecycleManagerProvider lifecycleManagerProvider,
             IFileSplitProvider fileSplitProvider, ITypeTraits[] typeTraits,
             IBinaryComparatorFactory[] comparatorFactories, int[] bloomFilterKeyFields, int[] fieldPermutation,
             float fillFactor, boolean verifyInput, long numElementsHint, boolean checkIfEmptyIndex,
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/dataflow/TreeIndexCreateOperatorDescriptor.java b/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/dataflow/TreeIndexCreateOperatorDescriptor.java
index c225d37..5448b7b 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/dataflow/TreeIndexCreateOperatorDescriptor.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/dataflow/TreeIndexCreateOperatorDescriptor.java
@@ -31,14 +31,14 @@
 import org.apache.hyracks.storage.am.common.api.IModificationOperationCallbackFactory;
 import org.apache.hyracks.storage.am.common.api.IPageManagerFactory;
 import org.apache.hyracks.storage.am.common.impls.NoOpOperationCallbackFactory;
-import org.apache.hyracks.storage.common.IStorageManagerInterface;
+import org.apache.hyracks.storage.common.IStorageManager;
 import org.apache.hyracks.storage.common.file.ILocalResourceFactoryProvider;
 
 public class TreeIndexCreateOperatorDescriptor extends AbstractTreeIndexOperatorDescriptor {
 
     private static final long serialVersionUID = 1L;
 
-    public TreeIndexCreateOperatorDescriptor(IOperatorDescriptorRegistry spec, IStorageManagerInterface storageManager,
+    public TreeIndexCreateOperatorDescriptor(IOperatorDescriptorRegistry spec, IStorageManager storageManager,
             IIndexLifecycleManagerProvider lifecycleManagerProvider, IFileSplitProvider fileSplitProvider,
             ITypeTraits[] typeTraits, IBinaryComparatorFactory[] comparatorFactories, int[] bloomFilterKeyFields,
             IIndexDataflowHelperFactory dataflowHelperFactory,
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/dataflow/TreeIndexDiskOrderScanOperatorDescriptor.java b/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/dataflow/TreeIndexDiskOrderScanOperatorDescriptor.java
index 58b63a8..378c49b 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/dataflow/TreeIndexDiskOrderScanOperatorDescriptor.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/dataflow/TreeIndexDiskOrderScanOperatorDescriptor.java
@@ -31,7 +31,7 @@
 import org.apache.hyracks.storage.am.common.api.IPageManagerFactory;
 import org.apache.hyracks.storage.am.common.api.ISearchOperationCallbackFactory;
 import org.apache.hyracks.storage.am.common.impls.NoOpOperationCallbackFactory;
-import org.apache.hyracks.storage.common.IStorageManagerInterface;
+import org.apache.hyracks.storage.common.IStorageManager;
 import org.apache.hyracks.storage.common.file.NoOpLocalResourceFactoryProvider;
 
 public class TreeIndexDiskOrderScanOperatorDescriptor extends AbstractTreeIndexOperatorDescriptor {
@@ -39,7 +39,7 @@
     private static final long serialVersionUID = 1L;
 
     public TreeIndexDiskOrderScanOperatorDescriptor(IOperatorDescriptorRegistry spec, RecordDescriptor recDesc,
-            IStorageManagerInterface storageManager, IIndexLifecycleManagerProvider lifecycleManagerProvider,
+            IStorageManager storageManager, IIndexLifecycleManagerProvider lifecycleManagerProvider,
             IFileSplitProvider fileSplitProvider, ITypeTraits[] typeTraits,
             IIndexDataflowHelperFactory dataflowHelperFactory,
             ISearchOperationCallbackFactory searchOpCallbackProvider, IPageManagerFactory pageManagerFactory) {
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/dataflow/TreeIndexInsertUpdateDeleteOperatorDescriptor.java b/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/dataflow/TreeIndexInsertUpdateDeleteOperatorDescriptor.java
index 9e1b172..c9a5be7 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/dataflow/TreeIndexInsertUpdateDeleteOperatorDescriptor.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/dataflow/TreeIndexInsertUpdateDeleteOperatorDescriptor.java
@@ -34,7 +34,7 @@
 import org.apache.hyracks.storage.am.common.api.ITupleFilterFactory;
 import org.apache.hyracks.storage.am.common.impls.NoOpOperationCallbackFactory;
 import org.apache.hyracks.storage.am.common.ophelpers.IndexOperation;
-import org.apache.hyracks.storage.common.IStorageManagerInterface;
+import org.apache.hyracks.storage.common.IStorageManager;
 import org.apache.hyracks.storage.common.file.NoOpLocalResourceFactoryProvider;
 
 public class TreeIndexInsertUpdateDeleteOperatorDescriptor extends AbstractTreeIndexOperatorDescriptor {
@@ -45,7 +45,7 @@
     private final IndexOperation op;
 
     public TreeIndexInsertUpdateDeleteOperatorDescriptor(IOperatorDescriptorRegistry spec, RecordDescriptor recDesc,
-            IStorageManagerInterface storageManager, IIndexLifecycleManagerProvider lifecycleManagerProvider,
+            IStorageManager storageManager, IIndexLifecycleManagerProvider lifecycleManagerProvider,
             IFileSplitProvider fileSplitProvider, ITypeTraits[] typeTraits,
             IBinaryComparatorFactory[] comparatorFactories, int[] bloomFilterKeyFields, int[] fieldPermutation,
             IndexOperation op, IIndexDataflowHelperFactory dataflowHelperFactory,
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/dataflow/TreeIndexStatsOperatorDescriptor.java b/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/dataflow/TreeIndexStatsOperatorDescriptor.java
index ecedc21..c145ea9 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/dataflow/TreeIndexStatsOperatorDescriptor.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/dataflow/TreeIndexStatsOperatorDescriptor.java
@@ -33,7 +33,7 @@
 import org.apache.hyracks.storage.am.common.api.IPageManagerFactory;
 import org.apache.hyracks.storage.am.common.api.ISearchOperationCallbackFactory;
 import org.apache.hyracks.storage.am.common.impls.NoOpOperationCallbackFactory;
-import org.apache.hyracks.storage.common.IStorageManagerInterface;
+import org.apache.hyracks.storage.common.IStorageManager;
 import org.apache.hyracks.storage.common.file.NoOpLocalResourceFactoryProvider;
 
 public class TreeIndexStatsOperatorDescriptor extends AbstractTreeIndexOperatorDescriptor {
@@ -42,7 +42,7 @@
     private static final RecordDescriptor recDesc = new RecordDescriptor(
             new ISerializerDeserializer[] { new UTF8StringSerializerDeserializer() });
 
-    public TreeIndexStatsOperatorDescriptor(IOperatorDescriptorRegistry spec, IStorageManagerInterface storageManager,
+    public TreeIndexStatsOperatorDescriptor(IOperatorDescriptorRegistry spec, IStorageManager storageManager,
             IIndexLifecycleManagerProvider lifecycleManagerProvider, IFileSplitProvider fileSplitProvider,
             ITypeTraits[] typeTraits, IBinaryComparatorFactory[] comparatorFactories, int[] bloomFilterKeyFields,
             IIndexDataflowHelperFactory dataflowHelperFactory,
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/freepage/AppendOnlyLinkedMetadataPageManagerFactory.java b/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/freepage/AppendOnlyLinkedMetadataPageManagerFactory.java
index fea128d..2c581b0 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/freepage/AppendOnlyLinkedMetadataPageManagerFactory.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/freepage/AppendOnlyLinkedMetadataPageManagerFactory.java
@@ -25,6 +25,11 @@
 
 public class AppendOnlyLinkedMetadataPageManagerFactory implements IMetadataPageManagerFactory {
     private static final long serialVersionUID = 1L;
+    public static final AppendOnlyLinkedMetadataPageManagerFactory INSTANCE =
+            new AppendOnlyLinkedMetadataPageManagerFactory();
+
+    private AppendOnlyLinkedMetadataPageManagerFactory() {
+    }
 
     @Override
     public IMetadataPageManager createPageManager(IBufferCache bufferCache) {
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/ophelpers/IndexOperation.java b/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/ophelpers/IndexOperation.java
index f579064..43e0889 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/ophelpers/IndexOperation.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/ophelpers/IndexOperation.java
@@ -20,6 +20,7 @@
 package org.apache.hyracks.storage.am.common.ophelpers;
 
 public enum IndexOperation {
+    CREATE,
     INSERT,
     DELETE,
     UPDATE,
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/util/TreeIndexUtils.java b/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/util/TreeIndexUtils.java
index a4afe80..37ecfc0 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/util/TreeIndexUtils.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/util/TreeIndexUtils.java
@@ -21,7 +21,7 @@
 
 import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
-import org.apache.hyracks.dataflow.common.util.TupleUtils;
+import org.apache.hyracks.dataflow.common.utils.TupleUtils;
 import org.apache.hyracks.storage.am.common.api.ITreeIndexFrame;
 import org.apache.hyracks.storage.am.common.api.ITreeIndexTupleReference;
 
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/dataflow/ExternalBTreeDataflowHelper.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/dataflow/ExternalBTreeDataflowHelper.java
index b11b364..0d8f32e 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/dataflow/ExternalBTreeDataflowHelper.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/dataflow/ExternalBTreeDataflowHelper.java
@@ -27,11 +27,11 @@
 import org.apache.hyracks.storage.am.common.dataflow.AbstractTreeIndexOperatorDescriptor;
 import org.apache.hyracks.storage.am.common.dataflow.IIndexOperatorDescriptor;
 import org.apache.hyracks.storage.am.common.util.IndexFileNameUtil;
-import org.apache.hyracks.storage.am.lsm.btree.util.LSMBTreeUtils;
+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.ILSMIOOperationScheduler;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMMergePolicy;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMOperationTrackerProvider;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMOperationTrackerFactory;
 
 public class ExternalBTreeDataflowHelper extends LSMBTreeDataflowHelper {
 
@@ -39,7 +39,7 @@
 
     public ExternalBTreeDataflowHelper(IIndexOperatorDescriptor opDesc, IHyracksTaskContext ctx, int partition,
             double bloomFilterFalsePositiveRate, ILSMMergePolicy mergePolicy,
-            ILSMOperationTrackerProvider opTrackerFactory, ILSMIOOperationScheduler ioScheduler,
+            ILSMOperationTrackerFactory opTrackerFactory, ILSMIOOperationScheduler ioScheduler,
             ILSMIOOperationCallbackFactory ioOpCallbackFactory, int version, boolean durable)
             throws HyracksDataException {
         super(opDesc, ctx, partition, null, bloomFilterFalsePositiveRate, mergePolicy, opTrackerFactory, ioScheduler,
@@ -66,11 +66,12 @@
         AbstractTreeIndexOperatorDescriptor treeOpDesc = (AbstractTreeIndexOperatorDescriptor) opDesc;
         FileReference fileRef = IndexFileNameUtil.getIndexAbsoluteFileRef(treeOpDesc, ctx.getTaskAttemptId()
                 .getTaskId().getPartition(), ctx.getIOManager());
-        return LSMBTreeUtils.createExternalBTree(ctx.getIOManager(), fileRef, opDesc.getStorageManager().getBufferCache(
-                ctx), opDesc
+        return LSMBTreeUtil.createExternalBTree(ctx.getIOManager(), fileRef, opDesc.getStorageManager()
+                .getBufferCache(ctx), opDesc
                 .getStorageManager().getFileMapProvider(ctx), treeOpDesc.getTreeIndexTypeTraits(), treeOpDesc
                 .getTreeIndexComparatorFactories(), treeOpDesc.getTreeIndexBloomFilterKeyFields(),
-                bloomFilterFalsePositiveRate, mergePolicy, opTrackerFactory.getOperationTracker(ctx), ioScheduler,
+                bloomFilterFalsePositiveRate, mergePolicy, opTrackerFactory.getOperationTracker(ctx.getJobletContext()
+                        .getApplicationContext()), ioScheduler,
                 ioOpCallbackFactory.createIOOperationCallback(), getVersion(), durable,
                 (IMetadataPageManagerFactory) opDesc.getPageManagerFactory());
     }
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/dataflow/ExternalBTreeDataflowHelperFactory.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/dataflow/ExternalBTreeDataflowHelperFactory.java
index 6334fb3..9d0d2ac 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/dataflow/ExternalBTreeDataflowHelperFactory.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/dataflow/ExternalBTreeDataflowHelperFactory.java
@@ -27,7 +27,7 @@
 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.ILSMOperationTrackerProvider;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMOperationTrackerFactory;
 import org.apache.hyracks.storage.am.lsm.common.dataflow.AbstractLSMIndexDataflowHelperFactory;
 
 public class ExternalBTreeDataflowHelperFactory extends AbstractLSMIndexDataflowHelperFactory {
@@ -37,7 +37,7 @@
     private final int version;
 
     public ExternalBTreeDataflowHelperFactory(ILSMMergePolicyFactory mergePolicyFactory,
-            Map<String, String> mergePolicyProperties, ILSMOperationTrackerProvider opTrackerFactory,
+            Map<String, String> mergePolicyProperties, ILSMOperationTrackerFactory opTrackerFactory,
             ILSMIOOperationSchedulerProvider ioSchedulerProvider, ILSMIOOperationCallbackFactory ioOpCallbackFactory,
             double bloomFilterFalsePositiveRate, int version, boolean durable) {
         super(null, mergePolicyFactory, mergePolicyProperties, opTrackerFactory, ioSchedulerProvider,
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/dataflow/ExternalBTreeWithBuddyDataflowHelper.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/dataflow/ExternalBTreeWithBuddyDataflowHelper.java
index dfbeb11..b4d2797 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/dataflow/ExternalBTreeWithBuddyDataflowHelper.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/dataflow/ExternalBTreeWithBuddyDataflowHelper.java
@@ -26,11 +26,11 @@
 import org.apache.hyracks.storage.am.common.dataflow.AbstractTreeIndexOperatorDescriptor;
 import org.apache.hyracks.storage.am.common.dataflow.IIndexOperatorDescriptor;
 import org.apache.hyracks.storage.am.common.util.IndexFileNameUtil;
-import org.apache.hyracks.storage.am.lsm.btree.util.LSMBTreeUtils;
+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.ILSMIOOperationScheduler;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMMergePolicy;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMOperationTrackerProvider;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMOperationTrackerFactory;
 import org.apache.hyracks.storage.am.lsm.common.dataflow.AbstractLSMIndexDataflowHelper;
 
 public class ExternalBTreeWithBuddyDataflowHelper extends AbstractLSMIndexDataflowHelper {
@@ -39,7 +39,7 @@
     private final int version;
 
     public ExternalBTreeWithBuddyDataflowHelper(IIndexOperatorDescriptor opDesc, IHyracksTaskContext ctx,
-            int partition, ILSMMergePolicy mergePolicy, ILSMOperationTrackerProvider opTrackerFactory,
+            int partition, ILSMMergePolicy mergePolicy, ILSMOperationTrackerFactory opTrackerFactory,
             ILSMIOOperationScheduler ioScheduler, ILSMIOOperationCallbackFactory ioOpCallbackFactory,
             int[] buddyBtreeFields, int version, boolean durable) throws HyracksDataException {
         super(opDesc, ctx, partition, null, mergePolicy, opTrackerFactory, ioScheduler, ioOpCallbackFactory, null,
@@ -50,7 +50,7 @@
 
     public ExternalBTreeWithBuddyDataflowHelper(IIndexOperatorDescriptor opDesc, IHyracksTaskContext ctx,
             int partition, double bloomFilterFalsePositiveRate, ILSMMergePolicy mergePolicy,
-            ILSMOperationTrackerProvider opTrackerFactory, ILSMIOOperationScheduler ioScheduler,
+            ILSMOperationTrackerFactory opTrackerFactory, ILSMIOOperationScheduler ioScheduler,
             ILSMIOOperationCallbackFactory ioOpCallbackFactory, int[] buddyBtreeFields, int version, boolean durable)
             throws HyracksDataException {
         super(opDesc, ctx, partition, null, bloomFilterFalsePositiveRate, mergePolicy, opTrackerFactory, ioScheduler,
@@ -78,17 +78,15 @@
         AbstractTreeIndexOperatorDescriptor treeOpDesc = (AbstractTreeIndexOperatorDescriptor) opDesc;
         FileReference fileRef = IndexFileNameUtil.getIndexAbsoluteFileRef(treeOpDesc, ctx.getTaskAttemptId()
                 .getTaskId().getPartition(), ctx.getIOManager());
-        return LSMBTreeUtils.createExternalBTreeWithBuddy(ctx.getIOManager(), fileRef, opDesc.getStorageManager()
-                .getBufferCache(ctx),
+        return LSMBTreeUtil.createExternalBTreeWithBuddy(ctx.getIOManager(), fileRef,
+                opDesc.getStorageManager().getBufferCache(ctx),
                 opDesc.getStorageManager().getFileMapProvider(ctx), treeOpDesc.getTreeIndexTypeTraits(),
                 treeOpDesc.getTreeIndexComparatorFactories(), bloomFilterFalsePositiveRate, mergePolicy,
-                opTrackerFactory.getOperationTracker(ctx), ioScheduler,
+                opTrackerFactory.getOperationTracker(ctx.getJobletContext().getApplicationContext()), ioScheduler,
                 ioOpCallbackFactory.createIOOperationCallback(), buddyBtreeFields, version, durable,
                 (IMetadataPageManagerFactory) opDesc.getPageManagerFactory());
     }
-
     public int getTargetVersion() {
         return version;
     }
-
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/dataflow/ExternalBTreeWithBuddyDataflowHelperFactory.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/dataflow/ExternalBTreeWithBuddyDataflowHelperFactory.java
index 319b5c1..3b49b0f 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/dataflow/ExternalBTreeWithBuddyDataflowHelperFactory.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/dataflow/ExternalBTreeWithBuddyDataflowHelperFactory.java
@@ -27,7 +27,7 @@
 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.ILSMOperationTrackerProvider;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMOperationTrackerFactory;
 import org.apache.hyracks.storage.am.lsm.common.dataflow.AbstractLSMIndexDataflowHelperFactory;
 
 public class ExternalBTreeWithBuddyDataflowHelperFactory extends AbstractLSMIndexDataflowHelperFactory {
@@ -37,7 +37,7 @@
     private final int version;
 
     public ExternalBTreeWithBuddyDataflowHelperFactory(ILSMMergePolicyFactory mergePolicyFactory,
-            Map<String, String> mergePolicyProperties, ILSMOperationTrackerProvider opTrackerFactory,
+            Map<String, String> mergePolicyProperties, ILSMOperationTrackerFactory opTrackerFactory,
             ILSMIOOperationSchedulerProvider ioSchedulerProvider, ILSMIOOperationCallbackFactory ioOpCallbackFactory,
             double bloomFilterFalsePositiveRate, int[] buddyBtreeFields, int version, boolean durable) {
         super(null, mergePolicyFactory, mergePolicyProperties, opTrackerFactory, ioSchedulerProvider,
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/dataflow/LSMBTreeDataflowHelper.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/dataflow/LSMBTreeDataflowHelper.java
index df029c76..5f8c951 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/dataflow/LSMBTreeDataflowHelper.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/dataflow/LSMBTreeDataflowHelper.java
@@ -31,11 +31,11 @@
 import org.apache.hyracks.storage.am.common.dataflow.AbstractTreeIndexOperatorDescriptor;
 import org.apache.hyracks.storage.am.common.dataflow.IIndexOperatorDescriptor;
 import org.apache.hyracks.storage.am.common.util.IndexFileNameUtil;
-import org.apache.hyracks.storage.am.lsm.btree.util.LSMBTreeUtils;
+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.ILSMIOOperationScheduler;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMMergePolicy;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMOperationTrackerProvider;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMOperationTrackerFactory;
 import org.apache.hyracks.storage.am.lsm.common.api.IVirtualBufferCache;
 import org.apache.hyracks.storage.am.lsm.common.dataflow.AbstractLSMIndexDataflowHelper;
 
@@ -46,7 +46,7 @@
 
     public LSMBTreeDataflowHelper(IIndexOperatorDescriptor opDesc, IHyracksTaskContext ctx, int partition,
             List<IVirtualBufferCache> virtualBufferCaches, ILSMMergePolicy mergePolicy,
-            ILSMOperationTrackerProvider opTrackerFactory, ILSMIOOperationScheduler ioScheduler,
+            ILSMOperationTrackerFactory opTrackerFactory, ILSMIOOperationScheduler ioScheduler,
             ILSMIOOperationCallbackFactory ioOpCallbackFactory, boolean needKeyDupCheck,
             ITypeTraits[] filterTypeTraits, IBinaryComparatorFactory[] filterCmpFactories, int[] btreeFields,
             int[] filterFields, boolean durable) throws HyracksDataException {
@@ -57,7 +57,7 @@
 
     public LSMBTreeDataflowHelper(IIndexOperatorDescriptor opDesc, IHyracksTaskContext ctx, int partition,
             List<IVirtualBufferCache> virtualBufferCaches, double bloomFilterFalsePositiveRate,
-            ILSMMergePolicy mergePolicy, ILSMOperationTrackerProvider opTrackerFactory,
+            ILSMMergePolicy mergePolicy, ILSMOperationTrackerFactory opTrackerFactory,
             ILSMIOOperationScheduler ioScheduler, ILSMIOOperationCallbackFactory ioOpCallbackFactory,
             boolean needKeyDupCheck, ITypeTraits[] filterTypeTraits, IBinaryComparatorFactory[] filterCmpFactories,
             int[] btreeFields, int[] filterFields, boolean durable) throws HyracksDataException {
@@ -72,11 +72,12 @@
         AbstractTreeIndexOperatorDescriptor treeOpDesc = (AbstractTreeIndexOperatorDescriptor) opDesc;
         FileReference fileRef = IndexFileNameUtil.getIndexAbsoluteFileRef(treeOpDesc, ctx.getTaskAttemptId()
                 .getTaskId().getPartition(), ctx.getIOManager());
-        return LSMBTreeUtils.createLSMTree(ctx.getIOManager(), virtualBufferCaches, fileRef, opDesc.getStorageManager()
-                .getBufferCache(ctx),
+        return LSMBTreeUtil.createLSMTree(ctx.getIOManager(), virtualBufferCaches, fileRef,
+                opDesc.getStorageManager().getBufferCache(ctx),
                 opDesc.getStorageManager().getFileMapProvider(ctx), treeOpDesc.getTreeIndexTypeTraits(),
                 treeOpDesc.getTreeIndexComparatorFactories(), treeOpDesc.getTreeIndexBloomFilterKeyFields(),
-                bloomFilterFalsePositiveRate, mergePolicy, opTrackerFactory.getOperationTracker(ctx), ioScheduler,
+                bloomFilterFalsePositiveRate, mergePolicy, opTrackerFactory.getOperationTracker(ctx.getJobletContext()
+                        .getApplicationContext()), ioScheduler,
                 ioOpCallbackFactory.createIOOperationCallback(), needKeyDupCheck, filterTypeTraits, filterCmpFactories,
                 btreeFields, filterFields, durable, (IMetadataPageManagerFactory) opDesc.getPageManagerFactory());
     }
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/dataflow/LSMBTreeDataflowHelperFactory.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/dataflow/LSMBTreeDataflowHelperFactory.java
index 4093938..d0defef 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/dataflow/LSMBTreeDataflowHelperFactory.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/dataflow/LSMBTreeDataflowHelperFactory.java
@@ -30,7 +30,7 @@
 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.ILSMOperationTrackerProvider;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMOperationTrackerFactory;
 import org.apache.hyracks.storage.am.lsm.common.api.IVirtualBufferCacheProvider;
 import org.apache.hyracks.storage.am.lsm.common.dataflow.AbstractLSMIndexDataflowHelperFactory;
 
@@ -42,7 +42,7 @@
 
     public LSMBTreeDataflowHelperFactory(IVirtualBufferCacheProvider virtualBufferCacheProvider,
             ILSMMergePolicyFactory mergePolicyFactory, Map<String, String> mergePolicyProperties,
-            ILSMOperationTrackerProvider opTrackerFactory, ILSMIOOperationSchedulerProvider ioSchedulerProvider,
+            ILSMOperationTrackerFactory opTrackerFactory, ILSMIOOperationSchedulerProvider ioSchedulerProvider,
             ILSMIOOperationCallbackFactory ioOpCallbackFactory, double bloomFilterFalsePositiveRate,
             boolean needKeyDupCheck, ITypeTraits[] filterTypeTraits, IBinaryComparatorFactory[] filterCmpFactories,
             int[] btreeFields, int[] filterFields, boolean durable) {
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreeRangeSearchCursor.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreeRangeSearchCursor.java
index 2e85fe9..d6c12e2 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreeRangeSearchCursor.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreeRangeSearchCursor.java
@@ -24,7 +24,7 @@
 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.util.TupleUtils;
+import org.apache.hyracks.dataflow.common.utils.TupleUtils;
 import org.apache.hyracks.storage.am.btree.api.IBTreeLeafFrame;
 import org.apache.hyracks.storage.am.btree.impls.BTree;
 import org.apache.hyracks.storage.am.btree.impls.BTree.BTreeAccessor;
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/util/LSMBTreeUtils.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/utils/LSMBTreeUtil.java
similarity index 64%
rename from hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/util/LSMBTreeUtils.java
rename to hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/utils/LSMBTreeUtil.java
index 9ca96e4..0bc8b8d 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/util/LSMBTreeUtils.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/utils/LSMBTreeUtil.java
@@ -17,7 +17,7 @@
  * under the License.
  */
 
-package org.apache.hyracks.storage.am.lsm.btree.util;
+package org.apache.hyracks.storage.am.lsm.btree.utils;
 
 import java.util.List;
 
@@ -55,7 +55,11 @@
 import org.apache.hyracks.storage.common.buffercache.IBufferCache;
 import org.apache.hyracks.storage.common.file.IFileMapProvider;
 
-public class LSMBTreeUtils {
+public class LSMBTreeUtil {
+
+    private LSMBTreeUtil() {
+    }
+
     public static LSMBTree createLSMTree(IIOManager ioManager, List<IVirtualBufferCache> virtualBufferCaches,
             FileReference file, IBufferCache diskBufferCache, IFileMapProvider diskFileMapProvider,
             ITypeTraits[] typeTraits, IBinaryComparatorFactory[] cmpFactories, int[] bloomFilterKeyFields,
@@ -64,22 +68,23 @@
             ITypeTraits[] filterTypeTraits, IBinaryComparatorFactory[] filterCmpFactories, int[] btreeFields,
             int[] filterFields, boolean durable, IMetadataPageManagerFactory freePageManagerFactory)
             throws HyracksDataException {
-        LSMBTreeTupleWriterFactory insertTupleWriterFactory = new LSMBTreeTupleWriterFactory(typeTraits,
-                cmpFactories.length, false);
-        LSMBTreeTupleWriterFactory deleteTupleWriterFactory = new LSMBTreeTupleWriterFactory(typeTraits,
-                cmpFactories.length, true);
-        LSMBTreeCopyTupleWriterFactory copyTupleWriterFactory = new LSMBTreeCopyTupleWriterFactory(typeTraits,
-                cmpFactories.length);
+        LSMBTreeTupleWriterFactory insertTupleWriterFactory =
+                new LSMBTreeTupleWriterFactory(typeTraits, cmpFactories.length, false);
+        LSMBTreeTupleWriterFactory deleteTupleWriterFactory =
+                new LSMBTreeTupleWriterFactory(typeTraits, cmpFactories.length, true);
+        LSMBTreeCopyTupleWriterFactory copyTupleWriterFactory =
+                new LSMBTreeCopyTupleWriterFactory(typeTraits, cmpFactories.length);
         ITreeIndexFrameFactory insertLeafFrameFactory = new BTreeNSMLeafFrameFactory(insertTupleWriterFactory);
         ITreeIndexFrameFactory copyTupleLeafFrameFactory = new BTreeNSMLeafFrameFactory(copyTupleWriterFactory);
         ITreeIndexFrameFactory deleteLeafFrameFactory = new BTreeNSMLeafFrameFactory(deleteTupleWriterFactory);
         ITreeIndexFrameFactory interiorFrameFactory = new BTreeNSMInteriorFrameFactory(insertTupleWriterFactory);
 
-        TreeIndexFactory<BTree> diskBTreeFactory = new BTreeFactory(ioManager, diskBufferCache, diskFileMapProvider,
-                freePageManagerFactory, interiorFrameFactory, copyTupleLeafFrameFactory, cmpFactories,
-                typeTraits.length);
-        TreeIndexFactory<BTree> bulkLoadBTreeFactory = new BTreeFactory(ioManager, diskBufferCache, diskFileMapProvider,
-                freePageManagerFactory, interiorFrameFactory, insertLeafFrameFactory, cmpFactories, typeTraits.length);
+        TreeIndexFactory<BTree> diskBTreeFactory =
+                new BTreeFactory(ioManager, diskBufferCache, diskFileMapProvider, freePageManagerFactory,
+                        interiorFrameFactory, copyTupleLeafFrameFactory, cmpFactories, typeTraits.length);
+        TreeIndexFactory<BTree> bulkLoadBTreeFactory =
+                new BTreeFactory(ioManager, diskBufferCache, diskFileMapProvider, freePageManagerFactory,
+                        interiorFrameFactory, insertLeafFrameFactory, cmpFactories, typeTraits.length);
 
         BloomFilterFactory bloomFilterFactory = needKeyDupCheck
                 ? new BloomFilterFactory(diskBufferCache, diskFileMapProvider, bloomFilterKeyFields) : null;
@@ -95,9 +100,8 @@
         }
 
         //Primary LSMBTree index has a BloomFilter.
-        ILSMIndexFileManager fileNameManager = new LSMBTreeFileManager(ioManager, diskFileMapProvider, file,
-                diskBTreeFactory,
-                needKeyDupCheck);
+        ILSMIndexFileManager fileNameManager =
+                new LSMBTreeFileManager(ioManager, diskFileMapProvider, file, diskBTreeFactory, needKeyDupCheck);
 
         LSMBTree lsmTree = new LSMBTree(ioManager, virtualBufferCaches, interiorFrameFactory, insertLeafFrameFactory,
                 deleteLeafFrameFactory, fileNameManager, diskBTreeFactory, bulkLoadBTreeFactory, bloomFilterFactory,
@@ -113,104 +117,102 @@
             ILSMMergePolicy mergePolicy, ILSMOperationTracker opTracker, ILSMIOOperationScheduler ioScheduler,
             ILSMIOOperationCallback ioOpCallback, int startWithVersion, boolean durable,
             IMetadataPageManagerFactory freePageManagerFactory) {
-        LSMBTreeTupleWriterFactory insertTupleWriterFactory = new LSMBTreeTupleWriterFactory(typeTraits,
-                cmpFactories.length, false);
-        LSMBTreeTupleWriterFactory deleteTupleWriterFactory = new LSMBTreeTupleWriterFactory(typeTraits,
-                cmpFactories.length, true);
-        LSMBTreeCopyTupleWriterFactory copyTupleWriterFactory = new LSMBTreeCopyTupleWriterFactory(typeTraits,
-                cmpFactories.length);
+        LSMBTreeTupleWriterFactory insertTupleWriterFactory =
+                new LSMBTreeTupleWriterFactory(typeTraits, cmpFactories.length, false);
+        LSMBTreeTupleWriterFactory deleteTupleWriterFactory =
+                new LSMBTreeTupleWriterFactory(typeTraits, cmpFactories.length, true);
+        LSMBTreeCopyTupleWriterFactory copyTupleWriterFactory =
+                new LSMBTreeCopyTupleWriterFactory(typeTraits, cmpFactories.length);
         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
-        LSMBTreeRefrencingTupleWriterFactory referencingTupleWriterFactory = new LSMBTreeRefrencingTupleWriterFactory(
-                typeTraits, cmpFactories.length, false);
+        LSMBTreeRefrencingTupleWriterFactory referencingTupleWriterFactory =
+                new LSMBTreeRefrencingTupleWriterFactory(typeTraits, cmpFactories.length, false);
         // This is the leaf frame factory for transaction components since it
         // can be used for both inserts and deletes
         ITreeIndexFrameFactory dualLeafFrameFactory = new BTreeNSMLeafFrameFactory(referencingTupleWriterFactory);
 
-        TreeIndexFactory<BTree> diskBTreeFactory = new BTreeFactory(ioManager, diskBufferCache, diskFileMapProvider,
-                freePageManagerFactory, interiorFrameFactory, copyTupleLeafFrameFactory, cmpFactories,
-                typeTraits.length);
-        TreeIndexFactory<BTree> bulkLoadBTreeFactory = new BTreeFactory(ioManager, diskBufferCache, diskFileMapProvider,
-                freePageManagerFactory, interiorFrameFactory, insertLeafFrameFactory, cmpFactories, typeTraits.length);
+        TreeIndexFactory<BTree> diskBTreeFactory =
+                new BTreeFactory(ioManager, diskBufferCache, diskFileMapProvider, freePageManagerFactory,
+                        interiorFrameFactory, copyTupleLeafFrameFactory, cmpFactories, typeTraits.length);
+        TreeIndexFactory<BTree> bulkLoadBTreeFactory =
+                new BTreeFactory(ioManager, diskBufferCache, diskFileMapProvider, freePageManagerFactory,
+                        interiorFrameFactory, insertLeafFrameFactory, cmpFactories, typeTraits.length);
 
-        BloomFilterFactory bloomFilterFactory = new BloomFilterFactory(diskBufferCache, diskFileMapProvider,
-                bloomFilterKeyFields);
+        BloomFilterFactory bloomFilterFactory =
+                new BloomFilterFactory(diskBufferCache, diskFileMapProvider, bloomFilterKeyFields);
 
         // This is the component factory for transactions
-        TreeIndexFactory<BTree> transactionBTreeFactory = new BTreeFactory(ioManager, diskBufferCache,
-                diskFileMapProvider,
-                freePageManagerFactory, interiorFrameFactory, dualLeafFrameFactory, cmpFactories, typeTraits.length);
-        //TODO remove BloomFilter from external dataset's secondary LSMBTree index 
-        ILSMIndexFileManager fileNameManager = new LSMBTreeFileManager(ioManager, diskFileMapProvider, file,
-                diskBTreeFactory,
-                true);
+        TreeIndexFactory<BTree> transactionBTreeFactory =
+                new BTreeFactory(ioManager, diskBufferCache, diskFileMapProvider, freePageManagerFactory,
+                        interiorFrameFactory, dualLeafFrameFactory, cmpFactories, typeTraits.length);
+        //TODO remove BloomFilter from external dataset's secondary LSMBTree index
+        ILSMIndexFileManager fileNameManager =
+                new LSMBTreeFileManager(ioManager, diskFileMapProvider, file, diskBTreeFactory, true);
         // the disk only index uses an empty ArrayList for virtual buffer caches
         ExternalBTree lsmTree = new ExternalBTree(ioManager, interiorFrameFactory, insertLeafFrameFactory,
-                deleteLeafFrameFactory,
-                fileNameManager, diskBTreeFactory, bulkLoadBTreeFactory, bloomFilterFactory,
+                deleteLeafFrameFactory, fileNameManager, diskBTreeFactory, bulkLoadBTreeFactory, bloomFilterFactory,
                 bloomFilterFalsePositiveRate, diskFileMapProvider, typeTraits.length, cmpFactories, mergePolicy,
                 opTracker, ioScheduler, ioOpCallback, transactionBTreeFactory, startWithVersion, durable);
         return lsmTree;
     }
 
     public static ExternalBTreeWithBuddy createExternalBTreeWithBuddy(IIOManager ioManager, FileReference file,
-            IBufferCache diskBufferCache,
-            IFileMapProvider diskFileMapProvider, ITypeTraits[] typeTraits, IBinaryComparatorFactory[] cmpFactories,
-            double bloomFilterFalsePositiveRate, ILSMMergePolicy mergePolicy, ILSMOperationTracker opTracker,
-            ILSMIOOperationScheduler ioScheduler, ILSMIOOperationCallback ioOpCallback, int[] buddyBTreeFields,
-            int startWithVersion, boolean durable, IMetadataPageManagerFactory freePageManagerFactory) {
+            IBufferCache diskBufferCache, IFileMapProvider diskFileMapProvider, ITypeTraits[] typeTraits,
+            IBinaryComparatorFactory[] cmpFactories, double bloomFilterFalsePositiveRate, ILSMMergePolicy mergePolicy,
+            ILSMOperationTracker opTracker, ILSMIOOperationScheduler ioScheduler, ILSMIOOperationCallback ioOpCallback,
+            int[] buddyBTreeFields, int startWithVersion, boolean durable,
+            IMetadataPageManagerFactory freePageManagerFactory) {
         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]];
         }
-        TypeAwareTupleWriterFactory buddyBtreeTupleWriterFactory = new TypeAwareTupleWriterFactory(
-                buddyBtreeTypeTraits);
-        ITreeIndexFrameFactory buddyBtreeInteriorFrameFactory = new BTreeNSMInteriorFrameFactory(
-                buddyBtreeTupleWriterFactory);
+        TypeAwareTupleWriterFactory buddyBtreeTupleWriterFactory =
+                new TypeAwareTupleWriterFactory(buddyBtreeTypeTraits);
+        ITreeIndexFrameFactory buddyBtreeInteriorFrameFactory =
+                new BTreeNSMInteriorFrameFactory(buddyBtreeTupleWriterFactory);
         ITreeIndexFrameFactory buddyBtreeLeafFrameFactory = new BTreeNSMLeafFrameFactory(buddyBtreeTupleWriterFactory);
 
-        LSMBTreeTupleWriterFactory insertTupleWriterFactory = new LSMBTreeTupleWriterFactory(typeTraits,
-                cmpFactories.length, false);
-        LSMBTreeCopyTupleWriterFactory copyTupleWriterFactory = new LSMBTreeCopyTupleWriterFactory(typeTraits,
-                cmpFactories.length);
+        LSMBTreeTupleWriterFactory insertTupleWriterFactory =
+                new LSMBTreeTupleWriterFactory(typeTraits, cmpFactories.length, false);
+        LSMBTreeCopyTupleWriterFactory copyTupleWriterFactory =
+                new LSMBTreeCopyTupleWriterFactory(typeTraits, cmpFactories.length);
         ITreeIndexFrameFactory insertLeafFrameFactory = new BTreeNSMLeafFrameFactory(insertTupleWriterFactory);
         ITreeIndexFrameFactory copyTupleLeafFrameFactory = new BTreeNSMLeafFrameFactory(copyTupleWriterFactory);
         ITreeIndexFrameFactory interiorFrameFactory = new BTreeNSMInteriorFrameFactory(insertTupleWriterFactory);
-        TreeIndexFactory<BTree> diskBTreeFactory = new BTreeFactory(ioManager, diskBufferCache, diskFileMapProvider,
-                freePageManagerFactory, interiorFrameFactory, copyTupleLeafFrameFactory, cmpFactories,
-                typeTraits.length);
+        TreeIndexFactory<BTree> diskBTreeFactory =
+                new BTreeFactory(ioManager, diskBufferCache, diskFileMapProvider, freePageManagerFactory,
+                        interiorFrameFactory, copyTupleLeafFrameFactory, cmpFactories, typeTraits.length);
 
-        TreeIndexFactory<BTree> bulkLoadBTreeFactory = new BTreeFactory(ioManager, diskBufferCache, diskFileMapProvider,
-                freePageManagerFactory, interiorFrameFactory, insertLeafFrameFactory, cmpFactories, typeTraits.length);
+        TreeIndexFactory<BTree> bulkLoadBTreeFactory =
+                new BTreeFactory(ioManager, diskBufferCache, diskFileMapProvider, 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, diskFileMapProvider,
-                bloomFilterKeyFields);
+        BloomFilterFactory bloomFilterFactory =
+                new BloomFilterFactory(diskBufferCache, diskFileMapProvider, bloomFilterKeyFields);
 
         // buddy b-tree factory
         TreeIndexFactory<BTree> diskBuddyBTreeFactory = new BTreeFactory(ioManager, diskBufferCache,
-                diskFileMapProvider,
-                freePageManagerFactory, buddyBtreeInteriorFrameFactory, buddyBtreeLeafFrameFactory,
-                buddyBtreeCmpFactories, buddyBtreeTypeTraits.length);
+                diskFileMapProvider, freePageManagerFactory, buddyBtreeInteriorFrameFactory,
+                buddyBtreeLeafFrameFactory, buddyBtreeCmpFactories, buddyBtreeTypeTraits.length);
 
         ILSMIndexFileManager fileNameManager = new LSMBTreeWithBuddyFileManager(ioManager, diskFileMapProvider, file,
                 diskBTreeFactory, diskBuddyBTreeFactory);
 
         // the disk only index uses an empty ArrayList for virtual buffer caches
         ExternalBTreeWithBuddy lsmTree = new ExternalBTreeWithBuddy(ioManager, interiorFrameFactory,
-                insertLeafFrameFactory,
-                buddyBtreeLeafFrameFactory, diskBufferCache, fileNameManager, bulkLoadBTreeFactory, diskBTreeFactory,
-                diskBuddyBTreeFactory, bloomFilterFactory, diskFileMapProvider, bloomFilterFalsePositiveRate,
-                mergePolicy, opTracker, ioScheduler, ioOpCallback, cmpFactories, buddyBtreeCmpFactories,
-                buddyBTreeFields, startWithVersion, durable);
+                insertLeafFrameFactory, buddyBtreeLeafFrameFactory, diskBufferCache, fileNameManager,
+                bulkLoadBTreeFactory, diskBTreeFactory, diskBuddyBTreeFactory, bloomFilterFactory, diskFileMapProvider,
+                bloomFilterFalsePositiveRate, mergePolicy, opTracker, ioScheduler, ioOpCallback, cmpFactories,
+                buddyBtreeCmpFactories, buddyBTreeFields, startWithVersion, durable);
         return lsmTree;
     }
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/ILSMIOOperationCallbackFactory.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/ILSMIOOperationCallbackFactory.java
index cfb795c..c91fa8b 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/ILSMIOOperationCallbackFactory.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/ILSMIOOperationCallbackFactory.java
@@ -20,6 +20,7 @@
 
 import java.io.Serializable;
 
+@FunctionalInterface
 public interface ILSMIOOperationCallbackFactory extends Serializable {
-    public ILSMIOOperationCallback createIOOperationCallback();
+    ILSMIOOperationCallback createIOOperationCallback();
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/ILSMIOOperationScheduler.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/ILSMIOOperationScheduler.java
index a808e70..57a3483 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/ILSMIOOperationScheduler.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/ILSMIOOperationScheduler.java
@@ -20,6 +20,10 @@
 
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 
+/**
+ * Schedules IO operations for LSM indexes
+ */
+@FunctionalInterface
 public interface ILSMIOOperationScheduler {
-    public void scheduleOperation(ILSMIOOperation operation) throws HyracksDataException;
+    void scheduleOperation(ILSMIOOperation operation) throws HyracksDataException;
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/ILSMIOOperationSchedulerProvider.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/ILSMIOOperationSchedulerProvider.java
index bf10a91..9a7ce57 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/ILSMIOOperationSchedulerProvider.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/ILSMIOOperationSchedulerProvider.java
@@ -22,6 +22,10 @@
 
 import org.apache.hyracks.api.context.IHyracksTaskContext;
 
+/**
+ * Provides the {@link ILSMIOOperationScheduler} for the application
+ */
+@FunctionalInterface
 public interface ILSMIOOperationSchedulerProvider extends Serializable {
-    public ILSMIOOperationScheduler getIOScheduler(IHyracksTaskContext ctx);
+    ILSMIOOperationScheduler getIOScheduler(IHyracksTaskContext ctx);
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/ILSMOperationTrackerProvider.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/ILSMOperationTrackerFactory.java
similarity index 79%
rename from hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/ILSMOperationTrackerProvider.java
rename to hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/ILSMOperationTrackerFactory.java
index 403b7b1..133d450 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/ILSMOperationTrackerProvider.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/ILSMOperationTrackerFactory.java
@@ -20,8 +20,9 @@
 
 import java.io.Serializable;
 
-import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.application.INCApplicationContext;
 
-public interface ILSMOperationTrackerProvider extends Serializable {
-    public ILSMOperationTracker getOperationTracker(IHyracksTaskContext ctx);
+@FunctionalInterface
+public interface ILSMOperationTrackerFactory extends Serializable {
+    ILSMOperationTracker getOperationTracker(INCApplicationContext ctx);
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/dataflow/AbstractLSMIndexDataflowHelper.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/dataflow/AbstractLSMIndexDataflowHelper.java
index fa5b623..f219c12 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/dataflow/AbstractLSMIndexDataflowHelper.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/dataflow/AbstractLSMIndexDataflowHelper.java
@@ -30,7 +30,7 @@
 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.ILSMMergePolicy;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMOperationTrackerProvider;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMOperationTrackerFactory;
 import org.apache.hyracks.storage.am.lsm.common.api.IVirtualBufferCache;
 
 public abstract class AbstractLSMIndexDataflowHelper extends IndexDataflowHelper {
@@ -42,7 +42,7 @@
     protected final List<IVirtualBufferCache> virtualBufferCaches;
     protected final ILSMMergePolicy mergePolicy;
     protected final ILSMIOOperationScheduler ioScheduler;
-    protected final ILSMOperationTrackerProvider opTrackerFactory;
+    protected final ILSMOperationTrackerFactory opTrackerFactory;
     protected final ILSMIOOperationCallbackFactory ioOpCallbackFactory;
     protected final ITypeTraits[] filterTypeTraits;
     protected final IBinaryComparatorFactory[] filterCmpFactories;
@@ -50,7 +50,7 @@
 
     public AbstractLSMIndexDataflowHelper(IIndexOperatorDescriptor opDesc, IHyracksTaskContext ctx, int partition,
             List<IVirtualBufferCache> virtualBufferCaches, ILSMMergePolicy mergePolicy,
-            ILSMOperationTrackerProvider opTrackerFactory, ILSMIOOperationScheduler ioScheduler,
+            ILSMOperationTrackerFactory opTrackerFactory, ILSMIOOperationScheduler ioScheduler,
             ILSMIOOperationCallbackFactory ioOpCallbackFactory, ITypeTraits[] filterTypeTraits,
             IBinaryComparatorFactory[] filterCmpFactories, int[] filterFields, boolean durable)
             throws HyracksDataException {
@@ -61,7 +61,7 @@
 
     public AbstractLSMIndexDataflowHelper(IIndexOperatorDescriptor opDesc, IHyracksTaskContext ctx, int partition,
             List<IVirtualBufferCache> virtualBufferCaches, double bloomFilterFalsePositiveRate,
-            ILSMMergePolicy mergePolicy, ILSMOperationTrackerProvider opTrackerFactory,
+            ILSMMergePolicy mergePolicy, ILSMOperationTrackerFactory opTrackerFactory,
             ILSMIOOperationScheduler ioScheduler, ILSMIOOperationCallbackFactory ioOpCallbackFactory,
             ITypeTraits[] filterTypeTraits, IBinaryComparatorFactory[] filterCmpFactories, int[] filterFields,
             boolean durable) throws HyracksDataException {
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/dataflow/AbstractLSMIndexDataflowHelperFactory.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/dataflow/AbstractLSMIndexDataflowHelperFactory.java
index a0affa3..a95574d 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/dataflow/AbstractLSMIndexDataflowHelperFactory.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/dataflow/AbstractLSMIndexDataflowHelperFactory.java
@@ -27,7 +27,7 @@
 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.ILSMOperationTrackerProvider;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMOperationTrackerFactory;
 import org.apache.hyracks.storage.am.lsm.common.api.IVirtualBufferCacheProvider;
 
 public abstract class AbstractLSMIndexDataflowHelperFactory implements IIndexDataflowHelperFactory {
@@ -36,7 +36,7 @@
     protected final IVirtualBufferCacheProvider virtualBufferCacheProvider;
     protected final ILSMMergePolicyFactory mergePolicyFactory;
     protected final Map<String, String> mergePolicyProperties;
-    protected final ILSMOperationTrackerProvider opTrackerFactory;
+    protected final ILSMOperationTrackerFactory opTrackerFactory;
     protected final ILSMIOOperationSchedulerProvider ioSchedulerProvider;
     protected final ILSMIOOperationCallbackFactory ioOpCallbackFactory;
     protected final double bloomFilterFalsePositiveRate;
@@ -47,7 +47,7 @@
 
     public AbstractLSMIndexDataflowHelperFactory(IVirtualBufferCacheProvider virtualBufferCacheProvider,
             ILSMMergePolicyFactory mergePolicyFactory, Map<String, String> mergePolicyProperties,
-            ILSMOperationTrackerProvider opTrackerFactory, ILSMIOOperationSchedulerProvider ioSchedulerProvider,
+            ILSMOperationTrackerFactory opTrackerFactory, ILSMIOOperationSchedulerProvider ioSchedulerProvider,
             ILSMIOOperationCallbackFactory ioOpCallbackFactory, double bloomFilterFalsePositiveRate,
             ITypeTraits[] filterTypeTraits, IBinaryComparatorFactory[] filterCmpFactories, int[] filterFields,
             boolean durable) {
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/dataflow/LSMTreeIndexCompactOperatorDescriptor.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/dataflow/LSMTreeIndexCompactOperatorDescriptor.java
index f009db9..9311599 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/dataflow/LSMTreeIndexCompactOperatorDescriptor.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/dataflow/LSMTreeIndexCompactOperatorDescriptor.java
@@ -33,7 +33,7 @@
 import org.apache.hyracks.storage.am.common.dataflow.AbstractTreeIndexOperatorDescriptor;
 import org.apache.hyracks.storage.am.common.dataflow.IIndexDataflowHelperFactory;
 import org.apache.hyracks.storage.am.common.impls.NoOpOperationCallbackFactory;
-import org.apache.hyracks.storage.common.IStorageManagerInterface;
+import org.apache.hyracks.storage.common.IStorageManager;
 import org.apache.hyracks.storage.common.file.NoOpLocalResourceFactoryProvider;
 
 public class LSMTreeIndexCompactOperatorDescriptor extends AbstractTreeIndexOperatorDescriptor {
@@ -41,7 +41,7 @@
     private static final long serialVersionUID = 1L;
 
     public LSMTreeIndexCompactOperatorDescriptor(IOperatorDescriptorRegistry spec,
-            IStorageManagerInterface storageManager, IIndexLifecycleManagerProvider lifecycleManagerProvider,
+            IStorageManager storageManager, IIndexLifecycleManagerProvider lifecycleManagerProvider,
             IFileSplitProvider fileSplitProvider, ITypeTraits[] typeTraits,
             IBinaryComparatorFactory[] comparatorFactories, int[] bloomFilterKeyFields,
             IIndexDataflowHelperFactory dataflowHelperFactory,
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/dataflow/LSMTreeIndexInsertUpdateDeleteOperatorDescriptor.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/dataflow/LSMTreeIndexInsertUpdateDeleteOperatorDescriptor.java
index 5403e79..f0a5352 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/dataflow/LSMTreeIndexInsertUpdateDeleteOperatorDescriptor.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/dataflow/LSMTreeIndexInsertUpdateDeleteOperatorDescriptor.java
@@ -37,7 +37,7 @@
 import org.apache.hyracks.storage.am.common.dataflow.AbstractTreeIndexOperatorDescriptor;
 import org.apache.hyracks.storage.am.common.dataflow.IIndexDataflowHelperFactory;
 import org.apache.hyracks.storage.am.common.ophelpers.IndexOperation;
-import org.apache.hyracks.storage.common.IStorageManagerInterface;
+import org.apache.hyracks.storage.common.IStorageManager;
 import org.apache.hyracks.storage.common.file.NoOpLocalResourceFactoryProvider;
 
 public class LSMTreeIndexInsertUpdateDeleteOperatorDescriptor extends AbstractTreeIndexOperatorDescriptor {
@@ -48,7 +48,7 @@
     protected final IndexOperation op;
 
     public LSMTreeIndexInsertUpdateDeleteOperatorDescriptor(IOperatorDescriptorRegistry spec, RecordDescriptor recDesc,
-            IStorageManagerInterface storageManager, IIndexLifecycleManagerProvider lifecycleManagerProvider,
+            IStorageManager storageManager, IIndexLifecycleManagerProvider lifecycleManagerProvider,
             IFileSplitProvider fileSplitProvider, ITypeTraits[] typeTraits,
             IBinaryComparatorFactory[] comparatorFactories, int[] bloomFilterKeyFields, int[] fieldPermutation,
             IndexOperation op, IIndexDataflowHelperFactory dataflowHelperFactory,
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/NoOpOperationTrackerFactory.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/NoOpOperationTrackerFactory.java
new file mode 100644
index 0000000..97e6978
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/NoOpOperationTrackerFactory.java
@@ -0,0 +1,70 @@
+/*
+ * 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 org.apache.hyracks.api.application.INCApplicationContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.storage.am.common.api.IModificationOperationCallback;
+import org.apache.hyracks.storage.am.common.api.ISearchOperationCallback;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndex;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMOperationTracker;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMOperationTrackerFactory;
+import org.apache.hyracks.storage.am.lsm.common.api.LSMOperationType;
+
+/**
+ * Operation tracker that does nothing.
+ * WARNING: This op tracker should only be used for specific testing purposes.
+ * It is assumed than an op tracker cooperates with an lsm index to synchronize flushes with
+ * regular operations, and this implementation does no such tracking at all.
+ */
+public class NoOpOperationTrackerFactory implements ILSMOperationTrackerFactory {
+    private static final long serialVersionUID = 1L;
+    public static final NoOpOperationTrackerFactory INSTANCE = new NoOpOperationTrackerFactory();
+    private static final NoOpOperationTracker tracker = new NoOpOperationTracker();
+
+    // Enforce singleton.
+    private NoOpOperationTrackerFactory() {
+    }
+
+    @Override
+    public ILSMOperationTracker getOperationTracker(INCApplicationContext ctx) {
+        return tracker;
+    }
+
+    private static final class NoOpOperationTracker implements ILSMOperationTracker {
+
+        @Override
+        public void beforeOperation(ILSMIndex index, LSMOperationType opType, ISearchOperationCallback searchCallback,
+                IModificationOperationCallback modificationCallback) throws HyracksDataException {
+            // No Op
+        }
+
+        @Override
+        public void afterOperation(ILSMIndex index, LSMOperationType opType, ISearchOperationCallback searchCallback,
+                IModificationOperationCallback modificationCallback) throws HyracksDataException {
+            // No Op
+        }
+
+        @Override
+        public void completeOperation(ILSMIndex index, LSMOperationType opType, ISearchOperationCallback searchCallback,
+                IModificationOperationCallback modificationCallback) throws HyracksDataException {
+            // No Op
+        }
+    }
+}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/NoOpOperationTrackerProvider.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/NoOpOperationTrackerProvider.java
deleted file mode 100644
index fd7c1f2..0000000
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/NoOpOperationTrackerProvider.java
+++ /dev/null
@@ -1,71 +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 org.apache.hyracks.api.context.IHyracksTaskContext;
-import org.apache.hyracks.api.exceptions.HyracksDataException;
-import org.apache.hyracks.storage.am.common.api.IModificationOperationCallback;
-import org.apache.hyracks.storage.am.common.api.ISearchOperationCallback;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndex;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMOperationTracker;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMOperationTrackerProvider;
-import org.apache.hyracks.storage.am.lsm.common.api.LSMOperationType;
-
-/**
- * Operation tracker that does nothing.
- * WARNING: This op tracker should only be used for specific testing purposes.
- * It is assumed than an op tracker cooperates with an lsm index to synchronize flushes with
- * regular operations, and this implementation does no such tracking at all.
- */
-public class NoOpOperationTrackerProvider implements ILSMOperationTrackerProvider {
-    private static final long serialVersionUID = 1L;
-
-    public static NoOpOperationTrackerProvider INSTANCE = new NoOpOperationTrackerProvider();
-
-    @Override
-    public ILSMOperationTracker getOperationTracker(IHyracksTaskContext ctx) {
-        return new ILSMOperationTracker() {
-
-            @Override
-            public void completeOperation(ILSMIndex index, LSMOperationType opType,
-                    ISearchOperationCallback searchCallback, IModificationOperationCallback modificationCallback)
-                    throws HyracksDataException {
-                // Do nothing.
-            }
-
-            @Override
-            public void beforeOperation(ILSMIndex index, LSMOperationType opType,
-                    ISearchOperationCallback searchCallback, IModificationOperationCallback modificationCallback)
-                    throws HyracksDataException {
-            }
-
-            @Override
-            public void afterOperation(ILSMIndex index, LSMOperationType opType,
-                    ISearchOperationCallback searchCallback, IModificationOperationCallback modificationCallback)
-                    throws HyracksDataException {
-                // Do nothing.
-            }
-        };
-    }
-
-    // Enforce singleton.
-    private NoOpOperationTrackerProvider() {
-    }
-
-};
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/ThreadCountingOperationTrackerProvider.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/ThreadCountingOperationTrackerFactory.java
similarity index 72%
rename from hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/ThreadCountingOperationTrackerProvider.java
rename to hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/ThreadCountingOperationTrackerFactory.java
index f17d1bc..a9ccc42 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/ThreadCountingOperationTrackerProvider.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/ThreadCountingOperationTrackerFactory.java
@@ -18,22 +18,21 @@
  */
 package org.apache.hyracks.storage.am.lsm.common.impls;
 
-import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.application.INCApplicationContext;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMOperationTracker;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMOperationTrackerProvider;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMOperationTrackerFactory;
 
-public class ThreadCountingOperationTrackerProvider implements ILSMOperationTrackerProvider {
+public class ThreadCountingOperationTrackerFactory implements ILSMOperationTrackerFactory {
 
     private static final long serialVersionUID = 1L;
-
-    public static ThreadCountingOperationTrackerProvider INSTANCE = new ThreadCountingOperationTrackerProvider();
-
-    @Override
-    public ILSMOperationTracker getOperationTracker(IHyracksTaskContext ctx) {
-        return new ThreadCountingTracker();
-    }
+    public static final ThreadCountingOperationTrackerFactory INSTANCE = new ThreadCountingOperationTrackerFactory();
 
     // Enforce singleton.
-    private ThreadCountingOperationTrackerProvider() {
+    private ThreadCountingOperationTrackerFactory() {
+    }
+
+    @Override
+    public ILSMOperationTracker getOperationTracker(INCApplicationContext ctx) {
+        return new ThreadCountingTracker();
     }
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/dataflow/AbstractLSMInvertedIndexOperatorDescriptor.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/dataflow/AbstractLSMInvertedIndexOperatorDescriptor.java
index 84a3267..a39d8e9 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/dataflow/AbstractLSMInvertedIndexOperatorDescriptor.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/dataflow/AbstractLSMInvertedIndexOperatorDescriptor.java
@@ -34,7 +34,7 @@
 import org.apache.hyracks.storage.am.common.dataflow.IIndexDataflowHelperFactory;
 import org.apache.hyracks.storage.am.lsm.invertedindex.api.IInvertedIndexOperatorDescriptor;
 import org.apache.hyracks.storage.am.lsm.invertedindex.tokenizers.IBinaryTokenizerFactory;
-import org.apache.hyracks.storage.common.IStorageManagerInterface;
+import org.apache.hyracks.storage.common.IStorageManager;
 import org.apache.hyracks.storage.common.file.ILocalResourceFactoryProvider;
 
 public abstract class AbstractLSMInvertedIndexOperatorDescriptor extends AbstractIndexOperatorDescriptor implements
@@ -49,7 +49,7 @@
     protected final IBinaryTokenizerFactory tokenizerFactory;
 
     public AbstractLSMInvertedIndexOperatorDescriptor(IOperatorDescriptorRegistry spec, int inputArity,
-            int outputArity, RecordDescriptor recDesc, IStorageManagerInterface storageManager,
+            int outputArity, RecordDescriptor recDesc, IStorageManager storageManager,
             IFileSplitProvider fileSplitProvider, IIndexLifecycleManagerProvider lifecycleManagerProvider,
             ITypeTraits[] tokenTypeTraits, IBinaryComparatorFactory[] tokenComparatorFactories,
             ITypeTraits[] invListsTypeTraits, IBinaryComparatorFactory[] invListComparatorFactories,
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/dataflow/LSMInvertedIndexBulkLoadOperatorDescriptor.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/dataflow/LSMInvertedIndexBulkLoadOperatorDescriptor.java
index 163439f..121c121 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/dataflow/LSMInvertedIndexBulkLoadOperatorDescriptor.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/dataflow/LSMInvertedIndexBulkLoadOperatorDescriptor.java
@@ -34,7 +34,7 @@
 import org.apache.hyracks.storage.am.common.dataflow.IndexBulkLoadOperatorNodePushable;
 import org.apache.hyracks.storage.am.common.impls.NoOpOperationCallbackFactory;
 import org.apache.hyracks.storage.am.lsm.invertedindex.tokenizers.IBinaryTokenizerFactory;
-import org.apache.hyracks.storage.common.IStorageManagerInterface;
+import org.apache.hyracks.storage.common.IStorageManager;
 import org.apache.hyracks.storage.common.file.NoOpLocalResourceFactoryProvider;
 
 public class LSMInvertedIndexBulkLoadOperatorDescriptor extends AbstractLSMInvertedIndexOperatorDescriptor {
@@ -48,7 +48,7 @@
 
     public LSMInvertedIndexBulkLoadOperatorDescriptor(IOperatorDescriptorRegistry spec, RecordDescriptor recDesc,
             int[] fieldPermutation, boolean verifyInput, long numElementsHint, boolean checkIfEmptyIndex,
-            IStorageManagerInterface storageManager, IFileSplitProvider fileSplitProvider,
+            IStorageManager storageManager, IFileSplitProvider fileSplitProvider,
             IIndexLifecycleManagerProvider lifecycleManagerProvider, ITypeTraits[] tokenTypeTraits,
             IBinaryComparatorFactory[] tokenComparatorFactories, ITypeTraits[] invListsTypeTraits,
             IBinaryComparatorFactory[] invListComparatorFactories, IBinaryTokenizerFactory tokenizerFactory,
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/dataflow/LSMInvertedIndexCompactOperator.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/dataflow/LSMInvertedIndexCompactOperator.java
index def8e71..08724e9 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/dataflow/LSMInvertedIndexCompactOperator.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/dataflow/LSMInvertedIndexCompactOperator.java
@@ -34,14 +34,14 @@
 import org.apache.hyracks.storage.am.common.impls.NoOpOperationCallbackFactory;
 import org.apache.hyracks.storage.am.lsm.common.dataflow.LSMIndexCompactOperatorNodePushable;
 import org.apache.hyracks.storage.am.lsm.invertedindex.tokenizers.IBinaryTokenizerFactory;
-import org.apache.hyracks.storage.common.IStorageManagerInterface;
+import org.apache.hyracks.storage.common.IStorageManager;
 import org.apache.hyracks.storage.common.file.NoOpLocalResourceFactoryProvider;
 
 public class LSMInvertedIndexCompactOperator extends AbstractLSMInvertedIndexOperatorDescriptor {
 
     private static final long serialVersionUID = 1L;
 
-    public LSMInvertedIndexCompactOperator(IOperatorDescriptorRegistry spec, IStorageManagerInterface storageManager,
+    public LSMInvertedIndexCompactOperator(IOperatorDescriptorRegistry spec, IStorageManager storageManager,
             IFileSplitProvider fileSplitProvider, IIndexLifecycleManagerProvider lifecycleManagerProvider,
             ITypeTraits[] tokenTypeTraits, IBinaryComparatorFactory[] tokenComparatorFactories,
             ITypeTraits[] invListsTypeTraits, IBinaryComparatorFactory[] invListComparatorFactories,
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/dataflow/LSMInvertedIndexCreateOperatorDescriptor.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/dataflow/LSMInvertedIndexCreateOperatorDescriptor.java
index 1151318..7a09649 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/dataflow/LSMInvertedIndexCreateOperatorDescriptor.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/dataflow/LSMInvertedIndexCreateOperatorDescriptor.java
@@ -34,7 +34,7 @@
 import org.apache.hyracks.storage.am.common.dataflow.IndexCreateOperatorNodePushable;
 import org.apache.hyracks.storage.am.common.impls.NoOpOperationCallbackFactory;
 import org.apache.hyracks.storage.am.lsm.invertedindex.tokenizers.IBinaryTokenizerFactory;
-import org.apache.hyracks.storage.common.IStorageManagerInterface;
+import org.apache.hyracks.storage.common.IStorageManager;
 import org.apache.hyracks.storage.common.file.ILocalResourceFactoryProvider;
 
 public class LSMInvertedIndexCreateOperatorDescriptor extends AbstractLSMInvertedIndexOperatorDescriptor {
@@ -42,7 +42,7 @@
     private static final long serialVersionUID = 1L;
 
     public LSMInvertedIndexCreateOperatorDescriptor(IOperatorDescriptorRegistry spec,
-            IStorageManagerInterface storageManager, IFileSplitProvider fileSplitProvider,
+            IStorageManager storageManager, IFileSplitProvider fileSplitProvider,
             IIndexLifecycleManagerProvider lifecycleManagerProvider, ITypeTraits[] tokenTypeTraits,
             IBinaryComparatorFactory[] tokenComparatorFactories, ITypeTraits[] invListsTypeTraits,
             IBinaryComparatorFactory[] invListComparatorFactories, IBinaryTokenizerFactory tokenizerFactory,
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/dataflow/LSMInvertedIndexDataflowHelper.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/dataflow/LSMInvertedIndexDataflowHelper.java
index 237b567..28cfadf 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/dataflow/LSMInvertedIndexDataflowHelper.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/dataflow/LSMInvertedIndexDataflowHelper.java
@@ -33,7 +33,7 @@
 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.ILSMMergePolicy;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMOperationTrackerProvider;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMOperationTrackerFactory;
 import org.apache.hyracks.storage.am.lsm.common.api.IVirtualBufferCache;
 import org.apache.hyracks.storage.am.lsm.common.dataflow.AbstractLSMIndexDataflowHelper;
 import org.apache.hyracks.storage.am.lsm.invertedindex.api.IInvertedIndexOperatorDescriptor;
@@ -50,7 +50,7 @@
 
     public LSMInvertedIndexDataflowHelper(IIndexOperatorDescriptor opDesc, IHyracksTaskContext ctx, int partition,
             List<IVirtualBufferCache> virtualBufferCaches, ILSMMergePolicy mergePolicy,
-            ILSMOperationTrackerProvider opTrackerFactory, ILSMIOOperationScheduler ioScheduler,
+            ILSMOperationTrackerFactory opTrackerFactory, ILSMIOOperationScheduler ioScheduler,
             ILSMIOOperationCallbackFactory ioOpCallbackFactory, int[] invertedIndexFields,
             ITypeTraits[] filterTypeTraits, IBinaryComparatorFactory[] filterCmpFactories, int[] filterFields,
             int[] filterFieldsForNonBulkLoadOps, int[] invertedIndexFieldsForNonBulkLoadOps, boolean durable)
@@ -63,7 +63,7 @@
 
     public LSMInvertedIndexDataflowHelper(IIndexOperatorDescriptor opDesc, IHyracksTaskContext ctx, int partition,
             List<IVirtualBufferCache> virtualBufferCaches, double bloomFilterFalsePositiveRate,
-            ILSMMergePolicy mergePolicy, ILSMOperationTrackerProvider opTrackerFactory,
+            ILSMMergePolicy mergePolicy, ILSMOperationTrackerFactory opTrackerFactory,
             ILSMIOOperationScheduler ioScheduler, ILSMIOOperationCallbackFactory ioOpCallbackFactory,
             int[] invertedIndexFields, ITypeTraits[] filterTypeTraits, IBinaryComparatorFactory[] filterCmpFactories,
             int[] filterFields, int[] filterFieldsForNonBulkLoadOps, int[] invertedIndexFieldsForNonBulkLoadOps,
@@ -89,7 +89,7 @@
                     invIndexOpDesc.getInvListsComparatorFactories(), invIndexOpDesc.getTokenTypeTraits(),
                     invIndexOpDesc.getTokenComparatorFactories(), invIndexOpDesc.getTokenizerFactory(),
                     diskBufferCache, fileRef.getFile().getAbsolutePath(), bloomFilterFalsePositiveRate, mergePolicy,
-                    opTrackerFactory.getOperationTracker(ctx), ioScheduler,
+                    opTrackerFactory.getOperationTracker(ctx.getJobletContext().getApplicationContext()), ioScheduler,
                     ioOpCallbackFactory.createIOOperationCallback(), invertedIndexFields, filterTypeTraits,
                     filterCmpFactories, filterFields, filterFieldsForNonBulkLoadOps,
                     invertedIndexFieldsForNonBulkLoadOps, durable, (IMetadataPageManagerFactory) opDesc
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/dataflow/LSMInvertedIndexDataflowHelperFactory.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/dataflow/LSMInvertedIndexDataflowHelperFactory.java
index f1cc153..f621bae 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/dataflow/LSMInvertedIndexDataflowHelperFactory.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/dataflow/LSMInvertedIndexDataflowHelperFactory.java
@@ -30,7 +30,7 @@
 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.ILSMOperationTrackerProvider;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMOperationTrackerFactory;
 import org.apache.hyracks.storage.am.lsm.common.api.IVirtualBufferCacheProvider;
 import org.apache.hyracks.storage.am.lsm.common.dataflow.AbstractLSMIndexDataflowHelperFactory;
 
@@ -43,7 +43,7 @@
 
     public LSMInvertedIndexDataflowHelperFactory(IVirtualBufferCacheProvider virtualBufferCacheProvider,
             ILSMMergePolicyFactory mergePolicyFactory, Map<String, String> mergePolicyProperties,
-            ILSMOperationTrackerProvider opTrackerProvider, ILSMIOOperationSchedulerProvider ioSchedulerProvider,
+            ILSMOperationTrackerFactory opTrackerProvider, ILSMIOOperationSchedulerProvider ioSchedulerProvider,
             ILSMIOOperationCallbackFactory ioOpCallbackFactory, double bloomFilterFalsePositiveRate,
             int[] invertedIndexFields, ITypeTraits[] filterTypeTraits, IBinaryComparatorFactory[] filterCmpFactories,
             int[] filterFields, int[] filterFieldsForNonBulkLoadOps, int[] invertedIndexFieldsForNonBulkLoadOps,
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/dataflow/LSMInvertedIndexInsertUpdateDeleteOperator.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/dataflow/LSMInvertedIndexInsertUpdateDeleteOperator.java
index 0ae65b7..46201d5 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/dataflow/LSMInvertedIndexInsertUpdateDeleteOperator.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/dataflow/LSMInvertedIndexInsertUpdateDeleteOperator.java
@@ -37,7 +37,7 @@
 import org.apache.hyracks.storage.am.common.ophelpers.IndexOperation;
 import org.apache.hyracks.storage.am.lsm.common.dataflow.LSMIndexInsertUpdateDeleteOperatorNodePushable;
 import org.apache.hyracks.storage.am.lsm.invertedindex.tokenizers.IBinaryTokenizerFactory;
-import org.apache.hyracks.storage.common.IStorageManagerInterface;
+import org.apache.hyracks.storage.common.IStorageManager;
 import org.apache.hyracks.storage.common.file.NoOpLocalResourceFactoryProvider;
 
 public class LSMInvertedIndexInsertUpdateDeleteOperator extends AbstractLSMInvertedIndexOperatorDescriptor {
@@ -48,7 +48,7 @@
     protected final IndexOperation op;
 
     public LSMInvertedIndexInsertUpdateDeleteOperator(IOperatorDescriptorRegistry spec, RecordDescriptor recDesc,
-            IStorageManagerInterface storageManager, IFileSplitProvider fileSplitProvider,
+            IStorageManager storageManager, IFileSplitProvider fileSplitProvider,
             IIndexLifecycleManagerProvider lifecycleManagerProvider, ITypeTraits[] tokenTypeTraits,
             IBinaryComparatorFactory[] tokenComparatorFactories, ITypeTraits[] invListsTypeTraits,
             IBinaryComparatorFactory[] invListComparatorFactories, IBinaryTokenizerFactory tokenizerFactory,
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/dataflow/LSMInvertedIndexSearchOperatorDescriptor.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/dataflow/LSMInvertedIndexSearchOperatorDescriptor.java
index 7c21c38..f628e76 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/dataflow/LSMInvertedIndexSearchOperatorDescriptor.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/dataflow/LSMInvertedIndexSearchOperatorDescriptor.java
@@ -37,7 +37,7 @@
 import org.apache.hyracks.storage.am.lsm.invertedindex.api.IInvertedIndexSearchModifier;
 import org.apache.hyracks.storage.am.lsm.invertedindex.api.IInvertedIndexSearchModifierFactory;
 import org.apache.hyracks.storage.am.lsm.invertedindex.tokenizers.IBinaryTokenizerFactory;
-import org.apache.hyracks.storage.common.IStorageManagerInterface;
+import org.apache.hyracks.storage.common.IStorageManager;
 import org.apache.hyracks.storage.common.file.NoOpLocalResourceFactoryProvider;
 
 public class LSMInvertedIndexSearchOperatorDescriptor extends AbstractLSMInvertedIndexOperatorDescriptor {
@@ -50,7 +50,7 @@
     private final boolean isFullTextSearchQuery;
 
     public LSMInvertedIndexSearchOperatorDescriptor(IOperatorDescriptorRegistry spec, int queryField,
-            IStorageManagerInterface storageManager, IFileSplitProvider fileSplitProvider,
+            IStorageManager storageManager, IFileSplitProvider fileSplitProvider,
             IIndexLifecycleManagerProvider lifecycleManagerProvider, ITypeTraits[] tokenTypeTraits,
             IBinaryComparatorFactory[] tokenComparatorFactories, ITypeTraits[] invListsTypeTraits,
             IBinaryComparatorFactory[] invListComparatorFactories,
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/dataflow/PartitionedLSMInvertedIndexDataflowHelper.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/dataflow/PartitionedLSMInvertedIndexDataflowHelper.java
index 3e8b634..5138642 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/dataflow/PartitionedLSMInvertedIndexDataflowHelper.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/dataflow/PartitionedLSMInvertedIndexDataflowHelper.java
@@ -32,7 +32,7 @@
 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.ILSMMergePolicy;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMOperationTrackerProvider;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMOperationTrackerFactory;
 import org.apache.hyracks.storage.am.lsm.common.api.IVirtualBufferCache;
 import org.apache.hyracks.storage.am.lsm.common.dataflow.AbstractLSMIndexDataflowHelper;
 import org.apache.hyracks.storage.am.lsm.invertedindex.api.IInvertedIndexOperatorDescriptor;
@@ -49,7 +49,7 @@
 
     public PartitionedLSMInvertedIndexDataflowHelper(IIndexOperatorDescriptor opDesc, IHyracksTaskContext ctx,
             int partition, List<IVirtualBufferCache> virtualBufferCache, ILSMMergePolicy mergePolicy,
-            ILSMOperationTrackerProvider opTrackerFactory, ILSMIOOperationScheduler ioScheduler,
+            ILSMOperationTrackerFactory opTrackerFactory, ILSMIOOperationScheduler ioScheduler,
             ILSMIOOperationCallbackFactory ioOpCallbackFactory, int[] invertedIndexFields,
             ITypeTraits[] filterTypeTraits, IBinaryComparatorFactory[] filterCmpFactories, int[] filterFields,
             int[] filterFieldsForNonBulkLoadOps, int[] invertedIndexFieldsForNonBulkLoadOps, boolean durable)
@@ -62,7 +62,7 @@
 
     public PartitionedLSMInvertedIndexDataflowHelper(IIndexOperatorDescriptor opDesc, IHyracksTaskContext ctx,
             int partition, List<IVirtualBufferCache> virtualBufferCaches, double bloomFilterFalsePositiveRate,
-            ILSMMergePolicy mergePolicy, ILSMOperationTrackerProvider opTrackerFactory,
+            ILSMMergePolicy mergePolicy, ILSMOperationTrackerFactory opTrackerFactory,
             ILSMIOOperationScheduler ioScheduler, ILSMIOOperationCallbackFactory ioOpCallbackFactory,
             int[] invertedIndexFields, ITypeTraits[] filterTypeTraits, IBinaryComparatorFactory[] filterCmpFactories,
             int[] filterFields, int[] filterFieldsForNonBulkLoadOps, int[] invertedIndexFieldsForNonBulkLoadOps,
@@ -88,7 +88,7 @@
                     invIndexOpDesc.getInvListsComparatorFactories(), invIndexOpDesc.getTokenTypeTraits(),
                     invIndexOpDesc.getTokenComparatorFactories(), invIndexOpDesc.getTokenizerFactory(),
                     diskBufferCache, fileRef.getFile().getAbsolutePath(), bloomFilterFalsePositiveRate, mergePolicy,
-                    opTrackerFactory.getOperationTracker(ctx), ioScheduler,
+                    opTrackerFactory.getOperationTracker(ctx.getJobletContext().getApplicationContext()), ioScheduler,
                     ioOpCallbackFactory.createIOOperationCallback(), invertedIndexFields, filterTypeTraits,
                     filterCmpFactories, filterFields, filterFieldsForNonBulkLoadOps,
                     invertedIndexFieldsForNonBulkLoadOps, durable, opDesc.getPageManagerFactory());
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/dataflow/PartitionedLSMInvertedIndexDataflowHelperFactory.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/dataflow/PartitionedLSMInvertedIndexDataflowHelperFactory.java
index bc1fc5b..8f4f2d9 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/dataflow/PartitionedLSMInvertedIndexDataflowHelperFactory.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/dataflow/PartitionedLSMInvertedIndexDataflowHelperFactory.java
@@ -30,7 +30,7 @@
 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.ILSMOperationTrackerProvider;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMOperationTrackerFactory;
 import org.apache.hyracks.storage.am.lsm.common.api.IVirtualBufferCacheProvider;
 import org.apache.hyracks.storage.am.lsm.common.dataflow.AbstractLSMIndexDataflowHelperFactory;
 
@@ -43,7 +43,7 @@
 
     public PartitionedLSMInvertedIndexDataflowHelperFactory(IVirtualBufferCacheProvider virtualBufferCacheProvider,
             ILSMMergePolicyFactory mergePolicyFactory, Map<String, String> mergePolicyProperties,
-            ILSMOperationTrackerProvider opTrackerProvider, ILSMIOOperationSchedulerProvider ioSchedulerProvider,
+            ILSMOperationTrackerFactory opTrackerProvider, ILSMIOOperationSchedulerProvider ioSchedulerProvider,
             ILSMIOOperationCallbackFactory ioOpCallbackFactory, double bloomFilterFalsePositiveRate,
             int[] invertedIndexFields, ITypeTraits[] filterTypeTraits, IBinaryComparatorFactory[] filterCmpFactories,
             int[] filterFields, int[] filterFieldsForNonBulkLoadOps, int[] invertedIndexFieldsForNonBulkLoadOps,
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/inmemory/InMemoryInvertedListCursor.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/inmemory/InMemoryInvertedListCursor.java
index be5358c..9299620 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/inmemory/InMemoryInvertedListCursor.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/inmemory/InMemoryInvertedListCursor.java
@@ -29,7 +29,7 @@
 import org.apache.hyracks.dataflow.common.comm.io.ArrayTupleReference;
 import org.apache.hyracks.dataflow.common.data.accessors.ITupleReference;
 import org.apache.hyracks.dataflow.common.data.marshalling.IntegerSerializerDeserializer;
-import org.apache.hyracks.dataflow.common.util.TupleUtils;
+import org.apache.hyracks.dataflow.common.utils.TupleUtils;
 import org.apache.hyracks.storage.am.btree.impls.BTree.BTreeAccessor;
 import org.apache.hyracks.storage.am.btree.impls.RangePredicate;
 import org.apache.hyracks.storage.am.common.api.IIndexCursor;
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/ondisk/OnDiskInvertedIndex.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/ondisk/OnDiskInvertedIndex.java
index e27edf5..4884190 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/ondisk/OnDiskInvertedIndex.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/ondisk/OnDiskInvertedIndex.java
@@ -33,7 +33,7 @@
 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.dataflow.common.util.TupleUtils;
+import org.apache.hyracks.dataflow.common.utils.TupleUtils;
 import org.apache.hyracks.storage.am.btree.frames.BTreeLeafFrameType;
 import org.apache.hyracks.storage.am.btree.impls.BTree;
 import org.apache.hyracks.storage.am.btree.impls.RangePredicate;
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/dataflow/AbstractLSMRTreeDataflowHelper.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/dataflow/AbstractLSMRTreeDataflowHelper.java
index abd11b1..00a2e3f 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/dataflow/AbstractLSMRTreeDataflowHelper.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/dataflow/AbstractLSMRTreeDataflowHelper.java
@@ -36,7 +36,7 @@
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationScheduler;
 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.ILSMOperationTrackerProvider;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMOperationTrackerFactory;
 import org.apache.hyracks.storage.am.lsm.common.api.IVirtualBufferCache;
 import org.apache.hyracks.storage.am.lsm.common.dataflow.AbstractLSMIndexDataflowHelper;
 import org.apache.hyracks.storage.am.rtree.frames.RTreePolicyType;
@@ -55,7 +55,7 @@
     public AbstractLSMRTreeDataflowHelper(IIndexOperatorDescriptor opDesc, IHyracksTaskContext ctx, int partition,
             List<IVirtualBufferCache> virtualBufferCaches, IBinaryComparatorFactory[] btreeComparatorFactories,
             IPrimitiveValueProviderFactory[] valueProviderFactories, RTreePolicyType rtreePolicyType,
-            ILSMMergePolicy mergePolicy, ILSMOperationTrackerProvider opTrackerFactory,
+            ILSMMergePolicy mergePolicy, ILSMOperationTrackerFactory opTrackerFactory,
             ILSMIOOperationScheduler ioScheduler, ILSMIOOperationCallbackFactory ioOpCallbackFactory,
             ILinearizeComparatorFactory linearizeCmpFactory, int[] rtreeFields, ITypeTraits[] filterTypeTraits,
             IBinaryComparatorFactory[] filterCmpFactories, int[] filterFields, boolean durable, boolean isPointMBR)
@@ -70,7 +70,7 @@
             List<IVirtualBufferCache> virtualBufferCaches, double bloomFilterFalsePositiveRate,
             IBinaryComparatorFactory[] btreeComparatorFactories,
             IPrimitiveValueProviderFactory[] valueProviderFactories, RTreePolicyType rtreePolicyType,
-            ILSMMergePolicy mergePolicy, ILSMOperationTrackerProvider opTrackerFactory,
+            ILSMMergePolicy mergePolicy, ILSMOperationTrackerFactory opTrackerFactory,
             ILSMIOOperationScheduler ioScheduler, ILSMIOOperationCallbackFactory ioOpCallbackFactory,
             ILinearizeComparatorFactory linearizeCmpFactory, int[] rtreeFields, ITypeTraits[] filterTypeTraits,
             IBinaryComparatorFactory[] filterCmpFactories, int[] filterFields, boolean durable, boolean isPointMBR)
@@ -93,7 +93,8 @@
         return createLSMTree(virtualBufferCaches, fileRef, opDesc.getStorageManager().getBufferCache(ctx),
                 opDesc.getStorageManager().getFileMapProvider(ctx), treeOpDesc.getTreeIndexTypeTraits(),
                 treeOpDesc.getTreeIndexComparatorFactories(), btreeComparatorFactories,
-                opTrackerFactory.getOperationTracker(ctx), valueProviderFactories, rtreePolicyType, linearizeCmpFactory,
+                opTrackerFactory.getOperationTracker(ctx.getJobletContext().getApplicationContext()),
+                valueProviderFactories, rtreePolicyType, linearizeCmpFactory,
                 rtreeFields, filterTypeTraits, filterCmpFactories, filterFields);
 
     }
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/dataflow/ExternalRTreeDataflowHelper.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/dataflow/ExternalRTreeDataflowHelper.java
index b4fc496..7c57bed 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/dataflow/ExternalRTreeDataflowHelper.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/dataflow/ExternalRTreeDataflowHelper.java
@@ -36,7 +36,7 @@
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationScheduler;
 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.ILSMOperationTrackerProvider;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMOperationTrackerFactory;
 import org.apache.hyracks.storage.am.lsm.common.api.IVirtualBufferCache;
 import org.apache.hyracks.storage.am.lsm.rtree.utils.LSMRTreeUtils;
 import org.apache.hyracks.storage.am.rtree.frames.RTreePolicyType;
@@ -50,7 +50,7 @@
     public ExternalRTreeDataflowHelper(IIndexOperatorDescriptor opDesc, IHyracksTaskContext ctx, int partition,
             IBinaryComparatorFactory[] btreeComparatorFactories,
             IPrimitiveValueProviderFactory[] valueProviderFactories, RTreePolicyType rtreePolicyType,
-            ILSMMergePolicy mergePolicy, ILSMOperationTrackerProvider opTrackerFactory,
+            ILSMMergePolicy mergePolicy, ILSMOperationTrackerFactory opTrackerFactory,
             ILSMIOOperationScheduler ioScheduler, ILSMIOOperationCallbackFactory ioOpCallbackFactory,
             ILinearizeComparatorFactory linearizeCmpFactory, int[] btreeFields, int version, boolean durable,
             boolean isPointMBR) throws HyracksDataException {
@@ -63,7 +63,7 @@
     public ExternalRTreeDataflowHelper(IIndexOperatorDescriptor opDesc, IHyracksTaskContext ctx, int partition,
             double bloomFilterFalsePositiveRate, IBinaryComparatorFactory[] btreeComparatorFactories,
             IPrimitiveValueProviderFactory[] valueProviderFactories, RTreePolicyType rtreePolicyType,
-            ILSMMergePolicy mergePolicy, ILSMOperationTrackerProvider opTrackerFactory,
+            ILSMMergePolicy mergePolicy, ILSMOperationTrackerFactory opTrackerFactory,
             ILSMIOOperationScheduler ioScheduler, ILSMIOOperationCallbackFactory ioOpCallbackFactory,
             ILinearizeComparatorFactory linearizeCmpFactory, int[] btreeFields, int version, boolean durable,
             boolean isPointMBR) throws HyracksDataException {
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/dataflow/ExternalRTreeDataflowHelperFactory.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/dataflow/ExternalRTreeDataflowHelperFactory.java
index 70e0580..8eab50e 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/dataflow/ExternalRTreeDataflowHelperFactory.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/dataflow/ExternalRTreeDataflowHelperFactory.java
@@ -30,7 +30,7 @@
 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.ILSMOperationTrackerProvider;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMOperationTrackerFactory;
 import org.apache.hyracks.storage.am.rtree.frames.RTreePolicyType;
 
 public class ExternalRTreeDataflowHelperFactory extends LSMRTreeDataflowHelperFactory {
@@ -41,7 +41,7 @@
     public ExternalRTreeDataflowHelperFactory(IPrimitiveValueProviderFactory[] valueProviderFactories,
             RTreePolicyType rtreePolicyType, IBinaryComparatorFactory[] btreeComparatorFactories,
             ILSMMergePolicyFactory mergePolicyFactory, Map<String, String> mergePolicyProperties,
-            ILSMOperationTrackerProvider opTrackerFactory, ILSMIOOperationSchedulerProvider ioSchedulerProvider,
+            ILSMOperationTrackerFactory opTrackerFactory, ILSMIOOperationSchedulerProvider ioSchedulerProvider,
             ILSMIOOperationCallbackFactory ioOpCallbackFactory, ILinearizeComparatorFactory linearizeCmpFactory,
             double bloomFilterFalsePositiveRate, int[] btreeFields, int version, boolean durable, boolean isPointMBR) {
         super(valueProviderFactories, rtreePolicyType, btreeComparatorFactories, null, mergePolicyFactory,
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/dataflow/LSMRTreeDataflowHelper.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/dataflow/LSMRTreeDataflowHelper.java
index c949737..3f69016 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/dataflow/LSMRTreeDataflowHelper.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/dataflow/LSMRTreeDataflowHelper.java
@@ -36,7 +36,7 @@
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationScheduler;
 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.ILSMOperationTrackerProvider;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMOperationTrackerFactory;
 import org.apache.hyracks.storage.am.lsm.common.api.IVirtualBufferCache;
 import org.apache.hyracks.storage.am.lsm.rtree.utils.LSMRTreeUtils;
 import org.apache.hyracks.storage.am.rtree.frames.RTreePolicyType;
@@ -49,7 +49,7 @@
     public LSMRTreeDataflowHelper(IIndexOperatorDescriptor opDesc, IHyracksTaskContext ctx, int partition,
             List<IVirtualBufferCache> virtualBufferCaches, IBinaryComparatorFactory[] btreeComparatorFactories,
             IPrimitiveValueProviderFactory[] valueProviderFactories, RTreePolicyType rtreePolicyType,
-            ILSMMergePolicy mergePolicy, ILSMOperationTrackerProvider opTrackerFactory,
+            ILSMMergePolicy mergePolicy, ILSMOperationTrackerFactory opTrackerFactory,
             ILSMIOOperationScheduler ioScheduler, ILSMIOOperationCallbackFactory ioOpCallbackFactory,
             ILinearizeComparatorFactory linearizeCmpFactory, int[] rtreeFields, int[] btreeFields,
             ITypeTraits[] filterTypeTraits, IBinaryComparatorFactory[] filterCmpFactories, int[] filterFields,
@@ -64,7 +64,7 @@
             List<IVirtualBufferCache> virtualBufferCaches, double bloomFilterFalsePositiveRate,
             IBinaryComparatorFactory[] btreeComparatorFactories,
             IPrimitiveValueProviderFactory[] valueProviderFactories, RTreePolicyType rtreePolicyType,
-            ILSMMergePolicy mergePolicy, ILSMOperationTrackerProvider opTrackerFactory,
+            ILSMMergePolicy mergePolicy, ILSMOperationTrackerFactory opTrackerFactory,
             ILSMIOOperationScheduler ioScheduler, ILSMIOOperationCallbackFactory ioOpCallbackFactory,
             ILinearizeComparatorFactory linearizeCmpFactory, int[] rtreeFields, int[] btreeFields,
             ITypeTraits[] filterTypeTraits, IBinaryComparatorFactory[] filterCmpFactories, int[] filterFields,
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/dataflow/LSMRTreeDataflowHelperFactory.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/dataflow/LSMRTreeDataflowHelperFactory.java
index ed0bd2e..93d9b54 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/dataflow/LSMRTreeDataflowHelperFactory.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/dataflow/LSMRTreeDataflowHelperFactory.java
@@ -32,7 +32,7 @@
 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.ILSMOperationTrackerProvider;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMOperationTrackerFactory;
 import org.apache.hyracks.storage.am.lsm.common.api.IVirtualBufferCacheProvider;
 import org.apache.hyracks.storage.am.lsm.common.dataflow.AbstractLSMIndexDataflowHelperFactory;
 import org.apache.hyracks.storage.am.rtree.frames.RTreePolicyType;
@@ -52,7 +52,7 @@
     public LSMRTreeDataflowHelperFactory(IPrimitiveValueProviderFactory[] valueProviderFactories,
             RTreePolicyType rtreePolicyType, IBinaryComparatorFactory[] btreeComparatorFactories,
             IVirtualBufferCacheProvider virtualBufferCacheProvider, ILSMMergePolicyFactory mergePolicyFactory,
-            Map<String, String> mergePolicyProperties, ILSMOperationTrackerProvider opTrackerFactory,
+            Map<String, String> mergePolicyProperties, ILSMOperationTrackerFactory opTrackerFactory,
             ILSMIOOperationSchedulerProvider ioSchedulerProvider, ILSMIOOperationCallbackFactory ioOpCallbackFactory,
             ILinearizeComparatorFactory linearizeCmpFactory, double bloomFilterFalsePositiveRate, int[] rtreeFields,
             int[] btreeFields, ITypeTraits[] filterTypeTraits, IBinaryComparatorFactory[] filterCmpFactories,
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/dataflow/LSMRTreeWithAntiMatterTuplesDataflowHelper.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/dataflow/LSMRTreeWithAntiMatterTuplesDataflowHelper.java
index 116cb51..8a807c2 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/dataflow/LSMRTreeWithAntiMatterTuplesDataflowHelper.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/dataflow/LSMRTreeWithAntiMatterTuplesDataflowHelper.java
@@ -36,7 +36,7 @@
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationScheduler;
 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.ILSMOperationTrackerProvider;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMOperationTrackerFactory;
 import org.apache.hyracks.storage.am.lsm.common.api.IVirtualBufferCache;
 import org.apache.hyracks.storage.am.lsm.rtree.utils.LSMRTreeUtils;
 import org.apache.hyracks.storage.am.rtree.frames.RTreePolicyType;
@@ -48,7 +48,7 @@
             int partition, List<IVirtualBufferCache> virtualBufferCaches,
             IBinaryComparatorFactory[] btreeComparatorFactories,
             IPrimitiveValueProviderFactory[] valueProviderFactories, RTreePolicyType rtreePolicyType,
-            ILSMMergePolicy mergePolicy, ILSMOperationTrackerProvider opTrackerFactory,
+            ILSMMergePolicy mergePolicy, ILSMOperationTrackerFactory opTrackerFactory,
             ILSMIOOperationScheduler ioScheduler, ILSMIOOperationCallbackFactory ioOpCallbackFactory,
             ILinearizeComparatorFactory linearizeCmpFactory, int[] rtreeFields, ITypeTraits[] filterTypeTraits,
             IBinaryComparatorFactory[] filterCmpFactories, int[] filterFields, boolean durable, boolean isPointMBR)
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/dataflow/LSMRTreeWithAntiMatterTuplesDataflowHelperFactory.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/dataflow/LSMRTreeWithAntiMatterTuplesDataflowHelperFactory.java
index 725c69a..297d82d 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/dataflow/LSMRTreeWithAntiMatterTuplesDataflowHelperFactory.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/dataflow/LSMRTreeWithAntiMatterTuplesDataflowHelperFactory.java
@@ -32,7 +32,7 @@
 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.ILSMOperationTrackerProvider;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMOperationTrackerFactory;
 import org.apache.hyracks.storage.am.lsm.common.api.IVirtualBufferCacheProvider;
 import org.apache.hyracks.storage.am.lsm.common.dataflow.AbstractLSMIndexDataflowHelperFactory;
 import org.apache.hyracks.storage.am.rtree.frames.RTreePolicyType;
@@ -51,7 +51,7 @@
     public LSMRTreeWithAntiMatterTuplesDataflowHelperFactory(IPrimitiveValueProviderFactory[] valueProviderFactories,
             RTreePolicyType rtreePolicyType, IBinaryComparatorFactory[] btreeComparatorFactories,
             IVirtualBufferCacheProvider virtualBufferCacheProvider, ILSMMergePolicyFactory mergePolicyFactory,
-            Map<String, String> mergePolicyProperties, ILSMOperationTrackerProvider opTrackerFactory,
+            Map<String, String> mergePolicyProperties, ILSMOperationTrackerFactory opTrackerFactory,
             ILSMIOOperationSchedulerProvider ioSchedulerProvider, ILSMIOOperationCallbackFactory ioOpCallbackFactory,
             ILinearizeComparatorFactory linearizeCmpFactory, int[] rtreeFields, ITypeTraits[] filterTypeTraits,
             IBinaryComparatorFactory[] filterCmpFactories, int[] filterFields, boolean durable, boolean isPointMBR) {
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-rtree/src/main/java/org/apache/hyracks/storage/am/rtree/dataflow/RTreeSearchOperatorDescriptor.java b/hyracks-fullstack/hyracks/hyracks-storage-am-rtree/src/main/java/org/apache/hyracks/storage/am/rtree/dataflow/RTreeSearchOperatorDescriptor.java
index 32c85e6..fad00ca 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-rtree/src/main/java/org/apache/hyracks/storage/am/rtree/dataflow/RTreeSearchOperatorDescriptor.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-rtree/src/main/java/org/apache/hyracks/storage/am/rtree/dataflow/RTreeSearchOperatorDescriptor.java
@@ -35,7 +35,7 @@
 import org.apache.hyracks.storage.am.common.dataflow.AbstractTreeIndexOperatorDescriptor;
 import org.apache.hyracks.storage.am.common.dataflow.IIndexDataflowHelperFactory;
 import org.apache.hyracks.storage.am.common.impls.NoOpOperationCallbackFactory;
-import org.apache.hyracks.storage.common.IStorageManagerInterface;
+import org.apache.hyracks.storage.common.IStorageManager;
 import org.apache.hyracks.storage.common.file.NoOpLocalResourceFactoryProvider;
 
 public class RTreeSearchOperatorDescriptor extends AbstractTreeIndexOperatorDescriptor {
@@ -47,7 +47,7 @@
     protected final int[] maxFilterFieldIndexes;
 
     public RTreeSearchOperatorDescriptor(IOperatorDescriptorRegistry spec, RecordDescriptor recDesc,
-            IStorageManagerInterface storageManager, IIndexLifecycleManagerProvider lifecycleManagerProvider,
+            IStorageManager storageManager, IIndexLifecycleManagerProvider lifecycleManagerProvider,
             IFileSplitProvider fileSplitProvider, ITypeTraits[] typeTraits,
             IBinaryComparatorFactory[] comparatorFactories, int[] keyFields,
             IIndexDataflowHelperFactory dataflowHelperFactory, boolean retainInput, boolean retainNull,
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/IStorageManager.java b/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/IStorageManager.java
new file mode 100644
index 0000000..24cf557
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/IStorageManager.java
@@ -0,0 +1,60 @@
+/*
+ * 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.common;
+
+import java.io.Serializable;
+
+import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.storage.common.buffercache.IBufferCache;
+import org.apache.hyracks.storage.common.file.IFileMapProvider;
+import org.apache.hyracks.storage.common.file.ILocalResourceRepository;
+import org.apache.hyracks.storage.common.file.IResourceIdFactory;
+
+/**
+ * Provides storage components during hyracks tasks execution
+ */
+public interface IStorageManager extends Serializable {
+    /**
+     * @param ctx
+     *            hyracks task context
+     * @return the disk buffer cache {@link org.apache.hyracks.storage.common.buffercache.IBufferCache}
+     */
+    IBufferCache getBufferCache(IHyracksTaskContext ctx);
+
+    /**
+     * @param ctx
+     *            the task context
+     * @return the file map provider {@link org.apache.hyracks.storage.common.file.IFileMapProvider}
+     */
+    IFileMapProvider getFileMapProvider(IHyracksTaskContext ctx);
+
+    /**
+     * @param ctx
+     *            the task context
+     * @return the local resource repository {@link org.apache.hyracks.storage.common.file.ILocalResourceRepository}
+     */
+    ILocalResourceRepository getLocalResourceRepository(IHyracksTaskContext ctx);
+
+    /**
+     * @param ctx
+     *            the task context
+     * @return the resource id factory {@link org.apache.hyracks.storage.common.file.IResourceIdFactory}
+     */
+    IResourceIdFactory getResourceIdFactory(IHyracksTaskContext ctx);
+}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/IStorageManagerInterface.java b/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/IStorageManagerInterface.java
deleted file mode 100644
index c1bdeab..0000000
--- a/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/IStorageManagerInterface.java
+++ /dev/null
@@ -1,37 +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.common;
-
-import java.io.Serializable;
-
-import org.apache.hyracks.api.context.IHyracksTaskContext;
-import org.apache.hyracks.storage.common.buffercache.IBufferCache;
-import org.apache.hyracks.storage.common.file.IFileMapProvider;
-import org.apache.hyracks.storage.common.file.ILocalResourceRepository;
-import org.apache.hyracks.storage.common.file.IResourceIdFactory;
-
-public interface IStorageManagerInterface extends Serializable {
-    public IBufferCache getBufferCache(IHyracksTaskContext ctx);
-
-    public IFileMapProvider getFileMapProvider(IHyracksTaskContext ctx);
-
-    public ILocalResourceRepository getLocalResourceRepository(IHyracksTaskContext ctx);
-
-    public IResourceIdFactory getResourceIdFactory(IHyracksTaskContext ctx);
-}
diff --git a/hyracks-fullstack/hyracks/hyracks-test-support/src/main/java/org/apache/hyracks/storage/am/btree/AbstractModificationOperationCallbackTest.java b/hyracks-fullstack/hyracks/hyracks-test-support/src/main/java/org/apache/hyracks/storage/am/btree/AbstractModificationOperationCallbackTest.java
index a9b2e1f..6da9334 100644
--- a/hyracks-fullstack/hyracks/hyracks-test-support/src/main/java/org/apache/hyracks/storage/am/btree/AbstractModificationOperationCallbackTest.java
+++ b/hyracks-fullstack/hyracks/hyracks-test-support/src/main/java/org/apache/hyracks/storage/am/btree/AbstractModificationOperationCallbackTest.java
@@ -22,7 +22,7 @@
 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.dataflow.common.util.TupleUtils;
+import org.apache.hyracks.dataflow.common.utils.TupleUtils;
 import org.apache.hyracks.storage.am.common.api.IIndexAccessor;
 import org.apache.hyracks.storage.am.common.api.IModificationOperationCallback;
 import org.apache.hyracks.storage.am.common.impls.NoOpOperationCallback;
diff --git a/hyracks-fullstack/hyracks/hyracks-test-support/src/main/java/org/apache/hyracks/storage/am/btree/AbstractOperationCallbackTest.java b/hyracks-fullstack/hyracks/hyracks-test-support/src/main/java/org/apache/hyracks/storage/am/btree/AbstractOperationCallbackTest.java
index 2fb52c9..831dafa 100644
--- a/hyracks-fullstack/hyracks/hyracks-test-support/src/main/java/org/apache/hyracks/storage/am/btree/AbstractOperationCallbackTest.java
+++ b/hyracks-fullstack/hyracks/hyracks-test-support/src/main/java/org/apache/hyracks/storage/am/btree/AbstractOperationCallbackTest.java
@@ -20,7 +20,7 @@
 
 import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
 import org.apache.hyracks.dataflow.common.data.marshalling.IntegerSerializerDeserializer;
-import org.apache.hyracks.dataflow.common.util.SerdeUtils;
+import org.apache.hyracks.dataflow.common.utils.SerdeUtils;
 import org.apache.hyracks.storage.am.common.api.IIndex;
 import org.apache.hyracks.storage.am.common.ophelpers.MultiComparator;
 
diff --git a/hyracks-fullstack/hyracks/hyracks-test-support/src/main/java/org/apache/hyracks/storage/am/btree/AbstractSearchOperationCallbackTest.java b/hyracks-fullstack/hyracks/hyracks-test-support/src/main/java/org/apache/hyracks/storage/am/btree/AbstractSearchOperationCallbackTest.java
index 20a2c0f..2bb357a 100644
--- a/hyracks-fullstack/hyracks/hyracks-test-support/src/main/java/org/apache/hyracks/storage/am/btree/AbstractSearchOperationCallbackTest.java
+++ b/hyracks-fullstack/hyracks/hyracks-test-support/src/main/java/org/apache/hyracks/storage/am/btree/AbstractSearchOperationCallbackTest.java
@@ -30,7 +30,7 @@
 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.dataflow.common.util.TupleUtils;
+import org.apache.hyracks.dataflow.common.utils.TupleUtils;
 import org.apache.hyracks.storage.am.btree.impls.RangePredicate;
 import org.apache.hyracks.storage.am.common.api.IIndexAccessor;
 import org.apache.hyracks.storage.am.common.api.IIndexCursor;
diff --git a/hyracks-fullstack/hyracks/hyracks-test-support/src/main/java/org/apache/hyracks/storage/am/btree/OrderedIndexExamplesTest.java b/hyracks-fullstack/hyracks/hyracks-test-support/src/main/java/org/apache/hyracks/storage/am/btree/OrderedIndexExamplesTest.java
index 6225697..8988605 100644
--- a/hyracks-fullstack/hyracks/hyracks-test-support/src/main/java/org/apache/hyracks/storage/am/btree/OrderedIndexExamplesTest.java
+++ b/hyracks-fullstack/hyracks/hyracks-test-support/src/main/java/org/apache/hyracks/storage/am/btree/OrderedIndexExamplesTest.java
@@ -37,7 +37,7 @@
 import org.apache.hyracks.dataflow.common.data.accessors.ITupleReference;
 import org.apache.hyracks.dataflow.common.data.marshalling.IntegerSerializerDeserializer;
 import org.apache.hyracks.dataflow.common.data.marshalling.UTF8StringSerializerDeserializer;
-import org.apache.hyracks.dataflow.common.util.TupleUtils;
+import org.apache.hyracks.dataflow.common.utils.TupleUtils;
 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.TestOperationCallback;
diff --git a/hyracks-fullstack/hyracks/hyracks-test-support/src/main/java/org/apache/hyracks/storage/am/btree/OrderedIndexMultiThreadTest.java b/hyracks-fullstack/hyracks/hyracks-test-support/src/main/java/org/apache/hyracks/storage/am/btree/OrderedIndexMultiThreadTest.java
index 95c3448..b9c0286 100644
--- a/hyracks-fullstack/hyracks/hyracks-test-support/src/main/java/org/apache/hyracks/storage/am/btree/OrderedIndexMultiThreadTest.java
+++ b/hyracks-fullstack/hyracks/hyracks-test-support/src/main/java/org/apache/hyracks/storage/am/btree/OrderedIndexMultiThreadTest.java
@@ -29,7 +29,7 @@
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.dataflow.common.data.marshalling.IntegerSerializerDeserializer;
 import org.apache.hyracks.dataflow.common.data.marshalling.UTF8StringSerializerDeserializer;
-import org.apache.hyracks.dataflow.common.util.SerdeUtils;
+import org.apache.hyracks.dataflow.common.utils.SerdeUtils;
 import org.apache.hyracks.storage.am.common.IIndexTestWorkerFactory;
 import org.apache.hyracks.storage.am.common.IndexMultiThreadTestDriver;
 import org.apache.hyracks.storage.am.common.TestWorkloadConf;
diff --git a/hyracks-fullstack/hyracks/hyracks-test-support/src/main/java/org/apache/hyracks/storage/am/btree/OrderedIndexTestDriver.java b/hyracks-fullstack/hyracks/hyracks-test-support/src/main/java/org/apache/hyracks/storage/am/btree/OrderedIndexTestDriver.java
index b1e8a8c..3aab8ef 100644
--- a/hyracks-fullstack/hyracks/hyracks-test-support/src/main/java/org/apache/hyracks/storage/am/btree/OrderedIndexTestDriver.java
+++ b/hyracks-fullstack/hyracks/hyracks-test-support/src/main/java/org/apache/hyracks/storage/am/btree/OrderedIndexTestDriver.java
@@ -29,7 +29,7 @@
 import org.apache.hyracks.dataflow.common.data.accessors.ITupleReference;
 import org.apache.hyracks.dataflow.common.data.marshalling.IntegerSerializerDeserializer;
 import org.apache.hyracks.dataflow.common.data.marshalling.UTF8StringSerializerDeserializer;
-import org.apache.hyracks.dataflow.common.util.TupleUtils;
+import org.apache.hyracks.dataflow.common.utils.TupleUtils;
 import org.apache.hyracks.storage.am.btree.frames.BTreeLeafFrameType;
 import org.apache.hyracks.storage.am.config.AccessMethodTestsConfig;
 
diff --git a/hyracks-fullstack/hyracks/hyracks-test-support/src/main/java/org/apache/hyracks/storage/am/btree/OrderedIndexTestUtils.java b/hyracks-fullstack/hyracks/hyracks-test-support/src/main/java/org/apache/hyracks/storage/am/btree/OrderedIndexTestUtils.java
index 40b9f05..2a16ebe36 100644
--- a/hyracks-fullstack/hyracks/hyracks-test-support/src/main/java/org/apache/hyracks/storage/am/btree/OrderedIndexTestUtils.java
+++ b/hyracks-fullstack/hyracks/hyracks-test-support/src/main/java/org/apache/hyracks/storage/am/btree/OrderedIndexTestUtils.java
@@ -38,7 +38,7 @@
 import org.apache.hyracks.dataflow.common.data.accessors.ITupleReference;
 import org.apache.hyracks.dataflow.common.data.marshalling.IntegerSerializerDeserializer;
 import org.apache.hyracks.dataflow.common.data.marshalling.UTF8StringSerializerDeserializer;
-import org.apache.hyracks.dataflow.common.util.TupleUtils;
+import org.apache.hyracks.dataflow.common.utils.TupleUtils;
 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.CheckTuple;
diff --git a/hyracks-fullstack/hyracks/hyracks-test-support/src/main/java/org/apache/hyracks/storage/am/common/TreeIndexTestUtils.java b/hyracks-fullstack/hyracks/hyracks-test-support/src/main/java/org/apache/hyracks/storage/am/common/TreeIndexTestUtils.java
index 464291a..94c0ab4 100644
--- a/hyracks-fullstack/hyracks/hyracks-test-support/src/main/java/org/apache/hyracks/storage/am/common/TreeIndexTestUtils.java
+++ b/hyracks-fullstack/hyracks/hyracks-test-support/src/main/java/org/apache/hyracks/storage/am/common/TreeIndexTestUtils.java
@@ -36,7 +36,7 @@
 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.dataflow.common.util.TupleUtils;
+import org.apache.hyracks.dataflow.common.utils.TupleUtils;
 import org.apache.hyracks.storage.am.common.api.IIndexBulkLoader;
 import org.apache.hyracks.storage.am.common.api.ISearchPredicate;
 import org.apache.hyracks.storage.am.common.api.ITreeIndexAccessor;
diff --git a/hyracks-fullstack/hyracks/hyracks-test-support/src/main/java/org/apache/hyracks/storage/am/rtree/AbstractRTreeExamplesTest.java b/hyracks-fullstack/hyracks/hyracks-test-support/src/main/java/org/apache/hyracks/storage/am/rtree/AbstractRTreeExamplesTest.java
index d4470fe..ba55a1d 100644
--- a/hyracks-fullstack/hyracks/hyracks-test-support/src/main/java/org/apache/hyracks/storage/am/rtree/AbstractRTreeExamplesTest.java
+++ b/hyracks-fullstack/hyracks/hyracks-test-support/src/main/java/org/apache/hyracks/storage/am/rtree/AbstractRTreeExamplesTest.java
@@ -37,7 +37,7 @@
 import org.apache.hyracks.dataflow.common.data.marshalling.DoubleSerializerDeserializer;
 import org.apache.hyracks.dataflow.common.data.marshalling.IntegerSerializerDeserializer;
 import org.apache.hyracks.dataflow.common.data.marshalling.UTF8StringSerializerDeserializer;
-import org.apache.hyracks.dataflow.common.util.TupleUtils;
+import org.apache.hyracks.dataflow.common.utils.TupleUtils;
 import org.apache.hyracks.storage.am.common.TestOperationCallback;
 import org.apache.hyracks.storage.am.common.api.IIndexAccessor;
 import org.apache.hyracks.storage.am.common.api.IIndexBulkLoader;
diff --git a/hyracks-fullstack/hyracks/hyracks-test-support/src/main/java/org/apache/hyracks/storage/am/rtree/AbstractRTreeMultiThreadTest.java b/hyracks-fullstack/hyracks/hyracks-test-support/src/main/java/org/apache/hyracks/storage/am/rtree/AbstractRTreeMultiThreadTest.java
index 27d7aa8..d6f07d9 100644
--- a/hyracks-fullstack/hyracks/hyracks-test-support/src/main/java/org/apache/hyracks/storage/am/rtree/AbstractRTreeMultiThreadTest.java
+++ b/hyracks-fullstack/hyracks/hyracks-test-support/src/main/java/org/apache/hyracks/storage/am/rtree/AbstractRTreeMultiThreadTest.java
@@ -31,7 +31,7 @@
 import org.apache.hyracks.data.std.primitive.IntegerPointable;
 import org.apache.hyracks.dataflow.common.data.marshalling.DoubleSerializerDeserializer;
 import org.apache.hyracks.dataflow.common.data.marshalling.IntegerSerializerDeserializer;
-import org.apache.hyracks.dataflow.common.util.SerdeUtils;
+import org.apache.hyracks.dataflow.common.utils.SerdeUtils;
 import org.apache.hyracks.storage.am.common.IIndexTestWorkerFactory;
 import org.apache.hyracks.storage.am.common.IndexMultiThreadTestDriver;
 import org.apache.hyracks.storage.am.common.TestWorkloadConf;
diff --git a/hyracks-fullstack/hyracks/hyracks-test-support/src/main/java/org/apache/hyracks/storage/am/rtree/AbstractRTreeTestDriver.java b/hyracks-fullstack/hyracks/hyracks-test-support/src/main/java/org/apache/hyracks/storage/am/rtree/AbstractRTreeTestDriver.java
index 7e56455..87f70fd 100644
--- a/hyracks-fullstack/hyracks/hyracks-test-support/src/main/java/org/apache/hyracks/storage/am/rtree/AbstractRTreeTestDriver.java
+++ b/hyracks-fullstack/hyracks/hyracks-test-support/src/main/java/org/apache/hyracks/storage/am/rtree/AbstractRTreeTestDriver.java
@@ -31,7 +31,7 @@
 import org.apache.hyracks.dataflow.common.data.accessors.ITupleReference;
 import org.apache.hyracks.dataflow.common.data.marshalling.DoubleSerializerDeserializer;
 import org.apache.hyracks.dataflow.common.data.marshalling.IntegerSerializerDeserializer;
-import org.apache.hyracks.dataflow.common.util.TupleUtils;
+import org.apache.hyracks.dataflow.common.utils.TupleUtils;
 import org.apache.hyracks.storage.am.common.api.IPrimitiveValueProviderFactory;
 import org.apache.hyracks.storage.am.config.AccessMethodTestsConfig;
 import org.apache.hyracks.storage.am.rtree.frames.RTreePolicyType;
diff --git a/hyracks-fullstack/hyracks/hyracks-test-support/src/main/java/org/apache/hyracks/storage/am/rtree/RTreeTestUtils.java b/hyracks-fullstack/hyracks/hyracks-test-support/src/main/java/org/apache/hyracks/storage/am/rtree/RTreeTestUtils.java
index 824d925..13a4fcb 100644
--- a/hyracks-fullstack/hyracks/hyracks-test-support/src/main/java/org/apache/hyracks/storage/am/rtree/RTreeTestUtils.java
+++ b/hyracks-fullstack/hyracks/hyracks-test-support/src/main/java/org/apache/hyracks/storage/am/rtree/RTreeTestUtils.java
@@ -31,7 +31,7 @@
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
 import org.apache.hyracks.dataflow.common.data.accessors.ITupleReference;
-import org.apache.hyracks.dataflow.common.util.TupleUtils;
+import org.apache.hyracks.dataflow.common.utils.TupleUtils;
 import org.apache.hyracks.storage.am.common.CheckTuple;
 import org.apache.hyracks.storage.am.common.IIndexTestContext;
 import org.apache.hyracks.storage.am.common.TreeIndexTestUtils;
diff --git a/hyracks-fullstack/hyracks/hyracks-test-support/src/main/java/org/apache/hyracks/test/support/TestStorageManagerInterface.java b/hyracks-fullstack/hyracks/hyracks-test-support/src/main/java/org/apache/hyracks/test/support/TestStorageManager.java
similarity index 92%
rename from hyracks-fullstack/hyracks/hyracks-test-support/src/main/java/org/apache/hyracks/test/support/TestStorageManagerInterface.java
rename to hyracks-fullstack/hyracks/hyracks-test-support/src/main/java/org/apache/hyracks/test/support/TestStorageManager.java
index 885ff9b..14ce345 100644
--- a/hyracks-fullstack/hyracks/hyracks-test-support/src/main/java/org/apache/hyracks/test/support/TestStorageManagerInterface.java
+++ b/hyracks-fullstack/hyracks/hyracks-test-support/src/main/java/org/apache/hyracks/test/support/TestStorageManager.java
@@ -19,13 +19,13 @@
 package org.apache.hyracks.test.support;
 
 import org.apache.hyracks.api.context.IHyracksTaskContext;
-import org.apache.hyracks.storage.common.IStorageManagerInterface;
+import org.apache.hyracks.storage.common.IStorageManager;
 import org.apache.hyracks.storage.common.buffercache.IBufferCache;
 import org.apache.hyracks.storage.common.file.IFileMapProvider;
 import org.apache.hyracks.storage.common.file.ILocalResourceRepository;
 import org.apache.hyracks.storage.common.file.ResourceIdFactory;
 
-public class TestStorageManagerInterface implements IStorageManagerInterface {
+public class TestStorageManager implements IStorageManager {
     private static final long serialVersionUID = 1L;
 
     @Override
diff --git a/hyracks-fullstack/hyracks/hyracks-test-support/src/main/java/org/apache/hyracks/test/support/TestStorageManagerComponentHolder.java b/hyracks-fullstack/hyracks/hyracks-test-support/src/main/java/org/apache/hyracks/test/support/TestStorageManagerComponentHolder.java
index cf59ada..259f737 100644
--- a/hyracks-fullstack/hyracks/hyracks-test-support/src/main/java/org/apache/hyracks/test/support/TestStorageManagerComponentHolder.java
+++ b/hyracks-fullstack/hyracks/hyracks-test-support/src/main/java/org/apache/hyracks/test/support/TestStorageManagerComponentHolder.java
@@ -58,7 +58,7 @@
     private static IResourceLifecycleManager<IIndex> lcManager;
     private static ResourceIdFactory resourceIdFactory;
     private static IMetadataPageManagerFactory metadataPageManagerFactory =
-            new AppendOnlyLinkedMetadataPageManagerFactory();
+            AppendOnlyLinkedMetadataPageManagerFactory.INSTANCE;
     private static int pageSize;
     private static int numPages;
     private static int maxOpenFiles;
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-bloomfilter-test/src/test/java/org/apache/hyracks/storage/am/bloomfilter/BloomFilterTest.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-bloomfilter-test/src/test/java/org/apache/hyracks/storage/am/bloomfilter/BloomFilterTest.java
index 4f57f19..afcc6c2 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-bloomfilter-test/src/test/java/org/apache/hyracks/storage/am/bloomfilter/BloomFilterTest.java
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-bloomfilter-test/src/test/java/org/apache/hyracks/storage/am/bloomfilter/BloomFilterTest.java
@@ -30,7 +30,7 @@
 import org.apache.hyracks.dataflow.common.comm.io.ArrayTupleReference;
 import org.apache.hyracks.dataflow.common.data.marshalling.IntegerSerializerDeserializer;
 import org.apache.hyracks.dataflow.common.data.marshalling.UTF8StringSerializerDeserializer;
-import org.apache.hyracks.dataflow.common.util.TupleUtils;
+import org.apache.hyracks.dataflow.common.utils.TupleUtils;
 import org.apache.hyracks.storage.am.bloomfilter.impls.BloomCalculations;
 import org.apache.hyracks.storage.am.bloomfilter.impls.BloomFilter;
 import org.apache.hyracks.storage.am.bloomfilter.impls.BloomFilterSpecification;
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-bloomfilter-test/src/test/java/org/apache/hyracks/storage/am/bloomfilter/MurmurHashForITupleReferenceTest.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-bloomfilter-test/src/test/java/org/apache/hyracks/storage/am/bloomfilter/MurmurHashForITupleReferenceTest.java
index 3284f8d..9d1b9be 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-bloomfilter-test/src/test/java/org/apache/hyracks/storage/am/bloomfilter/MurmurHashForITupleReferenceTest.java
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-bloomfilter-test/src/test/java/org/apache/hyracks/storage/am/bloomfilter/MurmurHashForITupleReferenceTest.java
@@ -33,7 +33,7 @@
 import org.apache.hyracks.dataflow.common.comm.io.ArrayTupleReference;
 import org.apache.hyracks.dataflow.common.data.accessors.ITupleReference;
 import org.apache.hyracks.dataflow.common.data.marshalling.UTF8StringSerializerDeserializer;
-import org.apache.hyracks.dataflow.common.util.TupleUtils;
+import org.apache.hyracks.dataflow.common.utils.TupleUtils;
 import org.apache.hyracks.storage.am.bloomfilter.impls.MurmurHash128Bit;
 import org.apache.hyracks.storage.am.bloomfilter.util.AbstractBloomFilterTest;
 
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/org/apache/hyracks/storage/am/btree/BTreeModificationOperationCallbackTest.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/org/apache/hyracks/storage/am/btree/BTreeModificationOperationCallbackTest.java
index 313510d..defb0bf 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/org/apache/hyracks/storage/am/btree/BTreeModificationOperationCallbackTest.java
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/org/apache/hyracks/storage/am/btree/BTreeModificationOperationCallbackTest.java
@@ -18,7 +18,7 @@
  */
 package org.apache.hyracks.storage.am.btree;
 
-import org.apache.hyracks.dataflow.common.util.SerdeUtils;
+import org.apache.hyracks.dataflow.common.utils.SerdeUtils;
 import org.apache.hyracks.storage.am.btree.frames.BTreeLeafFrameType;
 import org.apache.hyracks.storage.am.btree.util.BTreeTestHarness;
 import org.apache.hyracks.storage.am.btree.util.BTreeUtils;
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/org/apache/hyracks/storage/am/btree/BTreeSearchCursorTest.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/org/apache/hyracks/storage/am/btree/BTreeSearchCursorTest.java
index b880a01..3f71dd9 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/org/apache/hyracks/storage/am/btree/BTreeSearchCursorTest.java
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/org/apache/hyracks/storage/am/btree/BTreeSearchCursorTest.java
@@ -44,7 +44,7 @@
 import org.apache.hyracks.dataflow.common.comm.io.ArrayTupleReference;
 import org.apache.hyracks.dataflow.common.data.accessors.ITupleReference;
 import org.apache.hyracks.dataflow.common.data.marshalling.IntegerSerializerDeserializer;
-import org.apache.hyracks.dataflow.common.util.TupleUtils;
+import org.apache.hyracks.dataflow.common.utils.TupleUtils;
 import org.apache.hyracks.storage.am.btree.api.IBTreeInteriorFrame;
 import org.apache.hyracks.storage.am.btree.api.IBTreeLeafFrame;
 import org.apache.hyracks.storage.am.btree.exceptions.BTreeException;
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/org/apache/hyracks/storage/am/btree/BTreeSearchOperationCallbackTest.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/org/apache/hyracks/storage/am/btree/BTreeSearchOperationCallbackTest.java
index 797ee49..c9248a6 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/org/apache/hyracks/storage/am/btree/BTreeSearchOperationCallbackTest.java
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/org/apache/hyracks/storage/am/btree/BTreeSearchOperationCallbackTest.java
@@ -18,7 +18,7 @@
  */
 package org.apache.hyracks.storage.am.btree;
 
-import org.apache.hyracks.dataflow.common.util.SerdeUtils;
+import org.apache.hyracks.dataflow.common.utils.SerdeUtils;
 import org.apache.hyracks.storage.am.btree.frames.BTreeLeafFrameType;
 import org.apache.hyracks.storage.am.btree.util.BTreeTestHarness;
 import org.apache.hyracks.storage.am.btree.util.BTreeUtils;
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/org/apache/hyracks/storage/am/btree/BTreeUpdateSearchTest.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/org/apache/hyracks/storage/am/btree/BTreeUpdateSearchTest.java
index 493a7ab..8fc5a09 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/org/apache/hyracks/storage/am/btree/BTreeUpdateSearchTest.java
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/org/apache/hyracks/storage/am/btree/BTreeUpdateSearchTest.java
@@ -34,7 +34,7 @@
 import org.apache.hyracks.dataflow.common.comm.io.ArrayTupleReference;
 import org.apache.hyracks.dataflow.common.data.accessors.ITupleReference;
 import org.apache.hyracks.dataflow.common.data.marshalling.IntegerSerializerDeserializer;
-import org.apache.hyracks.dataflow.common.util.TupleUtils;
+import org.apache.hyracks.dataflow.common.utils.TupleUtils;
 import org.apache.hyracks.storage.am.btree.api.IBTreeLeafFrame;
 import org.apache.hyracks.storage.am.btree.frames.BTreeNSMInteriorFrameFactory;
 import org.apache.hyracks.storage.am.btree.frames.BTreeNSMLeafFrameFactory;
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/org/apache/hyracks/storage/am/btree/StorageManagerTest.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/org/apache/hyracks/storage/am/btree/StorageFileAccessTest.java
similarity index 99%
rename from hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/org/apache/hyracks/storage/am/btree/StorageManagerTest.java
rename to hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/org/apache/hyracks/storage/am/btree/StorageFileAccessTest.java
index 01863ca..d62d0ad 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/org/apache/hyracks/storage/am/btree/StorageManagerTest.java
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/org/apache/hyracks/storage/am/btree/StorageFileAccessTest.java
@@ -33,7 +33,7 @@
 import org.apache.hyracks.storage.common.file.BufferedFileHandle;
 import org.apache.hyracks.storage.common.sync.LatchType;
 
-public class StorageManagerTest extends AbstractBTreeTest {
+public class StorageFileAccessTest extends AbstractBTreeTest {
     public class PinnedLatchedPage {
         public final ICachedPage page;
         public final LatchType latch;
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/org/apache/hyracks/storage/am/btree/util/BTreeTestContext.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/org/apache/hyracks/storage/am/btree/util/BTreeTestContext.java
index f849e8e..440261a 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/org/apache/hyracks/storage/am/btree/util/BTreeTestContext.java
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/org/apache/hyracks/storage/am/btree/util/BTreeTestContext.java
@@ -24,7 +24,7 @@
 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.dataflow.common.util.SerdeUtils;
+import org.apache.hyracks.dataflow.common.utils.SerdeUtils;
 import org.apache.hyracks.storage.am.btree.OrderedIndexTestContext;
 import org.apache.hyracks.storage.am.btree.frames.BTreeLeafFrameType;
 import org.apache.hyracks.storage.am.btree.impls.BTree;
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/LSMBTreeExamplesTest.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/LSMBTreeExamplesTest.java
index ddfcae0..0cdcf48 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/LSMBTreeExamplesTest.java
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/LSMBTreeExamplesTest.java
@@ -30,14 +30,14 @@
 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.marshalling.IntegerSerializerDeserializer;
-import org.apache.hyracks.dataflow.common.util.TupleUtils;
+import org.apache.hyracks.dataflow.common.utils.TupleUtils;
 import org.apache.hyracks.storage.am.btree.OrderedIndexExamplesTest;
 import org.apache.hyracks.storage.am.common.TestOperationCallback;
 import org.apache.hyracks.storage.am.common.api.IIndexAccessor;
 import org.apache.hyracks.storage.am.common.api.ITreeIndex;
 import org.apache.hyracks.storage.am.common.api.TreeIndexException;
 import org.apache.hyracks.storage.am.lsm.btree.util.LSMBTreeTestHarness;
-import org.apache.hyracks.storage.am.lsm.btree.util.LSMBTreeUtils;
+import org.apache.hyracks.storage.am.lsm.btree.utils.LSMBTreeUtil;
 import org.junit.After;
 import org.junit.Before;
 import org.junit.Test;
@@ -49,7 +49,7 @@
     protected ITreeIndex createTreeIndex(ITypeTraits[] typeTraits, IBinaryComparatorFactory[] cmpFactories,
             int[] bloomFilterKeyFields, ITypeTraits[] filterTypeTraits, IBinaryComparatorFactory[] filterCmpFactories,
             int[] btreeFields, int[] filterFields) throws TreeIndexException, HyracksDataException {
-        return LSMBTreeUtils.createLSMTree(harness.getIOManager(), harness.getVirtualBufferCaches(), harness
+        return LSMBTreeUtil.createLSMTree(harness.getIOManager(), harness.getVirtualBufferCaches(), harness
                 .getFileReference(),
                 harness.getDiskBufferCache(), harness.getDiskFileMapProvider(), typeTraits, cmpFactories,
                 bloomFilterKeyFields, harness.getBoomFilterFalsePositiveRate(), harness.getMergePolicy(),
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/LSMBTreeModificationOperationCallbackTest.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/LSMBTreeModificationOperationCallbackTest.java
index 6ec706f..32e6bf6 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/LSMBTreeModificationOperationCallbackTest.java
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/LSMBTreeModificationOperationCallbackTest.java
@@ -19,16 +19,16 @@
 
 package org.apache.hyracks.storage.am.lsm.btree;
 
-import org.apache.hyracks.dataflow.common.util.SerdeUtils;
-import org.apache.hyracks.dataflow.common.util.TupleUtils;
+import org.apache.hyracks.dataflow.common.utils.SerdeUtils;
+import org.apache.hyracks.dataflow.common.utils.TupleUtils;
 import org.apache.hyracks.storage.am.btree.AbstractModificationOperationCallbackTest;
 import org.apache.hyracks.storage.am.common.impls.NoOpOperationCallback;
 import org.apache.hyracks.storage.am.lsm.btree.util.LSMBTreeTestHarness;
-import org.apache.hyracks.storage.am.lsm.btree.util.LSMBTreeUtils;
+import org.apache.hyracks.storage.am.lsm.btree.utils.LSMBTreeUtil;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndexAccessor;
 import org.apache.hyracks.storage.am.lsm.common.impls.BlockingIOOperationCallbackWrapper;
 import org.apache.hyracks.storage.am.lsm.common.impls.NoOpIOOperationCallback;
-import org.apache.hyracks.storage.am.lsm.common.impls.NoOpOperationTrackerProvider;
+import org.apache.hyracks.storage.am.lsm.common.impls.NoOpOperationTrackerFactory;
 import org.junit.Test;
 
 public class LSMBTreeModificationOperationCallbackTest extends AbstractModificationOperationCallbackTest {
@@ -45,13 +45,13 @@
 
     @Override
     protected void createIndexInstance() throws Exception {
-        index = LSMBTreeUtils.createLSMTree(harness.getIOManager(), harness.getVirtualBufferCaches(), harness
+        index = LSMBTreeUtil.createLSMTree(harness.getIOManager(), harness.getVirtualBufferCaches(), harness
                 .getFileReference(),
                 harness.getDiskBufferCache(), harness.getDiskFileMapProvider(),
                 SerdeUtils.serdesToTypeTraits(keySerdes),
                 SerdeUtils.serdesToComparatorFactories(keySerdes, keySerdes.length), bloomFilterKeyFields,
                 harness.getBoomFilterFalsePositiveRate(), harness.getMergePolicy(),
-                NoOpOperationTrackerProvider.INSTANCE.getOperationTracker(null), harness.getIOScheduler(),
+                NoOpOperationTrackerFactory.INSTANCE.getOperationTracker(null), harness.getIOScheduler(),
                 harness.getIOOperationCallback(), true, null, null, null, null, true, harness
                         .getMetadataPageManagerFactory());
     }
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/LSMBTreeSearchOperationCallbackTest.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/LSMBTreeSearchOperationCallbackTest.java
index 78ec635..f408bb9 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/LSMBTreeSearchOperationCallbackTest.java
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/LSMBTreeSearchOperationCallbackTest.java
@@ -26,8 +26,8 @@
 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.dataflow.common.util.SerdeUtils;
-import org.apache.hyracks.dataflow.common.util.TupleUtils;
+import org.apache.hyracks.dataflow.common.utils.SerdeUtils;
+import org.apache.hyracks.dataflow.common.utils.TupleUtils;
 import org.apache.hyracks.storage.am.btree.AbstractSearchOperationCallbackTest;
 import org.apache.hyracks.storage.am.btree.impls.RangePredicate;
 import org.apache.hyracks.storage.am.common.api.IIndexAccessor;
@@ -36,8 +36,8 @@
 import org.apache.hyracks.storage.am.common.api.ISearchOperationCallback;
 import org.apache.hyracks.storage.am.common.impls.NoOpOperationCallback;
 import org.apache.hyracks.storage.am.lsm.btree.util.LSMBTreeTestHarness;
-import org.apache.hyracks.storage.am.lsm.btree.util.LSMBTreeUtils;
-import org.apache.hyracks.storage.am.lsm.common.impls.NoOpOperationTrackerProvider;
+import org.apache.hyracks.storage.am.lsm.btree.utils.LSMBTreeUtil;
+import org.apache.hyracks.storage.am.lsm.common.impls.NoOpOperationTrackerFactory;
 import org.junit.Assert;
 import org.junit.Test;
 
@@ -52,13 +52,13 @@
 
     @Override
     protected void createIndexInstance() throws Exception {
-        index = LSMBTreeUtils.createLSMTree(harness.getIOManager(), harness.getVirtualBufferCaches(), harness
+        index = LSMBTreeUtil.createLSMTree(harness.getIOManager(), harness.getVirtualBufferCaches(), harness
                 .getFileReference(),
                 harness.getDiskBufferCache(), harness.getDiskFileMapProvider(),
                 SerdeUtils.serdesToTypeTraits(keySerdes),
                 SerdeUtils.serdesToComparatorFactories(keySerdes, keySerdes.length), bloomFilterKeyFields,
                 harness.getBoomFilterFalsePositiveRate(), harness.getMergePolicy(),
-                NoOpOperationTrackerProvider.INSTANCE.getOperationTracker(null), harness.getIOScheduler(),
+                NoOpOperationTrackerFactory.INSTANCE.getOperationTracker(null), harness.getIOScheduler(),
                 harness.getIOOperationCallback(), true, null, null, null, null, true, harness
                         .getMetadataPageManagerFactory());
     }
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/multithread/LSMBTreeMultiThreadTest.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/multithread/LSMBTreeMultiThreadTest.java
index 27424eb..cfd2196 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/multithread/LSMBTreeMultiThreadTest.java
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/multithread/LSMBTreeMultiThreadTest.java
@@ -32,7 +32,7 @@
 import org.apache.hyracks.storage.am.common.api.TreeIndexException;
 import org.apache.hyracks.storage.am.common.datagen.ProbabilityHelper;
 import org.apache.hyracks.storage.am.lsm.btree.util.LSMBTreeTestHarness;
-import org.apache.hyracks.storage.am.lsm.btree.util.LSMBTreeUtils;
+import org.apache.hyracks.storage.am.lsm.btree.utils.LSMBTreeUtil;
 
 public class LSMBTreeMultiThreadTest extends OrderedIndexMultiThreadTest {
 
@@ -53,7 +53,7 @@
     @Override
     protected ITreeIndex createIndex(ITypeTraits[] typeTraits, IBinaryComparatorFactory[] cmpFactories,
             int[] bloomFilterKeyFields) throws TreeIndexException, HyracksDataException {
-        return LSMBTreeUtils.createLSMTree(harness.getIOManager(), harness.getVirtualBufferCaches(), harness
+        return LSMBTreeUtil.createLSMTree(harness.getIOManager(), harness.getVirtualBufferCaches(), harness
                 .getFileReference(), harness.getDiskBufferCache(), harness.getDiskFileMapProvider(), typeTraits,
                 cmpFactories, bloomFilterKeyFields, harness.getBoomFilterFalsePositiveRate(), harness.getMergePolicy(),
                 harness.getOperationTracker(), harness.getIOScheduler(), harness.getIOOperationCallback(), true, null,
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/perf/BTreePageSizePerf.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/perf/BTreePageSizePerf.java
index 684ef57..2d355da 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/perf/BTreePageSizePerf.java
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/perf/BTreePageSizePerf.java
@@ -28,7 +28,7 @@
 import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
 import org.apache.hyracks.api.dataflow.value.ITypeTraits;
 import org.apache.hyracks.dataflow.common.data.marshalling.IntegerSerializerDeserializer;
-import org.apache.hyracks.dataflow.common.util.SerdeUtils;
+import org.apache.hyracks.dataflow.common.utils.SerdeUtils;
 import org.apache.hyracks.storage.am.common.datagen.DataGenThread;
 
 public class BTreePageSizePerf {
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/perf/LSMTreeRunner.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/perf/LSMTreeRunner.java
index 694ce44..d4e7886b 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/perf/LSMTreeRunner.java
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/perf/LSMTreeRunner.java
@@ -38,7 +38,7 @@
 import org.apache.hyracks.storage.am.common.datagen.TupleBatch;
 import org.apache.hyracks.storage.am.common.impls.NoOpOperationCallback;
 import org.apache.hyracks.storage.am.lsm.btree.impls.LSMBTree;
-import org.apache.hyracks.storage.am.lsm.btree.util.LSMBTreeUtils;
+import org.apache.hyracks.storage.am.lsm.btree.utils.LSMBTreeUtil;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationScheduler;
 import org.apache.hyracks.storage.am.lsm.common.api.IVirtualBufferCache;
 import org.apache.hyracks.storage.am.lsm.common.impls.AsynchronousScheduler;
@@ -109,7 +109,7 @@
         this.ioScheduler = AsynchronousScheduler.INSTANCE;
         AsynchronousScheduler.INSTANCE.init(threadFactory);
 
-        lsmtree = LSMBTreeUtils.createLSMTree(ioManager, virtualBufferCaches, file, bufferCache, fmp, typeTraits,
+        lsmtree = LSMBTreeUtil.createLSMTree(ioManager, virtualBufferCaches, file, bufferCache, fmp, typeTraits,
                 cmpFactories,
                 bloomFilterKeyFields, bloomFilterFalsePositiveRate, new NoMergePolicy(), new ThreadCountingTracker(),
                 ioScheduler, NoOpIOOperationCallback.INSTANCE, true, null, null, null, null, true,
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/perf/PerfExperiment.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/perf/PerfExperiment.java
index 56c97bf..3f14f62 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/perf/PerfExperiment.java
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/perf/PerfExperiment.java
@@ -28,7 +28,7 @@
 import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
 import org.apache.hyracks.api.dataflow.value.ITypeTraits;
 import org.apache.hyracks.dataflow.common.data.marshalling.IntegerSerializerDeserializer;
-import org.apache.hyracks.dataflow.common.util.SerdeUtils;
+import org.apache.hyracks.dataflow.common.utils.SerdeUtils;
 import org.apache.hyracks.storage.am.common.datagen.DataGenThread;
 
 public class PerfExperiment {
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/tuples/LSMBTreeTuplesTest.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/tuples/LSMBTreeTuplesTest.java
index 4743dee..028a7f9 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/tuples/LSMBTreeTuplesTest.java
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/tuples/LSMBTreeTuplesTest.java
@@ -34,8 +34,8 @@
 import org.apache.hyracks.dataflow.common.data.accessors.ITupleReference;
 import org.apache.hyracks.dataflow.common.data.marshalling.IntegerSerializerDeserializer;
 import org.apache.hyracks.dataflow.common.data.marshalling.UTF8StringSerializerDeserializer;
-import org.apache.hyracks.dataflow.common.util.SerdeUtils;
-import org.apache.hyracks.dataflow.common.util.TupleUtils;
+import org.apache.hyracks.dataflow.common.utils.SerdeUtils;
+import org.apache.hyracks.dataflow.common.utils.TupleUtils;
 import org.apache.hyracks.storage.am.common.datagen.DataGenUtils;
 import org.apache.hyracks.storage.am.common.datagen.IFieldValueGenerator;
 
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/util/LSMBTreeTestContext.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/util/LSMBTreeTestContext.java
index 0d50f5f..d474c02 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/util/LSMBTreeTestContext.java
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/util/LSMBTreeTestContext.java
@@ -28,12 +28,13 @@
 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.util.SerdeUtils;
+import org.apache.hyracks.dataflow.common.utils.SerdeUtils;
 import org.apache.hyracks.storage.am.btree.OrderedIndexTestContext;
 import org.apache.hyracks.storage.am.common.CheckTuple;
 import org.apache.hyracks.storage.am.common.api.IMetadataPageManagerFactory;
 import org.apache.hyracks.storage.am.common.api.ITreeIndex;
 import org.apache.hyracks.storage.am.lsm.btree.impls.LSMBTree;
+import org.apache.hyracks.storage.am.lsm.btree.utils.LSMBTreeUtil;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallback;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationScheduler;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMMergePolicy;
@@ -82,7 +83,7 @@
         for (int i = 0; i < numKeyFields; ++i) {
             bloomFilterKeyFields[i] = i;
         }
-        LSMBTree lsmTree = LSMBTreeUtils.createLSMTree(ioManager, virtualBufferCaches, file, diskBufferCache,
+        LSMBTree lsmTree = LSMBTreeUtil.createLSMTree(ioManager, virtualBufferCaches, file, diskBufferCache,
                 diskFileMapProvider, typeTraits, cmpFactories, bloomFilterKeyFields, bloomFilterFalsePositiveRate,
                 mergePolicy, opTracker, ioScheduler, ioOpCallback, true, null, null, null, null, true,
                 metadataPageManagerFactory);
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/util/LSMBTreeTestHarness.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/util/LSMBTreeTestHarness.java
index 9e7a2b6..ea0af0b 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/util/LSMBTreeTestHarness.java
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/util/LSMBTreeTestHarness.java
@@ -100,7 +100,7 @@
         this.opTracker = new ThreadCountingTracker();
         this.ioOpCallback = NoOpIOOperationCallback.INSTANCE;
         this.numMutableComponents = AccessMethodTestsConfig.LSM_BTREE_NUM_MUTABLE_COMPONENTS;
-        this.metadataPageManagerFactory = new AppendOnlyLinkedMetadataPageManagerFactory();
+        this.metadataPageManagerFactory = AppendOnlyLinkedMetadataPageManagerFactory.INSTANCE;
     }
 
     public void setUp() throws HyracksDataException {
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/org/apache/hyracks/storage/am/lsm/invertedindex/common/LSMInvertedIndexTestHarness.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/org/apache/hyracks/storage/am/lsm/invertedindex/common/LSMInvertedIndexTestHarness.java
index 030b3ed..8f5a6e7 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/org/apache/hyracks/storage/am/lsm/invertedindex/common/LSMInvertedIndexTestHarness.java
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/org/apache/hyracks/storage/am/lsm/invertedindex/common/LSMInvertedIndexTestHarness.java
@@ -84,7 +84,7 @@
     protected String invIndexFileName = "inv_index";
     protected FileReference invIndexFileRef;
     protected IMetadataPageManagerFactory metadataPageManagerFactory =
-            new AppendOnlyLinkedMetadataPageManagerFactory();
+            AppendOnlyLinkedMetadataPageManagerFactory.INSTANCE;
 
     public LSMInvertedIndexTestHarness() {
         this.diskPageSize = AccessMethodTestsConfig.LSM_INVINDEX_DISK_PAGE_SIZE;
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/org/apache/hyracks/storage/am/lsm/invertedindex/multithread/LSMInvertedIndexTestWorker.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/org/apache/hyracks/storage/am/lsm/invertedindex/multithread/LSMInvertedIndexTestWorker.java
index 9187e0e..9e24600 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/org/apache/hyracks/storage/am/lsm/invertedindex/multithread/LSMInvertedIndexTestWorker.java
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/org/apache/hyracks/storage/am/lsm/invertedindex/multithread/LSMInvertedIndexTestWorker.java
@@ -25,7 +25,7 @@
 
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.dataflow.common.data.accessors.ITupleReference;
-import org.apache.hyracks.dataflow.common.util.TupleUtils;
+import org.apache.hyracks.dataflow.common.utils.TupleUtils;
 import org.apache.hyracks.storage.am.btree.impls.RangePredicate;
 import org.apache.hyracks.storage.am.common.AbstractIndexTestWorker;
 import org.apache.hyracks.storage.am.common.TestOperationSelector;
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/org/apache/hyracks/storage/am/lsm/invertedindex/util/LSMInvertedIndexTestContext.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/org/apache/hyracks/storage/am/lsm/invertedindex/util/LSMInvertedIndexTestContext.java
index dd1b602..cae6a7d 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/org/apache/hyracks/storage/am/lsm/invertedindex/util/LSMInvertedIndexTestContext.java
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/org/apache/hyracks/storage/am/lsm/invertedindex/util/LSMInvertedIndexTestContext.java
@@ -33,8 +33,8 @@
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.control.nc.io.IOManager;
 import org.apache.hyracks.dataflow.common.data.accessors.ITupleReference;
-import org.apache.hyracks.dataflow.common.util.SerdeUtils;
-import org.apache.hyracks.dataflow.common.util.TupleUtils;
+import org.apache.hyracks.dataflow.common.utils.SerdeUtils;
+import org.apache.hyracks.dataflow.common.utils.TupleUtils;
 import org.apache.hyracks.storage.am.btree.OrderedIndexTestContext;
 import org.apache.hyracks.storage.am.common.CheckTuple;
 import org.apache.hyracks.storage.am.common.api.IIndex;
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/org/apache/hyracks/storage/am/lsm/rtree/AbstractLSMRTreeExamplesTest.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/org/apache/hyracks/storage/am/lsm/rtree/AbstractLSMRTreeExamplesTest.java
index 5b3c7aa..1c47373 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/org/apache/hyracks/storage/am/lsm/rtree/AbstractLSMRTreeExamplesTest.java
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/org/apache/hyracks/storage/am/lsm/rtree/AbstractLSMRTreeExamplesTest.java
@@ -31,7 +31,7 @@
 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.marshalling.IntegerSerializerDeserializer;
-import org.apache.hyracks.dataflow.common.util.TupleUtils;
+import org.apache.hyracks.dataflow.common.utils.TupleUtils;
 import org.apache.hyracks.storage.am.common.api.IIndexAccessor;
 import org.apache.hyracks.storage.am.common.api.IPrimitiveValueProviderFactory;
 import org.apache.hyracks.storage.am.common.api.ITreeIndex;
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/org/apache/hyracks/storage/am/lsm/rtree/util/LSMRTreeTestContext.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/org/apache/hyracks/storage/am/lsm/rtree/util/LSMRTreeTestContext.java
index 4f16e95..d55daef 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/org/apache/hyracks/storage/am/lsm/rtree/util/LSMRTreeTestContext.java
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/org/apache/hyracks/storage/am/lsm/rtree/util/LSMRTreeTestContext.java
@@ -28,7 +28,7 @@
 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.util.SerdeUtils;
+import org.apache.hyracks.dataflow.common.utils.SerdeUtils;
 import org.apache.hyracks.storage.am.common.api.IMetadataPageManagerFactory;
 import org.apache.hyracks.storage.am.common.api.IPrimitiveValueProviderFactory;
 import org.apache.hyracks.storage.am.common.api.ITreeIndex;
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/org/apache/hyracks/storage/am/lsm/rtree/util/LSMRTreeTestHarness.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/org/apache/hyracks/storage/am/lsm/rtree/util/LSMRTreeTestHarness.java
index 5a5913b..4753cbf 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/org/apache/hyracks/storage/am/lsm/rtree/util/LSMRTreeTestHarness.java
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/org/apache/hyracks/storage/am/lsm/rtree/util/LSMRTreeTestHarness.java
@@ -77,7 +77,7 @@
     protected ILSMMergePolicy mergePolicy;
     protected ILSMOperationTracker opTracker;
     protected IMetadataPageManagerFactory metadataPageManagerFactory =
-            new AppendOnlyLinkedMetadataPageManagerFactory();
+            AppendOnlyLinkedMetadataPageManagerFactory.INSTANCE;
 
     protected final Random rnd = new Random();
     protected final static SimpleDateFormat simpleDateFormat = new SimpleDateFormat("ddMMyy-hhmmssSS");
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/org/apache/hyracks/storage/am/lsm/rtree/util/LSMRTreeWithAntiMatterTuplesTestContext.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/org/apache/hyracks/storage/am/lsm/rtree/util/LSMRTreeWithAntiMatterTuplesTestContext.java
index 5c6f6b7..0653ddf 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/org/apache/hyracks/storage/am/lsm/rtree/util/LSMRTreeWithAntiMatterTuplesTestContext.java
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/org/apache/hyracks/storage/am/lsm/rtree/util/LSMRTreeWithAntiMatterTuplesTestContext.java
@@ -28,7 +28,7 @@
 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.util.SerdeUtils;
+import org.apache.hyracks.dataflow.common.utils.SerdeUtils;
 import org.apache.hyracks.storage.am.common.api.IMetadataPageManagerFactory;
 import org.apache.hyracks.storage.am.common.api.IPrimitiveValueProviderFactory;
 import org.apache.hyracks.storage.am.common.api.ITreeIndex;
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-rtree-test/src/test/java/org/apache/hyracks/storage/am/rtree/RTreeSearchCursorTest.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-rtree-test/src/test/java/org/apache/hyracks/storage/am/rtree/RTreeSearchCursorTest.java
index 6064c1b..256c98f 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-rtree-test/src/test/java/org/apache/hyracks/storage/am/rtree/RTreeSearchCursorTest.java
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-rtree-test/src/test/java/org/apache/hyracks/storage/am/rtree/RTreeSearchCursorTest.java
@@ -32,7 +32,7 @@
 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.marshalling.IntegerSerializerDeserializer;
-import org.apache.hyracks.dataflow.common.util.TupleUtils;
+import org.apache.hyracks.dataflow.common.utils.TupleUtils;
 import org.apache.hyracks.storage.am.common.api.IMetadataPageManager;
 import org.apache.hyracks.storage.am.common.api.IPrimitiveValueProviderFactory;
 import org.apache.hyracks.storage.am.common.api.ITreeIndexAccessor;
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-rtree-test/src/test/java/org/apache/hyracks/storage/am/rtree/utils/RTreeTestContext.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-rtree-test/src/test/java/org/apache/hyracks/storage/am/rtree/utils/RTreeTestContext.java
index fcfa811..a738e2f 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-rtree-test/src/test/java/org/apache/hyracks/storage/am/rtree/utils/RTreeTestContext.java
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-rtree-test/src/test/java/org/apache/hyracks/storage/am/rtree/utils/RTreeTestContext.java
@@ -24,7 +24,7 @@
 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.dataflow.common.util.SerdeUtils;
+import org.apache.hyracks.dataflow.common.utils.SerdeUtils;
 import org.apache.hyracks.storage.am.common.api.IPageManagerFactory;
 import org.apache.hyracks.storage.am.common.api.IPrimitiveValueProviderFactory;
 import org.apache.hyracks.storage.am.common.api.ITreeIndex;