[NO ISSUE][STO] Add consistency to flush lifecycle

- user model changes: no
- storage format changes: yes
  - renamed AbstractLSMIOOperationCallbackFactory
    to LSMIOOperationCallbackFactory
  - useless classes have been removed.
    - LSMBTreeIOOperationCallbackFactory
    - LSMBTreeWithBuddyIOOperationCallbackFactory
    - LSMInvertedIndexIOOperationCallbackFactory
    - LSMRTreeIOOperationCallbackFactory
- interface changes: yes

Details:
- Previously, flushes have different lifecycle depending
  on the memory component state
  - not allocated
  - allocated
  - modified
- In certain cases, flush operations are skipped alltogether
- IO Operation callbacks became complicated and difficult
  to maintain since calls are done differently in different
  cases.
- In certain cases, afterFinalize is called on the IO
  Operation callbacks even if beforeOperation was never
  called.
- In this change, flushes go through the same lifecycle
  events regardless of the state of the memory component.
- In addition, primary and secondary memory components
  would reside in different virtual buffer caches due
  to skipped flushes, or due to having the secondary
  index created when the primary index's memory component
  is residing on the virtual buffer cache with index !=0.
- Moreover, when flushes are lagging and all memory
  components are being flushed, search operations assumes
  the oldest of the memory component is the newest and
  produces incorrect results.
- In addition, in case of a failed flush of a component,
  the IO scheduler would skip it and flush the next
  component. This would produce a bad state on disk.
- In this change, a failed flush can be retried. otherwise,
  all future flushes of the component fail due to the failure
  of the previously failed flush.
- Previously, when a component fails to modify an index due
  to flush failures, it assumes disk is full.
- With this change, the modification failure reports the
  original cause of the failed flush.

Change-Id: I29f7992ec6c0f71c5b63d45800b2fb590d651e4b
Reviewed-on: https://asterix-gerrit.ics.uci.edu/2584
Reviewed-by: Murtadha Hubail <mhubail@apache.org>
Tested-by: Murtadha Hubail <mhubail@apache.org>
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/nc/HaltCallback.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/nc/HaltCallback.java
new file mode 100644
index 0000000..db34131
--- /dev/null
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/nc/HaltCallback.java
@@ -0,0 +1,49 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.app.nc;
+
+import org.apache.hyracks.storage.am.lsm.common.api.IIoOperationFailedCallback;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperation;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperation.LSMIOOperationType;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationScheduler;
+import org.apache.hyracks.util.ExitUtil;
+import org.apache.logging.log4j.LogManager;
+import org.apache.logging.log4j.Logger;
+
+public class HaltCallback implements IIoOperationFailedCallback {
+    private static final Logger LOGGER = LogManager.getLogger();
+    public static final HaltCallback INSTANCE = new HaltCallback();
+
+    private HaltCallback() {
+    }
+
+    @Override
+    public void schedulerFailed(ILSMIOOperationScheduler scheduler, Throwable failure) {
+        LOGGER.error("IO Scheduler has failed", failure);
+        ExitUtil.halt(ExitUtil.EC_IO_SCHEDULER_FAILED);
+    }
+
+    @Override
+    public void operationFailed(ILSMIOOperation operation, Throwable t) {
+        LOGGER.error("Operation {} has failed", t);
+        if (operation.getIOOpertionType() == LSMIOOperationType.FLUSH) {
+            ExitUtil.halt(ExitUtil.EC_FLUSH_FAILED);
+        }
+    }
+}
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/nc/NCAppRuntimeContext.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/nc/NCAppRuntimeContext.java
index 288e5f2..c3b1bae 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/nc/NCAppRuntimeContext.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/nc/NCAppRuntimeContext.java
@@ -52,7 +52,6 @@
 import org.apache.asterix.common.context.DatasetLifecycleManager;
 import org.apache.asterix.common.context.DatasetMemoryManager;
 import org.apache.asterix.common.context.IStorageComponentProvider;
-import org.apache.asterix.common.exceptions.ACIDException;
 import org.apache.asterix.common.exceptions.AsterixException;
 import org.apache.asterix.common.library.ILibraryManager;
 import org.apache.asterix.common.replication.IReplicationChannel;
@@ -169,7 +168,7 @@
     }
 
     @Override
-    public void initialize(boolean initialRun) throws IOException, ACIDException {
+    public void initialize(boolean initialRun) throws IOException {
         ioManager = getServiceContext().getIoManager();
         threadExecutor =
                 MaintainedThreadNameExecutorService.newCachedThreadPool(getServiceContext().getThreadFactory());
@@ -177,19 +176,13 @@
         IPageCleanerPolicy pcp = new DelayPageCleanerPolicy(600000);
         IPageReplacementStrategy prs = new ClockPageReplacementStrategy(allocator,
                 storageProperties.getBufferCachePageSize(), storageProperties.getBufferCacheNumPages());
-
-        AsynchronousScheduler.INSTANCE.init(getServiceContext().getThreadFactory());
-        lsmIOScheduler = AsynchronousScheduler.INSTANCE;
-
+        lsmIOScheduler = new AsynchronousScheduler(getServiceContext().getThreadFactory(), HaltCallback.INSTANCE);
         metadataMergePolicyFactory = new PrefixMergePolicyFactory();
         indexCheckpointManagerProvider = new IndexCheckpointManagerProvider(ioManager);
-
         ILocalResourceRepositoryFactory persistentLocalResourceRepositoryFactory =
                 new PersistentLocalResourceRepositoryFactory(ioManager, indexCheckpointManagerProvider);
-
         localResourceRepository =
                 (PersistentLocalResourceRepository) persistentLocalResourceRepositoryFactory.createRepository();
-
         txnSubsystem = new TransactionSubsystem(this);
         IRecoveryManager recoveryMgr = txnSubsystem.getRecoveryManager();
         SystemState systemState = recoveryMgr.getSystemState();
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/nc/RecoveryManager.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/nc/RecoveryManager.java
index d4e652d..efd173f 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/nc/RecoveryManager.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/nc/RecoveryManager.java
@@ -48,7 +48,7 @@
 import org.apache.asterix.common.context.IndexInfo;
 import org.apache.asterix.common.dataflow.DatasetLocalResource;
 import org.apache.asterix.common.exceptions.ACIDException;
-import org.apache.asterix.common.ioopcallbacks.AbstractLSMIOOperationCallback;
+import org.apache.asterix.common.ioopcallbacks.LSMIOOperationCallback;
 import org.apache.asterix.common.storage.DatasetResourceReference;
 import org.apache.asterix.common.storage.IIndexCheckpointManager;
 import org.apache.asterix.common.storage.IIndexCheckpointManagerProvider;
@@ -401,13 +401,12 @@
                                 if (iInfo.isOpen() && iInfo.getPartition() == partition) {
                                     maxDiskLastLsn = resourceId2MaxLSNMap.get(iInfo.getResourceId());
                                     index = iInfo.getIndex();
-                                    AbstractLSMIOOperationCallback ioCallback =
-                                            (AbstractLSMIOOperationCallback) index.getIOOperationCallback();
+                                    LSMIOOperationCallback ioCallback =
+                                            (LSMIOOperationCallback) index.getIOOperationCallback();
                                     if (logRecord.getLSN() > maxDiskLastLsn
                                             && !index.isCurrentMutableComponentEmpty()) {
                                         // schedule flush
-                                        ioCallback.updateLastLSN(logRecord.getLSN());
-                                        redoFlush(index, logRecord);
+                                        redoFlush(index, logRecord, idGenerator.getId());
                                         redoCount++;
                                     } else {
                                         if (index.isMemoryComponentsAllocated()) {
@@ -417,7 +416,7 @@
                                         } else {
                                             // otherwise, we refresh the id stored in ioCallback
                                             // to ensure the memory component receives correct Id upon activation
-                                            ioCallback.forceRefreshNextId();
+                                            ioCallback.forceRefreshNextId(idGenerator.getId());
                                         }
                                     }
                                 }
@@ -473,10 +472,10 @@
         long minFirstLSN = logMgr.getAppendLSN();
         if (!openIndexList.isEmpty()) {
             for (IIndex index : openIndexList) {
-                AbstractLSMIOOperationCallback ioCallback =
-                        (AbstractLSMIOOperationCallback) ((ILSMIndex) index).getIOOperationCallback();
+                LSMIOOperationCallback ioCallback =
+                        (LSMIOOperationCallback) ((ILSMIndex) index).getIOOperationCallback();
                 if (!((AbstractLSMIndex) index).isCurrentMutableComponentEmpty() || ioCallback.hasPendingFlush()) {
-                    firstLSN = ioCallback.getFirstLSN();
+                    firstLSN = ioCallback.getPersistenceLsn();
                     minFirstLSN = Math.min(minFirstLSN, firstLSN);
                 }
             }
@@ -823,11 +822,17 @@
         }
     }
 
-    private static void redoFlush(ILSMIndex index, ILogRecord logRecord) throws HyracksDataException {
+    private static void redoFlush(ILSMIndex index, ILogRecord logRecord, ILSMComponentId nextId)
+            throws HyracksDataException {
+        long flushLsn = logRecord.getLSN();
+        Map<String, Object> flushMap = new HashMap<>();
+        flushMap.put(LSMIOOperationCallback.KEY_FLUSH_LOG_LSN, flushLsn);
         ILSMIndexAccessor accessor = index.createAccessor(NoOpIndexAccessParameters.INSTANCE);
+        accessor.getOpContext().setParameters(flushMap);
         long minId = logRecord.getFlushingComponentMinId();
         long maxId = logRecord.getFlushingComponentMaxId();
         ILSMComponentId id = new LSMComponentId(minId, maxId);
+        flushMap.put(LSMIOOperationCallback.KEY_NEXT_COMPONENT_ID, nextId);
         if (!index.getDiskComponents().isEmpty()) {
             ILSMDiskComponent diskComponent = index.getDiskComponents().get(0);
             ILSMComponentId maxDiskComponentId = diskComponent.getId();
@@ -837,7 +842,7 @@
             }
         }
         index.getCurrentMemoryComponent().resetId(id, true);
-        accessor.scheduleFlush(index.getIOOperationCallback());
+        accessor.scheduleFlush();
     }
 
     private class JobEntityCommits {
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/ComponentRollbackTest.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/ComponentRollbackTest.java
index 9ef531e..a33bda1 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/ComponentRollbackTest.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/ComponentRollbackTest.java
@@ -20,7 +20,9 @@
 
 import java.io.File;
 import java.util.ArrayList;
+import java.util.HashMap;
 import java.util.List;
+import java.util.Map;
 import java.util.function.Predicate;
 
 import org.apache.asterix.app.bootstrap.TestNodeController;
@@ -29,7 +31,7 @@
 import org.apache.asterix.app.nc.NCAppRuntimeContext;
 import org.apache.asterix.common.api.IDatasetLifecycleManager;
 import org.apache.asterix.common.dataflow.LSMInsertDeleteOperatorNodePushable;
-import org.apache.asterix.common.ioopcallbacks.AbstractLSMIOOperationCallback;
+import org.apache.asterix.common.ioopcallbacks.LSMIOOperationCallback;
 import org.apache.asterix.common.transactions.ITransactionContext;
 import org.apache.asterix.common.transactions.ITransactionManager;
 import org.apache.asterix.common.transactions.TransactionOptions;
@@ -50,6 +52,7 @@
 import org.apache.hyracks.storage.am.lsm.btree.impl.AllowTestOpCallback;
 import org.apache.hyracks.storage.am.lsm.btree.impl.TestLsmBtree;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponent;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponentId;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMDiskComponent;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndexAccessor;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMMemoryComponent;
@@ -146,16 +149,27 @@
             StorageTestUtils.searchAndAssertCount(nc, PARTITION, StorageTestUtils.TOTAL_NUM_OF_RECORDS);
             ILSMIndexAccessor lsmAccessor = lsmBtree.createAccessor(NoOpIndexAccessParameters.INSTANCE);
             dsLifecycleMgr.getComponentIdGenerator(StorageTestUtils.DATASET_ID, PARTITION).refresh();
-            ((AbstractLSMIOOperationCallback) lsmBtree.getIOOperationCallback()).updateLastLSN(0);
+            ILSMComponentId next =
+                    dsLifecycleMgr.getComponentIdGenerator(StorageTestUtils.DATASET_ID, PARTITION).getId();
+            long flushLsn = nc.getTransactionSubsystem().getLogManager().getAppendLSN();
+            Map<String, Object> flushMap = new HashMap<>();
+            flushMap.put(LSMIOOperationCallback.KEY_FLUSH_LOG_LSN, flushLsn);
+            flushMap.put(LSMIOOperationCallback.KEY_NEXT_COMPONENT_ID, next);
+            lsmAccessor.getOpContext().setParameters(flushMap);
             // rollback a memory component
             lsmAccessor.deleteComponents(memoryComponentsPredicate);
             StorageTestUtils.searchAndAssertCount(nc, PARTITION,
                     StorageTestUtils.TOTAL_NUM_OF_RECORDS - StorageTestUtils.RECORDS_PER_COMPONENT);
             // rollback the last disk component
             lsmAccessor = lsmBtree.createAccessor(NoOpIndexAccessParameters.INSTANCE);
-            long lsn = AbstractLSMIOOperationCallback.getTreeIndexLSN(diskComponents.get(0).getMetadata());
+            long lsn = LSMIOOperationCallback.getTreeIndexLSN(diskComponents.get(0).getMetadata());
             dsLifecycleMgr.getComponentIdGenerator(StorageTestUtils.DATASET_ID, PARTITION).refresh();
-            ((AbstractLSMIOOperationCallback) lsmBtree.getIOOperationCallback()).updateLastLSN(0);
+            next = dsLifecycleMgr.getComponentIdGenerator(StorageTestUtils.DATASET_ID, PARTITION).getId();
+            flushLsn = nc.getTransactionSubsystem().getLogManager().getAppendLSN();
+            flushMap = new HashMap<>();
+            flushMap.put(LSMIOOperationCallback.KEY_FLUSH_LOG_LSN, flushLsn);
+            flushMap.put(LSMIOOperationCallback.KEY_NEXT_COMPONENT_ID, next);
+            lsmAccessor.getOpContext().setParameters(flushMap);
             DiskComponentLsnPredicate pred = new DiskComponentLsnPredicate(lsn);
             lsmAccessor.deleteComponents(pred);
             StorageTestUtils.searchAndAssertCount(nc, PARTITION,
@@ -203,7 +217,13 @@
             StorageTestUtils.searchAndAssertCount(nc, PARTITION, StorageTestUtils.TOTAL_NUM_OF_RECORDS);
             ILSMIndexAccessor lsmAccessor = lsmBtree.createAccessor(NoOpIndexAccessParameters.INSTANCE);
             dsLifecycleMgr.getComponentIdGenerator(StorageTestUtils.DATASET_ID, PARTITION).refresh();
-            ((AbstractLSMIOOperationCallback) lsmBtree.getIOOperationCallback()).updateLastLSN(0);
+            ILSMComponentId next =
+                    dsLifecycleMgr.getComponentIdGenerator(StorageTestUtils.DATASET_ID, PARTITION).getId();
+            long flushLsn = nc.getTransactionSubsystem().getLogManager().getAppendLSN();
+            Map<String, Object> flushMap = new HashMap<>();
+            flushMap.put(LSMIOOperationCallback.KEY_FLUSH_LOG_LSN, flushLsn);
+            flushMap.put(LSMIOOperationCallback.KEY_NEXT_COMPONENT_ID, next);
+            lsmAccessor.getOpContext().setParameters(flushMap);
             // rollback a memory component
             lsmAccessor.deleteComponents(memoryComponentsPredicate);
             StorageTestUtils.searchAndAssertCount(nc, PARTITION,
@@ -227,9 +247,14 @@
             StorageTestUtils.searchAndAssertCount(nc, PARTITION, StorageTestUtils.TOTAL_NUM_OF_RECORDS);
             // rollback the last disk component
             lsmAccessor = lsmBtree.createAccessor(NoOpIndexAccessParameters.INSTANCE);
-            long lsn = AbstractLSMIOOperationCallback.getTreeIndexLSN(diskComponents.get(0).getMetadata());
+            long lsn = LSMIOOperationCallback.getTreeIndexLSN(diskComponents.get(0).getMetadata());
             dsLifecycleMgr.getComponentIdGenerator(StorageTestUtils.DATASET_ID, PARTITION).refresh();
-            ((AbstractLSMIOOperationCallback) lsmBtree.getIOOperationCallback()).updateLastLSN(0);
+            next = dsLifecycleMgr.getComponentIdGenerator(StorageTestUtils.DATASET_ID, PARTITION).getId();
+            flushLsn = nc.getTransactionSubsystem().getLogManager().getAppendLSN();
+            flushMap = new HashMap<>();
+            flushMap.put(LSMIOOperationCallback.KEY_FLUSH_LOG_LSN, flushLsn);
+            flushMap.put(LSMIOOperationCallback.KEY_NEXT_COMPONENT_ID, next);
+            lsmAccessor.getOpContext().setParameters(flushMap);
             DiskComponentLsnPredicate pred = new DiskComponentLsnPredicate(lsn);
             lsmAccessor.deleteComponents(pred);
             StorageTestUtils.searchAndAssertCount(nc, PARTITION,
@@ -278,7 +303,13 @@
             // now that we enetered, we will rollback
             ILSMIndexAccessor lsmAccessor = lsmBtree.createAccessor(NoOpIndexAccessParameters.INSTANCE);
             dsLifecycleMgr.getComponentIdGenerator(StorageTestUtils.DATASET_ID, PARTITION).refresh();
-            ((AbstractLSMIOOperationCallback) lsmBtree.getIOOperationCallback()).updateLastLSN(0);
+            ILSMComponentId next =
+                    dsLifecycleMgr.getComponentIdGenerator(StorageTestUtils.DATASET_ID, PARTITION).getId();
+            long flushLsn = nc.getTransactionSubsystem().getLogManager().getAppendLSN();
+            Map<String, Object> flushMap = new HashMap<>();
+            flushMap.put(LSMIOOperationCallback.KEY_FLUSH_LOG_LSN, flushLsn);
+            flushMap.put(LSMIOOperationCallback.KEY_NEXT_COMPONENT_ID, next);
+            lsmAccessor.getOpContext().setParameters(flushMap);
             // rollback a memory component
             lsmAccessor.deleteComponents(
                     c -> (c instanceof ILSMMemoryComponent && ((ILSMMemoryComponent) c).isModified()));
@@ -297,10 +328,14 @@
             // wait till firstSearcher enter the components
             secondSearcher.waitUntilEntered();
             lsmAccessor = lsmBtree.createAccessor(NoOpIndexAccessParameters.INSTANCE);
-            long lsn = AbstractLSMIOOperationCallback.getTreeIndexLSN(diskComponents.get(0).getMetadata());
-
+            long lsn = LSMIOOperationCallback.getTreeIndexLSN(diskComponents.get(0).getMetadata());
             dsLifecycleMgr.getComponentIdGenerator(StorageTestUtils.DATASET_ID, PARTITION).refresh();
-            ((AbstractLSMIOOperationCallback) lsmBtree.getIOOperationCallback()).updateLastLSN(0);
+            next = dsLifecycleMgr.getComponentIdGenerator(StorageTestUtils.DATASET_ID, PARTITION).getId();
+            flushLsn = nc.getTransactionSubsystem().getLogManager().getAppendLSN();
+            flushMap = new HashMap<>();
+            flushMap.put(LSMIOOperationCallback.KEY_FLUSH_LOG_LSN, flushLsn);
+            flushMap.put(LSMIOOperationCallback.KEY_NEXT_COMPONENT_ID, next);
+            lsmAccessor.getOpContext().setParameters(flushMap);
             DiskComponentLsnPredicate pred = new DiskComponentLsnPredicate(lsn);
             lsmAccessor.deleteComponents(pred);
             // now that the rollback has completed, we will unblock the search
@@ -407,11 +442,11 @@
             // select the components to merge... the last three
             int numMergedComponents = 3;
             List<ILSMDiskComponent> mergedComponents = new ArrayList<>();
-            long lsn = AbstractLSMIOOperationCallback.getTreeIndexLSN(diskComponents.get(0).getMetadata());
+            long lsn = LSMIOOperationCallback.getTreeIndexLSN(diskComponents.get(0).getMetadata());
             for (int i = 0; i < numMergedComponents; i++) {
                 mergedComponents.add(diskComponents.get(i));
             }
-            mergeAccessor.scheduleMerge(lsmBtree.getIOOperationCallback(), mergedComponents);
+            mergeAccessor.scheduleMerge(mergedComponents);
             merger.waitUntilCount(1);
             // now that we enetered, we will rollback
             Rollerback rollerback = new Rollerback(lsmBtree, new DiskComponentLsnPredicate(lsn));
@@ -590,11 +625,11 @@
             // select the components to merge... the last three
             int numMergedComponents = 3;
             List<ILSMDiskComponent> mergedComponents = new ArrayList<>();
-            long lsn = AbstractLSMIOOperationCallback.getTreeIndexLSN(diskComponents.get(0).getMetadata());
+            long lsn = LSMIOOperationCallback.getTreeIndexLSN(diskComponents.get(0).getMetadata());
             for (int i = 0; i < numMergedComponents; i++) {
                 mergedComponents.add(diskComponents.get(i));
             }
-            mergeAccessor.scheduleMerge(lsmBtree.getIOOperationCallback(), mergedComponents);
+            mergeAccessor.scheduleMerge(mergedComponents);
             merger.waitUntilCount(1);
             // we will block search
             lsmBtree.clearSearchCallbacks();
@@ -659,12 +694,12 @@
             ILSMIndexAccessor mergeAccessor = lsmBtree.createAccessor(NoOpIndexAccessParameters.INSTANCE);
             // select the components to merge... the last three
             List<ILSMDiskComponent> mergedComponents = new ArrayList<>();
-            long lsn = AbstractLSMIOOperationCallback.getTreeIndexLSN(diskComponents.get(0).getMetadata());
+            long lsn = LSMIOOperationCallback.getTreeIndexLSN(diskComponents.get(0).getMetadata());
             int numMergedComponents = 3;
             for (int i = 0; i < numMergedComponents; i++) {
                 mergedComponents.add(diskComponents.get(i));
             }
-            mergeAccessor.scheduleMerge(lsmBtree.getIOOperationCallback(), mergedComponents);
+            mergeAccessor.scheduleMerge(mergedComponents);
             merger.waitUntilCount(1);
             // we will block search
             lsmBtree.clearSearchCallbacks();
@@ -704,7 +739,13 @@
                     ILSMIndexAccessor lsmAccessor = lsmBtree.createAccessor(NoOpIndexAccessParameters.INSTANCE);
                     try {
                         dsLifecycleMgr.getComponentIdGenerator(StorageTestUtils.DATASET_ID, PARTITION).refresh();
-                        ((AbstractLSMIOOperationCallback) lsmBtree.getIOOperationCallback()).updateLastLSN(0);
+                        ILSMComponentId next =
+                                dsLifecycleMgr.getComponentIdGenerator(StorageTestUtils.DATASET_ID, PARTITION).getId();
+                        long flushLsn = nc.getTransactionSubsystem().getLogManager().getAppendLSN();
+                        Map<String, Object> flushMap = new HashMap<>();
+                        flushMap.put(LSMIOOperationCallback.KEY_FLUSH_LOG_LSN, flushLsn);
+                        flushMap.put(LSMIOOperationCallback.KEY_NEXT_COMPONENT_ID, next);
+                        lsmAccessor.getOpContext().setParameters(flushMap);
                         lsmAccessor.deleteComponents(predicate);
                     } catch (HyracksDataException e) {
                         failure = e;
@@ -733,9 +774,8 @@
         @Override
         public boolean test(ILSMComponent c) {
             try {
-                return c instanceof ILSMMemoryComponent
-                        || (c instanceof ILSMDiskComponent && AbstractLSMIOOperationCallback
-                                .getTreeIndexLSN(((ILSMDiskComponent) c).getMetadata()) >= lsn);
+                return c instanceof ILSMMemoryComponent || (c instanceof ILSMDiskComponent
+                        && LSMIOOperationCallback.getTreeIndexLSN(((ILSMDiskComponent) c).getMetadata()) >= lsn);
             } catch (HyracksDataException e) {
                 e.printStackTrace();
                 return false;
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/IoCallbackFailureTest.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/IoCallbackFailureTest.java
index 8bafd32..eb16cf4 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/IoCallbackFailureTest.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/IoCallbackFailureTest.java
@@ -91,7 +91,7 @@
             }
 
             @Override
-            public void after() throws HyracksDataException {
+            public void after(ILSMMemoryComponent c) throws HyracksDataException {
                 // No Op
             }
         };
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/LSMFlushRecoveryTest.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/LSMFlushRecoveryTest.java
index dabb32c..e4623fd 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/LSMFlushRecoveryTest.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/LSMFlushRecoveryTest.java
@@ -19,6 +19,7 @@
 package org.apache.asterix.test.dataflow;
 
 import java.io.File;
+import java.lang.reflect.Field;
 import java.rmi.RemoteException;
 import java.util.ArrayList;
 import java.util.Arrays;
@@ -60,8 +61,14 @@
 import org.apache.hyracks.storage.am.lsm.btree.impl.AllowTestOpCallback;
 import org.apache.hyracks.storage.am.lsm.btree.impl.ITestOpCallback;
 import org.apache.hyracks.storage.am.lsm.btree.impl.TestLsmBtree;
+import org.apache.hyracks.storage.am.lsm.common.api.IIoOperationFailedCallback;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMDiskComponent;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperation;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationScheduler;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMMemoryComponent;
+import org.apache.hyracks.storage.am.lsm.common.impls.AsynchronousScheduler;
+import org.apache.logging.log4j.LogManager;
+import org.apache.logging.log4j.Logger;
 import org.junit.After;
 import org.junit.Assert;
 import org.junit.Before;
@@ -69,6 +76,7 @@
 import org.junit.Test;
 
 public class LSMFlushRecoveryTest {
+    public static final Logger LOGGER = LogManager.getLogger();
     private static TestNodeController nc;
     private static Dataset dataset;
     private static PrimaryIndexInfo[] primaryIndexInfos;
@@ -153,6 +161,22 @@
     private void initializeNc(boolean cleanUpOnStart) throws Exception {
         nc.init(cleanUpOnStart);
         ncAppCtx = nc.getAppRuntimeContext();
+        // Override the LSMIOScheduler to avoid halting on failure and enable
+        // testing failure scenario in a unit test setting
+        Field ioScheduler = ncAppCtx.getClass().getDeclaredField("lsmIOScheduler");
+        ioScheduler.setAccessible(true);
+        ioScheduler.set(ncAppCtx, new AsynchronousScheduler(ncAppCtx.getServiceContext().getThreadFactory(),
+                new IIoOperationFailedCallback() {
+                    @Override
+                    public void schedulerFailed(ILSMIOOperationScheduler scheduler, Throwable failure) {
+                        LOGGER.error("Scheduler Failed", failure);
+                    }
+
+                    @Override
+                    public void operationFailed(ILSMIOOperation operation, Throwable t) {
+                        LOGGER.warn("IO Operation failed", t);
+                    }
+                }));
         dsLifecycleMgr = ncAppCtx.getDatasetLifecycleManager();
     }
 
@@ -241,21 +265,21 @@
             }
 
             @Override
-            public void after() throws HyracksDataException {
+            public void after(Semaphore t) throws HyracksDataException {
 
             }
         });
 
         Semaphore primaryFlushSemaphore = new Semaphore(0);
         primaryIndexes[PARTITION_0].addFlushCallback(AllowTestOpCallback.INSTANCE);
-        primaryIndexes[PARTITION_0].addIoAfterFinalizeCallback(new ITestOpCallback<Void>() {
+        primaryIndexes[PARTITION_0].addIoCompletedCallback(new ITestOpCallback<Void>() {
             @Override
             public void before(Void t) throws HyracksDataException {
 
             }
 
             @Override
-            public void after() throws HyracksDataException {
+            public void after(Void t) throws HyracksDataException {
                 primaryFlushSemaphore.release();
             }
         });
@@ -283,21 +307,21 @@
             }
 
             @Override
-            public void after() throws HyracksDataException {
+            public void after(Semaphore t) throws HyracksDataException {
 
             }
         });
 
         Semaphore secondaryFlushSemaphore = new Semaphore(0);
         secondaryIndexes[PARTITION_0].addFlushCallback(AllowTestOpCallback.INSTANCE);
-        secondaryIndexes[PARTITION_0].addIoAfterFinalizeCallback(new ITestOpCallback<Void>() {
+        secondaryIndexes[PARTITION_0].addIoCompletedCallback(new ITestOpCallback<Void>() {
             @Override
             public void before(Void t) throws HyracksDataException {
 
             }
 
             @Override
-            public void after() throws HyracksDataException {
+            public void after(Void t) throws HyracksDataException {
                 secondaryFlushSemaphore.release();
             }
         });
@@ -335,21 +359,21 @@
             }
 
             @Override
-            public void after() throws HyracksDataException {
+            public void after(Semaphore t) throws HyracksDataException {
 
             }
         });
 
         Semaphore flushSemaphore = new Semaphore(0);
         secondaryIndexes[PARTITION_0].addFlushCallback(AllowTestOpCallback.INSTANCE);
-        secondaryIndexes[PARTITION_0].addIoAfterFinalizeCallback(new ITestOpCallback<Void>() {
+        secondaryIndexes[PARTITION_0].addIoCompletedCallback(new ITestOpCallback<Void>() {
             @Override
             public void before(Void t) throws HyracksDataException {
 
             }
 
             @Override
-            public void after() throws HyracksDataException {
+            public void after(Void t) throws HyracksDataException {
                 flushSemaphore.release();
             }
         });
@@ -387,7 +411,7 @@
             }
 
             @Override
-            public void after() throws HyracksDataException {
+            public void after(Semaphore t) throws HyracksDataException {
 
             }
         });
@@ -400,7 +424,7 @@
             }
 
             @Override
-            public void after() throws HyracksDataException {
+            public void after(Semaphore t) throws HyracksDataException {
 
             }
         });
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/MultiPartitionLSMIndexTest.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/MultiPartitionLSMIndexTest.java
index 62705cc..1795c93 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/MultiPartitionLSMIndexTest.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/MultiPartitionLSMIndexTest.java
@@ -302,7 +302,7 @@
                 }
 
                 @Override
-                public void after() {
+                public void after(Void t) {
                     synchronized (allocated) {
                         allocated.setValue(true);
                         allocated.notifyAll();
@@ -339,7 +339,7 @@
                 }
 
                 @Override
-                public void after() {
+                public void after(Semaphore t) {
                 }
             });
             synchronized (proceedToScheduleFlush) {
@@ -421,7 +421,7 @@
                 }
 
                 @Override
-                public void after() {
+                public void after(Void t) {
                     synchronized (finishedSchduleFlush) {
                         finishedSchduleFlush.set(true);
                         finishedSchduleFlush.notifyAll();
@@ -478,7 +478,7 @@
                 }
 
                 @Override
-                public void after() {
+                public void after(ILSMMemoryComponent t) {
                     synchronized (recycledPrimary) {
                         recycledPrimary.setValue(true);
                         recycledPrimary.notifyAll();
@@ -519,7 +519,7 @@
                 }
 
                 @Override
-                public void after() {
+                public void after(ILSMMemoryComponent t) {
                 }
             };
             secondaryLsmBtrees[0].addIoRecycleCallback(secondaryRecycleCallback);
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/SearchCursorComponentSwitchTest.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/SearchCursorComponentSwitchTest.java
index e4373f6..72026a2 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/SearchCursorComponentSwitchTest.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/SearchCursorComponentSwitchTest.java
@@ -60,8 +60,9 @@
 import org.apache.hyracks.storage.am.lsm.btree.impl.AllowTestOpCallback;
 import org.apache.hyracks.storage.am.lsm.btree.impl.TestLsmBtree;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMDiskComponent;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperation;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperation.LSMIOOperationStatus;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndexAccessor;
-import org.apache.hyracks.storage.am.lsm.common.impls.BlockingIOOperationCallbackWrapper;
 import org.apache.hyracks.storage.am.lsm.common.impls.NoMergePolicyFactory;
 import org.junit.After;
 import org.junit.AfterClass;
@@ -235,10 +236,11 @@
             // merge all components
             ILSMIndexAccessor mergeAccessor = lsmBtree.createAccessor(NoOpIndexAccessParameters.INSTANCE);
             List<ILSMDiskComponent> mergedComponents = new ArrayList<>(lsmBtree.getDiskComponents());
-            BlockingIOOperationCallbackWrapper ioCallback =
-                    new BlockingIOOperationCallbackWrapper(lsmBtree.getIOOperationCallback());
-            mergeAccessor.scheduleMerge(ioCallback, mergedComponents);
-            ioCallback.waitForIO();
+            ILSMIOOperation merge = mergeAccessor.scheduleMerge(mergedComponents);
+            merge.sync();
+            if (merge.getStatus() == LSMIOOperationStatus.FAILURE) {
+                throw HyracksDataException.create(merge.getFailure());
+            }
             // unblock the search
             unblockSearch(lsmBtree);
             // ensure the search got the correct number
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/StorageTestUtils.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/StorageTestUtils.java
index 390286a..d08fc72 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/StorageTestUtils.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/StorageTestUtils.java
@@ -253,7 +253,7 @@
                 }
 
                 @Override
-                public void after() {
+                public void after(Semaphore t) {
                 }
             });
             Callable<Boolean> callable = new Callable<Boolean>() {
@@ -292,7 +292,7 @@
                 }
 
                 @Override
-                public void after() {
+                public void after(Semaphore t) {
                 }
             });
         }
@@ -319,7 +319,7 @@
                 }
 
                 @Override
-                public void after() {
+                public void after(Semaphore t) {
                 }
             });
         }
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/TestDataset.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/TestDataset.java
index 0c4983a..20875a3 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/TestDataset.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/TestDataset.java
@@ -62,6 +62,6 @@
 
     @Override
     public ILSMIOOperationCallbackFactory getIoOperationCallbackFactory(Index index) throws AlgebricksException {
-        return new TestLsmBtreeIoOpCallbackFactory(getComponentIdGeneratorFactory());
+        return new TestLsmIoOpCallbackFactory(getComponentIdGeneratorFactory(), getDatasetInfoProvider());
     }
 }
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/TestLsmBtreeIoOpCallbackFactory.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/TestLsmIoOpCallbackFactory.java
similarity index 64%
rename from asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/TestLsmBtreeIoOpCallbackFactory.java
rename to asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/TestLsmIoOpCallbackFactory.java
index 5852ad9..c762c8c 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/TestLsmBtreeIoOpCallbackFactory.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/TestLsmIoOpCallbackFactory.java
@@ -18,21 +18,23 @@
  */
 package org.apache.asterix.test.dataflow;
 
-import org.apache.asterix.common.ioopcallbacks.LSMBTreeIOOperationCallback;
-import org.apache.asterix.common.ioopcallbacks.LSMBTreeIOOperationCallbackFactory;
+import org.apache.asterix.common.api.IDatasetInfoProvider;
+import org.apache.asterix.common.api.ILSMComponentIdGeneratorFactory;
+import org.apache.asterix.common.context.DatasetInfo;
+import org.apache.asterix.common.ioopcallbacks.LSMIOOperationCallback;
+import org.apache.asterix.common.ioopcallbacks.LSMIndexIOOperationCallbackFactory;
 import org.apache.asterix.common.storage.IIndexCheckpointManagerProvider;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.storage.am.lsm.btree.impl.TestLsmBtree;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponentIdGenerator;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponentIdGeneratorFactory;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponentId;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperation;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperation.LSMIOOperationType;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallback;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndex;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndexOperationContext;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMMemoryComponent;
 import org.apache.hyracks.storage.am.lsm.common.impls.EmptyComponent;
 
-public class TestLsmBtreeIoOpCallbackFactory extends LSMBTreeIOOperationCallbackFactory {
+public class TestLsmIoOpCallbackFactory extends LSMIndexIOOperationCallbackFactory {
 
     private static final long serialVersionUID = 1L;
 
@@ -43,8 +45,9 @@
     private static volatile int failedFlushes = 0;
     private static volatile int failedMerges = 0;
 
-    public TestLsmBtreeIoOpCallbackFactory(ILSMComponentIdGeneratorFactory idGeneratorFactory) {
-        super(idGeneratorFactory);
+    public TestLsmIoOpCallbackFactory(ILSMComponentIdGeneratorFactory idGeneratorFactory,
+            IDatasetInfoProvider datasetInfoProvider) {
+        super(idGeneratorFactory, datasetInfoProvider);
     }
 
     @Override
@@ -56,7 +59,8 @@
         // Whenever this is called, it resets the counter
         // However, the counters for the failed operations are never reset since we expect them
         // To be always 0
-        return new TestLsmBtreeIoOpCallback(index, getComponentIdGenerator(), getIndexCheckpointManagerProvider());
+        return new TestLsmIoOpCallback(datasetInfoProvider.getDatasetInfo(ncCtx), index,
+                getComponentIdGenerator().getId(), getIndexCheckpointManagerProvider());
     }
 
     public int getTotalFlushes() {
@@ -95,60 +99,78 @@
         return failedMerges;
     }
 
-    public class TestLsmBtreeIoOpCallback extends LSMBTreeIOOperationCallback {
+    public class TestLsmIoOpCallback extends LSMIOOperationCallback {
         private final TestLsmBtree lsmBtree;
 
-        public TestLsmBtreeIoOpCallback(ILSMIndex index, ILSMComponentIdGenerator idGenerator,
+        public TestLsmIoOpCallback(DatasetInfo dsInfo, ILSMIndex index, ILSMComponentId id,
                 IIndexCheckpointManagerProvider checkpointManagerProvider) {
-            super(index, idGenerator, checkpointManagerProvider);
+            super(dsInfo, index, id, checkpointManagerProvider);
             lsmBtree = (TestLsmBtree) index;
         }
 
         @Override
-        public void beforeOperation(ILSMIndexOperationContext opCtx) throws HyracksDataException {
+        public void scheduled(ILSMIOOperation op) throws HyracksDataException {
+            lsmBtree.ioScheduledCalled();
+            super.scheduled(op);
+            lsmBtree.ioScheduledReturned();
+        }
+
+        @Override
+        public void beforeOperation(ILSMIOOperation op) throws HyracksDataException {
             lsmBtree.beforeIoOperationCalled();
-            super.beforeOperation(opCtx);
+            super.beforeOperation(op);
             lsmBtree.beforeIoOperationReturned();
         }
 
         @Override
-        public void afterOperation(ILSMIndexOperationContext opCtx) throws HyracksDataException {
+        public void afterOperation(ILSMIOOperation op) throws HyracksDataException {
             lsmBtree.afterIoOperationCalled();
-            super.afterOperation(opCtx);
+            super.afterOperation(op);
             lsmBtree.afterIoOperationReturned();
         }
 
         @Override
-        public void afterFinalize(ILSMIndexOperationContext opCtx) throws HyracksDataException {
+        public void afterFinalize(ILSMIOOperation op) throws HyracksDataException {
             lsmBtree.afterIoFinalizeCalled();
-            super.afterFinalize(opCtx);
-            synchronized (TestLsmBtreeIoOpCallbackFactory.this) {
-                if (opCtx.getNewComponent() != null) {
-                    if (opCtx.getNewComponent() == EmptyComponent.INSTANCE) {
-                        if (opCtx.getIoOperationType() == LSMIOOperationType.FLUSH) {
+            super.afterFinalize(op);
+            synchronized (TestLsmIoOpCallbackFactory.this) {
+                if (op.getNewComponent() != null) {
+                    if (op.getNewComponent() == EmptyComponent.INSTANCE) {
+                        if (op.getIOOpertionType() == LSMIOOperationType.FLUSH) {
                             rollbackFlushes++;
                         } else {
                             rollbackMerges++;
                         }
                     } else {
-                        if (opCtx.getIoOperationType() == LSMIOOperationType.FLUSH) {
+                        if (op.getIOOpertionType() == LSMIOOperationType.FLUSH) {
                             completedFlushes++;
                         } else {
                             completedMerges++;
                         }
                     }
                 } else {
-                    recordFailure(opCtx.getIoOperationType());
+                    recordFailure(op.getIOOpertionType());
                 }
-                TestLsmBtreeIoOpCallbackFactory.this.notifyAll();
+                TestLsmIoOpCallbackFactory.this.notifyAll();
             }
             lsmBtree.afterIoFinalizeReturned();
         }
 
         @Override
-        public void recycled(ILSMMemoryComponent component, boolean advance) throws HyracksDataException {
+        public void completed(ILSMIOOperation operation) {
+            try {
+                lsmBtree.ioCompletedCalled();
+                super.completed(operation);
+                lsmBtree.ioCompletedReturned();
+            } catch (Exception e) {
+                throw new IllegalStateException(e);
+            }
+        }
+
+        @Override
+        public void recycled(ILSMMemoryComponent component) throws HyracksDataException {
             lsmBtree.recycledCalled(component);
-            super.recycled(component, advance);
+            super.recycled(component);
             lsmBtree.recycledReturned(component);
         }
 
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/TestPrimaryIndexOperationTracker.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/TestPrimaryIndexOperationTracker.java
index 9a528d3..250c25f 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/TestPrimaryIndexOperationTracker.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/TestPrimaryIndexOperationTracker.java
@@ -60,7 +60,7 @@
         super.triggerScheduleFlush(logRecord);
         synchronized (callbacks) {
             for (ITestOpCallback<Void> callback : callbacks) {
-                callback.after();
+                callback.after(null);
             }
         }
     }
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/TransactionAbortTest.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/TransactionAbortTest.java
index 06b213d..7a3e475 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/TransactionAbortTest.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/TransactionAbortTest.java
@@ -199,7 +199,7 @@
             }
 
             @Override
-            public void after() throws HyracksDataException {
+            public void after(Semaphore t) throws HyracksDataException {
                 // manually set the current memory component as modified
                 index.getCurrentMemoryComponent().setModified();
                 throw new HyracksDataException("Fail the job");
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/metadata/MetadataTxnTest.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/metadata/MetadataTxnTest.java
index 0f6adf6..d5985b5 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/metadata/MetadataTxnTest.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/metadata/MetadataTxnTest.java
@@ -51,7 +51,6 @@
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndex;
 import org.apache.hyracks.storage.am.lsm.common.impls.AbstractLSMIndex;
 import org.apache.hyracks.storage.am.lsm.common.impls.NoMergePolicyFactory;
-import org.apache.hyracks.test.support.TestUtils;
 import org.junit.After;
 import org.junit.Assert;
 import org.junit.Before;
@@ -261,8 +260,9 @@
         final MetadataTransactionContext mdTxn = MetadataManager.INSTANCE.beginTransaction();
         metadataProvider.setMetadataTxnContext(mdTxn);
         final String nodeGroupName = "ng";
+        final String committedNodeGroup = "committed_ng";
+        final List<String> ngNodes = Collections.singletonList("asterix_nc1");
         try {
-            final List<String> ngNodes = Collections.singletonList("asterix_nc1");
             MetadataManager.INSTANCE.addNodegroup(mdTxn, new NodeGroup(nodeGroupName, ngNodes));
             MetadataManager.INSTANCE.commitTransaction(mdTxn);
         } finally {
@@ -281,6 +281,9 @@
         int diskComponentsBeforeFlush = index.getDiskComponents().size();
         // lock opTracker to prevent log flusher from triggering flush
         synchronized (opTracker) {
+            final MetadataTransactionContext committedMdTxn = MetadataManager.INSTANCE.beginTransaction();
+            MetadataManager.INSTANCE.addNodegroup(committedMdTxn, new NodeGroup(committedNodeGroup, ngNodes));
+            MetadataManager.INSTANCE.commitTransaction(committedMdTxn);
             opTracker.setFlushOnExit(true);
             opTracker.flushIfNeeded();
             Assert.assertTrue(opTracker.isFlushLogCreated());
@@ -288,7 +291,6 @@
             // make sure force operation will processed
             MetadataManager.INSTANCE.dropNodegroup(mdTxn2, nodeGroupName, false);
             Assert.assertEquals(1, opTracker.getNumActiveOperations());
-            Assert.assertFalse(index.hasFlushRequestForCurrentMutableComponent());
             // release opTracker lock now to allow log flusher to schedule the flush
             InvokeUtil.runWithTimeout(() -> {
                 synchronized (opTracker) {
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/storage/DiskIsFullTest.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/storage/DiskIsFullTest.java
deleted file mode 100644
index 17509a4..0000000
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/storage/DiskIsFullTest.java
+++ /dev/null
@@ -1,182 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.asterix.test.storage;
-
-import static org.apache.hyracks.util.StorageUtil.StorageUnit.MEGABYTE;
-
-import java.nio.file.Path;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Collections;
-import java.util.List;
-
-import org.apache.asterix.api.common.AsterixHyracksIntegrationUtil;
-import org.apache.asterix.app.bootstrap.TestNodeController;
-import org.apache.asterix.app.data.gen.TupleGenerator;
-import org.apache.asterix.app.data.gen.TupleGenerator.GenerationFunction;
-import org.apache.asterix.common.config.DatasetConfig.DatasetType;
-import org.apache.asterix.common.dataflow.LSMInsertDeleteOperatorNodePushable;
-import org.apache.asterix.common.exceptions.ExceptionUtils;
-import org.apache.asterix.common.transactions.ITransactionContext;
-import org.apache.asterix.common.transactions.ITransactionManager;
-import org.apache.asterix.common.transactions.TransactionOptions;
-import org.apache.asterix.external.util.DataflowUtils;
-import org.apache.asterix.file.StorageComponentProvider;
-import org.apache.asterix.metadata.entities.Dataset;
-import org.apache.asterix.metadata.entities.Index;
-import org.apache.asterix.metadata.entities.InternalDatasetDetails;
-import org.apache.asterix.metadata.entities.InternalDatasetDetails.PartitioningStrategy;
-import org.apache.asterix.om.types.ARecordType;
-import org.apache.asterix.om.types.BuiltinType;
-import org.apache.asterix.om.types.IAType;
-import org.apache.asterix.test.common.TestHelper;
-import org.apache.commons.lang3.SystemUtils;
-import org.apache.hyracks.api.comm.VSizeFrame;
-import org.apache.hyracks.api.context.IHyracksTaskContext;
-import org.apache.hyracks.api.exceptions.ErrorCode;
-import org.apache.hyracks.api.exceptions.HyracksDataException;
-import org.apache.hyracks.api.job.JobId;
-import org.apache.hyracks.dataflow.common.comm.io.FrameTupleAppender;
-import org.apache.hyracks.dataflow.common.data.accessors.ITupleReference;
-import org.apache.hyracks.storage.am.lsm.common.impls.NoMergePolicyFactory;
-import org.apache.hyracks.util.DiskUtil;
-import org.junit.After;
-import org.junit.Assert;
-import org.junit.Before;
-import org.junit.Test;
-
-public class DiskIsFullTest {
-
-    private static final IAType[] KEY_TYPES = { BuiltinType.AINT32 };
-    private static final ARecordType RECORD_TYPE = new ARecordType("TestRecordType", new String[] { "key", "value" },
-            new IAType[] { BuiltinType.AINT32, BuiltinType.AINT64 }, false);
-    private static final GenerationFunction[] RECORD_GEN_FUNCTION =
-            { GenerationFunction.DETERMINISTIC, GenerationFunction.DETERMINISTIC };
-    private static final boolean[] UNIQUE_RECORD_FIELDS = { true, false };
-    private static final ARecordType META_TYPE = null;
-    private static final GenerationFunction[] META_GEN_FUNCTION = null;
-    private static final boolean[] UNIQUE_META_FIELDS = null;
-    private static final int[] KEY_INDEXES = { 0 };
-    private static final int[] KEY_INDICATOR = { Index.RECORD_INDICATOR };
-    private static final List<Integer> KEY_INDICATOR_LIST = Arrays.asList(new Integer[] { Index.RECORD_INDICATOR });
-    private static final int DATASET_ID = 101;
-    private static final String DATAVERSE_NAME = "TestDV";
-    private static final String DATASET_NAME = "TestDS";
-    private static final String DATA_TYPE_NAME = "DUMMY";
-    private static final String NODE_GROUP_NAME = "DEFAULT";
-    private static final String TEST_DISK_NAME = "asterixdb_ram_disk";
-    private boolean shouldRun = true;
-
-    @Before
-    public void setUp() throws Exception {
-        if (!SystemUtils.IS_OS_MAC) {
-            System.out.println("Skipping test " + DiskIsFullTest.class.getName() + " due to unsupported OS");
-            shouldRun = false;
-            return;
-        }
-        System.out.println("SetUp: ");
-        TestHelper.deleteExistingInstanceFiles();
-        // create RAM disk
-        final Path ramDiskRoot = DiskUtil.mountRamDisk(TEST_DISK_NAME, 4, MEGABYTE);
-        // Use RAM disk for storage
-        AsterixHyracksIntegrationUtil.setStoragePath(ramDiskRoot.toAbsolutePath().toString());
-    }
-
-    @After
-    public void tearDown() throws Exception {
-        if (!shouldRun) {
-            return;
-        }
-        System.out.println("TearDown");
-        TestHelper.deleteExistingInstanceFiles();
-        DiskUtil.unmountRamDisk(TEST_DISK_NAME);
-        AsterixHyracksIntegrationUtil.restoreDefaultStoragePath();
-    }
-
-    @Test
-    public void testDiskIsFull() {
-        if (!shouldRun) {
-            return;
-        }
-        HyracksDataException expectedException =
-                HyracksDataException.create(ErrorCode.CANNOT_MODIFY_INDEX_DISK_IS_FULL);
-        try {
-            TestNodeController nc = new TestNodeController(null, false);
-            nc.init();
-            StorageComponentProvider storageManager = new StorageComponentProvider();
-            List<List<String>> partitioningKeys = new ArrayList<>();
-            partitioningKeys.add(Collections.singletonList("key"));
-            Dataset dataset = new Dataset(DATAVERSE_NAME, DATASET_NAME, DATAVERSE_NAME, DATA_TYPE_NAME, NODE_GROUP_NAME,
-                    NoMergePolicyFactory.NAME, null, new InternalDatasetDetails(null, PartitioningStrategy.HASH,
-                            partitioningKeys, null, null, null, false, null),
-                    null, DatasetType.INTERNAL, DATASET_ID, 0);
-            try {
-                nc.createPrimaryIndex(dataset, KEY_TYPES, RECORD_TYPE, META_TYPE, null, storageManager, KEY_INDEXES,
-                        KEY_INDICATOR_LIST, 0);
-                JobId jobId = nc.newJobId();
-                IHyracksTaskContext ctx = nc.createTestContext(jobId, 0, false);
-                ITransactionContext txnCtx = nc.getTransactionManager().beginTransaction(nc.getTxnJobId(ctx),
-                        new TransactionOptions(ITransactionManager.AtomicityLevel.ENTITY_LEVEL));
-                // Prepare insert operation
-                LSMInsertDeleteOperatorNodePushable insertOp = nc.getInsertPipeline(ctx, dataset, KEY_TYPES,
-                        RECORD_TYPE, META_TYPE, null, KEY_INDEXES, KEY_INDICATOR_LIST, storageManager, null).getLeft();
-                try {
-                    insertOp.open();
-                    TupleGenerator tupleGenerator =
-                            new TupleGenerator(RECORD_TYPE, META_TYPE, KEY_INDEXES, KEY_INDICATOR, RECORD_GEN_FUNCTION,
-                                    UNIQUE_RECORD_FIELDS, META_GEN_FUNCTION, UNIQUE_META_FIELDS);
-                    VSizeFrame frame = new VSizeFrame(ctx);
-                    FrameTupleAppender tupleAppender = new FrameTupleAppender(frame);
-                    // Insert records until disk becomes full
-                    int tupleCount = 100000;
-                    while (tupleCount > 0) {
-                        ITupleReference tuple = tupleGenerator.next();
-                        try {
-                            DataflowUtils.addTupleToFrame(tupleAppender, tuple, insertOp);
-                        } catch (Throwable t) {
-                            final Throwable rootCause = ExceptionUtils.getRootCause(t);
-                            rootCause.printStackTrace();
-                            if (rootCause instanceof HyracksDataException) {
-                                HyracksDataException cause = (HyracksDataException) rootCause;
-                                Assert.assertEquals(cause.getErrorCode(), expectedException.getErrorCode());
-                                Assert.assertEquals(cause.getMessage(), expectedException.getMessage());
-                                return;
-                            } else {
-                                break;
-                            }
-                        }
-                        tupleCount--;
-                    }
-                    Assert.fail("Expected exception (" + expectedException + ") was not thrown");
-                } finally {
-                    try {
-                        insertOp.close();
-                    } finally {
-                        nc.getTransactionManager().abortTransaction(txnCtx.getTxnId());
-                    }
-                }
-            } finally {
-                nc.deInit();
-            }
-        } catch (Throwable e) {
-            e.printStackTrace();
-            Assert.fail("Expected exception (" + expectedException + ") was not thrown");
-        }
-    }
-}
\ 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/ILSMComponentIdGeneratorFactory.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/api/IDatasetInfoProvider.java
similarity index 69%
copy from hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/ILSMComponentIdGeneratorFactory.java
copy to asterixdb/asterix-common/src/main/java/org/apache/asterix/common/api/IDatasetInfoProvider.java
index 4ec82c1..7075739 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/ILSMComponentIdGeneratorFactory.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/api/IDatasetInfoProvider.java
@@ -16,16 +16,21 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.hyracks.storage.am.lsm.common.api;
+package org.apache.asterix.common.api;
 
 import java.io.Serializable;
 
+import org.apache.asterix.common.context.DatasetInfo;
 import org.apache.hyracks.api.application.INCServiceContext;
-import org.apache.hyracks.api.exceptions.HyracksDataException;
-import org.apache.hyracks.storage.common.IResource;
 
 @FunctionalInterface
-public interface ILSMComponentIdGeneratorFactory extends Serializable {
-    ILSMComponentIdGenerator getComponentIdGenerator(INCServiceContext serviceCtx, IResource resource)
-            throws HyracksDataException;
+public interface IDatasetInfoProvider extends Serializable {
+    /**
+     * Get the dataset info
+     *
+     * @param serviceCtx
+     *            the nc service context
+     * @return the dataset info object
+     */
+    DatasetInfo getDatasetInfo(INCServiceContext serviceCtx);
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/ILSMComponentIdGeneratorFactory.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/api/ILSMComponentIdGeneratorFactory.java
similarity index 81%
rename from hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/ILSMComponentIdGeneratorFactory.java
rename to asterixdb/asterix-common/src/main/java/org/apache/asterix/common/api/ILSMComponentIdGeneratorFactory.java
index 4ec82c1..5fd37e9 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/ILSMComponentIdGeneratorFactory.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/api/ILSMComponentIdGeneratorFactory.java
@@ -16,16 +16,25 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.hyracks.storage.am.lsm.common.api;
+package org.apache.asterix.common.api;
 
 import java.io.Serializable;
 
 import org.apache.hyracks.api.application.INCServiceContext;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponentIdGenerator;
 import org.apache.hyracks.storage.common.IResource;
 
 @FunctionalInterface
 public interface ILSMComponentIdGeneratorFactory extends Serializable {
+    /**
+     * Get the Id generator for the resource
+     *
+     * @param serviceCtx
+     * @param resource
+     * @return
+     * @throws HyracksDataException
+     */
     ILSMComponentIdGenerator getComponentIdGenerator(INCServiceContext serviceCtx, IResource resource)
             throws HyracksDataException;
 }
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/context/BaseOperationTracker.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/context/BaseOperationTracker.java
index b7af0b6..3ff13cb 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/context/BaseOperationTracker.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/context/BaseOperationTracker.java
@@ -37,8 +37,7 @@
     @Override
     public void beforeOperation(ILSMIndex index, LSMOperationType opType, ISearchOperationCallback searchCallback,
             IModificationOperationCallback modificationCallback) throws HyracksDataException {
-        if (opType == LSMOperationType.FLUSH || opType == LSMOperationType.MERGE
-                || opType == LSMOperationType.REPLICATE) {
+        if (opType == LSMOperationType.REPLICATE) {
             dsInfo.declareActiveIOOperation();
         }
     }
@@ -47,14 +46,14 @@
     public void afterOperation(ILSMIndex index, LSMOperationType opType, ISearchOperationCallback searchCallback,
             IModificationOperationCallback modificationCallback) throws HyracksDataException {
         if (opType == LSMOperationType.REPLICATE) {
-            dsInfo.undeclareActiveIOOperation();
+            completeOperation(index, opType, searchCallback, modificationCallback);
         }
     }
 
     @Override
     public void completeOperation(ILSMIndex index, LSMOperationType opType, ISearchOperationCallback searchCallback,
             IModificationOperationCallback modificationCallback) throws HyracksDataException {
-        if (opType == LSMOperationType.FLUSH || opType == LSMOperationType.MERGE) {
+        if (opType == LSMOperationType.REPLICATE) {
             dsInfo.undeclareActiveIOOperation();
         }
     }
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/context/CorrelatedPrefixMergePolicy.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/context/CorrelatedPrefixMergePolicy.java
index 41461ec..a14fac8 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/context/CorrelatedPrefixMergePolicy.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/context/CorrelatedPrefixMergePolicy.java
@@ -125,7 +125,7 @@
                 }
             }
             ILSMIndexAccessor accessor = lsmIndex.createAccessor(NoOpIndexAccessParameters.INSTANCE);
-            accessor.scheduleMerge(lsmIndex.getIOOperationCallback(), mergableComponents);
+            accessor.scheduleMerge(mergableComponents);
         }
     }
 }
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/context/DatasetInfoProvider.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/context/DatasetInfoProvider.java
new file mode 100644
index 0000000..597b5da
--- /dev/null
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/context/DatasetInfoProvider.java
@@ -0,0 +1,42 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.common.context;
+
+import org.apache.asterix.common.api.IDatasetInfoProvider;
+import org.apache.asterix.common.api.IDatasetLifecycleManager;
+import org.apache.asterix.common.api.INcApplicationContext;
+import org.apache.hyracks.api.application.INCServiceContext;
+
+public class DatasetInfoProvider implements IDatasetInfoProvider {
+
+    private static final long serialVersionUID = 1L;
+    private final int datasetId;
+
+    public DatasetInfoProvider(int datasetId) {
+        this.datasetId = datasetId;
+    }
+
+    @Override
+    public DatasetInfo getDatasetInfo(INCServiceContext serviceCtx) {
+        IDatasetLifecycleManager dslcManager =
+                ((INcApplicationContext) serviceCtx.getApplicationContext()).getDatasetLifecycleManager();
+        return dslcManager.getDatasetInfo(datasetId);
+    }
+
+}
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/context/DatasetLSMComponentIdGeneratorFactory.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/context/DatasetLSMComponentIdGeneratorFactory.java
index 83e3144..85eacf8 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/context/DatasetLSMComponentIdGeneratorFactory.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/context/DatasetLSMComponentIdGeneratorFactory.java
@@ -20,12 +20,12 @@
 package org.apache.asterix.common.context;
 
 import org.apache.asterix.common.api.IDatasetLifecycleManager;
+import org.apache.asterix.common.api.ILSMComponentIdGeneratorFactory;
 import org.apache.asterix.common.api.INcApplicationContext;
 import org.apache.asterix.common.utils.StoragePathUtil;
 import org.apache.hyracks.api.application.INCServiceContext;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponentIdGenerator;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponentIdGeneratorFactory;
 import org.apache.hyracks.storage.common.IResource;
 
 /**
@@ -51,5 +51,4 @@
         int partition = StoragePathUtil.getPartitionNumFromRelativePath(resource.getPath());
         return dslcManager.getComponentIdGenerator(datasetId, partition);
     }
-
 }
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/context/DatasetLifecycleManager.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/context/DatasetLifecycleManager.java
index b715eec..e5d18cf 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/context/DatasetLifecycleManager.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/context/DatasetLifecycleManager.java
@@ -25,6 +25,7 @@
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.Collections;
+import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 import java.util.concurrent.ConcurrentHashMap;
@@ -34,7 +35,7 @@
 import org.apache.asterix.common.config.StorageProperties;
 import org.apache.asterix.common.dataflow.DatasetLocalResource;
 import org.apache.asterix.common.exceptions.ACIDException;
-import org.apache.asterix.common.ioopcallbacks.AbstractLSMIOOperationCallback;
+import org.apache.asterix.common.ioopcallbacks.LSMIOOperationCallback;
 import org.apache.asterix.common.metadata.MetadataIndexImmutableProperties;
 import org.apache.asterix.common.replication.IReplicationStrategy;
 import org.apache.asterix.common.storage.DatasetResourceReference;
@@ -47,6 +48,7 @@
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.api.lifecycle.ILifeCycleComponent;
 import org.apache.hyracks.storage.am.common.impls.NoOpIndexAccessParameters;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponentId;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponentIdGenerator;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndex;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndexAccessor;
@@ -348,7 +350,7 @@
     }
 
     private void populateOpTrackerAndIdGenerator(DatasetResource dataset, int partition) {
-        ILSMComponentIdGenerator idGenerator = new LSMComponentIdGenerator();
+        ILSMComponentIdGenerator idGenerator = new LSMComponentIdGenerator(storageProperties.getMemoryComponentsNum());
         PrimaryIndexOperationTracker opTracker = new PrimaryIndexOperationTracker(dataset.getDatasetID(), partition,
                 logManager, dataset.getDatasetInfo(), idGenerator);
         dataset.setPrimaryIndexOperationTracker(partition, opTracker);
@@ -398,11 +400,10 @@
             PrimaryIndexOperationTracker opTracker, long targetLSN) throws HyracksDataException {
         int partition = opTracker.getPartition();
         for (ILSMIndex lsmIndex : dsr.getDatasetInfo().getDatasetPartitionOpenIndexes(partition)) {
-            AbstractLSMIOOperationCallback ioCallback =
-                    (AbstractLSMIOOperationCallback) lsmIndex.getIOOperationCallback();
+            LSMIOOperationCallback ioCallback = (LSMIOOperationCallback) lsmIndex.getIOOperationCallback();
             if (!(lsmIndex.isCurrentMutableComponentEmpty() || ioCallback.hasPendingFlush()
                     || opTracker.isFlushLogCreated() || opTracker.isFlushOnExit())) {
-                long firstLSN = ioCallback.getFirstLSN();
+                long firstLSN = ioCallback.getPersistenceLsn();
                 if (firstLSN < targetLSN) {
                     LOGGER.info("Checkpoint flush dataset {} partition {}", dsr.getDatasetID(), partition);
                     opTracker.setFlushOnExit(true);
@@ -448,9 +449,7 @@
             LSMComponentId componentId = (LSMComponentId) flushIndex.getCurrentMemoryComponent().getId();
             ILSMComponentIdGenerator idGenerator = getComponentIdGenerator(dsInfo.getDatasetID(), partition);
             idGenerator.refresh();
-
             if (dsInfo.isDurable()) {
-
                 synchronized (logRecord) {
                     TransactionUtil.formFlushLogRecord(logRecord, dsInfo.getDatasetID(), partition,
                             componentId.getMinId(), componentId.getMaxId(), null);
@@ -469,32 +468,21 @@
                     }
                 }
             }
+            long flushLsn = logRecord.getLSN();
+            ILSMComponentId nextComponentId = idGenerator.getId();
+            Map<String, Object> flushMap = new HashMap<>();
+            flushMap.put(LSMIOOperationCallback.KEY_FLUSH_LOG_LSN, flushLsn);
+            flushMap.put(LSMIOOperationCallback.KEY_NEXT_COMPONENT_ID, nextComponentId);
             for (ILSMIndex index : indexes) {
-                //update resource lsn
-                AbstractLSMIOOperationCallback ioOpCallback =
-                        (AbstractLSMIOOperationCallback) index.getIOOperationCallback();
-                ioOpCallback.updateLastLSN(logRecord.getLSN());
+                ILSMIndexAccessor accessor = index.createAccessor(NoOpIndexAccessParameters.INSTANCE);
+                accessor.getOpContext().setParameters(flushMap);
+                accessor.scheduleFlush();
             }
-
-            if (asyncFlush) {
-                for (ILSMIndex index : indexes) {
-                    ILSMIndexAccessor accessor = index.createAccessor(NoOpIndexAccessParameters.INSTANCE);
-                    accessor.scheduleFlush(index.getIOOperationCallback());
-                }
-            } else {
-                for (ILSMIndex index : indexes) {
-                    // TODO: This is not efficient since we flush the indexes sequentially.
-                    // Think of a way to allow submitting the flush requests concurrently.
-                    // We don't do them concurrently because this may lead to a deadlock scenario
-                    // between the DatasetLifeCycleManager and the PrimaryIndexOperationTracker.
-                    ILSMIndexAccessor accessor = index.createAccessor(NoOpIndexAccessParameters.INSTANCE);
-                    accessor.scheduleFlush(index.getIOOperationCallback());
-                    // Wait for the above flush op.
-                    dsInfo.waitForIO();
-                }
+            if (!asyncFlush) {
+                // Wait for the above flush op.
+                dsInfo.waitForIO();
             }
         }
-
     }
 
     private void closeDataset(DatasetResource dsr) throws HyracksDataException {
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/context/PrimaryIndexOperationTracker.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/context/PrimaryIndexOperationTracker.java
index 74848d5..681669f 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/context/PrimaryIndexOperationTracker.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/context/PrimaryIndexOperationTracker.java
@@ -19,11 +19,13 @@
 
 package org.apache.asterix.common.context;
 
+import java.util.HashMap;
+import java.util.Map;
 import java.util.Set;
 import java.util.concurrent.atomic.AtomicInteger;
 
 import org.apache.asterix.common.exceptions.ACIDException;
-import org.apache.asterix.common.ioopcallbacks.AbstractLSMIOOperationCallback;
+import org.apache.asterix.common.ioopcallbacks.LSMIOOperationCallback;
 import org.apache.asterix.common.metadata.MetadataIndexImmutableProperties;
 import org.apache.asterix.common.transactions.AbstractOperationCallback;
 import org.apache.asterix.common.transactions.ILogManager;
@@ -33,6 +35,7 @@
 import org.apache.hyracks.storage.am.common.impls.NoOpIndexAccessParameters;
 import org.apache.hyracks.storage.am.common.impls.NoOpOperationCallback;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponent.ComponentState;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponentId;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponentIdGenerator;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndex;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndexAccessor;
@@ -44,7 +47,6 @@
 import org.apache.hyracks.storage.common.ISearchOperationCallback;
 
 public class PrimaryIndexOperationTracker extends BaseOperationTracker {
-
     private final int partition;
     // Number of active operations on an ILSMIndex instance.
     private final AtomicInteger numActiveOperations;
@@ -65,20 +67,9 @@
     @Override
     public void beforeOperation(ILSMIndex index, LSMOperationType opType, ISearchOperationCallback searchCallback,
             IModificationOperationCallback modificationCallback) throws HyracksDataException {
+        super.beforeOperation(index, opType, searchCallback, modificationCallback);
         if (opType == LSMOperationType.MODIFICATION || opType == LSMOperationType.FORCE_MODIFICATION) {
             incrementNumActiveOperations(modificationCallback);
-        } else if (opType == LSMOperationType.FLUSH || opType == LSMOperationType.MERGE
-                || opType == LSMOperationType.REPLICATE) {
-            dsInfo.declareActiveIOOperation();
-        }
-    }
-
-    @Override
-    public void afterOperation(ILSMIndex index, LSMOperationType opType, ISearchOperationCallback searchCallback,
-            IModificationOperationCallback modificationCallback) throws HyracksDataException {
-        // Searches are immediately considered complete, because they should not prevent the execution of flushes.
-        if (opType == LSMOperationType.REPLICATE) {
-            completeOperation(index, opType, searchCallback, modificationCallback);
         }
     }
 
@@ -86,12 +77,10 @@
     public synchronized void completeOperation(ILSMIndex index, LSMOperationType opType,
             ISearchOperationCallback searchCallback, IModificationOperationCallback modificationCallback)
             throws HyracksDataException {
+        super.completeOperation(index, opType, searchCallback, modificationCallback);
         if (opType == LSMOperationType.MODIFICATION || opType == LSMOperationType.FORCE_MODIFICATION) {
             decrementNumActiveOperations(modificationCallback);
             flushIfNeeded();
-        } else if (opType == LSMOperationType.FLUSH || opType == LSMOperationType.MERGE
-                || opType == LSMOperationType.REPLICATE) {
-            dsInfo.undeclareActiveIOOperation();
         }
     }
 
@@ -121,12 +110,29 @@
             // make the current mutable components READABLE_UNWRITABLE to stop coming modify operations from entering
             // them until the current flush is scheduled.
             LSMComponentId primaryId = null;
+            //Double check that the primary index has been modified
+            synchronized (this) {
+                if (numActiveOperations.get() > 0) {
+                    throw new IllegalStateException(
+                            "Can't request a flush on an index with active operations: " + numActiveOperations.get());
+                }
+                for (ILSMIndex lsmIndex : indexes) {
+                    if (lsmIndex.isPrimaryIndex() && lsmIndex.isCurrentMutableComponentEmpty()) {
+                        return;
+                    }
+                }
+            }
             for (ILSMIndex lsmIndex : indexes) {
                 ILSMOperationTracker opTracker = lsmIndex.getOperationTracker();
                 synchronized (opTracker) {
                     ILSMMemoryComponent memComponent = lsmIndex.getCurrentMemoryComponent();
+                    if (memComponent.getWriterCount() > 0) {
+                        throw new IllegalStateException(
+                                "Can't request a flush on a component with writers inside: Index:" + lsmIndex
+                                        + " Component:" + memComponent);
+                    }
                     if (memComponent.getState() == ComponentState.READABLE_WRITABLE && memComponent.isModified()) {
-                        memComponent.setState(ComponentState.READABLE_UNWRITABLE);
+                        memComponent.setUnwritable();
                     }
                     if (lsmIndex.isPrimaryIndex()) {
                         primaryId = (LSMComponentId) memComponent.getId();
@@ -171,15 +177,15 @@
                 throw new IllegalStateException("Operation started while index was pending scheduling a flush");
             }
             idGenerator.refresh();
+            long flushLsn = logRecord.getLSN();
+            ILSMComponentId nextComponentId = idGenerator.getId();
+            Map<String, Object> flushMap = new HashMap<>();
+            flushMap.put(LSMIOOperationCallback.KEY_FLUSH_LOG_LSN, flushLsn);
+            flushMap.put(LSMIOOperationCallback.KEY_NEXT_COMPONENT_ID, nextComponentId);
             for (ILSMIndex lsmIndex : dsInfo.getDatasetPartitionOpenIndexes(partition)) {
-                //get resource
                 ILSMIndexAccessor accessor = lsmIndex.createAccessor(NoOpIndexAccessParameters.INSTANCE);
-                //update resource lsn
-                AbstractLSMIOOperationCallback ioOpCallback =
-                        (AbstractLSMIOOperationCallback) lsmIndex.getIOOperationCallback();
-                ioOpCallback.updateLastLSN(logRecord.getLSN());
-                //schedule flush after update
-                accessor.scheduleFlush(lsmIndex.getIOOperationCallback());
+                accessor.getOpContext().setParameters(flushMap);
+                accessor.scheduleFlush();
             }
         } finally {
             flushLogCreated = false;
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/dataflow/LSMIndexUtil.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/dataflow/LSMIndexUtil.java
index e844192..99ab2d0 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/dataflow/LSMIndexUtil.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/dataflow/LSMIndexUtil.java
@@ -18,15 +18,16 @@
  */
 package org.apache.asterix.common.dataflow;
 
-import org.apache.asterix.common.ioopcallbacks.AbstractLSMIOOperationCallback;
+import org.apache.asterix.common.ioopcallbacks.LSMIOOperationCallback;
 import org.apache.asterix.common.transactions.ILogManager;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMDiskComponent;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndex;
 import org.apache.hyracks.storage.am.lsm.common.impls.AbstractLSMIndex;
 
 public class LSMIndexUtil {
 
+    private LSMIndexUtil() {
+    }
+
     public static void checkAndSetFirstLSN(AbstractLSMIndex lsmIndex, ILogManager logManager)
             throws HyracksDataException {
         // If the index has an empty memory component, we need to set its first LSN (For soft checkpoint)
@@ -34,9 +35,8 @@
             //prevent transactions from incorrectly setting the first LSN on a modified component by checking the index is still empty
             synchronized (lsmIndex.getOperationTracker()) {
                 if (lsmIndex.isCurrentMutableComponentEmpty()) {
-                    AbstractLSMIOOperationCallback ioOpCallback =
-                            (AbstractLSMIOOperationCallback) lsmIndex.getIOOperationCallback();
-                    ioOpCallback.setFirstLSN(logManager.getAppendLSN());
+                    LSMIOOperationCallback ioOpCallback = (LSMIOOperationCallback) lsmIndex.getIOOperationCallback();
+                    ioOpCallback.setFirstLsnForCurrentMemoryComponent(logManager.getAppendLSN());
                 }
             }
         }
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/exceptions/ExceptionUtils.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/exceptions/ExceptionUtils.java
index 256ce08..359054e 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/exceptions/ExceptionUtils.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/exceptions/ExceptionUtils.java
@@ -20,6 +20,8 @@
 
 import java.util.function.Predicate;
 
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
 public class ExceptionUtils {
     public static final String INCORRECT_PARAMETER = "Incorrect parameter.\n";
     public static final String PARAMETER_NAME = "Parameter name: ";
@@ -45,13 +47,28 @@
         Throwable current = e;
         Throwable cause = e.getCause();
         while (cause != null && cause != current) {
-            Throwable nextCause = current.getCause();
             current = cause;
-            cause = nextCause;
+            cause = current.getCause();
         }
         return current;
     }
 
+    public static Throwable getCause(Throwable e, String component, int code) {
+        Throwable current = e;
+        Throwable expected =
+                (current instanceof HyracksDataException && ((HyracksDataException) current).getErrorCode() == code
+                        && ((HyracksDataException) current).getComponent().equals(component)) ? current : null;
+        Throwable cause = e.getCause();
+        while (cause != null && cause != current) {
+            current = cause;
+            expected =
+                    (current instanceof HyracksDataException && ((HyracksDataException) current).getErrorCode() == code
+                            && ((HyracksDataException) current).getComponent().equals(component)) ? current : expected;
+            cause = current.getCause();
+        }
+        return expected == null ? current : expected;
+    }
+
     /**
      * Determines whether supplied exception contains a matching cause in its hierarchy, or is itself a match
      */
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/ioopcallbacks/AbstractLSMIOOperationCallback.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/ioopcallbacks/AbstractLSMIOOperationCallback.java
deleted file mode 100644
index f027979..0000000
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/ioopcallbacks/AbstractLSMIOOperationCallback.java
+++ /dev/null
@@ -1,287 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-package org.apache.asterix.common.ioopcallbacks;
-
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.Optional;
-
-import org.apache.asterix.common.storage.IIndexCheckpointManagerProvider;
-import org.apache.asterix.common.storage.ResourceReference;
-import org.apache.hyracks.api.exceptions.HyracksDataException;
-import org.apache.hyracks.data.std.primitive.LongPointable;
-import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
-import org.apache.hyracks.storage.am.common.api.IMetadataPageManager;
-import org.apache.hyracks.storage.am.common.freepage.MutableArrayValueReference;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponent;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponentId;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponentIdGenerator;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMDiskComponent;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperation.LSMIOOperationType;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallback;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndex;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndexOperationContext;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMMemoryComponent;
-import org.apache.hyracks.storage.am.lsm.common.impls.AbstractLSMIndexFileManager;
-import org.apache.hyracks.storage.am.lsm.common.impls.DiskComponentMetadata;
-import org.apache.hyracks.storage.am.lsm.common.util.ComponentUtils;
-import org.apache.hyracks.storage.am.lsm.common.util.LSMComponentIdUtils;
-
-// A single LSMIOOperationCallback per LSM index used to perform actions around Flush and Merge operations
-public abstract class AbstractLSMIOOperationCallback implements ILSMIOOperationCallback {
-    public static final MutableArrayValueReference LSN_KEY = new MutableArrayValueReference("LSN".getBytes());
-    public static final long INVALID = -1L;
-
-    protected final ILSMIndex lsmIndex;
-    // First LSN per mutable component. TODO: move from hyracks to asterixdb
-    protected final long[] firstLSNs;
-    // A boolean array to keep track of flush operations
-    protected final boolean[] flushRequested;
-    // TODO: move from hyracks to asterixdb
-    protected final long[] mutableLastLSNs;
-    // Index of the currently flushing or next to be flushed component
-    protected int readIndex;
-    // Index of the currently being written to component
-    protected int writeIndex;
-    // Index of the memory component to be recycled
-    protected int recycleIndex;
-    // Indicates whether this index has been scheduled to flush (no matter whether succeeds or not)
-    protected boolean hasFlushed;
-    // Keep track of the component Id of the next component being activated.
-    protected ILSMComponentId[] nextComponentIds;
-
-    protected final ILSMComponentIdGenerator idGenerator;
-    protected final ArrayBackedValueStorage buffer = new ArrayBackedValueStorage(Long.BYTES);
-    private final IIndexCheckpointManagerProvider indexCheckpointManagerProvider;
-    private final Map<ILSMComponentId, Long> componentLsnMap = new HashMap<>();
-
-    public AbstractLSMIOOperationCallback(ILSMIndex lsmIndex, ILSMComponentIdGenerator idGenerator,
-            IIndexCheckpointManagerProvider indexCheckpointManagerProvider) {
-        this.lsmIndex = lsmIndex;
-        this.idGenerator = idGenerator;
-        this.indexCheckpointManagerProvider = indexCheckpointManagerProvider;
-        int count = lsmIndex.getNumberOfAllMemoryComponents();
-        mutableLastLSNs = new long[count];
-        firstLSNs = new long[count];
-        flushRequested = new boolean[count];
-        readIndex = 0;
-        writeIndex = 0;
-        recycleIndex = 0;
-        hasFlushed = false;
-        nextComponentIds = new ILSMComponentId[count];
-        if (count > 0) {
-            nextComponentIds[0] = idGenerator.getId();
-        }
-    }
-
-    @Override
-    public void beforeOperation(ILSMIndexOperationContext opCtx) throws HyracksDataException {
-        if (opCtx.getIoOperationType() == LSMIOOperationType.FLUSH) {
-            /*
-             * This method was called on the scheduleFlush operation.
-             * We set the lastLSN to the last LSN for the index (the LSN for the flush log)
-             * We mark the component flushing flag
-             * We then move the write pointer to the next component and sets its first LSN to the flush log LSN
-             */
-            synchronized (this) {
-                flushRequested[writeIndex] = true;
-                writeIndex = (writeIndex + 1) % mutableLastLSNs.length;
-                // Set the firstLSN of the next component unless it is being flushed
-                if (writeIndex != readIndex) {
-                    firstLSNs[writeIndex] = mutableLastLSNs[writeIndex];
-                }
-
-            }
-        }
-    }
-
-    @Override
-    public void afterOperation(ILSMIndexOperationContext opCtx) throws HyracksDataException {
-        //TODO: Copying Filters and all content of the metadata pages for flush operation should be done here
-        if (opCtx.getNewComponent() == null) {
-            // failed operation. Nothing to do.
-            return;
-        }
-        putLSNIntoMetadata(opCtx.getNewComponent(), opCtx.getComponentsToBeMerged());
-        putComponentIdIntoMetadata(opCtx.getIoOperationType(), opCtx.getNewComponent(),
-                opCtx.getComponentsToBeMerged());
-        componentLsnMap.put(opCtx.getNewComponent().getId(), getComponentLSN(opCtx.getComponentsToBeMerged()));
-        if (opCtx.getIoOperationType() == LSMIOOperationType.MERGE) {
-            if (opCtx.getComponentsToBeMerged().isEmpty()) {
-                throw new IllegalStateException("Merge must have old components");
-            }
-            LongPointable markerLsn = LongPointable.FACTORY
-                    .createPointable(ComponentUtils.getLong(opCtx.getComponentsToBeMerged().get(0).getMetadata(),
-                            ComponentUtils.MARKER_LSN_KEY, ComponentUtils.NOT_FOUND, buffer));
-            opCtx.getNewComponent().getMetadata().put(ComponentUtils.MARKER_LSN_KEY, markerLsn);
-        } else if (opCtx.getIoOperationType() == LSMIOOperationType.FLUSH) {
-            // advance memory component indexes
-            synchronized (this) {
-                // we've already consumed the specified LSN/component id.
-                // Now we can advance to the next component
-                flushRequested[readIndex] = false;
-                // if the component which just finished flushing is the component that will be modified next,
-                // we set its first LSN to its previous LSN
-                if (readIndex == writeIndex) {
-                    firstLSNs[writeIndex] = mutableLastLSNs[writeIndex];
-                }
-                readIndex = (readIndex + 1) % mutableLastLSNs.length;
-            }
-        }
-    }
-
-    @Override
-    public void afterFinalize(ILSMIndexOperationContext opCtx) throws HyracksDataException {
-        // The operation was complete and the next I/O operation for the LSM index didn't start yet
-        if (opCtx.getIoOperationType() == LSMIOOperationType.FLUSH) {
-            hasFlushed = true;
-            if (opCtx.getNewComponent() != null) {
-                final Long lsn = componentLsnMap.remove(opCtx.getNewComponent().getId());
-                if (lsn == null) {
-                    throw new IllegalStateException("Unidentified flushed component: " + opCtx.getNewComponent());
-                }
-                // empty component doesn't have any files
-                final Optional<String> componentFile =
-                        opCtx.getNewComponent().getLSMComponentPhysicalFiles().stream().findAny();
-                if (componentFile.isPresent()) {
-                    final ResourceReference ref = ResourceReference.of(componentFile.get());
-                    final String componentEndTime = AbstractLSMIndexFileManager.getComponentEndTime(ref.getName());
-                    indexCheckpointManagerProvider.get(ref).flushed(componentEndTime, lsn);
-                }
-            }
-        }
-    }
-
-    private void putLSNIntoMetadata(ILSMDiskComponent newComponent, List<? extends ILSMComponent> oldComponents)
-            throws HyracksDataException {
-        newComponent.getMetadata().put(LSN_KEY, LongPointable.FACTORY.createPointable(getComponentLSN(oldComponents)));
-    }
-
-    public static long getTreeIndexLSN(DiskComponentMetadata md) throws HyracksDataException {
-        LongPointable pointable = new LongPointable();
-        IMetadataPageManager metadataPageManager = md.getMetadataPageManager();
-        metadataPageManager.get(metadataPageManager.createMetadataFrame(), LSN_KEY, pointable);
-        return pointable.getLength() == 0 ? INVALID : pointable.longValue();
-    }
-
-    private ILSMComponentId getMergedComponentId(List<? extends ILSMComponent> mergedComponents)
-            throws HyracksDataException {
-        if (mergedComponents.isEmpty()) {
-            return null;
-        }
-        return LSMComponentIdUtils.union(mergedComponents.get(0).getId(),
-                mergedComponents.get(mergedComponents.size() - 1).getId());
-
-    }
-
-    private void putComponentIdIntoMetadata(LSMIOOperationType opType, ILSMDiskComponent newComponent,
-            List<? extends ILSMComponent> oldComponents) throws HyracksDataException {
-        // the id of flushed component is set when we copy the metadata of the memory component
-        if (opType == LSMIOOperationType.MERGE) {
-            ILSMComponentId componentId = getMergedComponentId(oldComponents);
-            LSMComponentIdUtils.persist(componentId, newComponent.getMetadata());
-        }
-    }
-
-    public synchronized void updateLastLSN(long lastLSN) {
-        if (!flushRequested[writeIndex]) {
-            //if the memory component pointed by writeIndex is being flushed, we should ignore this update call
-            //since otherwise the original LSN is overwritten.
-            //Moreover, since the memory component is already being flushed, the next scheduleFlush request must fail.
-            //See https://issues.apache.org/jira/browse/ASTERIXDB-1917
-            mutableLastLSNs[writeIndex] = lastLSN;
-            if (hasFlushed || lsmIndex.isMemoryComponentsAllocated()) {
-                // we only (re)set next component id if either this index has been flushed (no matter succeed or not)
-                // or the memory component has been allocated
-                // This prevents the case where indexes in a partition are being allocated, while another partition
-                // tries to schedule flush
-                nextComponentIds[writeIndex] = idGenerator.getId();
-            }
-        }
-    }
-
-    /**
-     * Used during the recovery process to force refresh the next component id
-     */
-    public void forceRefreshNextId() {
-        nextComponentIds[writeIndex] = idGenerator.getId();
-    }
-
-    public synchronized void setFirstLSN(long firstLSN) {
-        // We make sure that this method is only called on an empty component so the first LSN is not set incorrectly
-        firstLSNs[writeIndex] = firstLSN;
-    }
-
-    public synchronized long getFirstLSN() {
-        // We make sure that this method is only called on a non-empty component so the returned LSN is meaningful
-        // The firstLSN is always the lsn of the currently being flushed component or the next
-        // to be flushed when no flush operation is on going
-        return firstLSNs[readIndex];
-    }
-
-    public synchronized boolean hasPendingFlush() {
-        for (int i = 0; i < flushRequested.length; i++) {
-            if (flushRequested[i]) {
-                return true;
-            }
-        }
-        return false;
-    }
-
-    public long getComponentLSN(List<? extends ILSMComponent> diskComponents) throws HyracksDataException {
-        if (diskComponents.isEmpty()) {
-            // Implies a flush IO operation. --> moves the flush pointer
-            // Flush operation of an LSM index are executed sequentially.
-            synchronized (this) {
-                return mutableLastLSNs[readIndex];
-            }
-        }
-        // Get max LSN from the diskComponents. Implies a merge IO operation or Recovery operation.
-        long maxLSN = -1L;
-        for (ILSMComponent c : diskComponents) {
-            DiskComponentMetadata md = ((ILSMDiskComponent) c).getMetadata();
-            maxLSN = Math.max(getTreeIndexLSN(md), maxLSN);
-        }
-        return maxLSN;
-    }
-
-    private synchronized ILSMComponentId getLSMComponentId() {
-        return nextComponentIds[recycleIndex];
-    }
-
-    @Override
-    public void recycled(ILSMMemoryComponent component, boolean componentSwitched) throws HyracksDataException {
-        ILSMComponentId componentId = getLSMComponentId();
-        component.resetId(componentId, false);
-        if (componentSwitched) {
-            recycleIndex = (recycleIndex + 1) % nextComponentIds.length;
-        }
-    }
-
-    @Override
-    public void allocated(ILSMMemoryComponent component) throws HyracksDataException {
-        if (component == lsmIndex.getCurrentMemoryComponent()) {
-            // only set the component id for the first (current) memory component
-            ILSMComponentId componentId = getLSMComponentId();
-            component.resetId(componentId, false);
-        }
-    }
-}
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/ioopcallbacks/AbstractLSMIndexIOOperationCallbackFactory.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/ioopcallbacks/AbstractLSMIndexIOOperationCallbackFactory.java
deleted file mode 100644
index 5b9883c..0000000
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/ioopcallbacks/AbstractLSMIndexIOOperationCallbackFactory.java
+++ /dev/null
@@ -1,86 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-package org.apache.asterix.common.ioopcallbacks;
-
-import java.io.ObjectStreamException;
-
-import org.apache.asterix.common.api.INcApplicationContext;
-import org.apache.asterix.common.storage.IIndexCheckpointManagerProvider;
-import org.apache.hyracks.api.application.INCServiceContext;
-import org.apache.hyracks.api.exceptions.HyracksDataException;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponentId;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponentIdGenerator;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponentIdGeneratorFactory;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallbackFactory;
-import org.apache.hyracks.storage.am.lsm.common.impls.LSMComponentId;
-import org.apache.hyracks.storage.common.IResource;
-
-public abstract class AbstractLSMIndexIOOperationCallbackFactory implements ILSMIOOperationCallbackFactory {
-
-    private static final long serialVersionUID = 1L;
-
-    protected ILSMComponentIdGeneratorFactory idGeneratorFactory;
-
-    protected transient INCServiceContext ncCtx;
-
-    protected transient IResource resource;
-
-    public AbstractLSMIndexIOOperationCallbackFactory(ILSMComponentIdGeneratorFactory idGeneratorFactory) {
-        this.idGeneratorFactory = idGeneratorFactory;
-    }
-
-    @Override
-    public void initialize(INCServiceContext ncCtx, IResource resource) {
-        this.ncCtx = ncCtx;
-        this.resource = resource;
-    }
-
-    protected ILSMComponentIdGenerator getComponentIdGenerator() throws HyracksDataException {
-        return idGeneratorFactory.getComponentIdGenerator(ncCtx, resource);
-    }
-
-    protected IIndexCheckpointManagerProvider getIndexCheckpointManagerProvider() {
-        return ((INcApplicationContext) ncCtx.getApplicationContext()).getIndexCheckpointManagerProvider();
-    }
-
-    private void readObjectNoData() throws ObjectStreamException {
-        idGeneratorFactory = new ILSMComponentIdGeneratorFactory() {
-            private static final long serialVersionUID = 1L;
-
-            @Override
-            public ILSMComponentIdGenerator getComponentIdGenerator(INCServiceContext serviceCtx, IResource resource) {
-                // used for backward compatibility
-                // if idGeneratorFactory is not set for legacy lsm indexes, we return a default
-                // component id generator which always generates the missing component id.
-                return new ILSMComponentIdGenerator() {
-                    @Override
-                    public void refresh() {
-                        // No op
-                    }
-
-                    @Override
-                    public ILSMComponentId getId() {
-                        return LSMComponentId.MISSING_COMPONENT_ID;
-                    }
-                };
-            }
-        };
-    }
-}
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/ioopcallbacks/LSMBTreeIOOperationCallback.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/ioopcallbacks/LSMBTreeIOOperationCallback.java
deleted file mode 100644
index db6c609..0000000
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/ioopcallbacks/LSMBTreeIOOperationCallback.java
+++ /dev/null
@@ -1,32 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-package org.apache.asterix.common.ioopcallbacks;
-
-import org.apache.asterix.common.storage.IIndexCheckpointManagerProvider;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponentIdGenerator;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndex;
-
-public class LSMBTreeIOOperationCallback extends AbstractLSMIOOperationCallback {
-
-    public LSMBTreeIOOperationCallback(ILSMIndex index, ILSMComponentIdGenerator idGenerator,
-            IIndexCheckpointManagerProvider checkpointManagerProvider) {
-        super(index, idGenerator, checkpointManagerProvider);
-    }
-}
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/ioopcallbacks/LSMBTreeIOOperationCallbackFactory.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/ioopcallbacks/LSMBTreeIOOperationCallbackFactory.java
deleted file mode 100644
index 97badb2..0000000
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/ioopcallbacks/LSMBTreeIOOperationCallbackFactory.java
+++ /dev/null
@@ -1,39 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-package org.apache.asterix.common.ioopcallbacks;
-
-import org.apache.hyracks.api.exceptions.HyracksDataException;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponentIdGeneratorFactory;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallback;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndex;
-
-public class LSMBTreeIOOperationCallbackFactory extends AbstractLSMIndexIOOperationCallbackFactory {
-
-    private static final long serialVersionUID = 1L;
-
-    public LSMBTreeIOOperationCallbackFactory(ILSMComponentIdGeneratorFactory idGeneratorFactory) {
-        super(idGeneratorFactory);
-    }
-
-    @Override
-    public ILSMIOOperationCallback createIoOpCallback(ILSMIndex index) throws HyracksDataException {
-        return new LSMBTreeIOOperationCallback(index, getComponentIdGenerator(), getIndexCheckpointManagerProvider());
-    }
-}
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/ioopcallbacks/LSMBTreeWithBuddyIOOperationCallback.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/ioopcallbacks/LSMBTreeWithBuddyIOOperationCallback.java
deleted file mode 100644
index da1446b..0000000
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/ioopcallbacks/LSMBTreeWithBuddyIOOperationCallback.java
+++ /dev/null
@@ -1,31 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.asterix.common.ioopcallbacks;
-
-import org.apache.asterix.common.storage.IIndexCheckpointManagerProvider;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponentIdGenerator;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndex;
-
-public class LSMBTreeWithBuddyIOOperationCallback extends AbstractLSMIOOperationCallback {
-
-    public LSMBTreeWithBuddyIOOperationCallback(ILSMIndex lsmIndex, ILSMComponentIdGenerator idGenerator,
-            IIndexCheckpointManagerProvider checkpointManagerProvider) {
-        super(lsmIndex, idGenerator, checkpointManagerProvider);
-    }
-}
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/ioopcallbacks/LSMIOOperationCallback.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/ioopcallbacks/LSMIOOperationCallback.java
new file mode 100644
index 0000000..50f5906
--- /dev/null
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/ioopcallbacks/LSMIOOperationCallback.java
@@ -0,0 +1,227 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.asterix.common.ioopcallbacks;
+
+import java.util.ArrayDeque;
+import java.util.Deque;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+
+import org.apache.asterix.common.context.DatasetInfo;
+import org.apache.asterix.common.storage.IIndexCheckpointManagerProvider;
+import org.apache.asterix.common.storage.ResourceReference;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.data.std.primitive.LongPointable;
+import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
+import org.apache.hyracks.storage.am.common.api.IMetadataPageManager;
+import org.apache.hyracks.storage.am.common.freepage.MutableArrayValueReference;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponent;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponentId;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMDiskComponent;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperation;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperation.LSMIOOperationStatus;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperation.LSMIOOperationType;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallback;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndex;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMMemoryComponent;
+import org.apache.hyracks.storage.am.lsm.common.impls.AbstractLSMIndexFileManager;
+import org.apache.hyracks.storage.am.lsm.common.impls.DiskComponentMetadata;
+import org.apache.hyracks.storage.am.lsm.common.impls.FlushOperation;
+import org.apache.hyracks.storage.am.lsm.common.util.ComponentUtils;
+import org.apache.hyracks.storage.am.lsm.common.util.LSMComponentIdUtils;
+
+// A single LSMIOOperationCallback per LSM index used to perform actions around Flush and Merge operations
+public class LSMIOOperationCallback implements ILSMIOOperationCallback {
+    public static final String KEY_FLUSH_LOG_LSN = "FlushLogLsn";
+    public static final String KEY_NEXT_COMPONENT_ID = "NextComponentId";
+    private static final String KEY_FIRST_LSN = "FirstLsn";
+    private static final MutableArrayValueReference KEY_METADATA_FLUSH_LOG_LSN =
+            new MutableArrayValueReference(KEY_FLUSH_LOG_LSN.getBytes());
+    public static final long INVALID_LSN = -1L;
+    private final ArrayBackedValueStorage buffer = new ArrayBackedValueStorage(Long.BYTES);
+    private final IIndexCheckpointManagerProvider indexCheckpointManagerProvider;
+    protected final DatasetInfo dsInfo;
+    protected final ILSMIndex lsmIndex;
+    private long firstLsnForCurrentMemoryComponent = 0L;
+    private long persistenceLsn = 0L;
+    private int pendingFlushes = 0;
+    private Deque<ILSMComponentId> componentIds = new ArrayDeque<>();
+
+    public LSMIOOperationCallback(DatasetInfo dsInfo, ILSMIndex lsmIndex, ILSMComponentId nextComponentId,
+            IIndexCheckpointManagerProvider indexCheckpointManagerProvider) {
+        this.dsInfo = dsInfo;
+        this.lsmIndex = lsmIndex;
+        this.indexCheckpointManagerProvider = indexCheckpointManagerProvider;
+        componentIds.add(nextComponentId);
+    }
+
+    @Override
+    public void beforeOperation(ILSMIOOperation operation) throws HyracksDataException {
+        // No Op
+    }
+
+    @Override
+    public void afterOperation(ILSMIOOperation operation) throws HyracksDataException {
+        if (operation.getStatus() == LSMIOOperationStatus.FAILURE) {
+            return;
+        }
+        if (operation.getIOOpertionType() == LSMIOOperationType.FLUSH) {
+            Map<String, Object> map = operation.getAccessor().getOpContext().getParameters();
+            putLSNIntoMetadata(operation.getNewComponent(), (Long) map.get(KEY_FLUSH_LOG_LSN));
+            putComponentIdIntoMetadata(operation.getNewComponent(),
+                    ((FlushOperation) operation).getFlushingComponent().getId());
+        } else if (operation.getIOOpertionType() == LSMIOOperationType.MERGE) {
+            List<ILSMDiskComponent> mergedComponents = operation.getAccessor().getOpContext().getComponentsToBeMerged();
+            putLSNIntoMetadata(operation.getNewComponent(), mergedComponents);
+            putComponentIdIntoMetadata(operation.getNewComponent(), mergedComponents);
+            LongPointable markerLsn =
+                    LongPointable.FACTORY.createPointable(ComponentUtils.getLong(mergedComponents.get(0).getMetadata(),
+                            ComponentUtils.MARKER_LSN_KEY, ComponentUtils.NOT_FOUND, buffer));
+            operation.getNewComponent().getMetadata().put(ComponentUtils.MARKER_LSN_KEY, markerLsn);
+        }
+    }
+
+    @Override
+    public void afterFinalize(ILSMIOOperation operation) throws HyracksDataException {
+        if (operation.getStatus() == LSMIOOperationStatus.FAILURE) {
+            return;
+        }
+        if (operation.getIOOpertionType() == LSMIOOperationType.FLUSH) {
+            Map<String, Object> map = operation.getAccessor().getOpContext().getParameters();
+            final Long lsn = (Long) map.get(KEY_FLUSH_LOG_LSN);
+            final Optional<String> componentFile =
+                    operation.getNewComponent().getLSMComponentPhysicalFiles().stream().findAny();
+            if (componentFile.isPresent()) {
+                final ResourceReference ref = ResourceReference.of(componentFile.get());
+                final String componentEndTime = AbstractLSMIndexFileManager.getComponentEndTime(ref.getName());
+                indexCheckpointManagerProvider.get(ref).flushed(componentEndTime, lsn);
+            }
+        }
+    }
+
+    private void putLSNIntoMetadata(ILSMDiskComponent newComponent, List<ILSMDiskComponent> oldComponents)
+            throws HyracksDataException {
+        putLSNIntoMetadata(newComponent, getComponentLSN(oldComponents));
+    }
+
+    private void putLSNIntoMetadata(ILSMDiskComponent newComponent, long lsn) throws HyracksDataException {
+        newComponent.getMetadata().put(KEY_METADATA_FLUSH_LOG_LSN, LongPointable.FACTORY.createPointable(lsn));
+    }
+
+    public static long getTreeIndexLSN(DiskComponentMetadata md) throws HyracksDataException {
+        LongPointable pointable = new LongPointable();
+        IMetadataPageManager metadataPageManager = md.getMetadataPageManager();
+        metadataPageManager.get(metadataPageManager.createMetadataFrame(), KEY_METADATA_FLUSH_LOG_LSN, pointable);
+        return pointable.getLength() == 0 ? INVALID_LSN : pointable.longValue();
+    }
+
+    private ILSMComponentId getMergedComponentId(List<? extends ILSMComponent> mergedComponents)
+            throws HyracksDataException {
+        if (mergedComponents.isEmpty()) {
+            return null;
+        }
+        return LSMComponentIdUtils.union(mergedComponents.get(0).getId(),
+                mergedComponents.get(mergedComponents.size() - 1).getId());
+    }
+
+    private void putComponentIdIntoMetadata(ILSMDiskComponent newComponent, List<ILSMDiskComponent> oldComponents)
+            throws HyracksDataException {
+        ILSMComponentId componentId = getMergedComponentId(oldComponents);
+        putComponentIdIntoMetadata(newComponent, componentId);
+    }
+
+    private void putComponentIdIntoMetadata(ILSMDiskComponent newComponent, ILSMComponentId componentId)
+            throws HyracksDataException {
+        LSMComponentIdUtils.persist(componentId, newComponent.getMetadata());
+    }
+
+    /**
+     * Used during the recovery process to force refresh the next component id
+     */
+    public void forceRefreshNextId(ILSMComponentId nextComponentId) {
+        componentIds.clear();
+        componentIds.add(nextComponentId);
+    }
+
+    public synchronized void setFirstLsnForCurrentMemoryComponent(long firstLsn) {
+        this.firstLsnForCurrentMemoryComponent = firstLsn;
+        if (pendingFlushes == 0) {
+            this.persistenceLsn = firstLsn;
+        }
+    }
+
+    public synchronized long getPersistenceLsn() {
+        return persistenceLsn;
+    }
+
+    public long getComponentLSN(List<ILSMDiskComponent> diskComponents) throws HyracksDataException {
+        if (diskComponents.isEmpty()) {
+            throw new IllegalArgumentException("Can't get LSN from an empty list of disk components");
+        }
+        // Get max LSN from the diskComponents. Implies a merge IO operation or Recovery operation.
+        long maxLSN = -1L;
+        for (ILSMDiskComponent c : diskComponents) {
+            DiskComponentMetadata md = c.getMetadata();
+            maxLSN = Math.max(getTreeIndexLSN(md), maxLSN);
+        }
+        return maxLSN;
+    }
+
+    @Override
+    public void recycled(ILSMMemoryComponent component) throws HyracksDataException {
+        component.resetId(componentIds.poll(), false);
+    }
+
+    @Override
+    public synchronized void scheduled(ILSMIOOperation operation) throws HyracksDataException {
+        dsInfo.declareActiveIOOperation();
+        if (operation.getIOOpertionType() == LSMIOOperationType.FLUSH) {
+            pendingFlushes++;
+            Map<String, Object> map = operation.getAccessor().getOpContext().getParameters();
+            Long flushLsn = (Long) map.get(KEY_FLUSH_LOG_LSN);
+            map.put(KEY_FIRST_LSN, firstLsnForCurrentMemoryComponent);
+            componentIds.add((ILSMComponentId) map.get(KEY_NEXT_COMPONENT_ID));
+            firstLsnForCurrentMemoryComponent = flushLsn; // Advance the first lsn for new component
+        }
+    }
+
+    @Override
+    public synchronized void completed(ILSMIOOperation operation) {
+        if (operation.getIOOpertionType() == LSMIOOperationType.FLUSH) {
+            pendingFlushes--;
+            if (operation.getStatus() == LSMIOOperationStatus.SUCCESS) {
+                Map<String, Object> map = operation.getAccessor().getOpContext().getParameters();
+                persistenceLsn =
+                        pendingFlushes == 0 ? firstLsnForCurrentMemoryComponent : (Long) map.get(KEY_FLUSH_LOG_LSN);
+            }
+        }
+        dsInfo.undeclareActiveIOOperation();
+    }
+
+    public synchronized boolean hasPendingFlush() {
+        return pendingFlushes > 0;
+    }
+
+    @Override
+    public void allocated(ILSMMemoryComponent component) throws HyracksDataException {
+        // No Op
+    }
+}
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/ioopcallbacks/LSMIndexIOOperationCallbackFactory.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/ioopcallbacks/LSMIndexIOOperationCallbackFactory.java
new file mode 100644
index 0000000..826ca50
--- /dev/null
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/ioopcallbacks/LSMIndexIOOperationCallbackFactory.java
@@ -0,0 +1,76 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.asterix.common.ioopcallbacks;
+
+import org.apache.asterix.common.api.IDatasetInfoProvider;
+import org.apache.asterix.common.api.ILSMComponentIdGeneratorFactory;
+import org.apache.asterix.common.api.INcApplicationContext;
+import org.apache.asterix.common.storage.IIndexCheckpointManagerProvider;
+import org.apache.hyracks.api.application.INCServiceContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponentIdGenerator;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallback;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallbackFactory;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndex;
+import org.apache.hyracks.storage.common.IResource;
+
+public class LSMIndexIOOperationCallbackFactory implements ILSMIOOperationCallbackFactory {
+
+    private static final long serialVersionUID = 1L;
+
+    private final ILSMComponentIdGeneratorFactory idGeneratorFactory;
+
+    protected final IDatasetInfoProvider datasetInfoProvider;
+
+    protected transient INCServiceContext ncCtx;
+
+    protected transient IResource resource;
+
+    public LSMIndexIOOperationCallbackFactory(ILSMComponentIdGeneratorFactory idGeneratorFactory,
+            IDatasetInfoProvider datasetInfoProvider) {
+        this.idGeneratorFactory = idGeneratorFactory;
+        this.datasetInfoProvider = datasetInfoProvider;
+    }
+
+    @Override
+    public void initialize(INCServiceContext ncCtx, IResource resource) {
+        this.ncCtx = ncCtx;
+        this.resource = resource;
+    }
+
+    protected ILSMComponentIdGenerator getComponentIdGenerator() throws HyracksDataException {
+        return idGeneratorFactory.getComponentIdGenerator(ncCtx, resource);
+    }
+
+    protected IIndexCheckpointManagerProvider getIndexCheckpointManagerProvider() {
+        return ((INcApplicationContext) ncCtx.getApplicationContext()).getIndexCheckpointManagerProvider();
+    }
+
+    @Override
+    public ILSMIOOperationCallback createIoOpCallback(ILSMIndex index) throws HyracksDataException {
+        return new LSMIOOperationCallback(datasetInfoProvider.getDatasetInfo(ncCtx), index,
+                getComponentIdGenerator().getId(), getIndexCheckpointManagerProvider());
+    }
+
+    @Override
+    public int getCurrentMemoryComponentIndex() throws HyracksDataException {
+        return idGeneratorFactory.getComponentIdGenerator(ncCtx, resource).getCurrentComponentIndex();
+    }
+}
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/ioopcallbacks/LSMInvertedIndexIOOperationCallback.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/ioopcallbacks/LSMInvertedIndexIOOperationCallback.java
deleted file mode 100644
index 3ba9bcd..0000000
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/ioopcallbacks/LSMInvertedIndexIOOperationCallback.java
+++ /dev/null
@@ -1,32 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-package org.apache.asterix.common.ioopcallbacks;
-
-import org.apache.asterix.common.storage.IIndexCheckpointManagerProvider;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponentIdGenerator;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndex;
-
-public class LSMInvertedIndexIOOperationCallback extends AbstractLSMIOOperationCallback {
-
-    public LSMInvertedIndexIOOperationCallback(ILSMIndex index, ILSMComponentIdGenerator idGenerator,
-            IIndexCheckpointManagerProvider checkpointManagerProvider) {
-        super(index, idGenerator, checkpointManagerProvider);
-    }
-}
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/ioopcallbacks/LSMInvertedIndexIOOperationCallbackFactory.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/ioopcallbacks/LSMInvertedIndexIOOperationCallbackFactory.java
deleted file mode 100644
index 766ef95..0000000
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/ioopcallbacks/LSMInvertedIndexIOOperationCallbackFactory.java
+++ /dev/null
@@ -1,40 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-package org.apache.asterix.common.ioopcallbacks;
-
-import org.apache.hyracks.api.exceptions.HyracksDataException;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponentIdGeneratorFactory;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallback;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndex;
-
-public class LSMInvertedIndexIOOperationCallbackFactory extends AbstractLSMIndexIOOperationCallbackFactory {
-
-    private static final long serialVersionUID = 1L;
-
-    public LSMInvertedIndexIOOperationCallbackFactory(ILSMComponentIdGeneratorFactory idGeneratorFactory) {
-        super(idGeneratorFactory);
-    }
-
-    @Override
-    public ILSMIOOperationCallback createIoOpCallback(ILSMIndex index) throws HyracksDataException {
-        return new LSMInvertedIndexIOOperationCallback(index, getComponentIdGenerator(),
-                getIndexCheckpointManagerProvider());
-    }
-}
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/ioopcallbacks/LSMRTreeIOOperationCallback.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/ioopcallbacks/LSMRTreeIOOperationCallback.java
deleted file mode 100644
index f3e80ec..0000000
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/ioopcallbacks/LSMRTreeIOOperationCallback.java
+++ /dev/null
@@ -1,32 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-package org.apache.asterix.common.ioopcallbacks;
-
-import org.apache.asterix.common.storage.IIndexCheckpointManagerProvider;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponentIdGenerator;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndex;
-
-public class LSMRTreeIOOperationCallback extends AbstractLSMIOOperationCallback {
-
-    public LSMRTreeIOOperationCallback(ILSMIndex index, ILSMComponentIdGenerator idGenerator,
-            IIndexCheckpointManagerProvider checkpointManagerProvodier) {
-        super(index, idGenerator, checkpointManagerProvodier);
-    }
-}
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/ioopcallbacks/LSMRTreeIOOperationCallbackFactory.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/ioopcallbacks/LSMRTreeIOOperationCallbackFactory.java
deleted file mode 100644
index 3a0afa8..0000000
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/ioopcallbacks/LSMRTreeIOOperationCallbackFactory.java
+++ /dev/null
@@ -1,39 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-package org.apache.asterix.common.ioopcallbacks;
-
-import org.apache.hyracks.api.exceptions.HyracksDataException;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponentIdGeneratorFactory;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallback;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndex;
-
-public class LSMRTreeIOOperationCallbackFactory extends AbstractLSMIndexIOOperationCallbackFactory {
-
-    private static final long serialVersionUID = 1L;
-
-    public LSMRTreeIOOperationCallbackFactory(ILSMComponentIdGeneratorFactory idGeneratorFactory) {
-        super(idGeneratorFactory);
-    }
-
-    @Override
-    public ILSMIOOperationCallback createIoOpCallback(ILSMIndex index) throws HyracksDataException {
-        return new LSMRTreeIOOperationCallback(index, getComponentIdGenerator(), getIndexCheckpointManagerProvider());
-    }
-}
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/utils/StorageConstants.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/utils/StorageConstants.java
index f6aa2c3..4bde9cf 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/utils/StorageConstants.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/utils/StorageConstants.java
@@ -41,7 +41,7 @@
     /**
      * The storage version of AsterixDB related artifacts (e.g. log files, checkpoint files, etc..).
      */
-    private static final int LOCAL_STORAGE_VERSION = 2;
+    private static final int LOCAL_STORAGE_VERSION = 3;
 
     /**
      * The storage version of AsterixDB stack.
diff --git a/asterixdb/asterix-common/src/test/java/org/apache/asterix/test/context/CorrelatedPrefixMergePolicyTest.java b/asterixdb/asterix-common/src/test/java/org/apache/asterix/test/context/CorrelatedPrefixMergePolicyTest.java
index befbeed..843cb21 100644
--- a/asterixdb/asterix-common/src/test/java/org/apache/asterix/test/context/CorrelatedPrefixMergePolicyTest.java
+++ b/asterixdb/asterix-common/src/test/java/org/apache/asterix/test/context/CorrelatedPrefixMergePolicyTest.java
@@ -34,7 +34,6 @@
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponent.ComponentState;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponentId;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMDiskComponent;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallback;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndex;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndexAccessor;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMMergePolicy;
@@ -220,7 +219,7 @@
 
             @Override
             public Void answer(InvocationOnMock invocation) throws Throwable {
-                List<ILSMDiskComponent> mergedComponents = invocation.getArgumentAt(1, List.class);
+                List<ILSMDiskComponent> mergedComponents = invocation.getArgumentAt(0, List.class);
                 mergedComponents.forEach(component -> {
                     try {
                         resultComponentIDs.add(component.getId());
@@ -230,8 +229,7 @@
                 });
                 return null;
             }
-        }).when(accessor).scheduleMerge(Mockito.any(ILSMIOOperationCallback.class),
-                Mockito.anyListOf(ILSMDiskComponent.class));
+        }).when(accessor).scheduleMerge(Mockito.anyListOf(ILSMDiskComponent.class));
 
         Mockito.when(index.createAccessor(Mockito.any(IIndexAccessParameters.class))).thenReturn(accessor);
         Mockito.when(index.isPrimaryIndex()).thenReturn(isPrimary);
diff --git a/asterixdb/asterix-common/src/test/java/org/apache/asterix/test/ioopcallbacks/AbstractLSMIOOperationCallbackTest.java b/asterixdb/asterix-common/src/test/java/org/apache/asterix/test/ioopcallbacks/AbstractLSMIOOperationCallbackTest.java
deleted file mode 100644
index c03af40..0000000
--- a/asterixdb/asterix-common/src/test/java/org/apache/asterix/test/ioopcallbacks/AbstractLSMIOOperationCallbackTest.java
+++ /dev/null
@@ -1,315 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-package org.apache.asterix.test.ioopcallbacks;
-
-import java.util.Collections;
-
-import org.apache.asterix.common.ioopcallbacks.AbstractLSMIOOperationCallback;
-import org.apache.asterix.common.ioopcallbacks.LSMBTreeIOOperationCallback;
-import org.apache.asterix.common.storage.IIndexCheckpointManager;
-import org.apache.asterix.common.storage.IIndexCheckpointManagerProvider;
-import org.apache.hyracks.api.exceptions.HyracksDataException;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponentId;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponentIdGenerator;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMDiskComponent;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperation.LSMIOOperationType;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndex;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndexOperationContext;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMMemoryComponent;
-import org.apache.hyracks.storage.am.lsm.common.impls.AbstractLSMMemoryComponent;
-import org.apache.hyracks.storage.am.lsm.common.impls.DiskComponentMetadata;
-import org.apache.hyracks.storage.am.lsm.common.impls.LSMComponentIdGenerator;
-import org.junit.Assert;
-import org.junit.Test;
-import org.mockito.ArgumentCaptor;
-import org.mockito.Mockito;
-
-import junit.framework.TestCase;
-
-public abstract class AbstractLSMIOOperationCallbackTest extends TestCase {
-
-    @Test
-    public void testNormalSequence() throws HyracksDataException {
-        ILSMIndex mockIndex = Mockito.mock(ILSMIndex.class);
-        Mockito.when(mockIndex.getNumberOfAllMemoryComponents()).thenReturn(2);
-        Mockito.when(mockIndex.getCurrentMemoryComponent()).thenReturn(Mockito.mock(AbstractLSMMemoryComponent.class));
-        LSMBTreeIOOperationCallback callback = new LSMBTreeIOOperationCallback(mockIndex, new LSMComponentIdGenerator(),
-                mockIndexCheckpointManagerProvider());
-        ILSMIndexOperationContext firstOpCtx = new TestLSMIndexOperationContext(mockIndex);
-
-        //request to flush first component
-        callback.updateLastLSN(1);
-        firstOpCtx.setIoOperationType(LSMIOOperationType.FLUSH);
-        callback.beforeOperation(firstOpCtx);
-
-        ILSMIndexOperationContext secondOpCtx = new TestLSMIndexOperationContext(mockIndex);
-        //request to flush second component
-        callback.updateLastLSN(2);
-        secondOpCtx.setIoOperationType(LSMIOOperationType.FLUSH);
-        callback.beforeOperation(secondOpCtx);
-
-        Assert.assertEquals(1, callback.getComponentLSN(Collections.emptyList()));
-        final ILSMDiskComponent diskComponent1 = mockDiskComponent();
-        firstOpCtx.setNewComponent(diskComponent1);
-        callback.afterOperation(firstOpCtx);
-        callback.afterFinalize(firstOpCtx);
-
-        Assert.assertEquals(2, callback.getComponentLSN(Collections.emptyList()));
-        final ILSMDiskComponent diskComponent2 = mockDiskComponent();
-        secondOpCtx.setNewComponent(diskComponent2);
-        callback.afterOperation(secondOpCtx);
-        callback.afterFinalize(secondOpCtx);
-    }
-
-    @Test
-    public void testOverWrittenLSN() throws HyracksDataException {
-        ILSMIndex mockIndex = Mockito.mock(ILSMIndex.class);
-        Mockito.when(mockIndex.getNumberOfAllMemoryComponents()).thenReturn(2);
-        Mockito.when(mockIndex.getCurrentMemoryComponent()).thenReturn(Mockito.mock(AbstractLSMMemoryComponent.class));
-        LSMBTreeIOOperationCallback callback = new LSMBTreeIOOperationCallback(mockIndex, new LSMComponentIdGenerator(),
-                mockIndexCheckpointManagerProvider());
-
-        //request to flush first component
-        ILSMIndexOperationContext firstOpCtx = new TestLSMIndexOperationContext(mockIndex);
-        callback.updateLastLSN(1);
-        firstOpCtx.setIoOperationType(LSMIOOperationType.FLUSH);
-        callback.beforeOperation(firstOpCtx);
-
-        //request to flush second component
-        ILSMIndexOperationContext secondOpCtx = new TestLSMIndexOperationContext(mockIndex);
-        callback.updateLastLSN(2);
-        secondOpCtx.setIoOperationType(LSMIOOperationType.FLUSH);
-        callback.beforeOperation(secondOpCtx);
-
-        //request to flush first component again
-        //this call should fail
-        callback.updateLastLSN(3);
-        //there is no corresponding beforeOperation, since the first component is being flush
-        //the scheduleFlush request would fail this time
-
-        Assert.assertEquals(1, callback.getComponentLSN(Collections.emptyList()));
-        final ILSMDiskComponent diskComponent1 = mockDiskComponent();
-        firstOpCtx.setNewComponent(diskComponent1);
-        callback.afterOperation(firstOpCtx);
-        callback.afterFinalize(firstOpCtx);
-        final ILSMDiskComponent diskComponent2 = mockDiskComponent();
-        secondOpCtx.setNewComponent(diskComponent2);
-        Assert.assertEquals(2, callback.getComponentLSN(Collections.emptyList()));
-        callback.afterOperation(secondOpCtx);
-        callback.afterFinalize(secondOpCtx);
-    }
-
-    @Test
-    public void testLostLSN() throws HyracksDataException {
-        ILSMIndex mockIndex = Mockito.mock(ILSMIndex.class);
-        Mockito.when(mockIndex.getNumberOfAllMemoryComponents()).thenReturn(2);
-        Mockito.when(mockIndex.getCurrentMemoryComponent()).thenReturn(Mockito.mock(AbstractLSMMemoryComponent.class));
-
-        LSMBTreeIOOperationCallback callback = new LSMBTreeIOOperationCallback(mockIndex, new LSMComponentIdGenerator(),
-                mockIndexCheckpointManagerProvider());
-        //request to flush first component
-        ILSMIndexOperationContext firstOpCtx = new TestLSMIndexOperationContext(mockIndex);
-        callback.updateLastLSN(1);
-        firstOpCtx.setIoOperationType(LSMIOOperationType.FLUSH);
-        callback.beforeOperation(firstOpCtx);
-
-        //request to flush second component
-        ILSMIndexOperationContext secondOpCtx = new TestLSMIndexOperationContext(mockIndex);
-        callback.updateLastLSN(2);
-        secondOpCtx.setIoOperationType(LSMIOOperationType.FLUSH);
-        callback.beforeOperation(secondOpCtx);
-
-        Assert.assertEquals(1, callback.getComponentLSN(Collections.emptyList()));
-
-        // the first flush is finished, but has not finalized yet (in codebase, these two calls
-        // are not synchronized)
-        firstOpCtx.setNewComponent(mockDiskComponent());
-        callback.afterOperation(firstOpCtx);
-
-        //request to flush first component again
-        callback.updateLastLSN(3);
-
-        // the first flush is finalized (it may be called after afterOperation for a while)
-        callback.afterFinalize(firstOpCtx);
-
-        // the second flush gets LSN 2
-        Assert.assertEquals(2, callback.getComponentLSN(Collections.emptyList()));
-        // the second flush is finished
-        secondOpCtx.setNewComponent(mockDiskComponent());
-        callback.afterOperation(secondOpCtx);
-        callback.afterFinalize(secondOpCtx);
-
-        // it should get new LSN 3
-        Assert.assertEquals(3, callback.getComponentLSN(Collections.emptyList()));
-    }
-
-    @Test
-    public void testAllocateComponentId() throws HyracksDataException {
-        ILSMComponentIdGenerator idGenerator = new LSMComponentIdGenerator();
-        ILSMIndex mockIndex = Mockito.mock(ILSMIndex.class);
-        Mockito.when(mockIndex.getNumberOfAllMemoryComponents()).thenReturn(2);
-        ILSMMemoryComponent mockComponent = Mockito.mock(AbstractLSMMemoryComponent.class);
-        Mockito.when(mockIndex.getCurrentMemoryComponent()).thenReturn(mockComponent);
-
-        LSMBTreeIOOperationCallback callback =
-                new LSMBTreeIOOperationCallback(mockIndex, idGenerator, mockIndexCheckpointManagerProvider());
-
-        ILSMComponentId initialId = idGenerator.getId();
-        // simulate a partition is flushed before allocated
-        idGenerator.refresh();
-        callback.updateLastLSN(0);
-
-        callback.allocated(mockComponent);
-        checkMemoryComponent(initialId, mockComponent);
-    }
-
-    @Test
-    public void testRecycleComponentId() throws HyracksDataException {
-        ILSMComponentIdGenerator idGenerator = new LSMComponentIdGenerator();
-        ILSMIndex mockIndex = Mockito.mock(ILSMIndex.class);
-        Mockito.when(mockIndex.getNumberOfAllMemoryComponents()).thenReturn(2);
-        ILSMMemoryComponent mockComponent = Mockito.mock(AbstractLSMMemoryComponent.class);
-        Mockito.when(mockIndex.getCurrentMemoryComponent()).thenReturn(mockComponent);
-        LSMBTreeIOOperationCallback callback =
-                new LSMBTreeIOOperationCallback(mockIndex, idGenerator, mockIndexCheckpointManagerProvider());
-
-        ILSMComponentId id = idGenerator.getId();
-        callback.allocated(mockComponent);
-        checkMemoryComponent(id, mockComponent);
-
-        Mockito.when(mockIndex.isMemoryComponentsAllocated()).thenReturn(true);
-        for (int i = 0; i < 100; i++) {
-            // schedule a flush
-            idGenerator.refresh();
-            ILSMComponentId expectedId = idGenerator.getId();
-            callback.updateLastLSN(0);
-            ILSMIndexOperationContext opCtx = new TestLSMIndexOperationContext(mockIndex);
-            opCtx.setIoOperationType(LSMIOOperationType.FLUSH);
-            callback.beforeOperation(opCtx);
-            callback.recycled(mockComponent, true);
-            opCtx.setNewComponent(mockDiskComponent());
-            callback.afterOperation(opCtx);
-            callback.afterFinalize(opCtx);
-            checkMemoryComponent(expectedId, mockComponent);
-        }
-    }
-
-    @Test
-    public void testRecycleWithoutSwitch() throws HyracksDataException {
-        ILSMComponentIdGenerator idGenerator = new LSMComponentIdGenerator();
-        ILSMIndex mockIndex = Mockito.mock(ILSMIndex.class);
-        Mockito.when(mockIndex.getNumberOfAllMemoryComponents()).thenReturn(2);
-        ILSMMemoryComponent mockComponent = Mockito.mock(AbstractLSMMemoryComponent.class);
-        Mockito.when(mockIndex.getCurrentMemoryComponent()).thenReturn(mockComponent);
-        LSMBTreeIOOperationCallback callback =
-                new LSMBTreeIOOperationCallback(mockIndex, idGenerator, mockIndexCheckpointManagerProvider());
-        ILSMComponentId id = idGenerator.getId();
-        callback.allocated(mockComponent);
-        checkMemoryComponent(id, mockComponent);
-        Mockito.when(mockIndex.isMemoryComponentsAllocated()).thenReturn(true);
-        for (int i = 0; i < 10; i++) {
-            idGenerator.refresh();
-            id = idGenerator.getId();
-            callback.updateLastLSN(0);
-            // Huh! There is no beforeOperation?
-            ILSMIndexOperationContext opCtx = new TestLSMIndexOperationContext(mockIndex);
-            opCtx.setIoOperationType(LSMIOOperationType.FLUSH);
-            callback.recycled(mockComponent, false);
-            callback.afterFinalize(opCtx);
-            checkMemoryComponent(id, mockComponent);
-        }
-    }
-
-    @Test
-    public void testConcurrentRecycleComponentId() throws HyracksDataException {
-        ILSMComponentIdGenerator idGenerator = new LSMComponentIdGenerator();
-        ILSMIndex mockIndex = Mockito.mock(ILSMIndex.class);
-        ILSMMemoryComponent mockComponent = Mockito.mock(AbstractLSMMemoryComponent.class);
-        Mockito.when(mockIndex.getCurrentMemoryComponent()).thenReturn(mockComponent);
-        Mockito.when(mockIndex.getNumberOfAllMemoryComponents()).thenReturn(2);
-        LSMBTreeIOOperationCallback callback =
-                new LSMBTreeIOOperationCallback(mockIndex, idGenerator, mockIndexCheckpointManagerProvider());
-
-        ILSMComponentId id = idGenerator.getId();
-        callback.allocated(mockComponent);
-        checkMemoryComponent(id, mockComponent);
-
-        Mockito.when(mockIndex.isMemoryComponentsAllocated()).thenReturn(true);
-
-        // schedule a flush
-        idGenerator.refresh();
-        ILSMComponentId expectedId = idGenerator.getId();
-
-        callback.updateLastLSN(0);
-        ILSMIndexOperationContext firstOpCtx = new TestLSMIndexOperationContext(mockIndex);
-        firstOpCtx.setIoOperationType(LSMIOOperationType.FLUSH);
-        callback.beforeOperation(firstOpCtx);
-        firstOpCtx.setNewComponent(mockDiskComponent());
-        callback.afterOperation(firstOpCtx);
-        callback.afterFinalize(firstOpCtx);
-
-        // another flush is to be scheduled before the component is recycled
-        idGenerator.refresh();
-        ILSMComponentId nextId = idGenerator.getId();
-
-        // recycle the component
-        callback.recycled(mockComponent, true);
-        checkMemoryComponent(expectedId, mockComponent);
-
-        // schedule the next flush
-        callback.updateLastLSN(0);
-        ILSMIndexOperationContext secondOpCtx = new TestLSMIndexOperationContext(mockIndex);
-        secondOpCtx.setIoOperationType(LSMIOOperationType.FLUSH);
-        callback.beforeOperation(secondOpCtx);
-        secondOpCtx.setNewComponent(mockDiskComponent());
-        callback.afterOperation(secondOpCtx);
-        callback.afterFinalize(secondOpCtx);
-        callback.recycled(mockComponent, true);
-        checkMemoryComponent(nextId, mockComponent);
-    }
-
-    private void checkMemoryComponent(ILSMComponentId expected, ILSMMemoryComponent memoryComponent)
-            throws HyracksDataException {
-        ArgumentCaptor<ILSMComponentId> idArgument = ArgumentCaptor.forClass(ILSMComponentId.class);
-        ArgumentCaptor<Boolean> forceArgument = ArgumentCaptor.forClass(Boolean.class);
-        Mockito.verify(memoryComponent).resetId(idArgument.capture(), forceArgument.capture());
-        assertEquals(expected, idArgument.getValue());
-        assertEquals(false, forceArgument.getValue().booleanValue());
-
-        Mockito.reset(memoryComponent);
-    }
-
-    private ILSMDiskComponent mockDiskComponent() {
-        ILSMDiskComponent component = Mockito.mock(ILSMDiskComponent.class);
-        Mockito.when(component.getMetadata()).thenReturn(Mockito.mock(DiskComponentMetadata.class));
-        return component;
-    }
-
-    protected IIndexCheckpointManagerProvider mockIndexCheckpointManagerProvider() throws HyracksDataException {
-        IIndexCheckpointManagerProvider indexCheckpointManagerProvider =
-                Mockito.mock(IIndexCheckpointManagerProvider.class);
-        IIndexCheckpointManager indexCheckpointManager = Mockito.mock(IIndexCheckpointManager.class);
-        Mockito.doNothing().when(indexCheckpointManager).flushed(Mockito.any(), Mockito.anyLong());
-        Mockito.doReturn(indexCheckpointManager).when(indexCheckpointManagerProvider).get(Mockito.any());
-        return indexCheckpointManagerProvider;
-    }
-
-    protected abstract AbstractLSMIOOperationCallback getIoCallback() throws HyracksDataException;
-}
diff --git a/asterixdb/asterix-common/src/test/java/org/apache/asterix/test/ioopcallbacks/LSMBTreeIOOperationCallbackTest.java b/asterixdb/asterix-common/src/test/java/org/apache/asterix/test/ioopcallbacks/LSMBTreeIOOperationCallbackTest.java
deleted file mode 100644
index a4bc399..0000000
--- a/asterixdb/asterix-common/src/test/java/org/apache/asterix/test/ioopcallbacks/LSMBTreeIOOperationCallbackTest.java
+++ /dev/null
@@ -1,39 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-package org.apache.asterix.test.ioopcallbacks;
-
-import org.apache.asterix.common.ioopcallbacks.AbstractLSMIOOperationCallback;
-import org.apache.asterix.common.ioopcallbacks.LSMBTreeIOOperationCallback;
-import org.apache.hyracks.api.exceptions.HyracksDataException;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndex;
-import org.apache.hyracks.storage.am.lsm.common.impls.LSMComponentIdGenerator;
-import org.mockito.Mockito;
-
-public class LSMBTreeIOOperationCallbackTest extends AbstractLSMIOOperationCallbackTest {
-
-    @Override
-    protected AbstractLSMIOOperationCallback getIoCallback() throws HyracksDataException {
-        ILSMIndex mockIndex = Mockito.mock(ILSMIndex.class);
-        Mockito.when(mockIndex.getNumberOfAllMemoryComponents()).thenReturn(2);
-        return new LSMBTreeIOOperationCallback(mockIndex, new LSMComponentIdGenerator(),
-                mockIndexCheckpointManagerProvider());
-    }
-
-}
diff --git a/asterixdb/asterix-common/src/test/java/org/apache/asterix/test/ioopcallbacks/LSMBTreeWithBuddyIOOperationCallbackTest.java b/asterixdb/asterix-common/src/test/java/org/apache/asterix/test/ioopcallbacks/LSMBTreeWithBuddyIOOperationCallbackTest.java
deleted file mode 100644
index 5f37c78..0000000
--- a/asterixdb/asterix-common/src/test/java/org/apache/asterix/test/ioopcallbacks/LSMBTreeWithBuddyIOOperationCallbackTest.java
+++ /dev/null
@@ -1,39 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-package org.apache.asterix.test.ioopcallbacks;
-
-import org.apache.asterix.common.ioopcallbacks.AbstractLSMIOOperationCallback;
-import org.apache.asterix.common.ioopcallbacks.LSMBTreeWithBuddyIOOperationCallback;
-import org.apache.hyracks.api.exceptions.HyracksDataException;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndex;
-import org.apache.hyracks.storage.am.lsm.common.impls.LSMComponentIdGenerator;
-import org.mockito.Mockito;
-
-public class LSMBTreeWithBuddyIOOperationCallbackTest extends AbstractLSMIOOperationCallbackTest {
-
-    @Override
-    protected AbstractLSMIOOperationCallback getIoCallback() throws HyracksDataException {
-        ILSMIndex mockIndex = Mockito.mock(ILSMIndex.class);
-        Mockito.when(mockIndex.getNumberOfAllMemoryComponents()).thenReturn(2);
-        return new LSMBTreeWithBuddyIOOperationCallback(mockIndex, new LSMComponentIdGenerator(),
-                mockIndexCheckpointManagerProvider());
-    }
-
-}
diff --git a/asterixdb/asterix-common/src/test/java/org/apache/asterix/test/ioopcallbacks/LSMIOOperationCallbackTest.java b/asterixdb/asterix-common/src/test/java/org/apache/asterix/test/ioopcallbacks/LSMIOOperationCallbackTest.java
new file mode 100644
index 0000000..7af7b6e
--- /dev/null
+++ b/asterixdb/asterix-common/src/test/java/org/apache/asterix/test/ioopcallbacks/LSMIOOperationCallbackTest.java
@@ -0,0 +1,217 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.asterix.test.ioopcallbacks;
+
+import java.util.HashMap;
+import java.util.Map;
+
+import org.apache.asterix.common.context.DatasetInfo;
+import org.apache.asterix.common.ioopcallbacks.LSMIOOperationCallback;
+import org.apache.asterix.common.storage.IIndexCheckpointManager;
+import org.apache.asterix.common.storage.IIndexCheckpointManagerProvider;
+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.storage.am.lsm.common.api.ILSMComponentId;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponentIdGenerator;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMDiskComponent;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndex;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndexAccessor;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMMemoryComponent;
+import org.apache.hyracks.storage.am.lsm.common.impls.AbstractLSMMemoryComponent;
+import org.apache.hyracks.storage.am.lsm.common.impls.DiskComponentMetadata;
+import org.apache.hyracks.storage.am.lsm.common.impls.FlushOperation;
+import org.apache.hyracks.storage.am.lsm.common.impls.LSMComponentFileReferences;
+import org.apache.hyracks.storage.am.lsm.common.impls.LSMComponentId;
+import org.apache.hyracks.storage.am.lsm.common.impls.LSMComponentIdGenerator;
+import org.junit.Assert;
+import org.junit.Test;
+import org.mockito.ArgumentCaptor;
+import org.mockito.Mockito;
+
+import junit.framework.TestCase;
+
+public class LSMIOOperationCallbackTest extends TestCase {
+    /*
+     * The normal sequence of calls:
+     * 1. refresh id generator
+     * 2. flushLsn
+     * 3. created
+     * 4. before
+     * 5. after
+     * 6. finalize
+     * 7. destroy
+     */
+
+    @Test
+    public void testNormalSequence() throws HyracksDataException {
+        int numMemoryComponents = 2;
+
+        ILSMIndex mockIndex = Mockito.mock(ILSMIndex.class);
+        String indexId = "mockIndexId";
+        Mockito.when(mockIndex.getNumberOfAllMemoryComponents()).thenReturn(numMemoryComponents);
+        Mockito.when(mockIndex.getCurrentMemoryComponent()).thenReturn(Mockito.mock(AbstractLSMMemoryComponent.class));
+        DatasetInfo dsInfo = new DatasetInfo(101, null);
+        LSMComponentIdGenerator idGenerator = new LSMComponentIdGenerator(numMemoryComponents);
+        LSMIOOperationCallback callback = new LSMIOOperationCallback(dsInfo, mockIndex, idGenerator.getId(),
+                mockIndexCheckpointManagerProvider());
+        //Flush first
+        idGenerator.refresh();
+        long flushLsn = 1L;
+        ILSMComponentId nextComponentId = idGenerator.getId();
+        Map<String, Object> flushMap = new HashMap<>();
+        flushMap.put(LSMIOOperationCallback.KEY_FLUSH_LOG_LSN, flushLsn);
+        flushMap.put(LSMIOOperationCallback.KEY_NEXT_COMPONENT_ID, nextComponentId);
+        ILSMIndexAccessor firstAccessor = new TestLSMIndexAccessor(new TestLSMIndexOperationContext(mockIndex));
+        firstAccessor.getOpContext().setParameters(flushMap);
+        FileReference firstTarget = new FileReference(Mockito.mock(IODeviceHandle.class), "path");
+        LSMComponentFileReferences firstFiles = new LSMComponentFileReferences(firstTarget, firstTarget, firstTarget);
+        FlushOperation firstFlush = new TestFlushOperation(firstAccessor, firstTarget, callback, indexId, firstFiles,
+                new LSMComponentId(0, 0));
+        callback.scheduled(firstFlush);
+        callback.beforeOperation(firstFlush);
+
+        //Flush second
+        idGenerator.refresh();
+        flushLsn = 2L;
+        nextComponentId = idGenerator.getId();
+        flushMap = new HashMap<>();
+        flushMap.put(LSMIOOperationCallback.KEY_FLUSH_LOG_LSN, flushLsn);
+        flushMap.put(LSMIOOperationCallback.KEY_NEXT_COMPONENT_ID, nextComponentId);
+        ILSMIndexAccessor secondAccessor = new TestLSMIndexAccessor(new TestLSMIndexOperationContext(mockIndex));
+        secondAccessor.getOpContext().setParameters(flushMap);
+        FileReference secondTarget = new FileReference(Mockito.mock(IODeviceHandle.class), "path");
+        LSMComponentFileReferences secondFiles =
+                new LSMComponentFileReferences(secondTarget, secondTarget, secondTarget);
+        FlushOperation secondFlush = new TestFlushOperation(secondAccessor, secondTarget, callback, indexId,
+                secondFiles, new LSMComponentId(1, 1));
+        callback.scheduled(secondFlush);
+        callback.beforeOperation(secondFlush);
+
+        Map<String, Object> firstFlushMap = firstFlush.getAccessor().getOpContext().getParameters();
+        long firstFlushLogLsn = (Long) firstFlushMap.get(LSMIOOperationCallback.KEY_FLUSH_LOG_LSN);
+        Assert.assertEquals(1, firstFlushLogLsn);
+        final ILSMDiskComponent diskComponent1 = mockDiskComponent();
+        firstFlush.setNewComponent(diskComponent1);
+        callback.afterOperation(firstFlush);
+        callback.afterFinalize(firstFlush);
+        callback.completed(firstFlush);
+
+        Map<String, Object> secondFlushMap = secondFlush.getAccessor().getOpContext().getParameters();
+        long secondFlushLogLsn = (Long) secondFlushMap.get(LSMIOOperationCallback.KEY_FLUSH_LOG_LSN);
+        Assert.assertEquals(2, secondFlushLogLsn);
+        final ILSMDiskComponent diskComponent2 = mockDiskComponent();
+        secondFlush.setNewComponent(diskComponent2);
+        callback.afterOperation(secondFlush);
+        callback.afterFinalize(secondFlush);
+        callback.completed(secondFlush);
+    }
+
+    @Test
+    public void testAllocateComponentId() throws HyracksDataException {
+        int numMemoryComponents = 2;
+        DatasetInfo dsInfo = new DatasetInfo(101, null);
+        ILSMComponentIdGenerator idGenerator = new LSMComponentIdGenerator(numMemoryComponents);
+        ILSMIndex mockIndex = Mockito.mock(ILSMIndex.class);
+        Mockito.when(mockIndex.getNumberOfAllMemoryComponents()).thenReturn(numMemoryComponents);
+        ILSMMemoryComponent mockComponent = Mockito.mock(AbstractLSMMemoryComponent.class);
+        Mockito.when(mockIndex.getCurrentMemoryComponent()).thenReturn(mockComponent);
+        LSMIOOperationCallback callback = new LSMIOOperationCallback(dsInfo, mockIndex, idGenerator.getId(),
+                mockIndexCheckpointManagerProvider());
+        ILSMComponentId initialId = idGenerator.getId();
+        // simulate a partition is flushed before allocated
+        idGenerator.refresh();
+        long flushLsn = 1L;
+        ILSMComponentId nextComponentId = idGenerator.getId();
+        Map<String, Object> flushMap = new HashMap<>();
+        flushMap.put(LSMIOOperationCallback.KEY_FLUSH_LOG_LSN, flushLsn);
+        flushMap.put(LSMIOOperationCallback.KEY_NEXT_COMPONENT_ID, nextComponentId);
+        callback.allocated(mockComponent);
+        callback.recycled(mockComponent);
+        checkMemoryComponent(initialId, mockComponent);
+    }
+
+    @Test
+    public void testRecycleComponentId() throws HyracksDataException {
+        int numMemoryComponents = 2;
+        DatasetInfo dsInfo = new DatasetInfo(101, null);
+        ILSMComponentIdGenerator idGenerator = new LSMComponentIdGenerator(numMemoryComponents);
+        ILSMIndex mockIndex = Mockito.mock(ILSMIndex.class);
+        Mockito.when(mockIndex.getNumberOfAllMemoryComponents()).thenReturn(numMemoryComponents);
+        ILSMMemoryComponent mockComponent = Mockito.mock(AbstractLSMMemoryComponent.class);
+        Mockito.when(mockIndex.getCurrentMemoryComponent()).thenReturn(mockComponent);
+        LSMIOOperationCallback callback = new LSMIOOperationCallback(dsInfo, mockIndex, idGenerator.getId(),
+                mockIndexCheckpointManagerProvider());
+        String indexId = "mockIndexId";
+        ILSMComponentId id = idGenerator.getId();
+        callback.recycled(mockComponent);
+        checkMemoryComponent(id, mockComponent);
+
+        Mockito.when(mockIndex.isMemoryComponentsAllocated()).thenReturn(true);
+        for (int i = 0; i < 100; i++) {
+            // schedule a flush
+            idGenerator.refresh();
+            ILSMComponentId expectedId = idGenerator.getId();
+            long flushLsn = 0L;
+            Map<String, Object> flushMap = new HashMap<>();
+            flushMap.put(LSMIOOperationCallback.KEY_FLUSH_LOG_LSN, flushLsn);
+            flushMap.put(LSMIOOperationCallback.KEY_NEXT_COMPONENT_ID, expectedId);
+            ILSMIndexAccessor accessor = new TestLSMIndexAccessor(new TestLSMIndexOperationContext(mockIndex));
+            accessor.getOpContext().setParameters(flushMap);
+            FileReference target = new FileReference(Mockito.mock(IODeviceHandle.class), "path");
+            LSMComponentFileReferences files = new LSMComponentFileReferences(target, target, target);
+            FlushOperation flush =
+                    new TestFlushOperation(accessor, target, callback, indexId, files, new LSMComponentId(0, 0));
+            callback.scheduled(flush);
+            callback.beforeOperation(flush);
+            callback.recycled(mockComponent);
+            flush.setNewComponent(mockDiskComponent());
+            callback.afterOperation(flush);
+            callback.afterFinalize(flush);
+            callback.completed(flush);
+            checkMemoryComponent(expectedId, mockComponent);
+        }
+    }
+
+    private void checkMemoryComponent(ILSMComponentId expected, ILSMMemoryComponent memoryComponent)
+            throws HyracksDataException {
+        ArgumentCaptor<ILSMComponentId> idArgument = ArgumentCaptor.forClass(ILSMComponentId.class);
+        ArgumentCaptor<Boolean> forceArgument = ArgumentCaptor.forClass(Boolean.class);
+        Mockito.verify(memoryComponent).resetId(idArgument.capture(), forceArgument.capture());
+        assertEquals(expected, idArgument.getValue());
+        assertEquals(false, forceArgument.getValue().booleanValue());
+        Mockito.reset(memoryComponent);
+    }
+
+    private ILSMDiskComponent mockDiskComponent() {
+        ILSMDiskComponent component = Mockito.mock(ILSMDiskComponent.class);
+        Mockito.when(component.getMetadata()).thenReturn(Mockito.mock(DiskComponentMetadata.class));
+        return component;
+    }
+
+    protected IIndexCheckpointManagerProvider mockIndexCheckpointManagerProvider() throws HyracksDataException {
+        IIndexCheckpointManagerProvider indexCheckpointManagerProvider =
+                Mockito.mock(IIndexCheckpointManagerProvider.class);
+        IIndexCheckpointManager indexCheckpointManager = Mockito.mock(IIndexCheckpointManager.class);
+        Mockito.doNothing().when(indexCheckpointManager).flushed(Mockito.any(), Mockito.anyLong());
+        Mockito.doReturn(indexCheckpointManager).when(indexCheckpointManagerProvider).get(Mockito.any());
+        return indexCheckpointManagerProvider;
+    }
+}
diff --git a/asterixdb/asterix-common/src/test/java/org/apache/asterix/test/ioopcallbacks/LSMInvertedIndexIOOperationCallbackTest.java b/asterixdb/asterix-common/src/test/java/org/apache/asterix/test/ioopcallbacks/LSMInvertedIndexIOOperationCallbackTest.java
deleted file mode 100644
index 343bc59..0000000
--- a/asterixdb/asterix-common/src/test/java/org/apache/asterix/test/ioopcallbacks/LSMInvertedIndexIOOperationCallbackTest.java
+++ /dev/null
@@ -1,39 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-package org.apache.asterix.test.ioopcallbacks;
-
-import org.apache.asterix.common.ioopcallbacks.AbstractLSMIOOperationCallback;
-import org.apache.asterix.common.ioopcallbacks.LSMInvertedIndexIOOperationCallback;
-import org.apache.hyracks.api.exceptions.HyracksDataException;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndex;
-import org.apache.hyracks.storage.am.lsm.common.impls.LSMComponentIdGenerator;
-import org.mockito.Mockito;
-
-public class LSMInvertedIndexIOOperationCallbackTest extends AbstractLSMIOOperationCallbackTest {
-
-    @Override
-    protected AbstractLSMIOOperationCallback getIoCallback() throws HyracksDataException {
-        ILSMIndex mockIndex = Mockito.mock(ILSMIndex.class);
-        Mockito.when(mockIndex.getNumberOfAllMemoryComponents()).thenReturn(2);
-        return new LSMInvertedIndexIOOperationCallback(mockIndex, new LSMComponentIdGenerator(),
-                mockIndexCheckpointManagerProvider());
-    }
-
-}
diff --git a/asterixdb/asterix-common/src/test/java/org/apache/asterix/test/ioopcallbacks/LSMRTreeIOOperationCallbackTest.java b/asterixdb/asterix-common/src/test/java/org/apache/asterix/test/ioopcallbacks/LSMRTreeIOOperationCallbackTest.java
deleted file mode 100644
index 10d95d8..0000000
--- a/asterixdb/asterix-common/src/test/java/org/apache/asterix/test/ioopcallbacks/LSMRTreeIOOperationCallbackTest.java
+++ /dev/null
@@ -1,39 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-package org.apache.asterix.test.ioopcallbacks;
-
-import org.apache.asterix.common.ioopcallbacks.AbstractLSMIOOperationCallback;
-import org.apache.asterix.common.ioopcallbacks.LSMRTreeIOOperationCallback;
-import org.apache.hyracks.api.exceptions.HyracksDataException;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndex;
-import org.apache.hyracks.storage.am.lsm.common.impls.LSMComponentIdGenerator;
-import org.mockito.Mockito;
-
-public class LSMRTreeIOOperationCallbackTest extends AbstractLSMIOOperationCallbackTest {
-
-    @Override
-    protected AbstractLSMIOOperationCallback getIoCallback() throws HyracksDataException {
-        ILSMIndex mockIndex = Mockito.mock(ILSMIndex.class);
-        Mockito.when(mockIndex.getNumberOfAllMemoryComponents()).thenReturn(2);
-        return new LSMRTreeIOOperationCallback(mockIndex, new LSMComponentIdGenerator(),
-                mockIndexCheckpointManagerProvider());
-    }
-
-}
diff --git a/asterixdb/asterix-common/src/test/java/org/apache/asterix/test/ioopcallbacks/TestFlushOperation.java b/asterixdb/asterix-common/src/test/java/org/apache/asterix/test/ioopcallbacks/TestFlushOperation.java
new file mode 100644
index 0000000..8a02aa3
--- /dev/null
+++ b/asterixdb/asterix-common/src/test/java/org/apache/asterix/test/ioopcallbacks/TestFlushOperation.java
@@ -0,0 +1,56 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.test.ioopcallbacks;
+
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.io.FileReference;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponent;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallback;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndexAccessor;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMMemoryComponent;
+import org.apache.hyracks.storage.am.lsm.common.impls.FlushOperation;
+import org.apache.hyracks.storage.am.lsm.common.impls.LSMComponentFileReferences;
+import org.apache.hyracks.storage.am.lsm.common.impls.LSMComponentId;
+import org.mockito.Mockito;
+
+public class TestFlushOperation extends FlushOperation {
+
+    private final LSMComponentFileReferences files;
+    private final ILSMMemoryComponent flushingComponent;
+
+    public TestFlushOperation(ILSMIndexAccessor accessor, FileReference target, ILSMIOOperationCallback callback,
+            String indexIdentifier, LSMComponentFileReferences files, LSMComponentId componentId)
+            throws HyracksDataException {
+        super(accessor, target, callback, indexIdentifier);
+        this.files = files;
+        flushingComponent = accessor.getOpContext().getIndex().getCurrentMemoryComponent();
+        Mockito.when(flushingComponent.getId()).thenReturn(componentId);
+    }
+
+    @Override
+    public LSMComponentFileReferences getComponentFiles() {
+        return files;
+    }
+
+    @Override
+    public ILSMComponent getFlushingComponent() {
+        return flushingComponent;
+    }
+
+}
diff --git a/asterixdb/asterix-common/src/test/java/org/apache/asterix/test/ioopcallbacks/TestLSMIndexAccessor.java b/asterixdb/asterix-common/src/test/java/org/apache/asterix/test/ioopcallbacks/TestLSMIndexAccessor.java
new file mode 100644
index 0000000..9ac143c
--- /dev/null
+++ b/asterixdb/asterix-common/src/test/java/org/apache/asterix/test/ioopcallbacks/TestLSMIndexAccessor.java
@@ -0,0 +1,183 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.test.ioopcallbacks;
+
+import java.util.List;
+import java.util.function.Predicate;
+
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.data.std.api.IValueReference;
+import org.apache.hyracks.dataflow.common.data.accessors.ITupleReference;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponent;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMDiskComponent;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperation;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndexAccessor;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndexOperationContext;
+import org.apache.hyracks.storage.am.lsm.common.api.LSMOperationType;
+import org.apache.hyracks.storage.common.IIndexCursor;
+import org.apache.hyracks.storage.common.ISearchPredicate;
+
+public class TestLSMIndexAccessor implements ILSMIndexAccessor {
+
+    private final ILSMIndexOperationContext opCtx;
+
+    public TestLSMIndexAccessor(ILSMIndexOperationContext opCtx) {
+        this.opCtx = opCtx;
+    }
+
+    @Override
+    public void insert(ITupleReference tuple) throws HyracksDataException {
+        throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public void update(ITupleReference tuple) throws HyracksDataException {
+        throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public void delete(ITupleReference tuple) throws HyracksDataException {
+        throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public void upsert(ITupleReference tuple) throws HyracksDataException {
+        throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public IIndexCursor createSearchCursor(boolean exclusive) throws HyracksDataException {
+        throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public void search(IIndexCursor cursor, ISearchPredicate searchPred) throws HyracksDataException {
+        throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public void destroy() throws HyracksDataException {
+    }
+
+    @Override
+    public ILSMIndexOperationContext getOpContext() {
+        return opCtx;
+    }
+
+    @Override
+    public ILSMIOOperation scheduleFlush() throws HyracksDataException {
+        throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public ILSMIOOperation scheduleMerge(List<ILSMDiskComponent> components) throws HyracksDataException {
+        throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public ILSMIOOperation scheduleFullMerge() throws HyracksDataException {
+        throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public void physicalDelete(ITupleReference tuple) throws HyracksDataException {
+        throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public boolean tryInsert(ITupleReference tuple) throws HyracksDataException {
+        throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public boolean tryDelete(ITupleReference tuple) throws HyracksDataException {
+        throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public boolean tryUpdate(ITupleReference tuple) throws HyracksDataException {
+        throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public boolean tryUpsert(ITupleReference tuple) throws HyracksDataException {
+        throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public void forcePhysicalDelete(ITupleReference tuple) throws HyracksDataException {
+        throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public void forceInsert(ITupleReference tuple) throws HyracksDataException {
+        throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public void forceDelete(ITupleReference tuple) throws HyracksDataException {
+        throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public void forceUpsert(ITupleReference tuple) throws HyracksDataException {
+        throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public void scheduleReplication(List<ILSMDiskComponent> diskComponents, boolean bulkload, LSMOperationType opType)
+            throws HyracksDataException {
+        throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public void flush(ILSMIOOperation operation) throws HyracksDataException {
+        throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public void merge(ILSMIOOperation operation) throws HyracksDataException {
+        throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public void updateMeta(IValueReference key, IValueReference value) throws HyracksDataException {
+        throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public void forceUpdateMeta(IValueReference key, IValueReference value) throws HyracksDataException {
+        throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public void scanDiskComponents(IIndexCursor cursor) throws HyracksDataException {
+        throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public void deleteComponents(Predicate<ILSMComponent> predicate) throws HyracksDataException {
+        throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public void updateFilter(ITupleReference tuple) throws HyracksDataException {
+        throw new UnsupportedOperationException();
+    }
+}
diff --git a/asterixdb/asterix-common/src/test/java/org/apache/asterix/test/ioopcallbacks/TestLSMIndexOperationContext.java b/asterixdb/asterix-common/src/test/java/org/apache/asterix/test/ioopcallbacks/TestLSMIndexOperationContext.java
index 9b749fa..4306dda 100644
--- a/asterixdb/asterix-common/src/test/java/org/apache/asterix/test/ioopcallbacks/TestLSMIndexOperationContext.java
+++ b/asterixdb/asterix-common/src/test/java/org/apache/asterix/test/ioopcallbacks/TestLSMIndexOperationContext.java
@@ -20,6 +20,7 @@
 
 import java.util.ArrayList;
 import java.util.List;
+import java.util.Map;
 
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.storage.am.common.api.IExtendedModificationOperationCallback;
@@ -28,10 +29,9 @@
 import org.apache.hyracks.storage.am.common.tuples.PermutingTupleReference;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponent;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMDiskComponent;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperation.LSMIOOperationType;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperation;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndex;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndexOperationContext;
-import org.apache.hyracks.storage.common.IModificationOperationCallback;
 import org.apache.hyracks.storage.common.ISearchOperationCallback;
 import org.apache.hyracks.storage.common.ISearchPredicate;
 import org.apache.hyracks.storage.common.MultiComparator;
@@ -44,8 +44,8 @@
     private final List<ILSMDiskComponent> componentsToBeReplicated = new ArrayList<>();
     private boolean isAccessingComponents;
     private IndexOperation op;
-    private LSMIOOperationType ioOperationType;
-    private ILSMDiskComponent newComponent;
+    private ILSMIOOperation ioOperation;
+    private Map<String, Object> map;
     private boolean filterSkip = false;
     private boolean isRecovery = false;
 
@@ -180,22 +180,23 @@
     }
 
     @Override
-    public LSMIOOperationType getIoOperationType() {
-        return ioOperationType;
+    public ILSMIOOperation getIoOperation() {
+        return ioOperation;
     }
 
     @Override
-    public void setIoOperationType(LSMIOOperationType ioOpType) {
-        this.ioOperationType = ioOpType;
+    public void setIoOperation(ILSMIOOperation ioOperation) {
+        this.ioOperation = ioOperation;
     }
 
     @Override
-    public ILSMDiskComponent getNewComponent() {
-        return newComponent;
+    public void setParameters(Map<String, Object> map) {
+        this.map = map;
     }
 
     @Override
-    public void setNewComponent(ILSMDiskComponent component) {
-        this.newComponent = component;
+    public Map<String, Object> getParameters() {
+        return map;
     }
+
 }
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/api/IActiveEntityController.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/api/IActiveEntityController.java
index 7103fd8..baface2 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/api/IActiveEntityController.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/api/IActiveEntityController.java
@@ -82,7 +82,7 @@
     void add(Dataset dataset) throws HyracksDataException;
 
     /**
-     * Remove dataset to the list of associated datasets
+     * Remove dataset from the list of associated datasets
      *
      * @param dataset
      *            the dataset to add
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/bootstrap/MetadataBootstrap.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/bootstrap/MetadataBootstrap.java
index d110644..95526f4 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/bootstrap/MetadataBootstrap.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/bootstrap/MetadataBootstrap.java
@@ -24,17 +24,19 @@
 import java.util.HashMap;
 import java.util.List;
 
+import org.apache.asterix.common.api.ILSMComponentIdGeneratorFactory;
 import org.apache.asterix.common.api.INcApplicationContext;
 import org.apache.asterix.common.config.DatasetConfig.DatasetType;
 import org.apache.asterix.common.config.GlobalConfig;
 import org.apache.asterix.common.config.MetadataProperties;
 import org.apache.asterix.common.context.AsterixVirtualBufferCacheProvider;
 import org.apache.asterix.common.context.CorrelatedPrefixMergePolicyFactory;
+import org.apache.asterix.common.context.DatasetInfoProvider;
 import org.apache.asterix.common.context.DatasetLSMComponentIdGeneratorFactory;
 import org.apache.asterix.common.context.IStorageComponentProvider;
 import org.apache.asterix.common.exceptions.ACIDException;
 import org.apache.asterix.common.exceptions.MetadataException;
-import org.apache.asterix.common.ioopcallbacks.LSMBTreeIOOperationCallbackFactory;
+import org.apache.asterix.common.ioopcallbacks.LSMIndexIOOperationCallbackFactory;
 import org.apache.asterix.common.utils.StoragePathUtil;
 import org.apache.asterix.external.adapter.factory.GenericAdapterFactory;
 import org.apache.asterix.external.api.IAdapterFactory;
@@ -77,7 +79,6 @@
 import org.apache.hyracks.storage.am.common.build.IndexBuilder;
 import org.apache.hyracks.storage.am.common.dataflow.IndexDataflowHelper;
 import org.apache.hyracks.storage.am.lsm.btree.dataflow.LSMBTreeLocalResourceFactory;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponentIdGeneratorFactory;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallbackFactory;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMMergePolicyFactory;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMOperationTrackerFactory;
@@ -323,10 +324,10 @@
         ILSMOperationTrackerFactory opTrackerFactory =
                 index.isPrimaryIndex() ? new PrimaryIndexOperationTrackerFactory(datasetId)
                         : new SecondaryIndexOperationTrackerFactory(datasetId);
-        ILSMComponentIdGeneratorFactory idGeneratorProvider =
-                new DatasetLSMComponentIdGeneratorFactory(index.getDatasetId().getId());
+        ILSMComponentIdGeneratorFactory idGeneratorProvider = new DatasetLSMComponentIdGeneratorFactory(datasetId);
+        DatasetInfoProvider datasetInfoProvider = new DatasetInfoProvider(datasetId);
         ILSMIOOperationCallbackFactory ioOpCallbackFactory =
-                new LSMBTreeIOOperationCallbackFactory(idGeneratorProvider);
+                new LSMIndexIOOperationCallbackFactory(idGeneratorProvider, datasetInfoProvider);
         IStorageComponentProvider storageComponentProvider = appContext.getStorageComponentProvider();
         if (isNewUniverse()) {
             final double bloomFilterFalsePositiveRate =
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entities/Dataset.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entities/Dataset.java
index 7981309..2975972 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entities/Dataset.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entities/Dataset.java
@@ -27,17 +27,17 @@
 
 import org.apache.asterix.active.IActiveEntityEventsListener;
 import org.apache.asterix.active.IActiveNotificationHandler;
+import org.apache.asterix.common.api.IDatasetInfoProvider;
+import org.apache.asterix.common.api.ILSMComponentIdGeneratorFactory;
 import org.apache.asterix.common.config.DatasetConfig.DatasetType;
 import org.apache.asterix.common.context.CorrelatedPrefixMergePolicyFactory;
+import org.apache.asterix.common.context.DatasetInfoProvider;
 import org.apache.asterix.common.context.DatasetLSMComponentIdGeneratorFactory;
 import org.apache.asterix.common.context.IStorageComponentProvider;
 import org.apache.asterix.common.dataflow.NoOpFrameOperationCallbackFactory;
 import org.apache.asterix.common.exceptions.CompilationException;
 import org.apache.asterix.common.exceptions.ErrorCode;
-import org.apache.asterix.common.ioopcallbacks.LSMBTreeIOOperationCallbackFactory;
-import org.apache.asterix.common.ioopcallbacks.LSMBTreeWithBuddyIOOperationCallbackFactory;
-import org.apache.asterix.common.ioopcallbacks.LSMInvertedIndexIOOperationCallbackFactory;
-import org.apache.asterix.common.ioopcallbacks.LSMRTreeIOOperationCallbackFactory;
+import org.apache.asterix.common.ioopcallbacks.LSMIndexIOOperationCallbackFactory;
 import org.apache.asterix.common.metadata.IDataset;
 import org.apache.asterix.common.transactions.IRecoveryManager.ResourceType;
 import org.apache.asterix.common.utils.JobUtils;
@@ -105,7 +105,6 @@
 import org.apache.hyracks.storage.am.common.impls.NoOpOperationCallbackFactory;
 import org.apache.hyracks.storage.am.common.ophelpers.IndexOperation;
 import org.apache.hyracks.storage.am.lsm.common.api.IFrameOperationCallbackFactory;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponentIdGeneratorFactory;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallbackFactory;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMMergePolicyFactory;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMOperationTrackerFactory;
@@ -502,24 +501,9 @@
      * @throws AlgebricksException
      *             if the factory could not be created for the index/dataset combination
      */
+    @SuppressWarnings("squid:S1172")
     public ILSMIOOperationCallbackFactory getIoOperationCallbackFactory(Index index) throws AlgebricksException {
-        switch (index.getIndexType()) {
-            case BTREE:
-                return getDatasetType() == DatasetType.EXTERNAL
-                        && !index.getIndexName().equals(IndexingConstants.getFilesIndexName(getDatasetName()))
-                                ? new LSMBTreeWithBuddyIOOperationCallbackFactory(getComponentIdGeneratorFactory())
-                                : new LSMBTreeIOOperationCallbackFactory(getComponentIdGeneratorFactory());
-            case RTREE:
-                return new LSMRTreeIOOperationCallbackFactory(getComponentIdGeneratorFactory());
-            case LENGTH_PARTITIONED_NGRAM_INVIX:
-            case LENGTH_PARTITIONED_WORD_INVIX:
-            case SINGLE_PARTITION_NGRAM_INVIX:
-            case SINGLE_PARTITION_WORD_INVIX:
-                return new LSMInvertedIndexIOOperationCallbackFactory(getComponentIdGeneratorFactory());
-            default:
-                throw new CompilationException(ErrorCode.COMPILATION_UNKNOWN_INDEX_TYPE,
-                        index.getIndexType().toString());
-        }
+        return new LSMIndexIOOperationCallbackFactory(getComponentIdGeneratorFactory(), getDatasetInfoProvider());
     }
 
     /**
@@ -538,6 +522,10 @@
         return new DatasetLSMComponentIdGeneratorFactory(getDatasetId());
     }
 
+    public IDatasetInfoProvider getDatasetInfoProvider() {
+        return new DatasetInfoProvider(getDatasetId());
+    }
+
     /**
      * Get search callback factory for this dataset with the passed index and operation
      *
@@ -563,7 +551,6 @@
             boolean proceedIndexOnlyPlan) throws AlgebricksException {
         if (index.isPrimaryIndex()) {
             /**
-            /*
              * Due to the read-committed isolation level,
              * we may acquire very short duration lock(i.e., instant lock) for readers.
              */
diff --git a/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/sync/IndexSynchronizer.java b/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/sync/IndexSynchronizer.java
index 95ae690..0e07ac7 100644
--- a/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/sync/IndexSynchronizer.java
+++ b/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/sync/IndexSynchronizer.java
@@ -24,7 +24,7 @@
 import java.io.IOException;
 
 import org.apache.asterix.common.api.INcApplicationContext;
-import org.apache.asterix.common.ioopcallbacks.AbstractLSMIOOperationCallback;
+import org.apache.asterix.common.ioopcallbacks.LSMIOOperationCallback;
 import org.apache.asterix.common.utils.StoragePathUtil;
 import org.apache.asterix.replication.api.PartitionReplica;
 import org.apache.asterix.replication.messaging.ComponentMaskTask;
@@ -119,11 +119,11 @@
     private long getReplicatedComponentLsn() throws HyracksDataException {
         final ILSMIndexReplicationJob indexReplJob = (ILSMIndexReplicationJob) job;
         if (indexReplJob.getLSMOpType() != LSMOperationType.FLUSH) {
-            return AbstractLSMIOOperationCallback.INVALID;
+            return LSMIOOperationCallback.INVALID_LSN;
         }
         final ILSMIndex lsmIndex = indexReplJob.getLSMIndex();
         final ILSMIndexOperationContext ctx = indexReplJob.getLSMIndexOperationContext();
-        return ((AbstractLSMIOOperationCallback) lsmIndex.getIOOperationCallback())
+        return ((LSMIOOperationCallback) lsmIndex.getIOOperationCallback())
                 .getComponentLSN(ctx.getComponentsToBeReplicated());
     }
 }
\ No newline at end of file
diff --git a/hyracks-fullstack/algebricks/algebricks-compiler/src/test/java/org/apache/hyracks/algebricks/compiler/rewriter/rulecontrollers/SequentialFirstRuleCheckFixpointRuleControllerTest.java b/hyracks-fullstack/algebricks/algebricks-compiler/src/test/java/org/apache/hyracks/algebricks/compiler/rewriter/rulecontrollers/SequentialFirstRuleCheckFixpointRuleControllerTest.java
index 339cb15..18aa104 100644
--- a/hyracks-fullstack/algebricks/algebricks-compiler/src/test/java/org/apache/hyracks/algebricks/compiler/rewriter/rulecontrollers/SequentialFirstRuleCheckFixpointRuleControllerTest.java
+++ b/hyracks-fullstack/algebricks/algebricks-compiler/src/test/java/org/apache/hyracks/algebricks/compiler/rewriter/rulecontrollers/SequentialFirstRuleCheckFixpointRuleControllerTest.java
@@ -36,10 +36,12 @@
 import org.junit.Test;
 import org.junit.runner.RunWith;
 import org.powermock.api.mockito.PowerMockito;
+import org.powermock.core.classloader.annotations.PowerMockIgnore;
 import org.powermock.core.classloader.annotations.PrepareForTest;
 import org.powermock.modules.junit4.PowerMockRunner;
 
 @RunWith(PowerMockRunner.class)
+@PowerMockIgnore("javax.management.*")
 @PrepareForTest({ SequentialFirstRuleCheckFixpointRuleController.class, AbstractLogicalOperator.class })
 public class SequentialFirstRuleCheckFixpointRuleControllerTest {
 
diff --git a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/exceptions/ErrorCode.java b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/exceptions/ErrorCode.java
index 51afac1..b6d7cc7 100644
--- a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/exceptions/ErrorCode.java
+++ b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/exceptions/ErrorCode.java
@@ -121,7 +121,7 @@
     public static final int FOUND_MULTIPLE_TRANSACTIONS = 85;
     public static final int UNRECOGNIZED_INDEX_COMPONENT_FILE = 86;
     public static final int UNEQUAL_NUM_FILTERS_TREES = 87;
-    public static final int CANNOT_MODIFY_INDEX_DISK_IS_FULL = 88;
+    public static final int INDEX_NOT_MODIFIABLE = 88;
     public static final int GROUP_BY_MEMORY_BUDGET_EXCEEDS = 89;
     public static final int ILLEGAL_MEMORY_BUDGET = 90;
     public static final int TIMEOUT = 91;
diff --git a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/util/ExceptionUtils.java b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/util/ExceptionUtils.java
index 9302f46..bba0de7 100644
--- a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/util/ExceptionUtils.java
+++ b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/util/ExceptionUtils.java
@@ -119,4 +119,15 @@
         stackThrowable.setStackTrace(thread.getStackTrace());
         return stackThrowable;
     }
+
+    public static Throwable getRootCause(Throwable e) {
+        Throwable current = e;
+        Throwable cause = e.getCause();
+        while (cause != null && cause != current) {
+            current = cause;
+            cause = current.getCause();
+        }
+        return current;
+    }
+
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/util/SingleThreadEventProcessor.java b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/util/SingleThreadEventProcessor.java
index ccf0163..85019c5 100644
--- a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/util/SingleThreadEventProcessor.java
+++ b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/util/SingleThreadEventProcessor.java
@@ -65,20 +65,20 @@
         }
     }
 
-    public void stop() throws HyracksDataException, InterruptedException {
+    public void stop() throws HyracksDataException {
         stopped = true;
         executorThread.interrupt();
-        executorThread.join(1000);
+        InvokeUtil.doUninterruptibly(() -> executorThread.join(1000));
         int attempt = 0;
         while (executorThread.isAlive()) {
             attempt++;
-            LOGGER.log(Level.WARN,
-                    "Failed to stop event processor after " + attempt + " attempts. Interrupted exception swallowed?");
+            LOGGER.log(Level.WARN, "Failed to stop event processor after {} attempts. Interrupted exception swallowed?",
+                    attempt, ExceptionUtils.fromThreadStack(executorThread));
             if (attempt == 10) {
                 throw HyracksDataException.create(ErrorCode.FAILED_TO_SHUTDOWN_EVENT_PROCESSOR, name);
             }
             executorThread.interrupt();
-            executorThread.join(1000);
+            InvokeUtil.doUninterruptibly(() -> executorThread.join(1000));
         }
     }
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-api/src/main/resources/errormsg/en.properties b/hyracks-fullstack/hyracks/hyracks-api/src/main/resources/errormsg/en.properties
index 452d379..ef07038 100644
--- a/hyracks-fullstack/hyracks/hyracks-api/src/main/resources/errormsg/en.properties
+++ b/hyracks-fullstack/hyracks/hyracks-api/src/main/resources/errormsg/en.properties
@@ -104,7 +104,7 @@
 85 = Found more than one transaction file in %1$s
 86 = Found an unrecognized index file %1$s
 87 = Unequal number of trees and filters found in %1$s
-88 = Cannot modify index (Disk is full)
+88 = Cannot modify index (%1$s)
 89 = The byte size of a single group (%1$s bytes) exceeds the budget for a group by operator (%2$s bytes)
 90 = Memory budget for the %1$s operator (%2$s bytes) is lower than the minimum (%3$s bytes)
 91 = Operation timed out
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/ophelpers/IndexOperation.java b/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/ophelpers/IndexOperation.java
index 636e4f5..56600b7 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/ophelpers/IndexOperation.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/ophelpers/IndexOperation.java
@@ -35,6 +35,5 @@
     FLUSH,
     REPLICATE,
     DISK_COMPONENT_SCAN,
-    DELETE_MEMORY_COMPONENT,
-    DELETE_DISK_COMPONENTS
+    DELETE_COMPONENTS
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/ExternalBTree.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/ExternalBTree.java
index 4c2fc3b..cfffbe1 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/ExternalBTree.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/ExternalBTree.java
@@ -40,8 +40,6 @@
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMDiskComponentBulkLoader;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMDiskComponentFactory;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperation;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperation.LSMIOOperationType;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallback;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallbackFactory;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationScheduler;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndexAccessor;
@@ -54,6 +52,7 @@
 import org.apache.hyracks.storage.am.lsm.common.impls.LSMComponentFileReferences;
 import org.apache.hyracks.storage.am.lsm.common.impls.LSMTreeIndexAccessor;
 import org.apache.hyracks.storage.am.lsm.common.impls.LSMTreeIndexAccessor.ICursorFactory;
+import org.apache.hyracks.storage.am.lsm.common.impls.LoadOperation;
 import org.apache.hyracks.storage.common.IIndexAccessParameters;
 import org.apache.hyracks.storage.common.IIndexBulkLoader;
 import org.apache.hyracks.storage.common.IIndexCursor;
@@ -170,8 +169,7 @@
     // The only reason to override the following method is that it uses a different context object
     // in addition, determining whether or not to keep deleted tuples is different here
     @Override
-    public void scheduleMerge(ILSMIndexOperationContext ctx, ILSMIOOperationCallback callback)
-            throws HyracksDataException {
+    public LSMBTreeMergeOperation createMergeOperation(ILSMIndexOperationContext ctx) throws HyracksDataException {
         ExternalBTreeOpContext opCtx = createOpContext(NoOpOperationCallback.INSTANCE, -1);
         opCtx.setOperation(IndexOperation.MERGE);
         List<ILSMComponent> mergingComponents = ctx.getComponentHolder();
@@ -195,9 +193,11 @@
         LSMComponentFileReferences relMergeFileRefs =
                 fileManager.getRelMergeFileReference(firstFile.getFile().getName(), lastFile.getFile().getName());
         ILSMIndexAccessor accessor = new LSMTreeIndexAccessor(getHarness(), opCtx, cursorFactory);
-        ioScheduler.scheduleOperation(new LSMBTreeMergeOperation(accessor, cursor,
+        LSMBTreeMergeOperation mergeOp = new LSMBTreeMergeOperation(accessor, cursor,
                 relMergeFileRefs.getInsertIndexFileReference(), relMergeFileRefs.getBloomFilterFileReference(),
-                callback, fileManager.getBaseDir().getAbsolutePath()));
+                ioOpCallback, fileManager.getBaseDir().getAbsolutePath());
+        ioOpCallback.scheduled(mergeOp);
+        return mergeOp;
     }
 
     // This function should only be used when a transaction fail. it doesn't
@@ -286,11 +286,6 @@
         if (!isActive) {
             throw new HyracksDataException("Failed to deactivate the index since it is already deactivated.");
         }
-        if (flushOnExit) {
-            ExternalBTreeOpContext opCtx = createOpContext(NoOpOperationCallback.INSTANCE, version);
-            opCtx.setIoOperationType(LSMIOOperationType.FLUSH);
-            ioOpCallback.afterFinalize(opCtx);
-        }
         for (ILSMDiskComponent c : diskComponents) {
             c.deactivateAndPurge();
         }
@@ -353,8 +348,7 @@
 
     // Not supported
     @Override
-    public void scheduleFlush(ILSMIndexOperationContext ctx, ILSMIOOperationCallback callback)
-            throws HyracksDataException {
+    public ILSMIOOperation createFlushOperation(ILSMIndexOperationContext ctx) throws HyracksDataException {
         throw new UnsupportedOperationException("flush not supported in LSM-Disk-Only-BTree");
     }
 
@@ -430,9 +424,9 @@
             } else {
                 component = createBulkLoadTarget();
             }
-
-            componentBulkLoader = component.createBulkLoader(LSMIOOperationType.LOAD, fillFactor, verifyInput,
-                    numElementsHint, false, true, true);
+            LoadOperation loadOp = new LoadOperation(ioOpCallback, getIndexIdentifier());
+            componentBulkLoader =
+                    component.createBulkLoader(loadOp, fillFactor, verifyInput, numElementsHint, false, true, true);
         }
 
         // It is expected that the mode was set to insert operation before
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/ExternalBTreeWithBuddy.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/ExternalBTreeWithBuddy.java
index 62fd850..3e762c1 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/ExternalBTreeWithBuddy.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/ExternalBTreeWithBuddy.java
@@ -35,7 +35,6 @@
 import org.apache.hyracks.storage.am.common.api.ITreeIndex;
 import org.apache.hyracks.storage.am.common.api.ITreeIndexFrameFactory;
 import org.apache.hyracks.storage.am.common.api.ITwoPCIndexBulkLoader;
-import org.apache.hyracks.storage.am.common.impls.NoOpIndexAccessParameters;
 import org.apache.hyracks.storage.am.common.impls.NoOpOperationCallback;
 import org.apache.hyracks.storage.am.common.ophelpers.IndexOperation;
 import org.apache.hyracks.storage.am.lsm.common.api.AbstractLSMWithBloomFilterDiskComponent;
@@ -44,7 +43,6 @@
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMDiskComponentBulkLoader;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMDiskComponentFactory;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperation;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperation.LSMIOOperationType;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallback;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallbackFactory;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationScheduler;
@@ -59,6 +57,7 @@
 import org.apache.hyracks.storage.am.lsm.common.impls.ExternalIndexHarness;
 import org.apache.hyracks.storage.am.lsm.common.impls.LSMComponentFileReferences;
 import org.apache.hyracks.storage.am.lsm.common.impls.LSMTreeIndexAccessor;
+import org.apache.hyracks.storage.am.lsm.common.impls.LoadOperation;
 import org.apache.hyracks.storage.common.IIndexAccessParameters;
 import org.apache.hyracks.storage.common.IIndexBulkLoader;
 import org.apache.hyracks.storage.common.IIndexCursor;
@@ -244,8 +243,7 @@
     }
 
     @Override
-    public void scheduleFlush(ILSMIndexOperationContext ctx, ILSMIOOperationCallback callback)
-            throws HyracksDataException {
+    public ILSMIOOperation createFlushOperation(ILSMIndexOperationContext ctx) throws HyracksDataException {
         throw HyracksDataException.create(ErrorCode.FLUSH_NOT_SUPPORTED_IN_EXTERNAL_INDEX);
     }
 
@@ -267,7 +265,7 @@
     }
 
     @Override
-    public void scheduleMerge(ILSMIndexOperationContext ctx, ILSMIOOperationCallback callback)
+    public LSMBTreeWithBuddyMergeOperation createMergeOperation(ILSMIndexOperationContext ctx)
             throws HyracksDataException {
         ILSMIndexOperationContext bctx = createOpContext(NoOpOperationCallback.INSTANCE, 0);
         bctx.setOperation(IndexOperation.MERGE);
@@ -289,10 +287,13 @@
                     .get(secondDiskComponents.size() - 1);
         }
 
-        ioScheduler.scheduleOperation(
+        LSMBTreeWithBuddyMergeOperation mergeOp =
                 new LSMBTreeWithBuddyMergeOperation(accessor, cursor, relMergeFileRefs.getInsertIndexFileReference(),
                         relMergeFileRefs.getDeleteIndexFileReference(), relMergeFileRefs.getBloomFilterFileReference(),
-                        callback, fileManager.getBaseDir().getAbsolutePath(), keepDeleteTuples));
+                        ioOpCallback, fileManager.getBaseDir().getAbsolutePath(), keepDeleteTuples);
+        ioOpCallback.scheduled(mergeOp);
+        return mergeOp;
+
     }
 
     // This method creates the appropriate opContext for the targeted version
@@ -331,8 +332,8 @@
                 numElements += ((AbstractLSMWithBloomFilterDiskComponent) mergeOp.getMergingComponents().get(i))
                         .getBloomFilter().getNumElements();
             }
-            componentBulkLoader = mergedComponent.createBulkLoader(LSMIOOperationType.MERGE, 1.0f, false, numElements,
-                    false, false, false);
+            componentBulkLoader =
+                    mergedComponent.createBulkLoader(operation, 1.0f, false, numElements, false, false, false);
             try {
                 while (buddyBtreeCursor.hasNext()) {
                     buddyBtreeCursor.next();
@@ -343,8 +344,7 @@
                 buddyBtreeCursor.close();
             }
         } else {
-            componentBulkLoader =
-                    mergedComponent.createBulkLoader(LSMIOOperationType.MERGE, 1.0f, false, 0L, false, false, false);
+            componentBulkLoader = mergedComponent.createBulkLoader(operation, 1.0f, false, 0L, false, false, false);
         }
 
         try {
@@ -432,11 +432,6 @@
         if (!isActive) {
             throw new HyracksDataException("Failed to deactivate the index since it is already deactivated.");
         }
-        if (flushOnExit) {
-            AbstractLSMIndexOperationContext opCtx = createOpContext(NoOpIndexAccessParameters.INSTANCE);
-            opCtx.setIoOperationType(LSMIOOperationType.FLUSH);
-            ioOpCallback.afterFinalize(opCtx);
-        }
         // Even though, we deactivate the index, we don't exit components or
         // modify any of the lists to make sure they
         // are there if the index was opened again
@@ -512,8 +507,9 @@
                 component = createBulkLoadTarget();
             }
 
-            componentBulkLoader = component.createBulkLoader(LSMIOOperationType.LOAD, fillFactor, verifyInput,
-                    numElementsHint, false, true, false);
+            LoadOperation loadOp = new LoadOperation(ioOpCallback, getIndexIdentifier());
+            componentBulkLoader =
+                    component.createBulkLoader(loadOp, fillFactor, verifyInput, numElementsHint, false, true, false);
         }
 
         @Override
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 f88947e..5332b1e 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
@@ -48,7 +48,6 @@
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMDiskComponentBulkLoader;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMDiskComponentFactory;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperation;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperation.LSMIOOperationType;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallback;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallbackFactory;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationScheduler;
@@ -75,7 +74,7 @@
 
 public class LSMBTree extends AbstractLSMIndex implements ITreeIndex {
 
-    private static final ICursorFactory cursorFactory = opCtx -> new LSMBTreeSearchCursor(opCtx);
+    private static final ICursorFactory cursorFactory = LSMBTreeSearchCursor::new;
     // Common for in-memory and on-disk components.
     protected final ITreeIndexFrameFactory insertLeafFrameFactory;
     protected final ITreeIndexFrameFactory deleteLeafFrameFactory;
@@ -110,8 +109,7 @@
                     new BTree(virtualBufferCache, new VirtualFreePageManager(virtualBufferCache), interiorFrameFactory,
                             insertLeafFrameFactory, cmpFactories, fieldCount,
                             ioManager.resolveAbsolutePath(fileManager.getBaseDir() + "_virtual_" + i)),
-                    virtualBufferCache, i == 0 ? true : false,
-                    filterHelper == null ? null : filterHelper.createFilter());
+                    virtualBufferCache, filterHelper == null ? null : filterHelper.createFilter());
             memoryComponents.add(mutableComponent);
             ++i;
         }
@@ -278,8 +276,7 @@
             }
             component = createDiskComponent(componentFactory, flushOp.getTarget(), null, flushOp.getBloomFilterTarget(),
                     true);
-            componentBulkLoader =
-                    component.createBulkLoader(LSMIOOperationType.MERGE, 1.0f, false, numElements, false, false, false);
+            componentBulkLoader = component.createBulkLoader(operation, 1.0f, false, numElements, false, false, false);
             IIndexCursor scanCursor = accessor.createSearchCursor(false);
             accessor.search(scanCursor, nullPred);
             try {
@@ -338,8 +335,8 @@
                     long numElements = getNumberOfElements(mergedComponents);
                     mergedComponent = createDiskComponent(componentFactory, mergeOp.getTarget(), null,
                             mergeOp.getBloomFilterTarget(), true);
-                    componentBulkLoader = mergedComponent.createBulkLoader(LSMIOOperationType.MERGE, 1.0f, false,
-                            numElements, false, false, false);
+                    componentBulkLoader =
+                            mergedComponent.createBulkLoader(operation, 1.0f, false, numElements, false, false, false);
                     while (cursor.hasNext()) {
                         cursor.next();
                         ITupleReference frameTuple = cursor.getTuple();
@@ -393,7 +390,7 @@
             LSMComponentFileReferences componentFileRefs, ILSMIOOperationCallback callback) {
         ILSMIndexAccessor accessor = createAccessor(opCtx);
         return new LSMBTreeFlushOperation(accessor, componentFileRefs.getInsertIndexFileReference(),
-                componentFileRefs.getBloomFilterFileReference(), callback, fileManager.getBaseDir().getAbsolutePath());
+                componentFileRefs.getBloomFilterFileReference(), callback, getIndexIdentifier());
     }
 
     @Override
@@ -478,6 +475,6 @@
         }
         LSMBTreeRangeSearchCursor cursor = new LSMBTreeRangeSearchCursor(opCtx, returnDeletedTuples);
         return new LSMBTreeMergeOperation(accessor, cursor, mergeFileRefs.getInsertIndexFileReference(),
-                mergeFileRefs.getBloomFilterFileReference(), callback, fileManager.getBaseDir().getAbsolutePath());
+                mergeFileRefs.getBloomFilterFileReference(), callback, getIndexIdentifier());
     }
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreeMemoryComponent.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreeMemoryComponent.java
index c7d555d..831562c 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreeMemoryComponent.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreeMemoryComponent.java
@@ -28,9 +28,9 @@
 
     private final BTree btree;
 
-    public LSMBTreeMemoryComponent(LSMBTree lsmIndex, BTree btree, IVirtualBufferCache vbc, boolean isActive,
+    public LSMBTreeMemoryComponent(LSMBTree lsmIndex, BTree btree, IVirtualBufferCache vbc,
             ILSMComponentFilter filter) {
-        super(lsmIndex, vbc, isActive, filter);
+        super(lsmIndex, vbc, filter);
         this.btree = btree;
     }
 
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreeOpContext.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreeOpContext.java
index 1cfc414..1312e30 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreeOpContext.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreeOpContext.java
@@ -32,7 +32,6 @@
 import org.apache.hyracks.storage.am.common.api.IExtendedModificationOperationCallback;
 import org.apache.hyracks.storage.am.common.api.ITreeIndexFrameFactory;
 import org.apache.hyracks.storage.am.common.impls.IndexAccessParameters;
-import org.apache.hyracks.storage.am.common.impls.NoOpIndexAccessParameters;
 import org.apache.hyracks.storage.am.common.impls.NoOpOperationCallback;
 import org.apache.hyracks.storage.am.common.ophelpers.IndexOperation;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMHarness;
@@ -40,7 +39,6 @@
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMMemoryComponent;
 import org.apache.hyracks.storage.am.lsm.common.impls.AbstractLSMIndexOperationContext;
 import org.apache.hyracks.storage.common.IIndexAccessParameters;
-import org.apache.hyracks.storage.common.IModificationOperationCallback;
 import org.apache.hyracks.storage.common.ISearchOperationCallback;
 import org.apache.hyracks.storage.common.MultiComparator;
 import org.apache.hyracks.util.trace.ITracer;
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreeRangeSearchCursor.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreeRangeSearchCursor.java
index 6eef5a9..81f4a83 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreeRangeSearchCursor.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreeRangeSearchCursor.java
@@ -214,7 +214,10 @@
         }
         opCtx.getIndex().getHarness().replaceMemoryComponentsWithDiskComponents(getOpCtx(), replaceFrom);
         // redo the search on the new component
-        for (int i = replaceFrom; i < switchRequest.length; i++) {
+        // switchRequest array has the size = number of memory components. which can be greater
+        // than operationalComponents size in certain cases (0 disk component, 1 memory component for example)
+        // To avoid index out of bound, we end the loop at the first of the two conditions
+        for (int i = replaceFrom; i < switchRequest.length && i < operationalComponents.size(); i++) {
             if (switchRequest[i]) {
                 ILSMComponent component = operationalComponents.get(i);
                 BTree btree = (BTree) component.getIndex();
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreeWithBuddyMemoryComponent.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreeWithBuddyMemoryComponent.java
index a1a1e45..ed2ee70 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreeWithBuddyMemoryComponent.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreeWithBuddyMemoryComponent.java
@@ -20,10 +20,10 @@
 
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.storage.am.btree.impls.BTree;
-import org.apache.hyracks.storage.am.lsm.common.api.AbstractLSMWithBuddyMemoryComponent;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponentFilter;
 import org.apache.hyracks.storage.am.lsm.common.api.IVirtualBufferCache;
 import org.apache.hyracks.storage.am.lsm.common.impls.AbstractLSMIndex;
+import org.apache.hyracks.storage.am.lsm.common.impls.AbstractLSMWithBuddyMemoryComponent;
 
 /*
  * This class is also not needed at the moment but is implemented anyway
@@ -34,8 +34,8 @@
     private final BTree buddyBtree;
 
     public LSMBTreeWithBuddyMemoryComponent(AbstractLSMIndex lsmIndex, BTree btree, BTree buddyBtree,
-            IVirtualBufferCache vbc, boolean isActive, ILSMComponentFilter filter) {
-        super(lsmIndex, vbc, isActive, filter);
+            IVirtualBufferCache vbc, ILSMComponentFilter filter) {
+        super(lsmIndex, vbc, filter);
         this.btree = btree;
         this.buddyBtree = buddyBtree;
     }
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 cc10a98..d071bac 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
@@ -153,29 +153,24 @@
         // can be used for both inserts and deletes
         ITreeIndexFrameFactory transactionLeafFrameFactory =
                 new BTreeNSMLeafFrameFactory(transactionTupleWriterFactory);
-
         TreeIndexFactory<DiskBTree> diskBTreeFactory =
                 new DiskBTreeFactory(ioManager, diskBufferCache, freePageManagerFactory, interiorFrameFactory,
                         copyTupleLeafFrameFactory, cmpFactories, typeTraits.length);
         TreeIndexFactory<DiskBTree> bulkLoadBTreeFactory = new DiskBTreeFactory(ioManager, diskBufferCache,
                 freePageManagerFactory, interiorFrameFactory, insertLeafFrameFactory, cmpFactories, typeTraits.length);
-
         BloomFilterFactory bloomFilterFactory = new BloomFilterFactory(diskBufferCache, bloomFilterKeyFields);
-
         // This is the component factory for transactions
         TreeIndexFactory<DiskBTree> transactionBTreeFactory =
                 new DiskBTreeFactory(ioManager, diskBufferCache, freePageManagerFactory, interiorFrameFactory,
                         transactionLeafFrameFactory, cmpFactories, typeTraits.length);
         //TODO remove BloomFilter from external dataset's secondary LSMBTree index
         ILSMIndexFileManager fileNameManager = new LSMBTreeFileManager(ioManager, file, diskBTreeFactory, true);
-
         ILSMDiskComponentFactory componentFactory =
                 new LSMBTreeWithBloomFilterDiskComponentFactory(diskBTreeFactory, bloomFilterFactory, null);
         ILSMDiskComponentFactory bulkLoadComponentFactory =
                 new LSMBTreeWithBloomFilterDiskComponentFactory(bulkLoadBTreeFactory, bloomFilterFactory, null);
         ILSMDiskComponentFactory transactionComponentFactory =
                 new LSMBTreeWithBloomFilterDiskComponentFactory(transactionBTreeFactory, bloomFilterFactory, null);
-
         // the disk only index uses an empty ArrayList for virtual buffer caches
         return new ExternalBTree(ioManager, interiorFrameFactory, insertLeafFrameFactory, deleteLeafFrameFactory,
                 diskBufferCache, fileNameManager, componentFactory, bulkLoadComponentFactory,
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 c98fa69..908af86 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
@@ -23,7 +23,6 @@
 import org.apache.hyracks.storage.am.bloomfilter.impls.BloomFilter;
 import org.apache.hyracks.storage.am.bloomfilter.impls.BloomFilterSpecification;
 import org.apache.hyracks.storage.am.common.api.IMetadataPageManager;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperation.LSMIOOperationType;
 import org.apache.hyracks.storage.am.lsm.common.impls.AbstractLSMDiskComponent;
 import org.apache.hyracks.storage.am.lsm.common.impls.AbstractLSMIndex;
 import org.apache.hyracks.storage.am.lsm.common.impls.BloomFilterBulkLoader;
@@ -93,10 +92,10 @@
     }
 
     @Override
-    public ChainedLSMDiskComponentBulkLoader createBulkLoader(LSMIOOperationType opType, float fillFactor,
+    public ChainedLSMDiskComponentBulkLoader createBulkLoader(ILSMIOOperation operation, float fillFactor,
             boolean verifyInput, long numElementsHint, boolean checkIfEmptyIndex, boolean withFilter,
             boolean cleanupEmptyComponent) throws HyracksDataException {
-        ChainedLSMDiskComponentBulkLoader chainedBulkLoader = super.createBulkLoader(opType, fillFactor, verifyInput,
+        ChainedLSMDiskComponentBulkLoader chainedBulkLoader = super.createBulkLoader(operation, fillFactor, verifyInput,
                 numElementsHint, checkIfEmptyIndex, withFilter, cleanupEmptyComponent);
         if (numElementsHint > 0) {
             chainedBulkLoader.addBulkLoader(createBloomFilterBulkLoader(numElementsHint));
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/IIoOperationFailedCallback.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/IIoOperationFailedCallback.java
new file mode 100644
index 0000000..a9b9cc0
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/IIoOperationFailedCallback.java
@@ -0,0 +1,44 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.hyracks.storage.am.lsm.common.api;
+
+/**
+ * A callback that will be called on failures of IO operations
+ */
+public interface IIoOperationFailedCallback {
+    /**
+     * Called on a failed IO operation
+     *
+     * @param operation
+     *            the failed operation
+     * @param failure
+     *            the failure
+     */
+    void operationFailed(ILSMIOOperation operation, Throwable failure);
+
+    /**
+     * Called on a scheduler failure.
+     *
+     * @param scheduler
+     *            the failed scheduler
+     * @param failure
+     *            the failure
+     */
+    void schedulerFailed(ILSMIOOperationScheduler scheduler, Throwable failure);
+}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/ILSMComponent.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/ILSMComponent.java
index fc55ce5..1bfea53 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/ILSMComponent.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/ILSMComponent.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.storage.am.lsm.common.api.ILSMIOOperation.LSMIOOperationType;
 import org.apache.hyracks.storage.common.IIndex;
 
 /**
@@ -139,4 +140,12 @@
      * @throws HyracksDataException
      */
     ILSMComponentId getId() throws HyracksDataException;
+
+    /**
+     * Prepare the component to be scheduled for an IO operation
+     *
+     * @param ioOperationType
+     * @throws HyracksDataException
+     */
+    void schedule(LSMIOOperationType ioOperationType) throws HyracksDataException;
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/ILSMComponentIdGenerator.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/ILSMComponentIdGenerator.java
index 5dd3061..e6aa2d1 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/ILSMComponentIdGenerator.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/ILSMComponentIdGenerator.java
@@ -26,12 +26,17 @@
     /**
      * @return An Id for LSM component
      */
-    public ILSMComponentId getId();
+    ILSMComponentId getId();
 
     /**
      * Refresh the component Id generator to generate the next Id.
      * {@link #getId()} would always return the same Id before this method is called.
      */
-    public void refresh();
+    void refresh();
+
+    /**
+     * @return the index of the current memory component
+     */
+    int getCurrentComponentIndex();
 
 }
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 1a0305f..1500f37 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
@@ -22,7 +22,6 @@
 
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.storage.am.common.api.ITreeIndex;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperation.LSMIOOperationType;
 import org.apache.hyracks.storage.am.lsm.common.impls.AbstractLSMIndex;
 import org.apache.hyracks.storage.am.lsm.common.impls.ChainedLSMDiskComponentBulkLoader;
 import org.apache.hyracks.storage.am.lsm.common.impls.DiskComponentMetadata;
@@ -144,7 +143,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 opType
+     * @param operation
      * @param fillFactor
      * @param verifyInput
      * @param numElementsHint
@@ -154,7 +153,7 @@
      * @return
      * @throws HyracksDataException
      */
-    ChainedLSMDiskComponentBulkLoader createBulkLoader(LSMIOOperationType opType, float fillFactor, boolean verifyInput,
+    ChainedLSMDiskComponentBulkLoader createBulkLoader(ILSMIOOperation operation, float fillFactor, boolean verifyInput,
             long numElementsHint, boolean checkIfEmptyIndex, boolean withFilter, boolean cleanupEmptyComponent)
             throws HyracksDataException;
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/ILSMDiskComponentBulkLoader.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/ILSMDiskComponentBulkLoader.java
index 13ce971..8284cee 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/ILSMDiskComponentBulkLoader.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/ILSMDiskComponentBulkLoader.java
@@ -41,4 +41,9 @@
      */
     void cleanupArtifacts() throws HyracksDataException;
 
+    /**
+     * @return The operation this bulkLoader is operating under
+     */
+    ILSMIOOperation getOperation();
+
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/ILSMHarness.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/ILSMHarness.java
index b32dd0f..8a041ec 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/ILSMHarness.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/ILSMHarness.java
@@ -108,7 +108,7 @@
      * @throws HyracksDataException
      * @throws IndexException
      */
-    void scheduleMerge(ILSMIndexOperationContext ctx, ILSMIOOperationCallback callback) throws HyracksDataException;
+    ILSMIOOperation scheduleMerge(ILSMIndexOperationContext ctx) throws HyracksDataException;
 
     /**
      * Schedule full merge
@@ -118,17 +118,16 @@
      * @throws HyracksDataException
      * @throws IndexException
      */
-    void scheduleFullMerge(ILSMIndexOperationContext ctx, ILSMIOOperationCallback callback) throws HyracksDataException;
+    ILSMIOOperation scheduleFullMerge(ILSMIndexOperationContext ctx) throws HyracksDataException;
 
     /**
      * Perform a merge operation
      *
-     * @param ctx
      * @param operation
      * @throws HyracksDataException
      * @throws IndexException
      */
-    void merge(ILSMIndexOperationContext ctx, ILSMIOOperation operation) throws HyracksDataException;
+    void merge(ILSMIOOperation operation) throws HyracksDataException;
 
     /**
      * Schedule a flush
@@ -137,17 +136,16 @@
      * @param callback
      * @throws HyracksDataException
      */
-    void scheduleFlush(ILSMIndexOperationContext ctx, ILSMIOOperationCallback callback) throws HyracksDataException;
+    ILSMIOOperation scheduleFlush(ILSMIndexOperationContext ctx) throws HyracksDataException;
 
     /**
      * Perform a flush
      *
-     * @param ctx
      * @param operation
      * @throws HyracksDataException
      * @throws IndexException
      */
-    void flush(ILSMIndexOperationContext ctx, ILSMIOOperation operation) throws HyracksDataException;
+    void flush(ILSMIOOperation operation) throws HyracksDataException;
 
     /**
      * Add bulk loaded component
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/ILSMIOOperation.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/ILSMIOOperation.java
index 65e7f64..88dcc5a 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/ILSMIOOperation.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/ILSMIOOperation.java
@@ -23,9 +23,10 @@
 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.storage.am.lsm.common.api.ILSMIOOperation.LSMIOOperationStatus;
 import org.apache.hyracks.storage.am.lsm.common.impls.LSMComponentFileReferences;
 
-public interface ILSMIOOperation extends Callable<Boolean> {
+public interface ILSMIOOperation extends Callable<LSMIOOperationStatus> {
 
     /**
      * Represents the io operation type
@@ -38,6 +39,20 @@
     }
 
     /**
+     * Represents the status of the IO operation
+     */
+    enum LSMIOOperationStatus {
+        /**
+         * Operation successful
+         */
+        SUCCESS,
+        /**
+         * Operation failed
+         */
+        FAILURE
+    }
+
+    /**
      * @return the device on which the operation is running
      */
     IODeviceHandle getDevice();
@@ -58,7 +73,7 @@
     LSMIOOperationType getIOOpertionType();
 
     @Override
-    Boolean call() throws HyracksDataException;
+    LSMIOOperationStatus call() throws HyracksDataException;
 
     /**
      * @return The target of the io operation
@@ -74,4 +89,50 @@
      * @return the component files produced by this operation
      */
     LSMComponentFileReferences getComponentFiles();
+
+    /**
+     * @return the failure in the io operation if any, null otherwise
+     */
+    Throwable getFailure();
+
+    /**
+     * @return set the failure in the io operation
+     */
+    void setFailure(Throwable failure);
+
+    /**
+     * @return the status of the IO operation
+     */
+    LSMIOOperationStatus getStatus();
+
+    /**
+     * Set the status of the IO operation
+     *
+     * @param status
+     */
+    void setStatus(LSMIOOperationStatus status);
+
+    /**
+     * @return the new component produced by this operation if any, null otherwise
+     */
+    ILSMDiskComponent getNewComponent();
+
+    /**
+     * Set the new component produced by this operation
+     *
+     * @param component
+     */
+    void setNewComponent(ILSMDiskComponent component);
+
+    /**
+     * Destroy the operation after the scheduler is done with it
+     */
+    void complete();
+
+    /**
+     * Wait for the operation to complete
+     *
+     * @throws InterruptedException
+     */
+    void sync() throws InterruptedException;
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/ILSMIOOperationCallback.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/ILSMIOOperationCallback.java
index acc9e89..e448ae0 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/ILSMIOOperationCallback.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/ILSMIOOperationCallback.java
@@ -23,14 +23,23 @@
 public interface ILSMIOOperationCallback {
 
     /**
-     * This method is called on an IO operation before the operation starts.
-     * (i.e. IO operations could be flush, or merge operations.)
-     * For flush, this is called immediately before switching the current memory component pointer
+     * This method is called on an IO operation before the operation is scheduled
+     * For operations that are not scheduled(i,e. Bulk load), this call is skipped.
+     *
+     * @param operation
+     * @throws HyracksDataException
      */
-    void beforeOperation(ILSMIndexOperationContext opCtx) throws HyracksDataException;
+    void scheduled(ILSMIOOperation operation) throws HyracksDataException;
 
     /**
-     * This method is called on an IO operation sometime after the operation was completed.
+     * This method is called on an IO operation before the operation starts.
+     * (i.e. IO operations could be flush, or merge operations.)
+     */
+    void beforeOperation(ILSMIOOperation operation) throws HyracksDataException;
+
+    /**
+     * This method is called on an IO operation sometime after the operation is completed but before the new component
+     * is marked as valid.
      * (i.e. IO operations could be flush or merge operations.)
      *
      * Copying content of metadata page from memory component to disk component should be done in this call
@@ -38,31 +47,37 @@
      *
      * @throws HyracksDataException
      */
-    void afterOperation(ILSMIndexOperationContext opCtx) throws HyracksDataException;
+    void afterOperation(ILSMIOOperation operation) throws HyracksDataException;
 
     /**
      * This method is called on an IO operation when the operation needs any cleanup works
      * regardless that the IO operation was executed or not. Once the IO operation is executed,
      * this method should be called after ILSMIOOperationCallback.afterOperation() was called.
      *
+     */
+    void afterFinalize(ILSMIOOperation operation) throws HyracksDataException;
+
+    /**
+     * This method is called after the schduler is done with the IO operation
+     * For operation that are not scheduled, this call is skipped
+     *
+     * @param operation
      * @throws HyracksDataException
      */
-    void afterFinalize(ILSMIndexOperationContext opCtx) throws HyracksDataException;
+    void completed(ILSMIOOperation operation);
 
     /**
      * This method is called when a memory component is recycled
      *
      * @param component
-     * @param componentSwitched
-     *            true if the component index was advanced for this recycle, false otherwise
      */
-    void recycled(ILSMMemoryComponent component, boolean componentSwitched) throws HyracksDataException;
+    void recycled(ILSMMemoryComponent component) throws HyracksDataException;
 
     /**
      * This method is called when a memory component is allocated
      *
      * @param component
+     *            the allocated component
      */
     void allocated(ILSMMemoryComponent component) throws HyracksDataException;
-
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/ILSMIOOperationCallbackFactory.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/ILSMIOOperationCallbackFactory.java
index e8742b5..8ccf41f 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/ILSMIOOperationCallbackFactory.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/ILSMIOOperationCallbackFactory.java
@@ -32,5 +32,18 @@
      */
     void initialize(INCServiceContext ncCtx, IResource resource);
 
+    /**
+     * Create the IO Operation Callback
+     *
+     * @param index
+     * @return
+     * @throws HyracksDataException
+     */
     ILSMIOOperationCallback createIoOpCallback(ILSMIndex index) throws HyracksDataException;
+
+    /**
+     * @return the current memory component index
+     * @throws HyracksDataException
+     */
+    int getCurrentMemoryComponentIndex() throws HyracksDataException;
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/ILSMIndex.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/ILSMIndex.java
index a8467d3..1374524 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/ILSMIndex.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/ILSMIndex.java
@@ -48,8 +48,6 @@
 
     ILSMOperationTracker getOperationTracker();
 
-    ILSMIOOperationScheduler getIOScheduler();
-
     ILSMIOOperationCallback getIOOperationCallback();
 
     /**
@@ -74,11 +72,33 @@
 
     public void scanDiskComponents(ILSMIndexOperationContext ctx, IIndexCursor cursor) throws HyracksDataException;
 
-    void scheduleFlush(ILSMIndexOperationContext ctx, ILSMIOOperationCallback callback) throws HyracksDataException;
+    /**
+     * Create a flush operation.
+     * This is an atomic operation. If an exception is thrown, no partial effect is left
+     *
+     * @return the flush operation
+     *
+     * @param ctx
+     *            the operation context
+     * @param callback
+     *            the IO callback
+     * @throws HyracksDataException
+     */
+    ILSMIOOperation createFlushOperation(ILSMIndexOperationContext ctx) throws HyracksDataException;
 
     ILSMDiskComponent flush(ILSMIOOperation operation) throws HyracksDataException;
 
-    void scheduleMerge(ILSMIndexOperationContext ctx, ILSMIOOperationCallback callback) throws HyracksDataException;
+    /**
+     * Create a merge operation.
+     * This is an atomic operation. If an exception is thrown, no partial effect is left
+     *
+     * @param ctx
+     *            the operation context
+     * @param callback
+     *            the IO callback
+     * @throws HyracksDataException
+     */
+    ILSMIOOperation createMergeOperation(ILSMIndexOperationContext ctx) throws HyracksDataException;
 
     ILSMDiskComponent merge(ILSMIOOperation operation) throws HyracksDataException;
 
@@ -157,4 +177,12 @@
      * @return the {@link ILSMHarness} of the index
      */
     ILSMHarness getHarness();
+
+    /**
+     * Cleanup the files of the failed operation
+     *
+     * @param operation
+     */
+    void cleanUpFilesForFailedOperation(ILSMIOOperation operation);
+
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/ILSMIndexAccessor.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/ILSMIndexAccessor.java
index 61ef6cf..34844d6 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/ILSMIndexAccessor.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/ILSMIndexAccessor.java
@@ -44,34 +44,27 @@
     /**
      * Schedule a flush operation
      *
-     * @param callback
-     *            the IO operation callback
      * @throws HyracksDataException
      */
-    void scheduleFlush(ILSMIOOperationCallback callback) throws HyracksDataException;
+    ILSMIOOperation scheduleFlush() throws HyracksDataException;
 
     /**
      * Schedule a merge operation
      *
-     * @param callback
-     *            the merge operation callback
      * @param components
      *            the components to be merged
      * @throws HyracksDataException
      * @throws IndexException
      */
-    void scheduleMerge(ILSMIOOperationCallback callback, List<ILSMDiskComponent> components)
-            throws HyracksDataException;
+    ILSMIOOperation scheduleMerge(List<ILSMDiskComponent> components) throws HyracksDataException;
 
     /**
      * Schedule a full merge
      *
-     * @param callback
-     *            the merge operation callback
      * @throws HyracksDataException
      * @throws IndexException
      */
-    void scheduleFullMerge(ILSMIOOperationCallback callback) throws HyracksDataException;
+    ILSMIOOperation scheduleFullMerge() throws HyracksDataException;
 
     /**
      * Delete the tuple from the memory component only. Don't replace with antimatter tuple
@@ -203,7 +196,7 @@
             throws HyracksDataException;
 
     /**
-     * Force a flush of the in-memory component.
+     * Flush an in-memory component.
      *
      * @throws HyracksDataException
      * @throws TreeIndexException
@@ -265,9 +258,10 @@
     void deleteComponents(Predicate<ILSMComponent> predicate) throws HyracksDataException;
 
     /**
-    * Update the filter of an LSM index
-    * @param tuple
-    * @throws HyracksDataException
-    */
+     * Update the filter of an LSM index
+     *
+     * @param tuple
+     * @throws HyracksDataException
+     */
     void updateFilter(ITupleReference tuple) throws HyracksDataException;
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/ILSMIndexOperationContext.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/ILSMIndexOperationContext.java
index b34b403..238d5f8 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/ILSMIndexOperationContext.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/ILSMIndexOperationContext.java
@@ -19,12 +19,11 @@
 package org.apache.hyracks.storage.am.lsm.common.api;
 
 import java.util.List;
+import java.util.Map;
 
 import org.apache.hyracks.storage.am.common.api.IExtendedModificationOperationCallback;
 import org.apache.hyracks.storage.am.common.api.IIndexOperationContext;
 import org.apache.hyracks.storage.am.common.tuples.PermutingTupleReference;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperation.LSMIOOperationType;
-import org.apache.hyracks.storage.common.IModificationOperationCallback;
 import org.apache.hyracks.storage.common.ISearchOperationCallback;
 import org.apache.hyracks.storage.common.ISearchPredicate;
 import org.apache.hyracks.storage.common.MultiComparator;
@@ -94,26 +93,26 @@
     void setRecovery(boolean recovery);
 
     /**
-     * @return the IO Operation type associated with this context
+     * @return the IO operation associated with this context
      */
-    LSMIOOperationType getIoOperationType();
+    ILSMIOOperation getIoOperation();
 
     /**
-     * Set the IO Operation type associated with this context
+     * Set the IO operation associated with this context
      *
-     * @param ioOpType
+     * @param ioOperation
      */
-    void setIoOperationType(LSMIOOperationType ioOpType);
+    void setIoOperation(ILSMIOOperation ioOperation);
 
     /**
-     * @return the new component produced by this operation if any, null otherwise
-     */
-    ILSMDiskComponent getNewComponent();
-
-    /**
-     * Set the new component produced by this operation
+     * Set a map in the context to pass pairs of keys and values
      *
-     * @param component
+     * @param map
      */
-    void setNewComponent(ILSMDiskComponent component);
+    void setParameters(Map<String, Object> map);
+
+    /**
+     * @return the key value map of the context
+     */
+    Map<String, Object> getParameters();
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/ILSMMemoryComponent.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/ILSMMemoryComponent.java
index 4ff6377..c86f7b9 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/ILSMMemoryComponent.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/ILSMMemoryComponent.java
@@ -63,19 +63,6 @@
     void setModified();
 
     /**
-     * request the component to be active
-     */
-    void requestActivation();
-
-    /**
-     * Set the component state
-     *
-     * @param state
-     *            the new state
-     */
-    void setState(ComponentState state);
-
-    /**
      * Allocates memory to this component, create and activate it.
      * This method is atomic. If an exception is thrown, then the call had no effect.
      *
@@ -108,8 +95,14 @@
      *
      * @param newId
      * @param force
-     *      Whether to force reset the Id to skip sanity checks
+     *            Whether to force reset the Id to skip sanity checks
      * @throws HyracksDataException
      */
     void resetId(ILSMComponentId newId, boolean force) throws HyracksDataException;
+
+    /**
+     * Set the component state to be unwritable to prevent future writers from non-force
+     * entry to the component
+     */
+    void setUnwritable();
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/dataflow/LSMIndexCompactOperatorNodePushable.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/dataflow/LSMIndexCompactOperatorNodePushable.java
index 4e8cf71..682ffef 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/dataflow/LSMIndexCompactOperatorNodePushable.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/dataflow/LSMIndexCompactOperatorNodePushable.java
@@ -57,7 +57,7 @@
         indexHelper.open();
         ILSMIndex index = (ILSMIndex) indexHelper.getIndexInstance();
         ILSMIndexAccessor accessor = index.createAccessor(NoOpIndexAccessParameters.INSTANCE);
-        accessor.scheduleFullMerge(index.getIOOperationCallback());
+        accessor.scheduleFullMerge();
     }
 
     @Override
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/AbstractIoOperation.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/AbstractIoOperation.java
index aee46f0..b3252d9 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/AbstractIoOperation.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/AbstractIoOperation.java
@@ -20,6 +20,8 @@
 
 import org.apache.hyracks.api.io.FileReference;
 import org.apache.hyracks.api.io.IODeviceHandle;
+import org.apache.hyracks.api.util.ExceptionUtils;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMDiskComponent;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperation;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallback;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndexAccessor;
@@ -30,6 +32,10 @@
     protected final FileReference target;
     protected final ILSMIOOperationCallback callback;
     protected final String indexIdentifier;
+    private Throwable failure;
+    private LSMIOOperationStatus status = LSMIOOperationStatus.SUCCESS;
+    private ILSMDiskComponent newComponent;
+    private boolean completed = false;
 
     public AbstractIoOperation(ILSMIndexAccessor accessor, FileReference target, ILSMIOOperationCallback callback,
             String indexIdentifier) {
@@ -63,4 +69,51 @@
     public String getIndexIdentifier() {
         return indexIdentifier;
     }
+
+    @Override
+    public Throwable getFailure() {
+        return failure;
+    }
+
+    @Override
+    public void setFailure(Throwable failure) {
+        this.failure = ExceptionUtils.suppress(this.failure, failure);
+    }
+
+    @Override
+    public LSMIOOperationStatus getStatus() {
+        return status;
+    }
+
+    @Override
+    public void setStatus(LSMIOOperationStatus status) {
+        this.status = status;
+    }
+
+    @Override
+    public ILSMDiskComponent getNewComponent() {
+        return newComponent;
+    }
+
+    @Override
+    public void setNewComponent(ILSMDiskComponent component) {
+        this.newComponent = component;
+    }
+
+    @Override
+    public synchronized void complete() {
+        if (completed) {
+            throw new IllegalStateException("Multiple destroy calls");
+        }
+        callback.completed(this);
+        completed = true;
+        notifyAll();
+    }
+
+    @Override
+    public synchronized void sync() throws InterruptedException {
+        while (!completed) {
+            wait();
+        }
+    }
 }
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 5e28b30..0aa5775 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
@@ -24,6 +24,7 @@
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponentFilter;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponentId;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMDiskComponent;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperation;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperation.LSMIOOperationType;
 import org.apache.hyracks.storage.am.lsm.common.api.LSMOperationType;
 import org.apache.hyracks.storage.am.lsm.common.util.ComponentUtils;
@@ -53,27 +54,34 @@
     }
 
     @Override
+    public void schedule(LSMIOOperationType ioOperationType) throws HyracksDataException {
+        if (ioOperationType != LSMIOOperationType.MERGE) {
+            throw new IllegalStateException("Unsupported operation type: " + ioOperationType);
+        }
+        if (state == ComponentState.INACTIVE) {
+            throw new IllegalStateException("Trying to schedule a merge of an inactive disk component");
+        }
+        if (state == ComponentState.READABLE_MERGING) {
+            // This should never happen unless there are two concurrent merges that were scheduled
+            // concurrently and they have interleaving components to be merged.
+            // This should be handled properly by the merge policy, but we guard against that here anyway.
+            throw new IllegalStateException("The disk component has already been scheduled for a merge");
+        }
+        state = ComponentState.READABLE_MERGING;
+    }
+
+    @Override
     public boolean threadEnter(LSMOperationType opType, boolean isMutableComponent) {
         if (state == ComponentState.INACTIVE) {
             throw new IllegalStateException("Trying to enter an inactive disk component");
         }
-
         switch (opType) {
             case FORCE_MODIFICATION:
             case MODIFICATION:
             case REPLICATE:
             case SEARCH:
             case DISK_COMPONENT_SCAN:
-                readerCount++;
-                break;
             case MERGE:
-                if (state == ComponentState.READABLE_MERGING) {
-                    // This should never happen unless there are two concurrent merges that were scheduled
-                    // concurrently and they have interleaving components to be merged.
-                    // This should be handled properly by the merge policy, but we guard against that here anyway.
-                    return false;
-                }
-                state = ComponentState.READABLE_MERGING;
                 readerCount++;
                 break;
             default:
@@ -87,19 +95,22 @@
             throws HyracksDataException {
         switch (opType) {
             case MERGE:
+                readerCount--;
                 // In case two merge operations were scheduled to merge an overlapping set of components,
                 // the second merge will fail and it must reset those components back to their previous state.
                 if (failedOperation) {
                     state = ComponentState.READABLE_UNWRITABLE;
+                } else {
+                    state = (readerCount == 0) ? ComponentState.INACTIVE : ComponentState.UNREADABLE_UNWRITABLE;
                 }
-                // Fallthrough
+                break;
             case FORCE_MODIFICATION:
             case MODIFICATION:
             case REPLICATE:
             case SEARCH:
             case DISK_COMPONENT_SCAN:
                 readerCount--;
-                if (readerCount == 0 && state == ComponentState.READABLE_MERGING) {
+                if (readerCount == 0 && state == ComponentState.UNREADABLE_UNWRITABLE) {
                     state = ComponentState.INACTIVE;
                 }
                 break;
@@ -213,15 +224,15 @@
     }
 
     @Override
-    public ChainedLSMDiskComponentBulkLoader createBulkLoader(LSMIOOperationType opType, float fillFactor,
+    public ChainedLSMDiskComponentBulkLoader createBulkLoader(ILSMIOOperation operation, float fillFactor,
             boolean verifyInput, long numElementsHint, boolean checkIfEmptyIndex, boolean withFilter,
             boolean cleanupEmptyComponent) throws HyracksDataException {
         ChainedLSMDiskComponentBulkLoader chainedBulkLoader =
-                new ChainedLSMDiskComponentBulkLoader(this, cleanupEmptyComponent);
+                new ChainedLSMDiskComponentBulkLoader(operation, this, cleanupEmptyComponent);
         if (withFilter && getLsmIndex().getFilterFields() != null) {
             chainedBulkLoader.addBulkLoader(createFilterBulkLoader());
         }
-        IChainedComponentBulkLoader indexBulkloader = opType == LSMIOOperationType.MERGE
+        IChainedComponentBulkLoader indexBulkloader = operation.getIOOpertionType() == LSMIOOperationType.MERGE
                 ? createMergeIndexBulkLoader(fillFactor, verifyInput, numElementsHint, checkIfEmptyIndex)
                 : createIndexBulkLoader(fillFactor, verifyInput, numElementsHint, checkIfEmptyIndex);
         chainedBulkLoader.addBulkLoader(indexBulkloader);
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 2db2f79..a999b25 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
@@ -21,9 +21,11 @@
 
 import java.io.IOException;
 import java.util.ArrayList;
+import java.util.HashMap;
 import java.util.HashSet;
 import java.util.LinkedList;
 import java.util.List;
+import java.util.Map;
 import java.util.Set;
 import java.util.concurrent.atomic.AtomicBoolean;
 import java.util.concurrent.atomic.AtomicInteger;
@@ -41,6 +43,7 @@
 import org.apache.hyracks.storage.am.common.ophelpers.IndexOperation;
 import org.apache.hyracks.storage.am.lsm.common.api.IComponentFilterHelper;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponent;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponent.ComponentState;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponentFilterFrameFactory;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponentId;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponentId.IdCompareResult;
@@ -74,7 +77,6 @@
     private static final Logger LOGGER = LogManager.getLogger();
     protected final ILSMHarness lsmHarness;
     protected final IIOManager ioManager;
-    protected final ILSMIOOperationScheduler ioScheduler;
     protected final ILSMIOOperationCallback ioOpCallback;
 
     // In-memory components.
@@ -95,6 +97,7 @@
     protected final int[] filterFields;
     protected final boolean durable;
     protected boolean isActive;
+    protected volatile boolean isDeactivating = false;
     protected final AtomicBoolean[] flushRequests;
     protected volatile boolean memoryComponentsAllocated = false;
     protected ITracer tracer;
@@ -102,6 +105,7 @@
     protected final ILSMDiskComponentFactory componentFactory;
     // Factory for creating on-disk index components during bulkload.
     protected final ILSMDiskComponentFactory bulkLoadComponentFactory;
+    private int numScheduledFlushes = 0;
 
     public AbstractLSMIndex(IIOManager ioManager, List<IVirtualBufferCache> virtualBufferCaches,
             IBufferCache diskBufferCache, ILSMIndexFileManager fileManager, double bloomFilterFalsePositiveRate,
@@ -115,7 +119,6 @@
         this.diskBufferCache = diskBufferCache;
         this.fileManager = fileManager;
         this.bloomFilterFalsePositiveRate = bloomFilterFalsePositiveRate;
-        this.ioScheduler = ioScheduler;
         this.ioOpCallback = ioOpCallbackFactory.createIoOpCallback(this);
         this.componentFactory = componentFactory;
         this.bulkLoadComponentFactory = bulkLoadComponentFactory;
@@ -127,11 +130,12 @@
         this.inactiveDiskComponents = new LinkedList<>();
         this.durable = durable;
         this.tracer = tracer;
-        lsmHarness = new LSMHarness(this, mergePolicy, opTracker, diskBufferCache.isReplicationEnabled(), tracer);
+        lsmHarness = new LSMHarness(this, ioScheduler, mergePolicy, opTracker, diskBufferCache.isReplicationEnabled(),
+                tracer);
         isActive = false;
         diskComponents = new ArrayList<>();
         memoryComponents = new ArrayList<>();
-        currentMutableComponentId = new AtomicInteger();
+        currentMutableComponentId = new AtomicInteger(ioOpCallbackFactory.getCurrentMemoryComponentIndex());
         flushRequests = new AtomicBoolean[virtualBufferCaches.size()];
         for (int i = 0; i < virtualBufferCaches.size(); i++) {
             flushRequests[i] = new AtomicBoolean();
@@ -148,13 +152,13 @@
         this.diskBufferCache = diskBufferCache;
         this.fileManager = fileManager;
         this.bloomFilterFalsePositiveRate = bloomFilterFalsePositiveRate;
-        this.ioScheduler = ioScheduler;
         this.ioOpCallback = ioOpCallbackFactory.createIoOpCallback(this);
         this.componentFactory = componentFactory;
         this.bulkLoadComponentFactory = bulkLoadComponentFactory;
         this.durable = durable;
         this.tracer = tracer;
-        lsmHarness = new ExternalIndexHarness(this, mergePolicy, opTracker, diskBufferCache.isReplicationEnabled());
+        lsmHarness = new ExternalIndexHarness(this, ioScheduler, mergePolicy, opTracker,
+                diskBufferCache.isReplicationEnabled());
         isActive = false;
         diskComponents = new LinkedList<>();
         this.inactiveDiskComponents = new LinkedList<>();
@@ -205,28 +209,35 @@
         deactivate(true);
     }
 
+    @SuppressWarnings({ "squid:S1181", "squid:S2142" })
     @Override
     public synchronized void deactivate(boolean flush) throws HyracksDataException {
         if (!isActive) {
             throw HyracksDataException.create(ErrorCode.CANNOT_DEACTIVATE_INACTIVE_INDEX);
         }
-        if (flush) {
-            flushMemoryComponent();
-        }
-        deactivateDiskComponents();
-        deallocateMemoryComponents();
-        isActive = false;
-    }
-
-    private void flushMemoryComponent() throws HyracksDataException {
-        BlockingIOOperationCallbackWrapper cb = new BlockingIOOperationCallbackWrapper(ioOpCallback);
-        ILSMIndexAccessor accessor = createAccessor(NoOpIndexAccessParameters.INSTANCE);
-        accessor.scheduleFlush(cb);
+        // The following member is used to prevent scheduling of new merges as memory components
+        // get flushed. This now works only if the caller of deactivate waited for all IO
+        // operations to complete. Otherwise, disk components can be evicted while background
+        // merges are ongoing.
+        isDeactivating = true;
         try {
-            cb.waitForIO();
-        } catch (InterruptedException e) {
-            Thread.currentThread().interrupt();
-            throw HyracksDataException.create(e);
+            LOGGER.log(Level.INFO, "Deactivating the index: {}. STARTED", this);
+            if (flush && memoryComponentsAllocated) {
+                try {
+                    createAccessor(NoOpIndexAccessParameters.INSTANCE).scheduleFlush().sync();
+                } catch (InterruptedException e) {
+                    throw HyracksDataException.create(e);
+                }
+                LOGGER.log(Level.INFO, "Deactivating the index: {}. Flushed", this);
+            }
+            LOGGER.log(Level.INFO, "Deactivating the disk components of: {}", this);
+            deactivateDiskComponents();
+            LOGGER.log(Level.INFO, "Deallocating memory components of: {}", this);
+            deallocateMemoryComponents();
+            isActive = false;
+            LOGGER.log(Level.INFO, "Deactivating the index: {}. COMPLETED", this);
+        } finally {
+            isDeactivating = false;
         }
     }
 
@@ -282,6 +293,8 @@
                 c.reset();
             }
         }
+        numScheduledFlushes = 0;
+        currentMutableComponentId.set(0);
     }
 
     @Override
@@ -297,19 +310,18 @@
         switch (ctx.getOperation()) {
             case UPDATE:
             case PHYSICALDELETE:
-            case FLUSH:
-            case DELETE_MEMORY_COMPONENT:
+            case DELETE_COMPONENTS:
             case DELETE:
             case UPSERT:
                 operationalComponents.add(memoryComponents.get(cmc));
                 break;
             case INSERT:
-                addOperationalMutableComponents(operationalComponents, true);
+                addOperationalMemoryComponents(operationalComponents, true);
                 operationalComponents.addAll(diskComponents);
                 break;
             case SEARCH:
                 if (memoryComponentsAllocated) {
-                    addOperationalMutableComponents(operationalComponents, false);
+                    addOperationalMemoryComponents(operationalComponents, false);
                 }
                 if (filterManager != null) {
                     for (int i = 0; i < diskComponents.size(); i++) {
@@ -326,13 +338,6 @@
                 }
 
                 break;
-            case MERGE:
-            case DELETE_DISK_COMPONENTS:
-                operationalComponents.addAll(ctx.getComponentsToBeMerged());
-                break;
-            case FULL_MERGE:
-                operationalComponents.addAll(diskComponents);
-                break;
             case REPLICATE:
                 operationalComponents.addAll(ctx.getComponentsToBeReplicated());
                 break;
@@ -350,49 +355,140 @@
     }
 
     @Override
-    public void scheduleFlush(ILSMIndexOperationContext ctx, ILSMIOOperationCallback callback)
-            throws HyracksDataException {
-        LSMComponentFileReferences componentFileRefs = fileManager.getRelFlushFileReference();
-        AbstractLSMIndexOperationContext opCtx = createOpContext(NoOpIndexAccessParameters.INSTANCE);
-        opCtx.setOperation(ctx.getOperation());
-        opCtx.getComponentHolder().addAll(ctx.getComponentHolder());
-        ILSMIOOperation flushOp = createFlushOperation(opCtx, componentFileRefs, callback);
-        ioScheduler.scheduleOperation(TracedIOOperation.wrap(flushOp, tracer));
+    public ILSMIOOperation createFlushOperation(ILSMIndexOperationContext ctx) throws HyracksDataException {
+        ILSMMemoryComponent flushingComponent = getCurrentMemoryComponent();
+        if (flushingComponent.getWriterCount() > 0) {
+            throw new IllegalStateException(
+                    "createFlushOperation is called on a component with writers: " + flushingComponent);
+        }
+        // take care of the flush cycling
+        ILSMIOOperation flushOp =
+                TracedIOOperation.wrap(createFlushOperation(createOpContext(NoOpIndexAccessParameters.INSTANCE),
+                        fileManager.getRelFlushFileReference(), ioOpCallback), tracer);
+        // Changing the flush status should *always* precede changing the mutable component.
+        flushingComponent.schedule(LSMIOOperationType.FLUSH);
+        numScheduledFlushes++;
+        changeFlushStatusForCurrentMutableCompoent(false);
+        changeMutableComponent();
+        ILSMIndexAccessor accessor = flushOp.getAccessor();
+        ILSMIndexOperationContext flushCtx = accessor.getOpContext();
+        flushCtx.setOperation(ctx.getOperation()); // Could be component delete
+        flushCtx.getComponentHolder().add(flushingComponent);
+        flushCtx.setIoOperation(flushOp);
+        propagateMap(ctx, flushCtx);
+        ioOpCallback.scheduled(flushOp);
+        return flushOp;
     }
 
     @Override
-    public void scheduleMerge(ILSMIndexOperationContext ctx, ILSMIOOperationCallback callback)
-            throws HyracksDataException {
-        List<ILSMComponent> mergingComponents = ctx.getComponentHolder();
-        // merge must create a different op ctx
-        AbstractLSMIndexOperationContext opCtx = createOpContext(NoOpIndexAccessParameters.INSTANCE);
-        opCtx.setOperation(ctx.getOperation());
-        opCtx.getComponentHolder().addAll(mergingComponents);
-        mergingComponents.stream().map(ILSMDiskComponent.class::cast).forEach(opCtx.getComponentsToBeMerged()::add);
-        ILSMDiskComponent firstComponent = (ILSMDiskComponent) mergingComponents.get(0);
-        ILSMDiskComponent lastComponent = (ILSMDiskComponent) mergingComponents.get(mergingComponents.size() - 1);
-        LSMComponentFileReferences mergeFileRefs = getMergeFileReferences(firstComponent, lastComponent);
-        ILSMIOOperation mergeOp = createMergeOperation(opCtx, mergeFileRefs, callback);
-        ioScheduler.scheduleOperation(TracedIOOperation.wrap(mergeOp, tracer));
-    }
-
-    private void addOperationalMutableComponents(List<ILSMComponent> operationalComponents, boolean modification) {
-        int cmc = currentMutableComponentId.get();
-        int numMutableComponents = memoryComponents.size();
-        for (int i = 0; i < numMutableComponents - 1; i++) {
-            ILSMMemoryComponent c = memoryComponents.get((cmc + i + 1) % numMutableComponents);
-            if (c.isReadable()) {
-                // Make sure newest components are added first if readable
-                operationalComponents.add(0, c);
+    public ILSMIOOperation createMergeOperation(ILSMIndexOperationContext ctx) throws HyracksDataException {
+        List<ILSMDiskComponent> mergingComponents = ctx.getComponentsToBeMerged();
+        // Merge operation can fail if another merge is already scheduled on those components
+        // This should be guarded against by the merge policy but we still protect against here
+        if (isDeactivating || mergingComponents.size() < 2 && ctx.getOperation() != IndexOperation.DELETE_COMPONENTS) {
+            return NoOpIoOperation.INSTANCE;
+        }
+        for (int i = 0; i < mergingComponents.size(); i++) {
+            if (mergingComponents.get(i).getState() == ComponentState.READABLE_MERGING) {
+                return NoOpIoOperation.INSTANCE;
             }
         }
-        // The current mutable component is added if modification operation or if readable
-        // This ensures that activation of new component only happens in case of modifications
-        // and allow for controlling that without stopping search operations
-        ILSMMemoryComponent c = memoryComponents.get(cmc);
-        if (modification || c.isReadable()) {
-            operationalComponents.add(0, c);
+        // merge must create a different op ctx
+        AbstractLSMIndexOperationContext mergeCtx = createOpContext(NoOpIndexAccessParameters.INSTANCE);
+        mergeCtx.setOperation(ctx.getOperation());
+        mergeCtx.getComponentHolder().addAll(mergingComponents);
+        propagateMap(ctx, mergeCtx);
+        mergingComponents.stream().map(ILSMDiskComponent.class::cast).forEach(mergeCtx.getComponentsToBeMerged()::add);
+        ILSMDiskComponent firstComponent = mergingComponents.get(0);
+        ILSMDiskComponent lastComponent = mergingComponents.get(mergingComponents.size() - 1);
+        LSMComponentFileReferences mergeFileRefs = getMergeFileReferences(firstComponent, lastComponent);
+        ILSMIOOperation mergeOp =
+                TracedIOOperation.wrap(createMergeOperation(mergeCtx, mergeFileRefs, ioOpCallback), tracer);
+        mergeCtx.setIoOperation(mergeOp);
+        for (int i = 0; i < mergingComponents.size(); i++) {
+            mergingComponents.get(i).schedule(LSMIOOperationType.MERGE);
         }
+        ioOpCallback.scheduled(mergeOp);
+        return mergeOp;
+    }
+
+    private static void propagateMap(ILSMIndexOperationContext src, ILSMIndexOperationContext destination) {
+        Map<String, Object> map = src.getParameters();
+        if (map != null && !map.isEmpty()) {
+            destination.setParameters(new HashMap<>(map));
+        }
+    }
+
+    private void addOperationalMemoryComponents(List<ILSMComponent> operationalComponents, boolean modification) {
+        // add current memory component first if needed
+        if (numScheduledFlushes < memoryComponents.size()) {
+            ILSMMemoryComponent c = memoryComponents.get(currentMutableComponentId.get());
+            // The current mutable component is added if modification or readable
+            // This ensures that activation of new component only happens in case of modifications
+            // and allow for controlling that without stopping search operations
+            if (modification || c.isReadable()) {
+                operationalComponents.add(c);
+            }
+        }
+        if (modification && numScheduledFlushes >= memoryComponents.size()) {
+            // will fail the enterComponent call and retry
+            operationalComponents.add(memoryComponents.get(0));
+            return;
+        }
+        addImmutableMemoryComponents(operationalComponents);
+    }
+
+    private void addImmutableMemoryComponents(List<ILSMComponent> operationalComponents) {
+        int cmc = currentMutableComponentId.get();
+        int numImmutableMemoryComponents = Integer.min(numScheduledFlushes, memoryComponents.size());
+        int next = numScheduledFlushes < memoryComponents.size() ? cmc : getNextToBeFlushed();
+        for (int i = 0; i < numImmutableMemoryComponents; i++) {
+            next--;
+            if (next < 0) {
+                next = memoryComponents.size() - 1;
+            }
+            //newer components first
+            ILSMMemoryComponent c = memoryComponents.get(next);
+            if (c.isReadable()) {
+                operationalComponents.add(c);
+            }
+        }
+    }
+
+    private ILSMMemoryComponent getOldestReadableMemoryComponent() {
+        synchronized (getOperationTracker()) {
+            int cmc = currentMutableComponentId.get();
+            int numImmutableMemoryComponents = Integer.min(numScheduledFlushes, memoryComponents.size());
+            int next = numScheduledFlushes < memoryComponents.size() ? cmc : getNextToBeFlushed();
+            for (int i = 0; i < numImmutableMemoryComponents; i++) {
+                next--;
+                if (next < 0) {
+                    next = memoryComponents.size() - 1;
+                }
+            }
+
+            // start going forward
+            for (int i = 0; i < numImmutableMemoryComponents; i++) {
+                if (memoryComponents.get(next).isReadable()) {
+                    return memoryComponents.get(next);
+                }
+                next++;
+                if (next == memoryComponents.size()) {
+                    next = 0;
+                }
+            }
+            throw new IllegalStateException("Couldn't find any readable component");
+        }
+    }
+
+    private int getNextToBeFlushed() {
+        // we have:
+        // 1. currentMemeoryComponent
+        // 2. numMemoryComponents
+        // 3. numScheduledFlushes
+        int diff = numScheduledFlushes % memoryComponents.size();
+        int cmc = currentMutableComponentId.get() - diff;
+        return cmc < 0 ? memoryComponents.size() + cmc : cmc;
     }
 
     @Override
@@ -407,8 +503,9 @@
     public IIndexBulkLoader createBulkLoader(float fillLevel, boolean verifyInput, long numElementsHint)
             throws HyracksDataException {
         AbstractLSMIndexOperationContext opCtx = createOpContext(NoOpIndexAccessParameters.INSTANCE);
-        opCtx.setIoOperationType(LSMIOOperationType.LOAD);
-        ioOpCallback.beforeOperation(opCtx);
+        LoadOperation loadOp = new LoadOperation(ioOpCallback, getIndexIdentifier());
+        ioOpCallback.scheduled(loadOp);
+        opCtx.setIoOperation(loadOp);
         return new LSMIndexDiskComponentBulkLoader(this, opCtx, fillLevel, verifyInput, numElementsHint);
     }
 
@@ -503,7 +600,6 @@
     @Override
     public void changeMutableComponent() {
         currentMutableComponentId.set((currentMutableComponentId.get() + 1) % memoryComponents.size());
-        memoryComponents.get(currentMutableComponentId.get()).requestActivation();
     }
 
     @Override
@@ -527,11 +623,6 @@
     }
 
     @Override
-    public ILSMIOOperationScheduler getIOScheduler() {
-        return ioScheduler;
-    }
-
-    @Override
     public ILSMIOOperationCallback getIOOperationCallback() {
         return ioOpCallback;
     }
@@ -556,8 +647,9 @@
     @Override
     public final String toString() {
         return "{\"class\" : \"" + getClass().getSimpleName() + "\", \"dir\" : \"" + fileManager.getBaseDir()
-                + "\", \"memory\" : " + (memoryComponents == null ? 0 : memoryComponents.size()) + ", \"disk\" : "
-                + diskComponents.size() + "}";
+                + "\", \"memory\" : " + (memoryComponents == null ? 0 : memoryComponents) + ", \"disk\" : "
+                + diskComponents.size() + ", \"num-scheduled-flushes\":" + numScheduledFlushes
+                + ", \"current-memory-component\":" + currentMutableComponentId.get() + "}";
     }
 
     @Override
@@ -567,8 +659,12 @@
 
     @Override
     public boolean isCurrentMutableComponentEmpty() throws HyracksDataException {
-        //check if the current memory component has been modified
-        return !memoryComponents.get(currentMutableComponentId.get()).isModified();
+        synchronized (getOperationTracker()) {
+            ILSMMemoryComponent cmc = getCurrentMemoryComponent();
+            ComponentState state = cmc.getState();
+            return state == ComponentState.READABLE_UNWRITABLE_FLUSHING || state == ComponentState.INACTIVE
+                    || !cmc.isModified();
+        }
     }
 
     @Override
@@ -703,37 +799,30 @@
     public final ILSMDiskComponent flush(ILSMIOOperation operation) throws HyracksDataException {
         ILSMIndexAccessor accessor = operation.getAccessor();
         ILSMIndexOperationContext opCtx = accessor.getOpContext();
-        if (opCtx.getOperation() == IndexOperation.DELETE_MEMORY_COMPONENT) {
+        ILSMMemoryComponent memoryComponent = (ILSMMemoryComponent) opCtx.getComponentHolder().get(0);
+        if (memoryComponent != getOldestReadableMemoryComponent()) {
+            throw new IllegalStateException("An attempt to flush a memory component that is not the oldest");
+        }
+        if (!memoryComponent.isModified() || opCtx.getOperation() == IndexOperation.DELETE_COMPONENTS) {
             return EmptyComponent.INSTANCE;
         }
         if (LOGGER.isInfoEnabled()) {
             FlushOperation flushOp = (FlushOperation) operation;
-            LOGGER.log(Level.INFO, "Flushing component with id: " + flushOp.getFlushingComponent().getId());
+            LOGGER.log(Level.INFO,
+                    "Flushing component with id: " + flushOp.getFlushingComponent().getId() + " in the index " + this);
         }
-        try {
-            return doFlush(operation);
-        } catch (Exception e) {
-            LOGGER.error("Fail to execute flush " + this, e);
-            cleanUpFiles(operation, e);
-            throw HyracksDataException.create(e);
-        }
+        return doFlush(operation);
     }
 
     @Override
     public final ILSMDiskComponent merge(ILSMIOOperation operation) throws HyracksDataException {
         ILSMIndexAccessor accessor = operation.getAccessor();
         ILSMIndexOperationContext opCtx = accessor.getOpContext();
-        try {
-            return opCtx.getOperation() == IndexOperation.DELETE_DISK_COMPONENTS ? EmptyComponent.INSTANCE
-                    : doMerge(operation);
-        } catch (Exception e) {
-            LOGGER.error("Fail to execute merge " + this, e);
-            cleanUpFiles(operation, e);
-            throw HyracksDataException.create(e);
-        }
+        return opCtx.getOperation() == IndexOperation.DELETE_COMPONENTS ? EmptyComponent.INSTANCE : doMerge(operation);
     }
 
-    protected void cleanUpFiles(ILSMIOOperation operation, Exception e) {
+    @Override
+    public void cleanUpFilesForFailedOperation(ILSMIOOperation operation) {
         LSMComponentFileReferences componentFiles = operation.getComponentFiles();
         if (componentFiles == null) {
             return;
@@ -742,14 +831,24 @@
         for (FileReference file : files) {
             try {
                 if (file != null) {
+                    diskBufferCache.closeFileIfOpen(file);
                     diskBufferCache.deleteFile(file);
                 }
-            } catch (HyracksDataException hde) {
-                e.addSuppressed(hde);
+            } catch (Throwable th) { // NOSONAR Must catch all failures
+                operation.getFailure().addSuppressed(th);
             }
         }
     }
 
+    protected String getIndexIdentifier() {
+        return fileManager.getBaseDir().getAbsolutePath();
+    }
+
+    //Called when a memory component is reset
+    public void memoryComponentsReset() {
+        numScheduledFlushes = Integer.max(0, numScheduledFlushes - 1);
+    }
+
     protected abstract LSMComponentFileReferences getMergeFileReferences(ILSMDiskComponent firstComponent,
             ILSMDiskComponent lastComponent) throws HyracksDataException;
 
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/AbstractLSMIndexOperationContext.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/AbstractLSMIndexOperationContext.java
index c993874..2e133a6 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/AbstractLSMIndexOperationContext.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/AbstractLSMIndexOperationContext.java
@@ -20,6 +20,7 @@
 
 import java.util.ArrayList;
 import java.util.List;
+import java.util.Map;
 
 import org.apache.hyracks.api.dataflow.value.IBinaryComparatorFactory;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
@@ -28,7 +29,7 @@
 import org.apache.hyracks.storage.am.common.tuples.PermutingTupleReference;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponent;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMDiskComponent;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperation.LSMIOOperationType;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperation;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndex;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndexOperationContext;
 import org.apache.hyracks.storage.common.ISearchOperationCallback;
@@ -57,8 +58,8 @@
     private long enterExitTime = 0L;
     protected boolean skipFilter = false;
     protected boolean recovery = false;
-    private LSMIOOperationType ioOpType = LSMIOOperationType.NOOP;
-    private ILSMDiskComponent newDiskComponent;
+    private ILSMIOOperation ioOperation;
+    private Map<String, Object> parametersMap;
 
     public AbstractLSMIndexOperationContext(ILSMIndex index, int[] treeFields, int[] filterFields,
             IBinaryComparatorFactory[] filterCmpFactories, ISearchOperationCallback searchCallback,
@@ -210,22 +211,23 @@
     }
 
     @Override
-    public LSMIOOperationType getIoOperationType() {
-        return ioOpType;
+    public ILSMIOOperation getIoOperation() {
+        return ioOperation;
     }
 
     @Override
-    public void setIoOperationType(LSMIOOperationType ioOpType) {
-        this.ioOpType = ioOpType;
+    public void setIoOperation(ILSMIOOperation ioOperation) {
+        this.ioOperation = ioOperation;
     }
 
     @Override
-    public ILSMDiskComponent getNewComponent() {
-        return newDiskComponent;
+    public void setParameters(Map<String, Object> map) {
+        this.parametersMap = map;
     }
 
     @Override
-    public void setNewComponent(ILSMDiskComponent component) {
-        this.newDiskComponent = component;
+    public Map<String, Object> getParameters() {
+        return parametersMap;
     }
+
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/AbstractLSMMemoryComponent.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/AbstractLSMMemoryComponent.java
index 9596495..9ee1ff8 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/AbstractLSMMemoryComponent.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/AbstractLSMMemoryComponent.java
@@ -24,6 +24,7 @@
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponentFilter;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponentId;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponentId.IdCompareResult;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperation.LSMIOOperationType;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMMemoryComponent;
 import org.apache.hyracks.storage.am.lsm.common.api.IVirtualBufferCache;
 import org.apache.hyracks.storage.am.lsm.common.api.LSMOperationType;
@@ -39,31 +40,56 @@
     private final IVirtualBufferCache vbc;
     private final AtomicBoolean isModified;
     private int writerCount;
-    private boolean requestedToBeActive;
+    private int pendingFlushes = 0;
     private final MemoryComponentMetadata metadata;
     private ILSMComponentId componentId;
 
-    public AbstractLSMMemoryComponent(AbstractLSMIndex lsmIndex, IVirtualBufferCache vbc, boolean isActive,
-            ILSMComponentFilter filter) {
+    public AbstractLSMMemoryComponent(AbstractLSMIndex lsmIndex, IVirtualBufferCache vbc, ILSMComponentFilter filter) {
         super(lsmIndex, filter);
         this.vbc = vbc;
         writerCount = 0;
-        if (isActive) {
-            state = ComponentState.READABLE_WRITABLE;
-        } else {
-            state = ComponentState.INACTIVE;
-        }
+        state = ComponentState.INACTIVE;
         isModified = new AtomicBoolean();
         metadata = new MemoryComponentMetadata();
     }
 
+    /**
+     * Prepare the component to be scheduled for an IO operation
+     *
+     * @param ioOperationType
+     * @throws HyracksDataException
+     */
+    @Override
+    public void schedule(LSMIOOperationType ioOperationType) throws HyracksDataException {
+        activeate();
+        if (ioOperationType == LSMIOOperationType.FLUSH) {
+            if (state == ComponentState.READABLE_WRITABLE || state == ComponentState.READABLE_UNWRITABLE) {
+                if (writerCount != 0) {
+                    throw new IllegalStateException("Trying to schedule a flush when writerCount != 0");
+                }
+                state = ComponentState.READABLE_UNWRITABLE_FLUSHING;
+            } else if (state == ComponentState.READABLE_UNWRITABLE_FLUSHING
+                    || state == ComponentState.UNREADABLE_UNWRITABLE) {
+                // There is an ongoing flush. Increase pending flush count
+                pendingFlushes++;
+            } else {
+                throw new IllegalStateException("Trying to schedule a flush when the component state = " + state);
+            }
+        } else {
+            throw new UnsupportedOperationException("Unsupported operation " + ioOperationType);
+        }
+    }
+
+    private void activeate() throws HyracksDataException {
+        if (state == ComponentState.INACTIVE) {
+            state = ComponentState.READABLE_WRITABLE;
+            lsmIndex.getIOOperationCallback().recycled(this);
+        }
+    }
+
     @Override
     public boolean threadEnter(LSMOperationType opType, boolean isMutableComponent) throws HyracksDataException {
-        if (state == ComponentState.INACTIVE && requestedToBeActive) {
-            state = ComponentState.READABLE_WRITABLE;
-            requestedToBeActive = false;
-            lsmIndex.getIOOperationCallback().recycled(this, true);
-        }
+        activeate();
         switch (opType) {
             case FORCE_MODIFICATION:
                 if (isMutableComponent) {
@@ -97,7 +123,6 @@
                     }
                 }
                 break;
-            case REPLICATE:
             case SEARCH:
                 if (state == ComponentState.READABLE_WRITABLE || state == ComponentState.READABLE_UNWRITABLE
                         || state == ComponentState.READABLE_UNWRITABLE_FLUSHING) {
@@ -107,16 +132,18 @@
                 }
                 break;
             case FLUSH:
-                if (state == ComponentState.READABLE_WRITABLE || state == ComponentState.READABLE_UNWRITABLE) {
-                    if (writerCount != 0) {
-                        throw new IllegalStateException("Trying to flush when writerCount != 0");
-                    }
-                    state = ComponentState.READABLE_UNWRITABLE_FLUSHING;
-                    readerCount++;
-                } else {
+                if (state == ComponentState.UNREADABLE_UNWRITABLE) {
                     return false;
                 }
-                break;
+                if (state != ComponentState.READABLE_UNWRITABLE_FLUSHING) {
+                    throw new IllegalStateException("Trying to flush when component state = " + state);
+                }
+                if (writerCount != 0) {
+                    throw new IllegalStateException("Trying to flush when writerCount = " + writerCount);
+                }
+                readerCount++;
+                return true;
+
             default:
                 throw new UnsupportedOperationException("Unsupported operation " + opType);
         }
@@ -139,15 +166,14 @@
                 } else {
                     readerCount--;
                     if (state == ComponentState.UNREADABLE_UNWRITABLE && readerCount == 0) {
-                        state = ComponentState.INACTIVE;
+                        reset();
                     }
                 }
                 break;
-            case REPLICATE:
             case SEARCH:
                 readerCount--;
                 if (state == ComponentState.UNREADABLE_UNWRITABLE && readerCount == 0) {
-                    state = ComponentState.INACTIVE;
+                    reset();
                 }
                 break;
             case FLUSH:
@@ -156,12 +182,12 @@
                 }
                 readerCount--;
                 if (failedOperation) {
-                    // if flush failed, return the component state to READABLE_UNWRITABLE
-                    state = ComponentState.READABLE_UNWRITABLE;
+                    // If flush failed, keep the component state to READABLE_UNWRITABLE_FLUSHING
                     return;
                 }
+                // operation succeeded
                 if (readerCount == 0) {
-                    state = ComponentState.INACTIVE;
+                    reset();
                 } else {
                     state = ComponentState.UNREADABLE_UNWRITABLE;
                 }
@@ -177,20 +203,15 @@
 
     @Override
     public boolean isReadable() {
-        if (state == ComponentState.INACTIVE || state == ComponentState.UNREADABLE_UNWRITABLE) {
-            return false;
+        return state != ComponentState.INACTIVE && state != ComponentState.UNREADABLE_UNWRITABLE;
+    }
+
+    @Override
+    public void setUnwritable() {
+        if (state != ComponentState.READABLE_WRITABLE) {
+            throw new IllegalStateException("Attempt to set unwritable a component that is " + state);
         }
-        return true;
-    }
-
-    @Override
-    public void setState(ComponentState state) {
-        this.state = state;
-    }
-
-    @Override
-    public void requestActivation() {
-        requestedToBeActive = true;
+        this.state = ComponentState.READABLE_UNWRITABLE;
     }
 
     @Override
@@ -210,12 +231,23 @@
 
     @Override
     public final void reset() throws HyracksDataException {
+        state = ComponentState.INACTIVE;
         isModified.set(false);
         metadata.reset();
         if (filter != null) {
             filter.reset();
         }
         doReset();
+        lsmIndex.memoryComponentsReset();
+        // a flush can be pending on a component that just completed its flush... here is when this can happen:
+        // primary index has 2 components, secondary index has 2 components.
+        // 2 flushes are scheduled on each p1, p2, s1, and s2.
+        // p1 and p2 both finish. primary component 1 gets full and secondary doesn't have any entries (optional field).
+        // then flush is scheduled on p1, s1 will have a pending flush in that case.
+        if (pendingFlushes > 0) {
+            schedule(LSMIOOperationType.FLUSH);
+            pendingFlushes--;
+        }
     }
 
     protected void doReset() throws HyracksDataException {
@@ -267,6 +299,7 @@
     @Override
     public final void deallocate() throws HyracksDataException {
         try {
+            state = ComponentState.INACTIVE;
             doDeallocate();
         } finally {
             getIndex().getBufferCache().close();
@@ -297,7 +330,7 @@
 
     @Override
     public void resetId(ILSMComponentId componentId, boolean force) throws HyracksDataException {
-        if (!force && this.componentId != null && !componentId.missing() // for backward compatibility
+        if (!force && this.componentId != null
                 && this.componentId.compareTo(componentId) != IdCompareResult.LESS_THAN) {
             throw new IllegalStateException(
                     this + " receives illegal id. Old id " + this.componentId + ", new id " + componentId);
@@ -308,4 +341,11 @@
         this.componentId = componentId;
         LSMComponentIdUtils.persist(this.componentId, metadata);
     }
+
+    @Override
+    public String toString() {
+        return "{\"class\":\"" + getClass().getSimpleName() + "\", \"state\":\"" + state + "\", \"writers\":"
+                + writerCount + ", \"readers\":" + readerCount + ", \"pendingFlushes\":" + pendingFlushes
+                + ", \"id\":\"" + componentId + "\"}";
+    }
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/AbstractLSMWithBuddyMemoryComponent.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/AbstractLSMWithBuddyMemoryComponent.java
similarity index 86%
rename from hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/AbstractLSMWithBuddyMemoryComponent.java
rename to hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/AbstractLSMWithBuddyMemoryComponent.java
index 551303f..373d7e7 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/AbstractLSMWithBuddyMemoryComponent.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/AbstractLSMWithBuddyMemoryComponent.java
@@ -16,18 +16,18 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.hyracks.storage.am.lsm.common.api;
+package org.apache.hyracks.storage.am.lsm.common.impls;
 
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.storage.am.common.impls.AbstractTreeIndex;
-import org.apache.hyracks.storage.am.lsm.common.impls.AbstractLSMIndex;
-import org.apache.hyracks.storage.am.lsm.common.impls.AbstractLSMMemoryComponent;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponentFilter;
+import org.apache.hyracks.storage.am.lsm.common.api.IVirtualBufferCache;
 
 public abstract class AbstractLSMWithBuddyMemoryComponent extends AbstractLSMMemoryComponent {
 
-    public AbstractLSMWithBuddyMemoryComponent(AbstractLSMIndex lsmIndex, IVirtualBufferCache vbc, boolean isActive,
+    public AbstractLSMWithBuddyMemoryComponent(AbstractLSMIndex lsmIndex, IVirtualBufferCache vbc,
             ILSMComponentFilter filter) {
-        super(lsmIndex, vbc, isActive, filter);
+        super(lsmIndex, vbc, filter);
     }
 
     public abstract AbstractTreeIndex getBuddyIndex();
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/AsynchronousScheduler.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/AsynchronousScheduler.java
index a439ace..e4b845a 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/AsynchronousScheduler.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/AsynchronousScheduler.java
@@ -18,94 +18,80 @@
  */
 package org.apache.hyracks.storage.am.lsm.common.impls;
 
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.ArrayDeque;
+import java.util.Deque;
 import java.util.HashMap;
 import java.util.Map;
-import java.util.PriorityQueue;
-import java.util.concurrent.Callable;
 import java.util.concurrent.ExecutorService;
-import java.util.concurrent.RunnableFuture;
-import java.util.concurrent.SynchronousQueue;
 import java.util.concurrent.ThreadFactory;
-import java.util.concurrent.ThreadPoolExecutor;
-import java.util.concurrent.TimeUnit;
 
-import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.storage.am.lsm.common.api.IIoOperationFailedCallback;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperation;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperation.LSMIOOperationType;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperation.LSMIOOperationStatus;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationScheduler;
 
-public class AsynchronousScheduler implements ILSMIOOperationScheduler {
+public class AsynchronousScheduler implements ILSMIOOperationScheduler, Closeable {
     // Since this is a asynchronous scheduler, we make sure that flush operations coming from the same lsm index
     // will be executed serially in same order of scheduling the operations. Look at asterix issue 630.
 
-    public final static AsynchronousScheduler INSTANCE = new AsynchronousScheduler();
-    private ExecutorService executor;
-    private final Map<String, ILSMIOOperation> runningFlushOperations = new HashMap<String, ILSMIOOperation>();
-    private final Map<String, PriorityQueue<ILSMIOOperation>> waitingFlushOperations =
-            new HashMap<String, PriorityQueue<ILSMIOOperation>>();
+    private final ExecutorService executor;
+    private final Map<String, ILSMIOOperation> runningFlushOperations = new HashMap<>();
+    private final Map<String, Deque<ILSMIOOperation>> waitingFlushOperations = new HashMap<>();
+    private final Map<String, Throwable> failedGroups = new HashMap<>();
 
-    public void init(ThreadFactory threadFactory) {
-        // Creating an executor with the same configuration of Executors.newCachedThreadPool.
-        executor = new ThreadPoolExecutor(0, Integer.MAX_VALUE, 60L, TimeUnit.SECONDS, new SynchronousQueue<Runnable>(),
-                threadFactory) {
-
-            @Override
-            protected <T> RunnableFuture<T> newTaskFor(Callable<T> callable) {
-                return new LSMIOOperationTask<T>(callable);
-            }
-
-            @SuppressWarnings("unchecked")
-            @Override
-            protected void afterExecute(Runnable r, Throwable t) {
-                super.afterExecute(r, t);
-                LSMIOOperationTask<Boolean> task = (LSMIOOperationTask<Boolean>) r;
-                ILSMIOOperation executedOp = task.getOperation();
-                if (executedOp.getIOOpertionType() == LSMIOOperationType.FLUSH) {
-                    String id = executedOp.getIndexIdentifier();
-                    synchronized (this) {
-                        runningFlushOperations.remove(id);
-                        if (waitingFlushOperations.containsKey(id)) {
-                            try {
-                                ILSMIOOperation op = waitingFlushOperations.get(id).poll();
-                                if (op != null) {
-                                    scheduleOperation(op);
-                                } else {
-                                    waitingFlushOperations.remove(id);
-                                }
-                            } catch (HyracksDataException e) {
-                                t = e.getCause();
-                            }
-                        }
-                    }
-                }
-            }
-        };
+    public AsynchronousScheduler(ThreadFactory threadFactory, final IIoOperationFailedCallback callback) {
+        executor = new IoOperationExecutor(threadFactory, this, callback, runningFlushOperations,
+                waitingFlushOperations, failedGroups);
     }
 
     @Override
-    public void scheduleOperation(ILSMIOOperation operation) throws HyracksDataException {
-        if (operation.getIOOpertionType() == LSMIOOperationType.MERGE) {
-            executor.submit(operation);
-        } else if (operation.getIOOpertionType() == LSMIOOperationType.FLUSH) {
-            String id = operation.getIndexIdentifier();
-            synchronized (executor) {
-                if (runningFlushOperations.containsKey(id)) {
-                    if (waitingFlushOperations.containsKey(id)) {
-                        waitingFlushOperations.get(id).offer(operation);
-                    } else {
-                        PriorityQueue<ILSMIOOperation> q = new PriorityQueue<ILSMIOOperation>();
-                        q.offer(operation);
-                        waitingFlushOperations.put(id, q);
-                    }
-                } else {
-                    runningFlushOperations.put(id, operation);
-                    executor.submit(operation);
-                }
-            }
-        } else {
-            // this should never happen
-            // just guard here to avoid silient failures in case of future extensions
-            throw new IllegalArgumentException("Unknown operation type " + operation.getIOOpertionType());
+    public void scheduleOperation(ILSMIOOperation operation) {
+        switch (operation.getIOOpertionType()) {
+            case FLUSH:
+                scheduleFlush(operation);
+                break;
+            case MERGE:
+                executor.submit(operation);
+                break;
+            case NOOP:
+                return;
+            default:
+                // this should never happen
+                // just guard here to avoid silent failures in case of future extensions
+                throw new IllegalArgumentException("Unknown operation type " + operation.getIOOpertionType());
         }
     }
+
+    private void scheduleFlush(ILSMIOOperation operation) {
+        String id = operation.getIndexIdentifier();
+        synchronized (executor) {
+            if (failedGroups.containsKey(id)) {
+                // Group failure. Fail the operation right away
+                operation.setStatus(LSMIOOperationStatus.FAILURE);
+                operation.setFailure(new RuntimeException("Operation group " + id + " has permanently failed",
+                        failedGroups.get(id)));
+                operation.complete();
+                return;
+            }
+            if (runningFlushOperations.containsKey(id)) {
+                if (waitingFlushOperations.containsKey(id)) {
+                    waitingFlushOperations.get(id).offer(operation);
+                } else {
+                    Deque<ILSMIOOperation> q = new ArrayDeque<>();
+                    q.offer(operation);
+                    waitingFlushOperations.put(id, q);
+                }
+            } else {
+                runningFlushOperations.put(id, operation);
+                executor.submit(operation);
+            }
+        }
+    }
+
+    @Override
+    public void close() throws IOException {
+        executor.shutdown();
+    }
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/BlockingIOOperationCallbackWrapper.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/BlockingIOOperationCallbackWrapper.java
deleted file mode 100644
index a8ee286..0000000
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/BlockingIOOperationCallbackWrapper.java
+++ /dev/null
@@ -1,69 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.hyracks.storage.am.lsm.common.impls;
-
-import org.apache.hyracks.api.exceptions.HyracksDataException;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallback;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndexOperationContext;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMMemoryComponent;
-
-public class BlockingIOOperationCallbackWrapper implements ILSMIOOperationCallback {
-
-    private boolean notified = false;
-
-    private final ILSMIOOperationCallback wrappedCallback;
-
-    public BlockingIOOperationCallbackWrapper(ILSMIOOperationCallback callback) {
-        this.wrappedCallback = callback;
-    }
-
-    public synchronized void waitForIO() throws InterruptedException {
-        while (!notified) {
-            wait();
-        }
-        notified = false;
-    }
-
-    @Override
-    public void beforeOperation(ILSMIndexOperationContext opCtx) throws HyracksDataException {
-        wrappedCallback.beforeOperation(opCtx);
-    }
-
-    @Override
-    public void afterOperation(ILSMIndexOperationContext opCtx) throws HyracksDataException {
-        wrappedCallback.afterOperation(opCtx);
-    }
-
-    @Override
-    public synchronized void afterFinalize(ILSMIndexOperationContext opCtx) throws HyracksDataException {
-        wrappedCallback.afterFinalize(opCtx);
-        notifyAll();
-        notified = true;
-    }
-
-    @Override
-    public void recycled(ILSMMemoryComponent component, boolean componentSwitched) throws HyracksDataException {
-        wrappedCallback.recycled(component, componentSwitched);
-    }
-
-    @Override
-    public void allocated(ILSMMemoryComponent component) throws HyracksDataException {
-        wrappedCallback.allocated(component);
-    }
-}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/ChainedLSMDiskComponentBulkLoader.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/ChainedLSMDiskComponentBulkLoader.java
index abb0c76..232b42c 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/ChainedLSMDiskComponentBulkLoader.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/ChainedLSMDiskComponentBulkLoader.java
@@ -26,6 +26,7 @@
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMDiskComponent;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMDiskComponentBulkLoader;
 import org.apache.hyracks.util.annotations.CriticalPath;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperation;
 
 /**
  * Class encapsulates a chain of operations, happening during an LSM disk component bulkload
@@ -33,12 +34,15 @@
 public class ChainedLSMDiskComponentBulkLoader implements ILSMDiskComponentBulkLoader {
 
     private List<IChainedComponentBulkLoader> bulkloaderChain = new ArrayList<>();
-    private boolean isEmptyComponent = true;
-    private boolean cleanedUpArtifacts = false;
+    private final ILSMIOOperation operation;
     private final ILSMDiskComponent diskComponent;
     private final boolean cleanupEmptyComponent;
+    private boolean isEmptyComponent = true;
+    private boolean cleanedUpArtifacts = false;
 
-    public ChainedLSMDiskComponentBulkLoader(ILSMDiskComponent diskComponent, boolean cleanupEmptyComponent) {
+    public ChainedLSMDiskComponentBulkLoader(ILSMIOOperation operation, ILSMDiskComponent diskComponent,
+            boolean cleanupEmptyComponent) {
+        this.operation = operation;
         this.diskComponent = diskComponent;
         this.cleanupEmptyComponent = cleanupEmptyComponent;
     }
@@ -112,4 +116,9 @@
             lsmOperation.abort();
         }
     }
+
+    @Override
+    public ILSMIOOperation getOperation() {
+        return operation;
+    }
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/ComponentReplacementContext.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/ComponentReplacementContext.java
index 7bb24dc..dad3bdc 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/ComponentReplacementContext.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/ComponentReplacementContext.java
@@ -21,6 +21,7 @@
 import java.util.ArrayList;
 import java.util.Collections;
 import java.util.List;
+import java.util.Map;
 
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.storage.am.common.api.IExtendedModificationOperationCallback;
@@ -30,10 +31,9 @@
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponent.LSMComponentType;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponentId;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMDiskComponent;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperation.LSMIOOperationType;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperation;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndex;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndexOperationContext;
-import org.apache.hyracks.storage.common.IModificationOperationCallback;
 import org.apache.hyracks.storage.common.ISearchOperationCallback;
 import org.apache.hyracks.storage.common.ISearchPredicate;
 import org.apache.hyracks.storage.common.MultiComparator;
@@ -242,22 +242,22 @@
     }
 
     @Override
-    public LSMIOOperationType getIoOperationType() {
+    public ILSMIOOperation getIoOperation() {
         throw new UnsupportedOperationException();
     }
 
     @Override
-    public void setIoOperationType(LSMIOOperationType ioOpType) {
+    public void setIoOperation(ILSMIOOperation ioOperation) {
         throw new UnsupportedOperationException();
     }
 
     @Override
-    public ILSMDiskComponent getNewComponent() {
+    public void setParameters(Map<String, Object> map) {
         throw new UnsupportedOperationException();
     }
 
     @Override
-    public void setNewComponent(ILSMDiskComponent component) {
+    public Map<String, Object> getParameters() {
         throw new UnsupportedOperationException();
     }
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/ConstantMergePolicy.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/ConstantMergePolicy.java
index 13911ef..c642d82 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/ConstantMergePolicy.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/ConstantMergePolicy.java
@@ -49,10 +49,10 @@
             IIndexAccessParameters iap =
                     new IndexAccessParameters(NoOpOperationCallback.INSTANCE, NoOpOperationCallback.INSTANCE);
             ILSMIndexAccessor accessor = index.createAccessor(iap);
-            accessor.scheduleFullMerge(index.getIOOperationCallback());
+            accessor.scheduleFullMerge();
         } else if (immutableComponents.size() >= numComponents) {
             ILSMIndexAccessor accessor = index.createAccessor(NoOpIndexAccessParameters.INSTANCE);
-            accessor.scheduleMerge(index.getIOOperationCallback(), immutableComponents);
+            accessor.scheduleMerge(immutableComponents);
         }
     }
 
@@ -108,7 +108,7 @@
                 throw new IllegalStateException();
             }
             ILSMIndexAccessor accessor = index.createAccessor(NoOpIndexAccessParameters.INSTANCE);
-            accessor.scheduleMerge(index.getIOOperationCallback(), immutableComponents);
+            accessor.scheduleMerge(immutableComponents);
             return true;
         }
     }
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 466ef24..ced04e7 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
@@ -27,6 +27,7 @@
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponentFilter;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponentId;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMDiskComponent;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperation;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperation.LSMIOOperationType;
 import org.apache.hyracks.storage.am.lsm.common.api.LSMOperationType;
 import org.apache.hyracks.storage.common.IIndex;
@@ -145,9 +146,14 @@
     }
 
     @Override
-    public ChainedLSMDiskComponentBulkLoader createBulkLoader(LSMIOOperationType opType, float fillFactor,
+    public ChainedLSMDiskComponentBulkLoader createBulkLoader(ILSMIOOperation operation, float fillFactor,
             boolean verifyInput, long numElementsHint, boolean checkIfEmptyIndex, boolean withFilter,
             boolean cleanupEmptyComponent) throws HyracksDataException {
         return null;
     }
+
+    @Override
+    public void schedule(LSMIOOperationType ioOperationType) throws HyracksDataException {
+        // Do nothing
+    }
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/ExternalIndexHarness.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/ExternalIndexHarness.java
index 404c8e9..1ca91b3 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/ExternalIndexHarness.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/ExternalIndexHarness.java
@@ -28,8 +28,7 @@
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponent;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMDiskComponent;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperation;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperation.LSMIOOperationType;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallback;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationScheduler;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndex;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndexOperationContext;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMMergePolicy;
@@ -39,15 +38,11 @@
 import org.apache.hyracks.storage.common.IIndexCursor;
 import org.apache.hyracks.storage.common.ISearchPredicate;
 import org.apache.hyracks.util.trace.ITracer;
-import org.apache.logging.log4j.LogManager;
-import org.apache.logging.log4j.Logger;
 
 public class ExternalIndexHarness extends LSMHarness {
-    private static final Logger LOGGER = LogManager.getLogger();
-
-    public ExternalIndexHarness(ILSMIndex lsmIndex, ILSMMergePolicy mergePolicy, ILSMOperationTracker opTracker,
-            boolean replicationEnabled) {
-        super(lsmIndex, mergePolicy, opTracker, replicationEnabled, ITracer.NONE);
+    public ExternalIndexHarness(ILSMIndex lsmIndex, ILSMIOOperationScheduler ioScheduler, ILSMMergePolicy mergePolicy,
+            ILSMOperationTracker opTracker, boolean replicationEnabled) {
+        super(lsmIndex, ioScheduler, mergePolicy, opTracker, replicationEnabled, ITracer.NONE);
     }
 
     @Override
@@ -104,14 +99,6 @@
             }
             ctx.setAccessingComponents(true);
         }
-        // Check if there is any action that is needed to be taken based on the operation type
-        switch (opType) {
-            case MERGE:
-                ctx.setIoOperationType(LSMIOOperationType.MERGE);
-                lsmIndex.getIOOperationCallback().beforeOperation(ctx);
-            default:
-                break;
-        }
         opTracker.beforeOperation(lsmIndex, opType, ctx.getSearchOperationCallback(), ctx.getModificationCallback());
         return true;
     }
@@ -206,54 +193,6 @@
     }
 
     @Override
-    public void scheduleMerge(ILSMIndexOperationContext ctx, ILSMIOOperationCallback callback)
-            throws HyracksDataException {
-        if (!getAndEnterComponents(ctx, LSMOperationType.MERGE, true)) {
-            ctx.setIoOperationType(LSMIOOperationType.MERGE);
-            callback.afterFinalize(ctx);
-            return;
-        }
-        lsmIndex.scheduleMerge(ctx, callback);
-    }
-
-    @Override
-    public void scheduleFullMerge(ILSMIndexOperationContext ctx, ILSMIOOperationCallback callback)
-            throws HyracksDataException {
-        fullMergeIsRequested.set(true);
-        if (!getAndEnterComponents(ctx, LSMOperationType.MERGE, true)) {
-            // If the merge cannot be scheduled because there is already an ongoing merge on subset/all of the components, then
-            // whenever the current merge has finished, it will schedule the full merge again.
-            ctx.setIoOperationType(LSMIOOperationType.MERGE);
-            callback.afterFinalize(ctx);
-            return;
-        }
-        fullMergeIsRequested.set(false);
-        lsmIndex.scheduleMerge(ctx, callback);
-    }
-
-    @Override
-    public void merge(ILSMIndexOperationContext ctx, ILSMIOOperation operation) throws HyracksDataException {
-        if (LOGGER.isInfoEnabled()) {
-            LOGGER.info("Started a merge operation for index: " + lsmIndex + " ...");
-        }
-
-        ILSMDiskComponent newComponent = null;
-        try {
-            newComponent = lsmIndex.merge(operation);
-            ctx.setNewComponent(newComponent);
-            ctx.setIoOperationType(LSMIOOperationType.MERGE);
-            operation.getCallback().afterOperation(ctx);
-            newComponent.markAsValid(lsmIndex.isDurable());
-        } finally {
-            exitComponents(ctx, LSMOperationType.MERGE, newComponent, false);
-            operation.getCallback().afterFinalize(ctx);
-        }
-        if (LOGGER.isInfoEnabled()) {
-            LOGGER.info("Finished the merge operation for index: " + lsmIndex);
-        }
-    }
-
-    @Override
     public void addBulkLoadedComponent(ILSMDiskComponent c) throws HyracksDataException {
         c.markAsValid(lsmIndex.isDurable());
         synchronized (opTracker) {
@@ -304,14 +243,13 @@
     }
 
     @Override
-    public void scheduleFlush(ILSMIndexOperationContext ctx, ILSMIOOperationCallback callback)
-            throws HyracksDataException {
-        ctx.setIoOperationType(LSMIOOperationType.FLUSH);
-        callback.afterFinalize(ctx);
+    public ILSMIOOperation scheduleFlush(ILSMIndexOperationContext ctx) throws HyracksDataException {
+        return NoOpIoOperation.INSTANCE;
     }
 
     @Override
-    public void flush(ILSMIndexOperationContext ctx, ILSMIOOperation operation) throws HyracksDataException {
+    public void flush(ILSMIOOperation operation) throws HyracksDataException {
+        throw new UnsupportedOperationException();
     }
 
     @Override
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/FlushOperation.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/FlushOperation.java
index d835021..b61355f 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/FlushOperation.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/FlushOperation.java
@@ -35,9 +35,9 @@
     }
 
     @Override
-    public Boolean call() throws HyracksDataException {
+    public LSMIOOperationStatus call() throws HyracksDataException {
         accessor.flush(this);
-        return true;
+        return getStatus();
     }
 
     @Override
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/IoOperationExecutor.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/IoOperationExecutor.java
new file mode 100644
index 0000000..354b1af
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/IoOperationExecutor.java
@@ -0,0 +1,121 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.hyracks.storage.am.lsm.common.impls;
+
+import java.util.Deque;
+import java.util.Map;
+import java.util.concurrent.Callable;
+import java.util.concurrent.RunnableFuture;
+import java.util.concurrent.SynchronousQueue;
+import java.util.concurrent.ThreadFactory;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.storage.am.lsm.common.api.IIoOperationFailedCallback;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperation;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperation.LSMIOOperationStatus;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperation.LSMIOOperationType;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationScheduler;
+
+public class IoOperationExecutor extends ThreadPoolExecutor {
+
+    private final ILSMIOOperationScheduler scheduler;
+    private final IIoOperationFailedCallback callback;
+    private final Map<String, ILSMIOOperation> runningFlushOperations;
+    private final Map<String, Throwable> failedGroups;
+    private final Map<String, Deque<ILSMIOOperation>> waitingFlushOperations;
+
+    public IoOperationExecutor(ThreadFactory threadFactory, ILSMIOOperationScheduler scheduler,
+            IIoOperationFailedCallback callback, Map<String, ILSMIOOperation> runningFlushOperations,
+            Map<String, Deque<ILSMIOOperation>> waitingFlushOperations, Map<String, Throwable> failedGroups) {
+        super(0, Integer.MAX_VALUE, 60L, TimeUnit.SECONDS, new SynchronousQueue<>(), threadFactory);
+        this.scheduler = scheduler;
+        this.callback = callback;
+        this.runningFlushOperations = runningFlushOperations;
+        this.waitingFlushOperations = waitingFlushOperations;
+        this.failedGroups = failedGroups;
+    }
+
+    @Override
+    protected <T> RunnableFuture<T> newTaskFor(Callable<T> callable) {
+        return new LSMIOOperationTask<>(callable);
+    }
+
+    @Override
+    protected void afterExecute(Runnable r, Throwable t) {
+        super.afterExecute(r, t);
+        LSMIOOperationTask<?> task = (LSMIOOperationTask<?>) r;
+        ILSMIOOperation executedOp = task.getOperation();
+        try {
+            doAfterExecute(executedOp, t);
+        } catch (Throwable th) { // NOSONAR must catch all
+            callback.schedulerFailed(scheduler, th);
+            shutdown();
+        }
+    }
+
+    private void doAfterExecute(ILSMIOOperation executedOp, Throwable t) throws HyracksDataException {
+        final boolean failed = (t != null) || (executedOp.getStatus() == LSMIOOperationStatus.FAILURE);
+        if (failed) {
+            fail(executedOp, t != null ? t : executedOp.getFailure());
+        }
+        if (!failed || executedOp.getIOOpertionType() != LSMIOOperationType.FLUSH) {
+            executedOp.complete(); // destroy if merge or successful flush
+        }
+        if (executedOp.getIOOpertionType() == LSMIOOperationType.FLUSH) {
+            String id = executedOp.getIndexIdentifier();
+            synchronized (this) {
+                runningFlushOperations.remove(id);
+                if (waitingFlushOperations.containsKey(id)) {
+                    ILSMIOOperation op = waitingFlushOperations.get(id).poll();
+                    if (op != null) {
+                        scheduler.scheduleOperation(op);
+                    } else {
+                        waitingFlushOperations.remove(id);
+                    }
+                }
+            }
+        }
+    }
+
+    private void fail(ILSMIOOperation executedOp, Throwable t) throws HyracksDataException {
+        callback.operationFailed(executedOp, t);
+        if (executedOp.getIOOpertionType() == LSMIOOperationType.FLUSH) {
+            executedOp.complete();
+            // Doesn't make sense to process further flush requests... Mark the operation group permanently failed
+            // Fail other scheduled operations
+            synchronized (this) {
+                String id = executedOp.getIndexIdentifier();
+                failedGroups.put(id, t);
+                runningFlushOperations.remove(id);
+                if (waitingFlushOperations.containsKey(id)) {
+                    Deque<ILSMIOOperation> ops = waitingFlushOperations.remove(id);
+                    ILSMIOOperation next = ops.poll();
+                    while (next != null) {
+                        next.setFailure(new RuntimeException("Operation group " + id + " has permanently failed", t));
+                        next.setStatus(LSMIOOperationStatus.FAILURE);
+                        next.complete();
+                        next = ops.poll();
+                    }
+                }
+            }
+        }
+    }
+}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/LSMComponentIdGenerator.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/LSMComponentIdGenerator.java
index e174153..3da57fd 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/LSMComponentIdGenerator.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/LSMComponentIdGenerator.java
@@ -28,18 +28,22 @@
  */
 public class LSMComponentIdGenerator implements ILSMComponentIdGenerator {
 
+    private final int numComponents;
+    private int currentComponentIndex;
     protected long previousTimestamp = -1L;
-
     private ILSMComponentId componentId;
 
-    public LSMComponentIdGenerator() {
+    public LSMComponentIdGenerator(int numComponents) {
+        this.numComponents = numComponents;
         refresh();
+        currentComponentIndex = 0;
     }
 
     @Override
     public void refresh() {
         long ts = getCurrentTimestamp();
         componentId = new LSMComponentId(ts, ts);
+        currentComponentIndex = (currentComponentIndex + 1) % numComponents;
     }
 
     @Override
@@ -47,6 +51,11 @@
         return componentId;
     }
 
+    @Override
+    public int getCurrentComponentIndex() {
+        return currentComponentIndex;
+    }
+
     protected long getCurrentTimestamp() {
         long timestamp = System.currentTimeMillis();
         while (timestamp <= previousTimestamp) {
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/LSMComponentIdGeneratorFactory.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/LSMComponentIdGeneratorFactory.java
deleted file mode 100644
index d288ec8..0000000
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/LSMComponentIdGeneratorFactory.java
+++ /dev/null
@@ -1,40 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-package org.apache.hyracks.storage.am.lsm.common.impls;
-
-import org.apache.hyracks.api.application.INCServiceContext;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponentIdGenerator;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponentIdGeneratorFactory;
-import org.apache.hyracks.storage.common.IResource;
-
-/**
- * A default implementation of {@link ILSMComponentIdGeneratorFactory}.
- *
- */
-public class LSMComponentIdGeneratorFactory implements ILSMComponentIdGeneratorFactory {
-
-    private static final long serialVersionUID = 1L;
-
-    @Override
-    public ILSMComponentIdGenerator getComponentIdGenerator(INCServiceContext serviceCtx, IResource resource) {
-        return new LSMComponentIdGenerator();
-    }
-
-}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/LSMHarness.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/LSMHarness.java
index 1dbaa3c..2a3e983 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/LSMHarness.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/LSMHarness.java
@@ -42,8 +42,9 @@
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMDiskComponent;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMHarness;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperation;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperation.LSMIOOperationStatus;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperation.LSMIOOperationType;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallback;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationScheduler;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndex;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndexAccessor;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndexOperationContext;
@@ -51,7 +52,6 @@
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMMergePolicy;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMOperationTracker;
 import org.apache.hyracks.storage.am.lsm.common.api.LSMOperationType;
-import org.apache.hyracks.storage.am.lsm.common.util.IOOperationUtils;
 import org.apache.hyracks.storage.common.IIndexCursor;
 import org.apache.hyracks.storage.common.ISearchPredicate;
 import org.apache.hyracks.util.annotations.CriticalPath;
@@ -65,6 +65,7 @@
     private static final Logger LOGGER = LogManager.getLogger();
 
     protected final ILSMIndex lsmIndex;
+    protected final ILSMIOOperationScheduler ioScheduler;
     protected final ComponentReplacementContext componentReplacementCtx;
     protected final ILSMMergePolicy mergePolicy;
     protected final ILSMOperationTracker opTracker;
@@ -74,9 +75,10 @@
     protected ITracer tracer;
     protected long traceCategory;
 
-    public LSMHarness(ILSMIndex lsmIndex, ILSMMergePolicy mergePolicy, ILSMOperationTracker opTracker,
-            boolean replicationEnabled, ITracer tracer) {
+    public LSMHarness(ILSMIndex lsmIndex, ILSMIOOperationScheduler ioScheduler, ILSMMergePolicy mergePolicy,
+            ILSMOperationTracker opTracker, boolean replicationEnabled, ITracer tracer) {
         this.lsmIndex = lsmIndex;
+        this.ioScheduler = ioScheduler;
         this.opTracker = opTracker;
         this.mergePolicy = mergePolicy;
         this.tracer = tracer;
@@ -101,58 +103,12 @@
             synchronized (opTracker) {
                 while (true) {
                     lsmIndex.getOperationalComponents(ctx);
-                    // Before entering the components, prune those corner cases that indeed should not proceed.
-                    switch (opType) {
-                        case FLUSH:
-                            // if the lsm index does not have memory components allocated, then nothing to flush
-                            if (!lsmIndex.isMemoryComponentsAllocated()) {
-                                return false;
-                            }
-                            ILSMMemoryComponent flushingComponent =
-                                    (ILSMMemoryComponent) ctx.getComponentHolder().get(0);
-                            if (!flushingComponent.isModified()) {
-                                recycle(flushingComponent);
-                                return false;
-                            }
-                            if (flushingComponent.getWriterCount() > 0) {
-                                /*
-                                 * This case is a case where even though FLUSH log was flushed to disk
-                                 * and scheduleFlush is triggered, the current in-memory component (whose state was
-                                 * changed to READABLE_WRITABLE (RW) from READABLE_UNWRITABLE(RU) before FLUSH log
-                                 * was written to log tail (which is memory buffer of log file) and then the state was
-                                 * changed back to RW (as shown in the following scenario)) can have writers based on
-                                 * the current code base/design. Thus, the writer count of the component may be greater
-                                 * than 0. if this happens, intead of throwing exception, scheduleFlush() deal with
-                                 * this situation by not flushing the component.
-                                 * for more detailed information: ASTERIXDB-1027
-                                 */
-                                return false;
-                            }
-                            break;
-                        case MERGE:
-                            if (ctx.getComponentHolder().size() < 2
-                                    && ctx.getOperation() != IndexOperation.DELETE_DISK_COMPONENTS) {
-                                // There is only a single component. There is nothing to merge.
-                                return false;
-                            }
-                            break;
-                        default:
-                            break;
-                    }
                     if (enterComponents(ctx, opType)) {
                         return true;
                     } else if (isTryOperation) {
                         return false;
                     }
                     try {
-                        // Flush and merge operations should never reach this wait call,
-                        // because they are always try operations. If they fail to enter the components,
-                        // then it means that there are an ongoing flush/merge operation on
-                        // the same components, so they should not proceed.
-                        if (opType == LSMOperationType.MODIFICATION) {
-                            // before waiting, make sure the index is in a modifiable state to avoid waiting forever.
-                            ensureIndexModifiable();
-                        }
                         opTracker.wait();
                     } catch (InterruptedException e) {
                         Thread.currentThread().interrupt();
@@ -167,20 +123,6 @@
         }
     }
 
-    private void recycle(ILSMMemoryComponent flushingComponent) throws HyracksDataException {
-        if (flushingComponent.getState() == ComponentState.READABLE_UNWRITABLE) {
-            //The mutable component has not been modified by any writer.
-            // There is nothing to flush. Since the component is empty, set its state back
-            // to READABLE_WRITABLE only when it's state has been set to READABLE_UNWRITABLE
-            flushingComponent.setState(ComponentState.READABLE_WRITABLE);
-            opTracker.notifyAll(); // NOSONAR: Always synchronized from caller
-            // Call recycled only when we change it's state is reset back to READABLE_WRITABLE
-            // Otherwise, if the component is in other state, e.g., INACTIVE, or
-            // READABLE_UNWRITABLE_FLUSHING, it's not considered as being recycled here.
-            lsmIndex.getIOOperationCallback().recycled(flushingComponent, false);
-        }
-    }
-
     @CriticalPath
     protected boolean enterComponents(ILSMIndexOperationContext ctx, LSMOperationType opType)
             throws HyracksDataException {
@@ -222,25 +164,6 @@
             return false;
         }
         ctx.setAccessingComponents(true);
-        // Check if there is any action that is needed to be taken based on the operation type
-        switch (opType) {
-            case FLUSH:
-                ctx.setIoOperationType(LSMIOOperationType.FLUSH);
-                lsmIndex.getIOOperationCallback().beforeOperation(ctx);
-                // Changing the flush status should *always* precede changing the mutable component.
-                lsmIndex.changeFlushStatusForCurrentMutableCompoent(false);
-                lsmIndex.changeMutableComponent();
-                // Notify all waiting threads whenever a flush has been scheduled since they will check
-                // again if they can grab and enter the mutable component.
-                opTracker.notifyAll(); // NOSONAR: Always called from a synchronized block
-                break;
-            case MERGE:
-                ctx.setIoOperationType(LSMIOOperationType.MERGE);
-                lsmIndex.getIOOperationCallback().beforeOperation(ctx);
-                break;
-            default:
-                break;
-        }
         opTracker.beforeOperation(lsmIndex, opType, ctx.getSearchOperationCallback(), ctx.getModificationCallback());
         return true;
     }
@@ -394,7 +317,6 @@
                         break;
                     case INACTIVE:
                         tracer.instant(c.toString(), traceCategory, Scope.p, lsmIndex.toString());
-                        ((AbstractLSMMemoryComponent) c).reset();
                         // Notify all waiting threads whenever the mutable component's state
                         // has changed to inactive. This is important because even though we switched
                         // the mutable components, it is possible that the component that we just
@@ -556,129 +478,135 @@
     }
 
     @Override
-    public void scheduleFlush(ILSMIndexOperationContext ctx, ILSMIOOperationCallback callback)
-            throws HyracksDataException {
-        if (!getAndEnterComponents(ctx, LSMOperationType.FLUSH, true)) {
-            ctx.setIoOperationType(LSMIOOperationType.FLUSH);
-            callback.afterFinalize(ctx);
-            return;
+    public ILSMIOOperation scheduleFlush(ILSMIndexOperationContext ctx) throws HyracksDataException {
+        ILSMIOOperation flush;
+        LOGGER.info("Flush is being scheduled on {}", lsmIndex);
+        synchronized (opTracker) {
+            if (!lsmIndex.isMemoryComponentsAllocated()) {
+                lsmIndex.allocateMemoryComponents();
+            }
+            try {
+                flush = lsmIndex.createFlushOperation(ctx);
+            } finally {
+                // Notify all waiting threads whenever a flush has been scheduled since they will check
+                // again if they can grab and enter the mutable component.
+                opTracker.notifyAll();
+            }
         }
-        lsmIndex.scheduleFlush(ctx, callback);
+        ioScheduler.scheduleOperation(flush);
+        return flush;
     }
 
+    @SuppressWarnings("squid:S2142")
     @Override
-    public void flush(ILSMIndexOperationContext ctx, ILSMIOOperation operation) throws HyracksDataException {
+    public void flush(ILSMIOOperation operation) throws HyracksDataException {
         if (LOGGER.isInfoEnabled()) {
-            LOGGER.info("Started a flush operation for index: " + lsmIndex + " ...");
+            LOGGER.info("Started a flush operation for index: {}", lsmIndex);
+        }
+        synchronized (opTracker) {
+            while (!enterComponents(operation.getAccessor().getOpContext(), LSMOperationType.FLUSH)) {
+                try {
+                    opTracker.wait();
+                } catch (InterruptedException e) {
+                    Thread.currentThread().interrupt();
+                    throw HyracksDataException.create(e);
+                }
+            }
         }
         try {
-            ILSMDiskComponent newComponent = null;
-            boolean failedOperation = false;
-            try {
-                newComponent = lsmIndex.flush(operation);
-                ctx.setNewComponent(newComponent);
-                ctx.setIoOperationType(LSMIOOperationType.FLUSH);
-                operation.getCallback().afterOperation(ctx);
-                newComponent.markAsValid(lsmIndex.isDurable());
-            } catch (Throwable e) { // NOSONAR Log and re-throw
-                failedOperation = true;
-                if (LOGGER.isErrorEnabled()) {
-                    LOGGER.log(Level.ERROR, "Flush failed on " + lsmIndex, e);
-                }
-                throw e;
-            } finally {
-                exitComponents(ctx, LSMOperationType.FLUSH, newComponent, failedOperation);
-                ctx.setIoOperationType(LSMIOOperationType.FLUSH);
-                operation.getCallback().afterFinalize(ctx);
-
-            }
+            doIo(operation);
         } finally {
-            /*
-             * Completion of flush/merge operations is done explicitly here to make sure all generated files during
-             * io operations is completed before the io operation is declared complete
-             */
-            opTracker.completeOperation(lsmIndex, LSMOperationType.FLUSH, ctx.getSearchOperationCallback(),
-                    ctx.getModificationCallback());
+            exitComponents(operation.getAccessor().getOpContext(), LSMOperationType.FLUSH, operation.getNewComponent(),
+                    operation.getStatus() == LSMIOOperationStatus.FAILURE);
+            opTracker.completeOperation(lsmIndex, LSMOperationType.FLUSH,
+                    operation.getAccessor().getOpContext().getSearchOperationCallback(),
+                    operation.getAccessor().getOpContext().getModificationCallback());
         }
         if (LOGGER.isInfoEnabled()) {
-            LOGGER.info("Finished the flush operation for index: " + lsmIndex);
+            LOGGER.info("Finished the flush operation for index: {}. Result: ", lsmIndex, operation.getStatus());
+        }
+    }
+
+    public void doIo(ILSMIOOperation operation) {
+        try {
+            operation.getCallback().beforeOperation(operation);
+            ILSMDiskComponent newComponent = operation.getIOOpertionType() == LSMIOOperationType.FLUSH
+                    ? lsmIndex.flush(operation) : lsmIndex.merge(operation);
+            operation.setNewComponent(newComponent);
+            operation.getCallback().afterOperation(operation);
+            if (newComponent != null) {
+                newComponent.markAsValid(lsmIndex.isDurable());
+            }
+        } catch (Throwable e) { // NOSONAR Must catch all
+            operation.setStatus(LSMIOOperationStatus.FAILURE);
+            operation.setFailure(e);
+            if (LOGGER.isErrorEnabled()) {
+                LOGGER.log(Level.ERROR, "{} operation failed on {}", operation.getIOOpertionType(), lsmIndex, e);
+            }
+        } finally {
+            try {
+                operation.getCallback().afterFinalize(operation);
+            } catch (Throwable th) {// NOSONAR Must catch all
+                operation.setStatus(LSMIOOperationStatus.FAILURE);
+                operation.setFailure(th);
+                if (LOGGER.isErrorEnabled()) {
+                    LOGGER.log(Level.ERROR, "{} operation.afterFinalize failed on {}", operation.getIOOpertionType(),
+                            lsmIndex, th);
+                }
+            }
+        }
+        // if the operation failed, we need to cleanup files
+        if (operation.getStatus() == LSMIOOperationStatus.FAILURE) {
+            lsmIndex.cleanUpFilesForFailedOperation(operation);
         }
     }
 
     @Override
-    public void scheduleMerge(ILSMIndexOperationContext ctx, ILSMIOOperationCallback callback)
-            throws HyracksDataException {
-        if (!getAndEnterComponents(ctx, LSMOperationType.MERGE, true)) {
-            LOGGER.info("Failed to enter components for merge operation. Calling finalize");
-            ctx.setIoOperationType(LSMIOOperationType.MERGE);
-            callback.afterFinalize(ctx);
-            return;
+    public void merge(ILSMIOOperation operation) throws HyracksDataException {
+        if (LOGGER.isInfoEnabled()) {
+            LOGGER.info("Started a merge operation for index: {}", lsmIndex);
         }
-        lsmIndex.scheduleMerge(ctx, callback);
+        enterComponents(operation.getAccessor().getOpContext(), LSMOperationType.MERGE);
+        try {
+            doIo(operation);
+        } finally {
+            exitComponents(operation.getAccessor().getOpContext(), LSMOperationType.MERGE, operation.getNewComponent(),
+                    operation.getStatus() == LSMIOOperationStatus.FAILURE);
+            opTracker.completeOperation(lsmIndex, LSMOperationType.MERGE,
+                    operation.getAccessor().getOpContext().getSearchOperationCallback(),
+                    operation.getAccessor().getOpContext().getModificationCallback());
+        }
+        if (LOGGER.isInfoEnabled()) {
+            LOGGER.info("Finished the merge operation for index: {}. Result: ", lsmIndex, operation.getStatus());
+        }
     }
 
     @Override
-    public void scheduleFullMerge(ILSMIndexOperationContext ctx, ILSMIOOperationCallback callback)
-            throws HyracksDataException {
-        fullMergeIsRequested.set(true);
-        if (!getAndEnterComponents(ctx, LSMOperationType.MERGE, true)) {
+    public ILSMIOOperation scheduleMerge(ILSMIndexOperationContext ctx) throws HyracksDataException {
+        ILSMIOOperation operation;
+        synchronized (opTracker) {
+            operation = lsmIndex.createMergeOperation(ctx);
+        }
+        ioScheduler.scheduleOperation(operation);
+        return operation;
+    }
+
+    @Override
+    public ILSMIOOperation scheduleFullMerge(ILSMIndexOperationContext ctx) throws HyracksDataException {
+        ILSMIOOperation operation;
+        synchronized (opTracker) {
+            fullMergeIsRequested.set(true);
+            ctx.getComponentsToBeMerged().addAll(lsmIndex.getDiskComponents());
+            operation = lsmIndex.createMergeOperation(ctx);
+            if (operation != NoOpIoOperation.INSTANCE) {
+                fullMergeIsRequested.set(false);
+            }
             // If the merge cannot be scheduled because there is already an ongoing merge on
             // subset/all of the components, then whenever the current merge has finished,
             // it will schedule the full merge again.
-            ctx.setIoOperationType(LSMIOOperationType.MERGE);
-            callback.afterFinalize(ctx);
-            return;
         }
-        fullMergeIsRequested.set(false);
-        lsmIndex.scheduleMerge(ctx, callback);
-    }
-
-    @Override
-    public void merge(ILSMIndexOperationContext ctx, ILSMIOOperation operation) throws HyracksDataException {
-        if (LOGGER.isInfoEnabled()) {
-            LOGGER.info("Started a merge operation for index: " + lsmIndex + " ...");
-        }
-        try {
-            ILSMDiskComponent newComponent = null;
-            boolean failedOperation = false;
-            try {
-                newComponent = lsmIndex.merge(operation);
-                ctx.setNewComponent(newComponent);
-                ctx.setIoOperationType(LSMIOOperationType.MERGE);
-                operation.getCallback().afterOperation(ctx);
-                newComponent.markAsValid(lsmIndex.isDurable());
-            } catch (Throwable e) { // NOSONAR: Log and re-throw
-                failedOperation = true;
-                if (LOGGER.isErrorEnabled()) {
-                    LOGGER.log(Level.ERROR, "Failed merge operation on " + lsmIndex, e);
-                }
-                throw e;
-            } finally {
-                exitComponents(ctx, LSMOperationType.MERGE, newComponent, failedOperation);
-                operation.getCallback().afterFinalize(ctx);
-            }
-        } finally {
-            /*
-             * Completion of the merge operation is called here to and not on afterOperation because
-             * deletion of old components comes after afterOperation is called and the number of
-             * io operation should not be decremented before the operation is complete to avoid
-             * index destroy from competing with the merge on deletion of the files.
-             * The order becomes:
-             * 1. scheduleMerge
-             * 2. enterComponents
-             * 3. beforeOperation (increment the numOfIoOperations)
-             * 4. merge
-             * 5. exitComponents
-             * 6. afterOperation (no op)
-             * 7. delete components
-             * 8. completeOperation (decrement the numOfIoOperations)
-             */
-            opTracker.completeOperation(lsmIndex, LSMOperationType.MERGE, ctx.getSearchOperationCallback(),
-                    ctx.getModificationCallback());
-        }
-        if (LOGGER.isInfoEnabled()) {
-            LOGGER.info("Finished the merge operation for index: " + lsmIndex);
-        }
+        ioScheduler.scheduleOperation(operation);
+        return operation;
     }
 
     @Override
@@ -779,36 +707,6 @@
         }
     }
 
-    /***
-     * Ensures the index is in a modifiable state (no failed flushes)
-     *
-     * @throws HyracksDataException
-     *             if the index is not in a modifiable state
-     */
-    private void ensureIndexModifiable() throws HyracksDataException {
-        // if current memory component has a flush request, it means that flush didn't start for it
-        if (lsmIndex.hasFlushRequestForCurrentMutableComponent()) {
-            return;
-        }
-        // find if there is any memory component which is in a writable state or eventually will be in a writable state
-        for (ILSMMemoryComponent memoryComponent : lsmIndex.getMemoryComponents()) {
-            switch (memoryComponent.getState()) {
-                case INACTIVE:
-                    // will be activated on next modification
-                case UNREADABLE_UNWRITABLE:
-                    // flush completed successfully but readers are still inside
-                case READABLE_WRITABLE:
-                    // writable
-                case READABLE_UNWRITABLE_FLUSHING:
-                    // flush is ongoing
-                    return;
-                default:
-                    // continue to the next component
-            }
-        }
-        throw HyracksDataException.create(ErrorCode.CANNOT_MODIFY_INDEX_DISK_IS_FULL);
-    }
-
     /**
      * Waits for any lagging merge operations to finish to avoid breaking
      * the merge policy (i.e. adding a new disk component can make the
@@ -832,12 +730,12 @@
         }
     }
 
+    @SuppressWarnings("squid:S2142")
     @Override
     public void deleteComponents(ILSMIndexOperationContext ctx, Predicate<ILSMComponent> predicate)
             throws HyracksDataException {
-        BlockingIOOperationCallbackWrapper ioCallback =
-                new BlockingIOOperationCallbackWrapper(lsmIndex.getIOOperationCallback());
         boolean deleteMemoryComponent;
+        ILSMIOOperation ioOperation = null;
         synchronized (opTracker) {
             waitForFlushesAndMerges();
             ensureNoFailedFlush();
@@ -847,19 +745,25 @@
             if (deleteMemoryComponent) {
                 // schedule a delete for flushed component
                 ctx.reset();
-                ctx.setOperation(IndexOperation.DELETE_MEMORY_COMPONENT);
-                // ScheduleFlush is actually a try operation
-                scheduleFlush(ctx, ioCallback);
+                ctx.setOperation(IndexOperation.DELETE_COMPONENTS);
+                ioOperation = scheduleFlush(ctx);
             }
         }
         // Here, we are releasing the opTracker to allow other operations:
         // (searches, delete flush we will schedule, delete merge we will schedule).
         if (deleteMemoryComponent) {
-            IOOperationUtils.waitForIoOperation(ioCallback);
+            try {
+                ioOperation.sync();
+            } catch (InterruptedException e) {
+                Thread.currentThread().interrupt();
+                throw HyracksDataException.create(e);
+            }
+            if (ioOperation.getStatus() == LSMIOOperationStatus.FAILURE) {
+                throw HyracksDataException.create(ioOperation.getFailure());
+            }
         }
         ctx.reset();
-        ioCallback = new BlockingIOOperationCallbackWrapper(lsmIndex.getIOOperationCallback());
-        ctx.setOperation(IndexOperation.DELETE_DISK_COMPONENTS);
+        ctx.setOperation(IndexOperation.DELETE_COMPONENTS);
         List<ILSMDiskComponent> toBeDeleted;
         synchronized (opTracker) {
             waitForFlushesAndMerges();
@@ -877,9 +781,17 @@
             }
             toBeDeleted = new ArrayList<>(ctx.getComponentsToBeMerged());
             // ScheduleMerge is actually a try operation
-            scheduleMerge(ctx, ioCallback);
+            ioOperation = scheduleMerge(ctx);
         }
-        IOOperationUtils.waitForIoOperation(ioCallback);
+        try {
+            ioOperation.sync();
+        } catch (InterruptedException e) {
+            Thread.currentThread().interrupt();
+            throw HyracksDataException.create(e);
+        }
+        if (ioOperation.getStatus() == LSMIOOperationStatus.FAILURE) {
+            throw HyracksDataException.create(ioOperation.getFailure());
+        }
         synchronized (opTracker) {
             // ensure that merge has succeeded
             for (ILSMDiskComponent component : toBeDeleted) {
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 2ef6169..416106d 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
@@ -22,7 +22,7 @@
 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;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperation.LSMIOOperationType;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperation.LSMIOOperationStatus;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndexOperationContext;
 import org.apache.hyracks.storage.common.IIndexBulkLoader;
 
@@ -35,15 +35,14 @@
             boolean verifyInput, long numElementsHint) throws HyracksDataException {
         this.lsmIndex = lsmIndex;
         this.opCtx = opCtx;
-        // Note that by using a flush target file name, we state that the
-        // new bulk loaded component is "newer" than any other merged component.
-        opCtx.setNewComponent(lsmIndex.createBulkLoadTarget());
-        this.componentBulkLoader = opCtx.getNewComponent().createBulkLoader(LSMIOOperationType.LOAD, fillFactor,
-                verifyInput, numElementsHint, false, true, true);
+        opCtx.getIoOperation().setNewComponent(lsmIndex.createBulkLoadTarget());
+        this.componentBulkLoader = opCtx.getIoOperation().getNewComponent().createBulkLoader(opCtx.getIoOperation(),
+                fillFactor, verifyInput, numElementsHint, false, true, true);
+        lsmIndex.getIOOperationCallback().beforeOperation(opCtx.getIoOperation());
     }
 
     public ILSMDiskComponent getComponent() {
-        return opCtx.getNewComponent();
+        return opCtx.getIoOperation().getNewComponent();
     }
 
     @Override
@@ -58,27 +57,38 @@
     @Override
     public void end() throws HyracksDataException {
         try {
-            componentBulkLoader.end();
-            if (opCtx.getNewComponent().getComponentSize() > 0) {
-                //TODO(amoudi): Ensure Bulk load follow the same lifecycle Other Operations (Flush, Merge, etc).
-                //then after operation should be called from harness as well
-                //https://issues.apache.org/jira/browse/ASTERIXDB-1764
-                lsmIndex.getIOOperationCallback().afterOperation(opCtx);
-                lsmIndex.getHarness().addBulkLoadedComponent(opCtx.getNewComponent());
+            try {
+                componentBulkLoader.end();
+                if (opCtx.getIoOperation().getNewComponent().getComponentSize() > 0) {
+                    lsmIndex.getIOOperationCallback().afterOperation(opCtx.getIoOperation());
+                }
+            } catch (Throwable th) { // NOSONAR Must not call afterFinalize without setting failure
+                opCtx.getIoOperation().setStatus(LSMIOOperationStatus.FAILURE);
+                opCtx.getIoOperation().setFailure(th);
+                throw th;
+            } finally {
+                lsmIndex.getIOOperationCallback().afterFinalize(opCtx.getIoOperation());
+            }
+            if (opCtx.getIoOperation().getNewComponent().getComponentSize() > 0) {
+                lsmIndex.getHarness().addBulkLoadedComponent(opCtx.getIoOperation().getNewComponent());
             }
         } finally {
-            lsmIndex.getIOOperationCallback().afterFinalize(opCtx);
+            lsmIndex.getIOOperationCallback().completed(opCtx.getIoOperation());
         }
     }
 
     @Override
     public void abort() throws HyracksDataException {
+        opCtx.getIoOperation().setStatus(LSMIOOperationStatus.FAILURE);
+        opCtx.getIoOperation().setNewComponent(null);
         try {
-            componentBulkLoader.abort();
-            opCtx.setNewComponent(null);
-            lsmIndex.getIOOperationCallback().afterOperation(opCtx);
+            try {
+                componentBulkLoader.abort();
+            } finally {
+                lsmIndex.getIOOperationCallback().afterFinalize(opCtx.getIoOperation());
+            }
         } finally {
-            lsmIndex.getIOOperationCallback().afterFinalize(opCtx);
+            lsmIndex.getIOOperationCallback().completed(opCtx.getIoOperation());
         }
     }
 
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/LSMTreeIndexAccessor.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/LSMTreeIndexAccessor.java
index 1548f86..f2c8d35 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/LSMTreeIndexAccessor.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/LSMTreeIndexAccessor.java
@@ -34,7 +34,6 @@
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMDiskComponent;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMHarness;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperation;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallback;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndexAccessor;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndexOperationContext;
 import org.apache.hyracks.storage.am.lsm.common.api.LSMOperationType;
@@ -120,12 +119,12 @@
 
     @Override
     public void flush(ILSMIOOperation operation) throws HyracksDataException {
-        lsmHarness.flush(ctx, operation);
+        lsmHarness.flush(operation);
     }
 
     @Override
     public void merge(ILSMIOOperation operation) throws HyracksDataException {
-        lsmHarness.merge(ctx, operation);
+        lsmHarness.merge(operation);
     }
 
     @Override
@@ -135,18 +134,17 @@
     }
 
     @Override
-    public void scheduleFlush(ILSMIOOperationCallback callback) throws HyracksDataException {
+    public ILSMIOOperation scheduleFlush() throws HyracksDataException {
         ctx.setOperation(IndexOperation.FLUSH);
-        lsmHarness.scheduleFlush(ctx, callback);
+        return lsmHarness.scheduleFlush(ctx);
     }
 
     @Override
-    public void scheduleMerge(ILSMIOOperationCallback callback, List<ILSMDiskComponent> components)
-            throws HyracksDataException {
+    public ILSMIOOperation scheduleMerge(List<ILSMDiskComponent> components) throws HyracksDataException {
         ctx.setOperation(IndexOperation.MERGE);
         ctx.getComponentsToBeMerged().clear();
         ctx.getComponentsToBeMerged().addAll(components);
-        lsmHarness.scheduleMerge(ctx, callback);
+        return lsmHarness.scheduleMerge(ctx);
     }
 
     @Override
@@ -159,9 +157,9 @@
     }
 
     @Override
-    public void scheduleFullMerge(ILSMIOOperationCallback callback) throws HyracksDataException {
+    public ILSMIOOperation scheduleFullMerge() throws HyracksDataException {
         ctx.setOperation(IndexOperation.FULL_MERGE);
-        lsmHarness.scheduleFullMerge(ctx, callback);
+        return lsmHarness.scheduleFullMerge(ctx);
     }
 
     @Override
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/LoadOperation.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/LoadOperation.java
new file mode 100644
index 0000000..3d63a6a
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/LoadOperation.java
@@ -0,0 +1,49 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.hyracks.storage.am.lsm.common.impls;
+
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallback;
+
+public class LoadOperation extends AbstractIoOperation {
+
+    public LoadOperation(ILSMIOOperationCallback callback, String indexIdentifier) {
+        super(null, null, callback, indexIdentifier);
+    }
+
+    @Override
+    public final LSMIOOperationType getIOOpertionType() {
+        return LSMIOOperationType.LOAD;
+    }
+
+    @Override
+    public LSMIOOperationStatus call() throws HyracksDataException {
+        throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public LSMComponentFileReferences getComponentFiles() {
+        return null;
+    }
+
+    @Override
+    public void sync() {
+        throw new UnsupportedOperationException();
+    }
+}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/MergeOperation.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/MergeOperation.java
index ec2305d..9d7c449 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/MergeOperation.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/MergeOperation.java
@@ -41,9 +41,9 @@
     }
 
     @Override
-    public Boolean call() throws HyracksDataException {
+    public LSMIOOperationStatus call() throws HyracksDataException {
         accessor.merge(this);
-        return true;
+        return getStatus();
     }
 
     @Override
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/MultitenantVirtualBufferCache.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/MultitenantVirtualBufferCache.java
index 9b25471..9bd5007 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/MultitenantVirtualBufferCache.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/MultitenantVirtualBufferCache.java
@@ -220,4 +220,8 @@
         vbc.deleteFile(file);
     }
 
+    @Override
+    public void closeFileIfOpen(FileReference fileRef) {
+        throw new UnsupportedOperationException();
+    }
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/NoOpIOOperationCallbackFactory.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/NoOpIOOperationCallbackFactory.java
index 3432624..600a02d 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/NoOpIOOperationCallbackFactory.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/NoOpIOOperationCallbackFactory.java
@@ -20,15 +20,20 @@
 
 import org.apache.hyracks.api.application.INCServiceContext;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperation;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallback;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallbackFactory;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndex;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndexOperationContext;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMMemoryComponent;
 import org.apache.hyracks.storage.common.IResource;
 
-public enum NoOpIOOperationCallbackFactory implements ILSMIOOperationCallbackFactory {
-    INSTANCE;
+public class NoOpIOOperationCallbackFactory implements ILSMIOOperationCallbackFactory {
+
+    private static final long serialVersionUID = 1L;
+    public static final NoOpIOOperationCallbackFactory INSTANCE = new NoOpIOOperationCallbackFactory();
+
+    private NoOpIOOperationCallbackFactory() {
+    }
 
     @Override
     public ILSMIOOperationCallback createIoOpCallback(ILSMIndex index) {
@@ -47,22 +52,7 @@
         }
 
         @Override
-        public void beforeOperation(ILSMIndexOperationContext opCtx) throws HyracksDataException {
-            // Do nothing.
-        }
-
-        @Override
-        public void afterOperation(ILSMIndexOperationContext opCtx) throws HyracksDataException {
-            // Do nothing.
-        }
-
-        @Override
-        public void afterFinalize(ILSMIndexOperationContext opCtx) throws HyracksDataException {
-            // Do nothing.
-        }
-
-        @Override
-        public void recycled(ILSMMemoryComponent component, boolean componentSwitched) {
+        public void recycled(ILSMMemoryComponent component) {
             // Do nothing.
         }
 
@@ -70,5 +60,35 @@
         public void allocated(ILSMMemoryComponent component) {
             // Do nothing.
         }
+
+        @Override
+        public void scheduled(ILSMIOOperation operation) throws HyracksDataException {
+            // Do nothing.
+        }
+
+        @Override
+        public void beforeOperation(ILSMIOOperation operation) throws HyracksDataException {
+            // Do nothing.
+        }
+
+        @Override
+        public void afterOperation(ILSMIOOperation operation) throws HyracksDataException {
+            // Do nothing.
+        }
+
+        @Override
+        public void afterFinalize(ILSMIOOperation operation) throws HyracksDataException {
+            // Do nothing.
+        }
+
+        @Override
+        public void completed(ILSMIOOperation operation) {
+            // Do nothing.
+        }
+    }
+
+    @Override
+    public int getCurrentMemoryComponentIndex() throws HyracksDataException {
+        return 0;
     }
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/NoOpIoOperation.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/NoOpIoOperation.java
new file mode 100644
index 0000000..6d7e7ec
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/NoOpIoOperation.java
@@ -0,0 +1,116 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.hyracks.storage.am.lsm.common.impls;
+
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.io.FileReference;
+import org.apache.hyracks.api.io.IODeviceHandle;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMDiskComponent;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperation;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallback;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndexAccessor;
+
+public class NoOpIoOperation implements ILSMIOOperation {
+    public static final NoOpIoOperation INSTANCE = new NoOpIoOperation();
+
+    private NoOpIoOperation() {
+    }
+
+    @Override
+    public IODeviceHandle getDevice() {
+        throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public ILSMIOOperationCallback getCallback() {
+        throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public String getIndexIdentifier() {
+        return NoOpIoOperation.class.getSimpleName();
+    }
+
+    @Override
+    public LSMIOOperationType getIOOpertionType() {
+        return LSMIOOperationType.NOOP;
+    }
+
+    @Override
+    public LSMIOOperationStatus call() throws HyracksDataException {
+        return LSMIOOperationStatus.SUCCESS;
+    }
+
+    @Override
+    public FileReference getTarget() {
+        throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public ILSMIndexAccessor getAccessor() {
+        throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public LSMComponentFileReferences getComponentFiles() {
+        throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public Throwable getFailure() {
+        throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public void setFailure(Throwable failure) {
+        // No Op
+    }
+
+    @Override
+    public LSMIOOperationStatus getStatus() {
+        return LSMIOOperationStatus.SUCCESS;
+    }
+
+    @Override
+    public void setStatus(LSMIOOperationStatus status) {
+        // No Op
+    }
+
+    @Override
+    public ILSMDiskComponent getNewComponent() {
+        throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public void setNewComponent(ILSMDiskComponent component) {
+        // No Op
+    }
+
+    @Override
+    public void complete() {
+        // No Op
+    }
+
+    @Override
+    public void sync() {
+        // No Op
+    }
+
+}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/NoOpIoOperationFailedCallback.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/NoOpIoOperationFailedCallback.java
new file mode 100644
index 0000000..3b2b2c5
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/NoOpIoOperationFailedCallback.java
@@ -0,0 +1,44 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.hyracks.storage.am.lsm.common.impls;
+
+import org.apache.hyracks.storage.am.lsm.common.api.IIoOperationFailedCallback;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperation;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationScheduler;
+import org.apache.logging.log4j.LogManager;
+import org.apache.logging.log4j.Logger;
+
+public class NoOpIoOperationFailedCallback implements IIoOperationFailedCallback {
+    private static final Logger LOGGER = LogManager.getLogger();
+
+    public static final NoOpIoOperationFailedCallback INSTANCE = new NoOpIoOperationFailedCallback();
+
+    private NoOpIoOperationFailedCallback() {
+    }
+
+    @Override
+    public void operationFailed(ILSMIOOperation operation, Throwable t) {
+        LOGGER.error("Operation {} failed", operation, t);
+    }
+
+    @Override
+    public void schedulerFailed(ILSMIOOperationScheduler scheduler, Throwable failure) {
+        LOGGER.error("IO Scheduler failed", failure);
+    }
+}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/PrefixMergePolicy.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/PrefixMergePolicy.java
index 73b5d20..b571e58 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/PrefixMergePolicy.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/PrefixMergePolicy.java
@@ -31,7 +31,6 @@
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponent.ComponentState;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMDiskComponent;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndex;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndexAccessor;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMMergePolicy;
 
 public class PrefixMergePolicy implements ILSMMergePolicy {
@@ -59,8 +58,7 @@
         }
 
         if (fullMergeIsRequested) {
-            ILSMIndexAccessor accessor = index.createAccessor(NoOpIndexAccessParameters.INSTANCE);
-            accessor.scheduleFullMerge(index.getIOOperationCallback());
+            index.createAccessor(NoOpIndexAccessParameters.INSTANCE).scheduleFullMerge();
             return;
         }
 
@@ -244,8 +242,7 @@
 
         // Reverse the components order back to its original order
         Collections.reverse(mergableComponents);
-        ILSMIndexAccessor accessor = index.createAccessor(NoOpIndexAccessParameters.INSTANCE);
-        accessor.scheduleMerge(index.getIOOperationCallback(), mergableComponents);
+        index.createAccessor(NoOpIndexAccessParameters.INSTANCE).scheduleMerge(mergableComponents);
     }
 
     /**
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/StubIOOperationCallback.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/StubIOOperationCallback.java
deleted file mode 100644
index 2c16be0..0000000
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/StubIOOperationCallback.java
+++ /dev/null
@@ -1,70 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.hyracks.storage.am.lsm.common.impls;
-
-import java.util.List;
-
-import org.apache.hyracks.api.exceptions.HyracksDataException;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMDiskComponent;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallback;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndexOperationContext;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMMemoryComponent;
-
-/**
- * This class is for testing. It's basically a way to get the new/old component info from the
- * harness callback simply.
- */
-
-public class StubIOOperationCallback implements ILSMIOOperationCallback {
-
-    private ILSMIndexOperationContext opCtx = null;
-
-    @Override
-    public void beforeOperation(ILSMIndexOperationContext opCtx) throws HyracksDataException {
-        // Not interested in this
-    }
-
-    @Override
-    public void afterOperation(ILSMIndexOperationContext opCtx) throws HyracksDataException {
-        this.opCtx = opCtx;
-    }
-
-    @Override
-    public void afterFinalize(ILSMIndexOperationContext opCtx) throws HyracksDataException {
-        // Redundant info from after
-    }
-
-    public List<ILSMDiskComponent> getLastOldComponents() {
-        return opCtx.getComponentsToBeMerged();
-    }
-
-    public ILSMDiskComponent getLastNewComponent() {
-        return opCtx.getNewComponent();
-    }
-
-    @Override
-    public void recycled(ILSMMemoryComponent component, boolean componentSwitched) {
-        // Not interested in this
-    }
-
-    @Override
-    public void allocated(ILSMMemoryComponent component) {
-        // Not interested in this
-    }
-}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/SynchronousScheduler.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/SynchronousScheduler.java
index 506dcea..ae3b4e3 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/SynchronousScheduler.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/SynchronousScheduler.java
@@ -18,8 +18,13 @@
  */
 package org.apache.hyracks.storage.am.lsm.common.impls;
 
-import org.apache.hyracks.api.exceptions.HyracksDataException;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+
+import org.apache.hyracks.storage.am.lsm.common.api.IIoOperationFailedCallback;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperation;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperation.LSMIOOperationStatus;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperation.LSMIOOperationType;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationScheduler;
 import org.apache.logging.log4j.Level;
 import org.apache.logging.log4j.LogManager;
@@ -27,18 +32,76 @@
 
 public class SynchronousScheduler implements ILSMIOOperationScheduler {
     private static final Logger LOGGER = LogManager.getLogger();
-    public static final SynchronousScheduler INSTANCE = new SynchronousScheduler();
+    private final Map<String, ILSMIOOperation> runningFlushOperations = new ConcurrentHashMap<>();
+    private final Map<String, Throwable> failedGroups = new ConcurrentHashMap<>();
+    private final IIoOperationFailedCallback failureCallback;
 
-    private SynchronousScheduler() {
+    public SynchronousScheduler(IIoOperationFailedCallback failureCallback) {
+        this.failureCallback = failureCallback;
     }
 
     @Override
-    public void scheduleOperation(ILSMIOOperation operation) throws HyracksDataException {
+    public void scheduleOperation(ILSMIOOperation operation) {
+        try {
+            before(operation);
+            if (operation.getStatus() == LSMIOOperationStatus.FAILURE) {
+                return;
+            }
+            run(operation);
+        } catch (Throwable e) { // NOSONAR: Must catch them all
+            throw new IllegalStateException(e);
+        } finally {
+            after(operation);
+        }
+    }
+
+    private void run(ILSMIOOperation operation) {
         try {
             operation.call();
-        } catch (Exception e) {
-            LOGGER.log(Level.ERROR, "IO Operation failed", e);
-            throw HyracksDataException.create(e);
+        } catch (Throwable th) { // NOSONAR Must catch all
+            LOGGER.log(Level.ERROR, "IO Operation failed", th);
+            operation.setStatus(LSMIOOperationStatus.FAILURE);
+            operation.setFailure(th);
+        }
+        if (operation.getStatus() == LSMIOOperationStatus.FAILURE) {
+            failureCallback.operationFailed(operation, operation.getFailure());
+        }
+    }
+
+    private void after(ILSMIOOperation operation) {
+        if (operation.getIOOpertionType() == LSMIOOperationType.FLUSH) {
+            synchronized (runningFlushOperations) {
+                runningFlushOperations.remove(operation.getIndexIdentifier());
+                if (operation.getStatus() == LSMIOOperationStatus.FAILURE) {
+                    failedGroups.putIfAbsent(operation.getIndexIdentifier(), operation.getFailure());
+                }
+                operation.complete();
+                runningFlushOperations.notifyAll();
+            }
+        } else {
+            operation.complete();
+        }
+    }
+
+    private void before(ILSMIOOperation operation) throws InterruptedException {
+        String id = operation.getIndexIdentifier();
+        if (operation.getIOOpertionType() == LSMIOOperationType.FLUSH) {
+            synchronized (runningFlushOperations) {
+                while (true) {
+                    if (failedGroups.containsKey(id)) {
+                        operation.setStatus(LSMIOOperationStatus.FAILURE);
+                        operation.setFailure(new RuntimeException("Operation group " + id + " has permanently failed",
+                                failedGroups.get(id)));
+                        return;
+                    }
+                    if (runningFlushOperations.containsKey(id)) {
+                        runningFlushOperations.wait();
+                    } else {
+                        runningFlushOperations.put(id, operation);
+                        break;
+                    }
+                }
+            }
         }
     }
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/SynchronousSchedulerProvider.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/SynchronousSchedulerProvider.java
index 823db04..132d5cf 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/SynchronousSchedulerProvider.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/SynchronousSchedulerProvider.java
@@ -22,12 +22,16 @@
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationScheduler;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationSchedulerProvider;
 
-public enum SynchronousSchedulerProvider implements ILSMIOOperationSchedulerProvider {
-    INSTANCE;
+public class SynchronousSchedulerProvider implements ILSMIOOperationSchedulerProvider {
+    private static final long serialVersionUID = 1L;
+    public static final SynchronousSchedulerProvider INSTANCE = new SynchronousSchedulerProvider();
+
+    private SynchronousSchedulerProvider() {
+    }
 
     @Override
     public ILSMIOOperationScheduler getIoScheduler(INCServiceContext ctx) {
-        return SynchronousScheduler.INSTANCE;
+        return new SynchronousScheduler(NoOpIoOperationFailedCallback.INSTANCE);
     }
 
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/ThreadCountingTracker.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/ThreadCountingTracker.java
index a7fbe06..07eeb28 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/ThreadCountingTracker.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/ThreadCountingTracker.java
@@ -58,7 +58,7 @@
         if (opType == LSMOperationType.MODIFICATION && threadRefCount.decrementAndGet() == 0
                 && index.hasFlushRequestForCurrentMutableComponent()) {
             ILSMIndexAccessor accessor = index.createAccessor(NoOpIndexAccessParameters.INSTANCE);
-            accessor.scheduleFlush(NoOpIOOperationCallbackFactory.INSTANCE.createIoOpCallback(index));
+            accessor.scheduleFlush();
         }
     }
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/TracedIOOperation.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/TracedIOOperation.java
index 7238f8e..5ada349 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/TracedIOOperation.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/TracedIOOperation.java
@@ -22,6 +22,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.storage.am.lsm.common.api.ILSMDiskComponent;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperation;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallback;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndexAccessor;
@@ -86,7 +87,7 @@
     }
 
     @Override
-    public Boolean call() throws HyracksDataException {
+    public LSMIOOperationStatus call() throws HyracksDataException {
         final String name = getTarget().getRelativePath();
         final long tid = tracer.durationB(name, traceCategory, null);
         try {
@@ -111,6 +112,46 @@
     public LSMComponentFileReferences getComponentFiles() {
         return ioOp.getComponentFiles();
     }
+
+    @Override
+    public Throwable getFailure() {
+        return ioOp.getFailure();
+    }
+
+    @Override
+    public void setFailure(Throwable failure) {
+        ioOp.setFailure(failure);
+    }
+
+    @Override
+    public LSMIOOperationStatus getStatus() {
+        return ioOp.getStatus();
+    }
+
+    @Override
+    public void setStatus(LSMIOOperationStatus status) {
+        ioOp.setStatus(status);
+    }
+
+    @Override
+    public ILSMDiskComponent getNewComponent() {
+        return ioOp.getNewComponent();
+    }
+
+    @Override
+    public void setNewComponent(ILSMDiskComponent component) {
+        ioOp.setNewComponent(component);
+    }
+
+    @Override
+    public void complete() {
+        ioOp.complete();
+    }
+
+    @Override
+    public void sync() throws InterruptedException {
+        ioOp.sync();
+    }
 }
 
 class ComparableTracedIOOperation extends TracedIOOperation implements Comparable<ILSMIOOperation> {
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/VirtualBufferCache.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/VirtualBufferCache.java
index ef9852d..e63a5b8 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/VirtualBufferCache.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/VirtualBufferCache.java
@@ -462,4 +462,10 @@
         map.put("open", open);
         return map;
     }
+
+    @Override
+    public void closeFileIfOpen(FileReference fileRef) {
+        throw new UnsupportedOperationException();
+    }
+
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/util/IOOperationUtils.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/util/IOOperationUtils.java
deleted file mode 100644
index 9bd873d..0000000
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/util/IOOperationUtils.java
+++ /dev/null
@@ -1,43 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.hyracks.storage.am.lsm.common.util;
-
-import org.apache.hyracks.api.exceptions.HyracksDataException;
-import org.apache.hyracks.storage.am.lsm.common.impls.BlockingIOOperationCallbackWrapper;
-import org.apache.logging.log4j.Level;
-import org.apache.logging.log4j.LogManager;
-import org.apache.logging.log4j.Logger;
-
-public class IOOperationUtils {
-    private static final Logger LOGGER = LogManager.getLogger();
-
-    private IOOperationUtils() {
-    }
-
-    public static void waitForIoOperation(BlockingIOOperationCallbackWrapper ioCallback) throws HyracksDataException {
-        // Note that the following call assumes that the io operation has succeeded.
-        try {
-            ioCallback.waitForIO();
-        } catch (InterruptedException e) {
-            LOGGER.log(Level.WARN, "Operation has been interrupted. returning");
-            Thread.currentThread().interrupt();
-            throw HyracksDataException.create(e);
-        }
-    }
-}
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 0fae1ac..22556af 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
@@ -45,7 +45,6 @@
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMDiskComponentBulkLoader;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMDiskComponentFactory;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperation;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperation.LSMIOOperationType;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallback;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallbackFactory;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationScheduler;
@@ -111,7 +110,6 @@
         this.tokenCmpFactories = tokenCmpFactories;
         this.filterFieldsForNonBulkLoadOps = filterFieldsForNonBulkLoadOps;
         this.invertedIndexFieldsForNonBulkLoadOps = invertedIndexFieldsForNonBulkLoadOps;
-
         int i = 0;
         for (IVirtualBufferCache virtualBufferCache : virtualBufferCaches) {
             InMemoryInvertedIndex memInvIndex =
@@ -120,9 +118,8 @@
                     BTreeUtils.createBTree(virtualBufferCache, new VirtualFreePageManager(virtualBufferCache),
                             invListTypeTraits, invListCmpFactories, BTreeLeafFrameType.REGULAR_NSM,
                             ioManager.resolveAbsolutePath(fileManager.getBaseDir() + "_virtual_del_" + i), false);
-            LSMInvertedIndexMemoryComponent mutableComponent =
-                    new LSMInvertedIndexMemoryComponent(this, memInvIndex, deleteKeysBTree, virtualBufferCache,
-                            i == 0 ? true : false, filterHelper == null ? null : filterHelper.createFilter());
+            LSMInvertedIndexMemoryComponent mutableComponent = new LSMInvertedIndexMemoryComponent(this, memInvIndex,
+                    deleteKeysBTree, virtualBufferCache, filterHelper == null ? null : filterHelper.createFilter());
             memoryComponents.add(mutableComponent);
             ++i;
         }
@@ -296,7 +293,7 @@
         }
 
         ILSMDiskComponentBulkLoader componentBulkLoader =
-                component.createBulkLoader(LSMIOOperationType.FLUSH, 1.0f, false, numBTreeTuples, false, false, false);
+                component.createBulkLoader(operation, 1.0f, false, numBTreeTuples, false, false, false);
 
         // Create a scan cursor on the deleted keys BTree underlying the in-memory inverted index.
         IIndexCursor deletedKeysScanCursor = deletedKeysBTreeAccessor.createSearchCursor(false);
@@ -368,15 +365,14 @@
                     numElements += ((LSMInvertedIndexDiskComponent) mergeOp.getMergingComponents().get(i))
                             .getBloomFilter().getNumElements();
                 }
-                componentBulkLoader = component.createBulkLoader(LSMIOOperationType.MERGE, 1.0f, false, numElements,
-                        false, false, false);
+                componentBulkLoader =
+                        component.createBulkLoader(operation, 1.0f, false, numElements, false, false, false);
                 loadDeleteTuples(opCtx, btreeCursor, mergePred, componentBulkLoader);
             } finally {
                 btreeCursor.destroy();
             }
         } else {
-            componentBulkLoader =
-                    component.createBulkLoader(LSMIOOperationType.MERGE, 1.0f, false, 0L, false, false, false);
+            componentBulkLoader = component.createBulkLoader(operation, 1.0f, false, 0L, false, false, false);
         }
         search(opCtx, cursor, mergePred);
         try {
@@ -490,7 +486,7 @@
             throws HyracksDataException {
         return new LSMInvertedIndexFlushOperation(new LSMInvertedIndexAccessor(getHarness(), opCtx),
                 componentFileRefs.getInsertIndexFileReference(), componentFileRefs.getDeleteIndexFileReference(),
-                componentFileRefs.getBloomFilterFileReference(), callback, fileManager.getBaseDir().getAbsolutePath());
+                componentFileRefs.getBloomFilterFileReference(), callback, getIndexIdentifier());
     }
 
     @Override
@@ -500,6 +496,6 @@
         IIndexCursor cursor = new LSMInvertedIndexMergeCursor(opCtx);
         return new LSMInvertedIndexMergeOperation(accessor, cursor, mergeFileRefs.getInsertIndexFileReference(),
                 mergeFileRefs.getDeleteIndexFileReference(), mergeFileRefs.getBloomFilterFileReference(), callback,
-                fileManager.getBaseDir().getAbsolutePath());
+                getIndexIdentifier());
     }
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndexAccessor.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndexAccessor.java
index 247e44c..a6471f8 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndexAccessor.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndexAccessor.java
@@ -30,7 +30,6 @@
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMDiskComponent;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMHarness;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperation;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallback;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndexAccessor;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndexOperationContext;
 import org.apache.hyracks.storage.am.lsm.common.api.LSMOperationType;
@@ -86,23 +85,22 @@
     }
 
     @Override
-    public void scheduleFlush(ILSMIOOperationCallback callback) throws HyracksDataException {
+    public ILSMIOOperation scheduleFlush() throws HyracksDataException {
         ctx.setOperation(IndexOperation.FLUSH);
-        lsmHarness.scheduleFlush(ctx, callback);
+        return lsmHarness.scheduleFlush(ctx);
     }
 
     @Override
     public void flush(ILSMIOOperation operation) throws HyracksDataException {
-        lsmHarness.flush(ctx, operation);
+        lsmHarness.flush(operation);
     }
 
     @Override
-    public void scheduleMerge(ILSMIOOperationCallback callback, List<ILSMDiskComponent> components)
-            throws HyracksDataException {
+    public ILSMIOOperation scheduleMerge(List<ILSMDiskComponent> components) throws HyracksDataException {
         ctx.setOperation(IndexOperation.MERGE);
         ctx.getComponentsToBeMerged().clear();
         ctx.getComponentsToBeMerged().addAll(components);
-        lsmHarness.scheduleMerge(ctx, callback);
+        return lsmHarness.scheduleMerge(ctx);
     }
 
     @Override
@@ -115,14 +113,14 @@
     }
 
     @Override
-    public void scheduleFullMerge(ILSMIOOperationCallback callback) throws HyracksDataException {
+    public ILSMIOOperation scheduleFullMerge() throws HyracksDataException {
         ctx.setOperation(IndexOperation.FULL_MERGE);
-        lsmHarness.scheduleFullMerge(ctx, callback);
+        return lsmHarness.scheduleFullMerge(ctx);
     }
 
     @Override
     public void merge(ILSMIOOperation operation) throws HyracksDataException {
-        lsmHarness.merge(ctx, operation);
+        lsmHarness.merge(operation);
     }
 
     @Override
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndexMemoryComponent.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndexMemoryComponent.java
index 77c62b4..a8005bc 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndexMemoryComponent.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndexMemoryComponent.java
@@ -20,9 +20,9 @@
 package org.apache.hyracks.storage.am.lsm.invertedindex.impls;
 
 import org.apache.hyracks.storage.am.btree.impls.BTree;
-import org.apache.hyracks.storage.am.lsm.common.api.AbstractLSMWithBuddyMemoryComponent;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponentFilter;
 import org.apache.hyracks.storage.am.lsm.common.api.IVirtualBufferCache;
+import org.apache.hyracks.storage.am.lsm.common.impls.AbstractLSMWithBuddyMemoryComponent;
 import org.apache.hyracks.storage.am.lsm.invertedindex.inmemory.InMemoryInvertedIndex;
 
 public class LSMInvertedIndexMemoryComponent extends AbstractLSMWithBuddyMemoryComponent {
@@ -31,8 +31,8 @@
     private final BTree deletedKeysBTree;
 
     public LSMInvertedIndexMemoryComponent(LSMInvertedIndex lsmIndex, InMemoryInvertedIndex invIndex,
-            BTree deletedKeysBTree, IVirtualBufferCache vbc, boolean isActive, ILSMComponentFilter filter) {
-        super(lsmIndex, vbc, isActive, filter);
+            BTree deletedKeysBTree, IVirtualBufferCache vbc, ILSMComponentFilter filter) {
+        super(lsmIndex, vbc, filter);
         this.invIndex = invIndex;
         this.deletedKeysBTree = deletedKeysBTree;
     }
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 e36abb4..c695a5c 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
@@ -97,9 +97,8 @@
             BTree memBTree = new BTree(virtualBufferCache, new VirtualFreePageManager(virtualBufferCache),
                     btreeInteriorFrameFactory, btreeLeafFrameFactory, btreeCmpFactories, btreeCmpFactories.length,
                     ioManager.resolveAbsolutePath(fileManager.getBaseDir() + "_virtual_b_" + i));
-            LSMRTreeMemoryComponent mutableComponent =
-                    new LSMRTreeMemoryComponent(this, memRTree, memBTree, virtualBufferCache, i == 0 ? true : false,
-                            filterHelper == null ? null : filterHelper.createFilter());
+            LSMRTreeMemoryComponent mutableComponent = new LSMRTreeMemoryComponent(this, memRTree, memBTree,
+                    virtualBufferCache, filterHelper == null ? null : filterHelper.createFilter());
             memoryComponents.add(mutableComponent);
             ++i;
         }
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/impls/ExternalRTree.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/impls/ExternalRTree.java
index 7b12250..f3162c9 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/impls/ExternalRTree.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/impls/ExternalRTree.java
@@ -40,8 +40,6 @@
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMDiskComponent;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMDiskComponentFactory;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperation;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperation.LSMIOOperationType;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallback;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallbackFactory;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationScheduler;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndexAccessor;
@@ -50,7 +48,6 @@
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMMergePolicy;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMOperationTracker;
 import org.apache.hyracks.storage.am.lsm.common.api.ITwoPCIndex;
-import org.apache.hyracks.storage.am.lsm.common.impls.AbstractLSMIndexOperationContext;
 import org.apache.hyracks.storage.am.lsm.common.impls.ExternalIndexHarness;
 import org.apache.hyracks.storage.am.lsm.common.impls.LSMComponentFileReferences;
 import org.apache.hyracks.storage.am.rtree.impls.SearchPredicate;
@@ -323,12 +320,6 @@
         if (!isActive) {
             throw new HyracksDataException("Failed to deactivate the index since it is already deactivated.");
         }
-        if (flushOnExit) {
-            AbstractLSMIndexOperationContext opCtx = createOpContext(NoOpOperationCallback.INSTANCE, version);
-            opCtx.setIoOperationType(LSMIOOperationType.FLUSH);
-            ioOpCallback.afterFinalize(opCtx);
-        }
-
         for (ILSMDiskComponent c : diskComponents) {
             c.deactivateAndPurge();
         }
@@ -383,16 +374,13 @@
         version = 0;
     }
 
-    // Not supported
     @Override
     public void modify(IIndexOperationContext ictx, ITupleReference tuple) throws HyracksDataException {
         throw new UnsupportedOperationException("tuple modify not supported in LSM-Disk-Only-RTree");
     }
 
-    // Not supported
     @Override
-    public void scheduleFlush(ILSMIndexOperationContext ctx, ILSMIOOperationCallback callback)
-            throws HyracksDataException {
+    public ILSMIOOperation createFlushOperation(ILSMIndexOperationContext ctx) throws HyracksDataException {
         throw new UnsupportedOperationException("flush not supported in LSM-Disk-Only-RTree");
     }
 
@@ -576,8 +564,7 @@
     // The only change the the schedule merge is the method used to create the
     // opCtx. first line <- in schedule merge, we->
     @Override
-    public void scheduleMerge(ILSMIndexOperationContext ctx, ILSMIOOperationCallback callback)
-            throws HyracksDataException {
+    public ILSMIOOperation createMergeOperation(ILSMIndexOperationContext ctx) throws HyracksDataException {
         ILSMIndexOperationContext rctx = createOpContext(NoOpOperationCallback.INSTANCE, -1);
         rctx.setOperation(IndexOperation.MERGE);
         List<ILSMComponent> mergingComponents = ctx.getComponentHolder();
@@ -587,10 +574,12 @@
                         (ILSMDiskComponent) mergingComponents.get(mergingComponents.size() - 1));
         ILSMIndexAccessor accessor = new LSMRTreeAccessor(getHarness(), rctx, buddyBTreeFields);
         // create the merge operation.
-        LSMRTreeMergeOperation mergeOp = new LSMRTreeMergeOperation(accessor, cursor,
-                relMergeFileRefs.getInsertIndexFileReference(), relMergeFileRefs.getDeleteIndexFileReference(),
-                relMergeFileRefs.getBloomFilterFileReference(), callback, fileManager.getBaseDir().getAbsolutePath());
-        ioScheduler.scheduleOperation(mergeOp);
+        LSMRTreeMergeOperation mergeOp =
+                new LSMRTreeMergeOperation(accessor, cursor, relMergeFileRefs.getInsertIndexFileReference(),
+                        relMergeFileRefs.getDeleteIndexFileReference(), relMergeFileRefs.getBloomFilterFileReference(),
+                        ioOpCallback, fileManager.getBaseDir().getAbsolutePath());
+        ioOpCallback.scheduled(mergeOp);
+        return mergeOp;
     }
 
     @Override
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 fae6e1a..4566d32 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
@@ -45,7 +45,6 @@
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMDiskComponentBulkLoader;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMDiskComponentFactory;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperation;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperation.LSMIOOperationType;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallback;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallbackFactory;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationScheduler;
@@ -135,8 +134,8 @@
                 rTreeTupleSorter.sort();
                 component = createDiskComponent(componentFactory, flushOp.getTarget(), flushOp.getBTreeTarget(),
                         flushOp.getBloomFilterTarget(), true);
-                componentBulkLoader = component.createBulkLoader(LSMIOOperationType.FLUSH, 1.0f, false,
-                        numBTreeTuples.longValue(), false, false, false);
+                componentBulkLoader = component.createBulkLoader(operation, 1.0f, false, numBTreeTuples.longValue(),
+                        false, false, false);
                 flushLoadRTree(isEmpty, rTreeTupleSorter, componentBulkLoader);
                 // scan the memory BTree and bulk load delete tuples
                 flushLoadBtree(memBTreeAccessor, componentBulkLoader, btreeNullPredicate);
@@ -332,13 +331,13 @@
                         numElements += ((LSMRTreeDiskComponent) mergeOp.getMergingComponents().get(i)).getBloomFilter()
                                 .getNumElements();
                     }
-                    componentBulkLoader = mergedComponent.createBulkLoader(LSMIOOperationType.MERGE, 1.0f, false,
-                            numElements, false, false, false);
+                    componentBulkLoader =
+                            mergedComponent.createBulkLoader(mergeOp, 1.0f, false, numElements, false, false, false);
                     mergeLoadBTree(opCtx, rtreeSearchPred, componentBulkLoader);
                 } else {
                     //no buddy-btree needed
-                    componentBulkLoader = mergedComponent.createBulkLoader(LSMIOOperationType.MERGE, 1.0f, false, 0L,
-                            false, false, false);
+                    componentBulkLoader =
+                            mergedComponent.createBulkLoader(mergeOp, 1.0f, false, 0L, false, false, false);
                 }
                 //search old rtree components
                 while (cursor.hasNext()) {
@@ -425,7 +424,7 @@
         LSMRTreeAccessor accessor = new LSMRTreeAccessor(getHarness(), opCtx, buddyBTreeFields);
         return new LSMRTreeFlushOperation(accessor, componentFileRefs.getInsertIndexFileReference(),
                 componentFileRefs.getDeleteIndexFileReference(), componentFileRefs.getBloomFilterFileReference(),
-                callback, fileManager.getBaseDir().getAbsolutePath());
+                callback, getIndexIdentifier());
     }
 
     @Override
@@ -435,6 +434,6 @@
         ILSMIndexAccessor accessor = new LSMRTreeAccessor(getHarness(), opCtx, buddyBTreeFields);
         return new LSMRTreeMergeOperation(accessor, cursor, mergeFileRefs.getInsertIndexFileReference(),
                 mergeFileRefs.getDeleteIndexFileReference(), mergeFileRefs.getBloomFilterFileReference(), callback,
-                fileManager.getBaseDir().getAbsolutePath());
+                getIndexIdentifier());
     }
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/impls/LSMRTreeMemoryComponent.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/impls/LSMRTreeMemoryComponent.java
index 2e1ba68..ef7b815 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/impls/LSMRTreeMemoryComponent.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/impls/LSMRTreeMemoryComponent.java
@@ -21,9 +21,9 @@
 
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.storage.am.btree.impls.BTree;
-import org.apache.hyracks.storage.am.lsm.common.api.AbstractLSMWithBuddyMemoryComponent;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponentFilter;
 import org.apache.hyracks.storage.am.lsm.common.api.IVirtualBufferCache;
+import org.apache.hyracks.storage.am.lsm.common.impls.AbstractLSMWithBuddyMemoryComponent;
 import org.apache.hyracks.storage.am.rtree.impls.RTree;
 
 public class LSMRTreeMemoryComponent extends AbstractLSMWithBuddyMemoryComponent {
@@ -32,8 +32,8 @@
     private final BTree btree;
 
     public LSMRTreeMemoryComponent(AbstractLSMRTree lsmIndex, RTree rtree, BTree btree, IVirtualBufferCache vbc,
-            boolean isActive, ILSMComponentFilter filter) {
-        super(lsmIndex, vbc, isActive, filter);
+            ILSMComponentFilter filter) {
+        super(lsmIndex, vbc, filter);
         this.rtree = rtree;
         this.btree = btree;
     }
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 f4e919a..2757595 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
@@ -40,7 +40,6 @@
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMDiskComponentBulkLoader;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMDiskComponentFactory;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperation;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperation.LSMIOOperationType;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallback;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallbackFactory;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationScheduler;
@@ -110,8 +109,7 @@
                 try {
                     memRTreeAccessor.search(rtreeScanCursor, rtreeNullPredicate);
                     component = createDiskComponent(componentFactory, flushOp.getTarget(), null, null, true);
-                    componentBulkLoader =
-                            component.createBulkLoader(LSMIOOperationType.FLUSH, 1.0f, false, 0L, false, false, false);
+                    componentBulkLoader = component.createBulkLoader(operation, 1.0f, false, 0L, false, false, false);
                     // Since the LSM-RTree is used as a secondary assumption, the
                     // primary key will be the last comparator in the BTree comparators
                     rTreeTupleSorter = new TreeTupleSorter(flushingComponent.getIndex().getFileId(), linearizerArray,
@@ -237,7 +235,7 @@
         ILSMDiskComponent component = createDiskComponent(componentFactory, mergeOp.getTarget(), null, null, true);
 
         ILSMDiskComponentBulkLoader componentBulkLoader =
-                component.createBulkLoader(LSMIOOperationType.MERGE, 1.0f, false, 0L, false, false, false);
+                component.createBulkLoader(operation, 1.0f, false, 0L, false, false, false);
         try {
             while (cursor.hasNext()) {
                 cursor.next();
@@ -275,7 +273,7 @@
             throws HyracksDataException {
         ILSMIndexAccessor accessor = new LSMTreeIndexAccessor(getHarness(), opCtx, cursorFactory);
         return new LSMRTreeFlushOperation(accessor, componentFileRefs.getInsertIndexFileReference(), null, null,
-                callback, fileManager.getBaseDir().getAbsolutePath());
+                callback, getIndexIdentifier());
     }
 
     @Override
@@ -290,6 +288,6 @@
                 new LSMRTreeWithAntiMatterTuplesSearchCursor(opCtx, returnDeletedTuples);
         ILSMIndexAccessor accessor = new LSMTreeIndexAccessor(getHarness(), opCtx, cursorFactory);
         return new LSMRTreeMergeOperation(accessor, cursor, mergeFileRefs.getInsertIndexFileReference(), null, null,
-                callback, fileManager.getBaseDir().getAbsolutePath());
+                callback, getIndexIdentifier());
     }
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/IIndexBulkLoader.java b/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/IIndexBulkLoader.java
index fc54903..d6c954e 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/IIndexBulkLoader.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/IIndexBulkLoader.java
@@ -35,7 +35,7 @@
     public void add(ITupleReference tuple) throws HyracksDataException;
 
     /**
-     * Finalize the bulk loading operation in the given context.
+     * Finalize the bulk loading operation in the given context and release all resources.
      *
      * @throws HyracksDataException
      *             If the BufferCache throws while un/pinning or un/latching.
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/buffercache/BufferCache.java b/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/buffercache/BufferCache.java
index 1443bbc..9f82f02 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/buffercache/BufferCache.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/buffercache/BufferCache.java
@@ -1451,4 +1451,23 @@
             return multiplier;
         }
     }
+
+    @Override
+    public void closeFileIfOpen(FileReference fileRef) {
+        synchronized (fileInfoMap) {
+            if (fileMapManager.isMapped(fileRef)) {
+                int fileId;
+                try {
+                    fileId = fileMapManager.lookupFileId(fileRef);
+                } catch (HyracksDataException e) {
+                    throw new IllegalStateException(e);
+                }
+                BufferedFileHandle fInfo = fileInfoMap.get(fileId);
+                if (fInfo != null && fInfo.getReferenceCount() > 0) {
+                    fInfo.decReferenceCount();
+                }
+            }
+        }
+    }
+
 }
\ No newline at end of file
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/buffercache/DebugBufferCache.java b/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/buffercache/DebugBufferCache.java
index 1cbe404..9e92f21 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/buffercache/DebugBufferCache.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/buffercache/DebugBufferCache.java
@@ -246,4 +246,10 @@
         deleteFileCount.incrementAndGet();
         bufferCache.deleteFile(file);
     }
+
+    @Override
+    public void closeFileIfOpen(FileReference fileRef) {
+        bufferCache.closeFileIfOpen(fileRef);
+    }
+
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/buffercache/IBufferCache.java b/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/buffercache/IBufferCache.java
index 8dccc4a..21d3677 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/buffercache/IBufferCache.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/buffercache/IBufferCache.java
@@ -269,4 +269,13 @@
      */
     void resizePage(ICachedPage page, int multiplier, IExtraPageBlockHelper extraPageBlockHelper)
             throws HyracksDataException;
+
+    /**
+     * Close the file if open.
+     *
+     * @param fileRef
+     * @throws HyracksDataException
+     */
+    void closeFileIfOpen(FileReference fileRef);
+
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/EncapsulatingIoCallback.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/EncapsulatingIoCallback.java
new file mode 100644
index 0000000..cb791c5
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/EncapsulatingIoCallback.java
@@ -0,0 +1,107 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.hyracks.storage.am.lsm.btree;
+
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.storage.am.lsm.btree.impl.ITestOpCallback;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperation;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallback;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMMemoryComponent;
+
+public class EncapsulatingIoCallback implements ILSMIOOperationCallback {
+
+    private final ILSMIOOperationCallback encapsulated;
+    private final ITestOpCallback<ILSMIOOperation> scheduledCallback;
+    private final ITestOpCallback<ILSMIOOperation> beforeOperationCallback;
+    private final ITestOpCallback<ILSMIOOperation> afterOperationCallback;
+    private final ITestOpCallback<ILSMIOOperation> afterFinalizeCallback;
+    private final ITestOpCallback<ILSMIOOperation> completedCallback;
+
+    public EncapsulatingIoCallback(ILSMIOOperationCallback inner, ITestOpCallback<ILSMIOOperation> scheduledCallback,
+            ITestOpCallback<ILSMIOOperation> beforeOperationCallback,
+            ITestOpCallback<ILSMIOOperation> afterOperationCallback,
+            ITestOpCallback<ILSMIOOperation> afterFinalizeCallback,
+            ITestOpCallback<ILSMIOOperation> completedCallback) {
+        this.encapsulated = inner;
+        this.scheduledCallback = scheduledCallback;
+        this.beforeOperationCallback = beforeOperationCallback;
+        this.afterOperationCallback = afterOperationCallback;
+        this.afterFinalizeCallback = afterFinalizeCallback;
+        this.completedCallback = completedCallback;
+    }
+
+    @Override
+    public void scheduled(ILSMIOOperation operation) throws HyracksDataException {
+        scheduledCallback.before(operation);
+        encapsulated.scheduled(operation);
+        scheduledCallback.after(operation);
+    }
+
+    @Override
+    public void beforeOperation(ILSMIOOperation operation) throws HyracksDataException {
+        beforeOperationCallback.before(operation);
+        encapsulated.beforeOperation(operation);
+        beforeOperationCallback.after(operation);
+    }
+
+    @Override
+    public void afterOperation(ILSMIOOperation operation) throws HyracksDataException {
+        afterOperationCallback.before(operation);
+        encapsulated.afterOperation(operation);
+        afterOperationCallback.after(operation);
+    }
+
+    @Override
+    public void afterFinalize(ILSMIOOperation operation) throws HyracksDataException {
+        afterFinalizeCallback.before(operation);
+        encapsulated.afterFinalize(operation);
+        afterFinalizeCallback.after(operation);
+    }
+
+    @Override
+    public void completed(ILSMIOOperation operation) {
+        try {
+            completedCallback.before(operation);
+        } catch (HyracksDataException e) {
+            throw new IllegalStateException(e);
+        }
+        encapsulated.completed(operation);
+        try {
+            completedCallback.after(operation);
+        } catch (HyracksDataException e) {
+            throw new IllegalStateException(e);
+        }
+    }
+
+    @Override
+    public void recycled(ILSMMemoryComponent component) throws HyracksDataException {
+        encapsulated.recycled(component);
+    }
+
+    @Override
+    public void allocated(ILSMMemoryComponent component) throws HyracksDataException {
+        encapsulated.allocated(component);
+    }
+
+    public ILSMIOOperationCallback getEncapsulated() {
+        return encapsulated;
+    }
+
+}
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/EncapsulatingIoCallbackFactory.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/EncapsulatingIoCallbackFactory.java
new file mode 100644
index 0000000..bb10236
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/EncapsulatingIoCallbackFactory.java
@@ -0,0 +1,71 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.hyracks.storage.am.lsm.btree;
+
+import org.apache.hyracks.api.application.INCServiceContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.storage.am.lsm.btree.impl.ITestOpCallback;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperation;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallback;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallbackFactory;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndex;
+import org.apache.hyracks.storage.common.IResource;
+
+public class EncapsulatingIoCallbackFactory implements ILSMIOOperationCallbackFactory {
+
+    private static final long serialVersionUID = 1L;
+    private final ILSMIOOperationCallbackFactory encapsulated;
+    private final ITestOpCallback<ILSMIOOperation> scheduledCallback;
+    private final ITestOpCallback<ILSMIOOperation> beforeOperationCallback;
+    private final ITestOpCallback<ILSMIOOperation> afterOperationCallback;
+    private final ITestOpCallback<ILSMIOOperation> afterFinalizeCallback;
+    private final ITestOpCallback<ILSMIOOperation> completedCallback;
+
+    public EncapsulatingIoCallbackFactory(ILSMIOOperationCallbackFactory factory,
+            ITestOpCallback<ILSMIOOperation> scheduledCallback,
+            ITestOpCallback<ILSMIOOperation> beforeOperationCallback,
+            ITestOpCallback<ILSMIOOperation> afterOperationCallback,
+            ITestOpCallback<ILSMIOOperation> afterFinalizeCallback,
+            ITestOpCallback<ILSMIOOperation> completedCallback) {
+        encapsulated = factory;
+        this.scheduledCallback = scheduledCallback;
+        this.beforeOperationCallback = beforeOperationCallback;
+        this.afterOperationCallback = afterOperationCallback;
+        this.afterFinalizeCallback = afterFinalizeCallback;
+        this.completedCallback = completedCallback;
+    }
+
+    @Override
+    public void initialize(INCServiceContext ncCtx, IResource resource) {
+        encapsulated.initialize(ncCtx, resource);
+    }
+
+    @Override
+    public ILSMIOOperationCallback createIoOpCallback(ILSMIndex index) throws HyracksDataException {
+        ILSMIOOperationCallback inner = encapsulated.createIoOpCallback(index);
+        return new EncapsulatingIoCallback(inner, scheduledCallback, beforeOperationCallback, afterOperationCallback,
+                afterFinalizeCallback, completedCallback);
+    }
+
+    @Override
+    public int getCurrentMemoryComponentIndex() throws HyracksDataException {
+        return encapsulated.getCurrentMemoryComponentIndex();
+    }
+
+}
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
new file mode 100644
index 0000000..21f3877
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/LSMBTreeComponentLifecycleTest.java
@@ -0,0 +1,470 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.hyracks.storage.am.lsm.btree;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.Semaphore;
+import java.util.concurrent.atomic.AtomicInteger;
+
+import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.util.SingleThreadEventProcessor;
+import org.apache.hyracks.dataflow.common.data.marshalling.IntegerSerializerDeserializer;
+import org.apache.hyracks.storage.am.btree.OrderedIndexTestContext;
+import org.apache.hyracks.storage.am.btree.OrderedIndexTestUtils;
+import org.apache.hyracks.storage.am.common.impls.NoOpIndexAccessParameters;
+import org.apache.hyracks.storage.am.common.ophelpers.IndexOperation;
+import org.apache.hyracks.storage.am.lsm.btree.impl.CountingIoOperationCallback;
+import org.apache.hyracks.storage.am.lsm.btree.impl.CountingIoOperationCallbackFactory;
+import org.apache.hyracks.storage.am.lsm.btree.impl.ITestOpCallback;
+import org.apache.hyracks.storage.am.lsm.btree.impl.NoOpTestCallback;
+import org.apache.hyracks.storage.am.lsm.btree.util.LSMBTreeTestContext;
+import org.apache.hyracks.storage.am.lsm.btree.util.LSMBTreeTestHarness;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponent;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponent.LSMComponentType;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperation;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperation.LSMIOOperationStatus;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallbackFactory;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationScheduler;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndex;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndexAccessor;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndexOperationContext;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMMemoryComponent;
+import org.apache.hyracks.storage.am.lsm.common.api.IIoOperationFailedCallback;
+import org.apache.hyracks.storage.am.lsm.common.impls.AsynchronousScheduler;
+import org.apache.logging.log4j.Level;
+import org.apache.logging.log4j.LogManager;
+import org.apache.logging.log4j.Logger;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+
+public class LSMBTreeComponentLifecycleTest {
+    private static final Logger LOGGER = LogManager.getLogger();
+    private final LSMBTreeTestHarness harness = new LSMBTreeTestHarness();
+    private final OrderedIndexTestUtils testUtils = new OrderedIndexTestUtils();
+    private final ISerializerDeserializer[] fieldSerdes =
+            { IntegerSerializerDeserializer.INSTANCE, IntegerSerializerDeserializer.INSTANCE };
+    private final int numKeys = 1;
+    private static final int numTuplesToInsert = 100;
+
+    @Before
+    public void setUp() throws HyracksDataException {
+        harness.setUp();
+    }
+
+    @After
+    public void tearDown() throws HyracksDataException {
+        harness.tearDown();
+    }
+
+    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.getMetadataPageManagerFactory(), false, true, false);
+    }
+
+    private OrderedIndexTestContext createTestContext(ISerializerDeserializer[] fieldSerdes, int numKeys)
+            throws Exception {
+        return createTestContext(fieldSerdes, numKeys, harness.getIOScheduler(),
+                harness.getIOOperationCallbackFactory());
+    }
+
+    @Test
+    public void testFlushUnallocatedIndex() throws Exception {
+        OrderedIndexTestContext ctx = createTestContext(fieldSerdes, numKeys);
+        ILSMIndex index = (ILSMIndex) ctx.getIndex();
+        index.create();
+        index.activate();
+        Assert.assertEquals(getExpectedMemoryComponentIndex(0), index.getCurrentMemoryComponentIndex());
+        flush(ctx);
+        Assert.assertEquals(getExpectedMemoryComponentIndex(1), index.getCurrentMemoryComponentIndex());
+        Assert.assertEquals(0, index.getDiskComponents().size());
+        CountingIoOperationCallback ioCallback = (CountingIoOperationCallback) index.getIOOperationCallback();
+        // assert equal before, after, after were called
+        Assert.assertEquals(ioCallback.getBeforeOperationCount(), ioCallback.getAfterOperationCount());
+        Assert.assertEquals(ioCallback.getBeforeOperationCount(), ioCallback.getAfterFinalizeCount());
+        // insert into the index
+        testUtils.insertIntTuples(ctx, numTuplesToInsert, harness.getRandom());
+        // flush
+        flush(ctx);
+        Assert.assertEquals(getExpectedMemoryComponentIndex(0), index.getCurrentMemoryComponentIndex());
+        Assert.assertEquals(1, index.getDiskComponents().size());
+        Assert.assertEquals(ioCallback.getBeforeOperationCount(), ioCallback.getAfterOperationCount());
+        Assert.assertEquals(ioCallback.getBeforeOperationCount(), ioCallback.getAfterFinalizeCount());
+        // insert more
+        testUtils.insertIntTuples(ctx, numTuplesToInsert, harness.getRandom());
+        // flush
+        flush(ctx);
+        Assert.assertEquals(getExpectedMemoryComponentIndex(1), index.getCurrentMemoryComponentIndex());
+        Assert.assertEquals(2, index.getDiskComponents().size());
+        Assert.assertEquals(ioCallback.getBeforeOperationCount(), ioCallback.getAfterOperationCount());
+        Assert.assertEquals(ioCallback.getBeforeOperationCount(), ioCallback.getAfterFinalizeCount());
+        ctx.getIndex().deactivate();
+        ctx.getIndex().destroy();
+    }
+
+    @Test
+    public void testFlushUnallocatedIndexStartFromSecondComponent() throws Exception {
+        CountingIoOperationCallbackFactory.STARTING_INDEX = 1;
+        try {
+            testFlushUnallocatedIndex();
+        } finally {
+            CountingIoOperationCallbackFactory.STARTING_INDEX = 0;
+        }
+    }
+
+    @Test
+    public void testNormalFlushOperation() throws Exception {
+        OrderedIndexTestContext ctx = createTestContext(fieldSerdes, numKeys);
+        ILSMIndex index = (ILSMIndex) ctx.getIndex();
+        index.create();
+        index.activate();
+        Assert.assertEquals(getExpectedMemoryComponentIndex(0), index.getCurrentMemoryComponentIndex());
+        testUtils.insertIntTuples(ctx, numTuplesToInsert, harness.getRandom());
+        flush(ctx);
+        Assert.assertEquals(getExpectedMemoryComponentIndex(1), index.getCurrentMemoryComponentIndex());
+        Assert.assertEquals(1, index.getDiskComponents().size());
+        CountingIoOperationCallback ioCallback = (CountingIoOperationCallback) index.getIOOperationCallback();
+        // assert equal before, after, after were called
+        Assert.assertEquals(ioCallback.getBeforeOperationCount(), ioCallback.getAfterOperationCount());
+        Assert.assertEquals(ioCallback.getBeforeOperationCount(), ioCallback.getAfterFinalizeCount());
+        // insert into the index
+        testUtils.insertIntTuples(ctx, numTuplesToInsert, harness.getRandom());
+        // flush
+        flush(ctx);
+        Assert.assertEquals(getExpectedMemoryComponentIndex(0), index.getCurrentMemoryComponentIndex());
+        Assert.assertEquals(2, index.getDiskComponents().size());
+        Assert.assertEquals(ioCallback.getBeforeOperationCount(), ioCallback.getAfterOperationCount());
+        Assert.assertEquals(ioCallback.getBeforeOperationCount(), ioCallback.getAfterFinalizeCount());
+        // insert more
+        testUtils.insertIntTuples(ctx, numTuplesToInsert, harness.getRandom());
+        // flush
+        flush(ctx);
+        Assert.assertEquals(getExpectedMemoryComponentIndex(1), index.getCurrentMemoryComponentIndex());
+        Assert.assertEquals(3, index.getDiskComponents().size());
+        Assert.assertEquals(ioCallback.getBeforeOperationCount(), ioCallback.getAfterOperationCount());
+        Assert.assertEquals(ioCallback.getBeforeOperationCount(), ioCallback.getAfterFinalizeCount());
+        ctx.getIndex().deactivate();
+        ctx.getIndex().destroy();
+    }
+
+    @Test
+    public void testNormalFlushOperationStartFromSecondComponent() throws Exception {
+        CountingIoOperationCallbackFactory.STARTING_INDEX = 1;
+        try {
+            testNormalFlushOperation();
+        } finally {
+            CountingIoOperationCallbackFactory.STARTING_INDEX = 0;
+        }
+    }
+
+    @Test
+    public void testFlushUnModifiedComponent() throws Exception {
+        OrderedIndexTestContext ctx = createTestContext(fieldSerdes, numKeys);
+        ILSMIndex index = (ILSMIndex) ctx.getIndex();
+        index.create();
+        index.activate();
+        Assert.assertEquals(getExpectedMemoryComponentIndex(0), index.getCurrentMemoryComponentIndex());
+        testUtils.insertIntTuples(ctx, numTuplesToInsert, harness.getRandom());
+        flush(ctx);
+        Assert.assertEquals(getExpectedMemoryComponentIndex(1), index.getCurrentMemoryComponentIndex());
+        Assert.assertEquals(1, index.getDiskComponents().size());
+        CountingIoOperationCallback ioCallback = (CountingIoOperationCallback) index.getIOOperationCallback();
+        // assert equal before, after, finalize were called
+        Assert.assertEquals(ioCallback.getBeforeOperationCount(), ioCallback.getAfterOperationCount());
+        Assert.assertEquals(ioCallback.getBeforeOperationCount(), ioCallback.getAfterFinalizeCount());
+        // flush, there was no insert before
+        flush(ctx);
+        Assert.assertEquals(getExpectedMemoryComponentIndex(0), index.getCurrentMemoryComponentIndex());
+        Assert.assertEquals(1, index.getDiskComponents().size());
+        Assert.assertEquals(ioCallback.getBeforeOperationCount(), ioCallback.getAfterOperationCount());
+        Assert.assertEquals(ioCallback.getBeforeOperationCount(), ioCallback.getAfterFinalizeCount());
+        // insert more
+        testUtils.insertIntTuples(ctx, numTuplesToInsert, harness.getRandom());
+        // flush
+        flush(ctx);
+        Assert.assertEquals(getExpectedMemoryComponentIndex(1), index.getCurrentMemoryComponentIndex());
+        Assert.assertEquals(2, index.getDiskComponents().size());
+        Assert.assertEquals(ioCallback.getBeforeOperationCount(), ioCallback.getAfterOperationCount());
+        Assert.assertEquals(ioCallback.getBeforeOperationCount(), ioCallback.getAfterFinalizeCount());
+        // insert more
+        testUtils.insertIntTuples(ctx, numTuplesToInsert, harness.getRandom());
+        // flush
+        flush(ctx);
+        Assert.assertEquals(getExpectedMemoryComponentIndex(0), index.getCurrentMemoryComponentIndex());
+        Assert.assertEquals(3, index.getDiskComponents().size());
+        Assert.assertEquals(ioCallback.getBeforeOperationCount(), ioCallback.getAfterOperationCount());
+        Assert.assertEquals(ioCallback.getBeforeOperationCount(), ioCallback.getAfterFinalizeCount());
+        ctx.getIndex().deactivate();
+        ctx.getIndex().destroy();
+    }
+
+    @Test
+    public void testFlushUnModifiedComponentStartFromSecondComponent() throws Exception {
+        CountingIoOperationCallbackFactory.STARTING_INDEX = 1;
+        try {
+            testFlushUnModifiedComponent();
+        } finally {
+            CountingIoOperationCallbackFactory.STARTING_INDEX = 0;
+        }
+    }
+
+    public int getExpectedMemoryComponentIndex(int expectedIndex) {
+        return (CountingIoOperationCallbackFactory.STARTING_INDEX + expectedIndex) % 2;
+    }
+
+    @Test
+    public void testScheduleMoreFlushesThanComponents() throws Exception {
+        final AtomicInteger counter = new AtomicInteger();
+        final Semaphore flushSemaphore = new Semaphore(0);
+        OrderedIndexTestContext ctx = createTestContext(fieldSerdes, numKeys, new AsynchronousScheduler(
+                r -> new Thread(r, "LsmIoThread-" + counter.getAndIncrement()), new IIoOperationFailedCallback() {
+                    @Override
+                    public void schedulerFailed(ILSMIOOperationScheduler scheduler, Throwable failure) {
+                        LOGGER.log(Level.ERROR, "Scheduler failed", failure);
+                    }
+
+                    @Override
+                    public void operationFailed(ILSMIOOperation operation, Throwable failure) {
+                        LOGGER.log(Level.ERROR, "Operation {} failed", operation, failure);
+                    }
+                }), new EncapsulatingIoCallbackFactory(harness.getIOOperationCallbackFactory(), NoOpTestCallback.get(),
+                        NoOpTestCallback.get(), new ITestOpCallback<ILSMIOOperation>() {
+                            @Override
+                            public void before(ILSMIOOperation t) throws HyracksDataException {
+                                try {
+                                    flushSemaphore.acquire();
+                                } catch (InterruptedException e) {
+                                    throw new IllegalStateException(e);
+                                }
+                            }
+
+                            @Override
+                            public void after(ILSMIOOperation t) throws HyracksDataException {
+                            }
+                        }, NoOpTestCallback.get(), NoOpTestCallback.get()));
+        ILSMIndex index = (ILSMIndex) ctx.getIndex();
+        index.create();
+        index.activate();
+        Assert.assertEquals(getExpectedMemoryComponentIndex(0), index.getCurrentMemoryComponentIndex());
+        int numMemoryComponents = index.getNumberOfAllMemoryComponents();
+        // create a flusher that will schedule 13 flushes.
+        // wait for all flushes to be scheduled.
+        // create an inserter that will insert some records.
+        // one by one allow flushes until one flush remains, and ensure no record went in.
+        // allow the last flush, then wait for the inserts to succeed, and ensure they went to
+        // the expected memory component
+        final int numFlushes = 13;
+        User firstUser = new User("FirstUser");
+        User secondUser = new User("SecondUser");
+        Request flushRequest = new Request(Request.Statement.FLUSH, ctx, numFlushes);
+        firstUser.add(flushRequest);
+        firstUser.step();
+        // wait until all flushes have been scheduled.. Not yet performed
+        flushRequest.await(1);
+        // create an inserter and allow it to go all the way
+        Request insertRequest = new Request(Request.Statement.INSERT, ctx, 1);
+        secondUser.add(insertRequest);
+        secondUser.step();
+        secondUser.step();
+        ILSMIndexAccessor accessor = index.createAccessor(NoOpIndexAccessParameters.INSTANCE);
+        ILSMIndexOperationContext opCtx = accessor.getOpContext();
+        assertCorrectSearchComponents(opCtx, index, 0);
+        // Allow one flush at a time and ensure that inserter didn't succeed
+        for (int i = 0; i < numFlushes - 1; i++) {
+            flushSemaphore.release();
+            firstUser.step();
+            flushRequest.await(2 + i);
+            Assert.assertEquals(0, insertRequest.getSteps());
+            // also ensure that you get the correct components when searching
+            assertCorrectSearchComponents(opCtx, index, i + 1);
+        }
+        // Allow last flush to proceed
+        flushSemaphore.release();
+        // wait for the insert to complete
+        insertRequest.await();
+        firstUser.step();
+        firstUser.step();
+        flushRequest.await();
+        firstUser.stop();
+        secondUser.stop();
+
+        int expectedMemoryComponent = numFlushes % numMemoryComponents;
+        Assert.assertEquals(getExpectedMemoryComponentIndex(expectedMemoryComponent),
+                index.getCurrentMemoryComponentIndex());
+        Assert.assertEquals(0, index.getDiskComponents().size());
+
+        EncapsulatingIoCallback encapsulating = (EncapsulatingIoCallback) index.getIOOperationCallback();
+        CountingIoOperationCallback ioCallback = (CountingIoOperationCallback) encapsulating.getEncapsulated();
+        // assert equal before, after, finalize were called
+        Assert.assertEquals(numFlushes, ioCallback.getAfterOperationCount());
+        Assert.assertEquals(ioCallback.getBeforeOperationCount(), ioCallback.getAfterOperationCount());
+        Assert.assertEquals(ioCallback.getBeforeOperationCount(), ioCallback.getAfterFinalizeCount());
+        // flush, there was no insert before
+        flushSemaphore.release();
+        flush(ctx);
+        Assert.assertEquals(getExpectedMemoryComponentIndex((expectedMemoryComponent + 1) % numMemoryComponents),
+                index.getCurrentMemoryComponentIndex());
+        Assert.assertEquals(1, index.getDiskComponents().size());
+        Assert.assertEquals(ioCallback.getBeforeOperationCount(), ioCallback.getAfterOperationCount());
+        Assert.assertEquals(ioCallback.getBeforeOperationCount(), ioCallback.getAfterFinalizeCount());
+        // deactivate will cause a flush
+        flushSemaphore.release();
+        ctx.getIndex().deactivate();
+        ctx.getIndex().destroy();
+    }
+
+    private void assertCorrectSearchComponents(ILSMIndexOperationContext opCtx, ILSMIndex index,
+            int numSuccesfullyCompletedFlushes) throws HyracksDataException {
+        opCtx.reset();
+        opCtx.setOperation(IndexOperation.SEARCH);
+        index.getOperationalComponents(opCtx);
+        List<ILSMMemoryComponent> memComponents = index.getMemoryComponents();
+        int first = numSuccesfullyCompletedFlushes % memComponents.size();
+        Assert.assertEquals(memComponents.get(first), getFirstMemoryComponent(opCtx));
+    }
+
+    private ILSMComponent getFirstMemoryComponent(ILSMIndexOperationContext opCtx) {
+        List<ILSMComponent> components = opCtx.getComponentHolder();
+        // backward
+        for (int i = components.size() - 1; i >= 0; i--) {
+            ILSMComponent next = components.get(i);
+            if (next.getType() == LSMComponentType.MEMORY) {
+                return next;
+            }
+        }
+        return null;
+    }
+
+    private void flush(OrderedIndexTestContext ctx) throws HyracksDataException, InterruptedException {
+        ILSMIOOperation flush = scheduleFlush(ctx);
+        flush.sync();
+        if (flush.getStatus() == LSMIOOperationStatus.FAILURE) {
+            throw HyracksDataException.create(flush.getFailure());
+        }
+    }
+
+    private ILSMIOOperation scheduleFlush(OrderedIndexTestContext ctx)
+            throws HyracksDataException, InterruptedException {
+        ILSMIndexAccessor accessor =
+                (ILSMIndexAccessor) ctx.getIndex().createAccessor(NoOpIndexAccessParameters.INSTANCE);
+        return accessor.scheduleFlush();
+    }
+
+    private static class Request {
+        private enum Statement {
+            FLUSH,
+            INSERT
+        }
+
+        private final Statement statement;
+        private final OrderedIndexTestContext ctx;
+        private final int repeats;
+        private boolean done = false;
+        private int step = 0;
+
+        public Request(Statement statement, OrderedIndexTestContext ctx, int repeats) {
+            this.statement = statement;
+            this.ctx = ctx;
+            this.repeats = repeats;
+        }
+
+        Statement statement() {
+            return statement;
+        }
+
+        synchronized void complete() {
+            done = true;
+            notifyAll();
+        }
+
+        synchronized void await() throws InterruptedException {
+            while (!done) {
+                wait();
+            }
+        }
+
+        synchronized void step() {
+            step++;
+            notifyAll();
+        }
+
+        synchronized int getSteps() {
+            return step;
+        }
+
+        synchronized void await(int step) throws InterruptedException {
+            while (this.step < step) {
+                wait();
+            }
+        }
+    }
+
+    private class User extends SingleThreadEventProcessor<Request> {
+
+        private Semaphore step = new Semaphore(0);
+
+        public User(String username) {
+            super(username);
+        }
+
+        public void step() {
+            step.release();
+        }
+
+        @Override
+        protected void handle(Request req) throws Exception {
+            try {
+                step.acquire();
+                switch (req.statement()) {
+                    case FLUSH:
+                        List<ILSMIOOperation> flushes = new ArrayList<>(req.repeats);
+                        for (int i = 0; i < req.repeats; i++) {
+                            flushes.add(scheduleFlush(req.ctx));
+                        }
+                        req.step();
+                        for (ILSMIOOperation op : flushes) {
+                            step.acquire();
+                            op.sync();
+                            if (op.getStatus() == LSMIOOperationStatus.FAILURE) {
+                                throw HyracksDataException.create(op.getFailure());
+                            }
+                            req.step(); // report after completion of each flush
+                        }
+                        break;
+                    case INSERT:
+                        testUtils.insertIntTuples(req.ctx, numTuplesToInsert, harness.getRandom());
+                        break;
+                    default:
+                        break;
+                }
+                req.step();
+                step.acquire();
+            } finally {
+                req.step();
+                req.complete();
+            }
+        }
+    }
+}
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 9d768aa..03b221e 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
@@ -29,7 +29,6 @@
 import org.apache.hyracks.storage.am.lsm.btree.impls.LSMBTreeWithBloomFilterDiskComponent;
 import org.apache.hyracks.storage.am.lsm.btree.util.LSMBTreeTestContext;
 import org.apache.hyracks.storage.am.lsm.btree.util.LSMBTreeTestHarness;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndex;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndexAccessor;
 import org.junit.After;
 import org.junit.Assert;
@@ -71,8 +70,8 @@
         ILSMIndexAccessor accessor = (ILSMIndexAccessor) ctx.getIndexAccessor();
         accessor.insert(tuple);
 
-        // Flush to generate a disk component
-        accessor.scheduleFlush(((ILSMIndex) ctx.getIndex()).getIOOperationCallback());
+        // Flush to generate a disk component. This uses synchronous scheduler
+        accessor.scheduleFlush();
 
         // Make sure the disk component was generated
         LSMBTree btree = (LSMBTree) ctx.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/LSMBTreeFilterMergeTestDriver.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/LSMBTreeFilterMergeTestDriver.java
index 815c8bc..1b9a7a5 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/LSMBTreeFilterMergeTestDriver.java
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/LSMBTreeFilterMergeTestDriver.java
@@ -40,9 +40,9 @@
 import org.apache.hyracks.storage.am.lsm.btree.impls.LSMBTree;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponentFilter;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMDiskComponent;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperation;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperation.LSMIOOperationStatus;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndexAccessor;
-import org.apache.hyracks.storage.am.lsm.common.impls.BlockingIOOperationCallbackWrapper;
-import org.apache.hyracks.storage.am.lsm.common.impls.StubIOOperationCallback;
 import org.junit.Assert;
 
 /**
@@ -113,13 +113,14 @@
                             TreeIndexTestUtils.compareFilterTuples(obsMinMax.getRight(), minMax.getRight(), comp));
                 }
 
-                StubIOOperationCallback stub = new StubIOOperationCallback();
-                BlockingIOOperationCallbackWrapper waiter = new BlockingIOOperationCallbackWrapper(stub);
-                accessor.scheduleFlush(waiter);
-                waiter.waitForIO();
+                ILSMIOOperation flush = accessor.scheduleFlush();
+                flush.sync();
+                if (flush.getStatus() == LSMIOOperationStatus.FAILURE) {
+                    throw HyracksDataException.create(flush.getFailure());
+                }
                 if (minMax != null) {
                     Pair<ITupleReference, ITupleReference> obsMinMax =
-                            filterToMinMax(stub.getLastNewComponent().getLSMComponentFilter());
+                            filterToMinMax(flush.getNewComponent().getLSMComponentFilter());
                     Assert.assertEquals(0,
                             TreeIndexTestUtils.compareFilterTuples(obsMinMax.getLeft(), minMax.getLeft(), comp));
                     Assert.assertEquals(0,
@@ -143,8 +144,7 @@
                     expectedMergeMinMax.setRight(componentMinMax.getRight());
                 }
             }
-            accessor.scheduleMerge(((LSMBTree) ctx.getIndex()).getIOOperationCallback(),
-                    ((LSMBTree) ctx.getIndex()).getDiskComponents());
+            accessor.scheduleMerge(((LSMBTree) ctx.getIndex()).getDiskComponents());
 
             flushedComponents = ((LSMBTree) ctx.getIndex()).getDiskComponents();
             Pair<ITupleReference, ITupleReference> mergedMinMax =
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 475ab9c..a8639c3 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
@@ -35,6 +35,7 @@
 import org.apache.hyracks.storage.am.lsm.btree.util.LSMBTreeTestContext;
 import org.apache.hyracks.storage.am.lsm.btree.util.LSMBTreeTestHarness;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperation;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperation.LSMIOOperationStatus;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationScheduler;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndexAccessor;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndexOperationContext;
@@ -76,32 +77,24 @@
         ITupleReference tuple1 = TupleUtils.createIntegerTuple(1, 1, 1);
         accessor.insert(tuple1);
         // flush component
-        accessor.scheduleFlush(btree.getIOOperationCallback());
+        accessor.scheduleFlush();
 
         ITupleReference tuple2 = TupleUtils.createIntegerTuple(2, 2, 2);
         accessor.insert(tuple2);
         // flush component
-        accessor.scheduleFlush(btree.getIOOperationCallback());
+        accessor.scheduleFlush();
 
         ITupleReference tuple3 = TupleUtils.createIntegerTuple(3, 3, 3);
         accessor.insert(tuple3);
         // flush component
-        accessor.scheduleFlush(btree.getIOOperationCallback());
-
+        accessor.scheduleFlush();
         scheduler.modify = true;
-
-        boolean exceptionThrown = false;
-        try {
-            accessor.scheduleMerge(btree.getIOOperationCallback(), btree.getDiskComponents());
-        } catch (HyracksDataException e) {
-            exceptionThrown = true;
-        }
-        Assert.assertTrue(exceptionThrown);
-
+        ILSMIOOperation merge = accessor.scheduleMerge(btree.getDiskComponents());
+        merge.sync();
+        Assert.assertEquals(LSMIOOperationStatus.FAILURE, merge.getStatus());
         scheduler.modify = false;
-        accessor.scheduleMerge(btree.getIOOperationCallback(), btree.getDiskComponents());
+        accessor.scheduleMerge(btree.getDiskComponents());
         Assert.assertEquals(1, btree.getDiskComponents().size());
-
         btree.deactivate();
         btree.destroy();
     }
@@ -120,14 +113,18 @@
 
         @Override
         public void scheduleOperation(ILSMIOOperation operation) throws HyracksDataException {
-            if (modify) {
-                try {
-                    modifyOperation(operation);
-                } catch (Exception e) {
-                    throw HyracksDataException.create(e);
+            try {
+                if (modify) {
+                    try {
+                        modifyOperation(operation);
+                    } catch (Exception e) {
+                        throw HyracksDataException.create(e);
+                    }
                 }
+                operation.call();
+            } finally {
+                operation.complete();
             }
-            operation.call();
         }
 
         private void modifyOperation(ILSMIOOperation operation) throws Exception {
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/LSMBTreeMergeTestDriver.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/LSMBTreeMergeTestDriver.java
index 87ddf33..7c59671 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/LSMBTreeMergeTestDriver.java
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/LSMBTreeMergeTestDriver.java
@@ -74,8 +74,7 @@
             }
 
             ILSMIndexAccessor accessor = (ILSMIndexAccessor) ctx.getIndexAccessor();
-            accessor.scheduleMerge(((LSMBTree) ctx.getIndex()).getIOOperationCallback(),
-                    ((LSMBTree) ctx.getIndex()).getDiskComponents());
+            accessor.scheduleMerge(((LSMBTree) ctx.getIndex()).getDiskComponents());
 
             orderedIndexTestUtils.checkPointSearches(ctx);
             orderedIndexTestUtils.checkScan(ctx);
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 9f17efa..4fafb38 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/LSMBTreeModificationOperationCallbackTest.java
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/LSMBTreeModificationOperationCallbackTest.java
@@ -19,6 +19,7 @@
 
 package org.apache.hyracks.storage.am.lsm.btree;
 
+import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.dataflow.common.utils.SerdeUtils;
 import org.apache.hyracks.dataflow.common.utils.TupleUtils;
 import org.apache.hyracks.storage.am.btree.AbstractModificationOperationCallbackTest;
@@ -26,9 +27,9 @@
 import org.apache.hyracks.storage.am.common.impls.NoOpOperationCallback;
 import org.apache.hyracks.storage.am.lsm.btree.util.LSMBTreeTestHarness;
 import org.apache.hyracks.storage.am.lsm.btree.utils.LSMBTreeUtil;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndex;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperation;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperation.LSMIOOperationStatus;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndexAccessor;
-import org.apache.hyracks.storage.am.lsm.common.impls.BlockingIOOperationCallbackWrapper;
 import org.apache.hyracks.storage.am.lsm.common.impls.NoOpOperationTrackerFactory;
 import org.apache.hyracks.util.trace.ITracer;
 import org.junit.Test;
@@ -72,8 +73,6 @@
         IndexAccessParameters actx = new IndexAccessParameters(cb, NoOpOperationCallback.INSTANCE);
         ILSMIndexAccessor accessor = (ILSMIndexAccessor) index.createAccessor(actx);
 
-        BlockingIOOperationCallbackWrapper ioOpCallback =
-                new BlockingIOOperationCallbackWrapper(((ILSMIndex) index).getIOOperationCallback());
         for (int j = 0; j < 2; j++) {
             isFoundNull = true;
             for (int i = 0; i < NUM_TUPLES; i++) {
@@ -82,8 +81,11 @@
             }
 
             if (j == 1) {
-                accessor.scheduleFlush(ioOpCallback);
-                ioOpCallback.waitForIO();
+                ILSMIOOperation flush = accessor.scheduleFlush();
+                flush.sync();
+                if (flush.getStatus() == LSMIOOperationStatus.FAILURE) {
+                    throw HyracksDataException.create(flush.getFailure());
+                }
                 isFoundNull = true;
             } else {
                 isFoundNull = false;
@@ -95,8 +97,7 @@
             }
 
             if (j == 1) {
-                accessor.scheduleFlush(ioOpCallback);
-                ioOpCallback.waitForIO();
+                accessor.scheduleFlush().sync();
                 isFoundNull = true;
             } else {
                 isFoundNull = false;
@@ -106,9 +107,7 @@
                 TupleUtils.createIntegerTuple(builder, tuple, i);
                 accessor.delete(tuple);
             }
-
-            accessor.scheduleFlush(ioOpCallback);
-            ioOpCallback.waitForIO();
+            accessor.scheduleFlush().sync();
         }
     }
 }
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 48812b4..2574c4d 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
@@ -44,8 +44,6 @@
 import org.apache.hyracks.storage.am.lsm.btree.impls.LSMBTree;
 import org.apache.hyracks.storage.am.lsm.btree.util.LSMBTreeTestContext;
 import org.apache.hyracks.storage.am.lsm.btree.util.LSMBTreeTestHarness;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallback;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndex;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndexAccessor;
 import org.apache.hyracks.storage.common.IIndexCursor;
 import org.junit.After;
@@ -113,21 +111,20 @@
     protected void test(OrderedIndexTestContext ctx, ISerializerDeserializer[] fieldSerdes)
             throws HyracksDataException {
         ILSMIndexAccessor accessor = (ILSMIndexAccessor) ctx.getIndexAccessor();
-        ILSMIOOperationCallback ioCallback = ((ILSMIndex) ctx.getIndex()).getIOOperationCallback();
         //component 2 contains 1 and 2
         upsertTuple(ctx, fieldSerdes, getValue(1, fieldSerdes));
         upsertTuple(ctx, fieldSerdes, getValue(2, fieldSerdes));
-        accessor.scheduleFlush(ioCallback);
+        accessor.scheduleFlush();
 
         //component 1 contains 1 and -2
         upsertTuple(ctx, fieldSerdes, getValue(1, fieldSerdes));
         deleteTuple(ctx, fieldSerdes, getValue(2, fieldSerdes));
-        accessor.scheduleFlush(ioCallback);
+        accessor.scheduleFlush();
 
         //component 0 contains 2 and 3
         upsertTuple(ctx, fieldSerdes, getValue(3, fieldSerdes));
         upsertTuple(ctx, fieldSerdes, getValue(2, fieldSerdes));
-        accessor.scheduleFlush(ioCallback);
+        accessor.scheduleFlush();
 
         LSMBTree btree = (LSMBTree) ctx.getIndex();
         Assert.assertEquals("Check disk components", 3, btree.getDiskComponents().size());
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 3a48160..afcb3c1 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
@@ -52,8 +52,6 @@
 import org.apache.hyracks.storage.am.lsm.btree.util.LSMBTreeTestContext;
 import org.apache.hyracks.storage.am.lsm.btree.util.LSMBTreeTestHarness;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMDiskComponent;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallback;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndex;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndexAccessor;
 import org.apache.hyracks.storage.common.IIndexCursor;
 import org.junit.After;
@@ -352,11 +350,10 @@
             throws HyracksDataException {
 
         ILSMIndexAccessor accessor = (ILSMIndexAccessor) ctx.getIndexAccessor();
-        ILSMIOOperationCallback ioCallback = ((ILSMIndex) ctx.getIndex()).getIOOperationCallback();
         op1.performOperation(ctx, AccessMethodTestsConfig.BTREE_NUM_TUPLES_TO_INSERT);
         op2.performOperation(ctx,
                 AccessMethodTestsConfig.BTREE_NUM_TUPLES_TO_INSERT / AccessMethodTestsConfig.BTREE_NUM_INSERT_ROUNDS);
-        accessor.scheduleFlush(ioCallback);
+        accessor.scheduleFlush();
         LSMBTree btree = (LSMBTree) ctx.getIndex();
         Assert.assertEquals("Check disk components", 1, btree.getDiskComponents().size());
 
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 fc852cd..0914541 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
@@ -35,9 +35,9 @@
 import org.apache.hyracks.storage.am.config.AccessMethodTestsConfig;
 import org.apache.hyracks.storage.am.lsm.btree.util.LSMBTreeTestHarness;
 import org.apache.hyracks.storage.am.lsm.btree.utils.LSMBTreeUtil;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndex;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperation;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperation.LSMIOOperationStatus;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndexAccessor;
-import org.apache.hyracks.storage.am.lsm.common.impls.BlockingIOOperationCallbackWrapper;
 import org.apache.hyracks.storage.am.lsm.common.impls.NoOpOperationTrackerFactory;
 import org.apache.hyracks.storage.common.IIndexAccessor;
 import org.apache.hyracks.storage.common.IModificationOperationCallback;
@@ -99,8 +99,6 @@
 
     private void test(IndexModification op1, IndexModification op2) throws Exception {
         ILSMIndexAccessor lsmAccessor = (ILSMIndexAccessor) accessor;
-        BlockingIOOperationCallbackWrapper ioOpCallback =
-                new BlockingIOOperationCallbackWrapper(((ILSMIndex) index).getIOOperationCallback());
         for (int j = 0; j < 2; j++) {
             index.clear();
             isFoundNull = true;
@@ -111,8 +109,11 @@
             }
 
             if (j == 1) {
-                lsmAccessor.scheduleFlush(ioOpCallback);
-                ioOpCallback.waitForIO();
+                ILSMIOOperation flush = lsmAccessor.scheduleFlush();
+                flush.sync();
+                if (flush.getStatus() == LSMIOOperationStatus.FAILURE) {
+                    throw HyracksDataException.create(flush.getFailure());
+                }
                 isFoundNull = true;
                 isUpdated = false;
             } else {
@@ -126,8 +127,7 @@
             }
 
             if (j == 1) {
-                lsmAccessor.scheduleFlush(ioOpCallback);
-                ioOpCallback.waitForIO();
+                lsmAccessor.scheduleFlush().sync();
             } else {
                 isFoundNull = false;
             }
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/AllowTestOpCallback.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/impl/AllowTestOpCallback.java
index 19a9872..2e43198 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/impl/AllowTestOpCallback.java
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/impl/AllowTestOpCallback.java
@@ -35,7 +35,7 @@
     }
 
     @Override
-    public void after() throws HyracksDataException {
+    public void after(Semaphore t) throws HyracksDataException {
 
     }
 }
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/CountingIoOperationCallback.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/impl/CountingIoOperationCallback.java
new file mode 100644
index 0000000..84b530a
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/impl/CountingIoOperationCallback.java
@@ -0,0 +1,101 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.hyracks.storage.am.lsm.btree.impl;
+
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperation;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallback;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMMemoryComponent;
+
+public class CountingIoOperationCallback implements ILSMIOOperationCallback {
+    private int beforeOperation;
+    private int afterOperation;
+    private int afterFinalize;
+    private int recycled;
+    private int allocated;
+    private int beforeSchedule;
+    private int destroy;
+
+    public CountingIoOperationCallback() {
+    }
+
+    public int getAfterFinalizeCount() {
+        return afterFinalize;
+    }
+
+    @Override
+    public void recycled(ILSMMemoryComponent component) throws HyracksDataException {
+        recycled++;
+    }
+
+    public int getRecycledCount() {
+        return recycled;
+    }
+
+    @Override
+    public void allocated(ILSMMemoryComponent component) throws HyracksDataException {
+        allocated++;
+    }
+
+    public int getAllocatedCount() {
+        return allocated;
+    }
+
+    @Override
+    public void scheduled(ILSMIOOperation operation) throws HyracksDataException {
+        beforeSchedule++;
+    }
+
+    public int getBeforeScheduleCount() {
+        return beforeSchedule;
+    }
+
+    @Override
+    public void beforeOperation(ILSMIOOperation operation) throws HyracksDataException {
+        beforeOperation++;
+    }
+
+    public int getBeforeOperationCount() {
+        return beforeOperation;
+    }
+
+    @Override
+    public void afterOperation(ILSMIOOperation operation) throws HyracksDataException {
+        afterOperation++;
+    }
+
+    public int getAfterOperationCount() {
+        return afterOperation;
+    }
+
+    @Override
+    public void afterFinalize(ILSMIOOperation operation) throws HyracksDataException {
+        afterFinalize++;
+    }
+
+    @Override
+    public void completed(ILSMIOOperation operation) {
+        destroy++;
+    }
+
+    public int getDestroyCount() {
+        return destroy;
+    }
+
+}
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/ioopcallbacks/LSMBTreeWithBuddyIOOperationCallbackFactory.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/impl/CountingIoOperationCallbackFactory.java
similarity index 63%
rename from asterixdb/asterix-common/src/main/java/org/apache/asterix/common/ioopcallbacks/LSMBTreeWithBuddyIOOperationCallbackFactory.java
rename to hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/impl/CountingIoOperationCallbackFactory.java
index 9b32345..b64b9b1 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/ioopcallbacks/LSMBTreeWithBuddyIOOperationCallbackFactory.java
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/impl/CountingIoOperationCallbackFactory.java
@@ -16,24 +16,33 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.asterix.common.ioopcallbacks;
+package org.apache.hyracks.storage.am.lsm.btree.impl;
 
+import org.apache.hyracks.api.application.INCServiceContext;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponentIdGeneratorFactory;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallback;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallbackFactory;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndex;
+import org.apache.hyracks.storage.common.IResource;
 
-public class LSMBTreeWithBuddyIOOperationCallbackFactory extends AbstractLSMIndexIOOperationCallbackFactory {
+public class CountingIoOperationCallbackFactory implements ILSMIOOperationCallbackFactory {
 
     private static final long serialVersionUID = 1L;
+    public static int STARTING_INDEX = 0;
 
-    public LSMBTreeWithBuddyIOOperationCallbackFactory(ILSMComponentIdGeneratorFactory idGeneratorFactory) {
-        super(idGeneratorFactory);
+    @Override
+    public void initialize(INCServiceContext ncCtx, IResource resource) {
+        // No op
     }
 
     @Override
     public ILSMIOOperationCallback createIoOpCallback(ILSMIndex index) throws HyracksDataException {
-        return new LSMBTreeWithBuddyIOOperationCallback(index, getComponentIdGenerator(),
-                getIndexCheckpointManagerProvider());
+        return new CountingIoOperationCallback();
     }
+
+    @Override
+    public int getCurrentMemoryComponentIndex() throws HyracksDataException {
+        return STARTING_INDEX;
+    }
+
 }
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/ITestOpCallback.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/impl/ITestOpCallback.java
index e888238..fa3097d 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/impl/ITestOpCallback.java
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/impl/ITestOpCallback.java
@@ -23,5 +23,5 @@
 public interface ITestOpCallback<T> {
     void before(T t) throws HyracksDataException;
 
-    void after() throws HyracksDataException;
+    void after(T t) throws HyracksDataException;
 }
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/NoOpTestCallback.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/impl/NoOpTestCallback.java
new file mode 100644
index 0000000..2370663
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/impl/NoOpTestCallback.java
@@ -0,0 +1,45 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.hyracks.storage.am.lsm.btree.impl;
+
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+@SuppressWarnings("rawtypes")
+public class NoOpTestCallback implements ITestOpCallback {
+    public static final NoOpTestCallback INSTANCE = new NoOpTestCallback();
+
+    private NoOpTestCallback() {
+    }
+
+    @SuppressWarnings("unchecked")
+    public static final <T> ITestOpCallback<T> get() {
+        return INSTANCE;
+    }
+
+    @Override
+    public void before(Object t) throws HyracksDataException {
+        // NoOp
+    }
+
+    @Override
+    public void after(Object t) throws HyracksDataException {
+        // NoOp
+    }
+
+}
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 1d4b7d6..4867c71 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
@@ -34,7 +34,6 @@
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMDiskComponent;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMDiskComponentFactory;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperation;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallback;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallbackFactory;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationScheduler;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndexAccessor;
@@ -65,9 +64,11 @@
 
     private final List<ITestOpCallback<ILSMMemoryComponent>> ioAllocateCallbacks = new ArrayList<>();
     private final List<ITestOpCallback<ILSMMemoryComponent>> ioRecycleCallbacks = new ArrayList<>();
+    private final List<ITestOpCallback<Void>> ioOpScheduledCallbacks = new ArrayList<>();
     private final List<ITestOpCallback<Void>> ioBeforeCallbacks = new ArrayList<>();
     private final List<ITestOpCallback<Void>> ioAfterOpCallbacks = new ArrayList<>();
     private final List<ITestOpCallback<Void>> ioAfterFinalizeCallbacks = new ArrayList<>();
+    private final List<ITestOpCallback<Void>> ioOpCompletedCallbacks = new ArrayList<>();
     private final List<ITestOpCallback<Void>> allocateComponentCallbacks = new ArrayList<>();
 
     private volatile int numScheduledFlushes;
@@ -110,7 +111,7 @@
         super.modify(ictx, tuple);
         synchronized (modifyCallbacks) {
             for (ITestOpCallback<Semaphore> callback : modifyCallbacks) {
-                callback.after();
+                callback.after(modifySemaphore);
             }
         }
     }
@@ -122,17 +123,15 @@
     }
 
     @Override
-    public void scheduleFlush(ILSMIndexOperationContext ctx, ILSMIOOperationCallback callback)
-            throws HyracksDataException {
-        super.scheduleFlush(ctx, callback);
+    public ILSMIOOperation createFlushOperation(ILSMIndexOperationContext ctx) throws HyracksDataException {
         numScheduledFlushes++;
+        return super.createFlushOperation(ctx);
     }
 
     @Override
-    public void scheduleMerge(ILSMIndexOperationContext ctx, ILSMIOOperationCallback callback)
-            throws HyracksDataException {
-        super.scheduleMerge(ctx, callback);
+    public ILSMIOOperation createMergeOperation(ILSMIndexOperationContext ctx) throws HyracksDataException {
         numScheduledMerges++;
+        return super.createMergeOperation(ctx);
     }
 
     @Override
@@ -148,7 +147,7 @@
         numFinishedFlushes++;
         synchronized (flushCallbacks) {
             for (ITestOpCallback<Semaphore> callback : flushCallbacks) {
-                callback.after();
+                callback.after(flushSemaphore);
             }
         }
         return c;
@@ -167,7 +166,7 @@
         numFinishedMerges++;
         synchronized (mergeCallbacks) {
             for (ITestOpCallback<Semaphore> callback : mergeCallbacks) {
-                callback.after();
+                callback.after(mergeSemaphore);
             }
         }
         return c;
@@ -347,6 +346,30 @@
         }
     }
 
+    public void addIoScheduledCallback(ITestOpCallback<Void> callback) {
+        synchronized (ioOpScheduledCallbacks) {
+            ioOpScheduledCallbacks.add(callback);
+        }
+    }
+
+    public void clearIoScheduledCallbacks() {
+        synchronized (ioOpScheduledCallbacks) {
+            ioOpScheduledCallbacks.clear();
+        }
+    }
+
+    public void addIoCompletedCallback(ITestOpCallback<Void> callback) {
+        synchronized (ioOpCompletedCallbacks) {
+            ioOpCompletedCallbacks.add(callback);
+        }
+    }
+
+    public void clearIoCompletedCallbacks() {
+        synchronized (ioOpCompletedCallbacks) {
+            ioOpCompletedCallbacks.clear();
+        }
+    }
+
     @Override
     public void allocateMemoryComponents() throws HyracksDataException {
         synchronized (allocateComponentCallbacks) {
@@ -357,7 +380,7 @@
         super.allocateMemoryComponents();
         synchronized (allocateComponentCallbacks) {
             for (ITestOpCallback<Void> callback : allocateComponentCallbacks) {
-                callback.after();
+                callback.after(null);
             }
         }
     }
@@ -373,7 +396,7 @@
     public void beforeIoOperationReturned() throws HyracksDataException {
         synchronized (ioBeforeCallbacks) {
             for (ITestOpCallback<Void> callback : ioBeforeCallbacks) {
-                callback.after();
+                callback.after(null);
             }
         }
     }
@@ -389,7 +412,7 @@
     public void afterIoOperationReturned() throws HyracksDataException {
         synchronized (ioAfterOpCallbacks) {
             for (ITestOpCallback<Void> callback : ioAfterOpCallbacks) {
-                callback.after();
+                callback.after(null);
             }
         }
     }
@@ -405,7 +428,39 @@
     public void afterIoFinalizeReturned() throws HyracksDataException {
         synchronized (ioAfterFinalizeCallbacks) {
             for (ITestOpCallback<Void> callback : ioAfterFinalizeCallbacks) {
-                callback.after();
+                callback.after(null);
+            }
+        }
+    }
+
+    public void ioScheduledCalled() throws HyracksDataException {
+        synchronized (ioOpScheduledCallbacks) {
+            for (ITestOpCallback<Void> callback : ioOpScheduledCallbacks) {
+                callback.before(null);
+            }
+        }
+    }
+
+    public void ioScheduledReturned() throws HyracksDataException {
+        synchronized (ioOpScheduledCallbacks) {
+            for (ITestOpCallback<Void> callback : ioOpScheduledCallbacks) {
+                callback.after(null);
+            }
+        }
+    }
+
+    public void ioCompletedCalled() throws HyracksDataException {
+        synchronized (ioOpCompletedCallbacks) {
+            for (ITestOpCallback<Void> callback : ioOpCompletedCallbacks) {
+                callback.before(null);
+            }
+        }
+    }
+
+    public void ioCompletedReturned() throws HyracksDataException {
+        synchronized (ioOpCompletedCallbacks) {
+            for (ITestOpCallback<Void> callback : ioOpCompletedCallbacks) {
+                callback.after(null);
             }
         }
     }
@@ -421,7 +476,7 @@
     public void recycledReturned(ILSMMemoryComponent component) throws HyracksDataException {
         synchronized (ioRecycleCallbacks) {
             for (ITestOpCallback<ILSMMemoryComponent> callback : ioRecycleCallbacks) {
-                callback.after();
+                callback.after(component);
             }
         }
     }
@@ -437,7 +492,7 @@
     public void allocatedReturned(ILSMMemoryComponent component) throws HyracksDataException {
         synchronized (ioAllocateCallbacks) {
             for (ITestOpCallback<ILSMMemoryComponent> callback : ioAllocateCallbacks) {
-                callback.after();
+                callback.after(component);
             }
         }
     }
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/TestVirtualBufferCache.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/impl/TestVirtualBufferCache.java
index c7e064f..2e7a9f0 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/impl/TestVirtualBufferCache.java
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/impl/TestVirtualBufferCache.java
@@ -212,4 +212,10 @@
     public IFileMapManager getFileMapProvider() {
         return vbc.getFileMapProvider();
     }
+
+    @Override
+    public void closeFileIfOpen(FileReference fileRef) {
+        throw new UnsupportedOperationException();
+    }
+
 }
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/LSMBTreeTestWorker.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/multithread/LSMBTreeTestWorker.java
index 4c07dea..583fd5a 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/multithread/LSMBTreeTestWorker.java
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/multithread/LSMBTreeTestWorker.java
@@ -116,7 +116,7 @@
                 consumeCursorTuples(searchCursor);
                 break;
             case MERGE:
-                accessor.scheduleMerge(lsmBTree.getIOOperationCallback(), lsmBTree.getDiskComponents());
+                accessor.scheduleMerge(lsmBTree.getDiskComponents());
                 break;
 
             default:
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 a8e0aee..2462c85 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
@@ -36,7 +36,9 @@
 import org.apache.hyracks.storage.am.common.impls.NoOpIndexAccessParameters;
 import org.apache.hyracks.storage.am.lsm.btree.impls.LSMBTree;
 import org.apache.hyracks.storage.am.lsm.btree.utils.LSMBTreeUtil;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperation;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationScheduler;
+import org.apache.hyracks.storage.am.lsm.common.api.IIoOperationFailedCallback;
 import org.apache.hyracks.storage.am.lsm.common.api.IVirtualBufferCache;
 import org.apache.hyracks.storage.am.lsm.common.impls.AsynchronousScheduler;
 import org.apache.hyracks.storage.am.lsm.common.impls.NoMergePolicy;
@@ -48,10 +50,14 @@
 import org.apache.hyracks.storage.common.buffercache.IBufferCache;
 import org.apache.hyracks.test.support.TestStorageManagerComponentHolder;
 import org.apache.hyracks.test.support.TestUtils;
+import org.apache.hyracks.util.ExitUtil;
 import org.apache.hyracks.util.trace.ITracer;
+import org.apache.logging.log4j.LogManager;
+import org.apache.logging.log4j.Logger;
 
 public class LSMTreeRunner implements IExperimentRunner {
 
+    private static final Logger LOGGER = LogManager.getLogger();
     private static final int MAX_OPEN_FILES = Integer.MAX_VALUE;
     private static final int HYRACKS_FRAME_SIZE = 131072;
 
@@ -103,8 +109,17 @@
             virtualBufferCaches.add(virtualBufferCache);
         }
 
-        this.ioScheduler = AsynchronousScheduler.INSTANCE;
-        AsynchronousScheduler.INSTANCE.init(threadFactory);
+        this.ioScheduler = new AsynchronousScheduler(threadFactory, new IIoOperationFailedCallback() {
+            @Override
+            public void operationFailed(ILSMIOOperation operation, Throwable t) {
+                LOGGER.error("Operation {} failed", operation, t);
+            }
+
+            @Override
+            public void schedulerFailed(ILSMIOOperationScheduler scheduler, Throwable failure) {
+                ExitUtil.exit(ExitUtil.EC_IO_SCHEDULER_FAILED);
+            }
+        });
 
         lsmtree = LSMBTreeUtil.createLSMTree(ioManager, virtualBufferCaches, file, bufferCache, typeTraits,
                 cmpFactories, bloomFilterKeyFields, bloomFilterFalsePositiveRate, new NoMergePolicy(),
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 803c5cb..707628b 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
@@ -36,14 +36,14 @@
 import org.apache.hyracks.storage.am.common.api.IMetadataPageManagerFactory;
 import org.apache.hyracks.storage.am.common.freepage.AppendOnlyLinkedMetadataPageManagerFactory;
 import org.apache.hyracks.storage.am.config.AccessMethodTestsConfig;
+import org.apache.hyracks.storage.am.lsm.btree.impl.CountingIoOperationCallbackFactory;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallbackFactory;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationScheduler;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMMergePolicy;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMOperationTracker;
 import org.apache.hyracks.storage.am.lsm.common.api.IVirtualBufferCache;
 import org.apache.hyracks.storage.am.lsm.common.impls.NoMergePolicy;
-import org.apache.hyracks.storage.am.lsm.common.impls.NoOpIOOperationCallbackFactory;
-import org.apache.hyracks.storage.am.lsm.common.impls.SynchronousScheduler;
+import org.apache.hyracks.storage.am.lsm.common.impls.SynchronousSchedulerProvider;
 import org.apache.hyracks.storage.am.lsm.common.impls.ThreadCountingTracker;
 import org.apache.hyracks.storage.am.lsm.common.impls.VirtualBufferCache;
 import org.apache.hyracks.storage.common.buffercache.HeapBufferAllocator;
@@ -95,12 +95,12 @@
         this.memNumPages = AccessMethodTestsConfig.LSM_BTREE_MEM_NUM_PAGES;
         this.hyracksFrameSize = AccessMethodTestsConfig.LSM_BTREE_HYRACKS_FRAME_SIZE;
         this.bloomFilterFalsePositiveRate = AccessMethodTestsConfig.LSM_BTREE_BLOOMFILTER_FALSE_POSITIVE_RATE;
-        this.ioScheduler = SynchronousScheduler.INSTANCE;
+        this.ioScheduler = SynchronousSchedulerProvider.INSTANCE.getIoScheduler(null);
         this.mergePolicy = new NoMergePolicy();
         this.opTracker = new ThreadCountingTracker();
         this.numMutableComponents = AccessMethodTestsConfig.LSM_BTREE_NUM_MUTABLE_COMPONENTS;
         this.metadataPageManagerFactory = AppendOnlyLinkedMetadataPageManagerFactory.INSTANCE;
-        this.ioOpCallbackFactory = NoOpIOOperationCallbackFactory.INSTANCE;
+        this.ioOpCallbackFactory = new CountingIoOperationCallbackFactory();
     }
 
     public void setUp() throws HyracksDataException {
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-common-test/src/test/java/org/apache/hyracks/storage/am/lsm/common/test/PrefixMergePolicyTest.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-common-test/src/test/java/org/apache/hyracks/storage/am/lsm/common/test/PrefixMergePolicyTest.java
index 2fd1dbc..54fd3bb 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-common-test/src/test/java/org/apache/hyracks/storage/am/lsm/common/test/PrefixMergePolicyTest.java
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-common-test/src/test/java/org/apache/hyracks/storage/am/lsm/common/test/PrefixMergePolicyTest.java
@@ -30,7 +30,6 @@
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponent.ComponentState;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMDiskComponent;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallback;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndex;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndexAccessor;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMMergePolicy;
@@ -197,7 +196,7 @@
         Mockito.doAnswer(new Answer<Void>() {
             @Override
             public Void answer(InvocationOnMock invocation) throws Throwable {
-                List<ILSMDiskComponent> mergedComponents = invocation.getArgumentAt(1, List.class);
+                List<ILSMDiskComponent> mergedComponents = invocation.getArgumentAt(0, List.class);
                 if (mergedSizes != null) {
                     mergedComponents.forEach(component -> {
                         mergedSizes.add(component.getComponentSize());
@@ -219,8 +218,7 @@
                 }
                 return null;
             }
-        }).when(accessor).scheduleMerge(Mockito.any(ILSMIOOperationCallback.class),
-                Mockito.anyListOf(ILSMDiskComponent.class));
+        }).when(accessor).scheduleMerge(Mockito.anyListOf(ILSMDiskComponent.class));
 
         Mockito.when(index.createAccessor(Mockito.any(IIndexAccessParameters.class))).thenReturn(accessor);
 
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/org/apache/hyracks/storage/am/lsm/invertedindex/LSMInvertedIndexMergeTest.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/org/apache/hyracks/storage/am/lsm/invertedindex/LSMInvertedIndexMergeTest.java
index 4657109..1a882b8 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/org/apache/hyracks/storage/am/lsm/invertedindex/LSMInvertedIndexMergeTest.java
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/org/apache/hyracks/storage/am/lsm/invertedindex/LSMInvertedIndexMergeTest.java
@@ -56,8 +56,7 @@
                 invIndex.activate();
             }
             // Perform merge.
-            invIndexAccessor.scheduleMerge(((LSMInvertedIndex) invIndex).getIOOperationCallback(),
-                    ((LSMInvertedIndex) invIndex).getDiskComponents());
+            invIndexAccessor.scheduleMerge(((LSMInvertedIndex) invIndex).getDiskComponents());
             validateAndCheckIndex(testCtx);
             runTinySearchWorkload(testCtx, tupleGen);
         }
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/org/apache/hyracks/storage/am/lsm/invertedindex/PartitionedLSMInvertedIndexMergeTest.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/org/apache/hyracks/storage/am/lsm/invertedindex/PartitionedLSMInvertedIndexMergeTest.java
index 584de0c..f5a7c9a 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/org/apache/hyracks/storage/am/lsm/invertedindex/PartitionedLSMInvertedIndexMergeTest.java
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/org/apache/hyracks/storage/am/lsm/invertedindex/PartitionedLSMInvertedIndexMergeTest.java
@@ -58,8 +58,7 @@
                 invIndex.activate();
             }
             // Perform merge.
-            invIndexAccessor.scheduleMerge(((LSMInvertedIndex) invIndex).getIOOperationCallback(),
-                    ((LSMInvertedIndex) invIndex).getDiskComponents());
+            invIndexAccessor.scheduleMerge(((LSMInvertedIndex) invIndex).getDiskComponents());
             validateAndCheckIndex(testCtx);
             runTinySearchWorkload(testCtx, tupleGen);
         }
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 8e41c86..c1e1b4c 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
@@ -43,7 +43,7 @@
 import org.apache.hyracks.storage.am.lsm.common.impls.MultitenantVirtualBufferCache;
 import org.apache.hyracks.storage.am.lsm.common.impls.NoMergePolicy;
 import org.apache.hyracks.storage.am.lsm.common.impls.NoOpIOOperationCallbackFactory;
-import org.apache.hyracks.storage.am.lsm.common.impls.SynchronousScheduler;
+import org.apache.hyracks.storage.am.lsm.common.impls.SynchronousSchedulerProvider;
 import org.apache.hyracks.storage.am.lsm.common.impls.ThreadCountingTracker;
 import org.apache.hyracks.storage.am.lsm.common.impls.VirtualBufferCache;
 import org.apache.hyracks.storage.common.buffercache.HeapBufferAllocator;
@@ -92,7 +92,7 @@
         this.memNumPages = AccessMethodTestsConfig.LSM_INVINDEX_MEM_NUM_PAGES;
         this.hyracksFrameSize = AccessMethodTestsConfig.LSM_INVINDEX_HYRACKS_FRAME_SIZE;
         this.bloomFilterFalsePositiveRate = AccessMethodTestsConfig.LSM_INVINDEX_BLOOMFILTER_FALSE_POSITIVE_RATE;
-        this.ioScheduler = SynchronousScheduler.INSTANCE;
+        this.ioScheduler = SynchronousSchedulerProvider.INSTANCE.getIoScheduler(null);
         this.mergePolicy = new NoMergePolicy();
         this.opTracker = new ThreadCountingTracker();
         this.ioOpCallbackFactory = NoOpIOOperationCallbackFactory.INSTANCE;
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/org/apache/hyracks/storage/am/lsm/invertedindex/multithread/LSMInvertedIndexTestWorker.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/org/apache/hyracks/storage/am/lsm/invertedindex/multithread/LSMInvertedIndexTestWorker.java
index 78b1658..6b06f4a 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/org/apache/hyracks/storage/am/lsm/invertedindex/multithread/LSMInvertedIndexTestWorker.java
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/org/apache/hyracks/storage/am/lsm/invertedindex/multithread/LSMInvertedIndexTestWorker.java
@@ -138,7 +138,7 @@
             }
 
             case MERGE: {
-                accessor.scheduleMerge(invIndex.getIOOperationCallback(), invIndex.getDiskComponents());
+                accessor.scheduleMerge(invIndex.getDiskComponents());
                 break;
             }
 
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/org/apache/hyracks/storage/am/lsm/rtree/LSMRTreeMergeTestDriver.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/org/apache/hyracks/storage/am/lsm/rtree/LSMRTreeMergeTestDriver.java
index 0e0f7e5..ee044bd 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/org/apache/hyracks/storage/am/lsm/rtree/LSMRTreeMergeTestDriver.java
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/org/apache/hyracks/storage/am/lsm/rtree/LSMRTreeMergeTestDriver.java
@@ -76,8 +76,7 @@
             }
 
             ILSMIndexAccessor accessor = (ILSMIndexAccessor) ctx.getIndexAccessor();
-            accessor.scheduleMerge(((AbstractLSMRTree) ctx.getIndex()).getIOOperationCallback(),
-                    ((AbstractLSMRTree) ctx.getIndex()).getDiskComponents());
+            accessor.scheduleMerge(((AbstractLSMRTree) ctx.getIndex()).getDiskComponents());
 
             rTreeTestUtils.checkScan(ctx);
             rTreeTestUtils.checkDiskOrderScan(ctx);
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 581dbf7..1fa9d95 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
@@ -25,7 +25,6 @@
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.storage.am.common.api.IPrimitiveValueProviderFactory;
 import org.apache.hyracks.storage.am.config.AccessMethodTestsConfig;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndex;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndexAccessor;
 import org.apache.hyracks.storage.am.lsm.rtree.util.LSMRTreeTestHarness;
 import org.apache.hyracks.storage.am.lsm.rtree.util.LSMRTreeWithAntiMatterTuplesTestContext;
@@ -75,9 +74,8 @@
     }
 
     protected void flush(AbstractRTreeTestContext ctx) throws HyracksDataException {
-        ILSMIndex lsmIndex = (ILSMIndex) ctx.getIndex();
         ILSMIndexAccessor accessor = (ILSMIndexAccessor) ctx.getIndexAccessor();
-        accessor.scheduleFlush(lsmIndex.getIOOperationCallback());
+        accessor.scheduleFlush();
     }
 
     @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/LSMRTreeTestWorker.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/org/apache/hyracks/storage/am/lsm/rtree/multithread/LSMRTreeTestWorker.java
index 2a14780..810c3c3 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/org/apache/hyracks/storage/am/lsm/rtree/multithread/LSMRTreeTestWorker.java
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/org/apache/hyracks/storage/am/lsm/rtree/multithread/LSMRTreeTestWorker.java
@@ -78,7 +78,7 @@
                 break;
 
             case MERGE:
-                accessor.scheduleMerge(lsmRTree.getIOOperationCallback(), lsmRTree.getDiskComponents());
+                accessor.scheduleMerge(lsmRTree.getDiskComponents());
                 break;
 
             default:
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/LSMRTreeWithAntiMatterTuplesTestWorker.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/org/apache/hyracks/storage/am/lsm/rtree/multithread/LSMRTreeWithAntiMatterTuplesTestWorker.java
index 653677c..595d675 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/org/apache/hyracks/storage/am/lsm/rtree/multithread/LSMRTreeWithAntiMatterTuplesTestWorker.java
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/org/apache/hyracks/storage/am/lsm/rtree/multithread/LSMRTreeWithAntiMatterTuplesTestWorker.java
@@ -66,8 +66,7 @@
                 break;
 
             case MERGE:
-                accessor.scheduleMerge(((AbstractLSMRTree) lsmRTree).getIOOperationCallback(),
-                        ((AbstractLSMRTree) lsmRTree).getDiskComponents());
+                accessor.scheduleMerge(((AbstractLSMRTree) lsmRTree).getDiskComponents());
                 break;
 
             default:
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 8a5d0c5..598cf18 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
@@ -42,7 +42,7 @@
 import org.apache.hyracks.storage.am.lsm.common.api.IVirtualBufferCache;
 import org.apache.hyracks.storage.am.lsm.common.impls.NoMergePolicy;
 import org.apache.hyracks.storage.am.lsm.common.impls.NoOpIOOperationCallbackFactory;
-import org.apache.hyracks.storage.am.lsm.common.impls.SynchronousScheduler;
+import org.apache.hyracks.storage.am.lsm.common.impls.SynchronousSchedulerProvider;
 import org.apache.hyracks.storage.am.lsm.common.impls.ThreadCountingTracker;
 import org.apache.hyracks.storage.am.lsm.common.impls.VirtualBufferCache;
 import org.apache.hyracks.storage.common.buffercache.HeapBufferAllocator;
@@ -89,7 +89,7 @@
         this.memNumPages = AccessMethodTestsConfig.LSM_RTREE_MEM_NUM_PAGES;
         this.bloomFilterFalsePositiveRate = AccessMethodTestsConfig.LSM_RTREE_BLOOMFILTER_FALSE_POSITIVE_RATE;
         this.hyracksFrameSize = AccessMethodTestsConfig.LSM_RTREE_HYRACKS_FRAME_SIZE;
-        this.ioScheduler = SynchronousScheduler.INSTANCE;
+        this.ioScheduler = SynchronousSchedulerProvider.INSTANCE.getIoScheduler(null);
         this.mergePolicy = new NoMergePolicy();
         this.opTracker = new ThreadCountingTracker();
         this.ioOpCallbackFactory = NoOpIOOperationCallbackFactory.INSTANCE;
diff --git a/hyracks-fullstack/hyracks/hyracks-util/src/main/java/org/apache/hyracks/util/ExitUtil.java b/hyracks-fullstack/hyracks/hyracks-util/src/main/java/org/apache/hyracks/util/ExitUtil.java
index 1a17012..4aa123b 100644
--- a/hyracks-fullstack/hyracks/hyracks-util/src/main/java/org/apache/hyracks/util/ExitUtil.java
+++ b/hyracks-fullstack/hyracks/hyracks-util/src/main/java/org/apache/hyracks/util/ExitUtil.java
@@ -37,12 +37,11 @@
     public static final int EC_UNHANDLED_EXCEPTION = 11;
     public static final int EC_IMMEDIATE_HALT = 33;
     public static final int EC_HALT_ABNORMAL_RESERVED_44 = 44;
-    public static final int EC_HALT_ABNORMAL_RESERVED_55 = 55;
+    public static final int EC_IO_SCHEDULER_FAILED = 55;
     public static final int EC_HALT_SHUTDOWN_TIMED_OUT = 66;
     public static final int EC_HALT_WATCHDOG_FAILED = 77;
-    public static final int EC_HALT_ABNORMAL_RESERVED_88 = 88;
+    public static final int EC_FLUSH_FAILED = 88;
     public static final int EC_TERMINATE_NC_SERVICE_DIRECTIVE = 99;
-
     private static final ExitThread exitThread = new ExitThread();
     private static final ShutdownWatchdog watchdogThread = new ShutdownWatchdog();
     private static final MutableLong shutdownHaltDelay = new MutableLong(10 * 60 * 1000L); // 10 minutes default