[ASTERIXDB-3601][STO][CONF] Passing storage config to Btrees

- user model changes: no
- storage format changes: no
- interface changes: no

Ext-ref: MB-66306
Change-Id: Ia0881be7822fba488e34e25a6a05f20105cf6749
Reviewed-on: https://asterix-gerrit.ics.uci.edu/c/asterixdb/+/19988
Reviewed-by: Peeyush Gupta <peeyush.gupta@couchbase.com>
Tested-by: Jenkins <jenkins@fulliautomatix.ics.uci.edu>
Integration-Tests: Jenkins <jenkins@fulliautomatix.ics.uci.edu>
diff --git a/asterixdb/asterix-app/src/main/resources/cc.conf b/asterixdb/asterix-app/src/main/resources/cc.conf
index f5a496c..666702d 100644
--- a/asterixdb/asterix-app/src/main/resources/cc.conf
+++ b/asterixdb/asterix-app/src/main/resources/cc.conf
@@ -41,6 +41,7 @@
 storage.memorycomponent.globalbudget=512MB
 storage.io.scheduler=greedy
 storage.filtered.memorycomponent.max.size=16MB
+storage.max.columns.in.zeroth.segment=800
 
 [cc]
 address = 127.0.0.1
diff --git a/asterixdb/asterix-app/src/main/resources/cc2.conf b/asterixdb/asterix-app/src/main/resources/cc2.conf
index bb2e457..2f437a4 100644
--- a/asterixdb/asterix-app/src/main/resources/cc2.conf
+++ b/asterixdb/asterix-app/src/main/resources/cc2.conf
@@ -37,6 +37,7 @@
 jvm.args=-Xmx4096m -Dnode.Resolver="org.apache.asterix.external.util.IdentitiyResolverFactory"
 storage.buffercache.size=128MB
 storage.memorycomponent.globalbudget=512MB
+storage.max.columns.in.zeroth.segment=800
 
 [cc]
 address = 127.0.0.1
diff --git a/asterixdb/asterix-app/src/main/resources/cc3.conf b/asterixdb/asterix-app/src/main/resources/cc3.conf
index cc83da0..c8903cb 100644
--- a/asterixdb/asterix-app/src/main/resources/cc3.conf
+++ b/asterixdb/asterix-app/src/main/resources/cc3.conf
@@ -37,6 +37,7 @@
 jvm.args=-Xmx4096m -Dnode.Resolver="org.apache.asterix.external.util.IdentitiyResolverFactory"
 storage.buffercache.size=128MB
 storage.memorycomponent.globalbudget=512MB
+storage.max.columns.in.zeroth.segment=800
 
 [cc]
 address = 127.0.0.1
diff --git a/asterixdb/asterix-app/src/test/resources/cc-analyze.conf b/asterixdb/asterix-app/src/test/resources/cc-analyze.conf
index c65bca8..7e5856a 100644
--- a/asterixdb/asterix-app/src/test/resources/cc-analyze.conf
+++ b/asterixdb/asterix-app/src/test/resources/cc-analyze.conf
@@ -41,6 +41,7 @@
 jvm.args=-Xmx4096m -Dnode.Resolver="org.apache.asterix.external.util.IdentitiyResolverFactory"
 storage.buffercache.size=128MB
 storage.memorycomponent.globalbudget=512MB
+storage.max.columns.in.zeroth.segment=800
 
 [cc]
 address = 127.0.0.1
diff --git a/asterixdb/asterix-app/src/test/resources/cc-cloud-storage-azblob.conf b/asterixdb/asterix-app/src/test/resources/cc-cloud-storage-azblob.conf
index 4af4fd0..409b99b 100644
--- a/asterixdb/asterix-app/src/test/resources/cc-cloud-storage-azblob.conf
+++ b/asterixdb/asterix-app/src/test/resources/cc-cloud-storage-azblob.conf
@@ -41,6 +41,7 @@
 jvm.args=-Xmx4096m -Dnode.Resolver="org.apache.asterix.external.util.IdentitiyResolverFactory"
 storage.buffercache.size=128MB
 storage.memorycomponent.globalbudget=512MB
+storage.max.columns.in.zeroth.segment=800
 
 [cc]
 address = 127.0.0.1
diff --git a/asterixdb/asterix-app/src/test/resources/cc-cloud-storage-gcs.conf b/asterixdb/asterix-app/src/test/resources/cc-cloud-storage-gcs.conf
index 376252e..ee65f3c 100644
--- a/asterixdb/asterix-app/src/test/resources/cc-cloud-storage-gcs.conf
+++ b/asterixdb/asterix-app/src/test/resources/cc-cloud-storage-gcs.conf
@@ -41,6 +41,7 @@
 jvm.args=-Xmx4096m -Dnode.Resolver="org.apache.asterix.external.util.IdentitiyResolverFactory"
 storage.buffercache.size=128MB
 storage.memorycomponent.globalbudget=512MB
+storage.max.columns.in.zeroth.segment=800
 
 [cc]
 address = 127.0.0.1
diff --git a/asterixdb/asterix-app/src/test/resources/cc-cloud-storage-main.conf b/asterixdb/asterix-app/src/test/resources/cc-cloud-storage-main.conf
index a9f2aac..8745d9d 100644
--- a/asterixdb/asterix-app/src/test/resources/cc-cloud-storage-main.conf
+++ b/asterixdb/asterix-app/src/test/resources/cc-cloud-storage-main.conf
@@ -41,6 +41,7 @@
 jvm.args=-Xmx4096m --add-opens=jdk.management/com.sun.management.internal=ALL-UNNAMED --add-opens=java.management/sun.management=ALL-UNNAMED --add-opens=java.base/java.lang=ALL-UNNAMED --add-opens=java.base/java.nio=ALL-UNNAMED --add-opens=java.base/java.util=ALL-UNNAMED --add-opens=java.base/java.io=ALL-UNNAMED --add-opens=java.base/sun.nio.ch=ALL-UNNAMED -Dnode.Resolver="org.apache.asterix.external.util.IdentitiyResolverFactory"
 storage.buffercache.size=128MB
 storage.memorycomponent.globalbudget=512MB
+storage.max.columns.in.zeroth.segment=800
 
 [cc]
 address = 127.0.0.1
diff --git a/asterixdb/asterix-app/src/test/resources/cc-cloud-storage.conf b/asterixdb/asterix-app/src/test/resources/cc-cloud-storage.conf
index 64fc40f..551d2c4 100644
--- a/asterixdb/asterix-app/src/test/resources/cc-cloud-storage.conf
+++ b/asterixdb/asterix-app/src/test/resources/cc-cloud-storage.conf
@@ -41,6 +41,7 @@
 jvm.args=-Xmx4096m --add-opens=jdk.management/com.sun.management.internal=ALL-UNNAMED --add-opens=java.management/sun.management=ALL-UNNAMED --add-opens=java.base/java.lang=ALL-UNNAMED --add-opens=java.base/java.nio=ALL-UNNAMED --add-opens=java.base/java.util=ALL-UNNAMED --add-opens=java.base/java.io=ALL-UNNAMED --add-opens=java.base/sun.nio.ch=ALL-UNNAMED -Dnode.Resolver="org.apache.asterix.external.util.IdentitiyResolverFactory"
 storage.buffercache.size=128MB
 storage.memorycomponent.globalbudget=512MB
+storage.max.columns.in.zeroth.segment=800
 
 [cc]
 address = 127.0.0.1
diff --git a/asterixdb/asterix-app/src/test/resources/cc-columnar.conf b/asterixdb/asterix-app/src/test/resources/cc-columnar.conf
index e01bbbd..9be2c16 100644
--- a/asterixdb/asterix-app/src/test/resources/cc-columnar.conf
+++ b/asterixdb/asterix-app/src/test/resources/cc-columnar.conf
@@ -40,6 +40,7 @@
 jvm.args=-Xmx4096m -Dnode.Resolver="org.apache.asterix.external.util.IdentitiyResolverFactory"
 storage.buffercache.size=128MB
 storage.memorycomponent.globalbudget=512MB
+storage.max.columns.in.zeroth.segment=800
 
 [cc]
 address = 127.0.0.1
diff --git a/asterixdb/asterix-app/src/test/resources/cc-single-partition.conf b/asterixdb/asterix-app/src/test/resources/cc-single-partition.conf
index aeb9361..d5837ad 100644
--- a/asterixdb/asterix-app/src/test/resources/cc-single-partition.conf
+++ b/asterixdb/asterix-app/src/test/resources/cc-single-partition.conf
@@ -32,6 +32,7 @@
 jvm.args=-Xmx4096m -Dnode.Resolver="org.apache.asterix.external.util.IdentitiyResolverFactory"
 storage.buffercache.size=128MB
 storage.memorycomponent.globalbudget=512MB
+storage.max.columns.in.zeroth.segment=800
 
 [cc]
 address = 127.0.0.1
diff --git a/asterixdb/asterix-app/src/test/resources/cc-single.conf b/asterixdb/asterix-app/src/test/resources/cc-single.conf
index 76b29d7..6acd5b5 100644
--- a/asterixdb/asterix-app/src/test/resources/cc-single.conf
+++ b/asterixdb/asterix-app/src/test/resources/cc-single.conf
@@ -29,6 +29,7 @@
 jvm.args=-Xmx4096m -Dnode.Resolver="org.apache.asterix.external.util.IdentitiyResolverFactory"
 storage.buffercache.size=128MB
 storage.memorycomponent.globalbudget=512MB
+storage.max.columns.in.zeroth.segment=800
 
 [cc]
 address = 127.0.0.1
diff --git a/asterixdb/asterix-app/src/test/resources/cc-ssl.conf b/asterixdb/asterix-app/src/test/resources/cc-ssl.conf
index 086944b..735f3a3 100644
--- a/asterixdb/asterix-app/src/test/resources/cc-ssl.conf
+++ b/asterixdb/asterix-app/src/test/resources/cc-ssl.conf
@@ -45,6 +45,7 @@
 jvm.args=-Xmx4096m -Dnode.Resolver="org.apache.asterix.external.util.IdentitiyResolverFactory"
 storage.buffercache.size=128MB
 storage.memorycomponent.globalbudget=512MB
+storage.max.columns.in.zeroth.segment=800
 
 [cc]
 address = 127.0.0.1
diff --git a/asterixdb/asterix-app/src/test/resources/cc.conf b/asterixdb/asterix-app/src/test/resources/cc.conf
index 075d865..bdcd537 100644
--- a/asterixdb/asterix-app/src/test/resources/cc.conf
+++ b/asterixdb/asterix-app/src/test/resources/cc.conf
@@ -41,6 +41,7 @@
 jvm.args=-Xmx4096m -Dnode.Resolver="org.apache.asterix.external.util.IdentitiyResolverFactory"
 storage.buffercache.size=128MB
 storage.memorycomponent.globalbudget=512MB
+storage.max.columns.in.zeroth.segment=800
 
 [cc]
 address = 127.0.0.1
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/controllers/NCConfig.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/controllers/NCConfig.java
index 44b2fdc..0e7de33 100644
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/controllers/NCConfig.java
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/controllers/NCConfig.java
@@ -106,7 +106,9 @@
                 OptionTypes.STRING,
                 (Function<IApplicationConfig, String>) appConfig -> FileUtil
                         .joinPath(appConfig.getString(ControllerConfig.Option.DEFAULT_DIR), "passwd"),
-                ControllerConfig.Option.DEFAULT_DIR.cmdline() + "/passwd");
+                ControllerConfig.Option.DEFAULT_DIR.cmdline() + "/passwd"),
+        STORAGE_MAX_COLUMNS_IN_ZEROTH_SEGMENT(INTEGER_BYTE_UNIT, 5000),
+        ADAPTIVE_PAGE_ZERO_WRITER_SELECTION(BOOLEAN, false);
 
         private final IOptionType parser;
         private final String defaultValueDescription;
@@ -257,6 +259,11 @@
                     return "Path to HTTP basic credentials";
                 case ABORT_TASKS_TIMEOUT:
                     return "The maximum time to wait for the tasks to be aborted";
+                case STORAGE_MAX_COLUMNS_IN_ZEROTH_SEGMENT:
+                    return "The maximum number of columns in zero segment (default: 5000).";
+                case ADAPTIVE_PAGE_ZERO_WRITER_SELECTION:
+                    return "The config to choose between writers dynamically (default: false, i.e. use the "
+                            + "default writer for all segments).";
                 default:
                     throw new IllegalStateException("Not yet implemented: " + this);
             }
@@ -636,4 +643,11 @@
         return appConfig.getInt(Option.ABORT_TASKS_TIMEOUT);
     }
 
+    public int getStorageMaxColumnsInZerothSegment() {
+        return appConfig.getInt(Option.STORAGE_MAX_COLUMNS_IN_ZEROTH_SEGMENT);
+    }
+
+    public boolean isAdaptivePageZeroWriterSelection() {
+        return appConfig.getBoolean(Option.ADAPTIVE_PAGE_ZERO_WRITER_SELECTION);
+    }
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree-column/pom.xml b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree-column/pom.xml
index bbd8d87..72eae65 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree-column/pom.xml
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree-column/pom.xml
@@ -94,6 +94,11 @@
     </dependency>
     <dependency>
       <groupId>org.apache.hyracks</groupId>
+      <artifactId>hyracks-control-common</artifactId>
+      <version>${project.version}</version>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hyracks</groupId>
       <artifactId>hyracks-cloud</artifactId>
       <version>${project.version}</version>
     </dependency>
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree-column/src/main/java/org/apache/hyracks/storage/am/lsm/btree/column/dataflow/LSMColumnBTreeLocalResource.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree-column/src/main/java/org/apache/hyracks/storage/am/lsm/btree/column/dataflow/LSMColumnBTreeLocalResource.java
index 7ea4e35..65e19bb 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree-column/src/main/java/org/apache/hyracks/storage/am/lsm/btree/column/dataflow/LSMColumnBTreeLocalResource.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree-column/src/main/java/org/apache/hyracks/storage/am/lsm/btree/column/dataflow/LSMColumnBTreeLocalResource.java
@@ -30,6 +30,8 @@
 import org.apache.hyracks.api.io.IIOManager;
 import org.apache.hyracks.api.io.IJsonSerializable;
 import org.apache.hyracks.api.io.IPersistedResourceRegistry;
+import org.apache.hyracks.control.common.controllers.NCConfig;
+import org.apache.hyracks.control.nc.NodeControllerService;
 import org.apache.hyracks.storage.am.common.api.IMetadataPageManagerFactory;
 import org.apache.hyracks.storage.am.common.api.INullIntrospector;
 import org.apache.hyracks.storage.am.lsm.btree.column.api.IColumnManagerFactory;
@@ -83,15 +85,16 @@
 
     @Override
     public ILSMIndex createInstance(INCServiceContext serviceCtx) throws HyracksDataException {
+        NCConfig config = ((NodeControllerService) serviceCtx.getControllerService()).getConfiguration();
         IIOManager ioManager = storageManager.getIoManager(serviceCtx);
         FileReference file = ioManager.resolve(path);
         List<IVirtualBufferCache> vbcs = vbcProvider.getVirtualBufferCaches(serviceCtx, file);
         ioOpCallbackFactory.initialize(serviceCtx, this);
         pageWriteCallbackFactory.initialize(serviceCtx, this);
         IDiskCacheMonitoringService diskCacheService = storageManager.getDiskCacheMonitoringService(serviceCtx);
-        return LSMColumnBTreeUtil.createLSMTree(ioManager, vbcs, file, storageManager.getBufferCache(serviceCtx),
-                typeTraits, cmpFactories, bloomFilterKeyFields, bloomFilterFalsePositiveRate,
-                mergePolicyFactory.createMergePolicy(mergePolicyProperties, serviceCtx),
+        return LSMColumnBTreeUtil.createLSMTree(config, ioManager, vbcs, file,
+                storageManager.getBufferCache(serviceCtx), typeTraits, cmpFactories, bloomFilterKeyFields,
+                bloomFilterFalsePositiveRate, mergePolicyFactory.createMergePolicy(mergePolicyProperties, serviceCtx),
                 opTrackerProvider.getOperationTracker(serviceCtx, this), ioSchedulerProvider.getIoScheduler(serviceCtx),
                 ioOpCallbackFactory, pageWriteCallbackFactory, btreeFields, metadataPageManagerFactory, false,
                 serviceCtx.getTracer(), compressorDecompressorFactory, nullTypeTraits, nullIntrospector,
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree-column/src/main/java/org/apache/hyracks/storage/am/lsm/btree/column/impls/btree/ColumnBTree.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree-column/src/main/java/org/apache/hyracks/storage/am/lsm/btree/column/impls/btree/ColumnBTree.java
index a7a07be..a863d13 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree-column/src/main/java/org/apache/hyracks/storage/am/lsm/btree/column/impls/btree/ColumnBTree.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree-column/src/main/java/org/apache/hyracks/storage/am/lsm/btree/column/impls/btree/ColumnBTree.java
@@ -22,6 +22,7 @@
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.api.io.FileReference;
 import org.apache.hyracks.api.util.HyracksConstants;
+import org.apache.hyracks.control.common.controllers.NCConfig;
 import org.apache.hyracks.storage.am.btree.impls.DiskBTree;
 import org.apache.hyracks.storage.am.common.api.IPageManager;
 import org.apache.hyracks.storage.am.common.api.ITreeIndexCursor;
@@ -51,12 +52,14 @@
         throw new IllegalAccessError("Missing write column metadata");
     }
 
-    public IIndexBulkLoader createBulkLoader(float fillFactor, boolean verifyInput, IPageWriteCallback callback,
-            IColumnMetadata columnMetadata, IColumnWriteContext writeContext) throws HyracksDataException {
+    public IIndexBulkLoader createBulkLoader(NCConfig storageConfig, float fillFactor, boolean verifyInput,
+            IPageWriteCallback callback, IColumnMetadata columnMetadata, IColumnWriteContext writeContext)
+            throws HyracksDataException {
         ColumnBTreeLeafFrameFactory columnLeafFrameFactory = (ColumnBTreeLeafFrameFactory) leafFrameFactory;
         ColumnBTreeWriteLeafFrame writeLeafFrame =
                 columnLeafFrameFactory.createWriterFrame(columnMetadata, writeContext);
-        return new ColumnBTreeBulkloader(fillFactor, verifyInput, callback, this, writeLeafFrame, writeContext);
+        return new ColumnBTreeBulkloader(storageConfig, fillFactor, verifyInput, callback, this, writeLeafFrame,
+                writeContext);
     }
 
     @Override
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree-column/src/main/java/org/apache/hyracks/storage/am/lsm/btree/column/impls/btree/ColumnBTreeBulkloader.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree-column/src/main/java/org/apache/hyracks/storage/am/lsm/btree/column/impls/btree/ColumnBTreeBulkloader.java
index 88fe5bf..6826f3b 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree-column/src/main/java/org/apache/hyracks/storage/am/lsm/btree/column/impls/btree/ColumnBTreeBulkloader.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree-column/src/main/java/org/apache/hyracks/storage/am/lsm/btree/column/impls/btree/ColumnBTreeBulkloader.java
@@ -23,6 +23,7 @@
 import java.util.List;
 
 import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.control.common.controllers.NCConfig;
 import org.apache.hyracks.dataflow.common.data.accessors.ITupleReference;
 import org.apache.hyracks.storage.am.btree.impls.BTreeNSMBulkLoader;
 import org.apache.hyracks.storage.am.btree.impls.BTreeSplitKey;
@@ -69,8 +70,9 @@
     private int maxTupleCount;
     private int lastRequiredFreeSpace;
 
-    public ColumnBTreeBulkloader(float fillFactor, boolean verifyInput, IPageWriteCallback callback, ITreeIndex index,
-            ITreeIndexFrame leafFrame, IBufferCacheWriteContext writeContext) throws HyracksDataException {
+    public ColumnBTreeBulkloader(NCConfig storageConfig, float fillFactor, boolean verifyInput,
+            IPageWriteCallback callback, ITreeIndex index, ITreeIndexFrame leafFrame,
+            IBufferCacheWriteContext writeContext) throws HyracksDataException {
         super(fillFactor, verifyInput, callback, index, leafFrame, writeContext);
         columnsPages = new ArrayList<>();
         pageZeroSegments = new ArrayList<>();
@@ -84,8 +86,8 @@
         setLowKey = true;
 
         // Writer config
-        maxColumnsInPageZerothSegment = 40; // setting a lower value for testing, should be coming from config.
-        adaptiveWriter = false; // should be coming from config.
+        maxColumnsInPageZerothSegment = storageConfig.getStorageMaxColumnsInZerothSegment();
+        adaptiveWriter = storageConfig.isAdaptivePageZeroWriterSelection();
 
         // For logging. Starts with 1 for page0
         numberOfPagesInCurrentLeafNode = 1;
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree-column/src/main/java/org/apache/hyracks/storage/am/lsm/btree/column/impls/lsm/LSMColumnBTree.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree-column/src/main/java/org/apache/hyracks/storage/am/lsm/btree/column/impls/lsm/LSMColumnBTree.java
index d95678e..0ba4509 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree-column/src/main/java/org/apache/hyracks/storage/am/lsm/btree/column/impls/lsm/LSMColumnBTree.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree-column/src/main/java/org/apache/hyracks/storage/am/lsm/btree/column/impls/lsm/LSMColumnBTree.java
@@ -23,6 +23,7 @@
 import org.apache.hyracks.api.dataflow.value.IBinaryComparatorFactory;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.api.io.IIOManager;
+import org.apache.hyracks.control.common.controllers.NCConfig;
 import org.apache.hyracks.storage.am.common.api.IExtendedModificationOperationCallback;
 import org.apache.hyracks.storage.am.common.api.ITreeIndexFrameFactory;
 import org.apache.hyracks.storage.am.lsm.btree.column.api.IColumnManager;
@@ -64,7 +65,7 @@
      */
     private IColumnMetadata columnMetadata;
 
-    public LSMColumnBTree(IIOManager ioManager, List<IVirtualBufferCache> virtualBufferCaches,
+    public LSMColumnBTree(NCConfig storageConfig, IIOManager ioManager, List<IVirtualBufferCache> virtualBufferCaches,
             ITreeIndexFrameFactory interiorFrameFactory, ITreeIndexFrameFactory insertLeafFrameFactory,
             ITreeIndexFrameFactory deleteLeafFrameFactory, IBufferCache diskBufferCache,
             ILSMIndexFileManager fileManager, ILSMDiskComponentFactory componentFactory,
@@ -74,9 +75,9 @@
             ILSMIOOperationCallbackFactory ioOpCallbackFactory, ILSMPageWriteCallbackFactory pageWriteCallbackFactory,
             int[] btreeFields, ITracer tracer, IColumnManager columnManager, boolean atomic,
             IColumnIndexDiskCacheManager diskCacheManager) throws HyracksDataException {
-        super(ioManager, virtualBufferCaches, interiorFrameFactory, insertLeafFrameFactory, deleteLeafFrameFactory,
-                diskBufferCache, fileManager, componentFactory, bulkloadComponentFactory, null, null, null,
-                bloomFilterFalsePositiveRate, fieldCount, cmpFactories, mergePolicy, opTracker, ioScheduler,
+        super(storageConfig, ioManager, virtualBufferCaches, interiorFrameFactory, insertLeafFrameFactory,
+                deleteLeafFrameFactory, diskBufferCache, fileManager, componentFactory, bulkloadComponentFactory, null,
+                null, null, bloomFilterFalsePositiveRate, fieldCount, cmpFactories, mergePolicy, opTracker, ioScheduler,
                 ioOpCallbackFactory, pageWriteCallbackFactory, true, true, btreeFields, null, true, false, tracer,
                 atomic);
         this.columnManager = columnManager;
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree-column/src/main/java/org/apache/hyracks/storage/am/lsm/btree/column/impls/lsm/LSMColumnBTreeWithBloomFilterDiskComponent.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree-column/src/main/java/org/apache/hyracks/storage/am/lsm/btree/column/impls/lsm/LSMColumnBTreeWithBloomFilterDiskComponent.java
index 6dde7e9..154aba9 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree-column/src/main/java/org/apache/hyracks/storage/am/lsm/btree/column/impls/lsm/LSMColumnBTreeWithBloomFilterDiskComponent.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree-column/src/main/java/org/apache/hyracks/storage/am/lsm/btree/column/impls/lsm/LSMColumnBTreeWithBloomFilterDiskComponent.java
@@ -21,6 +21,7 @@
 import java.util.List;
 
 import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.control.common.controllers.NCConfig;
 import org.apache.hyracks.data.std.api.IValueReference;
 import org.apache.hyracks.storage.am.bloomfilter.impls.BloomFilter;
 import org.apache.hyracks.storage.am.btree.impls.BTree;
@@ -50,8 +51,8 @@
     }
 
     @Override
-    public ChainedLSMDiskComponentBulkLoader createBulkLoader(ILSMIOOperation operation, float fillFactor,
-            boolean verifyInput, long numElementsHint, boolean checkIfEmptyIndex, boolean withFilter,
+    public ChainedLSMDiskComponentBulkLoader createBulkLoader(NCConfig storageConfig, ILSMIOOperation operation,
+            float fillFactor, boolean verifyInput, long numElementsHint, boolean checkIfEmptyIndex, boolean withFilter,
             boolean cleanupEmptyComponent, IPageWriteCallback callback) throws HyracksDataException {
         ChainedLSMDiskComponentBulkLoader chainedBulkLoader =
                 new ChainedLSMDiskComponentBulkLoader(operation, this, cleanupEmptyComponent);
@@ -60,7 +61,8 @@
             chainedBulkLoader.addBulkLoader(createFilterBulkLoader());
         }
         //Add index bulkloader
-        chainedBulkLoader.addBulkLoader(createColumnIndexBulkLoader(operation, fillFactor, verifyInput, callback));
+        chainedBulkLoader.addBulkLoader(
+                createColumnIndexBulkLoader(storageConfig, operation, fillFactor, verifyInput, callback));
 
         if (numElementsHint > 0) {
             chainedBulkLoader.addBulkLoader(createBloomFilterBulkLoader(numElementsHint, callback));
@@ -70,8 +72,8 @@
         return chainedBulkLoader;
     }
 
-    private IChainedComponentBulkLoader createColumnIndexBulkLoader(ILSMIOOperation operation, float fillFactor,
-            boolean verifyInput, IPageWriteCallback callback) throws HyracksDataException {
+    private IChainedComponentBulkLoader createColumnIndexBulkLoader(NCConfig storageConfig, ILSMIOOperation operation,
+            float fillFactor, boolean verifyInput, IPageWriteCallback callback) throws HyracksDataException {
         LSMIOOperationType operationType = operation.getIOOperationType();
         LSMColumnBTree lsmColumnBTree = (LSMColumnBTree) getLsmIndex();
         ColumnBTree columnBTree = (ColumnBTree) getIndex();
@@ -91,8 +93,8 @@
         int numberOfColumns = columnMetadata.getNumberOfColumns();
         IColumnIndexDiskCacheManager diskCacheManager = lsmColumnBTree.getDiskCacheManager();
         IColumnWriteContext writeContext = diskCacheManager.createWriteContext(numberOfColumns, operationType);
-        IIndexBulkLoader bulkLoader =
-                columnBTree.createBulkLoader(fillFactor, verifyInput, callback, columnMetadata, writeContext);
+        IIndexBulkLoader bulkLoader = columnBTree.createBulkLoader(storageConfig, fillFactor, verifyInput, callback,
+                columnMetadata, writeContext);
         return new LSMColumnIndexBulkloader(bulkLoader, columnMetadata, getMetadata());
     }
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree-column/src/main/java/org/apache/hyracks/storage/am/lsm/btree/column/utils/LSMColumnBTreeUtil.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree-column/src/main/java/org/apache/hyracks/storage/am/lsm/btree/column/utils/LSMColumnBTreeUtil.java
index 04b4984..c8815ae 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree-column/src/main/java/org/apache/hyracks/storage/am/lsm/btree/column/utils/LSMColumnBTreeUtil.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree-column/src/main/java/org/apache/hyracks/storage/am/lsm/btree/column/utils/LSMColumnBTreeUtil.java
@@ -26,6 +26,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.control.common.controllers.NCConfig;
 import org.apache.hyracks.storage.am.bloomfilter.impls.BloomFilterFactory;
 import org.apache.hyracks.storage.am.btree.frames.BTreeNSMInteriorFrameFactory;
 import org.apache.hyracks.storage.am.btree.frames.BTreeNSMLeafFrameFactory;
@@ -61,12 +62,13 @@
 
 public class LSMColumnBTreeUtil {
 
-    public static LSMBTree createLSMTree(IIOManager ioManager, List<IVirtualBufferCache> virtualBufferCaches,
-            FileReference file, IBufferCache diskBufferCache, ITypeTraits[] typeTraits,
-            IBinaryComparatorFactory[] cmpFactories, int[] bloomFilterKeyFields, double bloomFilterFalsePositiveRate,
-            ILSMMergePolicy mergePolicy, ILSMOperationTracker opTracker, ILSMIOOperationScheduler ioScheduler,
-            ILSMIOOperationCallbackFactory ioOpCallbackFactory, ILSMPageWriteCallbackFactory pageWriteCallbackFactory,
-            int[] btreeFields, IMetadataPageManagerFactory freePageManagerFactory, boolean updateAware, ITracer tracer,
+    public static LSMBTree createLSMTree(NCConfig storageConfig, IIOManager ioManager,
+            List<IVirtualBufferCache> virtualBufferCaches, FileReference file, IBufferCache diskBufferCache,
+            ITypeTraits[] typeTraits, IBinaryComparatorFactory[] cmpFactories, int[] bloomFilterKeyFields,
+            double bloomFilterFalsePositiveRate, ILSMMergePolicy mergePolicy, ILSMOperationTracker opTracker,
+            ILSMIOOperationScheduler ioScheduler, ILSMIOOperationCallbackFactory ioOpCallbackFactory,
+            ILSMPageWriteCallbackFactory pageWriteCallbackFactory, int[] btreeFields,
+            IMetadataPageManagerFactory freePageManagerFactory, boolean updateAware, ITracer tracer,
             ICompressorDecompressorFactory compressorDecompressorFactory, ITypeTraits nullTypeTraits,
             INullIntrospector nullIntrospector, IColumnManagerFactory columnManagerFactory, boolean atomic,
             IDiskCacheMonitoringService diskCacheService) throws HyracksDataException {
@@ -121,10 +123,10 @@
         ILSMDiskComponentFactory bulkLoadComponentFactory =
                 new LSMColumnBTreeWithBloomFilterDiskComponentFactory(bulkloadBTreeFactory, bloomFilterFactory);
 
-        return new LSMColumnBTree(ioManager, virtualBufferCaches, interiorFrameFactory, insertLeafFrameFactory,
-                deleteLeafFrameFactory, diskBufferCache, fileNameManager, flushComponentFactory, mergeComponentFactory,
-                bulkLoadComponentFactory, bloomFilterFalsePositiveRate, typeTraits.length, cmpFactories, mergePolicy,
-                opTracker, ioScheduler, ioOpCallbackFactory, pageWriteCallbackFactory, btreeFields, tracer,
-                columnManager, atomic, diskCacheManager);
+        return new LSMColumnBTree(storageConfig, ioManager, virtualBufferCaches, interiorFrameFactory,
+                insertLeafFrameFactory, deleteLeafFrameFactory, diskBufferCache, fileNameManager, flushComponentFactory,
+                mergeComponentFactory, bulkLoadComponentFactory, bloomFilterFalsePositiveRate, typeTraits.length,
+                cmpFactories, mergePolicy, opTracker, ioScheduler, ioOpCallbackFactory, pageWriteCallbackFactory,
+                btreeFields, tracer, columnManager, atomic, diskCacheManager);
     }
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/pom.xml b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/pom.xml
index 958923f..fec254f 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/pom.xml
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/pom.xml
@@ -95,5 +95,15 @@
       <groupId>org.apache.logging.log4j</groupId>
       <artifactId>log4j-api</artifactId>
     </dependency>
+    <dependency>
+      <groupId>org.apache.hyracks</groupId>
+      <artifactId>hyracks-control-common</artifactId>
+      <version>${project.version}</version>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hyracks</groupId>
+      <artifactId>hyracks-control-nc</artifactId>
+      <version>${project.version}</version>
+    </dependency>
   </dependencies>
 </project>
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/dataflow/LSMBTreeLocalResource.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/dataflow/LSMBTreeLocalResource.java
index 43555ca..d5c7e72 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/dataflow/LSMBTreeLocalResource.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/dataflow/LSMBTreeLocalResource.java
@@ -30,6 +30,8 @@
 import org.apache.hyracks.api.io.IIOManager;
 import org.apache.hyracks.api.io.IJsonSerializable;
 import org.apache.hyracks.api.io.IPersistedResourceRegistry;
+import org.apache.hyracks.control.common.controllers.NCConfig;
+import org.apache.hyracks.control.nc.NodeControllerService;
 import org.apache.hyracks.storage.am.common.api.IMetadataPageManagerFactory;
 import org.apache.hyracks.storage.am.common.api.INullIntrospector;
 import org.apache.hyracks.storage.am.lsm.btree.utils.LSMBTreeUtil;
@@ -105,6 +107,7 @@
 
     @Override
     public ILSMIndex createInstance(INCServiceContext serviceCtx) throws HyracksDataException {
+        NCConfig storageConfig = ((NodeControllerService) serviceCtx.getControllerService()).getConfiguration();
         IIOManager ioManager = storageManager.getIoManager(serviceCtx);
         FileReference file = ioManager.resolve(path);
         List<IVirtualBufferCache> vbcs = vbcProvider.getVirtualBufferCaches(serviceCtx, file);
@@ -112,9 +115,9 @@
         pageWriteCallbackFactory.initialize(serviceCtx, this);
         //TODO: enable updateAwareness for secondary LSMBTree indexes
         boolean updateAware = false;
-        return LSMBTreeUtil.createLSMTree(ioManager, vbcs, file, storageManager.getBufferCache(serviceCtx), typeTraits,
-                cmpFactories, bloomFilterKeyFields, bloomFilterFalsePositiveRate,
-                mergePolicyFactory.createMergePolicy(mergePolicyProperties, serviceCtx),
+        return LSMBTreeUtil.createLSMTree(storageConfig, ioManager, vbcs, file,
+                storageManager.getBufferCache(serviceCtx), typeTraits, cmpFactories, bloomFilterKeyFields,
+                bloomFilterFalsePositiveRate, mergePolicyFactory.createMergePolicy(mergePolicyProperties, serviceCtx),
                 opTrackerProvider.getOperationTracker(serviceCtx, this), ioSchedulerProvider.getIoScheduler(serviceCtx),
                 ioOpCallbackFactory, pageWriteCallbackFactory, isPrimary, filterTypeTraits, filterCmpFactories,
                 btreeFields, filterFields, durable, metadataPageManagerFactory, updateAware, serviceCtx.getTracer(),
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTree.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTree.java
index 3526d2f..a720312 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTree.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTree.java
@@ -27,6 +27,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.control.common.controllers.NCConfig;
 import org.apache.hyracks.data.std.primitive.IntegerPointable;
 import org.apache.hyracks.dataflow.common.data.accessors.ITupleReference;
 import org.apache.hyracks.storage.am.btree.impls.BTree;
@@ -90,7 +91,7 @@
     // Primary and Primary Key LSMBTree has a Bloomfilter, but Secondary one doesn't have.
     private final boolean hasBloomFilter;
 
-    public LSMBTree(IIOManager ioManager, List<IVirtualBufferCache> virtualBufferCaches,
+    public LSMBTree(NCConfig storageConfig, IIOManager ioManager, List<IVirtualBufferCache> virtualBufferCaches,
             ITreeIndexFrameFactory interiorFrameFactory, ITreeIndexFrameFactory insertLeafFrameFactory,
             ITreeIndexFrameFactory deleteLeafFrameFactory, IBufferCache diskBufferCache,
             ILSMIndexFileManager fileManager, ILSMDiskComponentFactory componentFactory,
@@ -101,8 +102,8 @@
             ILSMIOOperationCallbackFactory ioOpCallbackFactory, ILSMPageWriteCallbackFactory pageWriteCallbackFactory,
             boolean needKeyDupCheck, boolean hasBloomFilter, int[] btreeFields, int[] filterFields, boolean durable,
             boolean updateAware, ITracer tracer, boolean atomic) throws HyracksDataException {
-        super(ioManager, virtualBufferCaches, diskBufferCache, fileManager, bloomFilterFalsePositiveRate, mergePolicy,
-                opTracker, ioScheduler, ioOpCallbackFactory, pageWriteCallbackFactory, componentFactory,
+        super(storageConfig, ioManager, virtualBufferCaches, diskBufferCache, fileManager, bloomFilterFalsePositiveRate,
+                mergePolicy, opTracker, ioScheduler, ioOpCallbackFactory, pageWriteCallbackFactory, componentFactory,
                 bulkLoadComponentFactory, filterFrameFactory, filterManager, filterFields, durable, filterHelper,
                 btreeFields, tracer, atomic);
         this.insertLeafFrameFactory = insertLeafFrameFactory;
@@ -265,8 +266,8 @@
                 }
                 component = createDiskComponent(componentFactory, flushOp.getTarget(), null,
                         flushOp.getBloomFilterTarget(), true);
-                componentBulkLoader = component.createBulkLoader(operation, 1.0f, false, numElements, false, false,
-                        false, pageWriteCallbackFactory.createPageWriteCallback());
+                componentBulkLoader = component.createBulkLoader(storageConfig, operation, 1.0f, false, numElements,
+                        false, false, false, pageWriteCallbackFactory.createPageWriteCallback());
                 IIndexCursor scanCursor = accessor.createSearchCursor(false);
                 accessor.search(scanCursor, nullPred);
                 try {
@@ -335,8 +336,8 @@
                     mergedComponent = createDiskComponent(getMergeComponentFactory(), mergeOp.getTarget(), null,
                             mergeOp.getBloomFilterTarget(), true);
                     IPageWriteCallback pageWriteCallback = pageWriteCallbackFactory.createPageWriteCallback();
-                    componentBulkLoader = mergedComponent.createBulkLoader(operation, 1.0f, false, numElements, false,
-                            false, false, pageWriteCallback);
+                    componentBulkLoader = mergedComponent.createBulkLoader(storageConfig, operation, 1.0f, false,
+                            numElements, false, false, false, pageWriteCallback);
                     while (cursor.hasNext()) {
                         cursor.next();
                         ITupleReference frameTuple = cursor.getTuple();
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/utils/LSMBTreeUtil.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/utils/LSMBTreeUtil.java
index 6bd236c..a23b716 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/utils/LSMBTreeUtil.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/utils/LSMBTreeUtil.java
@@ -27,6 +27,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.control.common.controllers.NCConfig;
 import org.apache.hyracks.storage.am.bloomfilter.impls.BloomFilterFactory;
 import org.apache.hyracks.storage.am.btree.frames.BTreeNSMInteriorFrameFactory;
 import org.apache.hyracks.storage.am.btree.frames.BTreeNSMLeafFrameFactory;
@@ -62,30 +63,32 @@
     private LSMBTreeUtil() {
     }
 
-    public static LSMBTree createLSMTree(IIOManager ioManager, List<IVirtualBufferCache> virtualBufferCaches,
-            FileReference file, IBufferCache diskBufferCache, ITypeTraits[] typeTraits,
-            IBinaryComparatorFactory[] cmpFactories, int[] bloomFilterKeyFields, double bloomFilterFalsePositiveRate,
-            ILSMMergePolicy mergePolicy, ILSMOperationTracker opTracker, ILSMIOOperationScheduler ioScheduler,
-            ILSMIOOperationCallbackFactory ioOpCallbackFactory, ILSMPageWriteCallbackFactory pageWriteCallbackFactory,
-            boolean needKeyDupCheck, ITypeTraits[] filterTypeTraits, IBinaryComparatorFactory[] filterCmpFactories,
-            int[] btreeFields, int[] filterFields, boolean durable, IMetadataPageManagerFactory freePageManagerFactory,
+    public static LSMBTree createLSMTree(NCConfig storageConfig, IIOManager ioManager,
+            List<IVirtualBufferCache> virtualBufferCaches, FileReference file, IBufferCache diskBufferCache,
+            ITypeTraits[] typeTraits, IBinaryComparatorFactory[] cmpFactories, int[] bloomFilterKeyFields,
+            double bloomFilterFalsePositiveRate, ILSMMergePolicy mergePolicy, ILSMOperationTracker opTracker,
+            ILSMIOOperationScheduler ioScheduler, ILSMIOOperationCallbackFactory ioOpCallbackFactory,
+            ILSMPageWriteCallbackFactory pageWriteCallbackFactory, boolean needKeyDupCheck,
+            ITypeTraits[] filterTypeTraits, IBinaryComparatorFactory[] filterCmpFactories, int[] btreeFields,
+            int[] filterFields, boolean durable, IMetadataPageManagerFactory freePageManagerFactory,
             boolean updateAware, ITracer tracer, ICompressorDecompressorFactory compressorDecompressorFactory,
             boolean hasBloomFilter, ITypeTraits nullTypeTraits, INullIntrospector nullIntrospector)
             throws HyracksDataException {
-        return createLSMTree(ioManager, virtualBufferCaches, file, diskBufferCache, typeTraits, cmpFactories,
-                bloomFilterKeyFields, bloomFilterFalsePositiveRate, mergePolicy, opTracker, ioScheduler,
+        return createLSMTree(storageConfig, ioManager, virtualBufferCaches, file, diskBufferCache, typeTraits,
+                cmpFactories, bloomFilterKeyFields, bloomFilterFalsePositiveRate, mergePolicy, opTracker, ioScheduler,
                 ioOpCallbackFactory, pageWriteCallbackFactory, needKeyDupCheck, filterTypeTraits, filterCmpFactories,
                 btreeFields, filterFields, durable, freePageManagerFactory, updateAware, tracer,
                 compressorDecompressorFactory, hasBloomFilter, nullTypeTraits, nullIntrospector, false);
     }
 
-    public static LSMBTree createLSMTree(IIOManager ioManager, List<IVirtualBufferCache> virtualBufferCaches,
-            FileReference file, IBufferCache diskBufferCache, ITypeTraits[] typeTraits,
-            IBinaryComparatorFactory[] cmpFactories, int[] bloomFilterKeyFields, double bloomFilterFalsePositiveRate,
-            ILSMMergePolicy mergePolicy, ILSMOperationTracker opTracker, ILSMIOOperationScheduler ioScheduler,
-            ILSMIOOperationCallbackFactory ioOpCallbackFactory, ILSMPageWriteCallbackFactory pageWriteCallbackFactory,
-            boolean needKeyDupCheck, ITypeTraits[] filterTypeTraits, IBinaryComparatorFactory[] filterCmpFactories,
-            int[] btreeFields, int[] filterFields, boolean durable, IMetadataPageManagerFactory freePageManagerFactory,
+    public static LSMBTree createLSMTree(NCConfig storageConfig, IIOManager ioManager,
+            List<IVirtualBufferCache> virtualBufferCaches, FileReference file, IBufferCache diskBufferCache,
+            ITypeTraits[] typeTraits, IBinaryComparatorFactory[] cmpFactories, int[] bloomFilterKeyFields,
+            double bloomFilterFalsePositiveRate, ILSMMergePolicy mergePolicy, ILSMOperationTracker opTracker,
+            ILSMIOOperationScheduler ioScheduler, ILSMIOOperationCallbackFactory ioOpCallbackFactory,
+            ILSMPageWriteCallbackFactory pageWriteCallbackFactory, boolean needKeyDupCheck,
+            ITypeTraits[] filterTypeTraits, IBinaryComparatorFactory[] filterCmpFactories, int[] btreeFields,
+            int[] filterFields, boolean durable, IMetadataPageManagerFactory freePageManagerFactory,
             boolean updateAware, ITracer tracer, ICompressorDecompressorFactory compressorDecompressorFactory,
             boolean hasBloomFilter, ITypeTraits nullTypeTraits, INullIntrospector nullIntrospector, boolean atomic)
             throws HyracksDataException {
@@ -137,7 +140,7 @@
             bulkLoadComponentFactory = new LSMBTreeDiskComponentFactory(bulkLoadBTreeFactory, filterHelper);
         }
 
-        return new LSMBTree(ioManager, virtualBufferCaches, interiorFrameFactory, insertLeafFrameFactory,
+        return new LSMBTree(storageConfig, ioManager, virtualBufferCaches, interiorFrameFactory, insertLeafFrameFactory,
                 deleteLeafFrameFactory, diskBufferCache, fileNameManager, componentFactory, bulkLoadComponentFactory,
                 filterHelper, filterFrameFactory, filterManager, bloomFilterFalsePositiveRate, typeTraits.length,
                 cmpFactories, mergePolicy, opTracker, ioScheduler, ioOpCallbackFactory, pageWriteCallbackFactory,
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/pom.xml b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/pom.xml
index 39e5e9c..50e534a 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/pom.xml
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/pom.xml
@@ -102,5 +102,10 @@
       <groupId>com.fasterxml.jackson.core</groupId>
       <artifactId>jackson-annotations</artifactId>
     </dependency>
+    <dependency>
+      <groupId>org.apache.hyracks</groupId>
+      <artifactId>hyracks-control-common</artifactId>
+      <version>${project.version}</version>
+    </dependency>
   </dependencies>
 </project>
\ No newline at end of file
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/AbstractLSMWithBloomFilterDiskComponent.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/AbstractLSMWithBloomFilterDiskComponent.java
index c77d7ff..36c1435 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/AbstractLSMWithBloomFilterDiskComponent.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/AbstractLSMWithBloomFilterDiskComponent.java
@@ -19,6 +19,7 @@
 package org.apache.hyracks.storage.am.lsm.common.api;
 
 import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.control.common.controllers.NCConfig;
 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;
@@ -92,11 +93,12 @@
     }
 
     @Override
-    public ChainedLSMDiskComponentBulkLoader createBulkLoader(ILSMIOOperation operation, float fillFactor,
-            boolean verifyInput, long numElementsHint, boolean checkIfEmptyIndex, boolean withFilter,
+    public ChainedLSMDiskComponentBulkLoader createBulkLoader(NCConfig storageConfig, ILSMIOOperation operation,
+            float fillFactor, boolean verifyInput, long numElementsHint, boolean checkIfEmptyIndex, boolean withFilter,
             boolean cleanupEmptyComponent, IPageWriteCallback callback) throws HyracksDataException {
-        ChainedLSMDiskComponentBulkLoader chainedBulkLoader = super.createBulkLoader(operation, fillFactor, verifyInput,
-                numElementsHint, checkIfEmptyIndex, withFilter, cleanupEmptyComponent, callback);
+        ChainedLSMDiskComponentBulkLoader chainedBulkLoader =
+                super.createBulkLoader(storageConfig, operation, fillFactor, verifyInput, numElementsHint,
+                        checkIfEmptyIndex, withFilter, cleanupEmptyComponent, callback);
         if (numElementsHint > 0) {
             chainedBulkLoader.addBulkLoader(createBloomFilterBulkLoader(numElementsHint, callback));
         }
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/ILSMDiskComponent.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/ILSMDiskComponent.java
index e41a17e..ef2061c 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/ILSMDiskComponent.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/ILSMDiskComponent.java
@@ -21,6 +21,7 @@
 import java.util.Set;
 
 import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.control.common.controllers.NCConfig;
 import org.apache.hyracks.storage.am.common.api.ITreeIndex;
 import org.apache.hyracks.storage.am.lsm.common.impls.AbstractLSMIndex;
 import org.apache.hyracks.storage.am.lsm.common.impls.DiskComponentMetadata;
@@ -124,6 +125,7 @@
      * Creates a bulkloader pipeline which includes all chained operations, bulkloading individual elements of the
      * component: indexes, LSM filters, Bloom filters, buddy indexes, etc.
      *
+     * @param storageConfig
      * @param operation
      * @param fillFactor
      * @param verifyInput
@@ -134,9 +136,9 @@
      * @return the created disk component bulk loader
      * @throws HyracksDataException
      */
-    ILSMDiskComponentBulkLoader createBulkLoader(ILSMIOOperation operation, float fillFactor, boolean verifyInput,
-            long numElementsHint, boolean checkIfEmptyIndex, boolean withFilter, boolean cleanupEmptyComponent,
-            IPageWriteCallback callback) throws HyracksDataException;
+    ILSMDiskComponentBulkLoader createBulkLoader(NCConfig storageConfig, ILSMIOOperation operation, float fillFactor,
+            boolean verifyInput, long numElementsHint, boolean checkIfEmptyIndex, boolean withFilter,
+            boolean cleanupEmptyComponent, IPageWriteCallback callback) throws HyracksDataException;
 
     /**
      * Returns all pages of the component to the buffer cache
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/AbstractLSMDiskComponent.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/AbstractLSMDiskComponent.java
index b6066fc..ea10c9a 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/AbstractLSMDiskComponent.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/AbstractLSMDiskComponent.java
@@ -19,6 +19,7 @@
 package org.apache.hyracks.storage.am.lsm.common.impls;
 
 import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.control.common.controllers.NCConfig;
 import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
 import org.apache.hyracks.storage.am.common.api.IMetadataPageManager;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponentFilter;
@@ -237,8 +238,8 @@
     }
 
     @Override
-    public ChainedLSMDiskComponentBulkLoader createBulkLoader(ILSMIOOperation operation, float fillFactor,
-            boolean verifyInput, long numElementsHint, boolean checkIfEmptyIndex, boolean withFilter,
+    public ChainedLSMDiskComponentBulkLoader createBulkLoader(NCConfig storageConfig, ILSMIOOperation operation,
+            float fillFactor, boolean verifyInput, long numElementsHint, boolean checkIfEmptyIndex, boolean withFilter,
             boolean cleanupEmptyComponent, IPageWriteCallback callback) throws HyracksDataException {
         ChainedLSMDiskComponentBulkLoader chainedBulkLoader =
                 new ChainedLSMDiskComponentBulkLoader(operation, this, cleanupEmptyComponent);
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/AbstractLSMIndex.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/AbstractLSMIndex.java
index 3f60978..8b16e8c 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/AbstractLSMIndex.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/AbstractLSMIndex.java
@@ -40,6 +40,7 @@
 import org.apache.hyracks.api.replication.IReplicationJob.ReplicationExecutionType;
 import org.apache.hyracks.api.replication.IReplicationJob.ReplicationOperation;
 import org.apache.hyracks.api.util.HyracksConstants;
+import org.apache.hyracks.control.common.controllers.NCConfig;
 import org.apache.hyracks.dataflow.common.data.accessors.ITupleReference;
 import org.apache.hyracks.storage.am.common.impls.AbstractSearchPredicate;
 import org.apache.hyracks.storage.am.common.impls.NoOpIndexAccessParameters;
@@ -109,6 +110,7 @@
     protected final AtomicBoolean[] flushRequests;
     protected volatile boolean memoryComponentsAllocated = false;
     protected ITracer tracer;
+    protected NCConfig storageConfig;
     // Factory for creating on-disk index components during flush and merge.
     protected final ILSMDiskComponentFactory componentFactory;
     // Factory for creating on-disk index components during bulkload.
@@ -120,7 +122,7 @@
     private final ILSMMergePolicy mergePolicy;
     private final ILSMIOOperationScheduler ioScheduler;
 
-    public AbstractLSMIndex(IIOManager ioManager, List<IVirtualBufferCache> virtualBufferCaches,
+    public AbstractLSMIndex(NCConfig storageConfig, IIOManager ioManager, List<IVirtualBufferCache> virtualBufferCaches,
             IBufferCache diskBufferCache, ILSMIndexFileManager fileManager, double bloomFilterFalsePositiveRate,
             ILSMMergePolicy mergePolicy, ILSMOperationTracker opTracker, ILSMIOOperationScheduler ioScheduler,
             ILSMIOOperationCallbackFactory ioOpCallbackFactory, ILSMPageWriteCallbackFactory pageWriteCallbackFactory,
@@ -150,6 +152,7 @@
         this.temporaryDiskComponents = new ArrayList<>();
         this.mergePolicy = mergePolicy;
         this.ioScheduler = ioScheduler;
+        this.storageConfig = storageConfig;
 
         fileManager.initLastUsedSeq(ioOpCallback.getLastValidSequence());
         lsmHarness = new LSMHarness(this, ioScheduler, mergePolicy, opTracker, diskBufferCache.isReplicationEnabled(),
@@ -164,7 +167,7 @@
         }
     }
 
-    public AbstractLSMIndex(IIOManager ioManager, List<IVirtualBufferCache> virtualBufferCaches,
+    public AbstractLSMIndex(NCConfig storageConfig, IIOManager ioManager, List<IVirtualBufferCache> virtualBufferCaches,
             IBufferCache diskBufferCache, ILSMIndexFileManager fileManager, double bloomFilterFalsePositiveRate,
             ILSMMergePolicy mergePolicy, ILSMOperationTracker opTracker, ILSMIOOperationScheduler ioScheduler,
             ILSMIOOperationCallbackFactory ioOpCallbackFactory, ILSMPageWriteCallbackFactory pageWriteCallbackFactory,
@@ -172,8 +175,8 @@
             ILSMComponentFilterFrameFactory filterFrameFactory, LSMComponentFilterManager filterManager,
             int[] filterFields, boolean durable, IComponentFilterHelper filterHelper, int[] treeFields, ITracer tracer)
             throws HyracksDataException {
-        this(ioManager, virtualBufferCaches, diskBufferCache, fileManager, bloomFilterFalsePositiveRate, mergePolicy,
-                opTracker, ioScheduler, ioOpCallbackFactory, pageWriteCallbackFactory, componentFactory,
+        this(storageConfig, ioManager, virtualBufferCaches, diskBufferCache, fileManager, bloomFilterFalsePositiveRate,
+                mergePolicy, opTracker, ioScheduler, ioOpCallbackFactory, pageWriteCallbackFactory, componentFactory,
                 bulkLoadComponentFactory, filterFrameFactory, filterManager, filterFields, durable, filterHelper,
                 treeFields, tracer, false);
     }
@@ -568,7 +571,7 @@
                 componentFileRefs.getBloomFilterFileReference(), true));
         ioOpCallback.scheduled(loadOp);
         opCtx.setIoOperation(loadOp);
-        return new LSMIndexDiskComponentBulkLoader(this, opCtx, fillLevel, verifyInput, numElementsHint);
+        return new LSMIndexDiskComponentBulkLoader(storageConfig, this, opCtx, fillLevel, verifyInput, numElementsHint);
     }
 
     @Override
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/EmptyComponent.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/EmptyComponent.java
index 0dfb12e..f5a7ed1 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/EmptyComponent.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/EmptyComponent.java
@@ -23,6 +23,7 @@
 
 import org.apache.hyracks.api.exceptions.ErrorCode;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.control.common.controllers.NCConfig;
 import org.apache.hyracks.storage.am.common.api.ITreeIndex;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponentFilter;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponentId;
@@ -137,8 +138,8 @@
     }
 
     @Override
-    public ChainedLSMDiskComponentBulkLoader createBulkLoader(ILSMIOOperation operation, float fillFactor,
-            boolean verifyInput, long numElementsHint, boolean checkIfEmptyIndex, boolean withFilter,
+    public ChainedLSMDiskComponentBulkLoader createBulkLoader(NCConfig storageConfig, ILSMIOOperation operation,
+            float fillFactor, boolean verifyInput, long numElementsHint, boolean checkIfEmptyIndex, boolean withFilter,
             boolean cleanupEmptyComponent, IPageWriteCallback callback) throws HyracksDataException {
         return null;
     }
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/LSMIndexDiskComponentBulkLoader.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/LSMIndexDiskComponentBulkLoader.java
index 88fa105..841d389 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/LSMIndexDiskComponentBulkLoader.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/LSMIndexDiskComponentBulkLoader.java
@@ -19,6 +19,7 @@
 package org.apache.hyracks.storage.am.lsm.common.impls;
 
 import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.control.common.controllers.NCConfig;
 import org.apache.hyracks.dataflow.common.data.accessors.ITupleReference;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMDiskComponent;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMDiskComponentBulkLoader;
@@ -34,12 +35,13 @@
     private final ILSMIndexOperationContext opCtx;
     private boolean failed = false;
 
-    public LSMIndexDiskComponentBulkLoader(AbstractLSMIndex lsmIndex, ILSMIndexOperationContext opCtx, float fillFactor,
-            boolean verifyInput, long numElementsHint) throws HyracksDataException {
+    public LSMIndexDiskComponentBulkLoader(NCConfig storageConfig, AbstractLSMIndex lsmIndex,
+            ILSMIndexOperationContext opCtx, float fillFactor, boolean verifyInput, long numElementsHint)
+            throws HyracksDataException {
         this.lsmIndex = lsmIndex;
         this.opCtx = opCtx;
-        this.componentBulkLoader = opCtx.getIoOperation().getNewComponent().createBulkLoader(opCtx.getIoOperation(),
-                fillFactor, verifyInput, numElementsHint, false, true, true,
+        this.componentBulkLoader = opCtx.getIoOperation().getNewComponent().createBulkLoader(storageConfig,
+                opCtx.getIoOperation(), fillFactor, verifyInput, numElementsHint, false, true, true,
                 lsmIndex.getPageWriteCallbackFactory().createPageWriteCallback());
     }
 
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/pom.xml b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/pom.xml
index cf10370..68608cc 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/pom.xml
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/pom.xml
@@ -96,14 +96,23 @@
       <groupId>com.fasterxml.jackson.core</groupId>
       <artifactId>jackson-databind</artifactId>
     </dependency>
-  <dependency>
-    <groupId>org.apache.commons</groupId>
-    <artifactId>commons-lang3</artifactId>
-  </dependency>
-  <dependency>
-    <groupId>com.google.guava</groupId>
-    <artifactId>guava</artifactId>
-  </dependency>
-
+    <dependency>
+      <groupId>org.apache.commons</groupId>
+      <artifactId>commons-lang3</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>com.google.guava</groupId>
+      <artifactId>guava</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hyracks</groupId>
+      <artifactId>hyracks-control-common</artifactId>
+      <version>${project.version}</version>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hyracks</groupId>
+      <artifactId>hyracks-control-nc</artifactId>
+      <version>${project.version}</version>
+    </dependency>
   </dependencies>
 </project>
\ No newline at end of file
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/dataflow/LSMInvertedIndexLocalResource.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/dataflow/LSMInvertedIndexLocalResource.java
index 8048a93..f20c716 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/dataflow/LSMInvertedIndexLocalResource.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/dataflow/LSMInvertedIndexLocalResource.java
@@ -30,6 +30,8 @@
 import org.apache.hyracks.api.io.IIOManager;
 import org.apache.hyracks.api.io.IJsonSerializable;
 import org.apache.hyracks.api.io.IPersistedResourceRegistry;
+import org.apache.hyracks.control.common.controllers.NCConfig;
+import org.apache.hyracks.control.nc.NodeControllerService;
 import org.apache.hyracks.storage.am.common.api.IMetadataPageManagerFactory;
 import org.apache.hyracks.storage.am.common.api.INullIntrospector;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallbackFactory;
@@ -130,6 +132,7 @@
     @Override
     public ILSMIndex createInstance(INCServiceContext serviceCtx) throws HyracksDataException {
         IIOManager ioManager = storageManager.getIoManager(serviceCtx);
+        NCConfig storageConfig = ((NodeControllerService) serviceCtx.getControllerService()).getConfiguration();
         FileReference file = ioManager.resolve(path);
         List<IVirtualBufferCache> virtualBufferCaches = vbcProvider.getVirtualBufferCaches(serviceCtx, file);
         IBufferCache bufferCache = storageManager.getBufferCache(serviceCtx);
@@ -138,21 +141,22 @@
         ioOpCallbackFactory.initialize(serviceCtx, this);
         pageWriteCallbackFactory.initialize(serviceCtx, this);
         if (isPartitioned) {
-            return InvertedIndexUtils.createPartitionedLSMInvertedIndex(ioManager, virtualBufferCaches, typeTraits,
+            return InvertedIndexUtils.createPartitionedLSMInvertedIndex(storageConfig, ioManager, virtualBufferCaches,
+                    typeTraits, cmpFactories, tokenTypeTraits, tokenCmpFactories, tokenizerFactory,
+                    fullTextConfigEvaluatorFactory, bufferCache, file.getAbsolutePath(), bloomFilterFalsePositiveRate,
+                    mergePolicy, opTrackerProvider.getOperationTracker(serviceCtx, this), ioScheduler,
+                    ioOpCallbackFactory, pageWriteCallbackFactory, invertedIndexFields, filterTypeTraits,
+                    filterCmpFactories, filterFields, filterFieldsForNonBulkLoadOps,
+                    invertedIndexFieldsForNonBulkLoadOps, durable, metadataPageManagerFactory, serviceCtx.getTracer(),
+                    nullTypeTraits, nullIntrospector);
+        } else {
+            return InvertedIndexUtils.createLSMInvertedIndex(storageConfig, ioManager, virtualBufferCaches, typeTraits,
                     cmpFactories, tokenTypeTraits, tokenCmpFactories, tokenizerFactory, fullTextConfigEvaluatorFactory,
                     bufferCache, file.getAbsolutePath(), bloomFilterFalsePositiveRate, mergePolicy,
                     opTrackerProvider.getOperationTracker(serviceCtx, this), ioScheduler, ioOpCallbackFactory,
                     pageWriteCallbackFactory, invertedIndexFields, filterTypeTraits, filterCmpFactories, filterFields,
                     filterFieldsForNonBulkLoadOps, invertedIndexFieldsForNonBulkLoadOps, durable,
                     metadataPageManagerFactory, serviceCtx.getTracer(), nullTypeTraits, nullIntrospector);
-        } else {
-            return InvertedIndexUtils.createLSMInvertedIndex(ioManager, virtualBufferCaches, typeTraits, cmpFactories,
-                    tokenTypeTraits, tokenCmpFactories, tokenizerFactory, fullTextConfigEvaluatorFactory, bufferCache,
-                    file.getAbsolutePath(), bloomFilterFalsePositiveRate, mergePolicy,
-                    opTrackerProvider.getOperationTracker(serviceCtx, this), ioScheduler, ioOpCallbackFactory,
-                    pageWriteCallbackFactory, invertedIndexFields, filterTypeTraits, filterCmpFactories, filterFields,
-                    filterFieldsForNonBulkLoadOps, invertedIndexFieldsForNonBulkLoadOps, durable,
-                    metadataPageManagerFactory, serviceCtx.getTracer(), nullTypeTraits, nullIntrospector);
         }
     }
 
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndex.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndex.java
index f98a056..75743c5 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndex.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndex.java
@@ -29,6 +29,7 @@
 import org.apache.hyracks.api.exceptions.ErrorCode;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.api.io.IIOManager;
+import org.apache.hyracks.control.common.controllers.NCConfig;
 import org.apache.hyracks.data.std.primitive.IntegerPointable;
 import org.apache.hyracks.dataflow.common.data.accessors.ITupleReference;
 import org.apache.hyracks.dataflow.common.data.accessors.PermutingTupleReference;
@@ -99,7 +100,7 @@
     protected final ITypeTraits nullTypeTraits;
     protected final INullIntrospector nullIntrospector;
 
-    public LSMInvertedIndex(IIOManager ioManager, List<IVirtualBufferCache> virtualBufferCaches,
+    public LSMInvertedIndex(NCConfig storageConfig, IIOManager ioManager, List<IVirtualBufferCache> virtualBufferCaches,
             ILSMDiskComponentFactory componentFactory, IComponentFilterHelper filterHelper,
             ILSMComponentFilterFrameFactory filterFrameFactory, LSMComponentFilterManager filterManager,
             double bloomFilterFalsePositiveRate, IBufferCache diskBufferCache, ILSMIndexFileManager fileManager,
@@ -111,8 +112,8 @@
             int[] invertedIndexFields, int[] filterFields, int[] filterFieldsForNonBulkLoadOps,
             int[] invertedIndexFieldsForNonBulkLoadOps, boolean durable, ITracer tracer, ITypeTraits nullTypeTraits,
             INullIntrospector nullIntrospector) throws HyracksDataException {
-        super(ioManager, virtualBufferCaches, diskBufferCache, fileManager, bloomFilterFalsePositiveRate, mergePolicy,
-                opTracker, ioScheduler, ioOpCallbackFactory, pageWriteCallbackFactory, componentFactory,
+        super(storageConfig, ioManager, virtualBufferCaches, diskBufferCache, fileManager, bloomFilterFalsePositiveRate,
+                mergePolicy, opTracker, ioScheduler, ioOpCallbackFactory, pageWriteCallbackFactory, componentFactory,
                 componentFactory, filterFrameFactory, filterManager, filterFields, durable, filterHelper,
                 invertedIndexFields, tracer);
         this.tokenizerFactory = tokenizerFactory;
@@ -283,8 +284,8 @@
             btreeCountingCursor.destroy();
         }
 
-        ILSMDiskComponentBulkLoader componentBulkLoader = component.createBulkLoader(operation, 1.0f, false,
-                numBTreeTuples, false, false, false, pageWriteCallbackFactory.createPageWriteCallback());
+        ILSMDiskComponentBulkLoader componentBulkLoader = component.createBulkLoader(storageConfig, operation, 1.0f,
+                false, numBTreeTuples, false, false, false, pageWriteCallbackFactory.createPageWriteCallback());
 
         // Create a scan cursor on the deleted keys BTree underlying the in-memory inverted index.
         IIndexCursor deletedKeysScanCursor = deletedKeysBTreeAccessor.createSearchCursor(false);
@@ -367,15 +368,15 @@
                         numElements += ((LSMInvertedIndexDiskComponent) mergeOp.getMergingComponents().get(i))
                                 .getBloomFilter().getNumElements();
                     }
-                    componentBulkLoader = component.createBulkLoader(operation, 1.0f, false, numElements, false, false,
-                            false, pageWriteCallbackFactory.createPageWriteCallback());
+                    componentBulkLoader = component.createBulkLoader(storageConfig, operation, 1.0f, false, numElements,
+                            false, false, false, pageWriteCallbackFactory.createPageWriteCallback());
                     loadDeleteTuples(opCtx, btreeCursor, mergePred, componentBulkLoader);
                 } finally {
                     btreeCursor.destroy();
                 }
             } else {
-                componentBulkLoader = component.createBulkLoader(operation, 1.0f, false, 0L, false, false, false,
-                        pageWriteCallbackFactory.createPageWriteCallback());
+                componentBulkLoader = component.createBulkLoader(storageConfig, operation, 1.0f, false, 0L, false,
+                        false, false, pageWriteCallbackFactory.createPageWriteCallback());
             }
             search(opCtx, cursor, mergePred);
             try {
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/impls/PartitionedLSMInvertedIndex.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/impls/PartitionedLSMInvertedIndex.java
index a801146..61783d7 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/impls/PartitionedLSMInvertedIndex.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/impls/PartitionedLSMInvertedIndex.java
@@ -25,6 +25,7 @@
 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.control.common.controllers.NCConfig;
 import org.apache.hyracks.storage.am.common.api.INullIntrospector;
 import org.apache.hyracks.storage.am.lsm.common.api.IComponentFilterHelper;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponentFilterFrameFactory;
@@ -47,24 +48,25 @@
 
 public class PartitionedLSMInvertedIndex extends LSMInvertedIndex {
 
-    public PartitionedLSMInvertedIndex(IIOManager ioManager, List<IVirtualBufferCache> virtualBufferCaches,
-            ILSMDiskComponentFactory componentFactory, IComponentFilterHelper filterHelper,
-            ILSMComponentFilterFrameFactory filterFrameFactory, LSMComponentFilterManager filterManager,
-            double bloomFilterFalsePositiveRate, IBufferCache diskBufferCache, ILSMIndexFileManager fileManager,
-            ITypeTraits[] invListTypeTraits, IBinaryComparatorFactory[] invListCmpFactories,
-            ITypeTraits[] tokenTypeTraits, IBinaryComparatorFactory[] tokenCmpFactories,
-            IBinaryTokenizerFactory tokenizerFactory, IFullTextConfigEvaluatorFactory fullTextConfigEvaluatorFactory,
-            ILSMMergePolicy mergePolicy, ILSMOperationTracker opTracker, ILSMIOOperationScheduler ioScheduler,
+    public PartitionedLSMInvertedIndex(NCConfig storageConfig, IIOManager ioManager,
+            List<IVirtualBufferCache> virtualBufferCaches, ILSMDiskComponentFactory componentFactory,
+            IComponentFilterHelper filterHelper, ILSMComponentFilterFrameFactory filterFrameFactory,
+            LSMComponentFilterManager filterManager, double bloomFilterFalsePositiveRate, IBufferCache diskBufferCache,
+            ILSMIndexFileManager fileManager, ITypeTraits[] invListTypeTraits,
+            IBinaryComparatorFactory[] invListCmpFactories, ITypeTraits[] tokenTypeTraits,
+            IBinaryComparatorFactory[] tokenCmpFactories, IBinaryTokenizerFactory tokenizerFactory,
+            IFullTextConfigEvaluatorFactory fullTextConfigEvaluatorFactory, ILSMMergePolicy mergePolicy,
+            ILSMOperationTracker opTracker, ILSMIOOperationScheduler ioScheduler,
             ILSMIOOperationCallbackFactory ioOpCallbackFactory, ILSMPageWriteCallbackFactory pageWriteCallbackFactory,
             int[] invertedIndexFields, int[] filterFields, int[] filterFieldsForNonBulkLoadOps,
             int[] invertedIndexFieldsForNonBulkLoadOps, boolean durable, ITracer tracer, ITypeTraits nullTypeTraits,
             INullIntrospector nullIntrospector) throws HyracksDataException {
-        super(ioManager, virtualBufferCaches, componentFactory, filterHelper, filterFrameFactory, filterManager,
-                bloomFilterFalsePositiveRate, diskBufferCache, fileManager, invListTypeTraits, invListCmpFactories,
-                tokenTypeTraits, tokenCmpFactories, tokenizerFactory, fullTextConfigEvaluatorFactory, mergePolicy,
-                opTracker, ioScheduler, ioOpCallbackFactory, pageWriteCallbackFactory, invertedIndexFields,
-                filterFields, filterFieldsForNonBulkLoadOps, invertedIndexFieldsForNonBulkLoadOps, durable, tracer,
-                nullTypeTraits, nullIntrospector);
+        super(storageConfig, ioManager, virtualBufferCaches, componentFactory, filterHelper, filterFrameFactory,
+                filterManager, bloomFilterFalsePositiveRate, diskBufferCache, fileManager, invListTypeTraits,
+                invListCmpFactories, tokenTypeTraits, tokenCmpFactories, tokenizerFactory,
+                fullTextConfigEvaluatorFactory, mergePolicy, opTracker, ioScheduler, ioOpCallbackFactory,
+                pageWriteCallbackFactory, invertedIndexFields, filterFields, filterFieldsForNonBulkLoadOps,
+                invertedIndexFieldsForNonBulkLoadOps, durable, tracer, nullTypeTraits, nullIntrospector);
     }
 
     @Override
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/util/InvertedIndexUtils.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/util/InvertedIndexUtils.java
index aefb461..4dc3507 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/util/InvertedIndexUtils.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/util/InvertedIndexUtils.java
@@ -28,6 +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.control.common.controllers.NCConfig;
 import org.apache.hyracks.storage.am.bloomfilter.impls.BloomFilterFactory;
 import org.apache.hyracks.storage.am.btree.frames.BTreeLeafFrameType;
 import org.apache.hyracks.storage.am.btree.frames.BTreeNSMInteriorFrameFactory;
@@ -144,7 +145,7 @@
                 leafFrameFactory, invListCmpFactories, invListCmpFactories.length);
     }
 
-    public static LSMInvertedIndex createLSMInvertedIndex(IIOManager ioManager,
+    public static LSMInvertedIndex createLSMInvertedIndex(NCConfig storageConfig, IIOManager ioManager,
             List<IVirtualBufferCache> virtualBufferCaches, ITypeTraits[] invListTypeTraits,
             IBinaryComparatorFactory[] invListCmpFactories, ITypeTraits[] tokenTypeTraits,
             IBinaryComparatorFactory[] tokenCmpFactories, IBinaryTokenizerFactory tokenizerFactory,
@@ -189,16 +190,16 @@
         ILSMDiskComponentFactory componentFactory = new LSMInvertedIndexDiskComponentFactory(invIndexFactory,
                 deletedKeysBTreeFactory, bloomFilterFactory, filterHelper);
 
-        return new LSMInvertedIndex(ioManager, virtualBufferCaches, componentFactory, filterHelper, filterFrameFactory,
-                filterManager, bloomFilterFalsePositiveRate, diskBufferCache, fileManager, invListTypeTraits,
-                invListCmpFactories, tokenTypeTraits, tokenCmpFactories, tokenizerFactory,
+        return new LSMInvertedIndex(storageConfig, ioManager, virtualBufferCaches, componentFactory, filterHelper,
+                filterFrameFactory, filterManager, bloomFilterFalsePositiveRate, diskBufferCache, fileManager,
+                invListTypeTraits, invListCmpFactories, tokenTypeTraits, tokenCmpFactories, tokenizerFactory,
                 fullTextConfigEvaluatorFactory, mergePolicy, opTracker, ioScheduler, ioOpCallbackFactory,
                 pageWriteCallbackFactory, invertedIndexFields, filterFields, filterFieldsForNonBulkLoadOps,
                 invertedIndexFieldsForNonBulkLoadOps, durable, tracer, nullTypeTraits, nullIntrospector);
     }
 
-    public static PartitionedLSMInvertedIndex createPartitionedLSMInvertedIndex(IIOManager ioManager,
-            List<IVirtualBufferCache> virtualBufferCaches, ITypeTraits[] invListTypeTraits,
+    public static PartitionedLSMInvertedIndex createPartitionedLSMInvertedIndex(NCConfig storageConfig,
+            IIOManager ioManager, List<IVirtualBufferCache> virtualBufferCaches, ITypeTraits[] invListTypeTraits,
             IBinaryComparatorFactory[] invListCmpFactories, ITypeTraits[] tokenTypeTraits,
             IBinaryComparatorFactory[] tokenCmpFactories, IBinaryTokenizerFactory tokenizerFactory,
             IFullTextConfigEvaluatorFactory fullTextConfigEvaluatorFactory, IBufferCache diskBufferCache,
@@ -242,12 +243,13 @@
         ILSMDiskComponentFactory componentFactory = new LSMInvertedIndexDiskComponentFactory(invIndexFactory,
                 deletedKeysBTreeFactory, bloomFilterFactory, filterHelper);
 
-        return new PartitionedLSMInvertedIndex(ioManager, virtualBufferCaches, componentFactory, filterHelper,
-                filterFrameFactory, filterManager, bloomFilterFalsePositiveRate, diskBufferCache, fileManager,
-                invListTypeTraits, invListCmpFactories, tokenTypeTraits, tokenCmpFactories, tokenizerFactory,
-                fullTextConfigEvaluatorFactory, mergePolicy, opTracker, ioScheduler, ioOpCallbackFactory,
-                pageWriteCallbackFactory, invertedIndexFields, filterFields, filterFieldsForNonBulkLoadOps,
-                invertedIndexFieldsForNonBulkLoadOps, durable, tracer, nullTypeTraits, nullIntrospector);
+        return new PartitionedLSMInvertedIndex(storageConfig, ioManager, virtualBufferCaches, componentFactory,
+                filterHelper, filterFrameFactory, filterManager, bloomFilterFalsePositiveRate, diskBufferCache,
+                fileManager, invListTypeTraits, invListCmpFactories, tokenTypeTraits, tokenCmpFactories,
+                tokenizerFactory, fullTextConfigEvaluatorFactory, mergePolicy, opTracker, ioScheduler,
+                ioOpCallbackFactory, pageWriteCallbackFactory, invertedIndexFields, filterFields,
+                filterFieldsForNonBulkLoadOps, invertedIndexFieldsForNonBulkLoadOps, durable, tracer, nullTypeTraits,
+                nullIntrospector);
     }
 
     public static boolean checkTypeTraitsAllFixed(ITypeTraits[] typeTraits) {
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/pom.xml b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/pom.xml
index 84040ed..c7fbc51 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/pom.xml
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/pom.xml
@@ -88,6 +88,16 @@
       <version>${project.version}</version>
     </dependency>
     <dependency>
+      <groupId>org.apache.hyracks</groupId>
+      <artifactId>hyracks-control-nc</artifactId>
+      <version>${project.version}</version>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hyracks</groupId>
+      <artifactId>hyracks-control-common</artifactId>
+      <version>${project.version}</version>
+    </dependency>
+    <dependency>
       <groupId>org.apache.commons</groupId>
       <artifactId>commons-lang3</artifactId>
     </dependency>
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/dataflow/LSMRTreeLocalResource.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/dataflow/LSMRTreeLocalResource.java
index 3a89238..bf347ba 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/dataflow/LSMRTreeLocalResource.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/dataflow/LSMRTreeLocalResource.java
@@ -31,6 +31,8 @@
 import org.apache.hyracks.api.io.IIOManager;
 import org.apache.hyracks.api.io.IJsonSerializable;
 import org.apache.hyracks.api.io.IPersistedResourceRegistry;
+import org.apache.hyracks.control.common.controllers.NCConfig;
+import org.apache.hyracks.control.nc.NodeControllerService;
 import org.apache.hyracks.storage.am.common.api.IMetadataPageManagerFactory;
 import org.apache.hyracks.storage.am.common.api.INullIntrospector;
 import org.apache.hyracks.storage.am.common.api.IPrimitiveValueProviderFactory;
@@ -109,11 +111,12 @@
     @Override
     public IIndex createInstance(INCServiceContext ncServiceCtx) throws HyracksDataException {
         IIOManager ioManager = storageManager.getIoManager(ncServiceCtx);
+        NCConfig storageConfig = ((NodeControllerService) ncServiceCtx.getControllerService()).getConfiguration();
         FileReference fileRef = ioManager.resolve(path);
         List<IVirtualBufferCache> virtualBufferCaches = vbcProvider.getVirtualBufferCaches(ncServiceCtx, fileRef);
         ioOpCallbackFactory.initialize(ncServiceCtx, this);
         pageWriteCallbackFactory.initialize(ncServiceCtx, this);
-        return LSMRTreeUtils.createLSMTree(ioManager, virtualBufferCaches, fileRef,
+        return LSMRTreeUtils.createLSMTree(storageConfig, ioManager, virtualBufferCaches, fileRef,
                 storageManager.getBufferCache(ncServiceCtx), typeTraits, cmpFactories, btreeCmpFactories,
                 valueProviderFactories, rtreePolicyType, bloomFilterFalsePositiveRate,
                 mergePolicyFactory.createMergePolicy(mergePolicyProperties, ncServiceCtx),
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/dataflow/LSMRTreeWithAntiMatterLocalResource.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/dataflow/LSMRTreeWithAntiMatterLocalResource.java
index e359661..c63edd7 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/dataflow/LSMRTreeWithAntiMatterLocalResource.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/dataflow/LSMRTreeWithAntiMatterLocalResource.java
@@ -31,6 +31,8 @@
 import org.apache.hyracks.api.io.IIOManager;
 import org.apache.hyracks.api.io.IJsonSerializable;
 import org.apache.hyracks.api.io.IPersistedResourceRegistry;
+import org.apache.hyracks.control.common.controllers.NCConfig;
+import org.apache.hyracks.control.nc.NodeControllerService;
 import org.apache.hyracks.storage.am.common.api.IMetadataPageManagerFactory;
 import org.apache.hyracks.storage.am.common.api.INullIntrospector;
 import org.apache.hyracks.storage.am.common.api.IPrimitiveValueProviderFactory;
@@ -101,11 +103,12 @@
     @Override
     public ILSMIndex createInstance(INCServiceContext serviceCtx) throws HyracksDataException {
         IIOManager ioManager = storageManager.getIoManager(serviceCtx);
+        NCConfig storageConfig = ((NodeControllerService) serviceCtx.getControllerService()).getConfiguration();
         FileReference file = ioManager.resolve(path);
         List<IVirtualBufferCache> virtualBufferCaches = vbcProvider.getVirtualBufferCaches(serviceCtx, file);
         ioOpCallbackFactory.initialize(serviceCtx, this);
         pageWriteCallbackFactory.initialize(serviceCtx, this);
-        return LSMRTreeUtils.createLSMTreeWithAntiMatterTuples(ioManager, virtualBufferCaches, file,
+        return LSMRTreeUtils.createLSMTreeWithAntiMatterTuples(storageConfig, ioManager, virtualBufferCaches, file,
                 storageManager.getBufferCache(serviceCtx), typeTraits, cmpFactories, btreeCmpFactories,
                 valueProviderFactories, rtreePolicyType,
                 mergePolicyFactory.createMergePolicy(mergePolicyProperties, serviceCtx),
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/impls/AbstractLSMRTree.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/impls/AbstractLSMRTree.java
index e95ef67..34d2ed8 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/impls/AbstractLSMRTree.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/impls/AbstractLSMRTree.java
@@ -27,6 +27,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.control.common.controllers.NCConfig;
 import org.apache.hyracks.dataflow.common.data.accessors.ITupleReference;
 import org.apache.hyracks.storage.am.btree.impls.BTree;
 import org.apache.hyracks.storage.am.common.api.IExtendedModificationOperationCallback;
@@ -75,7 +76,7 @@
     protected final ITreeIndexFrameFactory rtreeLeafFrameFactory;
     protected final ITreeIndexFrameFactory btreeLeafFrameFactory;
 
-    public AbstractLSMRTree(IIOManager ioManager, List<IVirtualBufferCache> virtualBufferCaches,
+    public AbstractLSMRTree(NCConfig storageConfig, IIOManager ioManager, List<IVirtualBufferCache> virtualBufferCaches,
             RTreeFrameFactory rtreeInteriorFrameFactory, RTreeFrameFactory rtreeLeafFrameFactory,
             ITreeIndexFrameFactory btreeInteriorFrameFactory, ITreeIndexFrameFactory btreeLeafFrameFactory,
             IBufferCache diskBufferCache, ILSMIndexFileManager fileManager, ILSMDiskComponentFactory componentFactory,
@@ -87,8 +88,8 @@
             ILSMPageWriteCallbackFactory pageWriteCallbackFactory, IComponentFilterHelper filterHelper,
             ILSMComponentFilterFrameFactory filterFrameFactory, LSMComponentFilterManager filterManager,
             int[] rtreeFields, int[] filterFields, boolean durable, boolean isPointMBR) throws HyracksDataException {
-        super(ioManager, virtualBufferCaches, diskBufferCache, fileManager, bloomFilterFalsePositiveRate, mergePolicy,
-                opTracker, ioScheduler, ioOpCallbackFactory, pageWriteCallbackFactory, componentFactory,
+        super(storageConfig, ioManager, virtualBufferCaches, diskBufferCache, fileManager, bloomFilterFalsePositiveRate,
+                mergePolicy, opTracker, ioScheduler, ioOpCallbackFactory, pageWriteCallbackFactory, componentFactory,
                 bulkLoadComponentFactory, filterFrameFactory, filterManager, filterFields, durable, filterHelper,
                 rtreeFields, ITracer.NONE);
         int i = 0;
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/impls/LSMRTree.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/impls/LSMRTree.java
index 8b88e7e..cafab7a 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/impls/LSMRTree.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/impls/LSMRTree.java
@@ -29,6 +29,7 @@
 import org.apache.hyracks.api.exceptions.ErrorCode;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.api.io.IIOManager;
+import org.apache.hyracks.control.common.controllers.NCConfig;
 import org.apache.hyracks.data.std.primitive.IntegerPointable;
 import org.apache.hyracks.dataflow.common.data.accessors.ITupleReference;
 import org.apache.hyracks.storage.am.btree.impls.BTree.BTreeAccessor;
@@ -72,7 +73,7 @@
 public class LSMRTree extends AbstractLSMRTree {
     protected final int[] buddyBTreeFields;
 
-    public LSMRTree(IIOManager ioManager, List<IVirtualBufferCache> virtualBufferCaches,
+    public LSMRTree(NCConfig storageConfig, IIOManager ioManager, List<IVirtualBufferCache> virtualBufferCaches,
             RTreeFrameFactory rtreeInteriorFrameFactory, RTreeFrameFactory rtreeLeafFrameFactory,
             ITreeIndexFrameFactory btreeInteriorFrameFactory, ITreeIndexFrameFactory btreeLeafFrameFactory,
             IBufferCache diskBufferCache, ILSMIndexFileManager fileNameManager,
@@ -85,7 +86,7 @@
             ILSMIOOperationCallbackFactory ioOpCallbackFactory, ILSMPageWriteCallbackFactory pageWriteCallbackFactory,
             int[] rtreeFields, int[] buddyBTreeFields, int[] filterFields, boolean durable, boolean isPointMBR)
             throws HyracksDataException {
-        super(ioManager, virtualBufferCaches, rtreeInteriorFrameFactory, rtreeLeafFrameFactory,
+        super(storageConfig, ioManager, virtualBufferCaches, rtreeInteriorFrameFactory, rtreeLeafFrameFactory,
                 btreeInteriorFrameFactory, btreeLeafFrameFactory, diskBufferCache, fileNameManager, componentFactory,
                 componentFactory, fieldCount, rtreeCmpFactories, btreeCmpFactories, linearizer, comparatorFields,
                 linearizerArray, bloomFilterFalsePositiveRate, mergePolicy, opTracker, ioScheduler, ioOpCallbackFactory,
@@ -117,8 +118,9 @@
                 rTreeTupleSorter.sort();
                 component = createDiskComponent(componentFactory, flushOp.getTarget(), flushOp.getBTreeTarget(),
                         flushOp.getBloomFilterTarget(), true);
-                componentBulkLoader = component.createBulkLoader(operation, 1.0f, false, numBTreeTuples.longValue(),
-                        false, false, false, pageWriteCallbackFactory.createPageWriteCallback());
+                componentBulkLoader =
+                        component.createBulkLoader(storageConfig, operation, 1.0f, false, numBTreeTuples.longValue(),
+                                false, false, false, pageWriteCallbackFactory.createPageWriteCallback());
                 flushLoadRTree(isEmpty, rTreeTupleSorter, componentBulkLoader);
                 // scan the memory BTree and bulk load delete tuples
                 flushLoadBtree(memBTreeAccessor, componentBulkLoader, btreeNullPredicate);
@@ -319,13 +321,13 @@
                     numElements += ((LSMRTreeDiskComponent) mergeOp.getMergingComponents().get(i)).getBloomFilter()
                             .getNumElements();
                 }
-                componentBulkLoader = mergedComponent.createBulkLoader(mergeOp, 1.0f, false, numElements, false, false,
-                        false, pageWriteCallbackFactory.createPageWriteCallback());
+                componentBulkLoader = mergedComponent.createBulkLoader(storageConfig, mergeOp, 1.0f, false, numElements,
+                        false, false, false, pageWriteCallbackFactory.createPageWriteCallback());
                 mergeLoadBTree(mergeOp, opCtx, rtreeSearchPred, componentBulkLoader);
             } else {
                 //no buddy-btree needed
-                componentBulkLoader = mergedComponent.createBulkLoader(mergeOp, 1.0f, false, 0L, false, false, false,
-                        pageWriteCallbackFactory.createPageWriteCallback());
+                componentBulkLoader = mergedComponent.createBulkLoader(storageConfig, mergeOp, 1.0f, false, 0L, false,
+                        false, false, pageWriteCallbackFactory.createPageWriteCallback());
             }
             //search old rtree components
             while (cursor.hasNext()) {
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/impls/LSMRTreeWithAntiMatterTuples.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/impls/LSMRTreeWithAntiMatterTuples.java
index 6968b3c..2c223e4 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/impls/LSMRTreeWithAntiMatterTuples.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/impls/LSMRTreeWithAntiMatterTuples.java
@@ -26,6 +26,7 @@
 import org.apache.hyracks.api.dataflow.value.ILinearizeComparatorFactory;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.api.io.IIOManager;
+import org.apache.hyracks.control.common.controllers.NCConfig;
 import org.apache.hyracks.dataflow.common.data.accessors.ITupleReference;
 import org.apache.hyracks.storage.am.btree.impls.BTree.BTreeAccessor;
 import org.apache.hyracks.storage.am.btree.impls.BTreeRangeSearchCursor;
@@ -71,10 +72,11 @@
 public class LSMRTreeWithAntiMatterTuples extends AbstractLSMRTree {
     private static final ICursorFactory cursorFactory = opCtx -> new LSMRTreeWithAntiMatterTuplesSearchCursor(opCtx);
 
-    public LSMRTreeWithAntiMatterTuples(IIOManager ioManager, List<IVirtualBufferCache> virtualBufferCaches,
-            RTreeFrameFactory rtreeInteriorFrameFactory, RTreeFrameFactory rtreeLeafFrameFactory,
-            ITreeIndexFrameFactory btreeInteriorFrameFactory, ITreeIndexFrameFactory btreeLeafFrameFactory,
-            IBufferCache diskBufferCache, ILSMIndexFileManager fileManager, ILSMDiskComponentFactory componentFactory,
+    public LSMRTreeWithAntiMatterTuples(NCConfig storageConfig, IIOManager ioManager,
+            List<IVirtualBufferCache> virtualBufferCaches, RTreeFrameFactory rtreeInteriorFrameFactory,
+            RTreeFrameFactory rtreeLeafFrameFactory, ITreeIndexFrameFactory btreeInteriorFrameFactory,
+            ITreeIndexFrameFactory btreeLeafFrameFactory, IBufferCache diskBufferCache,
+            ILSMIndexFileManager fileManager, ILSMDiskComponentFactory componentFactory,
             ILSMDiskComponentFactory bulkLoadComponentFactory, IComponentFilterHelper filterHelper,
             ILSMComponentFilterFrameFactory filterFrameFactory, LSMComponentFilterManager filterManager, int fieldCount,
             IBinaryComparatorFactory[] rtreeCmpFactories, IBinaryComparatorFactory[] btreeComparatorFactories,
@@ -82,7 +84,7 @@
             ILSMMergePolicy mergePolicy, ILSMOperationTracker opTracker, ILSMIOOperationScheduler ioScheduler,
             ILSMIOOperationCallbackFactory ioOpCallbackFactory, ILSMPageWriteCallbackFactory pageWriteCallbackFactory,
             int[] rtreeFields, int[] filterFields, boolean durable, boolean isPointMBR) throws HyracksDataException {
-        super(ioManager, virtualBufferCaches, rtreeInteriorFrameFactory, rtreeLeafFrameFactory,
+        super(storageConfig, ioManager, virtualBufferCaches, rtreeInteriorFrameFactory, rtreeLeafFrameFactory,
                 btreeInteriorFrameFactory, btreeLeafFrameFactory, diskBufferCache, fileManager, componentFactory,
                 bulkLoadComponentFactory, fieldCount, rtreeCmpFactories, btreeComparatorFactories, linearizer,
                 comparatorFields, linearizerArray, 0, mergePolicy, opTracker, ioScheduler, ioOpCallbackFactory,
@@ -113,8 +115,8 @@
                     try {
                         memRTreeAccessor.search(rtreeScanCursor, rtreeNullPredicate);
                         component = createDiskComponent(componentFactory, flushOp.getTarget(), null, null, true);
-                        componentBulkLoader = component.createBulkLoader(operation, 1.0f, false, 0L, false, false,
-                                false, pageWriteCallbackFactory.createPageWriteCallback());
+                        componentBulkLoader = component.createBulkLoader(storageConfig, operation, 1.0f, false, 0L,
+                                false, false, false, pageWriteCallbackFactory.createPageWriteCallback());
                         // Since the LSM-RTree is used as a secondary assumption, the
                         // primary key will be the last comparator in the BTree comparators
                         rTreeTupleSorter =
@@ -244,8 +246,8 @@
         // Bulk load the tuples from all on-disk RTrees into the new RTree.
         ILSMDiskComponent component = createDiskComponent(componentFactory, mergeOp.getTarget(), null, null, true);
 
-        ILSMDiskComponentBulkLoader componentBulkLoader = component.createBulkLoader(operation, 1.0f, false, 0L, false,
-                false, false, pageWriteCallbackFactory.createPageWriteCallback());
+        ILSMDiskComponentBulkLoader componentBulkLoader = component.createBulkLoader(storageConfig, operation, 1.0f,
+                false, 0L, false, false, false, pageWriteCallbackFactory.createPageWriteCallback());
         try {
             try {
                 while (cursor.hasNext()) {
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/utils/LSMRTreeUtils.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/utils/LSMRTreeUtils.java
index 4b16f4a..bc5bb45 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/utils/LSMRTreeUtils.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/utils/LSMRTreeUtils.java
@@ -28,6 +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.control.common.controllers.NCConfig;
 import org.apache.hyracks.data.std.primitive.DoublePointable;
 import org.apache.hyracks.data.std.primitive.IntegerPointable;
 import org.apache.hyracks.storage.am.bloomfilter.impls.BloomFilterFactory;
@@ -75,17 +76,17 @@
 import org.apache.hyracks.storage.common.buffercache.IBufferCache;
 
 public class LSMRTreeUtils {
-    public static LSMRTree createLSMTree(IIOManager ioManager, List<IVirtualBufferCache> virtualBufferCaches,
-            FileReference file, IBufferCache diskBufferCache, ITypeTraits[] typeTraits,
-            IBinaryComparatorFactory[] rtreeCmpFactories, IBinaryComparatorFactory[] btreeCmpFactories,
-            IPrimitiveValueProviderFactory[] valueProviderFactories, RTreePolicyType rtreePolicyType,
-            double bloomFilterFalsePositiveRate, ILSMMergePolicy mergePolicy, ILSMOperationTracker opTracker,
-            ILSMIOOperationScheduler ioScheduler, ILSMIOOperationCallbackFactory ioOpCallbackFactory,
-            ILSMPageWriteCallbackFactory pageWriteCallbackFactory, ILinearizeComparatorFactory linearizeCmpFactory,
-            int[] rtreeFields, int[] buddyBTreeFields, ITypeTraits[] filterTypeTraits,
-            IBinaryComparatorFactory[] filterCmpFactories, int[] filterFields, boolean durable, boolean isPointMBR,
-            IMetadataPageManagerFactory freePageManagerFactory, ITypeTraits nullTypeTraits,
-            INullIntrospector nullIntrospector) throws HyracksDataException {
+    public static LSMRTree createLSMTree(NCConfig storageConfig, IIOManager ioManager,
+            List<IVirtualBufferCache> virtualBufferCaches, FileReference file, IBufferCache diskBufferCache,
+            ITypeTraits[] typeTraits, IBinaryComparatorFactory[] rtreeCmpFactories,
+            IBinaryComparatorFactory[] btreeCmpFactories, IPrimitiveValueProviderFactory[] valueProviderFactories,
+            RTreePolicyType rtreePolicyType, double bloomFilterFalsePositiveRate, ILSMMergePolicy mergePolicy,
+            ILSMOperationTracker opTracker, ILSMIOOperationScheduler ioScheduler,
+            ILSMIOOperationCallbackFactory ioOpCallbackFactory, ILSMPageWriteCallbackFactory pageWriteCallbackFactory,
+            ILinearizeComparatorFactory linearizeCmpFactory, int[] rtreeFields, int[] buddyBTreeFields,
+            ITypeTraits[] filterTypeTraits, IBinaryComparatorFactory[] filterCmpFactories, int[] filterFields,
+            boolean durable, boolean isPointMBR, IMetadataPageManagerFactory freePageManagerFactory,
+            ITypeTraits nullTypeTraits, INullIntrospector nullIntrospector) throws HyracksDataException {
         int valueFieldCount = buddyBTreeFields.length;
         int keyFieldCount = typeTraits.length - valueFieldCount;
         ITypeTraits[] btreeTypeTraits = new ITypeTraits[valueFieldCount];
@@ -138,17 +139,18 @@
         ILSMDiskComponentFactory componentFactory =
                 new LSMRTreeDiskComponentFactory(diskRTreeFactory, diskBTreeFactory, bloomFilterFactory, filterHelper);
 
-        return new LSMRTree(ioManager, virtualBufferCaches, rtreeInteriorFrameFactory, rtreeLeafFrameFactory,
-                btreeInteriorFrameFactory, btreeLeafFrameFactory, diskBufferCache, fileNameManager, componentFactory,
-                filterHelper, filterFrameFactory, filterManager, bloomFilterFalsePositiveRate, typeTraits.length,
-                rtreeCmpFactories, btreeCmpFactories, linearizeCmpFactory, comparatorFields, linearizerArray,
-                mergePolicy, opTracker, ioScheduler, ioOpCallbackFactory, pageWriteCallbackFactory, rtreeFields,
-                buddyBTreeFields, filterFields, durable, isPointMBR);
+        return new LSMRTree(storageConfig, ioManager, virtualBufferCaches, rtreeInteriorFrameFactory,
+                rtreeLeafFrameFactory, btreeInteriorFrameFactory, btreeLeafFrameFactory, diskBufferCache,
+                fileNameManager, componentFactory, filterHelper, filterFrameFactory, filterManager,
+                bloomFilterFalsePositiveRate, typeTraits.length, rtreeCmpFactories, btreeCmpFactories,
+                linearizeCmpFactory, comparatorFields, linearizerArray, mergePolicy, opTracker, ioScheduler,
+                ioOpCallbackFactory, pageWriteCallbackFactory, rtreeFields, buddyBTreeFields, filterFields, durable,
+                isPointMBR);
     }
 
-    public static LSMRTreeWithAntiMatterTuples createLSMTreeWithAntiMatterTuples(IIOManager ioManager,
-            List<IVirtualBufferCache> virtualBufferCaches, FileReference file, IBufferCache diskBufferCache,
-            ITypeTraits[] typeTraits, IBinaryComparatorFactory[] rtreeCmpFactories,
+    public static LSMRTreeWithAntiMatterTuples createLSMTreeWithAntiMatterTuples(NCConfig storageConfig,
+            IIOManager ioManager, List<IVirtualBufferCache> virtualBufferCaches, FileReference file,
+            IBufferCache diskBufferCache, ITypeTraits[] typeTraits, IBinaryComparatorFactory[] rtreeCmpFactories,
             IBinaryComparatorFactory[] btreeComparatorFactories,
             IPrimitiveValueProviderFactory[] valueProviderFactories, RTreePolicyType rtreePolicyType,
             ILSMMergePolicy mergePolicy, ILSMOperationTracker opTracker, ILSMIOOperationScheduler ioScheduler,
@@ -237,12 +239,12 @@
         ILSMDiskComponentFactory bulkLoadComponentFactory =
                 new LSMRTreeWithAntiMatterTuplesDiskComponentFactory(bulkLoadRTreeFactory, filterHelper);
 
-        return new LSMRTreeWithAntiMatterTuples(ioManager, virtualBufferCaches, rtreeInteriorFrameFactory,
-                rtreeLeafFrameFactory, btreeInteriorFrameFactory, btreeLeafFrameFactory, diskBufferCache,
-                fileNameManager, componentFactory, bulkLoadComponentFactory, filterHelper, filterFrameFactory,
-                filterManager, typeTraits.length, rtreeCmpFactories, btreeComparatorFactories, linearizerCmpFactory,
-                comparatorFields, linearizerArray, mergePolicy, opTracker, ioScheduler, ioOpCallbackFactory,
-                pageWriteCallbackFactory, rtreeFields, filterFields, durable, isPointMBR);
+        return new LSMRTreeWithAntiMatterTuples(storageConfig, ioManager, virtualBufferCaches,
+                rtreeInteriorFrameFactory, rtreeLeafFrameFactory, btreeInteriorFrameFactory, btreeLeafFrameFactory,
+                diskBufferCache, fileNameManager, componentFactory, bulkLoadComponentFactory, filterHelper,
+                filterFrameFactory, filterManager, typeTraits.length, rtreeCmpFactories, btreeComparatorFactories,
+                linearizerCmpFactory, comparatorFields, linearizerArray, mergePolicy, opTracker, ioScheduler,
+                ioOpCallbackFactory, pageWriteCallbackFactory, rtreeFields, filterFields, durable, isPointMBR);
     }
 
     public static ILinearizeComparatorFactory proposeBestLinearizer(ITypeTraits[] typeTraits, int numKeyFields)
diff --git a/hyracks-fullstack/hyracks/hyracks-test-support/src/main/java/org/apache/hyracks/test/support/TestNCServiceContext.java b/hyracks-fullstack/hyracks/hyracks-test-support/src/main/java/org/apache/hyracks/test/support/TestNCServiceContext.java
index 7b0abc2..f6d9e13 100644
--- a/hyracks-fullstack/hyracks/hyracks-test-support/src/main/java/org/apache/hyracks/test/support/TestNCServiceContext.java
+++ b/hyracks-fullstack/hyracks/hyracks-test-support/src/main/java/org/apache/hyracks/test/support/TestNCServiceContext.java
@@ -35,22 +35,26 @@
 import org.apache.hyracks.api.messages.IMessageBroker;
 import org.apache.hyracks.api.resources.memory.IMemoryManager;
 import org.apache.hyracks.api.service.IControllerService;
+import org.apache.hyracks.control.common.controllers.NCConfig;
+import org.apache.hyracks.control.nc.NodeControllerService;
 import org.apache.hyracks.util.trace.ITracer;
 
 public class TestNCServiceContext implements INCServiceContext {
     private final ILifeCycleComponentManager lccm;
     private final IIOManager ioManager;
     private final String nodeId;
+    private NodeControllerService ncs;
 
     private Serializable distributedState;
     private Object appCtx;
 
     private final IMemoryManager mm;
 
-    public TestNCServiceContext(IIOManager ioManager, String nodeId) {
+    public TestNCServiceContext(IIOManager ioManager, String nodeId) throws Exception {
         this.lccm = new LifeCycleComponentManager();
         this.ioManager = ioManager;
         this.nodeId = nodeId;
+        this.ncs = new NodeControllerService(new NCConfig(nodeId));
         mm = new IMemoryManager() {
             @Override
             public long getMaximumMemory() {
@@ -138,7 +142,7 @@
 
     @Override
     public IControllerService getControllerService() {
-        return null;
+        return ncs;
     }
 
     @Override
diff --git a/hyracks-fullstack/hyracks/hyracks-test-support/src/main/java/org/apache/hyracks/test/support/TestUtils.java b/hyracks-fullstack/hyracks/hyracks-test-support/src/main/java/org/apache/hyracks/test/support/TestUtils.java
index 2348a1a..d5a9a43 100644
--- a/hyracks-fullstack/hyracks/hyracks-test-support/src/main/java/org/apache/hyracks/test/support/TestUtils.java
+++ b/hyracks-fullstack/hyracks/hyracks-test-support/src/main/java/org/apache/hyracks/test/support/TestUtils.java
@@ -73,12 +73,12 @@
 
     public static IHyracksTaskContext create(int frameSize, IOManager ioManager) {
         try {
-            INCServiceContext serviceCtx = new TestNCServiceContext(ioManager, null);
+            INCServiceContext serviceCtx = new TestNCServiceContext(ioManager, "asterix_nc1");
             TestJobletContext jobletCtx = new TestJobletContext(frameSize, serviceCtx, new JobId(0));
             TaskAttemptId tid = new TaskAttemptId(new TaskId(new ActivityId(new OperatorDescriptorId(0), 0), 0), 0);
             IHyracksTaskContext taskCtx = new TestTaskContext(jobletCtx, tid, 1);
             return taskCtx;
-        } catch (HyracksException e) {
+        } catch (Exception e) {
             throw new RuntimeException(e);
         }
     }
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/pom.xml b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/pom.xml
index 1d16c62..788ccf3 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/pom.xml
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/pom.xml
@@ -116,6 +116,12 @@
     </dependency>
     <dependency>
       <groupId>org.apache.hyracks</groupId>
+      <artifactId>hyracks-control-common</artifactId>
+      <version>${project.version}</version>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hyracks</groupId>
       <artifactId>hyracks-storage-common</artifactId>
       <version>${project.version}</version>
       <scope>test</scope>
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/LSMBTreeBulkLoadTest.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/LSMBTreeBulkLoadTest.java
index 1477535..32a42bd 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/LSMBTreeBulkLoadTest.java
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/LSMBTreeBulkLoadTest.java
@@ -53,10 +53,10 @@
     @Override
     protected OrderedIndexTestContext createTestContext(ISerializerDeserializer[] fieldSerdes, int numKeys,
             BTreeLeafFrameType leafType, boolean filtered) throws Exception {
-        return LSMBTreeTestContext.create(harness.getIOManager(), harness.getVirtualBufferCaches(),
-                harness.getFileReference(), harness.getDiskBufferCache(), fieldSerdes, numKeys,
-                harness.getBoomFilterFalsePositiveRate(), harness.getMergePolicy(), harness.getOperationTracker(),
-                harness.getIOScheduler(), harness.getIOOperationCallbackFactory(),
+        return LSMBTreeTestContext.create(harness.getNCConfig(), harness.getIOManager(),
+                harness.getVirtualBufferCaches(), harness.getFileReference(), harness.getDiskBufferCache(), fieldSerdes,
+                numKeys, harness.getBoomFilterFalsePositiveRate(), harness.getMergePolicy(),
+                harness.getOperationTracker(), harness.getIOScheduler(), harness.getIOOperationCallbackFactory(),
                 harness.getPageWriteCallbackFactory(), harness.getMetadataPageManagerFactory(), false, true, false,
                 harness.getCompressorDecompressorFactory());
     }
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/LSMBTreeComponentLifecycleTest.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/LSMBTreeComponentLifecycleTest.java
index 8608cac..6ab4f06 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/LSMBTreeComponentLifecycleTest.java
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/LSMBTreeComponentLifecycleTest.java
@@ -78,10 +78,10 @@
 
     private OrderedIndexTestContext createTestContext(ISerializerDeserializer[] fieldSerdes, int numKeys,
             ILSMIOOperationScheduler scheduler, ILSMIOOperationCallbackFactory ioCallbackFactory) throws Exception {
-        return LSMBTreeTestContext.create(harness.getIOManager(), harness.getVirtualBufferCaches(),
-                harness.getFileReference(), harness.getDiskBufferCache(), fieldSerdes, numKeys,
-                harness.getBoomFilterFalsePositiveRate(), harness.getMergePolicy(), harness.getOperationTracker(),
-                scheduler, ioCallbackFactory, harness.getPageWriteCallbackFactory(),
+        return LSMBTreeTestContext.create(harness.getNCConfig(), harness.getIOManager(),
+                harness.getVirtualBufferCaches(), harness.getFileReference(), harness.getDiskBufferCache(), fieldSerdes,
+                numKeys, harness.getBoomFilterFalsePositiveRate(), harness.getMergePolicy(),
+                harness.getOperationTracker(), scheduler, ioCallbackFactory, harness.getPageWriteCallbackFactory(),
                 harness.getMetadataPageManagerFactory(), false, true, false,
                 harness.getCompressorDecompressorFactory());
     }
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/LSMBTreeDeleteTest.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/LSMBTreeDeleteTest.java
index fb25ee7..73f8090 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/LSMBTreeDeleteTest.java
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/LSMBTreeDeleteTest.java
@@ -53,10 +53,10 @@
     @Override
     protected OrderedIndexTestContext createTestContext(ISerializerDeserializer[] fieldSerdes, int numKeys,
             BTreeLeafFrameType leafType, boolean filtered) throws Exception {
-        return LSMBTreeTestContext.create(harness.getIOManager(), harness.getVirtualBufferCaches(),
-                harness.getFileReference(), harness.getDiskBufferCache(), fieldSerdes, numKeys,
-                harness.getBoomFilterFalsePositiveRate(), harness.getMergePolicy(), harness.getOperationTracker(),
-                harness.getIOScheduler(), harness.getIOOperationCallbackFactory(),
+        return LSMBTreeTestContext.create(harness.getNCConfig(), harness.getIOManager(),
+                harness.getVirtualBufferCaches(), harness.getFileReference(), harness.getDiskBufferCache(), fieldSerdes,
+                numKeys, harness.getBoomFilterFalsePositiveRate(), harness.getMergePolicy(),
+                harness.getOperationTracker(), harness.getIOScheduler(), harness.getIOOperationCallbackFactory(),
                 harness.getPageWriteCallbackFactory(), harness.getMetadataPageManagerFactory(), false, true, false,
                 harness.getCompressorDecompressorFactory());
     }
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 3313443..9019795 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
@@ -58,9 +58,9 @@
             IBinaryComparatorFactory[] cmpFactories, int[] bloomFilterKeyFields, ITypeTraits[] filterTypeTraits,
             IBinaryComparatorFactory[] filterCmpFactories, int[] btreeFields, int[] filterFields)
             throws HyracksDataException {
-        return LSMBTreeUtil.createLSMTree(harness.getIOManager(), harness.getVirtualBufferCaches(),
-                harness.getFileReference(), harness.getDiskBufferCache(), typeTraits, cmpFactories,
-                bloomFilterKeyFields, harness.getBoomFilterFalsePositiveRate(), harness.getMergePolicy(),
+        return LSMBTreeUtil.createLSMTree(harness.getNCConfig(), harness.getIOManager(),
+                harness.getVirtualBufferCaches(), harness.getFileReference(), harness.getDiskBufferCache(), typeTraits,
+                cmpFactories, bloomFilterKeyFields, harness.getBoomFilterFalsePositiveRate(), harness.getMergePolicy(),
                 harness.getOperationTracker(), harness.getIOScheduler(), harness.getIOOperationCallbackFactory(),
                 harness.getPageWriteCallbackFactory(), true, filterTypeTraits, filterCmpFactories, btreeFields,
                 filterFields, true, harness.getMetadataPageManagerFactory(), false, ITracer.NONE,
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/LSMBTreeFileManagerTest.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/LSMBTreeFileManagerTest.java
index c607cae..3b6111a 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/LSMBTreeFileManagerTest.java
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/LSMBTreeFileManagerTest.java
@@ -54,9 +54,9 @@
     @Test
     public void deleteOrphanedFilesTest() throws Exception {
         ISerializerDeserializer[] fieldSerdes = { IntegerSerializerDeserializer.INSTANCE };
-        LSMBTreeTestContext ctx = LSMBTreeTestContext.create(harness.getIOManager(), harness.getVirtualBufferCaches(),
-                harness.getFileReference(), harness.getDiskBufferCache(), fieldSerdes, 1,
-                harness.getBoomFilterFalsePositiveRate(), harness.getMergePolicy(), harness.getOperationTracker(),
+        LSMBTreeTestContext ctx = LSMBTreeTestContext.create(harness.getNCConfig(), harness.getIOManager(),
+                harness.getVirtualBufferCaches(), harness.getFileReference(), harness.getDiskBufferCache(), fieldSerdes,
+                1, harness.getBoomFilterFalsePositiveRate(), harness.getMergePolicy(), harness.getOperationTracker(),
                 harness.getIOScheduler(), harness.getIOOperationCallbackFactory(),
                 harness.getPageWriteCallbackFactory(), harness.getMetadataPageManagerFactory(), false, true, false,
                 harness.getCompressorDecompressorFactory());
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/LSMBTreeFilterMergeTest.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/LSMBTreeFilterMergeTest.java
index bf76e20..1cfc712 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/LSMBTreeFilterMergeTest.java
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/LSMBTreeFilterMergeTest.java
@@ -52,10 +52,10 @@
     @Override
     protected OrderedIndexTestContext createTestContext(ISerializerDeserializer[] fieldSerdes, int numKeys,
             BTreeLeafFrameType leafType, boolean filtered) throws Exception {
-        return LSMBTreeTestContext.create(harness.getIOManager(), harness.getVirtualBufferCaches(),
-                harness.getFileReference(), harness.getDiskBufferCache(), fieldSerdes, numKeys,
-                harness.getBoomFilterFalsePositiveRate(), harness.getMergePolicy(), harness.getOperationTracker(),
-                harness.getIOScheduler(), harness.getIOOperationCallbackFactory(),
+        return LSMBTreeTestContext.create(harness.getNCConfig(), harness.getIOManager(),
+                harness.getVirtualBufferCaches(), harness.getFileReference(), harness.getDiskBufferCache(), fieldSerdes,
+                numKeys, harness.getBoomFilterFalsePositiveRate(), harness.getMergePolicy(),
+                harness.getOperationTracker(), harness.getIOScheduler(), harness.getIOOperationCallbackFactory(),
                 harness.getPageWriteCallbackFactory(), harness.getMetadataPageManagerFactory(), filtered, true, false,
                 harness.getCompressorDecompressorFactory());
     }
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/LSMBTreeInsertTest.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/LSMBTreeInsertTest.java
index dace731..a07a11d 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/LSMBTreeInsertTest.java
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/LSMBTreeInsertTest.java
@@ -53,10 +53,10 @@
     @Override
     protected OrderedIndexTestContext createTestContext(ISerializerDeserializer[] fieldSerdes, int numKeys,
             BTreeLeafFrameType leafType, boolean filtered) throws Exception {
-        return LSMBTreeTestContext.create(harness.getIOManager(), harness.getVirtualBufferCaches(),
-                harness.getFileReference(), harness.getDiskBufferCache(), fieldSerdes, numKeys,
-                harness.getBoomFilterFalsePositiveRate(), harness.getMergePolicy(), harness.getOperationTracker(),
-                harness.getIOScheduler(), harness.getIOOperationCallbackFactory(),
+        return LSMBTreeTestContext.create(harness.getNCConfig(), harness.getIOManager(),
+                harness.getVirtualBufferCaches(), harness.getFileReference(), harness.getDiskBufferCache(), fieldSerdes,
+                numKeys, harness.getBoomFilterFalsePositiveRate(), harness.getMergePolicy(),
+                harness.getOperationTracker(), harness.getIOScheduler(), harness.getIOOperationCallbackFactory(),
                 harness.getPageWriteCallbackFactory(), harness.getMetadataPageManagerFactory(), false, true, false,
                 harness.getCompressorDecompressorFactory());
     }
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/LSMBTreeLifecycleTest.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/LSMBTreeLifecycleTest.java
index d131ccb..967eca9 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/LSMBTreeLifecycleTest.java
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/LSMBTreeLifecycleTest.java
@@ -54,10 +54,10 @@
     @Override
     public void setup() throws Exception {
         harness.setUp();
-        testCtx = LSMBTreeTestContext.create(harness.getIOManager(), harness.getVirtualBufferCaches(),
-                harness.getFileReference(), harness.getDiskBufferCache(), fieldSerdes, fieldSerdes.length,
-                harness.getBoomFilterFalsePositiveRate(), harness.getMergePolicy(), harness.getOperationTracker(),
-                harness.getIOScheduler(), harness.getIOOperationCallbackFactory(),
+        testCtx = LSMBTreeTestContext.create(harness.getNCConfig(), harness.getIOManager(),
+                harness.getVirtualBufferCaches(), harness.getFileReference(), harness.getDiskBufferCache(), fieldSerdes,
+                fieldSerdes.length, harness.getBoomFilterFalsePositiveRate(), harness.getMergePolicy(),
+                harness.getOperationTracker(), harness.getIOScheduler(), harness.getIOOperationCallbackFactory(),
                 harness.getPageWriteCallbackFactory(), harness.getMetadataPageManagerFactory(), false, true, false,
                 harness.getCompressorDecompressorFactory());
         index = testCtx.getIndex();
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/LSMBTreeMergeFailTest.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/LSMBTreeMergeFailTest.java
index a720ec8..8b66b82 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/LSMBTreeMergeFailTest.java
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/LSMBTreeMergeFailTest.java
@@ -100,11 +100,11 @@
 
     protected LSMBTreeTestContext createTestContext(ISerializerDeserializer[] fieldSerdes, int numKeys,
             BTreeLeafFrameType leafType, boolean filtered) throws Exception {
-        return LSMBTreeTestContext.create(harness.getIOManager(), harness.getVirtualBufferCaches(),
-                harness.getFileReference(), harness.getDiskBufferCache(), fieldSerdes, numKeys,
-                harness.getBoomFilterFalsePositiveRate(), harness.getMergePolicy(), harness.getOperationTracker(),
-                scheduler, harness.getIOOperationCallbackFactory(), harness.getPageWriteCallbackFactory(),
-                harness.getMetadataPageManagerFactory(), filtered, true, false,
+        return LSMBTreeTestContext.create(harness.getNCConfig(), harness.getIOManager(),
+                harness.getVirtualBufferCaches(), harness.getFileReference(), harness.getDiskBufferCache(), fieldSerdes,
+                numKeys, harness.getBoomFilterFalsePositiveRate(), harness.getMergePolicy(),
+                harness.getOperationTracker(), scheduler, harness.getIOOperationCallbackFactory(),
+                harness.getPageWriteCallbackFactory(), harness.getMetadataPageManagerFactory(), filtered, true, false,
                 harness.getCompressorDecompressorFactory());
     }
 
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/LSMBTreeMergeTest.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/LSMBTreeMergeTest.java
index b21a8b9..1cbc3cd 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/LSMBTreeMergeTest.java
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/LSMBTreeMergeTest.java
@@ -52,10 +52,10 @@
     @Override
     protected OrderedIndexTestContext createTestContext(ISerializerDeserializer[] fieldSerdes, int numKeys,
             BTreeLeafFrameType leafType, boolean filtered) throws Exception {
-        return LSMBTreeTestContext.create(harness.getIOManager(), harness.getVirtualBufferCaches(),
-                harness.getFileReference(), harness.getDiskBufferCache(), fieldSerdes, numKeys,
-                harness.getBoomFilterFalsePositiveRate(), harness.getMergePolicy(), harness.getOperationTracker(),
-                harness.getIOScheduler(), harness.getIOOperationCallbackFactory(),
+        return LSMBTreeTestContext.create(harness.getNCConfig(), harness.getIOManager(),
+                harness.getVirtualBufferCaches(), harness.getFileReference(), harness.getDiskBufferCache(), fieldSerdes,
+                numKeys, harness.getBoomFilterFalsePositiveRate(), harness.getMergePolicy(),
+                harness.getOperationTracker(), harness.getIOScheduler(), harness.getIOOperationCallbackFactory(),
                 harness.getPageWriteCallbackFactory(), harness.getMetadataPageManagerFactory(), filtered, true, false,
                 harness.getCompressorDecompressorFactory());
     }
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 2476761..d902dfe 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
@@ -47,8 +47,9 @@
 
     @Override
     protected void createIndexInstance() throws Exception {
-        index = LSMBTreeUtil.createLSMTree(harness.getIOManager(), harness.getVirtualBufferCaches(),
-                harness.getFileReference(), harness.getDiskBufferCache(), SerdeUtils.serdesToTypeTraits(keySerdes),
+        index = LSMBTreeUtil.createLSMTree(harness.getNCConfig(), harness.getIOManager(),
+                harness.getVirtualBufferCaches(), harness.getFileReference(), harness.getDiskBufferCache(),
+                SerdeUtils.serdesToTypeTraits(keySerdes),
                 SerdeUtils.serdesToComparatorFactories(keySerdes, keySerdes.length), bloomFilterKeyFields,
                 harness.getBoomFilterFalsePositiveRate(), harness.getMergePolicy(),
                 NoOpOperationTrackerFactory.INSTANCE.getOperationTracker(null, null), harness.getIOScheduler(),
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/LSMBTreeMultiBulkLoadTest.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/LSMBTreeMultiBulkLoadTest.java
index d131b1f..9850ab3 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/LSMBTreeMultiBulkLoadTest.java
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/LSMBTreeMultiBulkLoadTest.java
@@ -54,10 +54,10 @@
     @Override
     protected OrderedIndexTestContext createTestContext(ISerializerDeserializer[] fieldSerdes, int numKeys,
             BTreeLeafFrameType leafType, boolean filtered) throws Exception {
-        return LSMBTreeTestContext.create(harness.getIOManager(), harness.getVirtualBufferCaches(),
-                harness.getFileReference(), harness.getDiskBufferCache(), fieldSerdes, numKeys,
-                harness.getBoomFilterFalsePositiveRate(), harness.getMergePolicy(), harness.getOperationTracker(),
-                harness.getIOScheduler(), harness.getIOOperationCallbackFactory(),
+        return LSMBTreeTestContext.create(harness.getNCConfig(), harness.getIOManager(),
+                harness.getVirtualBufferCaches(), harness.getFileReference(), harness.getDiskBufferCache(), fieldSerdes,
+                numKeys, harness.getBoomFilterFalsePositiveRate(), harness.getMergePolicy(),
+                harness.getOperationTracker(), harness.getIOScheduler(), harness.getIOOperationCallbackFactory(),
                 harness.getPageWriteCallbackFactory(), harness.getMetadataPageManagerFactory(), false, true, false,
                 harness.getCompressorDecompressorFactory());
     }
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/LSMBTreePageWriteCallbackTest.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/LSMBTreePageWriteCallbackTest.java
index a14d4ec..44f7d6e 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/LSMBTreePageWriteCallbackTest.java
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/LSMBTreePageWriteCallbackTest.java
@@ -110,11 +110,11 @@
     @Override
     protected OrderedIndexTestContext createTestContext(ISerializerDeserializer[] fieldSerdes, int numKeys,
             BTreeLeafFrameType leafType, boolean filtered) throws Exception {
-        return LSMBTreeTestContext.create(harness.getIOManager(), harness.getVirtualBufferCaches(),
-                harness.getFileReference(), harness.getDiskBufferCache(), fieldSerdes, numKeys,
-                harness.getBoomFilterFalsePositiveRate(), harness.getMergePolicy(), harness.getOperationTracker(),
-                harness.getIOScheduler(), harness.getIOOperationCallbackFactory(), pageWriteCallbackFactory,
-                harness.getMetadataPageManagerFactory(), false, true, false,
+        return LSMBTreeTestContext.create(harness.getNCConfig(), harness.getIOManager(),
+                harness.getVirtualBufferCaches(), harness.getFileReference(), harness.getDiskBufferCache(), fieldSerdes,
+                numKeys, harness.getBoomFilterFalsePositiveRate(), harness.getMergePolicy(),
+                harness.getOperationTracker(), harness.getIOScheduler(), harness.getIOOperationCallbackFactory(),
+                pageWriteCallbackFactory, harness.getMetadataPageManagerFactory(), false, true, false,
                 harness.getCompressorDecompressorFactory());
     }
 
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/LSMBTreeScanDiskComponentsTest.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/LSMBTreeScanDiskComponentsTest.java
index 4f2b2f7..4628f3a 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/LSMBTreeScanDiskComponentsTest.java
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/LSMBTreeScanDiskComponentsTest.java
@@ -76,10 +76,10 @@
     @Override
     protected OrderedIndexTestContext createTestContext(ISerializerDeserializer[] fieldSerdes, int numKeys,
             BTreeLeafFrameType leafType, boolean filtered) throws Exception {
-        return LSMBTreeTestContext.create(harness.getIOManager(), harness.getVirtualBufferCaches(),
-                harness.getFileReference(), harness.getDiskBufferCache(), fieldSerdes, numKeys,
-                harness.getBoomFilterFalsePositiveRate(), harness.getMergePolicy(), harness.getOperationTracker(),
-                harness.getIOScheduler(), harness.getIOOperationCallbackFactory(),
+        return LSMBTreeTestContext.create(harness.getNCConfig(), harness.getIOManager(),
+                harness.getVirtualBufferCaches(), harness.getFileReference(), harness.getDiskBufferCache(), fieldSerdes,
+                numKeys, harness.getBoomFilterFalsePositiveRate(), harness.getMergePolicy(),
+                harness.getOperationTracker(), harness.getIOScheduler(), harness.getIOOperationCallbackFactory(),
                 harness.getPageWriteCallbackFactory(), harness.getMetadataPageManagerFactory(), false, true, false,
                 harness.getCompressorDecompressorFactory());
     }
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 6dec9f4..9de96db 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
@@ -56,8 +56,9 @@
 
     @Override
     protected void createIndexInstance() throws Exception {
-        index = LSMBTreeUtil.createLSMTree(harness.getIOManager(), harness.getVirtualBufferCaches(),
-                harness.getFileReference(), harness.getDiskBufferCache(), SerdeUtils.serdesToTypeTraits(keySerdes),
+        index = LSMBTreeUtil.createLSMTree(harness.getNCConfig(), harness.getIOManager(),
+                harness.getVirtualBufferCaches(), harness.getFileReference(), harness.getDiskBufferCache(),
+                SerdeUtils.serdesToTypeTraits(keySerdes),
                 SerdeUtils.serdesToComparatorFactories(keySerdes, keySerdes.length), bloomFilterKeyFields,
                 harness.getBoomFilterFalsePositiveRate(), harness.getMergePolicy(),
                 NoOpOperationTrackerFactory.INSTANCE.getOperationTracker(null, null), harness.getIOScheduler(),
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/LSMBTreeUpdateInPlaceScanDiskComponentsTest.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/LSMBTreeUpdateInPlaceScanDiskComponentsTest.java
index 0125318..d976243 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/LSMBTreeUpdateInPlaceScanDiskComponentsTest.java
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/LSMBTreeUpdateInPlaceScanDiskComponentsTest.java
@@ -119,10 +119,10 @@
     @Override
     protected OrderedIndexTestContext createTestContext(ISerializerDeserializer[] fieldSerdes, int numKeys,
             BTreeLeafFrameType leafType, boolean filtered) throws HyracksDataException {
-        return LSMBTreeTestContext.create(harness.getIOManager(), harness.getVirtualBufferCaches(),
-                harness.getFileReference(), harness.getDiskBufferCache(), fieldSerdes, numKeys,
-                harness.getBoomFilterFalsePositiveRate(), harness.getMergePolicy(), harness.getOperationTracker(),
-                harness.getIOScheduler(), harness.getIOOperationCallbackFactory(),
+        return LSMBTreeTestContext.create(harness.getNCConfig(), harness.getIOManager(),
+                harness.getVirtualBufferCaches(), harness.getFileReference(), harness.getDiskBufferCache(), fieldSerdes,
+                numKeys, harness.getBoomFilterFalsePositiveRate(), harness.getMergePolicy(),
+                harness.getOperationTracker(), harness.getIOScheduler(), harness.getIOOperationCallbackFactory(),
                 harness.getPageWriteCallbackFactory(), harness.getMetadataPageManagerFactory(), false, !hasOnlyKeys,
                 hasOnlyKeys, harness.getCompressorDecompressorFactory());
     }
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/LSMBTreeUpdateInPlaceTest.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/LSMBTreeUpdateInPlaceTest.java
index 0a4b37b..ee2b689 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/LSMBTreeUpdateInPlaceTest.java
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/LSMBTreeUpdateInPlaceTest.java
@@ -69,8 +69,9 @@
 
     @Override
     protected void createIndexInstance() throws Exception {
-        index = LSMBTreeUtil.createLSMTree(harness.getIOManager(), harness.getVirtualBufferCaches(),
-                harness.getFileReference(), harness.getDiskBufferCache(), SerdeUtils.serdesToTypeTraits(keySerdes),
+        index = LSMBTreeUtil.createLSMTree(harness.getNCConfig(), harness.getIOManager(),
+                harness.getVirtualBufferCaches(), harness.getFileReference(), harness.getDiskBufferCache(),
+                SerdeUtils.serdesToTypeTraits(keySerdes),
                 SerdeUtils.serdesToComparatorFactories(keySerdes, keySerdes.length), bloomFilterKeyFields,
                 harness.getBoomFilterFalsePositiveRate(), harness.getMergePolicy(),
                 NoOpOperationTrackerFactory.INSTANCE.getOperationTracker(null, null), harness.getIOScheduler(),
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/LSMBTreeUpdateTest.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/LSMBTreeUpdateTest.java
index de2dfa5..58b235e 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/LSMBTreeUpdateTest.java
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/LSMBTreeUpdateTest.java
@@ -53,10 +53,10 @@
     @Override
     protected OrderedIndexTestContext createTestContext(ISerializerDeserializer[] fieldSerdes, int numKeys,
             BTreeLeafFrameType leafType, boolean filtered) throws Exception {
-        return LSMBTreeTestContext.create(harness.getIOManager(), harness.getVirtualBufferCaches(),
-                harness.getFileReference(), harness.getDiskBufferCache(), fieldSerdes, numKeys,
-                harness.getBoomFilterFalsePositiveRate(), harness.getMergePolicy(), harness.getOperationTracker(),
-                harness.getIOScheduler(), harness.getIOOperationCallbackFactory(),
+        return LSMBTreeTestContext.create(harness.getNCConfig(), harness.getIOManager(),
+                harness.getVirtualBufferCaches(), harness.getFileReference(), harness.getDiskBufferCache(), fieldSerdes,
+                numKeys, harness.getBoomFilterFalsePositiveRate(), harness.getMergePolicy(),
+                harness.getOperationTracker(), harness.getIOScheduler(), harness.getIOOperationCallbackFactory(),
                 harness.getPageWriteCallbackFactory(), harness.getMetadataPageManagerFactory(), false, true, false,
                 harness.getCompressorDecompressorFactory());
     }
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/impl/TestLsmBtree.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/impl/TestLsmBtree.java
index 8107827..695395f 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/impl/TestLsmBtree.java
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/impl/TestLsmBtree.java
@@ -25,6 +25,7 @@
 import org.apache.hyracks.api.dataflow.value.IBinaryComparatorFactory;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.api.io.IIOManager;
+import org.apache.hyracks.control.common.controllers.NCConfig;
 import org.apache.hyracks.dataflow.common.data.accessors.ITupleReference;
 import org.apache.hyracks.storage.am.common.api.IIndexOperationContext;
 import org.apache.hyracks.storage.am.common.api.ITreeIndexFrameFactory;
@@ -79,7 +80,7 @@
     private volatile int numFinishedMerges;
     private volatile int numStartedMerges;
 
-    public TestLsmBtree(IIOManager ioManager, List<IVirtualBufferCache> virtualBufferCaches,
+    public TestLsmBtree(NCConfig storageConfig, IIOManager ioManager, List<IVirtualBufferCache> virtualBufferCaches,
             ITreeIndexFrameFactory interiorFrameFactory, ITreeIndexFrameFactory insertLeafFrameFactory,
             ITreeIndexFrameFactory deleteLeafFrameFactory, IBufferCache diskBufferCache,
             ILSMIndexFileManager fileManager, ILSMDiskComponentFactory componentFactory,
@@ -91,11 +92,11 @@
             ILSMPageWriteCallbackFactory pageWriteCallbackFactory, boolean needKeyDupCheck, boolean hasBloomFilter,
             int[] btreeFields, int[] filterFields, boolean durable, boolean updateAware, ITracer tracer)
             throws HyracksDataException {
-        super(ioManager, virtualBufferCaches, interiorFrameFactory, insertLeafFrameFactory, deleteLeafFrameFactory,
-                diskBufferCache, fileManager, componentFactory, bulkLoadComponentFactory, filterHelper,
-                filterFrameFactory, filterManager, bloomFilterFalsePositiveRate, fieldCount, cmpFactories, mergePolicy,
-                opTracker, ioScheduler, ioOperationCallbackFactory, pageWriteCallbackFactory, needKeyDupCheck,
-                hasBloomFilter, btreeFields, filterFields, durable, updateAware, tracer, false);
+        super(storageConfig, ioManager, virtualBufferCaches, interiorFrameFactory, insertLeafFrameFactory,
+                deleteLeafFrameFactory, diskBufferCache, fileManager, componentFactory, bulkLoadComponentFactory,
+                filterHelper, filterFrameFactory, filterManager, bloomFilterFalsePositiveRate, fieldCount, cmpFactories,
+                mergePolicy, opTracker, ioScheduler, ioOperationCallbackFactory, pageWriteCallbackFactory,
+                needKeyDupCheck, hasBloomFilter, btreeFields, filterFields, durable, updateAware, tracer, false);
         addModifyCallback(AllowTestOpCallback.INSTANCE);
         addSearchCallback(AllowTestOpCallback.INSTANCE);
         addFlushCallback(AllowTestOpCallback.INSTANCE);
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/impl/TestLsmBtreeLocalResource.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/impl/TestLsmBtreeLocalResource.java
index 04fa11f..5928d5b 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/impl/TestLsmBtreeLocalResource.java
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/impl/TestLsmBtreeLocalResource.java
@@ -29,6 +29,8 @@
 import org.apache.hyracks.api.io.IIOManager;
 import org.apache.hyracks.api.io.IJsonSerializable;
 import org.apache.hyracks.api.io.IPersistedResourceRegistry;
+import org.apache.hyracks.control.common.controllers.NCConfig;
+import org.apache.hyracks.control.nc.NodeControllerService;
 import org.apache.hyracks.storage.am.common.api.IMetadataPageManagerFactory;
 import org.apache.hyracks.storage.am.lsm.btree.dataflow.LSMBTreeLocalResource;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallbackFactory;
@@ -76,6 +78,7 @@
     @Override
     public ILSMIndex createInstance(INCServiceContext serviceCtx) throws HyracksDataException {
         IIOManager ioManager = storageManager.getIoManager(serviceCtx);
+        NCConfig storageConfig = ((NodeControllerService) serviceCtx.getControllerService()).getConfiguration();
         FileReference file = ioManager.resolve(path);
         List<IVirtualBufferCache> vbcs = vbcProvider.getVirtualBufferCaches(serviceCtx, file);
         for (int i = 0; i < vbcs.size(); i++) {
@@ -87,9 +90,9 @@
         }
         ioOpCallbackFactory.initialize(serviceCtx, this);
         pageWriteCallbackFactory.initialize(serviceCtx, this);
-        return TestLsmBtreeUtil.createLSMTree(ioManager, vbcs, file, storageManager.getBufferCache(serviceCtx),
-                typeTraits, cmpFactories, bloomFilterKeyFields, bloomFilterFalsePositiveRate,
-                mergePolicyFactory.createMergePolicy(mergePolicyProperties, serviceCtx),
+        return TestLsmBtreeUtil.createLSMTree(storageConfig, ioManager, vbcs, file,
+                storageManager.getBufferCache(serviceCtx), typeTraits, cmpFactories, bloomFilterKeyFields,
+                bloomFilterFalsePositiveRate, mergePolicyFactory.createMergePolicy(mergePolicyProperties, serviceCtx),
                 opTrackerProvider.getOperationTracker(serviceCtx, this), ioSchedulerProvider.getIoScheduler(serviceCtx),
                 ioOpCallbackFactory, pageWriteCallbackFactory, isPrimary, filterTypeTraits, filterCmpFactories,
                 btreeFields, filterFields, durable, metadataPageManagerFactory, false, serviceCtx.getTracer());
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/impl/TestLsmBtreeUtil.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/impl/TestLsmBtreeUtil.java
index dd42838..3fa5745 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/impl/TestLsmBtreeUtil.java
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/impl/TestLsmBtreeUtil.java
@@ -25,6 +25,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.control.common.controllers.NCConfig;
 import org.apache.hyracks.storage.am.bloomfilter.impls.BloomFilterFactory;
 import org.apache.hyracks.storage.am.btree.frames.BTreeNSMInteriorFrameFactory;
 import org.apache.hyracks.storage.am.btree.frames.BTreeNSMLeafFrameFactory;
@@ -59,13 +60,14 @@
     private TestLsmBtreeUtil() {
     }
 
-    public static LSMBTree createLSMTree(IIOManager ioManager, List<IVirtualBufferCache> virtualBufferCaches,
-            FileReference file, IBufferCache diskBufferCache, ITypeTraits[] typeTraits,
-            IBinaryComparatorFactory[] cmpFactories, int[] bloomFilterKeyFields, double bloomFilterFalsePositiveRate,
-            ILSMMergePolicy mergePolicy, ILSMOperationTracker opTracker, ILSMIOOperationScheduler ioScheduler,
-            ILSMIOOperationCallbackFactory ioOpCallbackFactory, ILSMPageWriteCallbackFactory pageWriteCallbackFactory,
-            boolean needKeyDupCheck, ITypeTraits[] filterTypeTraits, IBinaryComparatorFactory[] filterCmpFactories,
-            int[] btreeFields, int[] filterFields, boolean durable, IMetadataPageManagerFactory freePageManagerFactory,
+    public static LSMBTree createLSMTree(NCConfig storageConfig, IIOManager ioManager,
+            List<IVirtualBufferCache> virtualBufferCaches, FileReference file, IBufferCache diskBufferCache,
+            ITypeTraits[] typeTraits, IBinaryComparatorFactory[] cmpFactories, int[] bloomFilterKeyFields,
+            double bloomFilterFalsePositiveRate, ILSMMergePolicy mergePolicy, ILSMOperationTracker opTracker,
+            ILSMIOOperationScheduler ioScheduler, ILSMIOOperationCallbackFactory ioOpCallbackFactory,
+            ILSMPageWriteCallbackFactory pageWriteCallbackFactory, boolean needKeyDupCheck,
+            ITypeTraits[] filterTypeTraits, IBinaryComparatorFactory[] filterCmpFactories, int[] btreeFields,
+            int[] filterFields, boolean durable, IMetadataPageManagerFactory freePageManagerFactory,
             boolean updateAware, ITracer tracer) throws HyracksDataException {
         LSMBTreeTupleWriterFactory insertTupleWriterFactory =
                 new LSMBTreeTupleWriterFactory(typeTraits, cmpFactories.length, false, updateAware, null, null);
@@ -114,10 +116,11 @@
             bulkLoadComponentFactory = new LSMBTreeDiskComponentFactory(bulkLoadBTreeFactory, filterHelper);
         }
 
-        return new TestLsmBtree(ioManager, virtualBufferCaches, interiorFrameFactory, insertLeafFrameFactory,
-                deleteLeafFrameFactory, diskBufferCache, fileNameManager, componentFactory, bulkLoadComponentFactory,
-                filterHelper, filterFrameFactory, filterManager, bloomFilterFalsePositiveRate, typeTraits.length,
-                cmpFactories, mergePolicy, opTracker, ioScheduler, ioOpCallbackFactory, pageWriteCallbackFactory,
-                needKeyDupCheck, hasBloomFilter, btreeFields, filterFields, durable, updateAware, tracer);
+        return new TestLsmBtree(storageConfig, ioManager, virtualBufferCaches, interiorFrameFactory,
+                insertLeafFrameFactory, deleteLeafFrameFactory, diskBufferCache, fileNameManager, componentFactory,
+                bulkLoadComponentFactory, filterHelper, filterFrameFactory, filterManager, bloomFilterFalsePositiveRate,
+                typeTraits.length, cmpFactories, mergePolicy, opTracker, ioScheduler, ioOpCallbackFactory,
+                pageWriteCallbackFactory, needKeyDupCheck, hasBloomFilter, btreeFields, filterFields, durable,
+                updateAware, tracer);
     }
 }
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 66af4b2..e4004a9 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
@@ -54,9 +54,9 @@
     @Override
     protected ITreeIndex createIndex(ITypeTraits[] typeTraits, IBinaryComparatorFactory[] cmpFactories,
             int[] bloomFilterKeyFields) throws HyracksDataException {
-        return LSMBTreeUtil.createLSMTree(harness.getIOManager(), harness.getVirtualBufferCaches(),
-                harness.getFileReference(), harness.getDiskBufferCache(), typeTraits, cmpFactories,
-                bloomFilterKeyFields, harness.getBoomFilterFalsePositiveRate(), harness.getMergePolicy(),
+        return LSMBTreeUtil.createLSMTree(harness.getNCConfig(), harness.getIOManager(),
+                harness.getVirtualBufferCaches(), harness.getFileReference(), harness.getDiskBufferCache(), typeTraits,
+                cmpFactories, bloomFilterKeyFields, harness.getBoomFilterFalsePositiveRate(), harness.getMergePolicy(),
                 harness.getOperationTracker(), harness.getIOScheduler(), harness.getIOOperationCallbackFactory(),
                 harness.getPageWriteCallbackFactory(), true, null, null, null, null, true,
                 harness.getMetadataPageManagerFactory(), false, ITracer.NONE,
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 6126040..c099ad2 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
@@ -30,6 +30,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.control.common.controllers.NCConfig;
 import org.apache.hyracks.control.nc.io.IOManager;
 import org.apache.hyracks.storage.am.common.datagen.DataGenThread;
 import org.apache.hyracks.storage.am.common.datagen.TupleBatch;
@@ -66,6 +67,7 @@
     protected IHyracksTaskContext ctx;
     protected IOManager ioManager;
     protected int ioDeviceId;
+    protected NCConfig ncConfig;
     protected IBufferCache bufferCache;
     protected int lsmtreeFileId;
 
@@ -95,6 +97,7 @@
         TestStorageManagerComponentHolder.init(this.onDiskPageSize, this.onDiskNumPages, MAX_OPEN_FILES);
         bufferCache = TestStorageManagerComponentHolder.getBufferCache(ctx.getJobletContext().getServiceContext());
         ioManager = TestStorageManagerComponentHolder.getIOManager();
+        ncConfig = new NCConfig(null);
 
         ioDeviceId = 0;
         file = ioManager.resolveAbsolutePath(onDiskDir);
@@ -118,7 +121,7 @@
             }
         }, Integer.MAX_VALUE, Integer.MAX_VALUE);
 
-        lsmtree = LSMBTreeUtil.createLSMTree(ioManager, virtualBufferCaches, file, bufferCache, typeTraits,
+        lsmtree = LSMBTreeUtil.createLSMTree(ncConfig, ioManager, virtualBufferCaches, file, bufferCache, typeTraits,
                 cmpFactories, bloomFilterKeyFields, bloomFilterFalsePositiveRate, new NoMergePolicy(),
                 new ThreadCountingTracker(), ioScheduler, NoOpIOOperationCallbackFactory.INSTANCE,
                 NoOpPageWriteCallbackFactory.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/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 82f6f4b..8f60c1f 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
@@ -29,6 +29,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.control.common.controllers.NCConfig;
 import org.apache.hyracks.dataflow.common.utils.SerdeUtils;
 import org.apache.hyracks.storage.am.btree.OrderedIndexTestContext;
 import org.apache.hyracks.storage.am.common.CheckTuple;
@@ -76,11 +77,11 @@
         upsertCheckTuple(checkTuple, checkTuples);
     }
 
-    public static LSMBTreeTestContext create(IIOManager ioManager, List<IVirtualBufferCache> virtualBufferCaches,
-            FileReference file, IBufferCache diskBufferCache, ISerializerDeserializer[] fieldSerdes, int numKeyFields,
-            double bloomFilterFalsePositiveRate, ILSMMergePolicy mergePolicy, ILSMOperationTracker opTracker,
-            ILSMIOOperationScheduler ioScheduler, ILSMIOOperationCallbackFactory ioOpCallbackFactory,
-            ILSMPageWriteCallbackFactory pageWriteCallbackFactory,
+    public static LSMBTreeTestContext create(NCConfig storageConfig, IIOManager ioManager,
+            List<IVirtualBufferCache> virtualBufferCaches, FileReference file, IBufferCache diskBufferCache,
+            ISerializerDeserializer[] fieldSerdes, int numKeyFields, double bloomFilterFalsePositiveRate,
+            ILSMMergePolicy mergePolicy, ILSMOperationTracker opTracker, ILSMIOOperationScheduler ioScheduler,
+            ILSMIOOperationCallbackFactory ioOpCallbackFactory, ILSMPageWriteCallbackFactory pageWriteCallbackFactory,
             IMetadataPageManagerFactory metadataPageManagerFactory, boolean filtered, boolean needKeyDupCheck,
             boolean updateAware, ICompressorDecompressorFactory compressorDecompressorFactory)
             throws HyracksDataException {
@@ -100,16 +101,16 @@
             }
             int[] filterfields = { btreefields.length };
             IBinaryComparatorFactory[] filterCmp = { cmpFactories[0] };
-            lsmTree = LSMBTreeUtil.createLSMTree(ioManager, virtualBufferCaches, file, diskBufferCache, typeTraits,
-                    cmpFactories, bloomFilterKeyFields, bloomFilterFalsePositiveRate, mergePolicy, opTracker,
-                    ioScheduler, ioOpCallbackFactory, pageWriteCallbackFactory, needKeyDupCheck, filterTypeTraits,
-                    filterCmp, btreefields, filterfields, true, metadataPageManagerFactory, updateAware, ITracer.NONE,
-                    compressorDecompressorFactory, true, null, null);
+            lsmTree = LSMBTreeUtil.createLSMTree(storageConfig, ioManager, virtualBufferCaches, file, diskBufferCache,
+                    typeTraits, cmpFactories, bloomFilterKeyFields, bloomFilterFalsePositiveRate, mergePolicy,
+                    opTracker, ioScheduler, ioOpCallbackFactory, pageWriteCallbackFactory, needKeyDupCheck,
+                    filterTypeTraits, filterCmp, btreefields, filterfields, true, metadataPageManagerFactory,
+                    updateAware, ITracer.NONE, compressorDecompressorFactory, true, null, null);
         } else {
-            lsmTree = LSMBTreeUtil.createLSMTree(ioManager, virtualBufferCaches, file, diskBufferCache, typeTraits,
-                    cmpFactories, bloomFilterKeyFields, bloomFilterFalsePositiveRate, mergePolicy, opTracker,
-                    ioScheduler, ioOpCallbackFactory, pageWriteCallbackFactory, needKeyDupCheck, null, null, null, null,
-                    true, metadataPageManagerFactory,
+            lsmTree = LSMBTreeUtil.createLSMTree(storageConfig, ioManager, virtualBufferCaches, file, diskBufferCache,
+                    typeTraits, cmpFactories, bloomFilterKeyFields, bloomFilterFalsePositiveRate, mergePolicy,
+                    opTracker, ioScheduler, ioOpCallbackFactory, pageWriteCallbackFactory, needKeyDupCheck, null, null,
+                    null, null, true, metadataPageManagerFactory,
                     updateAware, new Tracer(LSMBTreeTestContext.class.getSimpleName(),
                             ITraceCategoryRegistry.CATEGORIES_ALL, new TraceCategoryRegistry()),
                     compressorDecompressorFactory, true, null, 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/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 4d2b086..3eb9bf0 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
@@ -32,6 +32,7 @@
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.api.io.FileReference;
 import org.apache.hyracks.api.io.IODeviceHandle;
+import org.apache.hyracks.control.common.controllers.NCConfig;
 import org.apache.hyracks.control.nc.io.IOManager;
 import org.apache.hyracks.storage.am.btree.frames.BTreeLeafFrameType;
 import org.apache.hyracks.storage.am.common.api.IMetadataPageManagerFactory;
@@ -76,6 +77,7 @@
     private final ICompressorDecompressorFactory compressorDecompressorFactory;
 
     protected IOManager ioManager;
+    protected NCConfig ncConfig;
     protected int ioDeviceId;
     protected IBufferCache diskBufferCache;
     protected List<IVirtualBufferCache> virtualBufferCaches;
@@ -113,6 +115,7 @@
         this.ioOpCallbackFactory = new CountingIoOperationCallbackFactory();
         this.pageWriteCallbackFactory = NoOpPageWriteCallbackFactory.INSTANCE;
         this.compressorDecompressorFactory = compressorDecompressorFactory;
+        this.ncConfig = new NCConfig(null);
     }
 
     public void setUp() throws HyracksDataException {
@@ -181,6 +184,10 @@
         return ioManager;
     }
 
+    public NCConfig getNCConfig() {
+        return ncConfig;
+    }
+
     public int getIODeviceId() {
         return ioDeviceId;
     }
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/pom.xml b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/pom.xml
index 9e2cd3d..6d17134 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/pom.xml
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/pom.xml
@@ -106,6 +106,12 @@
     </dependency>
     <dependency>
       <groupId>org.apache.hyracks</groupId>
+      <artifactId>hyracks-control-common</artifactId>
+      <version>${project.version}</version>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hyracks</groupId>
       <artifactId>hyracks-storage-common</artifactId>
       <version>${project.version}</version>
       <scope>test</scope>
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 c220697..0960608 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
@@ -31,6 +31,7 @@
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.api.io.FileReference;
 import org.apache.hyracks.api.io.IODeviceHandle;
+import org.apache.hyracks.control.common.controllers.NCConfig;
 import org.apache.hyracks.control.nc.io.IOManager;
 import org.apache.hyracks.storage.am.common.api.IMetadataPageManagerFactory;
 import org.apache.hyracks.storage.am.common.freepage.AppendOnlyLinkedMetadataPageManagerFactory;
@@ -67,6 +68,7 @@
     protected final int numMutableComponents;
 
     protected IOManager ioManager;
+    protected NCConfig ncConfig;
     protected int ioDeviceId;
     protected IBufferCache diskBufferCache;
     protected List<IVirtualBufferCache> virtualBufferCaches;
@@ -101,6 +103,7 @@
         this.ioOpCallbackFactory = NoOpIOOperationCallbackFactory.INSTANCE;
         this.pageWriteCallbackFactory = NoOpPageWriteCallbackFactory.INSTANCE;
         this.numMutableComponents = AccessMethodTestsConfig.LSM_INVINDEX_NUM_MUTABLE_COMPONENTS;
+        this.ncConfig = new NCConfig(null);
     }
 
     public void setUp() throws HyracksDataException {
@@ -177,6 +180,10 @@
         return ioManager;
     }
 
+    public NCConfig getNCConfig() {
+        return ncConfig;
+    }
+
     public int getIODeviceId() {
         return ioDeviceId;
     }
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 e745f5d..b5da79f 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
@@ -34,6 +34,7 @@
 import org.apache.hyracks.api.exceptions.ErrorCode;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.api.util.HyracksConstants;
+import org.apache.hyracks.control.common.controllers.NCConfig;
 import org.apache.hyracks.control.nc.io.IOManager;
 import org.apache.hyracks.dataflow.common.data.accessors.ITupleReference;
 import org.apache.hyracks.dataflow.common.utils.SerdeUtils;
@@ -134,6 +135,7 @@
             throws HyracksDataException {
         ITypeTraits[] allTypeTraits = SerdeUtils.serdesToTypeTraits(fieldSerdes);
         IOManager ioManager = harness.getIOManager();
+        NCConfig storageConfig = harness.getNCConfig();
         IBinaryComparatorFactory[] allCmpFactories =
                 SerdeUtils.serdesToComparatorFactories(fieldSerdes, fieldSerdes.length);
         // Set token type traits and comparators.
@@ -186,8 +188,8 @@
                 break;
             }
             case LSM: {
-                invIndex =
-                        InvertedIndexUtils.createLSMInvertedIndex(ioManager, harness.getVirtualBufferCaches(),
+                invIndex = InvertedIndexUtils
+                        .createLSMInvertedIndex(storageConfig, ioManager, harness.getVirtualBufferCaches(),
                                 invListTypeTraits, invListCmpFactories, tokenTypeTraits, tokenCmpFactories,
                                 tokenizerFactory, fullTextConfigEvaluatorFactory, harness.getDiskBufferCache(),
                                 harness.getOnDiskDir(), harness.getBoomFilterFalsePositiveRate(),
@@ -203,7 +205,7 @@
             }
             case PARTITIONED_LSM: {
                 invIndex = InvertedIndexUtils
-                        .createPartitionedLSMInvertedIndex(ioManager, harness.getVirtualBufferCaches(),
+                        .createPartitionedLSMInvertedIndex(storageConfig, ioManager, harness.getVirtualBufferCaches(),
                                 invListTypeTraits, invListCmpFactories, tokenTypeTraits, tokenCmpFactories,
                                 tokenizerFactory, fullTextConfigEvaluatorFactory, harness.getDiskBufferCache(),
                                 harness.getOnDiskDir(), harness.getBoomFilterFalsePositiveRate(),
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/pom.xml b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/pom.xml
index fca340e..ab764be 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/pom.xml
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/pom.xml
@@ -94,6 +94,12 @@
     </dependency>
     <dependency>
       <groupId>org.apache.hyracks</groupId>
+      <artifactId>hyracks-control-common</artifactId>
+      <version>${project.version}</version>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hyracks</groupId>
       <artifactId>hyracks-storage-common</artifactId>
       <version>${project.version}</version>
       <scope>test</scope>
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/org/apache/hyracks/storage/am/lsm/rtree/LSMRTreeBulkLoadTest.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/org/apache/hyracks/storage/am/lsm/rtree/LSMRTreeBulkLoadTest.java
index 21b6a70..e642de4 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/org/apache/hyracks/storage/am/lsm/rtree/LSMRTreeBulkLoadTest.java
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/org/apache/hyracks/storage/am/lsm/rtree/LSMRTreeBulkLoadTest.java
@@ -56,11 +56,12 @@
     protected AbstractRTreeTestContext createTestContext(ISerializerDeserializer[] fieldSerdes,
             IPrimitiveValueProviderFactory[] valueProviderFactories, int numKeys, RTreePolicyType rtreePolicyType)
             throws Exception {
-        return LSMRTreeTestContext.create(harness.getIOManager(), harness.getVirtualBufferCaches(),
-                harness.getFileReference(), harness.getDiskBufferCache(), fieldSerdes, valueProviderFactories, numKeys,
-                rtreePolicyType, harness.getBoomFilterFalsePositiveRate(), harness.getMergePolicy(),
-                harness.getOperationTracker(), harness.getIOScheduler(), harness.getIOOperationCallbackFactory(),
-                harness.getPageWriteCallbackFactory(), harness.getMetadataPageManagerFactory());
+        return LSMRTreeTestContext.create(harness.getNcConfig(), harness.getIOManager(),
+                harness.getVirtualBufferCaches(), harness.getFileReference(), harness.getDiskBufferCache(), fieldSerdes,
+                valueProviderFactories, numKeys, rtreePolicyType, harness.getBoomFilterFalsePositiveRate(),
+                harness.getMergePolicy(), harness.getOperationTracker(), harness.getIOScheduler(),
+                harness.getIOOperationCallbackFactory(), harness.getPageWriteCallbackFactory(),
+                harness.getMetadataPageManagerFactory());
     }
 
     @Override
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/org/apache/hyracks/storage/am/lsm/rtree/LSMRTreeDeleteTest.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/org/apache/hyracks/storage/am/lsm/rtree/LSMRTreeDeleteTest.java
index 0662540..dc23f3b 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/org/apache/hyracks/storage/am/lsm/rtree/LSMRTreeDeleteTest.java
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/org/apache/hyracks/storage/am/lsm/rtree/LSMRTreeDeleteTest.java
@@ -56,11 +56,12 @@
     protected AbstractRTreeTestContext createTestContext(ISerializerDeserializer[] fieldSerdes,
             IPrimitiveValueProviderFactory[] valueProviderFactories, int numKeys, RTreePolicyType rtreePolicyType)
             throws Exception {
-        return LSMRTreeTestContext.create(harness.getIOManager(), harness.getVirtualBufferCaches(),
-                harness.getFileReference(), harness.getDiskBufferCache(), fieldSerdes, valueProviderFactories, numKeys,
-                rtreePolicyType, harness.getBoomFilterFalsePositiveRate(), harness.getMergePolicy(),
-                harness.getOperationTracker(), harness.getIOScheduler(), harness.getIOOperationCallbackFactory(),
-                harness.getPageWriteCallbackFactory(), harness.getMetadataPageManagerFactory());
+        return LSMRTreeTestContext.create(harness.getNcConfig(), harness.getIOManager(),
+                harness.getVirtualBufferCaches(), harness.getFileReference(), harness.getDiskBufferCache(), fieldSerdes,
+                valueProviderFactories, numKeys, rtreePolicyType, harness.getBoomFilterFalsePositiveRate(),
+                harness.getMergePolicy(), harness.getOperationTracker(), harness.getIOScheduler(),
+                harness.getIOOperationCallbackFactory(), harness.getPageWriteCallbackFactory(),
+                harness.getMetadataPageManagerFactory());
     }
 
     @Override
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/org/apache/hyracks/storage/am/lsm/rtree/LSMRTreeExamplesTest.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/org/apache/hyracks/storage/am/lsm/rtree/LSMRTreeExamplesTest.java
index 9e6fb83..71e2eb3 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/org/apache/hyracks/storage/am/lsm/rtree/LSMRTreeExamplesTest.java
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/org/apache/hyracks/storage/am/lsm/rtree/LSMRTreeExamplesTest.java
@@ -43,11 +43,12 @@
             IBinaryComparatorFactory[] btreeCmpFactories, IPrimitiveValueProviderFactory[] valueProviderFactories,
             RTreePolicyType rtreePolicyType, int[] rtreeFields, int[] btreeFields, ITypeTraits[] filterTypeTraits,
             IBinaryComparatorFactory[] filterCmpFactories, int[] filterFields) throws HyracksDataException {
-        return LSMRTreeUtils.createLSMTree(harness.getIOManager(), harness.getVirtualBufferCaches(),
-                harness.getFileReference(), harness.getDiskBufferCache(), typeTraits, rtreeCmpFactories,
-                btreeCmpFactories, valueProviderFactories, rtreePolicyType, harness.getBoomFilterFalsePositiveRate(),
-                harness.getMergePolicy(), harness.getOperationTracker(), harness.getIOScheduler(),
-                harness.getIOOperationCallbackFactory(), harness.getPageWriteCallbackFactory(),
+        return LSMRTreeUtils.createLSMTree(harness.getNcConfig(), harness.getIOManager(),
+                harness.getVirtualBufferCaches(), harness.getFileReference(), harness.getDiskBufferCache(), typeTraits,
+                rtreeCmpFactories, btreeCmpFactories, valueProviderFactories, rtreePolicyType,
+                harness.getBoomFilterFalsePositiveRate(), harness.getMergePolicy(), harness.getOperationTracker(),
+                harness.getIOScheduler(), harness.getIOOperationCallbackFactory(),
+                harness.getPageWriteCallbackFactory(),
                 LSMRTreeUtils.proposeBestLinearizer(typeTraits, rtreeCmpFactories.length), rtreeFields, btreeFields,
                 filterTypeTraits, filterCmpFactories, filterFields, true, false,
                 harness.getMetadataPageManagerFactory(), null, null);
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/org/apache/hyracks/storage/am/lsm/rtree/LSMRTreeInsertTest.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/org/apache/hyracks/storage/am/lsm/rtree/LSMRTreeInsertTest.java
index a48bfaf..515f3de 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/org/apache/hyracks/storage/am/lsm/rtree/LSMRTreeInsertTest.java
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/org/apache/hyracks/storage/am/lsm/rtree/LSMRTreeInsertTest.java
@@ -56,11 +56,12 @@
     protected AbstractRTreeTestContext createTestContext(ISerializerDeserializer[] fieldSerdes,
             IPrimitiveValueProviderFactory[] valueProviderFactories, int numKeys, RTreePolicyType rtreePolicyType)
             throws Exception {
-        return LSMRTreeTestContext.create(harness.getIOManager(), harness.getVirtualBufferCaches(),
-                harness.getFileReference(), harness.getDiskBufferCache(), fieldSerdes, valueProviderFactories, numKeys,
-                rtreePolicyType, harness.getBoomFilterFalsePositiveRate(), harness.getMergePolicy(),
-                harness.getOperationTracker(), harness.getIOScheduler(), harness.getIOOperationCallbackFactory(),
-                harness.getPageWriteCallbackFactory(), harness.getMetadataPageManagerFactory());
+        return LSMRTreeTestContext.create(harness.getNcConfig(), harness.getIOManager(),
+                harness.getVirtualBufferCaches(), harness.getFileReference(), harness.getDiskBufferCache(), fieldSerdes,
+                valueProviderFactories, numKeys, rtreePolicyType, harness.getBoomFilterFalsePositiveRate(),
+                harness.getMergePolicy(), harness.getOperationTracker(), harness.getIOScheduler(),
+                harness.getIOOperationCallbackFactory(), harness.getPageWriteCallbackFactory(),
+                harness.getMetadataPageManagerFactory());
     }
 
     @Override
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/org/apache/hyracks/storage/am/lsm/rtree/LSMRTreeLifecycleTest.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/org/apache/hyracks/storage/am/lsm/rtree/LSMRTreeLifecycleTest.java
index aa2bb63..4afb98e 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/org/apache/hyracks/storage/am/lsm/rtree/LSMRTreeLifecycleTest.java
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/org/apache/hyracks/storage/am/lsm/rtree/LSMRTreeLifecycleTest.java
@@ -66,11 +66,12 @@
     @Override
     public void setup() throws Exception {
         harness.setUp();
-        testCtx = LSMRTreeTestContext.create(harness.getIOManager(), harness.getVirtualBufferCaches(),
-                harness.getFileReference(), harness.getDiskBufferCache(), fieldSerdes, valueProviderFactories, numKeys,
-                RTreePolicyType.RTREE, harness.getBoomFilterFalsePositiveRate(), harness.getMergePolicy(),
-                harness.getOperationTracker(), harness.getIOScheduler(), harness.getIOOperationCallbackFactory(),
-                harness.getPageWriteCallbackFactory(), harness.getMetadataPageManagerFactory());
+        testCtx = LSMRTreeTestContext.create(harness.getNcConfig(), harness.getIOManager(),
+                harness.getVirtualBufferCaches(), harness.getFileReference(), harness.getDiskBufferCache(), fieldSerdes,
+                valueProviderFactories, numKeys, RTreePolicyType.RTREE, harness.getBoomFilterFalsePositiveRate(),
+                harness.getMergePolicy(), harness.getOperationTracker(), harness.getIOScheduler(),
+                harness.getIOOperationCallbackFactory(), harness.getPageWriteCallbackFactory(),
+                harness.getMetadataPageManagerFactory());
         index = testCtx.getIndex();
     }
 
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/org/apache/hyracks/storage/am/lsm/rtree/LSMRTreeMergeTest.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/org/apache/hyracks/storage/am/lsm/rtree/LSMRTreeMergeTest.java
index 864d555..1bd6ce0 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/org/apache/hyracks/storage/am/lsm/rtree/LSMRTreeMergeTest.java
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/org/apache/hyracks/storage/am/lsm/rtree/LSMRTreeMergeTest.java
@@ -55,11 +55,12 @@
     protected AbstractRTreeTestContext createTestContext(ISerializerDeserializer[] fieldSerdes,
             IPrimitiveValueProviderFactory[] valueProviderFactories, int numKeys, RTreePolicyType rtreePolicyType)
             throws Exception {
-        return LSMRTreeTestContext.create(harness.getIOManager(), harness.getVirtualBufferCaches(),
-                harness.getFileReference(), harness.getDiskBufferCache(), fieldSerdes, valueProviderFactories, numKeys,
-                rtreePolicyType, harness.getBoomFilterFalsePositiveRate(), harness.getMergePolicy(),
-                harness.getOperationTracker(), harness.getIOScheduler(), harness.getIOOperationCallbackFactory(),
-                harness.getPageWriteCallbackFactory(), harness.getMetadataPageManagerFactory());
+        return LSMRTreeTestContext.create(harness.getNcConfig(), harness.getIOManager(),
+                harness.getVirtualBufferCaches(), harness.getFileReference(), harness.getDiskBufferCache(), fieldSerdes,
+                valueProviderFactories, numKeys, rtreePolicyType, harness.getBoomFilterFalsePositiveRate(),
+                harness.getMergePolicy(), harness.getOperationTracker(), harness.getIOScheduler(),
+                harness.getIOOperationCallbackFactory(), harness.getPageWriteCallbackFactory(),
+                harness.getMetadataPageManagerFactory());
     }
 
     @Override
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/org/apache/hyracks/storage/am/lsm/rtree/LSMRTreePointMBRBulkLoadTest.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/org/apache/hyracks/storage/am/lsm/rtree/LSMRTreePointMBRBulkLoadTest.java
index dcdc8b5..134d71b 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/org/apache/hyracks/storage/am/lsm/rtree/LSMRTreePointMBRBulkLoadTest.java
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/org/apache/hyracks/storage/am/lsm/rtree/LSMRTreePointMBRBulkLoadTest.java
@@ -56,11 +56,12 @@
     protected AbstractRTreeTestContext createTestContext(ISerializerDeserializer[] fieldSerdes,
             IPrimitiveValueProviderFactory[] valueProviderFactories, int numKeys, RTreePolicyType rtreePolicyType)
             throws Exception {
-        return LSMRTreeTestContext.create(harness.getIOManager(), harness.getVirtualBufferCaches(),
-                harness.getFileReference(), harness.getDiskBufferCache(), fieldSerdes, valueProviderFactories, numKeys,
-                rtreePolicyType, harness.getBoomFilterFalsePositiveRate(), harness.getMergePolicy(),
-                harness.getOperationTracker(), harness.getIOScheduler(), harness.getIOOperationCallbackFactory(),
-                harness.getPageWriteCallbackFactory(), harness.getMetadataPageManagerFactory(), true);
+        return LSMRTreeTestContext.create(harness.getNcConfig(), harness.getIOManager(),
+                harness.getVirtualBufferCaches(), harness.getFileReference(), harness.getDiskBufferCache(), fieldSerdes,
+                valueProviderFactories, numKeys, rtreePolicyType, harness.getBoomFilterFalsePositiveRate(),
+                harness.getMergePolicy(), harness.getOperationTracker(), harness.getIOScheduler(),
+                harness.getIOOperationCallbackFactory(), harness.getPageWriteCallbackFactory(),
+                harness.getMetadataPageManagerFactory(), true);
     }
 
     @Override
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/org/apache/hyracks/storage/am/lsm/rtree/LSMRTreeWithAntiMatterTuplesBulkLoadTest.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/org/apache/hyracks/storage/am/lsm/rtree/LSMRTreeWithAntiMatterTuplesBulkLoadTest.java
index 1ac7d0b..30606b4 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/org/apache/hyracks/storage/am/lsm/rtree/LSMRTreeWithAntiMatterTuplesBulkLoadTest.java
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/org/apache/hyracks/storage/am/lsm/rtree/LSMRTreeWithAntiMatterTuplesBulkLoadTest.java
@@ -56,11 +56,11 @@
     protected AbstractRTreeTestContext createTestContext(ISerializerDeserializer[] fieldSerdes,
             IPrimitiveValueProviderFactory[] valueProviderFactories, int numKeys, RTreePolicyType rtreePolicyType)
             throws Exception {
-        return LSMRTreeWithAntiMatterTuplesTestContext.create(harness.getIOManager(), harness.getVirtualBufferCaches(),
-                harness.getFileReference(), harness.getDiskBufferCache(), fieldSerdes, valueProviderFactories, numKeys,
-                rtreePolicyType, harness.getMergePolicy(), harness.getOperationTracker(), harness.getIOScheduler(),
-                harness.getIOOperationCallbackFactory(), harness.getPageWriteCallbackFactory(),
-                harness.getMetadataPageManagerFactory());
+        return LSMRTreeWithAntiMatterTuplesTestContext.create(harness.getNcConfig(), harness.getIOManager(),
+                harness.getVirtualBufferCaches(), harness.getFileReference(), harness.getDiskBufferCache(), fieldSerdes,
+                valueProviderFactories, numKeys, rtreePolicyType, harness.getMergePolicy(),
+                harness.getOperationTracker(), harness.getIOScheduler(), harness.getIOOperationCallbackFactory(),
+                harness.getPageWriteCallbackFactory(), harness.getMetadataPageManagerFactory());
     }
 
     @Override
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/org/apache/hyracks/storage/am/lsm/rtree/LSMRTreeWithAntiMatterTuplesDeleteTest.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/org/apache/hyracks/storage/am/lsm/rtree/LSMRTreeWithAntiMatterTuplesDeleteTest.java
index 2d74823..d55514b 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/org/apache/hyracks/storage/am/lsm/rtree/LSMRTreeWithAntiMatterTuplesDeleteTest.java
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/org/apache/hyracks/storage/am/lsm/rtree/LSMRTreeWithAntiMatterTuplesDeleteTest.java
@@ -57,11 +57,11 @@
     protected AbstractRTreeTestContext createTestContext(ISerializerDeserializer[] fieldSerdes,
             IPrimitiveValueProviderFactory[] valueProviderFactories, int numKeys, RTreePolicyType rtreePolicyType)
             throws Exception {
-        return LSMRTreeWithAntiMatterTuplesTestContext.create(harness.getIOManager(), harness.getVirtualBufferCaches(),
-                harness.getFileReference(), harness.getDiskBufferCache(), fieldSerdes, valueProviderFactories, numKeys,
-                rtreePolicyType, harness.getMergePolicy(), harness.getOperationTracker(), harness.getIOScheduler(),
-                harness.getIOOperationCallbackFactory(), harness.getPageWriteCallbackFactory(),
-                harness.getMetadataPageManagerFactory());
+        return LSMRTreeWithAntiMatterTuplesTestContext.create(harness.getNcConfig(), harness.getIOManager(),
+                harness.getVirtualBufferCaches(), harness.getFileReference(), harness.getDiskBufferCache(), fieldSerdes,
+                valueProviderFactories, numKeys, rtreePolicyType, harness.getMergePolicy(),
+                harness.getOperationTracker(), harness.getIOScheduler(), harness.getIOOperationCallbackFactory(),
+                harness.getPageWriteCallbackFactory(), harness.getMetadataPageManagerFactory());
     }
 
     @Override
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/org/apache/hyracks/storage/am/lsm/rtree/LSMRTreeWithAntiMatterTuplesExamplesTest.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/org/apache/hyracks/storage/am/lsm/rtree/LSMRTreeWithAntiMatterTuplesExamplesTest.java
index baf54f3..49fb157 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/org/apache/hyracks/storage/am/lsm/rtree/LSMRTreeWithAntiMatterTuplesExamplesTest.java
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/org/apache/hyracks/storage/am/lsm/rtree/LSMRTreeWithAntiMatterTuplesExamplesTest.java
@@ -43,9 +43,9 @@
             IBinaryComparatorFactory[] btreeCmpFactories, IPrimitiveValueProviderFactory[] valueProviderFactories,
             RTreePolicyType rtreePolicyType, int[] rtreeFields, int[] btreeFields, ITypeTraits[] filterTypeTraits,
             IBinaryComparatorFactory[] filterCmpFactories, int[] filterFields) throws HyracksDataException {
-        return LSMRTreeUtils.createLSMTreeWithAntiMatterTuples(harness.getIOManager(), harness.getVirtualBufferCaches(),
-                harness.getFileReference(), harness.getDiskBufferCache(), typeTraits, rtreeCmpFactories,
-                btreeCmpFactories, valueProviderFactories, rtreePolicyType, harness.getMergePolicy(),
+        return LSMRTreeUtils.createLSMTreeWithAntiMatterTuples(harness.getNcConfig(), harness.getIOManager(),
+                harness.getVirtualBufferCaches(), harness.getFileReference(), harness.getDiskBufferCache(), typeTraits,
+                rtreeCmpFactories, btreeCmpFactories, valueProviderFactories, rtreePolicyType, harness.getMergePolicy(),
                 harness.getOperationTracker(), harness.getIOScheduler(), harness.getIOOperationCallbackFactory(),
                 harness.getPageWriteCallbackFactory(),
                 LSMRTreeUtils.proposeBestLinearizer(typeTraits, rtreeCmpFactories.length), rtreeFields,
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/org/apache/hyracks/storage/am/lsm/rtree/LSMRTreeWithAntiMatterTuplesInsertTest.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/org/apache/hyracks/storage/am/lsm/rtree/LSMRTreeWithAntiMatterTuplesInsertTest.java
index 4872e68..15301de 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/org/apache/hyracks/storage/am/lsm/rtree/LSMRTreeWithAntiMatterTuplesInsertTest.java
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/org/apache/hyracks/storage/am/lsm/rtree/LSMRTreeWithAntiMatterTuplesInsertTest.java
@@ -56,11 +56,11 @@
     protected AbstractRTreeTestContext createTestContext(ISerializerDeserializer[] fieldSerdes,
             IPrimitiveValueProviderFactory[] valueProviderFactories, int numKeys, RTreePolicyType rtreePolicyType)
             throws Exception {
-        return LSMRTreeWithAntiMatterTuplesTestContext.create(harness.getIOManager(), harness.getVirtualBufferCaches(),
-                harness.getFileReference(), harness.getDiskBufferCache(), fieldSerdes, valueProviderFactories, numKeys,
-                rtreePolicyType, harness.getMergePolicy(), harness.getOperationTracker(), harness.getIOScheduler(),
-                harness.getIOOperationCallbackFactory(), harness.getPageWriteCallbackFactory(),
-                harness.getMetadataPageManagerFactory());
+        return LSMRTreeWithAntiMatterTuplesTestContext.create(harness.getNcConfig(), harness.getIOManager(),
+                harness.getVirtualBufferCaches(), harness.getFileReference(), harness.getDiskBufferCache(), fieldSerdes,
+                valueProviderFactories, numKeys, rtreePolicyType, harness.getMergePolicy(),
+                harness.getOperationTracker(), harness.getIOScheduler(), harness.getIOOperationCallbackFactory(),
+                harness.getPageWriteCallbackFactory(), harness.getMetadataPageManagerFactory());
     }
 
     @Override
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/org/apache/hyracks/storage/am/lsm/rtree/LSMRTreeWithAntiMatterTuplesLifecycleTest.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/org/apache/hyracks/storage/am/lsm/rtree/LSMRTreeWithAntiMatterTuplesLifecycleTest.java
index ccce785..5b08886 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/org/apache/hyracks/storage/am/lsm/rtree/LSMRTreeWithAntiMatterTuplesLifecycleTest.java
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/org/apache/hyracks/storage/am/lsm/rtree/LSMRTreeWithAntiMatterTuplesLifecycleTest.java
@@ -66,7 +66,7 @@
     @Override
     public void setup() throws Exception {
         harness.setUp();
-        testCtx = LSMRTreeWithAntiMatterTuplesTestContext.create(harness.getIOManager(),
+        testCtx = LSMRTreeWithAntiMatterTuplesTestContext.create(harness.getNcConfig(), harness.getIOManager(),
                 harness.getVirtualBufferCaches(), harness.getFileReference(), harness.getDiskBufferCache(), fieldSerdes,
                 valueProviderFactories, numKeys, RTreePolicyType.RTREE, harness.getMergePolicy(),
                 harness.getOperationTracker(), harness.getIOScheduler(), harness.getIOOperationCallbackFactory(),
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/org/apache/hyracks/storage/am/lsm/rtree/LSMRTreeWithAntiMatterTuplesMergeTest.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/org/apache/hyracks/storage/am/lsm/rtree/LSMRTreeWithAntiMatterTuplesMergeTest.java
index df52549..fb8f63e 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/org/apache/hyracks/storage/am/lsm/rtree/LSMRTreeWithAntiMatterTuplesMergeTest.java
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/org/apache/hyracks/storage/am/lsm/rtree/LSMRTreeWithAntiMatterTuplesMergeTest.java
@@ -55,11 +55,11 @@
     protected AbstractRTreeTestContext createTestContext(ISerializerDeserializer[] fieldSerdes,
             IPrimitiveValueProviderFactory[] valueProviderFactories, int numKeys, RTreePolicyType rtreePolicyType)
             throws Exception {
-        return LSMRTreeWithAntiMatterTuplesTestContext.create(harness.getIOManager(), harness.getVirtualBufferCaches(),
-                harness.getFileReference(), harness.getDiskBufferCache(), fieldSerdes, valueProviderFactories, numKeys,
-                rtreePolicyType, harness.getMergePolicy(), harness.getOperationTracker(), harness.getIOScheduler(),
-                harness.getIOOperationCallbackFactory(), harness.getPageWriteCallbackFactory(),
-                harness.getMetadataPageManagerFactory());
+        return LSMRTreeWithAntiMatterTuplesTestContext.create(harness.getNcConfig(), harness.getIOManager(),
+                harness.getVirtualBufferCaches(), harness.getFileReference(), harness.getDiskBufferCache(), fieldSerdes,
+                valueProviderFactories, numKeys, rtreePolicyType, harness.getMergePolicy(),
+                harness.getOperationTracker(), harness.getIOScheduler(), harness.getIOOperationCallbackFactory(),
+                harness.getPageWriteCallbackFactory(), harness.getMetadataPageManagerFactory());
     }
 
     @Override
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/org/apache/hyracks/storage/am/lsm/rtree/multithread/LSMRTreeMultiThreadTest.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/org/apache/hyracks/storage/am/lsm/rtree/multithread/LSMRTreeMultiThreadTest.java
index 980f21f..8d0ef95 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/org/apache/hyracks/storage/am/lsm/rtree/multithread/LSMRTreeMultiThreadTest.java
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/org/apache/hyracks/storage/am/lsm/rtree/multithread/LSMRTreeMultiThreadTest.java
@@ -60,11 +60,12 @@
     protected ITreeIndex createTreeIndex(ITypeTraits[] typeTraits, IBinaryComparatorFactory[] rtreeCmpFactories,
             IBinaryComparatorFactory[] btreeCmpFactories, IPrimitiveValueProviderFactory[] valueProviderFactories,
             RTreePolicyType rtreePolicyType, int[] btreeFields) throws HyracksDataException {
-        return LSMRTreeUtils.createLSMTree(harness.getIOManager(), harness.getVirtualBufferCaches(),
-                harness.getFileReference(), harness.getDiskBufferCache(), typeTraits, rtreeCmpFactories,
-                btreeCmpFactories, valueProviderFactories, rtreePolicyType, harness.getBoomFilterFalsePositiveRate(),
-                harness.getMergePolicy(), harness.getOperationTracker(), harness.getIOScheduler(),
-                harness.getIOOperationCallbackFactory(), harness.getPageWriteCallbackFactory(),
+        return LSMRTreeUtils.createLSMTree(harness.getNcConfig(), harness.getIOManager(),
+                harness.getVirtualBufferCaches(), harness.getFileReference(), harness.getDiskBufferCache(), typeTraits,
+                rtreeCmpFactories, btreeCmpFactories, valueProviderFactories, rtreePolicyType,
+                harness.getBoomFilterFalsePositiveRate(), harness.getMergePolicy(), harness.getOperationTracker(),
+                harness.getIOScheduler(), harness.getIOOperationCallbackFactory(),
+                harness.getPageWriteCallbackFactory(),
                 LSMRTreeUtils.proposeBestLinearizer(typeTraits, rtreeCmpFactories.length), null, btreeFields, null,
                 null, null, true, false, harness.getMetadataPageManagerFactory(), null, null);
     }
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/org/apache/hyracks/storage/am/lsm/rtree/multithread/LSMRTreeWithAntiMatterTuplesMultiThreadTest.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/org/apache/hyracks/storage/am/lsm/rtree/multithread/LSMRTreeWithAntiMatterTuplesMultiThreadTest.java
index f40add9..dd1857f 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/org/apache/hyracks/storage/am/lsm/rtree/multithread/LSMRTreeWithAntiMatterTuplesMultiThreadTest.java
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/org/apache/hyracks/storage/am/lsm/rtree/multithread/LSMRTreeWithAntiMatterTuplesMultiThreadTest.java
@@ -61,9 +61,9 @@
     protected ITreeIndex createTreeIndex(ITypeTraits[] typeTraits, IBinaryComparatorFactory[] rtreeCmpFactories,
             IBinaryComparatorFactory[] btreeCmpFactories, IPrimitiveValueProviderFactory[] valueProviderFactories,
             RTreePolicyType rtreePolicyType, int[] btreeFields) throws HyracksDataException {
-        return LSMRTreeUtils.createLSMTreeWithAntiMatterTuples(harness.getIOManager(), harness.getVirtualBufferCaches(),
-                harness.getFileReference(), harness.getDiskBufferCache(), typeTraits, rtreeCmpFactories,
-                btreeCmpFactories, valueProviderFactories, rtreePolicyType, harness.getMergePolicy(),
+        return LSMRTreeUtils.createLSMTreeWithAntiMatterTuples(harness.getNcConfig(), harness.getIOManager(),
+                harness.getVirtualBufferCaches(), harness.getFileReference(), harness.getDiskBufferCache(), typeTraits,
+                rtreeCmpFactories, btreeCmpFactories, valueProviderFactories, rtreePolicyType, harness.getMergePolicy(),
                 harness.getOperationTracker(), harness.getIOScheduler(), harness.getIOOperationCallbackFactory(),
                 harness.getPageWriteCallbackFactory(),
                 LSMRTreeUtils.proposeBestLinearizer(typeTraits, rtreeCmpFactories.length), null, null, null, null, true,
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 02742a8..3111745 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,6 +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.control.common.controllers.NCConfig;
 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;
@@ -74,24 +75,25 @@
         return lsmTree.getComparatorFactories();
     }
 
-    public static LSMRTreeTestContext create(IIOManager ioManager, List<IVirtualBufferCache> virtualBufferCaches,
-            FileReference file, IBufferCache diskBufferCache, ISerializerDeserializer[] fieldSerdes,
-            IPrimitiveValueProviderFactory[] valueProviderFactories, int numKeyFields, RTreePolicyType rtreePolicyType,
-            double bloomFilterFalsePositiveRate, ILSMMergePolicy mergePolicy, ILSMOperationTracker opTracker,
-            ILSMIOOperationScheduler ioScheduler, ILSMIOOperationCallbackFactory ioOpCallbackFactory,
-            ILSMPageWriteCallbackFactory pageWriteCallbackFactory,
+    public static LSMRTreeTestContext create(NCConfig storageConfig, IIOManager ioManager,
+            List<IVirtualBufferCache> virtualBufferCaches, FileReference file, IBufferCache diskBufferCache,
+            ISerializerDeserializer[] fieldSerdes, IPrimitiveValueProviderFactory[] valueProviderFactories,
+            int numKeyFields, RTreePolicyType rtreePolicyType, double bloomFilterFalsePositiveRate,
+            ILSMMergePolicy mergePolicy, ILSMOperationTracker opTracker, ILSMIOOperationScheduler ioScheduler,
+            ILSMIOOperationCallbackFactory ioOpCallbackFactory, ILSMPageWriteCallbackFactory pageWriteCallbackFactory,
             IMetadataPageManagerFactory metadataPageManagerFactory) throws Exception {
-        return create(ioManager, virtualBufferCaches, file, diskBufferCache, fieldSerdes, valueProviderFactories,
-                numKeyFields, rtreePolicyType, bloomFilterFalsePositiveRate, mergePolicy, opTracker, ioScheduler,
-                ioOpCallbackFactory, pageWriteCallbackFactory, metadataPageManagerFactory, false);
+        return create(storageConfig, ioManager, virtualBufferCaches, file, diskBufferCache, fieldSerdes,
+                valueProviderFactories, numKeyFields, rtreePolicyType, bloomFilterFalsePositiveRate, mergePolicy,
+                opTracker, ioScheduler, ioOpCallbackFactory, pageWriteCallbackFactory, metadataPageManagerFactory,
+                false);
     }
 
-    public static LSMRTreeTestContext create(IIOManager ioManager, List<IVirtualBufferCache> virtualBufferCaches,
-            FileReference file, IBufferCache diskBufferCache, ISerializerDeserializer[] fieldSerdes,
-            IPrimitiveValueProviderFactory[] valueProviderFactories, int numKeyFields, RTreePolicyType rtreePolicyType,
-            double bloomFilterFalsePositiveRate, ILSMMergePolicy mergePolicy, ILSMOperationTracker opTracker,
-            ILSMIOOperationScheduler ioScheduler, ILSMIOOperationCallbackFactory ioOpCallbackFactory,
-            ILSMPageWriteCallbackFactory pageWriteCallbackFactory,
+    public static LSMRTreeTestContext create(NCConfig storageConfig, IIOManager ioManager,
+            List<IVirtualBufferCache> virtualBufferCaches, FileReference file, IBufferCache diskBufferCache,
+            ISerializerDeserializer[] fieldSerdes, IPrimitiveValueProviderFactory[] valueProviderFactories,
+            int numKeyFields, RTreePolicyType rtreePolicyType, double bloomFilterFalsePositiveRate,
+            ILSMMergePolicy mergePolicy, ILSMOperationTracker opTracker, ILSMIOOperationScheduler ioScheduler,
+            ILSMIOOperationCallbackFactory ioOpCallbackFactory, ILSMPageWriteCallbackFactory pageWriteCallbackFactory,
             IMetadataPageManagerFactory metadataPageManagerFactory, boolean isPointMBR) throws Exception {
         ITypeTraits[] typeTraits = SerdeUtils.serdesToTypeTraits(fieldSerdes);
         IBinaryComparatorFactory[] rtreeCmpFactories =
@@ -105,9 +107,9 @@
         }
         IBinaryComparatorFactory[] btreeCmpFactories =
                 SerdeUtils.serdesToComparatorFactories(btreeFieldSerdes, numBtreeFields);
-        LSMRTree lsmTree = LSMRTreeUtils.createLSMTree(ioManager, virtualBufferCaches, file, diskBufferCache,
-                typeTraits, rtreeCmpFactories, btreeCmpFactories, valueProviderFactories, rtreePolicyType,
-                bloomFilterFalsePositiveRate, mergePolicy, opTracker, ioScheduler, ioOpCallbackFactory,
+        LSMRTree lsmTree = LSMRTreeUtils.createLSMTree(storageConfig, ioManager, virtualBufferCaches, file,
+                diskBufferCache, typeTraits, rtreeCmpFactories, btreeCmpFactories, valueProviderFactories,
+                rtreePolicyType, bloomFilterFalsePositiveRate, mergePolicy, opTracker, ioScheduler, ioOpCallbackFactory,
                 pageWriteCallbackFactory, LSMRTreeUtils.proposeBestLinearizer(typeTraits, rtreeCmpFactories.length),
                 null, btreeFields, null, null, null, true, isPointMBR, metadataPageManagerFactory, null, null);
         LSMRTreeTestContext testCtx = new LSMRTreeTestContext(fieldSerdes, lsmTree);
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 e189898..58b9a34 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
@@ -31,6 +31,7 @@
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.api.io.FileReference;
 import org.apache.hyracks.api.io.IODeviceHandle;
+import org.apache.hyracks.control.common.controllers.NCConfig;
 import org.apache.hyracks.control.nc.io.IOManager;
 import org.apache.hyracks.storage.am.common.api.IMetadataPageManagerFactory;
 import org.apache.hyracks.storage.am.common.freepage.AppendOnlyLinkedMetadataPageManagerFactory;
@@ -66,6 +67,7 @@
     protected final int numMutableComponents;
 
     protected IOManager ioManager;
+    protected NCConfig ncConfig;
     protected int ioDeviceId;
     protected IBufferCache diskBufferCache;
     protected List<IVirtualBufferCache> virtualBufferCaches;
@@ -98,6 +100,7 @@
         this.ioOpCallbackFactory = NoOpIOOperationCallbackFactory.INSTANCE;
         this.pageWriteCallbackFactory = NoOpPageWriteCallbackFactory.INSTANCE;
         this.numMutableComponents = AccessMethodTestsConfig.LSM_RTREE_NUM_MUTABLE_COMPONENTS;
+        this.ncConfig = new NCConfig(null);
     }
 
     public void setUp() throws HyracksDataException {
@@ -209,6 +212,10 @@
         return mergePolicy;
     }
 
+    public NCConfig getNcConfig() {
+        return ncConfig;
+    }
+
     public ILSMIOOperationCallbackFactory getIOOperationCallbackFactory() {
         return ioOpCallbackFactory;
     }
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 ff953f1..45b5911 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,6 +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.control.common.controllers.NCConfig;
 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;
@@ -74,7 +75,7 @@
         return lsmTree.getComparatorFactories();
     }
 
-    public static LSMRTreeWithAntiMatterTuplesTestContext create(IIOManager ioManager,
+    public static LSMRTreeWithAntiMatterTuplesTestContext create(NCConfig storageConfig, IIOManager ioManager,
             List<IVirtualBufferCache> virtualBufferCaches, FileReference file, IBufferCache diskBufferCache,
             ISerializerDeserializer[] fieldSerdes, IPrimitiveValueProviderFactory[] valueProviderFactories,
             int numKeyFields, RTreePolicyType rtreePolicyType, ILSMMergePolicy mergePolicy,
@@ -86,7 +87,7 @@
                 SerdeUtils.serdesToComparatorFactories(fieldSerdes, numKeyFields);
         IBinaryComparatorFactory[] btreeCmpFactories =
                 SerdeUtils.serdesToComparatorFactories(fieldSerdes, fieldSerdes.length);
-        LSMRTreeWithAntiMatterTuples lsmTree = LSMRTreeUtils.createLSMTreeWithAntiMatterTuples(ioManager,
+        LSMRTreeWithAntiMatterTuples lsmTree = LSMRTreeUtils.createLSMTreeWithAntiMatterTuples(storageConfig, ioManager,
                 virtualBufferCaches, file, diskBufferCache, typeTraits, rtreeCmpFactories, btreeCmpFactories,
                 valueProviderFactories, rtreePolicyType, mergePolicy, opTracker, ioScheduler, ioOpCallbackFactory,
                 pageWriteCallbackFactory, LSMRTreeUtils.proposeBestLinearizer(typeTraits, rtreeCmpFactories.length),