You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@asterixdb.apache.org by il...@apache.org on 2017/10/16 06:08:01 UTC

[1/6] asterixdb git commit: [NO ISSUE][STO][IDX] LSM storage cleanup

Repository: asterixdb
Updated Branches:
  refs/heads/master 536aa3f2b -> 77f895257


http://git-wip-us.apache.org/repos/asf/asterixdb/blob/77f89525/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/impls/LSMRTreeWithAntiMatterTuples.java
----------------------------------------------------------------------
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 1e15455..cfb05e7 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
@@ -20,18 +20,16 @@
 package org.apache.hyracks.storage.am.lsm.rtree.impls;
 
 import java.util.ArrayList;
-import java.util.HashSet;
 import java.util.List;
-import java.util.Set;
 
 import org.apache.hyracks.api.dataflow.value.IBinaryComparatorFactory;
 import org.apache.hyracks.api.dataflow.value.ILinearizeComparatorFactory;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.api.io.IIOManager;
 import org.apache.hyracks.dataflow.common.data.accessors.ITupleReference;
+import org.apache.hyracks.storage.am.btree.impls.BTree.BTreeAccessor;
 import org.apache.hyracks.storage.am.btree.impls.BTreeRangeSearchCursor;
 import org.apache.hyracks.storage.am.btree.impls.RangePredicate;
-import org.apache.hyracks.storage.am.common.api.ITreeIndexAccessor;
 import org.apache.hyracks.storage.am.common.api.ITreeIndexCursor;
 import org.apache.hyracks.storage.am.common.api.ITreeIndexFrameFactory;
 import org.apache.hyracks.storage.am.common.impls.NoOpOperationCallback;
@@ -50,7 +48,6 @@ import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndexOperationContext;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMMergePolicy;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMOperationTracker;
 import org.apache.hyracks.storage.am.lsm.common.api.IVirtualBufferCache;
-import org.apache.hyracks.storage.am.lsm.common.api.LSMOperationType;
 import org.apache.hyracks.storage.am.lsm.common.impls.AbstractLSMIndexOperationContext;
 import org.apache.hyracks.storage.am.lsm.common.impls.LSMComponentFileReferences;
 import org.apache.hyracks.storage.am.lsm.common.impls.LSMComponentFilterManager;
@@ -58,30 +55,25 @@ import org.apache.hyracks.storage.am.lsm.common.impls.LSMIndexSearchCursor;
 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.MergeOperation;
-import org.apache.hyracks.storage.am.lsm.common.impls.TreeIndexFactory;
 import org.apache.hyracks.storage.am.rtree.frames.RTreeFrameFactory;
 import org.apache.hyracks.storage.am.rtree.impls.RTree;
+import org.apache.hyracks.storage.am.rtree.impls.RTree.RTreeAccessor;
 import org.apache.hyracks.storage.am.rtree.impls.RTreeSearchCursor;
 import org.apache.hyracks.storage.am.rtree.impls.SearchPredicate;
-import org.apache.hyracks.storage.common.IIndexBulkLoader;
 import org.apache.hyracks.storage.common.IIndexCursor;
 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;
+import org.apache.hyracks.storage.common.buffercache.IBufferCache;
 
 public class LSMRTreeWithAntiMatterTuples extends AbstractLSMRTree {
     private static final ICursorFactory cursorFactory = opCtx -> new LSMRTreeWithAntiMatterTuplesSearchCursor(opCtx);
-    // On-disk components.
-    // For creating RTree's used in bulk load. Different from diskRTreeFactory
-    // because it should have a different tuple writer in it's leaf frames.
-    private final ILSMDiskComponentFactory bulkLoaComponentFactory;
 
     public LSMRTreeWithAntiMatterTuples(IIOManager ioManager, List<IVirtualBufferCache> virtualBufferCaches,
             RTreeFrameFactory rtreeInteriorFrameFactory, RTreeFrameFactory rtreeLeafFrameFactory,
             ITreeIndexFrameFactory btreeInteriorFrameFactory, ITreeIndexFrameFactory btreeLeafFrameFactory,
-            ILSMIndexFileManager fileManager, TreeIndexFactory<RTree> diskRTreeFactory,
-            TreeIndexFactory<RTree> bulkLoadRTreeFactory, IComponentFilterHelper filterHelper,
+            IBufferCache diskBufferCache, ILSMIndexFileManager fileManager, ILSMDiskComponentFactory componentFactory,
+            ILSMDiskComponentFactory bulkLoadComponentFactory, IComponentFilterHelper filterHelper,
             ILSMComponentFilterFrameFactory filterFrameFactory, LSMComponentFilterManager filterManager, int fieldCount,
             IBinaryComparatorFactory[] rtreeCmpFactories, IBinaryComparatorFactory[] btreeComparatorFactories,
             ILinearizeComparatorFactory linearizer, int[] comparatorFields, IBinaryComparatorFactory[] linearizerArray,
@@ -89,37 +81,10 @@ public class LSMRTreeWithAntiMatterTuples extends AbstractLSMRTree {
             ILSMIOOperationCallback ioOpCallback, int[] rtreeFields, int[] filterFields, boolean durable,
             boolean isPointMBR) throws HyracksDataException {
         super(ioManager, virtualBufferCaches, rtreeInteriorFrameFactory, rtreeLeafFrameFactory,
-                btreeInteriorFrameFactory, btreeLeafFrameFactory, fileManager,
-                new LSMRTreeWithAntiMatterTuplesDiskComponentFactory(diskRTreeFactory, filterHelper), fieldCount,
-                rtreeCmpFactories, btreeComparatorFactories, linearizer, comparatorFields, linearizerArray, 0,
-                mergePolicy, opTracker, ioScheduler, ioOpCallback, filterHelper, filterFrameFactory, filterManager,
-                rtreeFields, filterFields, durable, isPointMBR, diskRTreeFactory.getBufferCache());
-        bulkLoaComponentFactory =
-                new LSMRTreeWithAntiMatterTuplesDiskComponentFactory(bulkLoadRTreeFactory, filterHelper);
-    }
-
-    @Override
-    protected ILSMDiskComponent loadComponent(LSMComponentFileReferences refs) throws HyracksDataException {
-        return createDiskComponent(componentFactory, refs.getInsertIndexFileReference(), null, null, false);
-    }
-
-    @Override
-    protected void deactivateDiskComponent(ILSMDiskComponent c) throws HyracksDataException {
-        RTree rtree = ((LSMRTreeDiskComponent) c).getRTree();
-        rtree.deactivate();
-        rtree.purge();
-    }
-
-    @Override
-    protected void destroyDiskComponent(ILSMDiskComponent c) throws HyracksDataException {
-        ((LSMRTreeDiskComponent) c).getRTree().destroy();
-    }
-
-    @Override
-    protected void clearDiskComponent(ILSMDiskComponent c) throws HyracksDataException {
-        RTree rtree = ((LSMRTreeDiskComponent) c).getRTree();
-        rtree.deactivate();
-        rtree.destroy();
+                btreeInteriorFrameFactory, btreeLeafFrameFactory, diskBufferCache, fileManager, componentFactory,
+                bulkLoadComponentFactory, fieldCount, rtreeCmpFactories, btreeComparatorFactories, linearizer,
+                comparatorFields, linearizerArray, 0, mergePolicy, opTracker, ioScheduler, ioOpCallback, filterHelper,
+                filterFrameFactory, filterManager, rtreeFields, filterFields, durable, isPointMBR);
     }
 
     @Override
@@ -129,20 +94,20 @@ public class LSMRTreeWithAntiMatterTuples extends AbstractLSMRTree {
         // read the file names when we open the tree.
         // The RTree should be renamed before the BTree.
         LSMRTreeMemoryComponent flushingComponent = (LSMRTreeMemoryComponent) flushOp.getFlushingComponent();
-        ITreeIndexAccessor memRTreeAccessor = flushingComponent.getRTree()
-                .createAccessor(NoOpOperationCallback.INSTANCE, NoOpOperationCallback.INSTANCE);
-        RTreeSearchCursor rtreeScanCursor = (RTreeSearchCursor) memRTreeAccessor.createSearchCursor(false);
+        RTreeAccessor memRTreeAccessor = flushingComponent.getIndex().createAccessor(NoOpOperationCallback.INSTANCE,
+                NoOpOperationCallback.INSTANCE);
+        RTreeSearchCursor rtreeScanCursor = memRTreeAccessor.createSearchCursor(false);
         SearchPredicate rtreeNullPredicate = new SearchPredicate(null, null);
         memRTreeAccessor.search(rtreeScanCursor, rtreeNullPredicate);
-        LSMRTreeDiskComponent component = createDiskComponent(componentFactory, flushOp.getTarget(), null, null, true);
+        ILSMDiskComponent component = createDiskComponent(componentFactory, flushOp.getTarget(), null, null, true);
         ILSMDiskComponentBulkLoader componentBulkLoader =
-                createComponentBulkLoader(component, 1.0f, false, 0L, false, false, false);
+                component.createBulkLoader(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
-        TreeTupleSorter rTreeTupleSorter = new TreeTupleSorter(flushingComponent.getRTree().getFileId(),
+        TreeTupleSorter rTreeTupleSorter = new TreeTupleSorter(flushingComponent.getIndex().getFileId(),
                 linearizerArray, rtreeLeafFrameFactory.createFrame(), rtreeLeafFrameFactory.createFrame(),
-                flushingComponent.getRTree().getBufferCache(), comparatorFields);
+                flushingComponent.getIndex().getBufferCache(), comparatorFields);
 
         boolean isEmpty = true;
         try {
@@ -159,14 +124,14 @@ public class LSMRTreeWithAntiMatterTuples extends AbstractLSMRTree {
         }
 
         // scan the memory BTree
-        ITreeIndexAccessor memBTreeAccessor = flushingComponent.getBTree()
+        BTreeAccessor memBTreeAccessor = flushingComponent.getBuddyIndex()
                 .createAccessor(NoOpOperationCallback.INSTANCE, NoOpOperationCallback.INSTANCE);
-        BTreeRangeSearchCursor btreeScanCursor = (BTreeRangeSearchCursor) memBTreeAccessor.createSearchCursor(false);
+        BTreeRangeSearchCursor btreeScanCursor = memBTreeAccessor.createSearchCursor(false);
         RangePredicate btreeNullPredicate = new RangePredicate(null, null, true, true, null, null);
         memBTreeAccessor.search(btreeScanCursor, btreeNullPredicate);
-        TreeTupleSorter bTreeTupleSorter = new TreeTupleSorter(flushingComponent.getBTree().getFileId(),
+        TreeTupleSorter bTreeTupleSorter = new TreeTupleSorter(flushingComponent.getBuddyIndex().getFileId(),
                 linearizerArray, btreeLeafFrameFactory.createFrame(), btreeLeafFrameFactory.createFrame(),
-                flushingComponent.getBTree().getBufferCache(), comparatorFields);
+                flushingComponent.getBuddyIndex().getBufferCache(), comparatorFields);
 
         isEmpty = true;
         try {
@@ -202,7 +167,7 @@ public class LSMRTreeWithAntiMatterTuples extends AbstractLSMRTree {
             filterTuples.add(flushingComponent.getLSMComponentFilter().getMinTuple());
             filterTuples.add(flushingComponent.getLSMComponentFilter().getMaxTuple());
             getFilterManager().updateFilter(component.getLSMComponentFilter(), filterTuples);
-            getFilterManager().writeFilter(component.getLSMComponentFilter(), component.getRTree());
+            getFilterManager().writeFilter(component.getLSMComponentFilter(), component.getMetadataHolder());
         }
         flushingComponent.getMetadata().copy(component.getMetadata());
 
@@ -219,10 +184,10 @@ public class LSMRTreeWithAntiMatterTuples extends AbstractLSMRTree {
         search(opCtx, cursor, rtreeSearchPred);
 
         // Bulk load the tuples from all on-disk RTrees into the new RTree.
-        LSMRTreeDiskComponent component = createDiskComponent(componentFactory, mergeOp.getTarget(), null, null, true);
+        ILSMDiskComponent component = createDiskComponent(componentFactory, mergeOp.getTarget(), null, null, true);
 
         ILSMDiskComponentBulkLoader componentBulkLoader =
-                createComponentBulkLoader(component, 1.0f, false, 0L, false, false, false);
+                component.createBulkLoader(1.0f, false, 0L, false, false, false);
         try {
             while (cursor.hasNext()) {
                 cursor.next();
@@ -239,7 +204,7 @@ public class LSMRTreeWithAntiMatterTuples extends AbstractLSMRTree {
                 filterTuples.add(mergeOp.getMergingComponents().get(i).getLSMComponentFilter().getMaxTuple());
             }
             getFilterManager().updateFilter(component.getLSMComponentFilter(), filterTuples);
-            getFilterManager().writeFilter(component.getLSMComponentFilter(), component.getRTree());
+            getFilterManager().writeFilter(component.getLSMComponentFilter(), component.getMetadataHolder());
         }
 
         componentBulkLoader.end();
@@ -255,78 +220,6 @@ public class LSMRTreeWithAntiMatterTuples extends AbstractLSMRTree {
     }
 
     @Override
-    public ILSMDiskComponentBulkLoader createComponentBulkLoader(ILSMDiskComponent component, float fillFactor,
-            boolean verifyInput, long numElementsHint, boolean checkIfEmptyIndex, boolean withFilter,
-            boolean cleanupEmptyComponent) throws HyracksDataException {
-        if (withFilter && filterFields != null) {
-            return new LSMRTreeWithAntiMatterTuplesDiskComponentBulkLoader((LSMRTreeDiskComponent) component, null,
-                    fillFactor, verifyInput, numElementsHint, checkIfEmptyIndex, cleanupEmptyComponent, filterManager,
-                    treeFields, filterFields,
-                    MultiComparator.create(component.getLSMComponentFilter().getFilterCmpFactories()));
-        } else {
-            return new LSMRTreeWithAntiMatterTuplesDiskComponentBulkLoader((LSMRTreeDiskComponent) component, null,
-                    fillFactor, verifyInput, numElementsHint, checkIfEmptyIndex, cleanupEmptyComponent);
-        }
-    }
-
-    @Override
-    public IIndexBulkLoader createBulkLoader(float fillLevel, boolean verifyInput, long numElementsHint)
-            throws HyracksDataException {
-        return new LSMRTreeWithAntiMatterTuplesBulkLoader(fillLevel, verifyInput, numElementsHint);
-    }
-
-    public class LSMRTreeWithAntiMatterTuplesBulkLoader implements IIndexBulkLoader {
-        private final ILSMDiskComponent component;
-        private final IIndexBulkLoader componentBulkLoader;
-
-        public LSMRTreeWithAntiMatterTuplesBulkLoader(float fillFactor, boolean verifyInput, long numElementsHint)
-                throws HyracksDataException {
-            component = createBulkLoadTarget();
-
-            componentBulkLoader =
-                    createComponentBulkLoader(component, fillFactor, verifyInput, numElementsHint, false, true, true);
-        }
-
-        @Override
-        public void add(ITupleReference tuple) throws HyracksDataException {
-            componentBulkLoader.add(tuple);
-        }
-
-        @Override
-        public void end() throws HyracksDataException {
-
-            componentBulkLoader.end();
-            if (component.getComponentSize() > 0) {
-                ioOpCallback.afterOperation(LSMOperationType.FLUSH, null, component);
-                lsmHarness.addBulkLoadedComponent(component);
-            }
-        }
-
-        @Override
-        public void abort() throws HyracksDataException {
-            if (componentBulkLoader != null) {
-                componentBulkLoader.abort();
-            }
-        }
-
-    }
-
-    @Override
-    public ILSMDiskComponent createBulkLoadTarget() throws HyracksDataException {
-        LSMComponentFileReferences relFlushFileRefs = fileManager.getRelFlushFileReference();
-        return createDiskComponent(bulkLoaComponentFactory, relFlushFileRefs.getInsertIndexFileReference(), null, null,
-                true);
-    }
-
-    @Override
-    public Set<String> getLSMComponentPhysicalFiles(ILSMComponent lsmComponent) {
-        Set<String> files = new HashSet<>();
-        RTree rtree = ((LSMRTreeDiskComponent) lsmComponent).getRTree();
-        files.add(rtree.getFileReference().getFile().getAbsolutePath());
-        return files;
-    }
-
-    @Override
     protected ILSMIOOperation createFlushOperation(AbstractLSMIndexOperationContext opCtx,
             LSMComponentFileReferences componentFileRefs, ILSMIOOperationCallback callback)
             throws HyracksDataException {

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/77f89525/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/impls/LSMRTreeWithAntiMatterTuplesDiskComponentBulkLoader.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/impls/LSMRTreeWithAntiMatterTuplesDiskComponentBulkLoader.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/impls/LSMRTreeWithAntiMatterTuplesDiskComponentBulkLoader.java
deleted file mode 100644
index 22046aa..0000000
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/impls/LSMRTreeWithAntiMatterTuplesDiskComponentBulkLoader.java
+++ /dev/null
@@ -1,59 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.hyracks.storage.am.lsm.rtree.impls;
-
-import org.apache.hyracks.api.exceptions.HyracksDataException;
-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.ITreeIndex;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponentFilterManager;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMDiskComponent;
-import org.apache.hyracks.storage.am.lsm.common.impls.AbstractLSMDiskComponentBulkLoader;
-import org.apache.hyracks.storage.common.MultiComparator;
-
-public class LSMRTreeWithAntiMatterTuplesDiskComponentBulkLoader extends AbstractLSMDiskComponentBulkLoader {
-
-    //with filter
-    public LSMRTreeWithAntiMatterTuplesDiskComponentBulkLoader(LSMRTreeDiskComponent component,
-            BloomFilterSpecification bloomFilterSpec, float fillFactor, boolean verifyInput, long numElementsHint,
-            boolean checkIfEmptyIndex, boolean cleanupEmptyComponent, ILSMComponentFilterManager filterManager,
-            int[] indexFields, int[] filterFields, MultiComparator filterCmp) throws HyracksDataException {
-        super(component, bloomFilterSpec, fillFactor, verifyInput, numElementsHint, checkIfEmptyIndex,
-                cleanupEmptyComponent, filterManager, indexFields, filterFields, filterCmp);
-    }
-
-    //without filter
-    public LSMRTreeWithAntiMatterTuplesDiskComponentBulkLoader(LSMRTreeDiskComponent component,
-            BloomFilterSpecification bloomFilterSpec, float fillFactor, boolean verifyInput, long numElementsHint,
-            boolean checkIfEmptyIndex, boolean cleanupEmptyComponent) throws HyracksDataException {
-        super(component, bloomFilterSpec, fillFactor, verifyInput, numElementsHint, checkIfEmptyIndex,
-                cleanupEmptyComponent, null, null, null, null);
-    }
-
-    @Override
-    protected BloomFilter getBloomFilter(ILSMDiskComponent component) {
-        return ((LSMRTreeDiskComponent) component).getBloomFilter();
-    }
-
-    @Override
-    protected ITreeIndex getIndex(ILSMDiskComponent component) {
-        return ((LSMRTreeDiskComponent) component).getRTree();
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/77f89525/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/impls/LSMRTreeWithAntiMatterTuplesDiskComponentFactory.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/impls/LSMRTreeWithAntiMatterTuplesDiskComponentFactory.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/impls/LSMRTreeWithAntiMatterTuplesDiskComponentFactory.java
index a6216fc..6eae415 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/impls/LSMRTreeWithAntiMatterTuplesDiskComponentFactory.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/impls/LSMRTreeWithAntiMatterTuplesDiskComponentFactory.java
@@ -22,6 +22,7 @@ package org.apache.hyracks.storage.am.lsm.rtree.impls;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.storage.am.lsm.common.api.IComponentFilterHelper;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMDiskComponentFactory;
+import org.apache.hyracks.storage.am.lsm.common.impls.AbstractLSMIndex;
 import org.apache.hyracks.storage.am.lsm.common.impls.LSMComponentFileReferences;
 import org.apache.hyracks.storage.am.lsm.common.impls.TreeIndexFactory;
 import org.apache.hyracks.storage.am.rtree.impls.RTree;
@@ -37,8 +38,10 @@ public class LSMRTreeWithAntiMatterTuplesDiskComponentFactory implements ILSMDis
     }
 
     @Override
-    public LSMRTreeDiskComponent createComponent(LSMComponentFileReferences cfr) throws HyracksDataException {
-        return new LSMRTreeDiskComponent(rtreeFactory.createIndexInstance(cfr.getInsertIndexFileReference()), null,
-                null, filterHelper == null ? null : filterHelper.createFilter());
+    public LSMRTreeWithAntimatterDiskComponent createComponent(AbstractLSMIndex lsmIndex,
+            LSMComponentFileReferences cfr) throws HyracksDataException {
+        return new LSMRTreeWithAntimatterDiskComponent(lsmIndex,
+                rtreeFactory.createIndexInstance(cfr.getInsertIndexFileReference()),
+                filterHelper == null ? null : filterHelper.createFilter());
     }
 }

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/77f89525/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/impls/LSMRTreeWithAntiMatterTuplesSearchCursor.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/impls/LSMRTreeWithAntiMatterTuplesSearchCursor.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/impls/LSMRTreeWithAntiMatterTuplesSearchCursor.java
index 81917ee..4abab53 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/impls/LSMRTreeWithAntiMatterTuplesSearchCursor.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/impls/LSMRTreeWithAntiMatterTuplesSearchCursor.java
@@ -101,8 +101,8 @@ public class LSMRTreeWithAntiMatterTuplesSearchCursor extends LSMIndexSearchCurs
         btreeAccessors = new ITreeIndexAccessor[numMutableComponents];
         for (int i = 0; i < numMutableComponents; i++) {
             ILSMComponent component = operationalComponents.get(i);
-            RTree rtree = ((LSMRTreeMemoryComponent) component).getRTree();
-            BTree btree = ((LSMRTreeMemoryComponent) component).getBTree();
+            RTree rtree = ((LSMRTreeMemoryComponent) component).getIndex();
+            BTree btree = ((LSMRTreeMemoryComponent) component).getBuddyIndex();
             mutableRTreeCursors[i] = new RTreeSearchCursor(
                     (IRTreeInteriorFrame) lsmInitialState.getRTreeInteriorFrameFactory().createFrame(),
                     (IRTreeLeafFrame) lsmInitialState.getRTreeLeafFrameFactory().createFrame());
@@ -121,7 +121,7 @@ public class LSMRTreeWithAntiMatterTuplesSearchCursor extends LSMIndexSearchCurs
             rangeCursors[j] = new RTreeSearchCursor(
                     (IRTreeInteriorFrame) lsmInitialState.getRTreeInteriorFrameFactory().createFrame(),
                     (IRTreeLeafFrame) lsmInitialState.getRTreeLeafFrameFactory().createFrame());
-            RTree rtree = ((LSMRTreeDiskComponent) component).getRTree();
+            RTree rtree = ((LSMRTreeWithAntimatterDiskComponent) component).getIndex();
             immutableRTreeAccessors[j] =
                     rtree.createAccessor(NoOpOperationCallback.INSTANCE, NoOpOperationCallback.INSTANCE);
             immutableRTreeAccessors[j].search(rangeCursors[j], searchPred);

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/77f89525/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/impls/LSMRTreeWithAntimatterDiskComponent.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/impls/LSMRTreeWithAntimatterDiskComponent.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/impls/LSMRTreeWithAntimatterDiskComponent.java
new file mode 100644
index 0000000..eed01a8
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/impls/LSMRTreeWithAntimatterDiskComponent.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.rtree.impls;
+
+import java.util.Set;
+
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponentFilter;
+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.rtree.impls.RTree;
+
+public class LSMRTreeWithAntimatterDiskComponent extends AbstractLSMDiskComponent {
+    private final RTree rtree;
+
+    public LSMRTreeWithAntimatterDiskComponent(AbstractLSMIndex lsmIndex, RTree rtree, ILSMComponentFilter filter) {
+        super(lsmIndex, LSMRTreeDiskComponent.getMetadataPageManager(rtree), filter);
+        this.rtree = rtree;
+    }
+
+    @Override
+    public RTree getIndex() {
+        return rtree;
+    }
+
+    @Override
+    public RTree getMetadataHolder() {
+        return rtree;
+    }
+
+    @Override
+    public long getComponentSize() {
+        return LSMRTreeDiskComponent.getComponentSize(rtree);
+    }
+
+    @Override
+    public int getFileReferenceCount() {
+        return LSMRTreeDiskComponent.getFileReferenceCount(rtree);
+    }
+
+    @Override
+    public Set<String> getLSMComponentPhysicalFiles() {
+        return LSMRTreeDiskComponent.getFiles(rtree);
+    }
+
+    @Override
+    public String toString() {
+        return getClass().getSimpleName() + ":" + rtree.getFileReference().getRelativePath();
+    }
+
+    @Override
+    public void validate() throws HyracksDataException {
+        throw new UnsupportedOperationException("Validation not implemented for LSM R-Trees.");
+    }
+}

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/77f89525/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/utils/LSMRTreeUtils.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/utils/LSMRTreeUtils.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/utils/LSMRTreeUtils.java
index 5e3a198..d1a3195 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/utils/LSMRTreeUtils.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/utils/LSMRTreeUtils.java
@@ -39,6 +39,7 @@ import org.apache.hyracks.storage.am.common.api.IMetadataPageManagerFactory;
 import org.apache.hyracks.storage.am.common.api.IPrimitiveValueProviderFactory;
 import org.apache.hyracks.storage.am.common.api.ITreeIndexFrameFactory;
 import org.apache.hyracks.storage.am.common.tuples.TypeAwareTupleWriterFactory;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMDiskComponentFactory;
 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.ILSMIndexFileManager;
@@ -52,8 +53,10 @@ import org.apache.hyracks.storage.am.lsm.common.impls.LSMComponentFilterManager;
 import org.apache.hyracks.storage.am.lsm.common.impls.TreeIndexFactory;
 import org.apache.hyracks.storage.am.lsm.rtree.impls.ExternalRTree;
 import org.apache.hyracks.storage.am.lsm.rtree.impls.LSMRTree;
+import org.apache.hyracks.storage.am.lsm.rtree.impls.LSMRTreeDiskComponentFactory;
 import org.apache.hyracks.storage.am.lsm.rtree.impls.LSMRTreeFileManager;
 import org.apache.hyracks.storage.am.lsm.rtree.impls.LSMRTreeWithAntiMatterTuples;
+import org.apache.hyracks.storage.am.lsm.rtree.impls.LSMRTreeWithAntiMatterTuplesDiskComponentFactory;
 import org.apache.hyracks.storage.am.lsm.rtree.impls.LSMRTreeWithAntiMatterTuplesFileManager;
 import org.apache.hyracks.storage.am.lsm.rtree.impls.RTreeFactory;
 import org.apache.hyracks.storage.am.lsm.rtree.tuples.LSMRTreeCopyTupleWriterFactory;
@@ -129,12 +132,15 @@ public class LSMRTreeUtils {
         }
         ILSMIndexFileManager fileNameManager =
                 new LSMRTreeFileManager(ioManager, file, diskRTreeFactory, diskBTreeFactory);
+        ILSMDiskComponentFactory componentFactory =
+                new LSMRTreeDiskComponentFactory(diskRTreeFactory, diskBTreeFactory, bloomFilterFactory, filterHelper);
+
         return new LSMRTree(ioManager, virtualBufferCaches, rtreeInteriorFrameFactory, rtreeLeafFrameFactory,
-                btreeInteriorFrameFactory, btreeLeafFrameFactory, fileNameManager, diskRTreeFactory, diskBTreeFactory,
-                bloomFilterFactory, filterHelper, filterFrameFactory, filterManager, bloomFilterFalsePositiveRate,
-                typeTraits.length, rtreeCmpFactories, btreeCmpFactories, linearizeCmpFactory, comparatorFields,
-                linearizerArray, mergePolicy, opTracker, ioScheduler, ioOpCallback, rtreeFields, buddyBTreeFields,
-                filterFields, durable, isPointMBR);
+                btreeInteriorFrameFactory, btreeLeafFrameFactory, diskBufferCache, fileNameManager, componentFactory,
+                filterHelper, filterFrameFactory, filterManager, bloomFilterFalsePositiveRate, typeTraits.length,
+                rtreeCmpFactories, btreeCmpFactories, linearizeCmpFactory, comparatorFields, linearizerArray,
+                mergePolicy, opTracker, ioScheduler, ioOpCallback, rtreeFields, buddyBTreeFields, filterFields, durable,
+                isPointMBR);
     }
 
     public static LSMRTreeWithAntiMatterTuples createLSMTreeWithAntiMatterTuples(IIOManager ioManager,
@@ -216,12 +222,18 @@ public class LSMRTreeUtils {
         }
         ILSMIndexFileManager fileNameManager =
                 new LSMRTreeWithAntiMatterTuplesFileManager(ioManager, file, diskRTreeFactory);
+
+        ILSMDiskComponentFactory componentFactory =
+                new LSMRTreeWithAntiMatterTuplesDiskComponentFactory(diskRTreeFactory, filterHelper);
+        ILSMDiskComponentFactory bulkLoadComponentFactory =
+                new LSMRTreeWithAntiMatterTuplesDiskComponentFactory(bulkLoadRTreeFactory, filterHelper);
+
         return new LSMRTreeWithAntiMatterTuples(ioManager, virtualBufferCaches, rtreeInteriorFrameFactory,
-                rtreeLeafFrameFactory, btreeInteriorFrameFactory, btreeLeafFrameFactory, fileNameManager,
-                diskRTreeFactory, bulkLoadRTreeFactory, filterHelper, filterFrameFactory, filterManager,
-                typeTraits.length, rtreeCmpFactories, btreeComparatorFactories, linearizerCmpFactory, comparatorFields,
-                linearizerArray, mergePolicy, opTracker, ioScheduler, ioOpCallback, rtreeFields, filterFields, durable,
-                isPointMBR);
+                rtreeLeafFrameFactory, btreeInteriorFrameFactory, btreeLeafFrameFactory, diskBufferCache,
+                fileNameManager, componentFactory, bulkLoadComponentFactory, filterHelper, filterFrameFactory,
+                filterManager, typeTraits.length, rtreeCmpFactories, btreeComparatorFactories, linearizerCmpFactory,
+                comparatorFields, linearizerArray, mergePolicy, opTracker, ioScheduler, ioOpCallback, rtreeFields,
+                filterFields, durable, isPointMBR);
     }
 
     public static ExternalRTree createExternalRTree(IIOManager ioManager, FileReference file,
@@ -270,11 +282,14 @@ public class LSMRTreeUtils {
 
         ILSMIndexFileManager fileNameManager =
                 new LSMRTreeFileManager(ioManager, file, diskRTreeFactory, diskBTreeFactory);
+        ILSMDiskComponentFactory componentFactory =
+                new LSMRTreeDiskComponentFactory(diskRTreeFactory, diskBTreeFactory, bloomFilterFactory, null);
+
         return new ExternalRTree(ioManager, rtreeInteriorFrameFactory, rtreeLeafFrameFactory, btreeInteriorFrameFactory,
-                btreeLeafFrameFactory, fileNameManager, diskRTreeFactory, diskBTreeFactory, bloomFilterFactory,
-                bloomFilterFalsePositiveRate, typeTraits.length, rtreeCmpFactories, btreeCmpFactories,
-                linearizeCmpFactory, comparatorFields, linearizerArray, mergePolicy, opTracker, ioScheduler,
-                ioOpCallback, buddyBTreeFields, durable, isPointMBR);
+                btreeLeafFrameFactory, diskBufferCache, fileNameManager, componentFactory, bloomFilterFalsePositiveRate,
+                typeTraits.length, rtreeCmpFactories, btreeCmpFactories, linearizeCmpFactory, comparatorFields,
+                linearizerArray, mergePolicy, opTracker, ioScheduler, ioOpCallback, buddyBTreeFields, durable,
+                isPointMBR);
     }
 
     public static ILinearizeComparatorFactory proposeBestLinearizer(ITypeTraits[] typeTraits, int numKeyFields)

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/77f89525/hyracks-fullstack/hyracks/hyracks-storage-am-rtree/src/main/java/org/apache/hyracks/storage/am/rtree/impls/RTree.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-rtree/src/main/java/org/apache/hyracks/storage/am/rtree/impls/RTree.java b/hyracks-fullstack/hyracks/hyracks-storage-am-rtree/src/main/java/org/apache/hyracks/storage/am/rtree/impls/RTree.java
index 9a39630..ee7c827 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-rtree/src/main/java/org/apache/hyracks/storage/am/rtree/impls/RTree.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-rtree/src/main/java/org/apache/hyracks/storage/am/rtree/impls/RTree.java
@@ -756,12 +756,12 @@ public class RTree extends AbstractTreeIndex {
     }
 
     @Override
-    public ITreeIndexAccessor createAccessor(IModificationOperationCallback modificationCallback,
+    public RTreeAccessor createAccessor(IModificationOperationCallback modificationCallback,
             ISearchOperationCallback searchCallback) {
         return new RTreeAccessor(this, modificationCallback, searchCallback);
     }
 
-    public ITreeIndexAccessor createAccessor(IModificationOperationCallback modificationCallback,
+    public RTreeAccessor createAccessor(IModificationOperationCallback modificationCallback,
                                              ISearchOperationCallback searchCallback, int[] nonIndexFields) {
         return new RTreeAccessor(this, modificationCallback, searchCallback, nonIndexFields);
     }
@@ -807,7 +807,7 @@ public class RTree extends AbstractTreeIndex {
         }
 
         @Override
-        public ITreeIndexCursor createSearchCursor(boolean exclusive) {
+        public RTreeSearchCursor createSearchCursor(boolean exclusive) {
             return new RTreeSearchCursor((IRTreeInteriorFrame) interiorFrameFactory.createFrame(),
                     (IRTreeLeafFrame) leafFrameFactory.createFrame());
         }

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/77f89525/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/IIndex.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/IIndex.java b/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/IIndex.java
index 255e0d0..473cfcb 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/IIndex.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/IIndex.java
@@ -86,6 +86,15 @@ public interface IIndex {
     void destroy() throws HyracksDataException;
 
     /**
+     * Purge the index files out of the buffer cache.
+     * Can only be called if the caller is absolutely sure the files don't contain dirty pages
+     *
+     * @throws HyracksDataException
+     *             if the index is active
+     */
+    void purge() throws HyracksDataException;
+
+    /**
      * Creates an {@link IIndexAccessor} for performing operations on this index.
      * An IIndexAccessor is not thread safe, but different IIndexAccessors can concurrently operate
      * on the same {@link IIndex}.
@@ -101,6 +110,8 @@ public interface IIndex {
             ISearchOperationCallback searchCallback) throws HyracksDataException;
 
     /**
+     * TODO: Get rid of this method
+     *
      * Strictly a test method
      *
      * Ensures that all pages (and tuples) of the index are logically consistent.

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/77f89525/hyracks-fullstack/hyracks/hyracks-test-support/src/main/java/org/apache/hyracks/storage/am/common/IndexTestContext.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/hyracks/hyracks-test-support/src/main/java/org/apache/hyracks/storage/am/common/IndexTestContext.java b/hyracks-fullstack/hyracks/hyracks-test-support/src/main/java/org/apache/hyracks/storage/am/common/IndexTestContext.java
index 4213a34..3e1c922 100644
--- a/hyracks-fullstack/hyracks/hyracks-test-support/src/main/java/org/apache/hyracks/storage/am/common/IndexTestContext.java
+++ b/hyracks-fullstack/hyracks/hyracks-test-support/src/main/java/org/apache/hyracks/storage/am/common/IndexTestContext.java
@@ -40,8 +40,7 @@ public abstract class IndexTestContext<T extends CheckTuple> implements IIndexTe
             throws HyracksDataException {
         this.fieldSerdes = fieldSerdes;
         this.index = index;
-        this.indexAccessor = (IIndexAccessor) index.createAccessor(TestOperationCallback.INSTANCE,
-                TestOperationCallback.INSTANCE);
+        this.indexAccessor = index.createAccessor(TestOperationCallback.INSTANCE, TestOperationCallback.INSTANCE);
         this.tupleBuilder = filtered ? new ArrayTupleBuilder(fieldSerdes.length + 1)
                 : new ArrayTupleBuilder(fieldSerdes.length);
     }

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/77f89525/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/LSMBTreeFileManagerTest.java
----------------------------------------------------------------------
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 5d6d8de..7718ed9 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
@@ -26,7 +26,7 @@ import org.apache.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
 import org.apache.hyracks.dataflow.common.comm.io.ArrayTupleReference;
 import org.apache.hyracks.dataflow.common.data.marshalling.IntegerSerializerDeserializer;
 import org.apache.hyracks.storage.am.lsm.btree.impls.LSMBTree;
-import org.apache.hyracks.storage.am.lsm.btree.impls.LSMBTreeDiskComponent;
+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.ILSMIndexAccessor;
@@ -81,8 +81,9 @@ public class LSMBTreeFileManagerTest {
         ctx.getIndex().deactivate();
 
         // Delete the btree file and keep the bloom filter file from the disk component
-        LSMBTreeDiskComponent ilsmDiskComponent = (LSMBTreeDiskComponent) btree.getDiskComponents().get(0);
-        ilsmDiskComponent.getBTree().getFileReference().delete();
+        LSMBTreeWithBloomFilterDiskComponent ilsmDiskComponent =
+                (LSMBTreeWithBloomFilterDiskComponent) btree.getDiskComponents().get(0);
+        ilsmDiskComponent.getIndex().getFileReference().delete();
 
         File bloomFilterFile = ilsmDiskComponent.getBloomFilter().getFileReference().getFile().getAbsoluteFile();
         Assert.assertEquals("Check bloom filter file exists", true, bloomFilterFile.exists());

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/77f89525/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/LSMBTreeUpdateInPlaceScanDiskComponentsTest.java
----------------------------------------------------------------------
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 fbcbcc2..a5c9d21 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
@@ -48,10 +48,10 @@ import org.apache.hyracks.storage.am.common.TestOperationCallback;
 import org.apache.hyracks.storage.am.common.api.ITreeIndexCursor;
 import org.apache.hyracks.storage.am.config.AccessMethodTestsConfig;
 import org.apache.hyracks.storage.am.lsm.btree.impls.LSMBTree;
-import org.apache.hyracks.storage.am.lsm.btree.impls.LSMBTreeDiskComponent;
 import org.apache.hyracks.storage.am.lsm.btree.tuples.LSMBTreeTupleReference;
 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.ILSMIndexAccessor;
 import org.apache.hyracks.storage.am.lsm.common.impls.NoOpIOOperationCallbackFactory;
 import org.apache.hyracks.storage.common.IIndexCursor;
@@ -172,8 +172,9 @@ public class LSMBTreeUpdateInPlaceScanDiskComponentsTest extends OrderedIndexTes
             (IIndexTestContext ctx, ITupleReference tuple, UpdatedCheckTuple checkTuple) -> {
                 ctx.getIndexAccessor().delete(tuple);
                 // Remove check tuple from expected results.
-                if (!checkTuples.contains(checkTuple))
+                if (!checkTuples.contains(checkTuple)) {
                     fail("Trying to delete tuple " + checkTuple + " that does not exist");
+                }
                 checkTuple.setUpdated(!checkTuple.isUpdated());
                 checkTuple.setAntimatter(true);
             };
@@ -272,8 +273,9 @@ public class LSMBTreeUpdateInPlaceScanDiskComponentsTest extends OrderedIndexTes
             // Copy CheckTuple references into array, so we can randomly pick from there.
             UpdatedCheckTuple[] checkTuplesArray = new UpdatedCheckTuple[numCheckTuples];
             int idx = 0;
-            for (UpdatedCheckTuple t : checkTuples)
+            for (UpdatedCheckTuple t : checkTuples) {
                 checkTuplesArray[idx++] = t;
+            }
 
             for (int i = 0; i < numTuples && numCheckTuples > 0; i++) {
                 if (LOGGER.isLoggable(Level.INFO)) {
@@ -357,8 +359,8 @@ public class LSMBTreeUpdateInPlaceScanDiskComponentsTest extends OrderedIndexTes
         LSMBTree btree = (LSMBTree) ctx.getIndex();
         Assert.assertEquals("Check disk components", 1, btree.getDiskComponents().size());
 
-        LSMBTreeDiskComponent btreeComponent = (LSMBTreeDiskComponent) btree.getDiskComponents().get(0);
-        BTree.BTreeAccessor btreeAccessor = (BTree.BTreeAccessor) btreeComponent.getBTree()
+        ILSMDiskComponent btreeComponent = btree.getDiskComponents().get(0);
+        BTree.BTreeAccessor btreeAccessor = ((BTree) btreeComponent.getIndex())
                 .createAccessor(TestOperationCallback.INSTANCE, TestOperationCallback.INSTANCE);
 
         ITreeIndexCursor cursor = btreeAccessor.createDiskOrderScanCursor();

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/77f89525/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/impl/TestLsmBtree.java
----------------------------------------------------------------------
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 6c1a406..2e0f196 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
@@ -26,14 +26,13 @@ import org.apache.hyracks.api.dataflow.value.IBinaryComparatorFactory;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.api.io.IIOManager;
 import org.apache.hyracks.dataflow.common.data.accessors.ITupleReference;
-import org.apache.hyracks.storage.am.bloomfilter.impls.BloomFilterFactory;
-import org.apache.hyracks.storage.am.btree.impls.BTree;
 import org.apache.hyracks.storage.am.common.api.IIndexOperationContext;
 import org.apache.hyracks.storage.am.common.api.ITreeIndexFrameFactory;
 import org.apache.hyracks.storage.am.lsm.btree.impls.LSMBTree;
 import org.apache.hyracks.storage.am.lsm.common.api.IComponentFilterHelper;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponentFilterFrameFactory;
 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.ILSMIOOperationScheduler;
@@ -46,7 +45,7 @@ import org.apache.hyracks.storage.am.lsm.common.api.IVirtualBufferCache;
 import org.apache.hyracks.storage.am.lsm.common.impls.AbstractLSMIndexOperationContext;
 import org.apache.hyracks.storage.am.lsm.common.impls.LSMComponentFilterManager;
 import org.apache.hyracks.storage.am.lsm.common.impls.LSMTreeIndexAccessor;
-import org.apache.hyracks.storage.am.lsm.common.impls.TreeIndexFactory;
+import org.apache.hyracks.storage.common.buffercache.IBufferCache;
 import org.apache.hyracks.util.trace.ITracer;
 
 public class TestLsmBtree extends LSMBTree {
@@ -70,16 +69,16 @@ public class TestLsmBtree extends LSMBTree {
 
     public TestLsmBtree(IIOManager ioManager, List<IVirtualBufferCache> virtualBufferCaches,
             ITreeIndexFrameFactory interiorFrameFactory, ITreeIndexFrameFactory insertLeafFrameFactory,
-            ITreeIndexFrameFactory deleteLeafFrameFactory, ILSMIndexFileManager fileManager,
-            TreeIndexFactory<BTree> diskBTreeFactory, TreeIndexFactory<BTree> bulkLoadBTreeFactory,
-            BloomFilterFactory bloomFilterFactory, IComponentFilterHelper filterHelper,
+            ITreeIndexFrameFactory deleteLeafFrameFactory, IBufferCache diskBufferCache,
+            ILSMIndexFileManager fileManager, ILSMDiskComponentFactory componentFactory,
+            ILSMDiskComponentFactory bulkLoadComponentFactory, IComponentFilterHelper filterHelper,
             ILSMComponentFilterFrameFactory filterFrameFactory, LSMComponentFilterManager filterManager,
             double bloomFilterFalsePositiveRate, int fieldCount, IBinaryComparatorFactory[] cmpFactories,
             ILSMMergePolicy mergePolicy, ILSMOperationTracker opTracker, ILSMIOOperationScheduler ioScheduler,
             ILSMIOOperationCallback ioOpCallback, boolean needKeyDupCheck, int[] btreeFields, int[] filterFields,
             boolean durable, boolean updateAware, ITracer tracer) throws HyracksDataException {
         super(ioManager, virtualBufferCaches, interiorFrameFactory, insertLeafFrameFactory, deleteLeafFrameFactory,
-                fileManager, diskBTreeFactory, bulkLoadBTreeFactory, bloomFilterFactory, filterHelper,
+                diskBufferCache, fileManager, componentFactory, bulkLoadComponentFactory, filterHelper,
                 filterFrameFactory, filterManager, bloomFilterFalsePositiveRate, fieldCount, cmpFactories, mergePolicy,
                 opTracker, ioScheduler, ioOpCallback, needKeyDupCheck, btreeFields, filterFields, durable, updateAware,
                 tracer);

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/77f89525/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/impl/TestLsmBtreeUtil.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/impl/TestLsmBtreeUtil.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/impl/TestLsmBtreeUtil.java
index 049cb59..75539ad 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/impl/TestLsmBtreeUtil.java
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/impl/TestLsmBtreeUtil.java
@@ -33,9 +33,12 @@ import org.apache.hyracks.storage.am.common.api.IMetadataPageManagerFactory;
 import org.apache.hyracks.storage.am.common.api.ITreeIndexFrameFactory;
 import org.apache.hyracks.storage.am.common.tuples.TypeAwareTupleWriterFactory;
 import org.apache.hyracks.storage.am.lsm.btree.impls.LSMBTree;
+import org.apache.hyracks.storage.am.lsm.btree.impls.LSMBTreeDiskComponentFactory;
 import org.apache.hyracks.storage.am.lsm.btree.impls.LSMBTreeFileManager;
+import org.apache.hyracks.storage.am.lsm.btree.impls.LSMBTreeWithBloomFilterDiskComponentFactory;
 import org.apache.hyracks.storage.am.lsm.btree.tuples.LSMBTreeCopyTupleWriterFactory;
 import org.apache.hyracks.storage.am.lsm.btree.tuples.LSMBTreeTupleWriterFactory;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMDiskComponentFactory;
 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.ILSMIndexFileManager;
@@ -79,9 +82,6 @@ public class TestLsmBtreeUtil {
         TreeIndexFactory<BTree> bulkLoadBTreeFactory = new BTreeFactory(ioManager, diskBufferCache,
                 freePageManagerFactory, interiorFrameFactory, insertLeafFrameFactory, cmpFactories, typeTraits.length);
 
-        BloomFilterFactory bloomFilterFactory =
-                needKeyDupCheck ? new BloomFilterFactory(diskBufferCache, bloomFilterKeyFields) : null;
-
         ComponentFilterHelper filterHelper = null;
         LSMComponentFilterFrameFactory filterFrameFactory = null;
         LSMComponentFilterManager filterManager = null;
@@ -96,8 +96,21 @@ public class TestLsmBtreeUtil {
         ILSMIndexFileManager fileNameManager =
                 new LSMBTreeFileManager(ioManager, file, diskBTreeFactory, needKeyDupCheck);
 
+        ILSMDiskComponentFactory componentFactory;
+        ILSMDiskComponentFactory bulkLoadComponentFactory;
+        if (needKeyDupCheck) {
+            BloomFilterFactory bloomFilterFactory = new BloomFilterFactory(diskBufferCache, bloomFilterKeyFields);
+            componentFactory =
+                    new LSMBTreeWithBloomFilterDiskComponentFactory(diskBTreeFactory, bloomFilterFactory, filterHelper);
+            bulkLoadComponentFactory = new LSMBTreeWithBloomFilterDiskComponentFactory(bulkLoadBTreeFactory,
+                    bloomFilterFactory, filterHelper);
+        } else {
+            componentFactory = new LSMBTreeDiskComponentFactory(diskBTreeFactory, filterHelper);
+            bulkLoadComponentFactory = new LSMBTreeDiskComponentFactory(bulkLoadBTreeFactory, filterHelper);
+        }
+
         return new TestLsmBtree(ioManager, virtualBufferCaches, interiorFrameFactory, insertLeafFrameFactory,
-                deleteLeafFrameFactory, fileNameManager, diskBTreeFactory, bulkLoadBTreeFactory, bloomFilterFactory,
+                deleteLeafFrameFactory, diskBufferCache, fileNameManager, componentFactory, bulkLoadComponentFactory,
                 filterHelper, filterFrameFactory, filterManager, bloomFilterFalsePositiveRate, typeTraits.length,
                 cmpFactories, mergePolicy, opTracker, ioScheduler, ioOpCallback, needKeyDupCheck, btreeFields,
                 filterFields, durable, updateAware, tracer);


[3/6] asterixdb git commit: [NO ISSUE][STO][IDX] LSM storage cleanup

Posted by il...@apache.org.
http://git-wip-us.apache.org/repos/asf/asterixdb/blob/77f89525/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndex.java
----------------------------------------------------------------------
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 eb3924c..5a773cc 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
@@ -19,9 +19,7 @@
 package org.apache.hyracks.storage.am.lsm.invertedindex.impls;
 
 import java.util.ArrayList;
-import java.util.HashSet;
 import java.util.List;
-import java.util.Set;
 import java.util.logging.Level;
 import java.util.logging.Logger;
 
@@ -29,13 +27,9 @@ import org.apache.hyracks.api.dataflow.value.IBinaryComparatorFactory;
 import org.apache.hyracks.api.dataflow.value.ITypeTraits;
 import org.apache.hyracks.api.exceptions.ErrorCode;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
-import org.apache.hyracks.api.io.FileReference;
 import org.apache.hyracks.api.io.IIOManager;
 import org.apache.hyracks.data.std.primitive.IntegerPointable;
 import org.apache.hyracks.dataflow.common.data.accessors.ITupleReference;
-import org.apache.hyracks.storage.am.bloomfilter.impls.BloomCalculations;
-import org.apache.hyracks.storage.am.bloomfilter.impls.BloomFilterFactory;
-import org.apache.hyracks.storage.am.bloomfilter.impls.BloomFilterSpecification;
 import org.apache.hyracks.storage.am.btree.frames.BTreeLeafFrameType;
 import org.apache.hyracks.storage.am.btree.impls.BTree;
 import org.apache.hyracks.storage.am.btree.impls.BTree.BTreeAccessor;
@@ -61,11 +55,9 @@ import org.apache.hyracks.storage.am.lsm.common.api.ILSMMemoryComponent;
 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.api.LSMOperationType;
 import org.apache.hyracks.storage.am.lsm.common.freepage.VirtualFreePageManager;
 import org.apache.hyracks.storage.am.lsm.common.impls.AbstractLSMIndex;
 import org.apache.hyracks.storage.am.lsm.common.impls.AbstractLSMIndexOperationContext;
-import org.apache.hyracks.storage.am.lsm.common.impls.BTreeFactory;
 import org.apache.hyracks.storage.am.lsm.common.impls.LSMComponentFileReferences;
 import org.apache.hyracks.storage.am.lsm.common.impls.LSMComponentFilterManager;
 import org.apache.hyracks.storage.am.lsm.common.impls.LSMIndexSearchCursor;
@@ -73,13 +65,11 @@ import org.apache.hyracks.storage.am.lsm.invertedindex.api.IInvertedIndex;
 import org.apache.hyracks.storage.am.lsm.invertedindex.inmemory.InMemoryInvertedIndex;
 import org.apache.hyracks.storage.am.lsm.invertedindex.inmemory.InMemoryInvertedIndexAccessor;
 import org.apache.hyracks.storage.am.lsm.invertedindex.ondisk.OnDiskInvertedIndex;
-import org.apache.hyracks.storage.am.lsm.invertedindex.ondisk.OnDiskInvertedIndexFactory;
 import org.apache.hyracks.storage.am.lsm.invertedindex.search.InvertedIndexSearchPredicate;
 import org.apache.hyracks.storage.am.lsm.invertedindex.tokenizers.IBinaryTokenizerFactory;
 import org.apache.hyracks.storage.am.lsm.invertedindex.util.InvertedIndexUtils;
 import org.apache.hyracks.storage.common.ICursorInitialState;
 import org.apache.hyracks.storage.common.IIndexAccessor;
-import org.apache.hyracks.storage.common.IIndexBulkLoader;
 import org.apache.hyracks.storage.common.IIndexCursor;
 import org.apache.hyracks.storage.common.IModificationOperationCallback;
 import org.apache.hyracks.storage.common.ISearchOperationCallback;
@@ -93,10 +83,6 @@ public class LSMInvertedIndex extends AbstractLSMIndex implements IInvertedIndex
 
     protected final IBinaryTokenizerFactory tokenizerFactory;
 
-    // On-disk components.
-    // For creating inverted indexes in flush and merge.
-    protected final ILSMDiskComponentFactory componentFactory;
-
     // Type traits and comparators for tokens and inverted-list elements.
     protected final ITypeTraits[] invListTypeTraits;
     protected final IBinaryComparatorFactory[] invListCmpFactories;
@@ -106,18 +92,17 @@ public class LSMInvertedIndex extends AbstractLSMIndex implements IInvertedIndex
     private final int[] invertedIndexFieldsForNonBulkLoadOps;
 
     public LSMInvertedIndex(IIOManager ioManager, List<IVirtualBufferCache> virtualBufferCaches,
-            OnDiskInvertedIndexFactory diskInvIndexFactory, BTreeFactory deletedKeysBTreeFactory,
-            BloomFilterFactory bloomFilterFactory, IComponentFilterHelper filterHelper,
+            ILSMDiskComponentFactory componentFactory, IComponentFilterHelper filterHelper,
             ILSMComponentFilterFrameFactory filterFrameFactory, LSMComponentFilterManager filterManager,
-            double bloomFilterFalsePositiveRate, ILSMIndexFileManager fileManager, ITypeTraits[] invListTypeTraits,
-            IBinaryComparatorFactory[] invListCmpFactories, ITypeTraits[] tokenTypeTraits,
-            IBinaryComparatorFactory[] tokenCmpFactories, IBinaryTokenizerFactory tokenizerFactory,
-            ILSMMergePolicy mergePolicy, ILSMOperationTracker opTracker, ILSMIOOperationScheduler ioScheduler,
-            ILSMIOOperationCallback ioOpCallback, int[] invertedIndexFields, int[] filterFields,
-            int[] filterFieldsForNonBulkLoadOps, int[] invertedIndexFieldsForNonBulkLoadOps, boolean durable)
-            throws HyracksDataException {
-        super(ioManager, virtualBufferCaches, diskInvIndexFactory.getBufferCache(), fileManager,
-                bloomFilterFalsePositiveRate, mergePolicy, opTracker, ioScheduler, ioOpCallback, filterFrameFactory,
+            double bloomFilterFalsePositiveRate, IBufferCache diskBufferCache, ILSMIndexFileManager fileManager,
+            ITypeTraits[] invListTypeTraits, IBinaryComparatorFactory[] invListCmpFactories,
+            ITypeTraits[] tokenTypeTraits, IBinaryComparatorFactory[] tokenCmpFactories,
+            IBinaryTokenizerFactory tokenizerFactory, ILSMMergePolicy mergePolicy, ILSMOperationTracker opTracker,
+            ILSMIOOperationScheduler ioScheduler, ILSMIOOperationCallback ioOpCallback, int[] invertedIndexFields,
+            int[] filterFields, int[] filterFieldsForNonBulkLoadOps, int[] invertedIndexFieldsForNonBulkLoadOps,
+            boolean durable) throws HyracksDataException {
+        super(ioManager, virtualBufferCaches, diskBufferCache, fileManager, bloomFilterFalsePositiveRate, mergePolicy,
+                opTracker, ioScheduler, ioOpCallback, componentFactory, componentFactory, filterFrameFactory,
                 filterManager, filterFields, durable, filterHelper, invertedIndexFields, ITracer.NONE);
         this.tokenizerFactory = tokenizerFactory;
         this.invListTypeTraits = invListTypeTraits;
@@ -126,8 +111,6 @@ public class LSMInvertedIndex extends AbstractLSMIndex implements IInvertedIndex
         this.tokenCmpFactories = tokenCmpFactories;
         this.filterFieldsForNonBulkLoadOps = filterFieldsForNonBulkLoadOps;
         this.invertedIndexFieldsForNonBulkLoadOps = invertedIndexFieldsForNonBulkLoadOps;
-        componentFactory = new LSMInvertedIndexDiskComponentFactory(diskInvIndexFactory, deletedKeysBTreeFactory,
-                bloomFilterFactory, filterHelper);
 
         int i = 0;
         for (IVirtualBufferCache virtualBufferCache : virtualBufferCaches) {
@@ -145,52 +128,6 @@ public class LSMInvertedIndex extends AbstractLSMIndex implements IInvertedIndex
         }
     }
 
-    @Override
-    protected ILSMDiskComponent loadComponent(LSMComponentFileReferences refs) throws HyracksDataException {
-        return createDiskInvIndexComponent(componentFactory, refs.getInsertIndexFileReference(),
-                refs.getDeleteIndexFileReference(), refs.getBloomFilterFileReference(), false);
-    }
-
-    @Override
-    protected void clearDiskComponent(ILSMDiskComponent c) throws HyracksDataException {
-        LSMInvertedIndexDiskComponent component = (LSMInvertedIndexDiskComponent) c;
-        component.getBloomFilter().deactivate();
-        component.getInvIndex().deactivate();
-        component.getDeletedKeysBTree().deactivate();
-        component.getBloomFilter().destroy();
-        component.getInvIndex().destroy();
-        component.getDeletedKeysBTree().destroy();
-    }
-
-    @Override
-    protected void deactivateMemoryComponent(ILSMMemoryComponent c) throws HyracksDataException {
-        LSMInvertedIndexMemoryComponent mutableComponent = (LSMInvertedIndexMemoryComponent) c;
-        mutableComponent.getInvIndex().deactivate();
-        mutableComponent.getDeletedKeysBTree().deactivate();
-        mutableComponent.getInvIndex().destroy();
-        mutableComponent.getDeletedKeysBTree().destroy();
-        ((IVirtualBufferCache) mutableComponent.getInvIndex().getBufferCache()).close();
-    }
-
-    @Override
-    protected void deactivateDiskComponent(ILSMDiskComponent c) throws HyracksDataException {
-        LSMInvertedIndexDiskComponent component = (LSMInvertedIndexDiskComponent) c;
-        component.getBloomFilter().deactivate();
-        component.getBloomFilter().purge();
-        component.getInvIndex().deactivate();
-        component.getInvIndex().purge();
-        component.getDeletedKeysBTree().deactivate();
-        component.getDeletedKeysBTree().purge();
-    }
-
-    @Override
-    protected void destroyDiskComponent(ILSMDiskComponent c) throws HyracksDataException {
-        LSMInvertedIndexDiskComponent component = (LSMInvertedIndexDiskComponent) c;
-        component.getInvIndex().destroy();
-        component.getDeletedKeysBTree().destroy();
-        component.getBloomFilter().destroy();
-    }
-
     /**
      * The keys in the in-memory deleted-keys BTree only refer to on-disk components.
      * We delete documents from the in-memory inverted index by deleting its entries directly,
@@ -265,17 +202,17 @@ public class LSMInvertedIndex extends AbstractLSMIndex implements IInvertedIndex
             ILSMComponent component = operationalComponents.get(i);
             if (component.getType() == LSMComponentType.MEMORY) {
                 includeMutableComponent = true;
-                IIndexAccessor invIndexAccessor = ((LSMInvertedIndexMemoryComponent) component).getInvIndex()
+                IIndexAccessor invIndexAccessor = component.getIndex()
                         .createAccessor(NoOpOperationCallback.INSTANCE, NoOpOperationCallback.INSTANCE);
                 indexAccessors.add(invIndexAccessor);
-                IIndexAccessor deletedKeysAccessor = ((LSMInvertedIndexMemoryComponent) component).getDeletedKeysBTree()
+                IIndexAccessor deletedKeysAccessor = ((LSMInvertedIndexMemoryComponent) component).getBuddyIndex()
                         .createAccessor(NoOpOperationCallback.INSTANCE, NoOpOperationCallback.INSTANCE);
                 deletedKeysBTreeAccessors.add(deletedKeysAccessor);
             } else {
-                IIndexAccessor invIndexAccessor = ((LSMInvertedIndexDiskComponent) component).getInvIndex()
+                IIndexAccessor invIndexAccessor = component.getIndex()
                         .createAccessor(NoOpOperationCallback.INSTANCE, NoOpOperationCallback.INSTANCE);
                 indexAccessors.add(invIndexAccessor);
-                IIndexAccessor deletedKeysAccessor = ((LSMInvertedIndexDiskComponent) component).getDeletedKeysBTree()
+                IIndexAccessor deletedKeysAccessor = ((LSMInvertedIndexDiskComponent) component).getBuddyIndex()
                         .createAccessor(NoOpOperationCallback.INSTANCE, NoOpOperationCallback.INSTANCE);
                 deletedKeysBTreeAccessors.add(deletedKeysAccessor);
             }
@@ -299,16 +236,16 @@ public class LSMInvertedIndex extends AbstractLSMIndex implements IInvertedIndex
             initState = new LSMInvertedIndexSearchCursorInitialState(keyCmp, keysOnlyTuple, indexAccessors,
                     deletedKeysBTreeAccessors,
                     ((LSMInvertedIndexMemoryComponent) memoryComponents.get(currentMutableComponentId.get()))
-                            .getDeletedKeysBTree().getLeafFrameFactory(),
+                            .getBuddyIndex().getLeafFrameFactory(),
                     ictx, includeMutableComponent, getLsmHarness(), operationalComponents);
         } else {
             LSMInvertedIndexMemoryComponent mutableComponent =
                     (LSMInvertedIndexMemoryComponent) memoryComponents.get(currentMutableComponentId.get());
-            InMemoryInvertedIndex memInvIndex = (InMemoryInvertedIndex) mutableComponent.getInvIndex();
-            MultiComparator tokensAndKeysCmp = MultiComparator.create(memInvIndex.getBTree().getComparatorFactories());
+            MultiComparator tokensAndKeysCmp =
+                    MultiComparator.create(mutableComponent.getIndex().getBTree().getComparatorFactories());
             initState = new LSMInvertedIndexRangeSearchCursorInitialState(tokensAndKeysCmp, keyCmp, keysOnlyTuple,
                     ((LSMInvertedIndexMemoryComponent) memoryComponents.get(currentMutableComponentId.get()))
-                            .getDeletedKeysBTree().getLeafFrameFactory(),
+                            .getBuddyIndex().getLeafFrameFactory(),
                     includeMutableComponent, getLsmHarness(), indexAccessors, deletedKeysBTreeAccessors, pred,
                     operationalComponents);
         }
@@ -334,7 +271,7 @@ public class LSMInvertedIndex extends AbstractLSMIndex implements IInvertedIndex
         LSMInvertedIndexFlushOperation flushOp = (LSMInvertedIndexFlushOperation) operation;
 
         // Create an inverted index instance to be bulk loaded.
-        LSMInvertedIndexDiskComponent component = createDiskInvIndexComponent(componentFactory, flushOp.getTarget(),
+        ILSMDiskComponent component = createDiskComponent(componentFactory, flushOp.getTarget(),
                 flushOp.getDeletedKeysBTreeTarget(), flushOp.getBloomFilterTarget(), true);
 
         // Create a scan cursor on the BTree underlying the in-memory inverted index.
@@ -344,9 +281,9 @@ public class LSMInvertedIndex extends AbstractLSMIndex implements IInvertedIndex
         RangePredicate nullPred = new RangePredicate(null, null, true, true, null, null);
 
         // Search the deleted keys BTree to calculate the number of elements for BloomFilter
-        IIndexAccessor deletedKeysBTreeAccessor = flushingComponent.getDeletedKeysBTree()
+        BTreeAccessor deletedKeysBTreeAccessor = flushingComponent.getBuddyIndex()
                 .createAccessor(NoOpOperationCallback.INSTANCE, NoOpOperationCallback.INSTANCE);
-        IIndexCursor btreeCountingCursor = ((BTreeAccessor) deletedKeysBTreeAccessor).createCountingSearchCursor();
+        IIndexCursor btreeCountingCursor = deletedKeysBTreeAccessor.createCountingSearchCursor();
         deletedKeysBTreeAccessor.search(btreeCountingCursor, nullPred);
         long numBTreeTuples = 0L;
         try {
@@ -360,7 +297,7 @@ public class LSMInvertedIndex extends AbstractLSMIndex implements IInvertedIndex
         }
 
         ILSMDiskComponentBulkLoader componentBulkLoader =
-                createComponentBulkLoader(component, 1.0f, false, numBTreeTuples, false, false, false);
+                component.createBulkLoader(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);
@@ -369,16 +306,15 @@ public class LSMInvertedIndex extends AbstractLSMIndex implements IInvertedIndex
         try {
             while (deletedKeysScanCursor.hasNext()) {
                 deletedKeysScanCursor.next();
-                ((LSMInvertedIndexDiskComponentBulkLoader) componentBulkLoader)
-                        .delete(deletedKeysScanCursor.getTuple());
+                componentBulkLoader.delete(deletedKeysScanCursor.getTuple());
             }
         } finally {
             deletedKeysScanCursor.close();
         }
 
         // Scan the in-memory inverted index
-        InMemoryInvertedIndexAccessor memInvIndexAccessor = (InMemoryInvertedIndexAccessor) flushingComponent
-                .getInvIndex().createAccessor(NoOpOperationCallback.INSTANCE, NoOpOperationCallback.INSTANCE);
+        InMemoryInvertedIndexAccessor memInvIndexAccessor = flushingComponent.getIndex()
+                .createAccessor(NoOpOperationCallback.INSTANCE, NoOpOperationCallback.INSTANCE);
         BTreeAccessor memBTreeAccessor = memInvIndexAccessor.getBTreeAccessor();
         IIndexCursor scanCursor = memBTreeAccessor.createSearchCursor(false);
         memBTreeAccessor.search(scanCursor, nullPred);
@@ -397,8 +333,7 @@ public class LSMInvertedIndex extends AbstractLSMIndex implements IInvertedIndex
             filterTuples.add(flushingComponent.getLSMComponentFilter().getMinTuple());
             filterTuples.add(flushingComponent.getLSMComponentFilter().getMaxTuple());
             filterManager.updateFilter(component.getLSMComponentFilter(), filterTuples);
-            filterManager.writeFilter(component.getLSMComponentFilter(),
-                    ((OnDiskInvertedIndex) component.getInvIndex()).getBTree());
+            filterManager.writeFilter(component.getLSMComponentFilter(), component.getMetadataHolder());
         }
         flushingComponent.getMetadata().copy(component.getMetadata());
 
@@ -418,7 +353,7 @@ public class LSMInvertedIndex extends AbstractLSMIndex implements IInvertedIndex
         search(opCtx, cursor, mergePred);
 
         // Create an inverted index instance.
-        LSMInvertedIndexDiskComponent component = createDiskInvIndexComponent(componentFactory, mergeOp.getTarget(),
+        ILSMDiskComponent component = createDiskComponent(componentFactory, mergeOp.getTarget(),
                 mergeOp.getDeletedKeysBTreeTarget(), mergeOp.getBloomFilterTarget(), true);
 
         ILSMDiskComponentBulkLoader componentBulkLoader;
@@ -439,7 +374,7 @@ public class LSMInvertedIndex extends AbstractLSMIndex implements IInvertedIndex
                         .getNumElements();
             }
 
-            componentBulkLoader = createComponentBulkLoader(component, 1.0f, false, numElements, false, false, false);
+            componentBulkLoader = component.createBulkLoader(1.0f, false, numElements, false, false, false);
             try {
                 while (btreeCursor.hasNext()) {
                     btreeCursor.next();
@@ -450,7 +385,7 @@ public class LSMInvertedIndex extends AbstractLSMIndex implements IInvertedIndex
                 btreeCursor.close();
             }
         } else {
-            componentBulkLoader = createComponentBulkLoader(component, 1.0f, false, 0L, false, false, false);
+            componentBulkLoader = component.createBulkLoader(1.0f, false, 0L, false, false, false);
         }
 
         try {
@@ -475,8 +410,7 @@ public class LSMInvertedIndex extends AbstractLSMIndex implements IInvertedIndex
                 }
             }
             getFilterManager().updateFilter(component.getLSMComponentFilter(), filterTuples);
-            getFilterManager().writeFilter(component.getLSMComponentFilter(),
-                    ((OnDiskInvertedIndex) component.getInvIndex()).getBTree());
+            getFilterManager().writeFilter(component.getLSMComponentFilter(), component.getMetadataHolder());
         }
 
         componentBulkLoader.end();
@@ -484,75 +418,6 @@ public class LSMInvertedIndex extends AbstractLSMIndex implements IInvertedIndex
         return component;
     }
 
-    @Override
-    public ILSMDiskComponentBulkLoader createComponentBulkLoader(ILSMDiskComponent component, float fillFactor,
-            boolean verifyInput, long numElementsHint, boolean checkIfEmptyIndex, boolean withFilter,
-            boolean cleanupEmptyComponent) throws HyracksDataException {
-        BloomFilterSpecification bloomFilterSpec = null;
-        if (numElementsHint > 0) {
-            int maxBucketsPerElement = BloomCalculations.maxBucketsPerElement(numElementsHint);
-            bloomFilterSpec = BloomCalculations.computeBloomSpec(maxBucketsPerElement, bloomFilterFalsePositiveRate);
-        }
-        if (withFilter && filterFields != null) {
-            return new LSMInvertedIndexDiskComponentBulkLoader((LSMInvertedIndexDiskComponent) component,
-                    bloomFilterSpec, fillFactor, verifyInput, numElementsHint, checkIfEmptyIndex, cleanupEmptyComponent,
-                    filterManager, treeFields, filterFields,
-                    MultiComparator.create(component.getLSMComponentFilter().getFilterCmpFactories()));
-        } else {
-            return new LSMInvertedIndexDiskComponentBulkLoader((LSMInvertedIndexDiskComponent) component,
-                    bloomFilterSpec, fillFactor, verifyInput, numElementsHint, checkIfEmptyIndex,
-                    cleanupEmptyComponent);
-        }
-    }
-
-    @Override
-    public IIndexBulkLoader createBulkLoader(float fillFactor, boolean verifyInput, long numElementsHint)
-            throws HyracksDataException {
-        return new LSMInvertedIndexBulkLoader(fillFactor, verifyInput, numElementsHint);
-    }
-
-    public class LSMInvertedIndexBulkLoader implements IIndexBulkLoader {
-        private final ILSMDiskComponent component;
-        private final IIndexBulkLoader componentBulkLoader;
-
-        public LSMInvertedIndexBulkLoader(float fillFactor, boolean verifyInput, long numElementsHint)
-                throws HyracksDataException {
-            // Note that by using a flush target file name, we state that the
-            // new bulk loaded tree is "newer" than any other merged tree.
-            component = createBulkLoadTarget();
-
-            componentBulkLoader =
-                    createComponentBulkLoader(component, fillFactor, verifyInput, numElementsHint, false, true, true);
-        }
-
-        @Override
-        public void add(ITupleReference tuple) throws HyracksDataException {
-            componentBulkLoader.add(tuple);
-        }
-
-        @Override
-        public void end() throws HyracksDataException {
-            componentBulkLoader.end();
-            if (component.getComponentSize() > 0) {
-                ioOpCallback.afterOperation(LSMOperationType.FLUSH, null, component);
-                lsmHarness.addBulkLoadedComponent(component);
-            }
-        }
-
-        @Override
-        public void abort() throws HyracksDataException {
-            componentBulkLoader.abort();
-        }
-
-    }
-
-    @Override
-    public ILSMDiskComponent createBulkLoadTarget() throws HyracksDataException {
-        LSMComponentFileReferences componentFileRefs = fileManager.getRelFlushFileReference();
-        return createDiskInvIndexComponent(componentFactory, componentFileRefs.getInsertIndexFileReference(),
-                componentFileRefs.getDeleteIndexFileReference(), componentFileRefs.getBloomFilterFileReference(), true);
-    }
-
     protected InMemoryInvertedIndex createInMemoryInvertedIndex(IVirtualBufferCache virtualBufferCache,
             VirtualFreePageManager virtualFreePageManager, int id) throws HyracksDataException {
         return InvertedIndexUtils.createInMemoryBTreeInvertedindex(virtualBufferCache, virtualFreePageManager,
@@ -560,27 +425,6 @@ public class LSMInvertedIndex extends AbstractLSMIndex implements IInvertedIndex
                 ioManager.resolveAbsolutePath(fileManager.getBaseDir() + "_virtual_vocab_" + id));
     }
 
-    protected LSMInvertedIndexDiskComponent createDiskInvIndexComponent(ILSMDiskComponentFactory factory,
-            FileReference dictBTreeFileRef, FileReference btreeFileRef, FileReference bloomFilterFileRef,
-            boolean create) throws HyracksDataException {
-        LSMInvertedIndexDiskComponent component = (LSMInvertedIndexDiskComponent) factory
-                .createComponent(new LSMComponentFileReferences(dictBTreeFileRef, btreeFileRef, bloomFilterFileRef));
-        if (create) {
-            component.getInvIndex().create();
-            component.getDeletedKeysBTree().create();
-            component.getBloomFilter().create();
-        }
-        component.getInvIndex().activate();
-        component.getDeletedKeysBTree().activate();
-        component.getBloomFilter().activate();
-        // Will be closed during cleanup of merge().
-        if (component.getLSMComponentFilter() != null && !create) {
-            getFilterManager().readFilter(component.getLSMComponentFilter(),
-                    ((OnDiskInvertedIndex) component.getInvIndex()).getBTree());
-        }
-        return component;
-    }
-
     @Override
     public ILSMIndexAccessor createAccessor(IModificationOperationCallback modificationCallback,
             ISearchOperationCallback searchCallback) throws HyracksDataException {
@@ -624,65 +468,12 @@ public class LSMInvertedIndex extends AbstractLSMIndex implements IInvertedIndex
     }
 
     @Override
-    public Set<String> getLSMComponentPhysicalFiles(ILSMComponent lsmComponent) {
-        Set<String> files = new HashSet<>();
-        LSMInvertedIndexDiskComponent invIndexComponent = (LSMInvertedIndexDiskComponent) lsmComponent;
-        OnDiskInvertedIndex invIndex = (OnDiskInvertedIndex) invIndexComponent.getInvIndex();
-        files.add(invIndex.getInvListsFile().getFile().getAbsolutePath());
-        files.add(invIndex.getBTree().getFileReference().getFile().getAbsolutePath());
-        files.add(invIndexComponent.getBloomFilter().getFileReference().getFile().getAbsolutePath());
-        files.add(invIndexComponent.getDeletedKeysBTree().getFileReference().getFile().getAbsolutePath());
-        return files;
-    }
-
-    @Override
-    protected void clearMemoryComponent(ILSMMemoryComponent c) throws HyracksDataException {
-        LSMInvertedIndexMemoryComponent mutableComponent = (LSMInvertedIndexMemoryComponent) c;
-        mutableComponent.getInvIndex().clear();
-        mutableComponent.getDeletedKeysBTree().clear();
-        mutableComponent.reset();
-    }
-
-    @Override
-    protected long getMemoryComponentSize(ILSMMemoryComponent c) {
-        LSMInvertedIndexMemoryComponent mutableComponent = (LSMInvertedIndexMemoryComponent) c;
-        IBufferCache virtualBufferCache = mutableComponent.getInvIndex().getBufferCache();
-        return ((long) virtualBufferCache.getNumPages()) * virtualBufferCache.getPageSize();
-    }
-
-    @Override
-    protected void validateMemoryComponent(ILSMMemoryComponent c) throws HyracksDataException {
-        LSMInvertedIndexMemoryComponent mutableComponent = (LSMInvertedIndexMemoryComponent) c;
-        mutableComponent.getInvIndex().validate();
-        mutableComponent.getDeletedKeysBTree().validate();
-    }
-
-    @Override
-    protected void validateDiskComponent(ILSMDiskComponent c) throws HyracksDataException {
-        LSMInvertedIndexDiskComponent component = (LSMInvertedIndexDiskComponent) c;
-        component.getInvIndex().validate();
-        component.getDeletedKeysBTree().validate();
-    }
-
-    @Override
-    protected void allocateMemoryComponent(ILSMMemoryComponent c) throws HyracksDataException {
-        LSMInvertedIndexMemoryComponent mutableComponent = (LSMInvertedIndexMemoryComponent) c;
-        ((IVirtualBufferCache) mutableComponent.getInvIndex().getBufferCache()).open();
-        mutableComponent.getInvIndex().create();
-        mutableComponent.getInvIndex().activate();
-        mutableComponent.getDeletedKeysBTree().create();
-        mutableComponent.getDeletedKeysBTree().activate();
-    }
-
-    @Override
     protected LSMComponentFileReferences getMergeFileReferences(ILSMDiskComponent firstComponent,
             ILSMDiskComponent lastComponent) throws HyracksDataException {
         LSMInvertedIndexDiskComponent first = (LSMInvertedIndexDiskComponent) firstComponent;
-        OnDiskInvertedIndex firstInvIndex = (OnDiskInvertedIndex) first.getInvIndex();
-        String firstFileName = firstInvIndex.getBTree().getFileReference().getFile().getName();
+        String firstFileName = first.getMetadataHolder().getFileReference().getFile().getName();
         LSMInvertedIndexDiskComponent last = (LSMInvertedIndexDiskComponent) lastComponent;
-        OnDiskInvertedIndex lastInvIndex = (OnDiskInvertedIndex) last.getInvIndex();
-        String lastFileName = lastInvIndex.getBTree().getFileReference().getFile().getName();
+        String lastFileName = last.getMetadataHolder().getFileReference().getFile().getName();
         return fileManager.getRelMergeFileReference(firstFileName, lastFileName);
     }
 

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/77f89525/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndexDiskComponent.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndexDiskComponent.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndexDiskComponent.java
index b77f894..279a518 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndexDiskComponent.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndexDiskComponent.java
@@ -18,57 +18,62 @@
  */
 package org.apache.hyracks.storage.am.lsm.invertedindex.impls;
 
+import java.util.HashSet;
+import java.util.Set;
+
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.storage.am.bloomfilter.impls.BloomFilter;
 import org.apache.hyracks.storage.am.btree.impls.BTree;
 import org.apache.hyracks.storage.am.common.api.IMetadataPageManager;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponentFilter;
-import org.apache.hyracks.storage.am.lsm.common.impls.AbstractLSMDiskComponent;
+import org.apache.hyracks.storage.am.lsm.common.api.AbstractLSMWithBuddyDiskComponent;
+import org.apache.hyracks.storage.am.lsm.common.impls.AbstractLSMIndex;
 import org.apache.hyracks.storage.am.lsm.common.util.ComponentUtils;
-import org.apache.hyracks.storage.am.lsm.invertedindex.api.IInPlaceInvertedIndex;
 import org.apache.hyracks.storage.am.lsm.invertedindex.ondisk.OnDiskInvertedIndex;
 import org.apache.hyracks.storage.common.buffercache.IBufferCache;
 
-public class LSMInvertedIndexDiskComponent extends AbstractLSMDiskComponent {
+public class LSMInvertedIndexDiskComponent extends AbstractLSMWithBuddyDiskComponent {
 
-    private final IInPlaceInvertedIndex invIndex;
+    private final OnDiskInvertedIndex invIndex;
     private final BTree deletedKeysBTree;
     private final BloomFilter bloomFilter;
 
-    public LSMInvertedIndexDiskComponent(IInPlaceInvertedIndex invIndex, BTree deletedKeysBTree,
-            BloomFilter bloomFilter, ILSMComponentFilter filter) throws HyracksDataException {
-        super((IMetadataPageManager) deletedKeysBTree.getPageManager(), filter);
+    public LSMInvertedIndexDiskComponent(AbstractLSMIndex lsmIndex, OnDiskInvertedIndex invIndex,
+            BTree deletedKeysBTree, BloomFilter bloomFilter, ILSMComponentFilter filter) {
+        super(lsmIndex, (IMetadataPageManager) deletedKeysBTree.getPageManager(), filter);
         this.invIndex = invIndex;
         this.deletedKeysBTree = deletedKeysBTree;
         this.bloomFilter = bloomFilter;
     }
 
     @Override
-    public void destroy() throws HyracksDataException {
-        invIndex.deactivate();
-        invIndex.destroy();
-        deletedKeysBTree.deactivate();
-        deletedKeysBTree.destroy();
-        bloomFilter.deactivate();
-        bloomFilter.destroy();
+    public OnDiskInvertedIndex getIndex() {
+        return invIndex;
     }
 
-    public IInPlaceInvertedIndex getInvIndex() {
-        return invIndex;
+    @Override
+    public BTree getMetadataHolder() {
+        return invIndex.getBTree();
     }
 
-    public BTree getDeletedKeysBTree() {
+    @Override
+    public BTree getBuddyIndex() {
         return deletedKeysBTree;
     }
 
+    @Override
     public BloomFilter getBloomFilter() {
         return bloomFilter;
     }
 
     @Override
+    public IBufferCache getBloomFilterBufferCache() {
+        return invIndex.getBufferCache();
+    }
+
+    @Override
     public long getComponentSize() {
-        return ((OnDiskInvertedIndex) invIndex).getInvListsFile().getFile().length()
-                + ((OnDiskInvertedIndex) invIndex).getBTree().getFileReference().getFile().length()
+        return invIndex.getInvListsFile().getFile().length() + invIndex.getBTree().getFileReference().getFile().length()
                 + deletedKeysBTree.getFileReference().getFile().length()
                 + bloomFilter.getFileReference().getFile().length();
     }
@@ -79,20 +84,29 @@ public class LSMInvertedIndexDiskComponent extends AbstractLSMDiskComponent {
     }
 
     @Override
+    public Set<String> getLSMComponentPhysicalFiles() {
+        Set<String> files = new HashSet<>();
+        files.add(invIndex.getInvListsFile().getFile().getAbsolutePath());
+        files.add(invIndex.getBTree().getFileReference().getFile().getAbsolutePath());
+        files.add(bloomFilter.getFileReference().getFile().getAbsolutePath());
+        files.add(deletedKeysBTree.getFileReference().getFile().getAbsolutePath());
+        return files;
+    }
+
+    @Override
     public String toString() {
-        return getClass().getSimpleName() + ":" + ((OnDiskInvertedIndex) invIndex).getInvListsFile().getRelativePath();
+        return getClass().getSimpleName() + ":" + invIndex.getInvListsFile().getRelativePath();
     }
 
     @Override
     public void markAsValid(boolean persist) throws HyracksDataException {
-        IBufferCache bufferCache = invIndex.getBufferCache();
-        ComponentUtils.markAsValid(invIndex.getBufferCache(), bloomFilter, persist);
+        ComponentUtils.markAsValid(getBloomFilterBufferCache(), getBloomFilter(), persist);
 
         // Flush inverted index second.
-        bufferCache.force(((OnDiskInvertedIndex) invIndex).getInvListsFileId(), true);
-        ComponentUtils.markAsValid(((OnDiskInvertedIndex) invIndex).getBTree(), persist);
+        invIndex.getBufferCache().force((invIndex).getInvListsFileId(), true);
+        ComponentUtils.markAsValid(getMetadataHolder(), persist);
 
         // Flush deleted keys BTree.
-        ComponentUtils.markAsValid(deletedKeysBTree, persist);
+        ComponentUtils.markAsValid(getBuddyIndex(), persist);
     }
 }

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/77f89525/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndexDiskComponentBulkLoader.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndexDiskComponentBulkLoader.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndexDiskComponentBulkLoader.java
deleted file mode 100644
index daa93ae..0000000
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndexDiskComponentBulkLoader.java
+++ /dev/null
@@ -1,71 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.hyracks.storage.am.lsm.invertedindex.impls;
-
-import org.apache.hyracks.api.exceptions.HyracksDataException;
-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.ITreeIndex;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponentFilterManager;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMDiskComponent;
-import org.apache.hyracks.storage.am.lsm.common.impls.AbstractLSMDiskComponentWithBuddyBulkLoader;
-import org.apache.hyracks.storage.am.lsm.invertedindex.ondisk.OnDiskInvertedIndex;
-import org.apache.hyracks.storage.common.IIndex;
-import org.apache.hyracks.storage.common.MultiComparator;
-
-public class LSMInvertedIndexDiskComponentBulkLoader extends AbstractLSMDiskComponentWithBuddyBulkLoader {
-
-    //with filter
-    public LSMInvertedIndexDiskComponentBulkLoader(LSMInvertedIndexDiskComponent component,
-            BloomFilterSpecification bloomFilterSpec, float fillFactor, boolean verifyInput, long numElementsHint,
-            boolean checkIfEmptyIndex, boolean cleanupEmptyComponent, ILSMComponentFilterManager filterManager,
-            int[] indexFields, int[] filterFields, MultiComparator filterCmp) throws HyracksDataException {
-        super(component, bloomFilterSpec, fillFactor, verifyInput, numElementsHint, checkIfEmptyIndex,
-                cleanupEmptyComponent, filterManager, indexFields, filterFields, filterCmp);
-    }
-
-    //without filter
-    public LSMInvertedIndexDiskComponentBulkLoader(LSMInvertedIndexDiskComponent component,
-            BloomFilterSpecification bloomFilterSpec, float fillFactor, boolean verifyInput, long numElementsHint,
-            boolean checkIfEmptyIndex, boolean cleanupEmptyComponent) throws HyracksDataException {
-        super(component, bloomFilterSpec, fillFactor, verifyInput, numElementsHint, checkIfEmptyIndex,
-                cleanupEmptyComponent, null, null, null, null);
-    }
-
-    @Override
-    protected BloomFilter getBloomFilter(ILSMDiskComponent component) {
-        return ((LSMInvertedIndexDiskComponent) component).getBloomFilter();
-    }
-
-    @Override
-    protected IIndex getIndex(ILSMDiskComponent component) {
-        return ((LSMInvertedIndexDiskComponent) component).getInvIndex();
-    }
-
-    @Override
-    protected ITreeIndex getTreeIndex(ILSMDiskComponent component) {
-        return ((OnDiskInvertedIndex) ((LSMInvertedIndexDiskComponent) component).getInvIndex()).getBTree();
-    }
-
-    @Override
-    protected ITreeIndex getBuddyBTree(ILSMDiskComponent component) {
-        return ((LSMInvertedIndexDiskComponent) component).getDeletedKeysBTree();
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/77f89525/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndexDiskComponentFactory.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndexDiskComponentFactory.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndexDiskComponentFactory.java
index d018535..253ed20 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndexDiskComponentFactory.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndexDiskComponentFactory.java
@@ -24,6 +24,7 @@ import org.apache.hyracks.storage.am.bloomfilter.impls.BloomFilterFactory;
 import org.apache.hyracks.storage.am.btree.impls.BTree;
 import org.apache.hyracks.storage.am.lsm.common.api.IComponentFilterHelper;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMDiskComponentFactory;
+import org.apache.hyracks.storage.am.lsm.common.impls.AbstractLSMIndex;
 import org.apache.hyracks.storage.am.lsm.common.impls.LSMComponentFileReferences;
 import org.apache.hyracks.storage.am.lsm.common.impls.TreeIndexFactory;
 import org.apache.hyracks.storage.am.lsm.invertedindex.ondisk.OnDiskInvertedIndexFactory;
@@ -44,8 +45,9 @@ public class LSMInvertedIndexDiskComponentFactory implements ILSMDiskComponentFa
     }
 
     @Override
-    public LSMInvertedIndexDiskComponent createComponent(LSMComponentFileReferences cfr) throws HyracksDataException {
-        return new LSMInvertedIndexDiskComponent(
+    public LSMInvertedIndexDiskComponent createComponent(AbstractLSMIndex lsmIndex, LSMComponentFileReferences cfr)
+            throws HyracksDataException {
+        return new LSMInvertedIndexDiskComponent(lsmIndex,
                 diskInvIndexFactory.createIndexInstance(cfr.getInsertIndexFileReference()),
                 btreeFactory.createIndexInstance(cfr.getDeleteIndexFileReference()),
                 bloomFilterFactory.createBloomFiltertInstance(cfr.getBloomFilterFileReference()),

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/77f89525/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndexMemoryComponent.java
----------------------------------------------------------------------
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 c164cf9..2d2111c 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
@@ -19,43 +19,31 @@
 
 package org.apache.hyracks.storage.am.lsm.invertedindex.impls;
 
-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.ILSMComponentFilter;
+import org.apache.hyracks.storage.am.lsm.common.api.AbstractLSMWithBuddyMemoryComponent;
 import org.apache.hyracks.storage.am.lsm.common.api.IVirtualBufferCache;
-import org.apache.hyracks.storage.am.lsm.common.impls.AbstractLSMMemoryComponent;
-import org.apache.hyracks.storage.am.lsm.invertedindex.api.IInvertedIndex;
+import org.apache.hyracks.storage.am.lsm.invertedindex.inmemory.InMemoryInvertedIndex;
 
-public class LSMInvertedIndexMemoryComponent extends AbstractLSMMemoryComponent {
+public class LSMInvertedIndexMemoryComponent extends AbstractLSMWithBuddyMemoryComponent {
 
-    private final IInvertedIndex invIndex;
+    private final InMemoryInvertedIndex invIndex;
     private final BTree deletedKeysBTree;
 
-    public LSMInvertedIndexMemoryComponent(IInvertedIndex invIndex, BTree deletedKeysBTree, IVirtualBufferCache vbc,
-            boolean isActive, ILSMComponentFilter filter) {
+    public LSMInvertedIndexMemoryComponent(InMemoryInvertedIndex invIndex, BTree deletedKeysBTree,
+            IVirtualBufferCache vbc, boolean isActive, ILSMComponentFilter filter) {
         super(vbc, isActive, filter);
         this.invIndex = invIndex;
         this.deletedKeysBTree = deletedKeysBTree;
     }
 
-    public IInvertedIndex getInvIndex() {
+    @Override
+    public InMemoryInvertedIndex getIndex() {
         return invIndex;
     }
 
-    public BTree getDeletedKeysBTree() {
-        return deletedKeysBTree;
-    }
-
     @Override
-    public void reset() throws HyracksDataException {
-        super.reset();
-        invIndex.deactivate();
-        invIndex.destroy();
-        invIndex.create();
-        invIndex.activate();
-        deletedKeysBTree.deactivate();
-        deletedKeysBTree.destroy();
-        deletedKeysBTree.create();
-        deletedKeysBTree.activate();
+    public BTree getBuddyIndex() {
+        return deletedKeysBTree;
     }
 }

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/77f89525/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndexOpContext.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndexOpContext.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndexOpContext.java
index c725501..30300d3 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndexOpContext.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndexOpContext.java
@@ -28,8 +28,6 @@ import org.apache.hyracks.storage.am.common.tuples.PermutingTupleReference;
 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.am.lsm.invertedindex.api.IInvertedIndexAccessor;
-import org.apache.hyracks.storage.am.lsm.invertedindex.inmemory.InMemoryInvertedIndex;
-import org.apache.hyracks.storage.am.lsm.invertedindex.inmemory.InMemoryInvertedIndexAccessor;
 import org.apache.hyracks.storage.common.IIndexAccessor;
 import org.apache.hyracks.storage.common.IModificationOperationCallback;
 import org.apache.hyracks.storage.common.ISearchOperationCallback;
@@ -58,19 +56,17 @@ public class LSMInvertedIndexOpContext extends AbstractLSMIndexOperationContext
             LSMInvertedIndexMemoryComponent mutableComponent =
                     (LSMInvertedIndexMemoryComponent) mutableComponents.get(i);
             if (allFields != null) {
-                mutableInvIndexAccessors[i] = (IInvertedIndexAccessor) ((InMemoryInvertedIndex) mutableComponent
-                        .getInvIndex()).createAccessor(NoOpOperationCallback.INSTANCE, NoOpOperationCallback.INSTANCE,
-                                allFields);
+                mutableInvIndexAccessors[i] = mutableComponent.getIndex().createAccessor(allFields);
             } else {
-                mutableInvIndexAccessors[i] = (IInvertedIndexAccessor) mutableComponent.getInvIndex()
-                        .createAccessor(NoOpOperationCallback.INSTANCE, NoOpOperationCallback.INSTANCE);
+                mutableInvIndexAccessors[i] = mutableComponent.getIndex().createAccessor(NoOpOperationCallback.INSTANCE,
+                        NoOpOperationCallback.INSTANCE);
             }
-            deletedKeysBTreeAccessors[i] = mutableComponent.getDeletedKeysBTree()
+            deletedKeysBTreeAccessors[i] = mutableComponent.getBuddyIndex()
                     .createAccessor(NoOpOperationCallback.INSTANCE, NoOpOperationCallback.INSTANCE);
         }
         // Project away the document fields, leaving only the key fields.
         LSMInvertedIndexMemoryComponent c = (LSMInvertedIndexMemoryComponent) mutableComponents.get(0);
-        int numKeyFields = c.getInvIndex().getInvListTypeTraits().length;
+        int numKeyFields = c.getIndex().getInvListTypeTraits().length;
         int[] keyFieldPermutation = new int[numKeyFields];
         for (int i = 0; i < numKeyFields; i++) {
             keyFieldPermutation[i] = NUM_DOCUMENT_FIELDS + i;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/77f89525/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/impls/PartitionedLSMInvertedIndex.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/impls/PartitionedLSMInvertedIndex.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/impls/PartitionedLSMInvertedIndex.java
index d154356..b3a63af 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/impls/PartitionedLSMInvertedIndex.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/impls/PartitionedLSMInvertedIndex.java
@@ -25,9 +25,9 @@ import org.apache.hyracks.api.dataflow.value.IBinaryComparatorFactory;
 import org.apache.hyracks.api.dataflow.value.ITypeTraits;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.api.io.IIOManager;
-import org.apache.hyracks.storage.am.bloomfilter.impls.BloomFilterFactory;
 import org.apache.hyracks.storage.am.lsm.common.api.IComponentFilterHelper;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponentFilterFrameFactory;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMDiskComponentFactory;
 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.ILSMIndexFileManager;
@@ -35,31 +35,29 @@ 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.freepage.VirtualFreePageManager;
-import org.apache.hyracks.storage.am.lsm.common.impls.BTreeFactory;
 import org.apache.hyracks.storage.am.lsm.common.impls.LSMComponentFilterManager;
 import org.apache.hyracks.storage.am.lsm.invertedindex.inmemory.InMemoryInvertedIndex;
-import org.apache.hyracks.storage.am.lsm.invertedindex.ondisk.OnDiskInvertedIndexFactory;
 import org.apache.hyracks.storage.am.lsm.invertedindex.tokenizers.IBinaryTokenizerFactory;
 import org.apache.hyracks.storage.am.lsm.invertedindex.util.InvertedIndexUtils;
+import org.apache.hyracks.storage.common.buffercache.IBufferCache;
 
 public class PartitionedLSMInvertedIndex extends LSMInvertedIndex {
 
     public PartitionedLSMInvertedIndex(IIOManager ioManager, List<IVirtualBufferCache> virtualBufferCaches,
-            OnDiskInvertedIndexFactory diskInvIndexFactory, BTreeFactory deletedKeysBTreeFactory,
-            BloomFilterFactory bloomFilterFactory, IComponentFilterHelper filterHelper,
+            ILSMDiskComponentFactory componentFactory, IComponentFilterHelper filterHelper,
             ILSMComponentFilterFrameFactory filterFrameFactory, LSMComponentFilterManager filterManager,
-            double bloomFilterFalsePositiveRate, ILSMIndexFileManager fileManager, ITypeTraits[] invListTypeTraits,
-            IBinaryComparatorFactory[] invListCmpFactories, ITypeTraits[] tokenTypeTraits,
-            IBinaryComparatorFactory[] tokenCmpFactories, IBinaryTokenizerFactory tokenizerFactory,
-            ILSMMergePolicy mergePolicy, ILSMOperationTracker opTracker, ILSMIOOperationScheduler ioScheduler,
-            ILSMIOOperationCallback ioOpCallback, int[] invertedIndexFields, int[] filterFields,
-            int[] filterFieldsForNonBulkLoadOps, int[] invertedIndexFieldsForNonBulkLoadOps, boolean durable)
-            throws HyracksDataException {
-        super(ioManager, virtualBufferCaches, diskInvIndexFactory, deletedKeysBTreeFactory, bloomFilterFactory,
-                filterHelper, filterFrameFactory, filterManager, bloomFilterFalsePositiveRate, fileManager,
-                invListTypeTraits, invListCmpFactories, tokenTypeTraits, tokenCmpFactories, tokenizerFactory,
-                mergePolicy, opTracker, ioScheduler, ioOpCallback, invertedIndexFields, filterFields,
-                filterFieldsForNonBulkLoadOps, invertedIndexFieldsForNonBulkLoadOps, durable);
+            double bloomFilterFalsePositiveRate, IBufferCache diskBufferCache, ILSMIndexFileManager fileManager,
+            ITypeTraits[] invListTypeTraits, IBinaryComparatorFactory[] invListCmpFactories,
+            ITypeTraits[] tokenTypeTraits, IBinaryComparatorFactory[] tokenCmpFactories,
+            IBinaryTokenizerFactory tokenizerFactory, ILSMMergePolicy mergePolicy, ILSMOperationTracker opTracker,
+            ILSMIOOperationScheduler ioScheduler, ILSMIOOperationCallback ioOpCallback, int[] invertedIndexFields,
+            int[] filterFields, int[] filterFieldsForNonBulkLoadOps, int[] invertedIndexFieldsForNonBulkLoadOps,
+            boolean durable) throws HyracksDataException {
+        super(ioManager, virtualBufferCaches, componentFactory, filterHelper, filterFrameFactory, filterManager,
+                bloomFilterFalsePositiveRate, diskBufferCache, fileManager, invListTypeTraits, invListCmpFactories,
+                tokenTypeTraits, tokenCmpFactories, tokenizerFactory, mergePolicy, opTracker, ioScheduler, ioOpCallback,
+                invertedIndexFields, filterFields, filterFieldsForNonBulkLoadOps, invertedIndexFieldsForNonBulkLoadOps,
+                durable);
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/77f89525/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/inmemory/InMemoryInvertedIndex.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/inmemory/InMemoryInvertedIndex.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/inmemory/InMemoryInvertedIndex.java
index 54cae72..cf40a7a 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/inmemory/InMemoryInvertedIndex.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/inmemory/InMemoryInvertedIndex.java
@@ -34,7 +34,6 @@ import org.apache.hyracks.storage.am.common.ophelpers.IndexOperation;
 import org.apache.hyracks.storage.am.lsm.invertedindex.api.IInPlaceInvertedIndex;
 import org.apache.hyracks.storage.am.lsm.invertedindex.api.IInvertedListCursor;
 import org.apache.hyracks.storage.am.lsm.invertedindex.tokenizers.IBinaryTokenizerFactory;
-import org.apache.hyracks.storage.common.IIndexAccessor;
 import org.apache.hyracks.storage.common.IIndexBulkLoader;
 import org.apache.hyracks.storage.common.IModificationOperationCallback;
 import org.apache.hyracks.storage.common.ISearchOperationCallback;
@@ -167,14 +166,13 @@ public class InMemoryInvertedIndex implements IInPlaceInvertedIndex {
     }
 
     @Override
-    public IIndexAccessor createAccessor(IModificationOperationCallback modificationCallback,
+    public InMemoryInvertedIndexAccessor createAccessor(IModificationOperationCallback modificationCallback,
             ISearchOperationCallback searchCallback) throws HyracksDataException {
         return new InMemoryInvertedIndexAccessor(this,
                 new InMemoryInvertedIndexOpContext(btree, tokenCmpFactories, tokenizerFactory));
     }
 
-    public IIndexAccessor createAccessor(IModificationOperationCallback modificationCallback,
-            ISearchOperationCallback searchCallback, int[] nonIndexFields) throws HyracksDataException {
+    public InMemoryInvertedIndexAccessor createAccessor(int[] nonIndexFields) throws HyracksDataException {
         return new InMemoryInvertedIndexAccessor(this,
                 new InMemoryInvertedIndexOpContext(btree, tokenCmpFactories, tokenizerFactory), nonIndexFields);
     }

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/77f89525/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/inmemory/InMemoryInvertedIndexAccessor.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/inmemory/InMemoryInvertedIndexAccessor.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/inmemory/InMemoryInvertedIndexAccessor.java
index 2e0511c..73f90a6 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/inmemory/InMemoryInvertedIndexAccessor.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/inmemory/InMemoryInvertedIndexAccessor.java
@@ -51,8 +51,8 @@ public class InMemoryInvertedIndexAccessor implements IInvertedIndexAccessor {
         this.opCtx = opCtx;
         this.index = index;
         this.searcher = createSearcher();
-        this.btreeAccessor = (BTreeAccessor) index.getBTree().createAccessor(NoOpOperationCallback.INSTANCE,
-                NoOpOperationCallback.INSTANCE);
+        this.btreeAccessor =
+                index.getBTree().createAccessor(NoOpOperationCallback.INSTANCE, NoOpOperationCallback.INSTANCE);
     }
 
     public InMemoryInvertedIndexAccessor(InMemoryInvertedIndex index, IIndexOperationContext opCtx,
@@ -60,7 +60,7 @@ public class InMemoryInvertedIndexAccessor implements IInvertedIndexAccessor {
         this.opCtx = opCtx;
         this.index = index;
         this.searcher = createSearcher();
-        this.btreeAccessor = (BTreeAccessor) index.getBTree().createAccessor(NoOpOperationCallback.INSTANCE,
+        this.btreeAccessor = index.getBTree().createAccessor(NoOpOperationCallback.INSTANCE,
                 NoOpOperationCallback.INSTANCE, nonIndexFields);
     }
 

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/77f89525/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/inmemory/InMemoryInvertedIndexOpContext.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/inmemory/InMemoryInvertedIndexOpContext.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/inmemory/InMemoryInvertedIndexOpContext.java
index c37f880..90dbc81 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/inmemory/InMemoryInvertedIndexOpContext.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/inmemory/InMemoryInvertedIndexOpContext.java
@@ -67,8 +67,8 @@ public class InMemoryInvertedIndexOpContext implements IIndexOperationContext {
             case SEARCH: {
                 if (getBtreePred() == null) {
                     btreePred = new RangePredicate(null, null, true, true, null, null);
-                    btreeAccessor = (BTreeAccessor) btree.createAccessor(NoOpOperationCallback.INSTANCE,
-                            NoOpOperationCallback.INSTANCE);
+                    btreeAccessor =
+                            btree.createAccessor(NoOpOperationCallback.INSTANCE, NoOpOperationCallback.INSTANCE);
                     btreeCmp = MultiComparator.create(btree.getComparatorFactories());
                     tokenFieldsCmp = MultiComparator.create(tokenCmpFactories);
                 }

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/77f89525/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/inmemory/PartitionedInMemoryInvertedIndex.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/inmemory/PartitionedInMemoryInvertedIndex.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/inmemory/PartitionedInMemoryInvertedIndex.java
index e615bf0..933ef21 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/inmemory/PartitionedInMemoryInvertedIndex.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/inmemory/PartitionedInMemoryInvertedIndex.java
@@ -37,7 +37,6 @@ import org.apache.hyracks.storage.am.lsm.invertedindex.search.InvertedListPartit
 import org.apache.hyracks.storage.am.lsm.invertedindex.search.PartitionedTOccurrenceSearcher;
 import org.apache.hyracks.storage.am.lsm.invertedindex.tokenizers.IBinaryTokenizerFactory;
 import org.apache.hyracks.storage.am.lsm.invertedindex.util.PartitionedInvertedIndexTokenizingTupleIterator;
-import org.apache.hyracks.storage.common.IIndexAccessor;
 import org.apache.hyracks.storage.common.IModificationOperationCallback;
 import org.apache.hyracks.storage.common.ISearchOperationCallback;
 import org.apache.hyracks.storage.common.buffercache.IBufferCache;
@@ -88,15 +87,14 @@ public class PartitionedInMemoryInvertedIndex extends InMemoryInvertedIndex impl
     }
 
     @Override
-    public IIndexAccessor createAccessor(IModificationOperationCallback modificationCallback,
+    public PartitionedInMemoryInvertedIndexAccessor createAccessor(IModificationOperationCallback modificationCallback,
             ISearchOperationCallback searchCallback) throws HyracksDataException {
         return new PartitionedInMemoryInvertedIndexAccessor(this,
                 new PartitionedInMemoryInvertedIndexOpContext(btree, tokenCmpFactories, tokenizerFactory));
     }
 
     @Override
-    public IIndexAccessor createAccessor(IModificationOperationCallback modificationCallback,
-            ISearchOperationCallback searchCallback, int[] nonIndexFields) throws HyracksDataException {
+    public PartitionedInMemoryInvertedIndexAccessor createAccessor(int[] nonIndexFields) throws HyracksDataException {
         return new PartitionedInMemoryInvertedIndexAccessor(this,
                 new PartitionedInMemoryInvertedIndexOpContext(btree, tokenCmpFactories, tokenizerFactory),
                 nonIndexFields);

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/77f89525/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/ondisk/OnDiskInvertedIndex.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/ondisk/OnDiskInvertedIndex.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/ondisk/OnDiskInvertedIndex.java
index 5d8cafb..6244ee7 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/ondisk/OnDiskInvertedIndex.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/ondisk/OnDiskInvertedIndex.java
@@ -486,7 +486,7 @@ public class OnDiskInvertedIndex implements IInPlaceInvertedIndex {
     }
 
     @Override
-    public IIndexAccessor createAccessor(IModificationOperationCallback modificationCallback,
+    public OnDiskInvertedIndexAccessor createAccessor(IModificationOperationCallback modificationCallback,
             ISearchOperationCallback searchCallback) throws HyracksDataException {
         return new OnDiskInvertedIndexAccessor(this);
     }
@@ -554,7 +554,7 @@ public class OnDiskInvertedIndex implements IInPlaceInvertedIndex {
         PermutingTupleReference tokenTuple = new PermutingTupleReference(fieldPermutation);
 
         IInvertedIndexAccessor invIndexAccessor =
-                (IInvertedIndexAccessor) createAccessor(NoOpOperationCallback.INSTANCE, NoOpOperationCallback.INSTANCE);
+                createAccessor(NoOpOperationCallback.INSTANCE, NoOpOperationCallback.INSTANCE);
         IInvertedListCursor invListCursor = invIndexAccessor.createInvertedListCursor();
         MultiComparator invListCmp = MultiComparator.create(invListCmpFactories);
 

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/77f89525/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/ondisk/OnDiskInvertedIndexFactory.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/ondisk/OnDiskInvertedIndexFactory.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/ondisk/OnDiskInvertedIndexFactory.java
index 0edc63c..f058e4a 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/ondisk/OnDiskInvertedIndexFactory.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/ondisk/OnDiskInvertedIndexFactory.java
@@ -26,7 +26,6 @@ import org.apache.hyracks.api.io.IIOManager;
 import org.apache.hyracks.storage.am.common.api.IPageManagerFactory;
 import org.apache.hyracks.storage.am.lsm.common.impls.IndexFactory;
 import org.apache.hyracks.storage.am.lsm.invertedindex.api.IInvertedIndex;
-import org.apache.hyracks.storage.am.lsm.invertedindex.api.IInPlaceInvertedIndex;
 import org.apache.hyracks.storage.am.lsm.invertedindex.api.IInvertedIndexFileNameMapper;
 import org.apache.hyracks.storage.am.lsm.invertedindex.api.IInvertedListBuilder;
 import org.apache.hyracks.storage.am.lsm.invertedindex.api.IInvertedListBuilderFactory;
@@ -56,7 +55,7 @@ public class OnDiskInvertedIndexFactory extends IndexFactory<IInvertedIndex> {
     }
 
     @Override
-    public IInPlaceInvertedIndex createIndexInstance(FileReference dictBTreeFile) throws HyracksDataException {
+    public OnDiskInvertedIndex createIndexInstance(FileReference dictBTreeFile) throws HyracksDataException {
         String invListsFilePath = fileNameMapper.getInvListsFilePath(dictBTreeFile.getFile().getAbsolutePath());
         FileReference invListsFile = ioManager.resolveAbsolutePath(invListsFilePath);
         IInvertedListBuilder invListBuilder = invListBuilderFactory.create();

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/77f89525/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/ondisk/PartitionedOnDiskInvertedIndex.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/ondisk/PartitionedOnDiskInvertedIndex.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/ondisk/PartitionedOnDiskInvertedIndex.java
index 0c0110e..569c51d 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/ondisk/PartitionedOnDiskInvertedIndex.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/ondisk/PartitionedOnDiskInvertedIndex.java
@@ -35,7 +35,6 @@ import org.apache.hyracks.storage.am.lsm.invertedindex.api.IInvertedListCursor;
 import org.apache.hyracks.storage.am.lsm.invertedindex.api.IPartitionedInvertedIndex;
 import org.apache.hyracks.storage.am.lsm.invertedindex.search.InvertedListPartitions;
 import org.apache.hyracks.storage.am.lsm.invertedindex.search.PartitionedTOccurrenceSearcher;
-import org.apache.hyracks.storage.common.IIndexAccessor;
 import org.apache.hyracks.storage.common.IModificationOperationCallback;
 import org.apache.hyracks.storage.common.ISearchOperationCallback;
 import org.apache.hyracks.storage.common.buffercache.IBufferCache;
@@ -59,7 +58,7 @@ public class PartitionedOnDiskInvertedIndex extends OnDiskInvertedIndex implemen
     }
 
     @Override
-    public IIndexAccessor createAccessor(IModificationOperationCallback modificationCallback,
+    public PartitionedOnDiskInvertedIndexAccessor createAccessor(IModificationOperationCallback modificationCallback,
             ISearchOperationCallback searchCallback) throws HyracksDataException {
         return new PartitionedOnDiskInvertedIndexAccessor(this);
     }

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/77f89525/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/ondisk/PartitionedOnDiskInvertedIndexFactory.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/ondisk/PartitionedOnDiskInvertedIndexFactory.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/ondisk/PartitionedOnDiskInvertedIndexFactory.java
index e8d12ad..7f10658 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/ondisk/PartitionedOnDiskInvertedIndexFactory.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/ondisk/PartitionedOnDiskInvertedIndexFactory.java
@@ -24,7 +24,6 @@ import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.api.io.FileReference;
 import org.apache.hyracks.api.io.IIOManager;
 import org.apache.hyracks.storage.am.common.api.IPageManagerFactory;
-import org.apache.hyracks.storage.am.lsm.invertedindex.api.IInPlaceInvertedIndex;
 import org.apache.hyracks.storage.am.lsm.invertedindex.api.IInvertedIndexFileNameMapper;
 import org.apache.hyracks.storage.am.lsm.invertedindex.api.IInvertedListBuilder;
 import org.apache.hyracks.storage.am.lsm.invertedindex.api.IInvertedListBuilderFactory;
@@ -42,7 +41,7 @@ public class PartitionedOnDiskInvertedIndexFactory extends OnDiskInvertedIndexFa
     }
 
     @Override
-    public IInPlaceInvertedIndex createIndexInstance(FileReference dictBTreeFile) throws HyracksDataException {
+    public PartitionedOnDiskInvertedIndex createIndexInstance(FileReference dictBTreeFile) throws HyracksDataException {
         String invListsFilePath = fileNameMapper.getInvListsFilePath(dictBTreeFile.getFile().getAbsolutePath());
         FileReference invListsFile = ioManager.resolveAbsolutePath(invListsFilePath);
         IInvertedListBuilder invListBuilder = invListBuilderFactory.create();

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/77f89525/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/util/InvertedIndexUtils.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/util/InvertedIndexUtils.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/util/InvertedIndexUtils.java
index 48afdbc3..8f9b5de 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/util/InvertedIndexUtils.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/util/InvertedIndexUtils.java
@@ -36,6 +36,7 @@ import org.apache.hyracks.storage.am.common.api.IPageManager;
 import org.apache.hyracks.storage.am.common.api.IPageManagerFactory;
 import org.apache.hyracks.storage.am.common.api.ITreeIndexFrameFactory;
 import org.apache.hyracks.storage.am.common.tuples.TypeAwareTupleWriterFactory;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMDiskComponentFactory;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallback;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationScheduler;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMMergePolicy;
@@ -48,6 +49,7 @@ import org.apache.hyracks.storage.am.lsm.common.impls.LSMComponentFilterManager;
 import org.apache.hyracks.storage.am.lsm.invertedindex.api.IInvertedListBuilder;
 import org.apache.hyracks.storage.am.lsm.invertedindex.api.IInvertedListBuilderFactory;
 import org.apache.hyracks.storage.am.lsm.invertedindex.impls.LSMInvertedIndex;
+import org.apache.hyracks.storage.am.lsm.invertedindex.impls.LSMInvertedIndexDiskComponentFactory;
 import org.apache.hyracks.storage.am.lsm.invertedindex.impls.LSMInvertedIndexFileManager;
 import org.apache.hyracks.storage.am.lsm.invertedindex.impls.PartitionedLSMInvertedIndex;
 import org.apache.hyracks.storage.am.lsm.invertedindex.inmemory.InMemoryInvertedIndex;
@@ -157,11 +159,14 @@ public class InvertedIndexUtils {
             filterFrameFactory = new LSMComponentFilterFrameFactory(filterTupleWriterFactory);
             filterManager = new LSMComponentFilterManager(filterFrameFactory);
         }
-        return new LSMInvertedIndex(ioManager, virtualBufferCaches, invIndexFactory, deletedKeysBTreeFactory,
-                bloomFilterFactory, filterHelper, filterFrameFactory, filterManager, bloomFilterFalsePositiveRate,
-                fileManager, invListTypeTraits, invListCmpFactories, tokenTypeTraits, tokenCmpFactories,
-                tokenizerFactory, mergePolicy, opTracker, ioScheduler, ioOpCallback, invertedIndexFields, filterFields,
-                filterFieldsForNonBulkLoadOps, invertedIndexFieldsForNonBulkLoadOps, durable);
+        ILSMDiskComponentFactory componentFactory = new LSMInvertedIndexDiskComponentFactory(invIndexFactory,
+                deletedKeysBTreeFactory, bloomFilterFactory, filterHelper);
+
+        return new LSMInvertedIndex(ioManager, virtualBufferCaches, componentFactory, filterHelper, filterFrameFactory,
+                filterManager, bloomFilterFalsePositiveRate, diskBufferCache, fileManager, invListTypeTraits,
+                invListCmpFactories, tokenTypeTraits, tokenCmpFactories, tokenizerFactory, mergePolicy, opTracker,
+                ioScheduler, ioOpCallback, invertedIndexFields, filterFields, filterFieldsForNonBulkLoadOps,
+                invertedIndexFieldsForNonBulkLoadOps, durable);
     }
 
     public static PartitionedLSMInvertedIndex createPartitionedLSMInvertedIndex(IIOManager ioManager,
@@ -203,10 +208,13 @@ public class InvertedIndexUtils {
             filterFrameFactory = new LSMComponentFilterFrameFactory(filterTupleWriterFactory);
             filterManager = new LSMComponentFilterManager(filterFrameFactory);
         }
-        return new PartitionedLSMInvertedIndex(ioManager, virtualBufferCaches, invIndexFactory, deletedKeysBTreeFactory,
-                bloomFilterFactory, filterHelper, filterFrameFactory, filterManager, bloomFilterFalsePositiveRate,
-                fileManager, invListTypeTraits, invListCmpFactories, tokenTypeTraits, tokenCmpFactories,
-                tokenizerFactory, mergePolicy, opTracker, ioScheduler, ioOpCallback, invertedIndexFields, filterFields,
+        ILSMDiskComponentFactory componentFactory = new LSMInvertedIndexDiskComponentFactory(invIndexFactory,
+                deletedKeysBTreeFactory, bloomFilterFactory, filterHelper);
+
+        return new PartitionedLSMInvertedIndex(ioManager, virtualBufferCaches, componentFactory, filterHelper,
+                filterFrameFactory, filterManager, bloomFilterFalsePositiveRate, diskBufferCache, fileManager,
+                invListTypeTraits, invListCmpFactories, tokenTypeTraits, tokenCmpFactories, tokenizerFactory,
+                mergePolicy, opTracker, ioScheduler, ioOpCallback, invertedIndexFields, filterFields,
                 filterFieldsForNonBulkLoadOps, invertedIndexFieldsForNonBulkLoadOps, durable);
     }
 }


[5/6] asterixdb git commit: [NO ISSUE][STO][IDX] LSM storage cleanup

Posted by il...@apache.org.
http://git-wip-us.apache.org/repos/asf/asterixdb/blob/77f89525/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreeBulkLoader.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreeBulkLoader.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreeBulkLoader.java
deleted file mode 100644
index c169d85..0000000
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreeBulkLoader.java
+++ /dev/null
@@ -1,62 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.hyracks.storage.am.lsm.btree.impls;
-
-import org.apache.hyracks.api.exceptions.HyracksDataException;
-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.LSMOperationType;
-import org.apache.hyracks.storage.common.IIndexBulkLoader;
-
-public class LSMBTreeBulkLoader implements IIndexBulkLoader {
-    private final LSMBTree lsmIndex;
-    private final ILSMDiskComponent component;
-    private final IIndexBulkLoader componentBulkLoader;
-
-    public LSMBTreeBulkLoader(LSMBTree lsmIndex, float fillFactor, boolean verifyInput, long numElementsHint)
-            throws HyracksDataException {
-        this.lsmIndex = lsmIndex;
-        this.component = lsmIndex.createBulkLoadTarget();
-        this.componentBulkLoader = lsmIndex.createComponentBulkLoader(component, fillFactor, verifyInput,
-                numElementsHint, false, true, true);
-    }
-
-    @Override
-    public void add(ITupleReference tuple) throws HyracksDataException {
-        componentBulkLoader.add(tuple);
-    }
-
-    @Override
-    public void end() throws HyracksDataException {
-        componentBulkLoader.end();
-        if (component.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(LSMOperationType.FLUSH, null, component);
-            lsmIndex.getLsmHarness().addBulkLoadedComponent(component);
-        }
-    }
-
-    @Override
-    public void abort() throws HyracksDataException {
-        componentBulkLoader.end();
-    }
-
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/77f89525/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreeDiskComponent.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreeDiskComponent.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreeDiskComponent.java
index 4d60ab4..a8e707f 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreeDiskComponent.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreeDiskComponent.java
@@ -18,51 +18,47 @@
  */
 package org.apache.hyracks.storage.am.lsm.btree.impls;
 
-import org.apache.hyracks.api.exceptions.HyracksDataException;
-import org.apache.hyracks.storage.am.bloomfilter.impls.BloomFilter;
+import java.util.HashSet;
+import java.util.Set;
+
 import org.apache.hyracks.storage.am.btree.impls.BTree;
 import org.apache.hyracks.storage.am.common.api.IMetadataPageManager;
 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.impls.AbstractLSMDiskComponent;
-import org.apache.hyracks.storage.am.lsm.common.util.ComponentUtils;
+import org.apache.hyracks.storage.am.lsm.common.impls.AbstractLSMIndex;
 
-public class LSMBTreeDiskComponent extends AbstractLSMDiskComponent {
-    private final BTree btree;
-    private final BloomFilter bloomFilter;
+public class LSMBTreeDiskComponent extends AbstractLSMDiskComponent implements ILSMDiskComponent {
+    protected final BTree btree;
 
-    public LSMBTreeDiskComponent(BTree btree, BloomFilter bloomFilter, ILSMComponentFilter filter) {
-        super((IMetadataPageManager) btree.getPageManager(), filter);
+    public LSMBTreeDiskComponent(AbstractLSMIndex lsmIndex, BTree btree, ILSMComponentFilter filter) {
+        super(lsmIndex, getMetadataPageManager(btree), filter);
         this.btree = btree;
-        this.bloomFilter = bloomFilter;
     }
 
     @Override
-    public void destroy() throws HyracksDataException {
-        btree.deactivate();
-        btree.destroy();
-        if (bloomFilter != null) {
-            bloomFilter.deactivate();
-            bloomFilter.destroy();
-        }
-    }
-
-    public BTree getBTree() {
+    public BTree getIndex() {
         return btree;
     }
 
-    public BloomFilter getBloomFilter() {
-        return bloomFilter;
+    @Override
+    public BTree getMetadataHolder() {
+        return btree;
     }
 
     @Override
     public long getComponentSize() {
-        return btree.getFileReference().getFile().length()
-                + (bloomFilter == null ? 0 : bloomFilter.getFileReference().getFile().length());
+        return getComponentSize(btree);
     }
 
     @Override
     public int getFileReferenceCount() {
-        return btree.getBufferCache().getFileReferenceCount(btree.getFileId());
+        return getFileReferenceCount(btree);
+    }
+
+    @Override
+    public Set<String> getLSMComponentPhysicalFiles() {
+        return getFiles(btree);
     }
 
     @Override
@@ -70,13 +66,21 @@ public class LSMBTreeDiskComponent extends AbstractLSMDiskComponent {
         return getClass().getSimpleName() + ":" + btree.getFileReference().getRelativePath();
     }
 
-    @Override
-    public void markAsValid(boolean persist) throws HyracksDataException {
-        // The order of forcing the dirty page to be flushed is critical.
-        // The bloom filter must be always done first.
-        if (bloomFilter != null && persist) {
-            ComponentUtils.markAsValid(btree.getBufferCache(), bloomFilter, persist);
-        }
-        ComponentUtils.markAsValid(btree, persist);
+    static IMetadataPageManager getMetadataPageManager(BTree btree) {
+        return (IMetadataPageManager) btree.getPageManager();
+    }
+
+    static long getComponentSize(BTree btree) {
+        return btree.getFileReference().getFile().length();
+    }
+
+    static int getFileReferenceCount(BTree btree) {
+        return btree.getBufferCache().getFileReferenceCount(btree.getFileId());
+    }
+
+    static Set<String> getFiles(BTree btree) {
+        Set<String> files = new HashSet<>();
+        files.add(btree.getFileReference().getFile().getAbsolutePath());
+        return files;
     }
 }

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/77f89525/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreeDiskComponentBulkLoader.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreeDiskComponentBulkLoader.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreeDiskComponentBulkLoader.java
deleted file mode 100644
index 43373b5..0000000
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreeDiskComponentBulkLoader.java
+++ /dev/null
@@ -1,59 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.hyracks.storage.am.lsm.btree.impls;
-
-import org.apache.hyracks.api.exceptions.HyracksDataException;
-import org.apache.hyracks.storage.am.bloomfilter.impls.BloomFilter;
-import org.apache.hyracks.storage.am.bloomfilter.impls.BloomFilterSpecification;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponentFilterManager;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMDiskComponent;
-import org.apache.hyracks.storage.am.lsm.common.impls.AbstractLSMDiskComponentBulkLoader;
-import org.apache.hyracks.storage.common.IIndex;
-import org.apache.hyracks.storage.common.MultiComparator;
-
-public class LSMBTreeDiskComponentBulkLoader extends AbstractLSMDiskComponentBulkLoader {
-
-    //with filter
-    public LSMBTreeDiskComponentBulkLoader(LSMBTreeDiskComponent component, BloomFilterSpecification bloomFilterSpec,
-            float fillFactor, boolean verifyInput, long numElementsHint, boolean checkIfEmptyIndex,
-            boolean cleanupEmptyComponent, ILSMComponentFilterManager filterManager, int[] indexFields,
-            int[] filterFields, MultiComparator filterCmp) throws HyracksDataException {
-        super(component, bloomFilterSpec, fillFactor, verifyInput, numElementsHint, checkIfEmptyIndex,
-                cleanupEmptyComponent, filterManager, indexFields, filterFields, filterCmp);
-    }
-
-    //without filter
-    public LSMBTreeDiskComponentBulkLoader(LSMBTreeDiskComponent component, BloomFilterSpecification bloomFilterSpec,
-            float fillFactor, boolean verifyInput, long numElementsHint, boolean checkIfEmptyIndex,
-            boolean cleanupEmptyComponent) throws HyracksDataException {
-        super(component, bloomFilterSpec, fillFactor, verifyInput, numElementsHint, checkIfEmptyIndex,
-                cleanupEmptyComponent, null, null, null, null);
-    }
-
-    @Override
-    protected BloomFilter getBloomFilter(ILSMDiskComponent component) {
-        return ((LSMBTreeDiskComponent) component).getBloomFilter();
-    }
-
-    @Override
-    protected IIndex getIndex(ILSMDiskComponent component) {
-        return ((LSMBTreeDiskComponent) component).getBTree();
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/77f89525/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreeDiskComponentFactory.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreeDiskComponentFactory.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreeDiskComponentFactory.java
index 399904f..0e54f53 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreeDiskComponentFactory.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreeDiskComponentFactory.java
@@ -20,34 +20,27 @@
 package org.apache.hyracks.storage.am.lsm.btree.impls;
 
 import org.apache.hyracks.api.exceptions.HyracksDataException;
-import org.apache.hyracks.storage.am.bloomfilter.impls.BloomFilterFactory;
 import org.apache.hyracks.storage.am.btree.impls.BTree;
 import org.apache.hyracks.storage.am.lsm.common.api.IComponentFilterHelper;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMDiskComponentFactory;
+import org.apache.hyracks.storage.am.lsm.common.impls.AbstractLSMIndex;
 import org.apache.hyracks.storage.am.lsm.common.impls.LSMComponentFileReferences;
 import org.apache.hyracks.storage.am.lsm.common.impls.TreeIndexFactory;
 
 public class LSMBTreeDiskComponentFactory implements ILSMDiskComponentFactory {
-    private final TreeIndexFactory<BTree> btreeFactory;
-    private final BloomFilterFactory bloomFilterFactory;
-    private final IComponentFilterHelper filterHelper;
+    protected final TreeIndexFactory<BTree> btreeFactory;
+    protected final IComponentFilterHelper filterHelper;
 
-    public LSMBTreeDiskComponentFactory(TreeIndexFactory<BTree> btreeFactory, BloomFilterFactory bloomFilterFactory,
-            IComponentFilterHelper filterHelper) {
+    public LSMBTreeDiskComponentFactory(TreeIndexFactory<BTree> btreeFactory, IComponentFilterHelper filterHelper) {
         this.btreeFactory = btreeFactory;
-        this.bloomFilterFactory = bloomFilterFactory;
         this.filterHelper = filterHelper;
     }
 
     @Override
-    public LSMBTreeDiskComponent createComponent(LSMComponentFileReferences cfr) throws HyracksDataException {
-        return new LSMBTreeDiskComponent(btreeFactory.createIndexInstance(cfr.getInsertIndexFileReference()),
-                bloomFilterFactory == null ? null
-                        : bloomFilterFactory.createBloomFiltertInstance(cfr.getBloomFilterFileReference()),
+    public LSMBTreeDiskComponent createComponent(AbstractLSMIndex lsmIndex, LSMComponentFileReferences cfr)
+            throws HyracksDataException {
+        return new LSMBTreeDiskComponent(lsmIndex, btreeFactory.createIndexInstance(cfr.getInsertIndexFileReference()),
                 filterHelper == null ? null : filterHelper.createFilter());
     }
 
-    public int[] getBloomFilterKeyFields() {
-        return bloomFilterFactory == null ? null : bloomFilterFactory.getBloomFilterKeyFields();
-    }
 }

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/77f89525/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreeDiskComponentScanCursor.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreeDiskComponentScanCursor.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreeDiskComponentScanCursor.java
index ab9b5af..02e481c 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreeDiskComponentScanCursor.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreeDiskComponentScanCursor.java
@@ -69,7 +69,6 @@ public class LSMBTreeDiskComponentScanCursor extends LSMIndexSearchCursor {
         operationalComponents = lsmInitialState.getOperationalComponents();
         lsmHarness = lsmInitialState.getLSMHarness();
         includeMutableComponent = false;
-
         int numBTrees = operationalComponents.size();
         rangeCursors = new IIndexCursor[numBTrees];
         btreeAccessors = new BTreeAccessor[numBTrees];
@@ -77,10 +76,9 @@ public class LSMBTreeDiskComponentScanCursor extends LSMIndexSearchCursor {
             ILSMComponent component = operationalComponents.get(i);
             IBTreeLeafFrame leafFrame = (IBTreeLeafFrame) lsmInitialState.getLeafFrameFactory().createFrame();
             rangeCursors[i] = new BTreeRangeSearchCursor(leafFrame, false);
-            BTree btree = ((LSMBTreeDiskComponent) component).getBTree();
+            BTree btree = (BTree) component.getIndex();
 
-            btreeAccessors[i] = (BTreeAccessor) btree.createAccessor(NoOpOperationCallback.INSTANCE,
-                    NoOpOperationCallback.INSTANCE);
+            btreeAccessors[i] = btree.createAccessor(NoOpOperationCallback.INSTANCE, NoOpOperationCallback.INSTANCE);
             btreeAccessors[i].search(rangeCursors[i], searchPred);
         }
 

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/77f89525/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreeMemoryComponent.java
----------------------------------------------------------------------
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 32b93bb..b334d80 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
@@ -19,7 +19,6 @@
 
 package org.apache.hyracks.storage.am.lsm.btree.impls;
 
-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.ILSMComponentFilter;
 import org.apache.hyracks.storage.am.lsm.common.api.IVirtualBufferCache;
@@ -29,23 +28,13 @@ public class LSMBTreeMemoryComponent extends AbstractLSMMemoryComponent {
 
     private final BTree btree;
 
-    public LSMBTreeMemoryComponent(BTree btree, IVirtualBufferCache vbc, boolean isActive,
-            ILSMComponentFilter filter) {
+    public LSMBTreeMemoryComponent(BTree btree, IVirtualBufferCache vbc, boolean isActive, ILSMComponentFilter filter) {
         super(vbc, isActive, filter);
         this.btree = btree;
     }
 
-    public BTree getBTree() {
-        return btree;
-    }
-
     @Override
-    public void reset() throws HyracksDataException {
-        super.reset();
-        btree.deactivate();
-        btree.destroy();
-        btree.create();
-        btree.activate();
+    public BTree getIndex() {
+        return btree;
     }
-
 }

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/77f89525/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreeOpContext.java
----------------------------------------------------------------------
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 86502a7..08ba7af 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
@@ -66,29 +66,29 @@ public final class LSMBTreeOpContext extends AbstractLSMIndexOperationContext {
             ILSMHarness lsmHarness, IBinaryComparatorFactory[] filterCmpFactories) {
         super(btreeFields, filterFields, filterCmpFactories, searchCallback, modificationCallback);
         LSMBTreeMemoryComponent c = (LSMBTreeMemoryComponent) mutableComponents.get(0);
-        IBinaryComparatorFactory cmpFactories[] = c.getBTree().getComparatorFactories();
+        IBinaryComparatorFactory cmpFactories[] = c.getIndex().getComparatorFactories();
         if (cmpFactories[0] != null) {
-            this.cmp = MultiComparator.create(c.getBTree().getComparatorFactories());
+            this.cmp = MultiComparator.create(c.getIndex().getComparatorFactories());
         } else {
             this.cmp = null;
         }
 
         bloomFilterCmp = numBloomFilterKeyFields == 0 ? null
-                : MultiComparator.create(c.getBTree().getComparatorFactories(), 0, numBloomFilterKeyFields);
+                : MultiComparator.create(c.getIndex().getComparatorFactories(), 0, numBloomFilterKeyFields);
 
         mutableBTrees = new BTree[mutableComponents.size()];
         mutableBTreeAccessors = new BTree.BTreeAccessor[mutableComponents.size()];
         mutableBTreeOpCtxs = new BTreeOpContext[mutableComponents.size()];
         for (int i = 0; i < mutableComponents.size(); i++) {
             LSMBTreeMemoryComponent mutableComponent = (LSMBTreeMemoryComponent) mutableComponents.get(i);
-            mutableBTrees[i] = mutableComponent.getBTree();
+            mutableBTrees[i] = mutableComponent.getIndex();
             if (allFields != null) {
-                mutableBTreeAccessors[i] = (BTree.BTreeAccessor) mutableBTrees[i].createAccessor(modificationCallback,
+                mutableBTreeAccessors[i] = mutableBTrees[i].createAccessor(modificationCallback,
                         NoOpOperationCallback.INSTANCE, allFields);
             } else {
 
-                mutableBTreeAccessors[i] = (BTree.BTreeAccessor) mutableBTrees[i].createAccessor(modificationCallback,
-                        NoOpOperationCallback.INSTANCE);
+                mutableBTreeAccessors[i] =
+                        mutableBTrees[i].createAccessor(modificationCallback, NoOpOperationCallback.INSTANCE);
             }
             mutableBTreeOpCtxs[i] = mutableBTreeAccessors[i].getOpContext();
         }

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/77f89525/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreePointSearchCursor.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreePointSearchCursor.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreePointSearchCursor.java
index 6126c53..c178b51 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreePointSearchCursor.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreePointSearchCursor.java
@@ -178,24 +178,24 @@ public class LSMBTreePointSearchCursor implements ITreeIndexCursor {
                     // reset
                     rangeCursors[i].reset();
                 }
-                btree = ((LSMBTreeMemoryComponent) component).getBTree();
+                btree = ((LSMBTreeMemoryComponent) component).getIndex();
             } else {
                 if (rangeCursors[i] != null && rangeCursors[i].isBloomFilterAware()) {
                     // can re-use cursor
                     ((BloomFilterAwareBTreePointSearchCursor) rangeCursors[i])
-                            .resetBloomFilter(((LSMBTreeDiskComponent) component).getBloomFilter());
+                            .resetBloomFilter(((LSMBTreeWithBloomFilterDiskComponent) component).getBloomFilter());
                     rangeCursors[i].reset();
                 } else {
                     // create new cursor <should be relatively rare>
                     IBTreeLeafFrame leafFrame = (IBTreeLeafFrame) lsmInitialState.getLeafFrameFactory().createFrame();
                     rangeCursors[i] = new BloomFilterAwareBTreePointSearchCursor(leafFrame, false,
-                            ((LSMBTreeDiskComponent) component).getBloomFilter());
+                            ((LSMBTreeWithBloomFilterDiskComponent) component).getBloomFilter());
                 }
-                btree = ((LSMBTreeDiskComponent) component).getBTree();
+                btree = (BTree) component.getIndex();
             }
             if (btreeAccessors[i] == null) {
-                btreeAccessors[i] = (BTreeAccessor) btree.createAccessor(NoOpOperationCallback.INSTANCE,
-                        NoOpOperationCallback.INSTANCE);
+                btreeAccessors[i] =
+                        btree.createAccessor(NoOpOperationCallback.INSTANCE, NoOpOperationCallback.INSTANCE);
             } else {
                 // re-use
                 btreeAccessors[i].reset(btree, NoOpOperationCallback.INSTANCE, NoOpOperationCallback.INSTANCE);

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/77f89525/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreeRangeSearchCursor.java
----------------------------------------------------------------------
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 29d7c60..7f93b62 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
@@ -231,14 +231,14 @@ public class LSMBTreeRangeSearchCursor extends LSMIndexSearchCursor {
             }
             if (component.getType() == LSMComponentType.MEMORY) {
                 includeMutableComponent = true;
-                btree = ((LSMBTreeMemoryComponent) component).getBTree();
+                btree = (BTree) component.getIndex();
             } else {
-                btree = ((LSMBTreeDiskComponent) component).getBTree();
+                btree = (BTree) component.getIndex();
             }
 
             if (btreeAccessors[i] == null) {
-                btreeAccessors[i] = (BTreeAccessor) btree.createAccessor(NoOpOperationCallback.INSTANCE,
-                        NoOpOperationCallback.INSTANCE);
+                btreeAccessors[i] =
+                        btree.createAccessor(NoOpOperationCallback.INSTANCE, NoOpOperationCallback.INSTANCE);
             } else {
                 // re-use
                 btreeAccessors[i].reset(btree, NoOpOperationCallback.INSTANCE, NoOpOperationCallback.INSTANCE);

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/77f89525/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreeWithBloomFilterDiskComponent.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreeWithBloomFilterDiskComponent.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreeWithBloomFilterDiskComponent.java
new file mode 100644
index 0000000..fa17549
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreeWithBloomFilterDiskComponent.java
@@ -0,0 +1,86 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.hyracks.storage.am.lsm.btree.impls;
+
+import java.util.Set;
+
+import org.apache.hyracks.storage.am.bloomfilter.impls.BloomFilter;
+import org.apache.hyracks.storage.am.btree.impls.BTree;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponentFilter;
+import org.apache.hyracks.storage.am.lsm.common.api.AbstractLSMWithBloomFilterDiskComponent;
+import org.apache.hyracks.storage.am.lsm.common.impls.AbstractLSMIndex;
+import org.apache.hyracks.storage.common.buffercache.IBufferCache;
+
+public class LSMBTreeWithBloomFilterDiskComponent extends AbstractLSMWithBloomFilterDiskComponent {
+
+    private final BTree btree;
+    private final BloomFilter bloomFilter;
+
+    public LSMBTreeWithBloomFilterDiskComponent(AbstractLSMIndex lsmIndex, BTree btree, BloomFilter bloomFilter,
+            ILSMComponentFilter filter) {
+        super(lsmIndex, LSMBTreeDiskComponent.getMetadataPageManager(btree), filter);
+        this.btree = btree;
+        this.bloomFilter = bloomFilter;
+    }
+
+    @Override
+    public BloomFilter getBloomFilter() {
+        return bloomFilter;
+    }
+
+    @Override
+    public IBufferCache getBloomFilterBufferCache() {
+        return getMetadataHolder().getBufferCache();
+    }
+
+    @Override
+    public long getComponentSize() {
+        return LSMBTreeDiskComponent.getComponentSize(btree) + getComponentSize(bloomFilter);
+    }
+
+    @Override
+    public Set<String> getLSMComponentPhysicalFiles() {
+        Set<String> files = LSMBTreeDiskComponent.getFiles(btree);
+        addFiles(files, bloomFilter);
+        return files;
+    }
+
+    static void addFiles(Set<String> files, BloomFilter bloomFilter) {
+        files.add(bloomFilter.getFileReference().getFile().getAbsolutePath());
+    }
+
+    @Override
+    public int getFileReferenceCount() {
+        return LSMBTreeDiskComponent.getFileReferenceCount(btree);
+    }
+
+    @Override
+    public BTree getMetadataHolder() {
+        return btree;
+    }
+
+    @Override
+    public BTree getIndex() {
+        return btree;
+    }
+
+    static long getComponentSize(BloomFilter bloomFilter) {
+        return bloomFilter.getFileReference().getFile().length();
+    }
+}

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/77f89525/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreeWithBloomFilterDiskComponentFactory.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreeWithBloomFilterDiskComponentFactory.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreeWithBloomFilterDiskComponentFactory.java
new file mode 100644
index 0000000..09e83cc
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreeWithBloomFilterDiskComponentFactory.java
@@ -0,0 +1,54 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.hyracks.storage.am.lsm.btree.impls;
+
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.storage.am.bloomfilter.impls.BloomFilterFactory;
+import org.apache.hyracks.storage.am.btree.impls.BTree;
+import org.apache.hyracks.storage.am.lsm.common.api.IComponentFilterHelper;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMDiskComponentFactory;
+import org.apache.hyracks.storage.am.lsm.common.impls.AbstractLSMIndex;
+import org.apache.hyracks.storage.am.lsm.common.impls.LSMComponentFileReferences;
+import org.apache.hyracks.storage.am.lsm.common.impls.TreeIndexFactory;
+
+public class LSMBTreeWithBloomFilterDiskComponentFactory implements ILSMDiskComponentFactory {
+    protected final TreeIndexFactory<BTree> btreeFactory;
+    protected final IComponentFilterHelper filterHelper;
+    protected final BloomFilterFactory bloomFilterFactory;
+
+    public LSMBTreeWithBloomFilterDiskComponentFactory(TreeIndexFactory<BTree> btreeFactory,
+            BloomFilterFactory bloomFilterFactory, IComponentFilterHelper filterHelper) {
+        this.btreeFactory = btreeFactory;
+        this.filterHelper = filterHelper;
+        this.bloomFilterFactory = bloomFilterFactory;
+    }
+
+    @Override
+    public LSMBTreeWithBloomFilterDiskComponent createComponent(AbstractLSMIndex lsmIndex,
+            LSMComponentFileReferences cfr) throws HyracksDataException {
+        return new LSMBTreeWithBloomFilterDiskComponent(lsmIndex,
+                btreeFactory.createIndexInstance(cfr.getInsertIndexFileReference()),
+                bloomFilterFactory.createBloomFiltertInstance(cfr.getBloomFilterFileReference()),
+                filterHelper == null ? null : filterHelper.createFilter());
+    }
+
+    public int[] getBloomFilterKeyFields() {
+        return bloomFilterFactory == null ? null : bloomFilterFactory.getBloomFilterKeyFields();
+    }
+}

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/77f89525/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreeWithBuddyAbstractCursor.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreeWithBuddyAbstractCursor.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreeWithBuddyAbstractCursor.java
index 03d28d0..5122e43 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreeWithBuddyAbstractCursor.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreeWithBuddyAbstractCursor.java
@@ -33,6 +33,7 @@ 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.ILSMHarness;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndexOperationContext;
+import org.apache.hyracks.storage.am.lsm.common.api.AbstractLSMWithBuddyDiskComponent;
 import org.apache.hyracks.storage.am.lsm.common.impls.BloomFilterAwareBTreePointSearchCursor;
 import org.apache.hyracks.storage.common.ICursorInitialState;
 import org.apache.hyracks.storage.common.ISearchPredicate;
@@ -104,26 +105,26 @@ public abstract class LSMBTreeWithBuddyAbstractCursor implements ITreeIndexCurso
                 } else {
                     buddyBtreeCursors[i].reset();
                 }
-                btree = ((LSMBTreeWithBuddyMemoryComponent) component).getBTree();
-                buddyBtree = ((LSMBTreeWithBuddyMemoryComponent) component).getBuddyBTree();
+                btree = ((LSMBTreeWithBuddyMemoryComponent) component).getIndex();
+                buddyBtree = ((LSMBTreeWithBuddyMemoryComponent) component).getBuddyIndex();
             } else {
                 if (buddyBtreeCursors[i] == null || !buddyBtreeCursors[i].isBloomFilterAware()) {
                     buddyBtreeCursors[i] = new BloomFilterAwareBTreePointSearchCursor(
                             (IBTreeLeafFrame) lsmInitialState.getBuddyBTreeLeafFrameFactory().createFrame(), false,
-                            ((LSMBTreeWithBuddyDiskComponent) operationalComponents.get(i)).getBloomFilter());
+                            ((AbstractLSMWithBuddyDiskComponent) operationalComponents.get(i)).getBloomFilter());
                 } else {
                     buddyBtreeCursors[i].reset();
                 }
-                btree = ((LSMBTreeWithBuddyDiskComponent) component).getBTree();
-                buddyBtree = ((LSMBTreeWithBuddyDiskComponent) component).getBuddyBTree();
+                btree = (BTree) component.getIndex();
+                buddyBtree = (BTree) ((AbstractLSMWithBuddyDiskComponent) component).getBuddyIndex();
             }
             IBTreeLeafFrame leafFrame = (IBTreeLeafFrame) lsmInitialState.getBTreeLeafFrameFactory().createFrame();
             if (btreeAccessors[i] == null) {
                 btreeCursors[i] = new BTreeRangeSearchCursor(leafFrame, false);
-                btreeAccessors[i] = (BTreeAccessor) btree.createAccessor(NoOpOperationCallback.INSTANCE,
-                        NoOpOperationCallback.INSTANCE);
-                buddyBtreeAccessors[i] = (BTreeAccessor) buddyBtree.createAccessor(NoOpOperationCallback.INSTANCE,
-                        NoOpOperationCallback.INSTANCE);
+                btreeAccessors[i] =
+                        btree.createAccessor(NoOpOperationCallback.INSTANCE, NoOpOperationCallback.INSTANCE);
+                buddyBtreeAccessors[i] =
+                        buddyBtree.createAccessor(NoOpOperationCallback.INSTANCE, NoOpOperationCallback.INSTANCE);
             } else {
                 btreeCursors[i].reset();
                 btreeAccessors[i].reset(btree, NoOpOperationCallback.INSTANCE, NoOpOperationCallback.INSTANCE);

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/77f89525/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreeWithBuddyDiskComponent.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreeWithBuddyDiskComponent.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreeWithBuddyDiskComponent.java
index 4aee950..1e9dade 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreeWithBuddyDiskComponent.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreeWithBuddyDiskComponent.java
@@ -18,72 +18,76 @@
  */
 package org.apache.hyracks.storage.am.lsm.btree.impls;
 
+import java.util.Set;
+
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.storage.am.bloomfilter.impls.BloomFilter;
 import org.apache.hyracks.storage.am.btree.impls.BTree;
-import org.apache.hyracks.storage.am.common.api.IMetadataPageManager;
+import org.apache.hyracks.storage.am.lsm.common.api.AbstractLSMWithBuddyDiskComponent;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponentFilter;
-import org.apache.hyracks.storage.am.lsm.common.impls.AbstractLSMDiskComponent;
-import org.apache.hyracks.storage.am.lsm.common.util.ComponentUtils;
-
-public class LSMBTreeWithBuddyDiskComponent extends AbstractLSMDiskComponent {
+import org.apache.hyracks.storage.am.lsm.common.impls.AbstractLSMIndex;
+import org.apache.hyracks.storage.common.buffercache.IBufferCache;
 
+public class LSMBTreeWithBuddyDiskComponent extends AbstractLSMWithBuddyDiskComponent {
     private final BTree btree;
-    private final BTree buddyBtree;
     private final BloomFilter bloomFilter;
+    private final BTree buddyBtree;
 
-    public LSMBTreeWithBuddyDiskComponent(BTree btree, BTree buddyBtree, BloomFilter bloomFilter,
-            ILSMComponentFilter filter) {
-        super((IMetadataPageManager) btree.getPageManager(), filter);
+    public LSMBTreeWithBuddyDiskComponent(AbstractLSMIndex lsmIndex, BTree btree, BTree buddyBtree,
+            BloomFilter bloomFilter, ILSMComponentFilter filter) {
+        super(lsmIndex, LSMBTreeDiskComponent.getMetadataPageManager(btree), filter);
         this.btree = btree;
-        this.buddyBtree = buddyBtree;
         this.bloomFilter = bloomFilter;
+        this.buddyBtree = buddyBtree;
     }
 
     @Override
-    public void destroy() throws HyracksDataException {
-        btree.deactivate();
-        btree.destroy();
-        buddyBtree.deactivate();
-        buddyBtree.destroy();
-        bloomFilter.deactivate();
-        bloomFilter.destroy();
+    public BTree getBuddyIndex() {
+        return buddyBtree;
     }
 
-    public BTree getBTree() {
-        return btree;
+    @Override
+    public long getComponentSize() {
+        return LSMBTreeDiskComponent.getComponentSize(btree)
+                + LSMBTreeWithBloomFilterDiskComponent.getComponentSize(bloomFilter)
+                + buddyBtree.getFileReference().getFile().length();
     }
 
-    public BTree getBuddyBTree() {
-        return buddyBtree;
+    @Override
+    public Set<String> getLSMComponentPhysicalFiles() {
+        Set<String> files = LSMBTreeDiskComponent.getFiles(btree);
+        LSMBTreeWithBloomFilterDiskComponent.addFiles(files, bloomFilter);
+        files.add(buddyBtree.getFileReference().getFile().getAbsolutePath());
+        return files;
     }
 
-    public BloomFilter getBloomFilter() {
-        return bloomFilter;
+    @Override
+    public void validate() throws HyracksDataException {
+        throw new UnsupportedOperationException("Validation not implemented for LSM B-Trees with Buddy B-Tree.");
     }
 
     @Override
-    public long getComponentSize() {
-        long size = btree.getFileReference().getFile().length();
-        size += buddyBtree.getFileReference().getFile().length();
-        size += bloomFilter.getFileReference().getFile().length();
-        return size;
+    public int getFileReferenceCount() {
+        return LSMBTreeDiskComponent.getFileReferenceCount(btree);
     }
 
     @Override
-    public int getFileReferenceCount() {
-        return btree.getBufferCache().getFileReferenceCount(btree.getFileId());
+    public BTree getMetadataHolder() {
+        return btree;
     }
 
     @Override
-    public String toString() {
-        return getClass().getSimpleName() + ":" + btree.getFileReference().getRelativePath();
+    public BTree getIndex() {
+        return btree;
+    }
+
+    @Override
+    public BloomFilter getBloomFilter() {
+        return bloomFilter;
     }
 
     @Override
-    public void markAsValid(boolean persist) throws HyracksDataException {
-        ComponentUtils.markAsValid(btree.getBufferCache(), bloomFilter, persist);
-        ComponentUtils.markAsValid(btree, persist);
-        ComponentUtils.markAsValid(buddyBtree, persist);
+    public IBufferCache getBloomFilterBufferCache() {
+        return getMetadataHolder().getBufferCache();
     }
 }

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/77f89525/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreeWithBuddyDiskComponentBulkLoader.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreeWithBuddyDiskComponentBulkLoader.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreeWithBuddyDiskComponentBulkLoader.java
deleted file mode 100644
index b3e7c6c..0000000
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreeWithBuddyDiskComponentBulkLoader.java
+++ /dev/null
@@ -1,65 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.hyracks.storage.am.lsm.btree.impls;
-
-import org.apache.hyracks.api.exceptions.HyracksDataException;
-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.ITreeIndex;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponentFilterManager;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMDiskComponent;
-import org.apache.hyracks.storage.am.lsm.common.impls.AbstractLSMDiskComponentWithBuddyBulkLoader;
-import org.apache.hyracks.storage.common.IIndex;
-import org.apache.hyracks.storage.common.MultiComparator;
-
-public class LSMBTreeWithBuddyDiskComponentBulkLoader extends AbstractLSMDiskComponentWithBuddyBulkLoader {
-
-    //with filter
-    public LSMBTreeWithBuddyDiskComponentBulkLoader(LSMBTreeWithBuddyDiskComponent component,
-            BloomFilterSpecification bloomFilterSpec, float fillFactor, boolean verifyInput, long numElementsHint,
-            boolean checkIfEmptyIndex, boolean cleanupEmptyComponent, ILSMComponentFilterManager filterManager,
-            int[] indexFields, int[] filterFields, MultiComparator filterCmp) throws HyracksDataException {
-        super(component, bloomFilterSpec, fillFactor, verifyInput, numElementsHint, checkIfEmptyIndex,
-                cleanupEmptyComponent, filterManager, indexFields, filterFields, filterCmp);
-    }
-
-    //without filter
-    public LSMBTreeWithBuddyDiskComponentBulkLoader(LSMBTreeWithBuddyDiskComponent component,
-            BloomFilterSpecification bloomFilterSpec, float fillFactor, boolean verifyInput, long numElementsHint,
-            boolean checkIfEmptyIndex, boolean cleanupEmptyComponent) throws HyracksDataException {
-        super(component, bloomFilterSpec, fillFactor, verifyInput, numElementsHint, checkIfEmptyIndex,
-                cleanupEmptyComponent, null, null, null, null);
-    }
-
-    @Override
-    protected BloomFilter getBloomFilter(ILSMDiskComponent component) {
-        return ((LSMBTreeWithBuddyDiskComponent) component).getBloomFilter();
-    }
-
-    @Override
-    protected IIndex getIndex(ILSMDiskComponent component) {
-        return ((LSMBTreeWithBuddyDiskComponent) component).getBTree();
-    }
-
-    @Override
-    protected ITreeIndex getBuddyBTree(ILSMDiskComponent component) {
-        return ((LSMBTreeWithBuddyDiskComponent) component).getBuddyBTree();
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/77f89525/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreeWithBuddyDiskComponentFactory.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreeWithBuddyDiskComponentFactory.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreeWithBuddyDiskComponentFactory.java
index 4223954..bfa2314 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreeWithBuddyDiskComponentFactory.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreeWithBuddyDiskComponentFactory.java
@@ -21,27 +21,39 @@ package org.apache.hyracks.storage.am.lsm.btree.impls;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.storage.am.bloomfilter.impls.BloomFilterFactory;
 import org.apache.hyracks.storage.am.btree.impls.BTree;
+import org.apache.hyracks.storage.am.lsm.common.api.IComponentFilterHelper;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMDiskComponentFactory;
+import org.apache.hyracks.storage.am.lsm.common.impls.AbstractLSMIndex;
 import org.apache.hyracks.storage.am.lsm.common.impls.LSMComponentFileReferences;
 import org.apache.hyracks.storage.am.lsm.common.impls.TreeIndexFactory;
 
 public class LSMBTreeWithBuddyDiskComponentFactory implements ILSMDiskComponentFactory {
 
-    private final TreeIndexFactory<BTree> btreeFactory;
-    private final TreeIndexFactory<BTree> buddyBtreeFactory;
-    private final BloomFilterFactory bloomFilterFactory;
+    protected final TreeIndexFactory<BTree> btreeFactory;
+    protected final IComponentFilterHelper filterHelper;
+    protected final BloomFilterFactory bloomFilterFactory;
+    protected final TreeIndexFactory<BTree> buddyBtreeFactory;
 
     public LSMBTreeWithBuddyDiskComponentFactory(TreeIndexFactory<BTree> btreeFactory,
-            TreeIndexFactory<BTree> buddyBtreeFactory, BloomFilterFactory bloomFilterFactory) {
+            TreeIndexFactory<BTree> buddyBtreeFactory, BloomFilterFactory bloomFilterFactory,
+            IComponentFilterHelper filterHelper) {
         this.btreeFactory = btreeFactory;
-        this.buddyBtreeFactory = buddyBtreeFactory;
+        this.filterHelper = filterHelper;
         this.bloomFilterFactory = bloomFilterFactory;
+        this.buddyBtreeFactory = buddyBtreeFactory;
+    }
+
+    public int[] getBloomFilterKeyFields() {
+        return bloomFilterFactory == null ? null : bloomFilterFactory.getBloomFilterKeyFields();
     }
 
     @Override
-    public LSMBTreeWithBuddyDiskComponent createComponent(LSMComponentFileReferences cfr) throws HyracksDataException {
-        return new LSMBTreeWithBuddyDiskComponent(btreeFactory.createIndexInstance(cfr.getInsertIndexFileReference()),
+    public LSMBTreeWithBuddyDiskComponent createComponent(AbstractLSMIndex lsmIndex, LSMComponentFileReferences cfr)
+            throws HyracksDataException {
+        return new LSMBTreeWithBuddyDiskComponent(lsmIndex,
+                btreeFactory.createIndexInstance(cfr.getInsertIndexFileReference()),
                 buddyBtreeFactory.createIndexInstance(cfr.getDeleteIndexFileReference()),
-                bloomFilterFactory.createBloomFiltertInstance(cfr.getBloomFilterFileReference()), null);
+                bloomFilterFactory.createBloomFiltertInstance(cfr.getBloomFilterFileReference()),
+                filterHelper == null ? null : filterHelper.createFilter());
     }
 }

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/77f89525/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreeWithBuddyMemoryComponent.java
----------------------------------------------------------------------
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 f58c737..828b58e 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
@@ -21,13 +21,13 @@ package org.apache.hyracks.storage.am.lsm.btree.impls;
 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.ILSMComponentFilter;
+import org.apache.hyracks.storage.am.lsm.common.api.AbstractLSMWithBuddyMemoryComponent;
 import org.apache.hyracks.storage.am.lsm.common.api.IVirtualBufferCache;
-import org.apache.hyracks.storage.am.lsm.common.impls.AbstractLSMMemoryComponent;
 
 /*
  * This class is also not needed at the moment but is implemented anyway
  */
-public class LSMBTreeWithBuddyMemoryComponent extends AbstractLSMMemoryComponent {
+public class LSMBTreeWithBuddyMemoryComponent extends AbstractLSMWithBuddyMemoryComponent {
 
     private final BTree btree;
     private final BTree buddyBtree;
@@ -39,25 +39,23 @@ public class LSMBTreeWithBuddyMemoryComponent extends AbstractLSMMemoryComponent
         this.buddyBtree = buddyBtree;
     }
 
-    public BTree getBTree() {
+    @Override
+    public BTree getIndex() {
         return btree;
     }
 
-    public BTree getBuddyBTree() {
+    @Override
+    public BTree getBuddyIndex() {
         return buddyBtree;
     }
 
     @Override
-    public void reset() throws HyracksDataException {
-        super.reset();
-        btree.deactivate();
-        btree.destroy();
-        btree.create();
-        btree.activate();
-        buddyBtree.deactivate();
-        buddyBtree.destroy();
-        buddyBtree.create();
-        buddyBtree.activate();
+    public void validate() throws HyracksDataException {
+        throw new UnsupportedOperationException("Validation not implemented for LSM B-Trees with Buddy B-Tree.");
     }
 
+    @Override
+    public long getSize() {
+        return 0L;
+    }
 }

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/77f89525/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBuddyBTreeMergeCursor.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBuddyBTreeMergeCursor.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBuddyBTreeMergeCursor.java
index 79d6b0f..061cad5 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBuddyBTreeMergeCursor.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBuddyBTreeMergeCursor.java
@@ -62,7 +62,7 @@ public class LSMBuddyBTreeMergeCursor extends LSMIndexSearchCursor {
             ILSMComponent component = operationalComponents.get(i);
             IBTreeLeafFrame leafFrame = (IBTreeLeafFrame) lsmInitialState.getBuddyBTreeLeafFrameFactory().createFrame();
             rangeCursors[i] = new BTreeRangeSearchCursor(leafFrame, false);
-            BTree buddyBtree = ((LSMBTreeWithBuddyDiskComponent) component).getBuddyBTree();
+            BTree buddyBtree = ((LSMBTreeWithBuddyDiskComponent) component).getBuddyIndex();
             btreeAccessors[i] =
                     buddyBtree.createAccessor(NoOpOperationCallback.INSTANCE, NoOpOperationCallback.INSTANCE);
             btreeAccessors[i].search(rangeCursors[i], btreePredicate);

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/77f89525/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/utils/LSMBTreeUtil.java
----------------------------------------------------------------------
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 4aafbe9..2c4e6b1 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
@@ -37,10 +37,14 @@ import org.apache.hyracks.storage.am.common.tuples.TypeAwareTupleWriterFactory;
 import org.apache.hyracks.storage.am.lsm.btree.impls.ExternalBTree;
 import org.apache.hyracks.storage.am.lsm.btree.impls.ExternalBTreeWithBuddy;
 import org.apache.hyracks.storage.am.lsm.btree.impls.LSMBTree;
+import org.apache.hyracks.storage.am.lsm.btree.impls.LSMBTreeDiskComponentFactory;
 import org.apache.hyracks.storage.am.lsm.btree.impls.LSMBTreeFileManager;
+import org.apache.hyracks.storage.am.lsm.btree.impls.LSMBTreeWithBloomFilterDiskComponentFactory;
+import org.apache.hyracks.storage.am.lsm.btree.impls.LSMBTreeWithBuddyDiskComponentFactory;
 import org.apache.hyracks.storage.am.lsm.btree.impls.LSMBTreeWithBuddyFileManager;
 import org.apache.hyracks.storage.am.lsm.btree.tuples.LSMBTreeCopyTupleWriterFactory;
 import org.apache.hyracks.storage.am.lsm.btree.tuples.LSMBTreeTupleWriterFactory;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMDiskComponentFactory;
 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.ILSMIndexFileManager;
@@ -89,9 +93,6 @@ public class LSMBTreeUtil {
                 new BTreeFactory(ioManager, diskBufferCache, freePageManagerFactory, interiorFrameFactory,
                         bulkLoadLeafFrameFactory, cmpFactories, typeTraits.length);
 
-        BloomFilterFactory bloomFilterFactory =
-                needKeyDupCheck ? new BloomFilterFactory(diskBufferCache, bloomFilterKeyFields) : null;
-
         ComponentFilterHelper filterHelper = null;
         LSMComponentFilterFrameFactory filterFrameFactory = null;
         LSMComponentFilterManager filterManager = null;
@@ -106,8 +107,21 @@ public class LSMBTreeUtil {
         ILSMIndexFileManager fileNameManager =
                 new LSMBTreeFileManager(ioManager, file, diskBTreeFactory, needKeyDupCheck);
 
+        ILSMDiskComponentFactory componentFactory;
+        ILSMDiskComponentFactory bulkLoadComponentFactory;
+        if (needKeyDupCheck) {
+            BloomFilterFactory bloomFilterFactory = new BloomFilterFactory(diskBufferCache, bloomFilterKeyFields);
+            componentFactory =
+                    new LSMBTreeWithBloomFilterDiskComponentFactory(diskBTreeFactory, bloomFilterFactory, filterHelper);
+            bulkLoadComponentFactory = new LSMBTreeWithBloomFilterDiskComponentFactory(bulkLoadBTreeFactory,
+                    bloomFilterFactory, filterHelper);
+        } else {
+            componentFactory = new LSMBTreeDiskComponentFactory(diskBTreeFactory, filterHelper);
+            bulkLoadComponentFactory = new LSMBTreeDiskComponentFactory(bulkLoadBTreeFactory, filterHelper);
+        }
+
         return new LSMBTree(ioManager, virtualBufferCaches, interiorFrameFactory, insertLeafFrameFactory,
-                deleteLeafFrameFactory, fileNameManager, diskBTreeFactory, bulkLoadBTreeFactory, bloomFilterFactory,
+                deleteLeafFrameFactory, diskBufferCache, fileNameManager, componentFactory, bulkLoadComponentFactory,
                 filterHelper, filterFrameFactory, filterManager, bloomFilterFalsePositiveRate, typeTraits.length,
                 cmpFactories, mergePolicy, opTracker, ioScheduler, ioOpCallback, needKeyDupCheck, btreeFields,
                 filterFields, durable, updateAware, tracer);
@@ -149,11 +163,19 @@ public class LSMBTreeUtil {
                         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,
-                fileNameManager, diskBTreeFactory, bulkLoadBTreeFactory, bloomFilterFactory,
-                bloomFilterFalsePositiveRate, cmpFactories, mergePolicy, opTracker, ioScheduler, ioOpCallback,
-                transactionBTreeFactory, durable);
+                diskBufferCache, fileNameManager, componentFactory, bulkLoadComponentFactory,
+                transactionComponentFactory, bloomFilterFalsePositiveRate, cmpFactories, mergePolicy, opTracker,
+                ioScheduler, ioOpCallback, durable);
     }
 
     public static ExternalBTreeWithBuddy createExternalBTreeWithBuddy(IIOManager ioManager, FileReference file,
@@ -200,12 +222,15 @@ public class LSMBTreeUtil {
         ILSMIndexFileManager fileNameManager =
                 new LSMBTreeWithBuddyFileManager(ioManager, file, diskBTreeFactory, diskBuddyBTreeFactory);
 
+        ILSMDiskComponentFactory componentFactory = new LSMBTreeWithBuddyDiskComponentFactory(diskBTreeFactory,
+                diskBuddyBTreeFactory, bloomFilterFactory, null);
+        ILSMDiskComponentFactory bulkLoadComponentFactory = new LSMBTreeWithBuddyDiskComponentFactory(
+                bulkLoadBTreeFactory, diskBuddyBTreeFactory, bloomFilterFactory, null);
+
         // the disk only index uses an empty ArrayList for virtual buffer caches
-        ExternalBTreeWithBuddy lsmTree = new ExternalBTreeWithBuddy(ioManager, interiorFrameFactory,
-                insertLeafFrameFactory, buddyBtreeLeafFrameFactory, diskBufferCache, fileNameManager,
-                bulkLoadBTreeFactory, diskBTreeFactory, diskBuddyBTreeFactory, bloomFilterFactory,
-                bloomFilterFalsePositiveRate, mergePolicy, opTracker, ioScheduler, ioOpCallback, cmpFactories,
-                buddyBtreeCmpFactories, buddyBTreeFields, durable);
-        return lsmTree;
+        return new ExternalBTreeWithBuddy(ioManager, interiorFrameFactory, insertLeafFrameFactory,
+                buddyBtreeLeafFrameFactory, diskBufferCache, fileNameManager, componentFactory,
+                bulkLoadComponentFactory, bloomFilterFalsePositiveRate, mergePolicy, opTracker, ioScheduler,
+                ioOpCallback, cmpFactories, buddyBtreeCmpFactories, buddyBTreeFields, durable);
     }
 }

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/77f89525/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/AbstractLSMWithBloomFilterDiskComponent.java
----------------------------------------------------------------------
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
new file mode 100644
index 0000000..107190d
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/AbstractLSMWithBloomFilterDiskComponent.java
@@ -0,0 +1,105 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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;
+
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.storage.am.bloomfilter.impls.BloomCalculations;
+import org.apache.hyracks.storage.am.bloomfilter.impls.BloomFilter;
+import org.apache.hyracks.storage.am.bloomfilter.impls.BloomFilterSpecification;
+import org.apache.hyracks.storage.am.common.api.IMetadataPageManager;
+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;
+import org.apache.hyracks.storage.am.lsm.common.impls.ChainedLSMDiskComponentBulkLoader;
+import org.apache.hyracks.storage.am.lsm.common.impls.IChainedComponentBulkLoader;
+import org.apache.hyracks.storage.am.lsm.common.util.ComponentUtils;
+import org.apache.hyracks.storage.common.buffercache.IBufferCache;
+
+public abstract class AbstractLSMWithBloomFilterDiskComponent extends AbstractLSMDiskComponent {
+    public AbstractLSMWithBloomFilterDiskComponent(AbstractLSMIndex lsmIndex, IMetadataPageManager mdPageManager,
+            ILSMComponentFilter filter) {
+        super(lsmIndex, mdPageManager, filter);
+    }
+
+    public abstract BloomFilter getBloomFilter();
+
+    public abstract IBufferCache getBloomFilterBufferCache();
+
+    @Override
+    public void markAsValid(boolean persist) throws HyracksDataException {
+        // The order of forcing the dirty page to be flushed is critical. The
+        // bloom filter must be always done first.
+        ComponentUtils.markAsValid(getBloomFilterBufferCache(), getBloomFilter(), persist);
+        super.markAsValid(persist);
+    }
+
+    @Override
+    public void activate(boolean createNewComponent) throws HyracksDataException {
+        super.activate(createNewComponent);
+        if (createNewComponent) {
+            getBloomFilter().create();
+        }
+        getBloomFilter().activate();
+    }
+
+    @Override
+    public void deactivateAndDestroy() throws HyracksDataException {
+        super.deactivateAndDestroy();
+        getBloomFilter().deactivate();
+        getBloomFilter().destroy();
+    }
+
+    @Override
+    public void destroy() throws HyracksDataException {
+        super.destroy();
+        getBloomFilter().destroy();
+    }
+
+    @Override
+    public void deactivate() throws HyracksDataException {
+        super.deactivate();
+        getBloomFilter().deactivate();
+    }
+
+    @Override
+    public void deactivateAndPurge() throws HyracksDataException {
+        super.deactivateAndPurge();
+        getBloomFilter().deactivate();
+        getBloomFilter().purge();
+    }
+
+    public IChainedComponentBulkLoader createBloomFilterBulkLoader(long numElementsHint) throws HyracksDataException {
+        BloomFilterSpecification bloomFilterSpec = BloomCalculations.computeBloomSpec(
+                BloomCalculations.maxBucketsPerElement(numElementsHint), getLsmIndex().bloomFilterFalsePositiveRate());
+        return new BloomFilterBulkLoader(getBloomFilter().createBuilder(numElementsHint, bloomFilterSpec.getNumHashes(),
+                bloomFilterSpec.getNumBucketsPerElements()));
+    }
+
+    @Override
+    public ChainedLSMDiskComponentBulkLoader createBulkLoader(float fillFactor, boolean verifyInput,
+            long numElementsHint, boolean checkIfEmptyIndex, boolean withFilter, boolean cleanupEmptyComponent)
+            throws HyracksDataException {
+        ChainedLSMDiskComponentBulkLoader chainedBulkLoader = super.createBulkLoader(fillFactor, verifyInput,
+                numElementsHint, checkIfEmptyIndex, withFilter, cleanupEmptyComponent);
+        if (numElementsHint > 0) {
+            chainedBulkLoader.addBulkLoader(createBloomFilterBulkLoader(numElementsHint));
+        }
+        return chainedBulkLoader;
+    }
+}

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/77f89525/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/AbstractLSMWithBuddyDiskComponent.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/AbstractLSMWithBuddyDiskComponent.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/AbstractLSMWithBuddyDiskComponent.java
new file mode 100644
index 0000000..c2f52e0
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/AbstractLSMWithBuddyDiskComponent.java
@@ -0,0 +1,96 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.hyracks.storage.am.lsm.common.api;
+
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.storage.am.common.api.IMetadataPageManager;
+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.IChainedComponentBulkLoader;
+import org.apache.hyracks.storage.am.lsm.common.impls.IndexWithBuddyBulkLoader;
+import org.apache.hyracks.storage.am.lsm.common.util.ComponentUtils;
+import org.apache.hyracks.storage.common.IIndexBulkLoader;
+
+public abstract class AbstractLSMWithBuddyDiskComponent extends AbstractLSMWithBloomFilterDiskComponent {
+
+    public AbstractLSMWithBuddyDiskComponent(AbstractLSMIndex lsmIndex, IMetadataPageManager mdPageManager,
+            ILSMComponentFilter filter) {
+        super(lsmIndex, mdPageManager, filter);
+    }
+
+    public abstract AbstractTreeIndex getBuddyIndex();
+
+    @Override
+    public void markAsValid(boolean persist) throws HyracksDataException {
+        super.markAsValid(persist);
+        ComponentUtils.markAsValid(getBuddyIndex(), persist);
+    }
+
+    @Override
+    public void activate(boolean createNewComponent) throws HyracksDataException {
+        super.activate(createNewComponent);
+        if (createNewComponent) {
+            getBuddyIndex().create();
+        }
+        getBuddyIndex().activate();
+    }
+
+    @Override
+    public void deactivateAndDestroy() throws HyracksDataException {
+        super.deactivateAndDestroy();
+        getBuddyIndex().deactivate();
+        getBuddyIndex().destroy();
+    }
+
+    @Override
+    public void destroy() throws HyracksDataException {
+        super.destroy();
+        getBuddyIndex().destroy();
+    }
+
+    @Override
+    public void deactivate() throws HyracksDataException {
+        super.deactivate();
+        getBuddyIndex().deactivate();
+    }
+
+    @Override
+    public void deactivateAndPurge() throws HyracksDataException {
+        super.deactivateAndPurge();
+        getBuddyIndex().deactivate();
+        getBuddyIndex().purge();
+    }
+
+    @Override
+    public void validate() throws HyracksDataException {
+        super.validate();
+        getBuddyIndex().validate();
+    }
+
+    @Override
+    public IChainedComponentBulkLoader createIndexBulkLoader(float fillFactor, boolean verifyInput,
+            long numElementsHint, boolean checkIfEmptyIndex) throws HyracksDataException {
+        IIndexBulkLoader indexBulkLoader =
+                getIndex().createBulkLoader(fillFactor, verifyInput, numElementsHint, checkIfEmptyIndex);
+        IIndexBulkLoader buddyBulkLoader =
+                getBuddyIndex().createBulkLoader(fillFactor, verifyInput, numElementsHint, checkIfEmptyIndex);
+        return new IndexWithBuddyBulkLoader(indexBulkLoader, buddyBulkLoader);
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/77f89525/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/AbstractLSMWithBuddyMemoryComponent.java
----------------------------------------------------------------------
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/api/AbstractLSMWithBuddyMemoryComponent.java
new file mode 100644
index 0000000..b445142
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/AbstractLSMWithBuddyMemoryComponent.java
@@ -0,0 +1,61 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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;
+
+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.AbstractLSMMemoryComponent;
+
+public abstract class AbstractLSMWithBuddyMemoryComponent extends AbstractLSMMemoryComponent {
+
+    public AbstractLSMWithBuddyMemoryComponent(IVirtualBufferCache vbc, boolean isActive, ILSMComponentFilter filter) {
+        super(vbc, isActive, filter);
+    }
+
+    public abstract AbstractTreeIndex getBuddyIndex();
+
+    @Override
+    public void doReset() throws HyracksDataException {
+        super.doReset();
+        getBuddyIndex().deactivate();
+        getBuddyIndex().destroy();
+        getBuddyIndex().create();
+        getBuddyIndex().activate();
+    }
+
+    @Override
+    public void doAllocate() throws HyracksDataException {
+        super.doAllocate();
+        getBuddyIndex().create();
+        getBuddyIndex().activate();
+    }
+
+    @Override
+    public void doDeallocate() throws HyracksDataException {
+        super.doDeallocate();
+        getBuddyIndex().deactivate();
+        getBuddyIndex().destroy();
+    }
+
+    @Override
+    public void validate() throws HyracksDataException {
+        super.validate();
+        getBuddyIndex().validate();
+    }
+}

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/77f89525/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/ILSMComponent.java
----------------------------------------------------------------------
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 a446f4e..a60f544 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.common.IIndex;
 
 /**
  * An LSM index component. can be an in memory or on disk. Can be readable or unreadable, writable or unwritable
@@ -121,4 +122,9 @@ public interface ILSMComponent {
      * @return the component filter
      */
     ILSMComponentFilter getLSMComponentFilter();
+
+    /**
+     * @return index data structure that is the stored in the component
+     */
+    IIndex getIndex();
 }

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/77f89525/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/ILSMDiskComponent.java
----------------------------------------------------------------------
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 b101315..43c5482 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
@@ -18,8 +18,14 @@
  */
 package org.apache.hyracks.storage.am.lsm.common.api;
 
+import java.util.Set;
+
 import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.storage.am.common.api.ITreeIndex;
+import org.apache.hyracks.storage.am.lsm.common.impls.AbstractLSMIndex;
+import org.apache.hyracks.storage.am.lsm.common.impls.ChainedLSMDiskComponentBulkLoader;
 import org.apache.hyracks.storage.am.lsm.common.impls.DiskComponentMetadata;
+import org.apache.hyracks.storage.am.lsm.common.impls.IChainedComponentBulkLoader;
 
 public interface ILSMDiskComponent extends ILSMComponent {
 
@@ -42,13 +48,6 @@ public interface ILSMDiskComponent extends ILSMComponent {
     int getFileReferenceCount();
 
     /**
-     * Delete the component from disk
-     *
-     * @throws HyracksDataException
-     */
-    void destroy() throws HyracksDataException;
-
-    /**
      * Return the component Id of this disk component from its metadata
      *
      * @return
@@ -57,6 +56,21 @@ public interface ILSMDiskComponent extends ILSMComponent {
     ILSMDiskComponentId getComponentId() throws HyracksDataException;
 
     /**
+     * @return LsmIndex of the component
+     */
+    AbstractLSMIndex getLsmIndex();
+
+    /**
+     * @return the TreeIndex which holds metadata for the disk component
+     */
+    ITreeIndex getMetadataHolder();
+
+    /**
+     * @return a set of files describing the contents of the disk component
+     */
+    Set<String> getLSMComponentPhysicalFiles();
+
+    /**
      * Mark the component as valid
      *
      * @param persist
@@ -65,4 +79,86 @@ public interface ILSMDiskComponent extends ILSMComponent {
      */
     void markAsValid(boolean persist) throws HyracksDataException;
 
+    /**
+     * Activates the component
+     *
+     * @param create
+     *            whether a new component should be created
+     * @throws HyracksDataException
+     */
+    void activate(boolean create) throws HyracksDataException;
+
+    /**
+     * Deactivate and destroy the component (Deletes it from disk)
+     *
+     * @throws HyracksDataException
+     */
+    void deactivateAndDestroy() throws HyracksDataException;
+
+    /**
+     * Destroy the component (Deletes it from disk)
+     *
+     * @throws HyracksDataException
+     */
+    void destroy() throws HyracksDataException;
+
+    /**
+     * Deactivate the component
+     * The pages are still in the buffer cache
+     *
+     * @throws HyracksDataException
+     */
+    void deactivate() throws HyracksDataException;
+
+    /**
+     * Deactivate the component and purge it out of the buffer cache
+     *
+     * @throws HyracksDataException
+     */
+    void deactivateAndPurge() throws HyracksDataException;
+
+    /**
+     * Test method. validates the content of the component
+     * TODO: Remove this method from the interface
+     *
+     * @throws HyracksDataException
+     */
+    void validate() throws HyracksDataException;
+
+    /**
+     * Creates a chained bulkloader which populates component's LSM filter
+     *
+     * @return
+     * @throws HyracksDataException
+     */
+    IChainedComponentBulkLoader createFilterBulkLoader() throws HyracksDataException;
+
+    /**
+     * Creates a chained bulkloader which populates component's index
+     *
+     * @param fillFactor
+     * @param verifyInput
+     * @param numElementsHint
+     * @param checkIfEmptyIndex
+     * @return
+     * @throws HyracksDataException
+     */
+    IChainedComponentBulkLoader createIndexBulkLoader(float fillFactor, boolean verifyInput, long numElementsHint,
+            boolean checkIfEmptyIndex) throws HyracksDataException;
+
+    /**
+     * Creates a bulkloader pipeline which includes all chained operations, bulkloading individual elements of the
+     * component: indexes, LSM filters, Bloom filters, buddy indexes, etc.
+     *
+     * @param fillFactor
+     * @param verifyInput
+     * @param numElementsHint
+     * @param checkIfEmptyIndex
+     * @param withFilter
+     * @param cleanupEmptyComponent
+     * @return
+     * @throws HyracksDataException
+     */
+    ChainedLSMDiskComponentBulkLoader createBulkLoader(float fillFactor, boolean verifyInput, long numElementsHint,
+            boolean checkIfEmptyIndex, boolean withFilter, boolean cleanupEmptyComponent) throws HyracksDataException;
 }

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/77f89525/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/ILSMDiskComponentBulkLoader.java
----------------------------------------------------------------------
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 f1d1ce1..13ce971 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
@@ -34,4 +34,11 @@ public interface ILSMDiskComponentBulkLoader extends IIndexBulkLoader {
      */
     void delete(ITupleReference tuple) throws HyracksDataException;
 
+    /**
+     * Releases all resources allocated during the bulkloading process
+     *
+     * @throws HyracksDataException
+     */
+    void cleanupArtifacts() throws HyracksDataException;
+
 }

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/77f89525/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/ILSMDiskComponentFactory.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/ILSMDiskComponentFactory.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/ILSMDiskComponentFactory.java
index 9daf30b..a904670 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/ILSMDiskComponentFactory.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/ILSMDiskComponentFactory.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.impls.AbstractLSMIndex;
 import org.apache.hyracks.storage.am.lsm.common.impls.LSMComponentFileReferences;
 
 @FunctionalInterface
@@ -32,5 +33,6 @@ public interface ILSMDiskComponentFactory {
      * @return a disk component
      * @throws HyracksDataException
      */
-    ILSMDiskComponent createComponent(LSMComponentFileReferences cfr) throws HyracksDataException;
+    ILSMDiskComponent createComponent(AbstractLSMIndex lsmIndex, LSMComponentFileReferences cfr)
+            throws HyracksDataException;
 }

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/77f89525/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/ILSMIndex.java
----------------------------------------------------------------------
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 addeb27..e4a9436 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
@@ -134,23 +134,6 @@ public interface ILSMIndex extends IIndex {
     void updateFilter(ILSMIndexOperationContext ictx, ITupleReference tuple) throws HyracksDataException;
 
     /**
-     * Create a component bulk loader for the given component
-     *
-     * @param component
-     * @param fillFactor
-     * @param verifyInput
-     * @param numElementsHint
-     * @param checkIfEmptyIndex
-     * @param withFilter
-     * @param cleanupEmptyComponent
-     * @return
-     * @throws HyracksDataException
-     */
-    ILSMDiskComponentBulkLoader createComponentBulkLoader(ILSMDiskComponent component, float fillFactor,
-            boolean verifyInput, long numElementsHint, boolean checkIfEmptyIndex, boolean withFilter,
-            boolean cleanupEmptyComponent) throws HyracksDataException;
-
-    /**
      * Creates a disk component for the bulk load operation
      *
      * @return


[2/6] asterixdb git commit: [NO ISSUE][STO][IDX] LSM storage cleanup

Posted by il...@apache.org.
http://git-wip-us.apache.org/repos/asf/asterixdb/blob/77f89525/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/impls/AbstractLSMRTree.java
----------------------------------------------------------------------
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 ec8a857..02a4221 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
@@ -42,7 +42,6 @@ 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.ILSMIndexFileManager;
 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.ILSMMergePolicy;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMOperationTracker;
 import org.apache.hyracks.storage.am.lsm.common.api.IVirtualBufferCache;
@@ -66,10 +65,6 @@ public abstract class AbstractLSMRTree extends AbstractLSMIndex implements ITree
     protected final IBinaryComparatorFactory[] linearizerArray;
     protected final boolean isPointMBR;
 
-    // On-disk components.
-    // For creating RTree's used in flush and merge.
-    protected final ILSMDiskComponentFactory componentFactory;
-
     protected IBinaryComparatorFactory[] btreeCmpFactories;
     protected IBinaryComparatorFactory[] rtreeCmpFactories;
 
@@ -82,17 +77,18 @@ public abstract class AbstractLSMRTree extends AbstractLSMIndex implements ITree
     public AbstractLSMRTree(IIOManager ioManager, List<IVirtualBufferCache> virtualBufferCaches,
             RTreeFrameFactory rtreeInteriorFrameFactory, RTreeFrameFactory rtreeLeafFrameFactory,
             ITreeIndexFrameFactory btreeInteriorFrameFactory, ITreeIndexFrameFactory btreeLeafFrameFactory,
-            ILSMIndexFileManager fileManager, ILSMDiskComponentFactory componentFactory, int fieldCount,
+            IBufferCache diskBufferCache, ILSMIndexFileManager fileManager, ILSMDiskComponentFactory componentFactory,
+            ILSMDiskComponentFactory bulkLoadComponentFactory, int fieldCount,
             IBinaryComparatorFactory[] rtreeCmpFactories, IBinaryComparatorFactory[] btreeCmpFactories,
             ILinearizeComparatorFactory linearizer, int[] comparatorFields, IBinaryComparatorFactory[] linearizerArray,
             double bloomFilterFalsePositiveRate, ILSMMergePolicy mergePolicy, ILSMOperationTracker opTracker,
             ILSMIOOperationScheduler ioScheduler, ILSMIOOperationCallback ioOpCallback,
             IComponentFilterHelper filterHelper, ILSMComponentFilterFrameFactory filterFrameFactory,
             LSMComponentFilterManager filterManager, int[] rtreeFields, int[] filterFields, boolean durable,
-            boolean isPointMBR, IBufferCache diskBufferCache) throws HyracksDataException {
+            boolean isPointMBR) throws HyracksDataException {
         super(ioManager, virtualBufferCaches, diskBufferCache, fileManager, bloomFilterFalsePositiveRate, mergePolicy,
-                opTracker, ioScheduler, ioOpCallback, filterFrameFactory, filterManager, filterFields, durable,
-                filterHelper, rtreeFields, ITracer.NONE);
+                opTracker, ioScheduler, ioOpCallback, componentFactory, bulkLoadComponentFactory, filterFrameFactory,
+                filterManager, filterFields, durable, filterHelper, rtreeFields, ITracer.NONE);
         int i = 0;
         for (IVirtualBufferCache virtualBufferCache : virtualBufferCaches) {
             RTree memRTree = new RTree(virtualBufferCache, new VirtualFreePageManager(virtualBufferCache),
@@ -112,7 +108,6 @@ public abstract class AbstractLSMRTree extends AbstractLSMIndex implements ITree
         this.rtreeLeafFrameFactory = rtreeLeafFrameFactory;
         this.btreeInteriorFrameFactory = btreeInteriorFrameFactory;
         this.btreeLeafFrameFactory = btreeLeafFrameFactory;
-        this.componentFactory = componentFactory;
         this.btreeCmpFactories = btreeCmpFactories;
         this.rtreeCmpFactories = rtreeCmpFactories;
         this.linearizer = linearizer;
@@ -126,19 +121,19 @@ public abstract class AbstractLSMRTree extends AbstractLSMIndex implements ITree
      */
     public AbstractLSMRTree(IIOManager ioManager, ITreeIndexFrameFactory rtreeInteriorFrameFactory,
             ITreeIndexFrameFactory rtreeLeafFrameFactory, ITreeIndexFrameFactory btreeInteriorFrameFactory,
-            ITreeIndexFrameFactory btreeLeafFrameFactory, ILSMIndexFileManager fileManager,
-            ILSMDiskComponentFactory componentFactory, IBinaryComparatorFactory[] rtreeCmpFactories,
-            IBinaryComparatorFactory[] btreeCmpFactories, ILinearizeComparatorFactory linearizer,
-            int[] comparatorFields, IBinaryComparatorFactory[] linearizerArray, double bloomFilterFalsePositiveRate,
-            ILSMMergePolicy mergePolicy, ILSMOperationTracker opTracker, ILSMIOOperationScheduler ioScheduler,
-            ILSMIOOperationCallback ioOpCallback, boolean durable, boolean isPointMBR, IBufferCache diskBufferCache) {
+            ITreeIndexFrameFactory btreeLeafFrameFactory, IBufferCache diskBufferCache,
+            ILSMIndexFileManager fileManager, ILSMDiskComponentFactory componentFactory,
+            IBinaryComparatorFactory[] rtreeCmpFactories, IBinaryComparatorFactory[] btreeCmpFactories,
+            ILinearizeComparatorFactory linearizer, int[] comparatorFields, IBinaryComparatorFactory[] linearizerArray,
+            double bloomFilterFalsePositiveRate, ILSMMergePolicy mergePolicy, ILSMOperationTracker opTracker,
+            ILSMIOOperationScheduler ioScheduler, ILSMIOOperationCallback ioOpCallback, boolean durable,
+            boolean isPointMBR) {
         super(ioManager, diskBufferCache, fileManager, bloomFilterFalsePositiveRate, mergePolicy, opTracker,
-                ioScheduler, ioOpCallback, durable);
+                ioScheduler, ioOpCallback, componentFactory, componentFactory, durable);
         this.rtreeInteriorFrameFactory = rtreeInteriorFrameFactory;
         this.rtreeLeafFrameFactory = rtreeLeafFrameFactory;
         this.btreeInteriorFrameFactory = btreeInteriorFrameFactory;
         this.btreeLeafFrameFactory = btreeLeafFrameFactory;
-        this.componentFactory = componentFactory;
         this.btreeCmpFactories = btreeCmpFactories;
         this.rtreeCmpFactories = rtreeCmpFactories;
         this.linearizer = linearizer;
@@ -154,71 +149,46 @@ public abstract class AbstractLSMRTree extends AbstractLSMIndex implements ITree
         cursor.open(ctx.getSearchInitialState(), pred);
     }
 
-    protected LSMRTreeDiskComponent createDiskComponent(ILSMDiskComponentFactory factory, FileReference insertFileRef,
-            FileReference deleteFileRef, FileReference bloomFilterFileRef, boolean createComponent)
-            throws HyracksDataException {
-        // Create new tree instance.
-        LSMRTreeDiskComponent component = (LSMRTreeDiskComponent) factory
-                .createComponent(new LSMComponentFileReferences(insertFileRef, deleteFileRef, bloomFilterFileRef));
-        // Tree will be closed during cleanup of merge().
-        if (createComponent) {
-            component.getRTree().create();
-        }
-        component.getRTree().activate();
-        if (component.getBTree() != null) {
-            if (createComponent) {
-                component.getBTree().create();
-                component.getBloomFilter().create();
-            }
-            component.getBTree().activate();
-            component.getBloomFilter().activate();
-        }
-        if (component.getLSMComponentFilter() != null && !createComponent) {
-            getFilterManager().readFilter(component.getLSMComponentFilter(), component.getRTree());
-        }
-        return component;
-    }
-
     @Override
     public ITreeIndexFrameFactory getLeafFrameFactory() {
         LSMRTreeMemoryComponent mutableComponent =
                 (LSMRTreeMemoryComponent) memoryComponents.get(currentMutableComponentId.get());
-        return mutableComponent.getRTree().getLeafFrameFactory();
+        return mutableComponent.getIndex().getLeafFrameFactory();
     }
 
     @Override
     public ITreeIndexFrameFactory getInteriorFrameFactory() {
         LSMRTreeMemoryComponent mutableComponent =
                 (LSMRTreeMemoryComponent) memoryComponents.get(currentMutableComponentId.get());
-        return mutableComponent.getRTree().getInteriorFrameFactory();
+        return mutableComponent.getIndex().getInteriorFrameFactory();
     }
 
     @Override
     public IPageManager getPageManager() {
         LSMRTreeMemoryComponent mutableComponent =
                 (LSMRTreeMemoryComponent) memoryComponents.get(currentMutableComponentId.get());
-        return mutableComponent.getRTree().getPageManager();
+        return mutableComponent.getIndex().getPageManager();
     }
 
     @Override
     public int getFieldCount() {
         LSMRTreeMemoryComponent mutableComponent =
                 (LSMRTreeMemoryComponent) memoryComponents.get(currentMutableComponentId.get());
-        return mutableComponent.getRTree().getFieldCount();
+        return mutableComponent.getIndex().getFieldCount();
     }
 
     @Override
     public int getRootPageId() {
         LSMRTreeMemoryComponent mutableComponent =
                 (LSMRTreeMemoryComponent) memoryComponents.get(currentMutableComponentId.get());
-        return mutableComponent.getRTree().getRootPageId();
+        return mutableComponent.getIndex().getRootPageId();
     }
 
     @Override
     public int getFileId() {
         LSMRTreeMemoryComponent mutableComponent =
                 (LSMRTreeMemoryComponent) memoryComponents.get(currentMutableComponentId.get());
-        return mutableComponent.getRTree().getFileId();
+        return mutableComponent.getIndex().getFileId();
     }
 
     @Override
@@ -272,60 +242,15 @@ public abstract class AbstractLSMRTree extends AbstractLSMIndex implements ITree
     }
 
     @Override
-    protected void validateMemoryComponent(ILSMMemoryComponent c) throws HyracksDataException {
-        throw new UnsupportedOperationException("Validation not implemented for LSM R-Trees.");
-    }
-
-    @Override
-    protected void validateDiskComponent(ILSMDiskComponent c) throws HyracksDataException {
-        throw new UnsupportedOperationException("Validation not implemented for LSM R-Trees.");
-    }
-
-    @Override
-    protected long getMemoryComponentSize(ILSMMemoryComponent c) {
-        LSMRTreeMemoryComponent mutableComponent = (LSMRTreeMemoryComponent) c;
-        IBufferCache virtualBufferCache = mutableComponent.getRTree().getBufferCache();
-        return virtualBufferCache.getNumPages() * (long) virtualBufferCache.getPageSize();
-    }
-
-    @Override
     public boolean isPrimaryIndex() {
         return false;
     }
 
     @Override
-    protected void allocateMemoryComponent(ILSMMemoryComponent c) throws HyracksDataException {
-        LSMRTreeMemoryComponent mutableComponent = (LSMRTreeMemoryComponent) c;
-        ((IVirtualBufferCache) mutableComponent.getRTree().getBufferCache()).open();
-        mutableComponent.getRTree().create();
-        mutableComponent.getBTree().create();
-        mutableComponent.getRTree().activate();
-        mutableComponent.getBTree().activate();
-    }
-
-    @Override
-    protected void deactivateMemoryComponent(ILSMMemoryComponent c) throws HyracksDataException {
-        LSMRTreeMemoryComponent mutableComponent = (LSMRTreeMemoryComponent) c;
-        mutableComponent.getRTree().deactivate();
-        mutableComponent.getBTree().deactivate();
-        mutableComponent.getRTree().destroy();
-        mutableComponent.getBTree().destroy();
-        ((IVirtualBufferCache) mutableComponent.getRTree().getBufferCache()).close();
-    }
-
-    @Override
-    protected void clearMemoryComponent(ILSMMemoryComponent c) throws HyracksDataException {
-        LSMRTreeMemoryComponent mutableComponent = (LSMRTreeMemoryComponent) c;
-        mutableComponent.getRTree().clear();
-        mutableComponent.getBTree().clear();
-        mutableComponent.reset();
-    }
-
-    @Override
     protected LSMComponentFileReferences getMergeFileReferences(ILSMDiskComponent firstComponent,
             ILSMDiskComponent lastComponent) throws HyracksDataException {
-        RTree firstTree = ((LSMRTreeDiskComponent) firstComponent).getRTree();
-        RTree lastTree = ((LSMRTreeDiskComponent) lastComponent).getRTree();
+        RTree firstTree = (RTree) firstComponent.getIndex();
+        RTree lastTree = (RTree) lastComponent.getIndex();
         FileReference firstFile = firstTree.getFileReference();
         FileReference lastFile = lastTree.getFileReference();
         return fileManager.getRelMergeFileReference(firstFile.getFile().getName(), lastFile.getFile().getName());

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/77f89525/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/impls/ExternalRTree.java
----------------------------------------------------------------------
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 6595403..228f359 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
@@ -28,8 +28,6 @@ import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.api.io.IIOManager;
 import org.apache.hyracks.dataflow.common.data.accessors.ITupleReference;
 import org.apache.hyracks.storage.am.bloomfilter.impls.BloomCalculations;
-import org.apache.hyracks.storage.am.bloomfilter.impls.BloomFilter;
-import org.apache.hyracks.storage.am.bloomfilter.impls.BloomFilterFactory;
 import org.apache.hyracks.storage.am.bloomfilter.impls.BloomFilterSpecification;
 import org.apache.hyracks.storage.am.btree.impls.BTree;
 import org.apache.hyracks.storage.am.btree.impls.BTree.BTreeBulkLoader;
@@ -41,6 +39,7 @@ import org.apache.hyracks.storage.am.common.impls.NoOpOperationCallback;
 import org.apache.hyracks.storage.am.common.ophelpers.IndexOperation;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponent;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMDiskComponent;
+import org.apache.hyracks.storage.am.lsm.common.api.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.ILSMIOOperationScheduler;
@@ -54,14 +53,13 @@ import org.apache.hyracks.storage.am.lsm.common.api.LSMOperationType;
 import org.apache.hyracks.storage.am.lsm.common.impls.BlockingIOOperationCallbackWrapper;
 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.TreeIndexFactory;
-import org.apache.hyracks.storage.am.rtree.impls.RTree;
 import org.apache.hyracks.storage.am.rtree.impls.SearchPredicate;
 import org.apache.hyracks.storage.common.IIndexBulkLoader;
 import org.apache.hyracks.storage.common.IIndexCursor;
 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.buffercache.IBufferCache;
 
 /**
  * This is an lsm r-tree that does not have memory component and is modified
@@ -83,18 +81,17 @@ public class ExternalRTree extends LSMRTree implements ITwoPCIndex {
 
     public ExternalRTree(IIOManager ioManager, ITreeIndexFrameFactory rtreeInteriorFrameFactory,
             ITreeIndexFrameFactory rtreeLeafFrameFactory, ITreeIndexFrameFactory btreeInteriorFrameFactory,
-            ITreeIndexFrameFactory btreeLeafFrameFactory, ILSMIndexFileManager fileNameManager,
-            TreeIndexFactory<RTree> diskRTreeFactory, TreeIndexFactory<BTree> diskBTreeFactory,
-            BloomFilterFactory bloomFilterFactory, double bloomFilterFalsePositiveRate, int fieldCount,
-            IBinaryComparatorFactory[] rtreeCmpFactories, IBinaryComparatorFactory[] btreeCmpFactories,
-            ILinearizeComparatorFactory linearizer, int[] comparatorFields, IBinaryComparatorFactory[] linearizerArray,
-            ILSMMergePolicy mergePolicy, ILSMOperationTracker opTracker, ILSMIOOperationScheduler ioScheduler,
-            ILSMIOOperationCallback ioOpCallback, int[] buddyBTreeFields, boolean durable, boolean isPointMBR) {
+            ITreeIndexFrameFactory btreeLeafFrameFactory, IBufferCache diskBufferCache,
+            ILSMIndexFileManager fileNameManager, ILSMDiskComponentFactory componentFactory,
+            double bloomFilterFalsePositiveRate, int fieldCount, IBinaryComparatorFactory[] rtreeCmpFactories,
+            IBinaryComparatorFactory[] btreeCmpFactories, ILinearizeComparatorFactory linearizer,
+            int[] comparatorFields, IBinaryComparatorFactory[] linearizerArray, ILSMMergePolicy mergePolicy,
+            ILSMOperationTracker opTracker, ILSMIOOperationScheduler ioScheduler, ILSMIOOperationCallback ioOpCallback,
+            int[] buddyBTreeFields, boolean durable, boolean isPointMBR) {
         super(ioManager, rtreeInteriorFrameFactory, rtreeLeafFrameFactory, btreeInteriorFrameFactory,
-                btreeLeafFrameFactory, fileNameManager, diskRTreeFactory, diskBTreeFactory, bloomFilterFactory,
-                bloomFilterFalsePositiveRate, rtreeCmpFactories, btreeCmpFactories, linearizer, comparatorFields,
-                linearizerArray, mergePolicy, opTracker, ioScheduler, ioOpCallback, buddyBTreeFields, durable,
-                isPointMBR);
+                btreeLeafFrameFactory, diskBufferCache, fileNameManager, componentFactory, bloomFilterFalsePositiveRate,
+                rtreeCmpFactories, btreeCmpFactories, linearizer, comparatorFields, linearizerArray, mergePolicy,
+                opTracker, ioScheduler, ioOpCallback, buddyBTreeFields, durable, isPointMBR);
         this.secondDiskComponents = new LinkedList<>();
         this.fieldCount = fieldCount;
     }
@@ -200,7 +197,7 @@ public class ExternalRTree extends LSMRTree implements ITwoPCIndex {
             List<LSMComponentFileReferences> validFileReferences;
             validFileReferences = fileManager.cleanupAndGetValidFiles();
             for (LSMComponentFileReferences lsmComonentFileReference : validFileReferences) {
-                LSMRTreeDiskComponent component;
+                ILSMDiskComponent component;
                 component =
                         createDiskComponent(componentFactory, lsmComonentFileReference.getInsertIndexFileReference(),
                                 lsmComonentFileReference.getDeleteIndexFileReference(),
@@ -214,23 +211,13 @@ public class ExternalRTree extends LSMRTree implements ITwoPCIndex {
             // components. It should also maintain the version pointer
             for (ILSMComponent c : diskComponents) {
                 LSMRTreeDiskComponent component = (LSMRTreeDiskComponent) c;
-                RTree rtree = component.getRTree();
-                BTree btree = component.getBTree();
-                BloomFilter bloomFilter = component.getBloomFilter();
-                rtree.activate();
-                btree.activate();
-                bloomFilter.activate();
+                component.activate(false);
             }
             for (ILSMComponent c : secondDiskComponents) {
                 // Only activate non shared components
                 if (!diskComponents.contains(c)) {
                     LSMRTreeDiskComponent component = (LSMRTreeDiskComponent) c;
-                    RTree rtree = component.getRTree();
-                    BTree btree = component.getBTree();
-                    BloomFilter bloomFilter = component.getBloomFilter();
-                    rtree.activate();
-                    btree.activate();
-                    bloomFilter.activate();
+                    component.activate(false);
                 }
             }
         }
@@ -265,8 +252,8 @@ public class ExternalRTree extends LSMRTree implements ITwoPCIndex {
         ILSMIndexOperationContext opCtx = ((LSMRTreeSortedCursor) cursor).getOpCtx();
         search(opCtx, cursor, rtreeSearchPred);
 
-        LSMRTreeDiskComponent mergedComponent = createDiskComponent(componentFactory, mergeOp.getTarget(),
-                mergeOp.getBTreeTarget(), mergeOp.getBloomFilterTarget(), true);
+        LSMRTreeDiskComponent mergedComponent = (LSMRTreeDiskComponent) createDiskComponent(componentFactory,
+                mergeOp.getTarget(), mergeOp.getBTreeTarget(), mergeOp.getBloomFilterTarget(), true);
 
         // In case we must keep the deleted-keys BTrees, then they must be
         // merged *before* merging the r-trees so that
@@ -288,7 +275,7 @@ public class ExternalRTree extends LSMRTree implements ITwoPCIndex {
             LSMRTreeDeletedKeysBTreeMergeCursor btreeCursor = new LSMRTreeDeletedKeysBTreeMergeCursor(opCtx);
             search(opCtx, btreeCursor, rtreeSearchPred);
 
-            BTree btree = mergedComponent.getBTree();
+            BTree btree = mergedComponent.getBuddyIndex();
             IIndexBulkLoader btreeBulkLoader = btree.createBulkLoader(1.0f, true, 0L, false);
 
             long numElements = 0L;
@@ -317,7 +304,7 @@ public class ExternalRTree extends LSMRTree implements ITwoPCIndex {
             btreeBulkLoader.end();
         }
 
-        IIndexBulkLoader bulkLoader = mergedComponent.getRTree().createBulkLoader(1.0f, false, 0L, false);
+        IIndexBulkLoader bulkLoader = mergedComponent.getIndex().createBulkLoader(1.0f, false, 0L, false);
         try {
             while (cursor.hasNext()) {
                 cursor.next();
@@ -343,11 +330,11 @@ public class ExternalRTree extends LSMRTree implements ITwoPCIndex {
         }
 
         for (ILSMDiskComponent c : diskComponents) {
-            deactivateDiskComponent(c);
+            c.deactivateAndPurge();
         }
         for (ILSMDiskComponent c : secondDiskComponents) {
             if (!diskComponents.contains(c)) {
-                deactivateDiskComponent(c);
+                c.deactivateAndPurge();
             }
         }
         isActive = false;
@@ -362,26 +349,14 @@ public class ExternalRTree extends LSMRTree implements ITwoPCIndex {
         }
         getLsmHarness().indexClear();
 
-        for (ILSMComponent c : diskComponents) {
-            LSMRTreeDiskComponent component = (LSMRTreeDiskComponent) c;
-            component.getRTree().deactivate();
-            component.getBloomFilter().deactivate();
-            component.getBTree().deactivate();
-            component.getRTree().destroy();
-            component.getBloomFilter().destroy();
-            component.getBTree().destroy();
+        for (ILSMDiskComponent c : diskComponents) {
+            c.deactivateAndDestroy();
             // Remove from second list to avoid destroying twice
             secondDiskComponents.remove(c);
         }
 
-        for (ILSMComponent c : secondDiskComponents) {
-            LSMRTreeDiskComponent component = (LSMRTreeDiskComponent) c;
-            component.getRTree().deactivate();
-            component.getBloomFilter().deactivate();
-            component.getBTree().deactivate();
-            component.getRTree().destroy();
-            component.getBloomFilter().destroy();
-            component.getBTree().destroy();
+        for (ILSMDiskComponent c : secondDiskComponents) {
+            c.deactivateAndDestroy();
         }
 
         diskComponents.clear();
@@ -395,12 +370,12 @@ public class ExternalRTree extends LSMRTree implements ITwoPCIndex {
             throw new HyracksDataException("Failed to destroy the index since it is activated.");
         }
         for (ILSMDiskComponent c : diskComponents) {
-            destroyDiskComponent(c);
+            c.destroy();
             // Remove from second list to avoid destroying twice
             secondDiskComponents.remove(c);
         }
         for (ILSMDiskComponent c : secondDiskComponents) {
-            destroyDiskComponent(c);
+            c.destroy();
         }
         diskComponents.clear();
         secondDiskComponents.clear();
@@ -505,9 +480,9 @@ public class ExternalRTree extends LSMRTree implements ITwoPCIndex {
             }
 
             // Create the three loaders
-            rtreeBulkLoader = ((LSMRTreeDiskComponent) component).getRTree().createBulkLoader(fillFactor, verifyInput,
+            rtreeBulkLoader = ((LSMRTreeDiskComponent) component).getIndex().createBulkLoader(fillFactor, verifyInput,
                     numElementsHint, false);
-            btreeBulkLoader = (BTreeBulkLoader) ((LSMRTreeDiskComponent) component).getBTree()
+            btreeBulkLoader = (BTreeBulkLoader) ((LSMRTreeDiskComponent) component).getBuddyIndex()
                     .createBulkLoader(fillFactor, verifyInput, numElementsHint, false);
             int maxBucketsPerElement = BloomCalculations.maxBucketsPerElement(numElementsHint);
             BloomFilterSpecification bloomFilterSpec =
@@ -534,24 +509,7 @@ public class ExternalRTree extends LSMRTree implements ITwoPCIndex {
         public void cleanupArtifacts() throws HyracksDataException {
             if (!cleanedUpArtifacts) {
                 cleanedUpArtifacts = true;
-                try {
-                    ((LSMRTreeDiskComponent) component).getRTree().deactivate();
-                } catch (Exception e) {
-
-                }
-                ((LSMRTreeDiskComponent) component).getRTree().destroy();
-                try {
-                    ((LSMRTreeDiskComponent) component).getBTree().deactivate();
-                } catch (Exception e) {
-
-                }
-                ((LSMRTreeDiskComponent) component).getBTree().destroy();
-                try {
-                    ((LSMRTreeDiskComponent) component).getBloomFilter().deactivate();
-                } catch (Exception e) {
-
-                }
-                ((LSMRTreeDiskComponent) component).getBloomFilter().destroy();
+                ((LSMRTreeDiskComponent) component).deactivateAndDestroy();
             }
         }
 
@@ -570,12 +528,7 @@ public class ExternalRTree extends LSMRTree implements ITwoPCIndex {
                     // Since this is a transaction component, validate and
                     // deactivate. it could later be added or deleted
                     component.markAsValid(durable);
-                    RTree rtree = ((LSMRTreeDiskComponent) component).getRTree();
-                    BTree btree = ((LSMRTreeDiskComponent) component).getBTree();
-                    BloomFilter bloomFilter = ((LSMRTreeDiskComponent) component).getBloomFilter();
-                    rtree.deactivate();
-                    btree.deactivate();
-                    bloomFilter.deactivate();
+                    component.deactivate();
                 } else {
                     getLsmHarness().addBulkLoadedComponent(component);
                 }
@@ -685,7 +638,7 @@ public class ExternalRTree extends LSMRTree implements ITwoPCIndex {
     @Override
     public void commitTransaction() throws HyracksDataException {
         LSMComponentFileReferences componentFileRefrences = fileManager.getTransactionFileReferenceForCommit();
-        LSMRTreeDiskComponent component = null;
+        ILSMDiskComponent component = null;
         if (componentFileRefrences != null) {
             component = createDiskComponent(componentFactory, componentFileRefrences.getInsertIndexFileReference(),
                     componentFileRefrences.getDeleteIndexFileReference(),

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/77f89525/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/impls/LSMRTree.java
----------------------------------------------------------------------
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 ca0e4e1..9ba96a6 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
@@ -20,9 +20,7 @@
 package org.apache.hyracks.storage.am.lsm.rtree.impls;
 
 import java.util.ArrayList;
-import java.util.HashSet;
 import java.util.List;
-import java.util.Set;
 
 import org.apache.hyracks.api.dataflow.value.IBinaryComparatorFactory;
 import org.apache.hyracks.api.dataflow.value.ILinearizeComparatorFactory;
@@ -31,25 +29,19 @@ import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.api.io.IIOManager;
 import org.apache.hyracks.data.std.primitive.IntegerPointable;
 import org.apache.hyracks.dataflow.common.data.accessors.ITupleReference;
-import org.apache.hyracks.storage.am.bloomfilter.impls.BloomCalculations;
-import org.apache.hyracks.storage.am.bloomfilter.impls.BloomFilter;
-import org.apache.hyracks.storage.am.bloomfilter.impls.BloomFilterFactory;
-import org.apache.hyracks.storage.am.bloomfilter.impls.BloomFilterSpecification;
-import org.apache.hyracks.storage.am.btree.impls.BTree;
 import org.apache.hyracks.storage.am.btree.impls.BTree.BTreeAccessor;
 import org.apache.hyracks.storage.am.btree.impls.RangePredicate;
 import org.apache.hyracks.storage.am.common.api.IIndexOperationContext;
-import org.apache.hyracks.storage.am.common.api.ITreeIndexAccessor;
 import org.apache.hyracks.storage.am.common.api.ITreeIndexCursor;
 import org.apache.hyracks.storage.am.common.api.ITreeIndexFrameFactory;
 import org.apache.hyracks.storage.am.common.impls.NoOpOperationCallback;
 import org.apache.hyracks.storage.am.common.ophelpers.IndexOperation;
 import org.apache.hyracks.storage.am.common.tuples.DualTupleReference;
 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.ILSMComponentFilterFrameFactory;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMDiskComponent;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMDiskComponentBulkLoader;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMDiskComponentFactory;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperation;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallback;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationScheduler;
@@ -62,17 +54,15 @@ import org.apache.hyracks.storage.am.lsm.common.api.IVirtualBufferCache;
 import org.apache.hyracks.storage.am.lsm.common.impls.AbstractLSMIndexOperationContext;
 import org.apache.hyracks.storage.am.lsm.common.impls.LSMComponentFileReferences;
 import org.apache.hyracks.storage.am.lsm.common.impls.LSMComponentFilterManager;
-import org.apache.hyracks.storage.am.lsm.common.impls.TreeIndexFactory;
 import org.apache.hyracks.storage.am.rtree.frames.RTreeFrameFactory;
-import org.apache.hyracks.storage.am.rtree.impls.RTree;
+import org.apache.hyracks.storage.am.rtree.impls.RTree.RTreeAccessor;
 import org.apache.hyracks.storage.am.rtree.impls.RTreeSearchCursor;
 import org.apache.hyracks.storage.am.rtree.impls.SearchPredicate;
-import org.apache.hyracks.storage.common.IIndexBulkLoader;
 import org.apache.hyracks.storage.common.IIndexCursor;
 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;
+import org.apache.hyracks.storage.common.buffercache.IBufferCache;
 
 public class LSMRTree extends AbstractLSMRTree {
     protected final int[] buddyBTreeFields;
@@ -80,22 +70,20 @@ public class LSMRTree extends AbstractLSMRTree {
     public LSMRTree(IIOManager ioManager, List<IVirtualBufferCache> virtualBufferCaches,
             RTreeFrameFactory rtreeInteriorFrameFactory, RTreeFrameFactory rtreeLeafFrameFactory,
             ITreeIndexFrameFactory btreeInteriorFrameFactory, ITreeIndexFrameFactory btreeLeafFrameFactory,
-            ILSMIndexFileManager fileNameManager, TreeIndexFactory<RTree> diskRTreeFactory,
-            TreeIndexFactory<BTree> diskBTreeFactory, BloomFilterFactory bloomFilterFactory,
-            IComponentFilterHelper filterHelper, ILSMComponentFilterFrameFactory filterFrameFactory,
-            LSMComponentFilterManager filterManager, double bloomFilterFalsePositiveRate, int fieldCount,
-            IBinaryComparatorFactory[] rtreeCmpFactories, IBinaryComparatorFactory[] btreeCmpFactories,
-            ILinearizeComparatorFactory linearizer, int[] comparatorFields, IBinaryComparatorFactory[] linearizerArray,
-            ILSMMergePolicy mergePolicy, ILSMOperationTracker opTracker, ILSMIOOperationScheduler ioScheduler,
-            ILSMIOOperationCallback ioOpCallback, int[] rtreeFields, int[] buddyBTreeFields, int[] filterFields,
-            boolean durable, boolean isPointMBR) throws HyracksDataException {
+            IBufferCache diskBufferCache, ILSMIndexFileManager fileNameManager,
+            ILSMDiskComponentFactory componentFactory, IComponentFilterHelper filterHelper,
+            ILSMComponentFilterFrameFactory filterFrameFactory, LSMComponentFilterManager filterManager,
+            double bloomFilterFalsePositiveRate, int fieldCount, IBinaryComparatorFactory[] rtreeCmpFactories,
+            IBinaryComparatorFactory[] btreeCmpFactories, ILinearizeComparatorFactory linearizer,
+            int[] comparatorFields, IBinaryComparatorFactory[] linearizerArray, ILSMMergePolicy mergePolicy,
+            ILSMOperationTracker opTracker, ILSMIOOperationScheduler ioScheduler, ILSMIOOperationCallback ioOpCallback,
+            int[] rtreeFields, int[] buddyBTreeFields, int[] filterFields, boolean durable, boolean isPointMBR)
+            throws HyracksDataException {
         super(ioManager, virtualBufferCaches, rtreeInteriorFrameFactory, rtreeLeafFrameFactory,
-                btreeInteriorFrameFactory, btreeLeafFrameFactory, fileNameManager,
-                new LSMRTreeDiskComponentFactory(diskRTreeFactory, diskBTreeFactory, bloomFilterFactory, filterHelper),
-                fieldCount, rtreeCmpFactories, btreeCmpFactories, linearizer, comparatorFields, linearizerArray,
-                bloomFilterFalsePositiveRate, mergePolicy, opTracker, ioScheduler, ioOpCallback, filterHelper,
-                filterFrameFactory, filterManager, rtreeFields, filterFields, durable, isPointMBR,
-                diskRTreeFactory.getBufferCache());
+                btreeInteriorFrameFactory, btreeLeafFrameFactory, diskBufferCache, fileNameManager, componentFactory,
+                componentFactory, fieldCount, rtreeCmpFactories, btreeCmpFactories, linearizer, comparatorFields,
+                linearizerArray, bloomFilterFalsePositiveRate, mergePolicy, opTracker, ioScheduler, ioOpCallback,
+                filterHelper, filterFrameFactory, filterManager, rtreeFields, filterFields, durable, isPointMBR);
         this.buddyBTreeFields = buddyBTreeFields;
     }
 
@@ -104,64 +92,21 @@ public class LSMRTree extends AbstractLSMRTree {
      */
     public LSMRTree(IIOManager ioManager, ITreeIndexFrameFactory rtreeInteriorFrameFactory,
             ITreeIndexFrameFactory rtreeLeafFrameFactory, ITreeIndexFrameFactory btreeInteriorFrameFactory,
-            ITreeIndexFrameFactory btreeLeafFrameFactory, ILSMIndexFileManager fileNameManager,
-            TreeIndexFactory<RTree> diskRTreeFactory, TreeIndexFactory<BTree> diskBTreeFactory,
-            BloomFilterFactory bloomFilterFactory, double bloomFilterFalsePositiveRate,
-            IBinaryComparatorFactory[] rtreeCmpFactories, IBinaryComparatorFactory[] btreeCmpFactories,
-            ILinearizeComparatorFactory linearizer, int[] comparatorFields, IBinaryComparatorFactory[] linearizerArray,
-            ILSMMergePolicy mergePolicy, ILSMOperationTracker opTracker, ILSMIOOperationScheduler ioScheduler,
-            ILSMIOOperationCallback ioOpCallback, int[] buddyBTreeFields, boolean durable, boolean isPointMBR) {
+            ITreeIndexFrameFactory btreeLeafFrameFactory, IBufferCache diskBufferCache,
+            ILSMIndexFileManager fileNameManager, ILSMDiskComponentFactory componentFactory,
+            double bloomFilterFalsePositiveRate, IBinaryComparatorFactory[] rtreeCmpFactories,
+            IBinaryComparatorFactory[] btreeCmpFactories, ILinearizeComparatorFactory linearizer,
+            int[] comparatorFields, IBinaryComparatorFactory[] linearizerArray, ILSMMergePolicy mergePolicy,
+            ILSMOperationTracker opTracker, ILSMIOOperationScheduler ioScheduler, ILSMIOOperationCallback ioOpCallback,
+            int[] buddyBTreeFields, boolean durable, boolean isPointMBR) {
         super(ioManager, rtreeInteriorFrameFactory, rtreeLeafFrameFactory, btreeInteriorFrameFactory,
-                btreeLeafFrameFactory, fileNameManager,
-                new LSMRTreeDiskComponentFactory(diskRTreeFactory, diskBTreeFactory, bloomFilterFactory, null),
-                rtreeCmpFactories, btreeCmpFactories, linearizer, comparatorFields, linearizerArray,
-                bloomFilterFalsePositiveRate, mergePolicy, opTracker, ioScheduler, ioOpCallback, durable, isPointMBR,
-                diskRTreeFactory.getBufferCache());
+                btreeLeafFrameFactory, diskBufferCache, fileNameManager, componentFactory, rtreeCmpFactories,
+                btreeCmpFactories, linearizer, comparatorFields, linearizerArray, bloomFilterFalsePositiveRate,
+                mergePolicy, opTracker, ioScheduler, ioOpCallback, durable, isPointMBR);
         this.buddyBTreeFields = buddyBTreeFields;
     }
 
     @Override
-    protected ILSMDiskComponent loadComponent(LSMComponentFileReferences lsmComonentFileReferences)
-            throws HyracksDataException {
-        return createDiskComponent(componentFactory, lsmComonentFileReferences.getInsertIndexFileReference(),
-                lsmComonentFileReferences.getDeleteIndexFileReference(),
-                lsmComonentFileReferences.getBloomFilterFileReference(), false);
-    }
-
-    @Override
-    protected void deactivateDiskComponent(ILSMDiskComponent c) throws HyracksDataException {
-        LSMRTreeDiskComponent component = (LSMRTreeDiskComponent) c;
-        RTree rtree = component.getRTree();
-        BTree btree = component.getBTree();
-        BloomFilter bloomFilter = component.getBloomFilter();
-        rtree.deactivate();
-        btree.deactivate();
-        bloomFilter.deactivate();
-        rtree.purge();
-        btree.purge();
-        bloomFilter.purge();
-    }
-
-    @Override
-    protected void destroyDiskComponent(ILSMDiskComponent c) throws HyracksDataException {
-        LSMRTreeDiskComponent component = (LSMRTreeDiskComponent) c;
-        component.getBTree().destroy();
-        component.getBloomFilter().destroy();
-        component.getRTree().destroy();
-    }
-
-    @Override
-    protected void clearDiskComponent(ILSMDiskComponent c) throws HyracksDataException {
-        LSMRTreeDiskComponent component = (LSMRTreeDiskComponent) c;
-        component.getBTree().deactivate();
-        component.getBloomFilter().deactivate();
-        component.getRTree().deactivate();
-        component.getBTree().destroy();
-        component.getBloomFilter().destroy();
-        component.getRTree().destroy();
-    }
-
-    @Override
     public ILSMDiskComponent doFlush(ILSMIOOperation operation) throws HyracksDataException {
         LSMRTreeFlushOperation flushOp = (LSMRTreeFlushOperation) operation;
         LSMRTreeMemoryComponent flushingComponent = (LSMRTreeMemoryComponent) flushOp.getFlushingComponent();
@@ -170,20 +115,20 @@ public class LSMRTree extends AbstractLSMRTree {
         // The RTree should be renamed before the BTree.
 
         // scan the memory RTree
-        ITreeIndexAccessor memRTreeAccessor = flushingComponent.getRTree()
-                .createAccessor(NoOpOperationCallback.INSTANCE, NoOpOperationCallback.INSTANCE);
-        RTreeSearchCursor rtreeScanCursor = (RTreeSearchCursor) memRTreeAccessor.createSearchCursor(false);
+        RTreeAccessor memRTreeAccessor = flushingComponent.getIndex().createAccessor(NoOpOperationCallback.INSTANCE,
+                NoOpOperationCallback.INSTANCE);
+        RTreeSearchCursor rtreeScanCursor = memRTreeAccessor.createSearchCursor(false);
         SearchPredicate rtreeNullPredicate = new SearchPredicate(null, null);
         memRTreeAccessor.search(rtreeScanCursor, rtreeNullPredicate);
 
-        LSMRTreeDiskComponent component = createDiskComponent(componentFactory, flushOp.getTarget(),
+        ILSMDiskComponent component = createDiskComponent(componentFactory, flushOp.getTarget(),
                 flushOp.getBTreeTarget(), flushOp.getBloomFilterTarget(), true);
 
         //count the number of tuples in the buddy btree
-        ITreeIndexAccessor memBTreeAccessor = flushingComponent.getBTree()
+        BTreeAccessor memBTreeAccessor = flushingComponent.getBuddyIndex()
                 .createAccessor(NoOpOperationCallback.INSTANCE, NoOpOperationCallback.INSTANCE);
         RangePredicate btreeNullPredicate = new RangePredicate(null, null, true, true, null, null);
-        IIndexCursor btreeCountingCursor = ((BTreeAccessor) memBTreeAccessor).createCountingSearchCursor();
+        IIndexCursor btreeCountingCursor = memBTreeAccessor.createCountingSearchCursor();
         memBTreeAccessor.search(btreeCountingCursor, btreeNullPredicate);
         long numBTreeTuples = 0L;
         try {
@@ -197,14 +142,14 @@ public class LSMRTree extends AbstractLSMRTree {
         }
 
         ILSMDiskComponentBulkLoader componentBulkLoader =
-                createComponentBulkLoader(component, 1.0f, false, numBTreeTuples, false, false, false);
+                component.createBulkLoader(1.0f, false, numBTreeTuples, false, false, false);
 
         ITreeIndexCursor cursor;
         IBinaryComparatorFactory[] linearizerArray = { linearizer };
 
-        TreeTupleSorter rTreeTupleSorter = new TreeTupleSorter(flushingComponent.getRTree().getFileId(),
+        TreeTupleSorter rTreeTupleSorter = new TreeTupleSorter(flushingComponent.getIndex().getFileId(),
                 linearizerArray, rtreeLeafFrameFactory.createFrame(), rtreeLeafFrameFactory.createFrame(),
-                flushingComponent.getRTree().getBufferCache(), comparatorFields);
+                flushingComponent.getIndex().getBufferCache(), comparatorFields);
 
         // BulkLoad the tuples from the in-memory tree into the new disk
         // RTree.
@@ -252,7 +197,7 @@ public class LSMRTree extends AbstractLSMRTree {
             filterTuples.add(flushingComponent.getLSMComponentFilter().getMinTuple());
             filterTuples.add(flushingComponent.getLSMComponentFilter().getMaxTuple());
             getFilterManager().updateFilter(component.getLSMComponentFilter(), filterTuples);
-            getFilterManager().writeFilter(component.getLSMComponentFilter(), component.getRTree());
+            getFilterManager().writeFilter(component.getLSMComponentFilter(), component.getMetadataHolder());
         }
         // Note. If we change the filter to write to metadata object, we don't need the if block above
         flushingComponent.getMetadata().copy(component.getMetadata());
@@ -268,7 +213,8 @@ public class LSMRTree extends AbstractLSMRTree {
         ISearchPredicate rtreeSearchPred = new SearchPredicate(null, null);
         ILSMIndexOperationContext opCtx = ((LSMRTreeSortedCursor) cursor).getOpCtx();
         search(opCtx, cursor, rtreeSearchPred);
-        LSMRTreeDiskComponent mergedComponent = createDiskComponent(componentFactory, mergeOp.getTarget(),
+
+        ILSMDiskComponent mergedComponent = createDiskComponent(componentFactory, mergeOp.getTarget(),
                 mergeOp.getBTreeTarget(), mergeOp.getBloomFilterTarget(), true);
         ILSMDiskComponentBulkLoader componentBulkLoader;
         // In case we must keep the deleted-keys BTrees, then they must be merged *before* merging the r-trees so that
@@ -282,8 +228,7 @@ public class LSMRTree extends AbstractLSMRTree {
                 numElements += ((LSMRTreeDiskComponent) mergeOp.getMergingComponents().get(i)).getBloomFilter()
                         .getNumElements();
             }
-            componentBulkLoader =
-                    createComponentBulkLoader(mergedComponent, 1.0f, false, numElements, false, false, false);
+            componentBulkLoader = mergedComponent.createBulkLoader(1.0f, false, numElements, false, false, false);
 
             LSMRTreeDeletedKeysBTreeMergeCursor btreeCursor = new LSMRTreeDeletedKeysBTreeMergeCursor(opCtx);
             search(opCtx, btreeCursor, rtreeSearchPred);
@@ -298,7 +243,7 @@ public class LSMRTree extends AbstractLSMRTree {
             }
         } else {
             //no buddy-btree needed
-            componentBulkLoader = createComponentBulkLoader(mergedComponent, 1.0f, false, 0L, false, false, false);
+            componentBulkLoader = mergedComponent.createBulkLoader(1.0f, false, 0L, false, false, false);
         }
 
         //search old rtree components
@@ -319,7 +264,8 @@ public class LSMRTree extends AbstractLSMRTree {
                 filterTuples.add(mergeOp.getMergingComponents().get(i).getLSMComponentFilter().getMaxTuple());
             }
             getFilterManager().updateFilter(mergedComponent.getLSMComponentFilter(), filterTuples);
-            getFilterManager().writeFilter(mergedComponent.getLSMComponentFilter(), mergedComponent.getRTree());
+            getFilterManager().writeFilter(mergedComponent.getLSMComponentFilter(),
+                    mergedComponent.getMetadataHolder());
         }
 
         componentBulkLoader.end();
@@ -334,38 +280,6 @@ public class LSMRTree extends AbstractLSMRTree {
                 buddyBTreeFields);
     }
 
-    @Override
-    public ILSMDiskComponent createBulkLoadTarget() throws HyracksDataException {
-        LSMComponentFileReferences componentFileRefs = fileManager.getRelFlushFileReference();
-        return createDiskComponent(componentFactory, componentFileRefs.getInsertIndexFileReference(),
-                componentFileRefs.getDeleteIndexFileReference(), componentFileRefs.getBloomFilterFileReference(), true);
-    }
-
-    @Override
-    public ILSMDiskComponentBulkLoader createComponentBulkLoader(ILSMDiskComponent component, float fillFactor,
-            boolean verifyInput, long numElementsHint, boolean checkIfEmptyIndex, boolean withFilter,
-            boolean cleanupEmptyComponent) throws HyracksDataException {
-        BloomFilterSpecification bloomFilterSpec = null;
-        if (numElementsHint > 0) {
-            int maxBucketsPerElement = BloomCalculations.maxBucketsPerElement(numElementsHint);
-            bloomFilterSpec = BloomCalculations.computeBloomSpec(maxBucketsPerElement, bloomFilterFalsePositiveRate);
-        }
-        if (withFilter && filterFields != null) {
-            return new LSMRTreeDiskComponentBulkLoader((LSMRTreeDiskComponent) component, bloomFilterSpec, fillFactor,
-                    verifyInput, numElementsHint, checkIfEmptyIndex, cleanupEmptyComponent, filterManager, treeFields,
-                    filterFields, MultiComparator.create(component.getLSMComponentFilter().getFilterCmpFactories()));
-        } else {
-            return new LSMRTreeDiskComponentBulkLoader((LSMRTreeDiskComponent) component, bloomFilterSpec, fillFactor,
-                    verifyInput, numElementsHint, checkIfEmptyIndex, cleanupEmptyComponent);
-        }
-    }
-
-    @Override
-    public IIndexBulkLoader createBulkLoader(float fillLevel, boolean verifyInput, long numElementsHint)
-            throws HyracksDataException {
-        return new LSMRTreeBulkLoader(this, fillLevel, verifyInput, numElementsHint);
-    }
-
     // This function is modified for R-Trees without antimatter tuples to allow buddy B-Tree to have only primary keys
     @Override
     public void modify(IIndexOperationContext ictx, ITupleReference tuple) throws HyracksDataException {
@@ -404,16 +318,6 @@ public class LSMRTree extends AbstractLSMRTree {
     }
 
     @Override
-    public Set<String> getLSMComponentPhysicalFiles(ILSMComponent lsmComponent) {
-        Set<String> files = new HashSet<>();
-        LSMRTreeDiskComponent component = (LSMRTreeDiskComponent) lsmComponent;
-        files.add(component.getBTree().getFileReference().getFile().getAbsolutePath());
-        files.add(component.getRTree().getFileReference().getFile().getAbsolutePath());
-        files.add(component.getBloomFilter().getFileReference().getFile().getAbsolutePath());
-        return files;
-    }
-
-    @Override
     protected ILSMIOOperation createFlushOperation(AbstractLSMIndexOperationContext opCtx,
             LSMComponentFileReferences componentFileRefs, ILSMIOOperationCallback callback)
             throws HyracksDataException {

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/77f89525/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/impls/LSMRTreeAbstractCursor.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/impls/LSMRTreeAbstractCursor.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/impls/LSMRTreeAbstractCursor.java
index edb7481..7cc38f9 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/impls/LSMRTreeAbstractCursor.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/impls/LSMRTreeAbstractCursor.java
@@ -110,8 +110,8 @@ public abstract class LSMRTreeAbstractCursor implements ITreeIndexCursor {
                     //re-use
                     btreeCursors[i].reset();
                 }
-                rtree = ((LSMRTreeMemoryComponent) component).getRTree();
-                btree = ((LSMRTreeMemoryComponent) component).getBTree();
+                rtree = ((LSMRTreeMemoryComponent) component).getIndex();
+                btree = ((LSMRTreeMemoryComponent) component).getBuddyIndex();
             } else {
                 if (btreeCursors[i] == null || !btreeCursors[i].isBloomFilterAware()) {
                     // need to create a new one
@@ -124,8 +124,8 @@ public abstract class LSMRTreeAbstractCursor implements ITreeIndexCursor {
                             .resetBloomFilter(((LSMRTreeDiskComponent) operationalComponents.get(i)).getBloomFilter());
                     btreeCursors[i].reset();
                 }
-                rtree = ((LSMRTreeDiskComponent) component).getRTree();
-                btree = ((LSMRTreeDiskComponent) component).getBTree();
+                rtree = ((LSMRTreeDiskComponent) component).getIndex();
+                btree = ((LSMRTreeDiskComponent) component).getBuddyIndex();
             }
             if (rtreeCursors[i] == null) {
                 rtreeCursors[i] = new RTreeSearchCursor(
@@ -135,10 +135,10 @@ public abstract class LSMRTreeAbstractCursor implements ITreeIndexCursor {
                 rtreeCursors[i].reset();
             }
             if (rtreeAccessors[i] == null) {
-                rtreeAccessors[i] = (RTreeAccessor) rtree.createAccessor(NoOpOperationCallback.INSTANCE,
-                        NoOpOperationCallback.INSTANCE);
-                btreeAccessors[i] = (BTreeAccessor) btree.createAccessor(NoOpOperationCallback.INSTANCE,
-                        NoOpOperationCallback.INSTANCE);
+                rtreeAccessors[i] =
+                        rtree.createAccessor(NoOpOperationCallback.INSTANCE, NoOpOperationCallback.INSTANCE);
+                btreeAccessors[i] =
+                        btree.createAccessor(NoOpOperationCallback.INSTANCE, NoOpOperationCallback.INSTANCE);
             } else {
                 rtreeAccessors[i].reset(rtree, NoOpOperationCallback.INSTANCE);
                 btreeAccessors[i].reset(btree, NoOpOperationCallback.INSTANCE, NoOpOperationCallback.INSTANCE);

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/77f89525/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/impls/LSMRTreeBulkLoader.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/impls/LSMRTreeBulkLoader.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/impls/LSMRTreeBulkLoader.java
deleted file mode 100644
index b30e2fa..0000000
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/impls/LSMRTreeBulkLoader.java
+++ /dev/null
@@ -1,60 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.hyracks.storage.am.lsm.rtree.impls;
-
-import org.apache.hyracks.api.exceptions.HyracksDataException;
-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.LSMOperationType;
-import org.apache.hyracks.storage.common.IIndexBulkLoader;
-
-public class LSMRTreeBulkLoader implements IIndexBulkLoader {
-    private final ILSMDiskComponent component;
-    private final LSMRTree lsmIndex;
-    private final IIndexBulkLoader componentBulkLoader;
-
-    public LSMRTreeBulkLoader(LSMRTree lsmIndex, float fillFactor, boolean verifyInput, long numElementsHint)
-            throws HyracksDataException {
-        this.lsmIndex = lsmIndex;
-        // Note that by using a flush target file name, we state that the
-        // new bulk loaded tree is "newer" than any other merged tree.
-        this.component = lsmIndex.createBulkLoadTarget();
-        this.componentBulkLoader = lsmIndex.createComponentBulkLoader(component, fillFactor, verifyInput,
-                numElementsHint, false, true, true);
-    }
-
-    @Override
-    public void add(ITupleReference tuple) throws HyracksDataException {
-        componentBulkLoader.add(tuple);
-    }
-
-    @Override
-    public void end() throws HyracksDataException {
-        componentBulkLoader.end();
-        if (component.getComponentSize() > 0) {
-            lsmIndex.getIOOperationCallback().afterOperation(LSMOperationType.FLUSH, null, component);
-            lsmIndex.getLsmHarness().addBulkLoadedComponent(component);
-        }
-    }
-
-    @Override
-    public void abort() throws HyracksDataException {
-        componentBulkLoader.abort();
-    }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/77f89525/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/impls/LSMRTreeDeletedKeysBTreeMergeCursor.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/impls/LSMRTreeDeletedKeysBTreeMergeCursor.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/impls/LSMRTreeDeletedKeysBTreeMergeCursor.java
index dc140a2..d20b1bd 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/impls/LSMRTreeDeletedKeysBTreeMergeCursor.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/impls/LSMRTreeDeletedKeysBTreeMergeCursor.java
@@ -61,7 +61,7 @@ public class LSMRTreeDeletedKeysBTreeMergeCursor extends LSMIndexSearchCursor {
             ILSMComponent component = operationalComponents.get(i);
             IBTreeLeafFrame leafFrame = (IBTreeLeafFrame) lsmInitialState.getBTreeLeafFrameFactory().createFrame();
             rangeCursors[i] = new BTreeRangeSearchCursor(leafFrame, false);
-            BTree btree = ((LSMRTreeDiskComponent) component).getBTree();
+            BTree btree = ((LSMRTreeDiskComponent) component).getBuddyIndex();
             btreeAccessors[i] = btree.createAccessor(NoOpOperationCallback.INSTANCE, NoOpOperationCallback.INSTANCE);
             btreeAccessors[i].search(rangeCursors[i], btreePredicate);
         }

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/77f89525/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/impls/LSMRTreeDiskComponent.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/impls/LSMRTreeDiskComponent.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/impls/LSMRTreeDiskComponent.java
index 0f7943d..e04868d 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/impls/LSMRTreeDiskComponent.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/impls/LSMRTreeDiskComponent.java
@@ -18,79 +18,92 @@
  */
 package org.apache.hyracks.storage.am.lsm.rtree.impls;
 
-import org.apache.hyracks.api.exceptions.HyracksDataException;
+import java.util.HashSet;
+import java.util.Set;
+
 import org.apache.hyracks.storage.am.bloomfilter.impls.BloomFilter;
 import org.apache.hyracks.storage.am.btree.impls.BTree;
 import org.apache.hyracks.storage.am.common.api.IMetadataPageManager;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponentFilter;
-import org.apache.hyracks.storage.am.lsm.common.impls.AbstractLSMDiskComponent;
-import org.apache.hyracks.storage.am.lsm.common.util.ComponentUtils;
+import org.apache.hyracks.storage.am.lsm.common.api.AbstractLSMWithBuddyDiskComponent;
+import org.apache.hyracks.storage.am.lsm.common.impls.AbstractLSMIndex;
 import org.apache.hyracks.storage.am.rtree.impls.RTree;
+import org.apache.hyracks.storage.common.buffercache.IBufferCache;
 
-public class LSMRTreeDiskComponent extends AbstractLSMDiskComponent {
+public class LSMRTreeDiskComponent extends AbstractLSMWithBuddyDiskComponent {
     private final RTree rtree;
     private final BTree btree;
     private final BloomFilter bloomFilter;
 
-    public LSMRTreeDiskComponent(RTree rtree, BTree btree, BloomFilter bloomFilter, ILSMComponentFilter filter) {
-        super((IMetadataPageManager) rtree.getPageManager(), filter);
+    public LSMRTreeDiskComponent(AbstractLSMIndex lsmIndex, RTree rtree, BTree btree, BloomFilter bloomFilter,
+            ILSMComponentFilter filter) {
+        super(lsmIndex, getMetadataPageManager(rtree), filter);
         this.rtree = rtree;
         this.btree = btree;
         this.bloomFilter = bloomFilter;
     }
 
     @Override
-    public void destroy() throws HyracksDataException {
-        rtree.deactivate();
-        rtree.destroy();
-        if (btree != null) {
-            btree.deactivate();
-            btree.destroy();
-            bloomFilter.deactivate();
-            bloomFilter.destroy();
-        }
-    }
-
-    public RTree getRTree() {
-        return rtree;
-    }
-
-    public BTree getBTree() {
+    public BTree getBuddyIndex() {
         return btree;
     }
 
+    @Override
     public BloomFilter getBloomFilter() {
         return bloomFilter;
     }
 
     @Override
+    public IBufferCache getBloomFilterBufferCache() {
+        return btree.getBufferCache();
+    }
+
+    @Override
     public long getComponentSize() {
-        long size = rtree.getFileReference().getFile().length();
-        if (btree != null) {
-            size += btree.getFileReference().getFile().length();
-            size += bloomFilter.getFileReference().getFile().length();
-        }
+        long size = getComponentSize(rtree);
+        size += btree.getFileReference().getFile().length();
+        size += bloomFilter.getFileReference().getFile().length();
         return size;
     }
 
     @Override
+    public Set<String> getLSMComponentPhysicalFiles() {
+        Set<String> files = getFiles(rtree);
+        files.add(btree.getFileReference().getFile().getAbsolutePath());
+        files.add(bloomFilter.getFileReference().getFile().getAbsolutePath());
+        return files;
+    }
+
+    @Override
     public int getFileReferenceCount() {
-        return rtree.getBufferCache().getFileReferenceCount(rtree.getFileId());
+        return getFileReferenceCount(rtree);
     }
 
     @Override
-    public String toString() {
-        return getClass().getSimpleName() + ":" + rtree.getFileReference().getRelativePath();
+    public RTree getMetadataHolder() {
+        return rtree;
     }
 
     @Override
-    public void markAsValid(boolean persist) throws HyracksDataException {
-        if (bloomFilter != null) {
-            ComponentUtils.markAsValid(btree.getBufferCache(), bloomFilter, persist);
-        }
-        if (btree != null) {
-            ComponentUtils.markAsValid(btree, persist);
-        }
-        ComponentUtils.markAsValid(rtree, persist);
+    public RTree getIndex() {
+        return rtree;
+    }
+
+    static IMetadataPageManager getMetadataPageManager(RTree rtree) {
+        return (IMetadataPageManager) rtree.getPageManager();
+    }
+
+    static long getComponentSize(RTree rtree) {
+        return rtree.getFileReference().getFile().length();
+    }
+
+    static int getFileReferenceCount(RTree rtree) {
+        return rtree.getBufferCache().getFileReferenceCount(rtree.getFileId());
+    }
+
+    static Set<String> getFiles(RTree rtree) {
+        Set<String> files = new HashSet<>();
+        files.add(rtree.getFileReference().getFile().getAbsolutePath());
+        return files;
     }
 }

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/77f89525/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/impls/LSMRTreeDiskComponentBulkLoader.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/impls/LSMRTreeDiskComponentBulkLoader.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/impls/LSMRTreeDiskComponentBulkLoader.java
deleted file mode 100644
index e72918a..0000000
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/impls/LSMRTreeDiskComponentBulkLoader.java
+++ /dev/null
@@ -1,65 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.hyracks.storage.am.lsm.rtree.impls;
-
-import org.apache.hyracks.api.exceptions.HyracksDataException;
-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.ITreeIndex;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponentFilterManager;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMDiskComponent;
-import org.apache.hyracks.storage.am.lsm.common.impls.AbstractLSMDiskComponentWithBuddyBulkLoader;
-import org.apache.hyracks.storage.common.IIndex;
-import org.apache.hyracks.storage.common.MultiComparator;
-
-public class LSMRTreeDiskComponentBulkLoader extends AbstractLSMDiskComponentWithBuddyBulkLoader {
-
-    //with filter
-    public LSMRTreeDiskComponentBulkLoader(LSMRTreeDiskComponent component, BloomFilterSpecification bloomFilterSpec,
-            float fillFactor, boolean verifyInput, long numElementsHint, boolean checkIfEmptyIndex,
-            boolean cleanupEmptyComponent, ILSMComponentFilterManager filterManager, int[] indexFields,
-            int[] filterFields, MultiComparator filterCmp) throws HyracksDataException {
-        super(component, bloomFilterSpec, fillFactor, verifyInput, numElementsHint, checkIfEmptyIndex,
-                cleanupEmptyComponent, filterManager, indexFields, filterFields, filterCmp);
-    }
-
-    //without filter
-    public LSMRTreeDiskComponentBulkLoader(LSMRTreeDiskComponent component, BloomFilterSpecification bloomFilterSpec,
-            float fillFactor, boolean verifyInput, long numElementsHint, boolean checkIfEmptyIndex,
-            boolean cleanupEmptyComponent) throws HyracksDataException {
-        super(component, bloomFilterSpec, fillFactor, verifyInput, numElementsHint, checkIfEmptyIndex,
-                cleanupEmptyComponent, null, null, null, null);
-    }
-
-    @Override
-    protected BloomFilter getBloomFilter(ILSMDiskComponent component) {
-        return ((LSMRTreeDiskComponent) component).getBloomFilter();
-    }
-
-    @Override
-    protected IIndex getIndex(ILSMDiskComponent component) {
-        return ((LSMRTreeDiskComponent) component).getRTree();
-    }
-
-    @Override
-    protected ITreeIndex getBuddyBTree(ILSMDiskComponent component) {
-        return ((LSMRTreeDiskComponent) component).getBTree();
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/77f89525/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/impls/LSMRTreeDiskComponentFactory.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/impls/LSMRTreeDiskComponentFactory.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/impls/LSMRTreeDiskComponentFactory.java
index c3d8282..ae25e93 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/impls/LSMRTreeDiskComponentFactory.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/impls/LSMRTreeDiskComponentFactory.java
@@ -24,6 +24,7 @@ import org.apache.hyracks.storage.am.bloomfilter.impls.BloomFilterFactory;
 import org.apache.hyracks.storage.am.btree.impls.BTree;
 import org.apache.hyracks.storage.am.lsm.common.api.IComponentFilterHelper;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMDiskComponentFactory;
+import org.apache.hyracks.storage.am.lsm.common.impls.AbstractLSMIndex;
 import org.apache.hyracks.storage.am.lsm.common.impls.LSMComponentFileReferences;
 import org.apache.hyracks.storage.am.lsm.common.impls.TreeIndexFactory;
 import org.apache.hyracks.storage.am.rtree.impls.RTree;
@@ -43,8 +44,9 @@ public class LSMRTreeDiskComponentFactory implements ILSMDiskComponentFactory {
     }
 
     @Override
-    public LSMRTreeDiskComponent createComponent(LSMComponentFileReferences cfr) throws HyracksDataException {
-        return new LSMRTreeDiskComponent(rtreeFactory.createIndexInstance(cfr.getInsertIndexFileReference()),
+    public LSMRTreeDiskComponent createComponent(AbstractLSMIndex lsmIndex, LSMComponentFileReferences cfr)
+            throws HyracksDataException {
+        return new LSMRTreeDiskComponent(lsmIndex, rtreeFactory.createIndexInstance(cfr.getInsertIndexFileReference()),
                 btreeFactory.createIndexInstance(cfr.getDeleteIndexFileReference()),
                 bloomFilterFactory.createBloomFiltertInstance(cfr.getBloomFilterFileReference()),
                 filterHelper == null ? null : filterHelper.createFilter());

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/77f89525/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/impls/LSMRTreeMemoryComponent.java
----------------------------------------------------------------------
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 957e2a9..abb3ce1 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
@@ -22,11 +22,11 @@ package org.apache.hyracks.storage.am.lsm.rtree.impls;
 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.ILSMComponentFilter;
+import org.apache.hyracks.storage.am.lsm.common.api.AbstractLSMWithBuddyMemoryComponent;
 import org.apache.hyracks.storage.am.lsm.common.api.IVirtualBufferCache;
-import org.apache.hyracks.storage.am.lsm.common.impls.AbstractLSMMemoryComponent;
 import org.apache.hyracks.storage.am.rtree.impls.RTree;
 
-public class LSMRTreeMemoryComponent extends AbstractLSMMemoryComponent {
+public class LSMRTreeMemoryComponent extends AbstractLSMWithBuddyMemoryComponent {
 
     private final RTree rtree;
     private final BTree btree;
@@ -38,27 +38,19 @@ public class LSMRTreeMemoryComponent extends AbstractLSMMemoryComponent {
         this.btree = btree;
     }
 
-    public RTree getRTree() {
+    @Override
+    public RTree getIndex() {
         return rtree;
     }
 
-    public BTree getBTree() {
+    @Override
+    public BTree getBuddyIndex() {
         return btree;
     }
 
     @Override
-    public void reset() throws HyracksDataException {
-        super.reset();
-        rtree.deactivate();
-        rtree.destroy();
-        rtree.create();
-        rtree.activate();
-        if (btree != null) {
-            btree.deactivate();
-            btree.destroy();
-            btree.create();
-            btree.activate();
-        }
+    public void validate() throws HyracksDataException {
+        throw new UnsupportedOperationException("Validation not implemented for LSM R-Trees.");
     }
 
 }

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/77f89525/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/impls/LSMRTreeOpContext.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/impls/LSMRTreeOpContext.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/impls/LSMRTreeOpContext.java
index 670406e..74d9685 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/impls/LSMRTreeOpContext.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/impls/LSMRTreeOpContext.java
@@ -62,14 +62,14 @@ public final class LSMRTreeOpContext extends AbstractLSMIndexOperationContext {
         for (int i = 0; i < mutableComponents.size(); i++) {
             LSMRTreeMemoryComponent mutableComponent = (LSMRTreeMemoryComponent) mutableComponents.get(i);
             if (allFields != null) {
-                mutableRTreeAccessors[i] = (RTree.RTreeAccessor) mutableComponent.getRTree()
-                        .createAccessor(NoOpOperationCallback.INSTANCE, NoOpOperationCallback.INSTANCE, allFields);
+                mutableRTreeAccessors[i] = mutableComponent.getIndex().createAccessor(NoOpOperationCallback.INSTANCE,
+                        NoOpOperationCallback.INSTANCE, allFields);
             } else {
-                mutableRTreeAccessors[i] = (RTree.RTreeAccessor) mutableComponent.getRTree()
-                        .createAccessor(NoOpOperationCallback.INSTANCE, NoOpOperationCallback.INSTANCE);
+                mutableRTreeAccessors[i] = mutableComponent.getIndex().createAccessor(NoOpOperationCallback.INSTANCE,
+                        NoOpOperationCallback.INSTANCE);
             }
-            mutableBTreeAccessors[i] = (BTree.BTreeAccessor) mutableComponent.getBTree()
-                    .createAccessor(NoOpOperationCallback.INSTANCE, NoOpOperationCallback.INSTANCE);
+            mutableBTreeAccessors[i] = mutableComponent.getBuddyIndex().createAccessor(NoOpOperationCallback.INSTANCE,
+                    NoOpOperationCallback.INSTANCE);
 
             rtreeOpContexts[i] = mutableRTreeAccessors[i].getOpContext();
             btreeOpContexts[i] = mutableBTreeAccessors[i].getOpContext();


[4/6] asterixdb git commit: [NO ISSUE][STO][IDX] LSM storage cleanup

Posted by il...@apache.org.
http://git-wip-us.apache.org/repos/asf/asterixdb/blob/77f89525/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/ILSMMemoryComponent.java
----------------------------------------------------------------------
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 18121e0..13543e4 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
@@ -65,7 +65,7 @@ public interface ILSMMemoryComponent extends ILSMComponent {
     /**
      * request the component to be active
      */
-    void activate();
+    void requestActivation();
 
     /**
      * Set the component state
@@ -74,4 +74,31 @@ public interface ILSMMemoryComponent extends ILSMComponent {
      *            the new state
      */
     void setState(ComponentState state);
+
+    /**
+     * Allocates memory to this component, create and activate it
+     *
+     * @throws HyracksDataException
+     */
+    void allocate() throws HyracksDataException;
+
+    /**
+     * Deactivete the memory component, destroy it, and deallocates its memory
+     *
+     * @throws HyracksDataException
+     */
+    void deallocate() throws HyracksDataException;
+
+    /**
+     * Test method
+     * TODO: Get rid of it
+     *
+     * @throws HyracksDataException
+     */
+    void validate() throws HyracksDataException;
+
+    /**
+     * @return the size of the memory component
+     */
+    long getSize();
 }

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/77f89525/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/AbstractLSMDiskComponent.java
----------------------------------------------------------------------
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 4386d52..280cc52 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
@@ -25,18 +25,27 @@ import org.apache.hyracks.storage.am.lsm.common.api.ILSMDiskComponent;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMDiskComponentId;
 import org.apache.hyracks.storage.am.lsm.common.api.LSMOperationType;
 import org.apache.hyracks.storage.am.lsm.common.util.ComponentUtils;
+import org.apache.hyracks.storage.common.MultiComparator;
 
 public abstract class AbstractLSMDiskComponent extends AbstractLSMComponent implements ILSMDiskComponent {
 
     private final DiskComponentMetadata metadata;
+    private final AbstractLSMIndex lsmIndex;
 
-    public AbstractLSMDiskComponent(IMetadataPageManager mdPageManager, ILSMComponentFilter filter) {
+    public AbstractLSMDiskComponent(AbstractLSMIndex lsmIndex, IMetadataPageManager mdPageManager,
+            ILSMComponentFilter filter) {
         super(filter);
+        this.lsmIndex = lsmIndex;
         state = ComponentState.READABLE_UNWRITABLE;
         metadata = new DiskComponentMetadata(mdPageManager);
     }
 
     @Override
+    public AbstractLSMIndex getLsmIndex() {
+        return lsmIndex;
+    }
+
+    @Override
     public boolean threadEnter(LSMOperationType opType, boolean isMutableComponent) {
         if (state == ComponentState.INACTIVE) {
             throw new IllegalStateException("Trying to enter an inactive disk component");
@@ -110,4 +119,82 @@ public abstract class AbstractLSMDiskComponent extends AbstractLSMComponent impl
         //TODO: do we need to throw an exception when ID is not found?
         return new LSMDiskComponentId(minID, maxID);
     }
+
+    /**
+     * Mark the component as valid
+     *
+     * @param persist
+     *            whether the call should force data to disk before returning
+     * @throws HyracksDataException
+     */
+    @Override
+    public void markAsValid(boolean persist) throws HyracksDataException {
+        ComponentUtils.markAsValid(getMetadataHolder(), persist);
+    }
+
+    @Override
+    public void activate(boolean createNewComponent) throws HyracksDataException {
+        if (createNewComponent) {
+            getIndex().create();
+        }
+        getIndex().activate();
+        if (getLSMComponentFilter() != null && !createNewComponent) {
+            getLsmIndex().getFilterManager().readFilter(getLSMComponentFilter(), getMetadataHolder());
+        }
+    }
+
+    @Override
+    public void deactivateAndDestroy() throws HyracksDataException {
+        getIndex().deactivate();
+        getIndex().destroy();
+    }
+
+    @Override
+    public void destroy() throws HyracksDataException {
+        getIndex().destroy();
+    }
+
+    @Override
+    public void deactivate() throws HyracksDataException {
+        getIndex().deactivate();
+    }
+
+    @Override
+    public void deactivateAndPurge() throws HyracksDataException {
+        getIndex().deactivate();
+        getIndex().purge();
+    }
+
+    @Override
+    public void validate() throws HyracksDataException {
+        getIndex().validate();
+    }
+
+    @Override
+    public IChainedComponentBulkLoader createFilterBulkLoader() throws HyracksDataException {
+        return new FilterBulkLoader(getLSMComponentFilter(), getMetadataHolder(), getLsmIndex().getFilterManager(),
+                getLsmIndex().getTreeFields(), getLsmIndex().getFilterFields(),
+                MultiComparator.create(getLSMComponentFilter().getFilterCmpFactories()));
+    }
+
+    @Override
+    public IChainedComponentBulkLoader createIndexBulkLoader(float fillFactor, boolean verifyInput,
+            long numElementsHint, boolean checkIfEmptyIndex) throws HyracksDataException {
+        return new LSMIndexBulkLoader(
+                getIndex().createBulkLoader(fillFactor, verifyInput, numElementsHint, checkIfEmptyIndex));
+    }
+
+    @Override
+    public ChainedLSMDiskComponentBulkLoader createBulkLoader(float fillFactor, boolean verifyInput,
+            long numElementsHint, boolean checkIfEmptyIndex, boolean withFilter, boolean cleanupEmptyComponent)
+            throws HyracksDataException {
+        ChainedLSMDiskComponentBulkLoader chainedBulkLoader =
+                new ChainedLSMDiskComponentBulkLoader(this, cleanupEmptyComponent);
+        if (withFilter && getLsmIndex().getFilterFields() != null) {
+            chainedBulkLoader.addBulkLoader(createFilterBulkLoader());
+        }
+        chainedBulkLoader
+                .addBulkLoader(createIndexBulkLoader(fillFactor, verifyInput, numElementsHint, checkIfEmptyIndex));
+        return chainedBulkLoader;
+    }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/77f89525/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/AbstractLSMDiskComponentBulkLoader.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/AbstractLSMDiskComponentBulkLoader.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/AbstractLSMDiskComponentBulkLoader.java
deleted file mode 100644
index 351f619..0000000
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/AbstractLSMDiskComponentBulkLoader.java
+++ /dev/null
@@ -1,202 +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.dataflow.common.data.accessors.ITupleReference;
-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.ITreeIndex;
-import org.apache.hyracks.storage.am.common.impls.AbstractTreeIndex.AbstractTreeIndexBulkLoader;
-import org.apache.hyracks.storage.am.common.tuples.PermutingTupleReference;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponentFilterManager;
-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.ILSMTreeTupleWriter;
-import org.apache.hyracks.storage.common.IIndex;
-import org.apache.hyracks.storage.common.IIndexBulkLoader;
-import org.apache.hyracks.storage.common.MultiComparator;
-
-public abstract class AbstractLSMDiskComponentBulkLoader implements ILSMDiskComponentBulkLoader {
-    protected final ILSMDiskComponent component;
-
-    protected final IIndexBulkLoader indexBulkLoader;
-    protected final IIndexBulkLoader bloomFilterBuilder;
-
-    protected final ILSMComponentFilterManager filterManager;
-    protected final PermutingTupleReference indexTuple;
-    protected final PermutingTupleReference filterTuple;
-    protected final MultiComparator filterCmp;
-    protected final boolean cleanupEmptyComponent;
-
-    protected boolean cleanedUpArtifacts = false;
-    protected boolean isEmptyComponent = true;
-    protected boolean endedBloomFilterLoad = false;
-
-    //with filter
-    public AbstractLSMDiskComponentBulkLoader(ILSMDiskComponent component, BloomFilterSpecification bloomFilterSpec,
-            float fillFactor, boolean verifyInput, long numElementsHint, boolean checkIfEmptyIndex,
-            boolean cleanupEmptyComponent, ILSMComponentFilterManager filterManager, int[] indexFields,
-            int[] filterFields, MultiComparator filterCmp) throws HyracksDataException {
-        this.component = component;
-        this.indexBulkLoader =
-                getIndex(component).createBulkLoader(fillFactor, verifyInput, numElementsHint, checkIfEmptyIndex);
-        if (bloomFilterSpec != null) {
-            this.bloomFilterBuilder = getBloomFilter(component).createBuilder(numElementsHint,
-                    bloomFilterSpec.getNumHashes(), bloomFilterSpec.getNumBucketsPerElements());
-        } else {
-            this.bloomFilterBuilder = null;
-        }
-        this.cleanupEmptyComponent = cleanupEmptyComponent;
-        if (filterManager != null) {
-            this.filterManager = filterManager;
-            this.indexTuple = new PermutingTupleReference(indexFields);
-            this.filterTuple = new PermutingTupleReference(filterFields);
-            this.filterCmp = filterCmp;
-        } else {
-            this.filterManager = null;
-            this.indexTuple = null;
-            this.filterTuple = null;
-            this.filterCmp = null;
-        }
-    }
-
-    @Override
-    public void add(ITupleReference tuple) throws HyracksDataException {
-        try {
-            ITupleReference t;
-            if (indexTuple != null) {
-                indexTuple.reset(tuple);
-                t = indexTuple;
-            } else {
-                t = tuple;
-            }
-
-            indexBulkLoader.add(t);
-            if (bloomFilterBuilder != null) {
-                bloomFilterBuilder.add(t);
-            }
-            updateFilter(tuple);
-
-        } catch (Exception e) {
-            cleanupArtifacts();
-            throw e;
-        }
-        if (isEmptyComponent) {
-            isEmptyComponent = false;
-        }
-    }
-
-    @Override
-    public void delete(ITupleReference tuple) throws HyracksDataException {
-        ILSMTreeTupleWriter tupleWriter =
-                (ILSMTreeTupleWriter) ((AbstractTreeIndexBulkLoader) indexBulkLoader).getLeafFrame().getTupleWriter();
-        tupleWriter.setAntimatter(true);
-        try {
-            ITupleReference t;
-            if (indexTuple != null) {
-                indexTuple.reset(tuple);
-                t = indexTuple;
-            } else {
-                t = tuple;
-            }
-
-            indexBulkLoader.add(t);
-
-            updateFilter(tuple);
-        } catch (Exception e) {
-            cleanupArtifacts();
-            throw e;
-        } finally {
-            tupleWriter.setAntimatter(false);
-        }
-        if (isEmptyComponent) {
-            isEmptyComponent = false;
-        }
-    }
-
-    @Override
-    public void abort() throws HyracksDataException {
-        if (indexBulkLoader != null) {
-            indexBulkLoader.abort();
-        }
-        if (bloomFilterBuilder != null) {
-            bloomFilterBuilder.abort();
-        }
-
-    }
-
-    @Override
-    public void end() throws HyracksDataException {
-        if (!cleanedUpArtifacts) {
-            if (bloomFilterBuilder != null && !endedBloomFilterLoad) {
-                bloomFilterBuilder.end();
-                endedBloomFilterLoad = true;
-            }
-
-            //use filter
-            if (filterManager != null && component.getLSMComponentFilter() != null) {
-                filterManager.writeFilter(component.getLSMComponentFilter(), getTreeIndex(component));
-            }
-            indexBulkLoader.end();
-
-            if (isEmptyComponent && cleanupEmptyComponent) {
-                cleanupArtifacts();
-            }
-        }
-    }
-
-    protected void cleanupArtifacts() throws HyracksDataException {
-        if (!cleanedUpArtifacts) {
-            cleanedUpArtifacts = true;
-            if (bloomFilterBuilder != null && !endedBloomFilterLoad) {
-                bloomFilterBuilder.abort();
-                endedBloomFilterLoad = true;
-            }
-            getIndex(component).deactivate();
-            getIndex(component).destroy();
-            if (bloomFilterBuilder != null) {
-                getBloomFilter(component).deactivate();
-                getBloomFilter(component).destroy();
-            }
-        }
-    }
-
-    protected void updateFilter(ITupleReference tuple) throws HyracksDataException {
-        if (filterTuple != null) {
-            filterTuple.reset(tuple);
-            component.getLSMComponentFilter().update(filterTuple, filterCmp);
-        }
-    }
-
-    /**
-     * TreeIndex is used to hold the filter tuple values
-     *
-     * @param component
-     * @return
-     */
-    protected ITreeIndex getTreeIndex(ILSMDiskComponent component) {
-        return (ITreeIndex) getIndex(component);
-    }
-
-    protected abstract IIndex getIndex(ILSMDiskComponent component);
-
-    protected abstract BloomFilter getBloomFilter(ILSMDiskComponent component);
-
-}

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/77f89525/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/AbstractLSMDiskComponentWithBuddyBulkLoader.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/AbstractLSMDiskComponentWithBuddyBulkLoader.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/AbstractLSMDiskComponentWithBuddyBulkLoader.java
deleted file mode 100644
index fc7310c..0000000
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/AbstractLSMDiskComponentWithBuddyBulkLoader.java
+++ /dev/null
@@ -1,156 +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.ErrorCode;
-import org.apache.hyracks.api.exceptions.HyracksDataException;
-import org.apache.hyracks.dataflow.common.data.accessors.ITupleReference;
-import org.apache.hyracks.storage.am.bloomfilter.impls.BloomFilterSpecification;
-import org.apache.hyracks.storage.am.common.api.ITreeIndex;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponentFilterManager;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMDiskComponent;
-import org.apache.hyracks.storage.common.IIndexBulkLoader;
-import org.apache.hyracks.storage.common.MultiComparator;
-
-public abstract class AbstractLSMDiskComponentWithBuddyBulkLoader extends AbstractLSMDiskComponentBulkLoader {
-
-    protected final IIndexBulkLoader buddyBTreeBulkLoader;
-
-    //with filter
-    public AbstractLSMDiskComponentWithBuddyBulkLoader(ILSMDiskComponent component,
-            BloomFilterSpecification bloomFilterSpec, float fillFactor, boolean verifyInput, long numElementsHint,
-            boolean checkIfEmptyIndex, boolean cleanupEmptyComponent, ILSMComponentFilterManager filterManager,
-            int[] indexFields, int[] filterFields, MultiComparator filterCmp) throws HyracksDataException {
-        super(component, bloomFilterSpec, fillFactor, verifyInput, numElementsHint, checkIfEmptyIndex,
-                cleanupEmptyComponent, filterManager, indexFields, filterFields, filterCmp);
-
-        // BuddyBTree must be created even if it could be empty,
-        // since without it the component is not considered as valid.
-        buddyBTreeBulkLoader =
-                getBuddyBTree(component).createBulkLoader(fillFactor, verifyInput, numElementsHint, checkIfEmptyIndex);
-    }
-
-    @Override
-    public void add(ITupleReference tuple) throws HyracksDataException {
-        try {
-            ITupleReference t;
-            if (indexTuple != null) {
-                indexTuple.reset(tuple);
-                t = indexTuple;
-            } else {
-                t = tuple;
-            }
-
-            indexBulkLoader.add(t);
-
-            updateFilter(tuple);
-        } catch (Exception e) {
-            cleanupArtifacts();
-            throw e;
-        }
-        if (isEmptyComponent) {
-            isEmptyComponent = false;
-        }
-    }
-
-    @Override
-    public void delete(ITupleReference tuple) throws HyracksDataException {
-        try {
-            ITupleReference t;
-            if (indexTuple != null) {
-                indexTuple.reset(tuple);
-                t = indexTuple;
-            } else {
-                t = tuple;
-            }
-
-            buddyBTreeBulkLoader.add(t);
-            if (bloomFilterBuilder != null) {
-                bloomFilterBuilder.add(t);
-            }
-
-            updateFilter(tuple);
-        } catch (HyracksDataException e) {
-            //deleting a key multiple times is OK
-            if (e.getErrorCode() != ErrorCode.DUPLICATE_KEY) {
-                cleanupArtifacts();
-                throw e;
-            }
-        } catch (Exception e) {
-            cleanupArtifacts();
-            throw e;
-        }
-        if (isEmptyComponent) {
-            isEmptyComponent = false;
-        }
-    }
-
-    @Override
-    public void abort() throws HyracksDataException {
-        super.abort();
-        if (buddyBTreeBulkLoader != null) {
-            buddyBTreeBulkLoader.abort();
-        }
-    }
-
-    @Override
-    public void end() throws HyracksDataException {
-        if (!cleanedUpArtifacts) {
-            if (bloomFilterBuilder != null && !endedBloomFilterLoad) {
-                bloomFilterBuilder.end();
-                endedBloomFilterLoad = true;
-            }
-
-            //use filter
-            if (filterManager != null && component.getLSMComponentFilter() != null) {
-                filterManager.writeFilter(component.getLSMComponentFilter(), getTreeIndex(component));
-            }
-            indexBulkLoader.end();
-            buddyBTreeBulkLoader.end();
-
-            if (isEmptyComponent && cleanupEmptyComponent) {
-                cleanupArtifacts();
-            }
-        }
-    }
-
-    @Override
-    protected void cleanupArtifacts() throws HyracksDataException {
-        if (!cleanedUpArtifacts) {
-            cleanedUpArtifacts = true;
-            if (bloomFilterBuilder != null && !endedBloomFilterLoad) {
-                bloomFilterBuilder.abort();
-                endedBloomFilterLoad = true;
-            }
-            getIndex(component).deactivate();
-            getIndex(component).destroy();
-
-            getBuddyBTree(component).deactivate();
-            getBuddyBTree(component).destroy();
-
-            if (bloomFilterBuilder != null) {
-                getBloomFilter(component).deactivate();
-                getBloomFilter(component).destroy();
-            }
-        }
-    }
-
-    protected abstract ITreeIndex getBuddyBTree(ILSMDiskComponent component);
-
-}

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/77f89525/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/AbstractLSMIndex.java
----------------------------------------------------------------------
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 dc64f9b..50c7720 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
@@ -31,6 +31,7 @@ import java.util.concurrent.atomic.AtomicInteger;
 import org.apache.hyracks.api.dataflow.value.IBinaryComparatorFactory;
 import org.apache.hyracks.api.exceptions.ErrorCode;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.io.FileReference;
 import org.apache.hyracks.api.io.IIOManager;
 import org.apache.hyracks.api.replication.IReplicationJob.ReplicationExecutionType;
 import org.apache.hyracks.api.replication.IReplicationJob.ReplicationOperation;
@@ -43,6 +44,7 @@ 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.ILSMDiskComponent;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMDiskComponentFactory;
 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;
@@ -90,11 +92,16 @@ public abstract class AbstractLSMIndex implements ILSMIndex {
     protected final AtomicBoolean[] flushRequests;
     protected boolean memoryComponentsAllocated = false;
     protected ITracer tracer;
+    // Factory for creating on-disk index components during flush and merge.
+    protected final ILSMDiskComponentFactory componentFactory;
+    // Factory for creating on-disk index components during bulkload.
+    protected final ILSMDiskComponentFactory bulkLoadComponentFactory;
 
     public AbstractLSMIndex(IIOManager ioManager, List<IVirtualBufferCache> virtualBufferCaches,
             IBufferCache diskBufferCache, ILSMIndexFileManager fileManager, double bloomFilterFalsePositiveRate,
             ILSMMergePolicy mergePolicy, ILSMOperationTracker opTracker, ILSMIOOperationScheduler ioScheduler,
-            ILSMIOOperationCallback ioOpCallback, ILSMComponentFilterFrameFactory filterFrameFactory,
+            ILSMIOOperationCallback ioOpCallback, ILSMDiskComponentFactory componentFactory,
+            ILSMDiskComponentFactory bulkLoadComponentFactory, ILSMComponentFilterFrameFactory filterFrameFactory,
             LSMComponentFilterManager filterManager, int[] filterFields, boolean durable,
             IComponentFilterHelper filterHelper, int[] treeFields, ITracer tracer) {
         this.ioManager = ioManager;
@@ -104,6 +111,8 @@ public abstract class AbstractLSMIndex implements ILSMIndex {
         this.bloomFilterFalsePositiveRate = bloomFilterFalsePositiveRate;
         this.ioScheduler = ioScheduler;
         this.ioOpCallback = ioOpCallback;
+        this.componentFactory = componentFactory;
+        this.bulkLoadComponentFactory = bulkLoadComponentFactory;
         this.ioOpCallback.setNumOfMutableComponents(virtualBufferCaches.size());
         this.filterHelper = filterHelper;
         this.filterFrameFactory = filterFrameFactory;
@@ -127,13 +136,17 @@ public abstract class AbstractLSMIndex implements ILSMIndex {
     // The constructor used by external indexes
     public AbstractLSMIndex(IIOManager ioManager, IBufferCache diskBufferCache, ILSMIndexFileManager fileManager,
             double bloomFilterFalsePositiveRate, ILSMMergePolicy mergePolicy, ILSMOperationTracker opTracker,
-            ILSMIOOperationScheduler ioScheduler, ILSMIOOperationCallback ioOpCallback, boolean durable) {
+            ILSMIOOperationScheduler ioScheduler, ILSMIOOperationCallback ioOpCallback,
+            ILSMDiskComponentFactory componentFactory, ILSMDiskComponentFactory bulkLoadComponentFactory,
+            boolean durable) {
         this.ioManager = ioManager;
         this.diskBufferCache = diskBufferCache;
         this.fileManager = fileManager;
         this.bloomFilterFalsePositiveRate = bloomFilterFalsePositiveRate;
         this.ioScheduler = ioScheduler;
         this.ioOpCallback = ioOpCallback;
+        this.componentFactory = componentFactory;
+        this.bulkLoadComponentFactory = bulkLoadComponentFactory;
         this.durable = durable;
         lsmHarness = new ExternalIndexHarness(this, mergePolicy, opTracker, diskBufferCache.isReplicationEnabled());
         isActive = false;
@@ -169,11 +182,14 @@ public abstract class AbstractLSMIndex implements ILSMIndex {
         isActive = true;
     }
 
-    protected void loadDiskComponents() throws HyracksDataException {
+    private void loadDiskComponents() throws HyracksDataException {
         diskComponents.clear();
         List<LSMComponentFileReferences> validFileReferences = fileManager.cleanupAndGetValidFiles();
-        for (LSMComponentFileReferences lsmComonentFileReference : validFileReferences) {
-            ILSMDiskComponent component = loadComponent(lsmComonentFileReference);
+        for (LSMComponentFileReferences lsmComponentFileReferences : validFileReferences) {
+            ILSMDiskComponent component =
+                    createDiskComponent(componentFactory, lsmComponentFileReferences.getInsertIndexFileReference(),
+                            lsmComponentFileReferences.getDeleteIndexFileReference(),
+                            lsmComponentFileReferences.getBloomFilterFileReference(), false);
             diskComponents.add(component);
         }
     }
@@ -192,7 +208,7 @@ public abstract class AbstractLSMIndex implements ILSMIndex {
             flushMemoryComponent();
         }
         deactivateDiskComponents();
-        deactivateMemoryComponents();
+        deallocateMemoryComponents();
         isActive = false;
     }
 
@@ -208,17 +224,16 @@ public abstract class AbstractLSMIndex implements ILSMIndex {
         }
     }
 
-    protected void deactivateDiskComponents() throws HyracksDataException {
-        List<ILSMDiskComponent> immutableComponents = diskComponents;
-        for (ILSMDiskComponent c : immutableComponents) {
-            deactivateDiskComponent(c);
+    private void deactivateDiskComponents() throws HyracksDataException {
+        for (ILSMDiskComponent c : diskComponents) {
+            c.deactivateAndPurge();
         }
     }
 
-    protected void deactivateMemoryComponents() throws HyracksDataException {
+    private void deallocateMemoryComponents() throws HyracksDataException {
         if (memoryComponentsAllocated) {
             for (ILSMMemoryComponent c : memoryComponents) {
-                deactivateMemoryComponent(c);
+                c.deallocate();
             }
             memoryComponentsAllocated = false;
         }
@@ -233,10 +248,9 @@ public abstract class AbstractLSMIndex implements ILSMIndex {
         fileManager.deleteDirs();
     }
 
-    protected void destroyDiskComponents() throws HyracksDataException {
-        List<ILSMDiskComponent> immutableComponents = diskComponents;
-        for (ILSMDiskComponent c : immutableComponents) {
-            destroyDiskComponent(c);
+    private void destroyDiskComponents() throws HyracksDataException {
+        for (ILSMDiskComponent c : diskComponents) {
+            c.destroy();
         }
     }
 
@@ -245,26 +259,30 @@ public abstract class AbstractLSMIndex implements ILSMIndex {
         if (!isActive) {
             throw HyracksDataException.create(ErrorCode.CANNOT_CLEAR_INACTIVE_INDEX);
         }
-        clearMemoryComponents();
-        clearDiskComponents();
+        resetMemoryComponents();
+        deactivateAndDestroyDiskComponents();
     }
 
-    private void clearDiskComponents() throws HyracksDataException {
+    private void deactivateAndDestroyDiskComponents() throws HyracksDataException {
         for (ILSMDiskComponent c : diskComponents) {
-            clearDiskComponent(c);
+            c.deactivateAndDestroy();
         }
         diskComponents.clear();
     }
 
-    protected void clearMemoryComponents() throws HyracksDataException {
-        if (memoryComponentsAllocated) {
+    private void resetMemoryComponents() throws HyracksDataException {
+        if (memoryComponentsAllocated && memoryComponents != null) {
             for (ILSMMemoryComponent c : memoryComponents) {
-                clearMemoryComponent(c);
+                c.reset();
             }
         }
     }
 
     @Override
+    public void purge() throws HyracksDataException {
+    }
+
+    @Override
     public void getOperationalComponents(ILSMIndexOperationContext ctx) throws HyracksDataException {
         List<ILSMDiskComponent> immutableComponents = diskComponents;
         List<ILSMComponent> operationalComponents = ctx.getComponentHolder();
@@ -376,16 +394,37 @@ public abstract class AbstractLSMIndex implements ILSMIndex {
         return createBulkLoader(fillLevel, verifyInput, numElementsHint);
     }
 
+    public IIndexBulkLoader createBulkLoader(float fillLevel, boolean verifyInput, long numElementsHint)
+            throws HyracksDataException {
+        return new LSMIndexDiskComponentBulkLoader(this, fillLevel, verifyInput, numElementsHint);
+    }
+
+    @Override
+    public ILSMDiskComponent createBulkLoadTarget() throws HyracksDataException {
+        LSMComponentFileReferences componentFileRefs = fileManager.getRelFlushFileReference();
+        return createDiskComponent(bulkLoadComponentFactory, componentFileRefs.getInsertIndexFileReference(),
+                componentFileRefs.getDeleteIndexFileReference(), componentFileRefs.getBloomFilterFileReference(), true);
+    }
+
+    protected ILSMDiskComponent createDiskComponent(ILSMDiskComponentFactory factory, FileReference insertFileReference,
+            FileReference deleteIndexFileReference, FileReference bloomFilterFileRef, boolean createComponent)
+            throws HyracksDataException {
+        ILSMDiskComponent component = factory.createComponent(this,
+                new LSMComponentFileReferences(insertFileReference, deleteIndexFileReference, bloomFilterFileRef));
+        component.activate(createComponent);
+        return component;
+    }
+
     @Override
     public final synchronized void allocateMemoryComponents() throws HyracksDataException {
         if (!isActive) {
             throw HyracksDataException.create(ErrorCode.CANNOT_ALLOCATE_MEMORY_FOR_INACTIVE_INDEX);
         }
-        if (memoryComponentsAllocated) {
+        if (memoryComponentsAllocated || memoryComponents == null) {
             return;
         }
         for (ILSMMemoryComponent c : memoryComponents) {
-            allocateMemoryComponent(c);
+            c.allocate();
         }
         memoryComponentsAllocated = true;
     }
@@ -410,7 +449,7 @@ public abstract class AbstractLSMIndex implements ILSMIndex {
     @Override
     public void changeMutableComponent() {
         currentMutableComponentId.set((currentMutableComponentId.get() + 1) % memoryComponents.size());
-        memoryComponents.get(currentMutableComponentId.get()).activate();
+        memoryComponents.get(currentMutableComponentId.get()).requestActivation();
     }
 
     @Override
@@ -507,8 +546,8 @@ public abstract class AbstractLSMIndex implements ILSMIndex {
         Set<String> componentFiles = new HashSet<>();
 
         //get set of files to be replicated for each component
-        for (ILSMComponent lsmComponent : lsmComponents) {
-            componentFiles.addAll(getLSMComponentPhysicalFiles(lsmComponent));
+        for (ILSMDiskComponent lsmComponent : lsmComponents) {
+            componentFiles.addAll(lsmComponent.getLSMComponentPhysicalFiles());
         }
 
         ReplicationExecutionType executionType;
@@ -595,11 +634,11 @@ public abstract class AbstractLSMIndex implements ILSMIndex {
     public final void validate() throws HyracksDataException {
         if (memoryComponentsAllocated) {
             for (ILSMMemoryComponent c : memoryComponents) {
-                validateMemoryComponent(c);
+                c.validate();
             }
         }
         for (ILSMDiskComponent c : diskComponents) {
-            validateDiskComponent(c);
+            c.validate();
         }
     }
 
@@ -607,7 +646,7 @@ public abstract class AbstractLSMIndex implements ILSMIndex {
     public long getMemoryAllocationSize() {
         long size = 0;
         for (ILSMMemoryComponent c : memoryComponents) {
-            size += getMemoryComponentSize(c);
+            size += c.getSize();
         }
         return size;
     }
@@ -628,31 +667,6 @@ public abstract class AbstractLSMIndex implements ILSMIndex {
                 : doMerge(operation);
     }
 
-    public abstract Set<String> getLSMComponentPhysicalFiles(ILSMComponent newComponent);
-
-    protected abstract void allocateMemoryComponent(ILSMMemoryComponent c) throws HyracksDataException;
-
-    protected abstract IIndexBulkLoader createBulkLoader(float fillLevel, boolean verifyInput, long numElementsHint)
-            throws HyracksDataException;
-
-    protected abstract ILSMDiskComponent loadComponent(LSMComponentFileReferences refs) throws HyracksDataException;
-
-    protected abstract void deactivateMemoryComponent(ILSMMemoryComponent c) throws HyracksDataException;
-
-    protected abstract void deactivateDiskComponent(ILSMDiskComponent c) throws HyracksDataException;
-
-    protected abstract void destroyDiskComponent(ILSMDiskComponent c) throws HyracksDataException;
-
-    protected abstract void clearDiskComponent(ILSMDiskComponent c) throws HyracksDataException;
-
-    protected abstract void clearMemoryComponent(ILSMMemoryComponent c) throws HyracksDataException;
-
-    protected abstract void validateMemoryComponent(ILSMMemoryComponent c) throws HyracksDataException;
-
-    protected abstract void validateDiskComponent(ILSMDiskComponent c) throws HyracksDataException;
-
-    protected abstract long getMemoryComponentSize(ILSMMemoryComponent c);
-
     protected abstract LSMComponentFileReferences getMergeFileReferences(ILSMDiskComponent firstComponent,
             ILSMDiskComponent lastComponent) throws HyracksDataException;
 

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/77f89525/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/AbstractLSMMemoryComponent.java
----------------------------------------------------------------------
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 1ee68d9..a4fe35c 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
@@ -25,6 +25,7 @@ import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponentFilter;
 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;
+import org.apache.hyracks.storage.common.buffercache.IBufferCache;
 
 public abstract class AbstractLSMMemoryComponent extends AbstractLSMComponent implements ILSMMemoryComponent {
 
@@ -178,7 +179,7 @@ public abstract class AbstractLSMMemoryComponent extends AbstractLSMComponent im
     }
 
     @Override
-    public void activate() {
+    public void requestActivation() {
         requestedToBeActive = true;
     }
 
@@ -198,12 +199,20 @@ public abstract class AbstractLSMMemoryComponent extends AbstractLSMComponent im
     }
 
     @Override
-    public void reset() throws HyracksDataException {
+    public final void reset() throws HyracksDataException {
         isModified.set(false);
         metadata.reset();
         if (filter != null) {
             filter.reset();
         }
+        doReset();
+    }
+
+    protected void doReset() throws HyracksDataException {
+        getIndex().deactivate();
+        getIndex().destroy();
+        getIndex().create();
+        getIndex().activate();
     }
 
     @Override
@@ -215,4 +224,37 @@ public abstract class AbstractLSMMemoryComponent extends AbstractLSMComponent im
     public MemoryComponentMetadata getMetadata() {
         return metadata;
     }
+
+    @Override
+    public final void allocate() throws HyracksDataException {
+        ((IVirtualBufferCache) getIndex().getBufferCache()).open();
+        doAllocate();
+    }
+
+    protected void doAllocate() throws HyracksDataException {
+        getIndex().create();
+        getIndex().activate();
+    }
+
+    @Override
+    public final void deallocate() throws HyracksDataException {
+        doDeallocate();
+        getIndex().getBufferCache().close();
+    }
+
+    protected void doDeallocate() throws HyracksDataException {
+        getIndex().deactivate();
+        getIndex().destroy();
+    }
+
+    @Override
+    public void validate() throws HyracksDataException {
+        getIndex().validate();
+    }
+
+    @Override
+    public long getSize() {
+        IBufferCache virtualBufferCache = getIndex().getBufferCache();
+        return virtualBufferCache.getNumPages() * (long) virtualBufferCache.getPageSize();
+    }
 }

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/77f89525/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/BloomFilterBulkLoader.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/BloomFilterBulkLoader.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/BloomFilterBulkLoader.java
new file mode 100644
index 0000000..0dcf349
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/BloomFilterBulkLoader.java
@@ -0,0 +1,67 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.hyracks.storage.am.lsm.common.impls;
+
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.dataflow.common.data.accessors.ITupleReference;
+import org.apache.hyracks.storage.common.IIndexBulkLoader;
+
+public class BloomFilterBulkLoader implements IChainedComponentBulkLoader {
+
+    private final IIndexBulkLoader bulkLoader;
+
+    public BloomFilterBulkLoader(IIndexBulkLoader bulkLoader) {
+        this.bulkLoader = bulkLoader;
+    }
+
+    private boolean endedBloomFilterLoad = false;
+
+    @Override
+    public ITupleReference add(ITupleReference tuple) throws HyracksDataException {
+        bulkLoader.add(tuple);
+        return tuple;
+    }
+
+    @Override
+    public ITupleReference delete(ITupleReference tuple) throws HyracksDataException {
+        //Noop
+        return tuple;
+    }
+
+    @Override
+    public void end() throws HyracksDataException {
+        if (!endedBloomFilterLoad) {
+            bulkLoader.end();
+            endedBloomFilterLoad = true;
+        }
+    }
+
+    @Override
+    public void abort() throws HyracksDataException {
+        bulkLoader.abort();
+    }
+
+    @Override
+    public void cleanupArtifacts() throws HyracksDataException {
+        if (!endedBloomFilterLoad) {
+            bulkLoader.abort();
+            endedBloomFilterLoad = true;
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/77f89525/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/ChainedLSMDiskComponentBulkLoader.java
----------------------------------------------------------------------
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
new file mode 100644
index 0000000..f38614c
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/ChainedLSMDiskComponentBulkLoader.java
@@ -0,0 +1,110 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.LinkedList;
+import java.util.List;
+
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+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;
+
+/**
+ * Class encapsulates a chain of operations, happening during an LSM disk component bulkload
+ */
+public class ChainedLSMDiskComponentBulkLoader implements ILSMDiskComponentBulkLoader {
+
+    private List<IChainedComponentBulkLoader> bulkloaderChain = new LinkedList<>();
+    private boolean isEmptyComponent = true;
+    private boolean cleanedUpArtifacts = false;
+    private final ILSMDiskComponent diskComponent;
+    private final boolean cleanupEmptyComponent;
+
+    public ChainedLSMDiskComponentBulkLoader(ILSMDiskComponent diskComponent, boolean cleanupEmptyComponent) {
+        this.diskComponent = diskComponent;
+        this.cleanupEmptyComponent = cleanupEmptyComponent;
+    }
+
+    public void addBulkLoader(IChainedComponentBulkLoader bulkloader) {
+        bulkloaderChain.add(bulkloader);
+    }
+
+    @Override
+    public void add(ITupleReference tuple) throws HyracksDataException {
+        try {
+            ITupleReference t = tuple;
+            for (IChainedComponentBulkLoader lsmBulkloader : bulkloaderChain) {
+                t = lsmBulkloader.add(t);
+            }
+        } catch (Exception e) {
+            cleanupArtifacts();
+            throw e;
+        }
+        if (isEmptyComponent) {
+            isEmptyComponent = false;
+        }
+    }
+
+    @Override
+    public void delete(ITupleReference tuple) throws HyracksDataException {
+        try {
+            ITupleReference t = tuple;
+            for (IChainedComponentBulkLoader lsmOperation : bulkloaderChain) {
+                t = lsmOperation.delete(t);
+            }
+        } catch (Exception e) {
+            cleanupArtifacts();
+            throw e;
+        }
+        if (isEmptyComponent) {
+            isEmptyComponent = false;
+        }
+    }
+
+    @Override
+    public void cleanupArtifacts() throws HyracksDataException {
+        if (!cleanedUpArtifacts) {
+            cleanedUpArtifacts = true;
+            for (IChainedComponentBulkLoader lsmOperation : bulkloaderChain) {
+                lsmOperation.cleanupArtifacts();
+            }
+        }
+        diskComponent.deactivateAndDestroy();
+    }
+
+    @Override
+    public void end() throws HyracksDataException {
+        if (!cleanedUpArtifacts) {
+            for (IChainedComponentBulkLoader lsmOperation : bulkloaderChain) {
+                lsmOperation.end();
+            }
+            if (isEmptyComponent && cleanupEmptyComponent) {
+                cleanupArtifacts();
+            }
+        }
+    }
+
+    @Override
+    public void abort() throws HyracksDataException {
+        for (IChainedComponentBulkLoader lsmOperation : bulkloaderChain) {
+            lsmOperation.abort();
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/77f89525/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/EmptyComponent.java
----------------------------------------------------------------------
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 0134dca..f2751bf 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
@@ -18,12 +18,17 @@
  */
 package org.apache.hyracks.storage.am.lsm.common.impls;
 
+import java.util.Collections;
+import java.util.Set;
+
 import org.apache.hyracks.api.exceptions.ErrorCode;
 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.ILSMComponentFilter;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMDiskComponent;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMDiskComponentId;
 import org.apache.hyracks.storage.am.lsm.common.api.LSMOperationType;
+import org.apache.hyracks.storage.common.IIndex;
 
 public class EmptyComponent implements ILSMDiskComponent {
     public static final EmptyComponent INSTANCE = new EmptyComponent();
@@ -53,6 +58,11 @@ public class EmptyComponent implements ILSMDiskComponent {
     }
 
     @Override
+    public IIndex getIndex() {
+        return null;
+    }
+
+    @Override
     public DiskComponentMetadata getMetadata() {
         return EmptyDiskComponentMetadata.INSTANCE;
     }
@@ -78,8 +88,65 @@ public class EmptyComponent implements ILSMDiskComponent {
     }
 
     @Override
+    public AbstractLSMIndex getLsmIndex() {
+        return null;
+    }
+
+    @Override
+    public ITreeIndex getMetadataHolder() {
+        return null;
+    }
+
+    @Override
+    public Set<String> getLSMComponentPhysicalFiles() {
+        return Collections.emptySet();
+    }
+
+    @Override
     public void markAsValid(boolean persist) throws HyracksDataException {
         // No Op
     }
 
+    @Override
+    public void activate(boolean createNewComponent) throws HyracksDataException {
+        // No Op
+    }
+
+    @Override
+    public void deactivateAndDestroy() throws HyracksDataException {
+        // No Op
+    }
+
+    @Override
+    public void deactivate() throws HyracksDataException {
+        // No Op
+    }
+
+    @Override
+    public void deactivateAndPurge() throws HyracksDataException {
+        // No Op
+    }
+
+    @Override
+    public void validate() throws HyracksDataException {
+        // No Op
+    }
+
+    @Override
+    public IChainedComponentBulkLoader createFilterBulkLoader() throws HyracksDataException {
+        return null;
+    }
+
+    @Override
+    public IChainedComponentBulkLoader createIndexBulkLoader(float fillFactor, boolean verifyInput,
+            long numElementsHint, boolean checkIfEmptyIndex) throws HyracksDataException {
+        return null;
+    }
+
+    @Override
+    public ChainedLSMDiskComponentBulkLoader createBulkLoader(float fillFactor, boolean verifyInput,
+            long numElementsHint, boolean checkIfEmptyIndex, boolean withFilter, boolean cleanupEmptyComponent)
+            throws HyracksDataException {
+        return null;
+    }
 }

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/77f89525/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/ExternalIndexHarness.java
----------------------------------------------------------------------
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 2f65b18..11f2441 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
@@ -135,7 +135,7 @@ public class ExternalIndexHarness extends LSMHarness {
                                 lsmIndex.scheduleReplication(null, componentsToBeReplicated, false,
                                         ReplicationOperation.DELETE, opType);
                             }
-                            ((ILSMDiskComponent) c).destroy();
+                            ((ILSMDiskComponent) c).deactivateAndDestroy();
                             break;
                         default:
                             break;
@@ -348,7 +348,7 @@ public class ExternalIndexHarness extends LSMHarness {
                 componentsToBeReplicated.add(diskComponent);
                 lsmIndex.scheduleReplication(null, componentsToBeReplicated, false, ReplicationOperation.DELETE, null);
             }
-            diskComponent.destroy();
+            diskComponent.deactivateAndDestroy();
         }
     }
 

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/77f89525/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/FilterBulkLoader.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/FilterBulkLoader.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/FilterBulkLoader.java
new file mode 100644
index 0000000..7359d2b
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/FilterBulkLoader.java
@@ -0,0 +1,82 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.dataflow.common.data.accessors.ITupleReference;
+import org.apache.hyracks.storage.am.common.api.ITreeIndex;
+import org.apache.hyracks.storage.am.common.tuples.PermutingTupleReference;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponentFilter;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponentFilterManager;
+import org.apache.hyracks.storage.common.MultiComparator;
+
+public class FilterBulkLoader implements IChainedComponentBulkLoader {
+
+    private final ILSMComponentFilter filter;
+    private final ITreeIndex treeIndex;
+    protected final ILSMComponentFilterManager filterManager;
+    protected final PermutingTupleReference indexTuple;
+    protected final PermutingTupleReference filterTuple;
+    protected final MultiComparator filterCmp;
+
+    public FilterBulkLoader(ILSMComponentFilter filter, ITreeIndex treeIndex,
+            ILSMComponentFilterManager filterManager, int[] indexFields, int[] filterFields,
+            MultiComparator filterCmp) {
+        this.filter = filter;
+        this.treeIndex = treeIndex;
+        this.filterManager = filterManager;
+        this.indexTuple = new PermutingTupleReference(indexFields);
+        this.filterTuple = new PermutingTupleReference(filterFields);
+        this.filterCmp = filterCmp;
+    }
+
+    @Override
+    public ITupleReference delete(ITupleReference tuple) throws HyracksDataException {
+        indexTuple.reset(tuple);
+        updateFilter(tuple);
+        return indexTuple;
+    }
+
+    @Override
+    public void cleanupArtifacts() throws HyracksDataException {
+        //Noop
+    }
+
+    @Override
+    public ITupleReference add(ITupleReference tuple) throws HyracksDataException {
+        indexTuple.reset(tuple);
+        updateFilter(tuple);
+        return indexTuple;
+    }
+
+    @Override
+    public void end() throws HyracksDataException {
+        filterManager.writeFilter(filter, treeIndex);
+    }
+
+    @Override
+    public void abort() throws HyracksDataException {
+        //Noop
+    }
+
+    private void updateFilter(ITupleReference tuple) throws HyracksDataException {
+        filterTuple.reset(tuple);
+        filter.update(filterTuple, filterCmp);
+    }
+}

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/77f89525/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/IChainedComponentBulkLoader.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/IChainedComponentBulkLoader.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/IChainedComponentBulkLoader.java
new file mode 100644
index 0000000..90ef127
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/IChainedComponentBulkLoader.java
@@ -0,0 +1,63 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.dataflow.common.data.accessors.ITupleReference;
+
+public interface IChainedComponentBulkLoader {
+    /**
+     * Adds a tuple to the bulkloaded component
+     *
+     * @param tuple
+     * @return Potentially modified tuple, which is used as an input for downstream bulkloaders
+     * @throws HyracksDataException
+     */
+    ITupleReference add(ITupleReference tuple) throws HyracksDataException;
+
+    /**
+     * Deletes a tuple (i.e. appends anti-matter tuple or deleted-key tuple) from the bulkloaded component
+     *
+     * @param tuple
+     * @return Potentially modified tuple, which is used as an input for downstream bulkloaders
+     * @throws HyracksDataException
+     */
+    ITupleReference delete(ITupleReference tuple) throws HyracksDataException;
+
+    /**
+     * Correctly finalizes bulkloading process and releases all resources
+     *
+     * @throws HyracksDataException
+     */
+    void end() throws HyracksDataException;
+
+    /**
+     * Aborts bulkloading process without releasing associated resources
+     *
+     * @throws HyracksDataException
+     */
+    void abort() throws HyracksDataException;
+
+    /**
+     * Releases all resources allocated during the bulkloading process
+     *
+     * @throws HyracksDataException
+     */
+    void cleanupArtifacts() throws HyracksDataException;
+}

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/77f89525/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/IndexWithBuddyBulkLoader.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/IndexWithBuddyBulkLoader.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/IndexWithBuddyBulkLoader.java
new file mode 100644
index 0000000..4fb2919
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/IndexWithBuddyBulkLoader.java
@@ -0,0 +1,72 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.ErrorCode;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.dataflow.common.data.accessors.ITupleReference;
+import org.apache.hyracks.storage.common.IIndexBulkLoader;
+
+public class IndexWithBuddyBulkLoader implements IChainedComponentBulkLoader {
+
+    private final IIndexBulkLoader bulkLoader;
+    private final IIndexBulkLoader buddyBTreeBulkLoader;
+
+    public IndexWithBuddyBulkLoader(IIndexBulkLoader bulkLoader, IIndexBulkLoader buddyBTreeBulkLoader) {
+        this.bulkLoader = bulkLoader;
+        this.buddyBTreeBulkLoader = buddyBTreeBulkLoader;
+    }
+
+    @Override
+    public ITupleReference delete(ITupleReference tuple) throws HyracksDataException {
+        try {
+            buddyBTreeBulkLoader.add(tuple);
+        } catch (HyracksDataException e) {
+            //deleting a key multiple times is OK
+            if (e.getErrorCode() != ErrorCode.DUPLICATE_KEY) {
+                cleanupArtifacts();
+                throw e;
+            }
+        }
+        return tuple;
+    }
+
+    @Override
+    public void cleanupArtifacts() throws HyracksDataException {
+        //Noop
+    }
+
+    @Override
+    public ITupleReference add(ITupleReference tuple) throws HyracksDataException {
+        bulkLoader.add(tuple);
+        return tuple;
+    }
+
+    @Override
+    public void end() throws HyracksDataException {
+        bulkLoader.end();
+        buddyBTreeBulkLoader.end();
+    }
+
+    @Override
+    public void abort() throws HyracksDataException {
+        bulkLoader.abort();
+        buddyBTreeBulkLoader.abort();
+    }
+}

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/77f89525/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/LSMHarness.java
----------------------------------------------------------------------
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 1bdd250..c8be9b9 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
@@ -356,8 +356,8 @@ public class LSMHarness implements ILSMHarness {
                         lsmIndex.scheduleReplication(null, inactiveDiskComponentsToBeDeleted, false,
                                 ReplicationOperation.DELETE, opType);
                     }
-                    for (ILSMComponent c : inactiveDiskComponentsToBeDeleted) {
-                        ((AbstractLSMDiskComponent) c).destroy();
+                    for (ILSMDiskComponent c : inactiveDiskComponentsToBeDeleted) {
+                        c.deactivateAndDestroy();
                     }
                 } catch (Throwable e) {
                     if (LOGGER.isLoggable(Level.WARNING)) {

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/77f89525/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/LSMIndexBulkLoader.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/LSMIndexBulkLoader.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/LSMIndexBulkLoader.java
new file mode 100644
index 0000000..84857f4
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/LSMIndexBulkLoader.java
@@ -0,0 +1,67 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.hyracks.storage.am.lsm.common.impls;
+
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.dataflow.common.data.accessors.ITupleReference;
+import org.apache.hyracks.storage.am.common.impls.AbstractTreeIndex.AbstractTreeIndexBulkLoader;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMTreeTupleWriter;
+import org.apache.hyracks.storage.common.IIndexBulkLoader;
+
+public class LSMIndexBulkLoader implements IChainedComponentBulkLoader {
+    private final IIndexBulkLoader bulkLoader;
+
+    public LSMIndexBulkLoader(IIndexBulkLoader bulkLoader) {
+        this.bulkLoader = bulkLoader;
+    }
+
+    @Override
+    public ITupleReference delete(ITupleReference tuple) throws HyracksDataException {
+        ILSMTreeTupleWriter tupleWriter =
+                (ILSMTreeTupleWriter) ((AbstractTreeIndexBulkLoader) bulkLoader).getLeafFrame().getTupleWriter();
+        tupleWriter.setAntimatter(true);
+        try {
+            bulkLoader.add(tuple);
+        } finally {
+            tupleWriter.setAntimatter(false);
+        }
+        return tuple;
+    }
+
+    @Override
+    public void cleanupArtifacts() throws HyracksDataException {
+        //Noop
+    }
+
+    @Override
+    public ITupleReference add(ITupleReference tuple) throws HyracksDataException {
+        bulkLoader.add(tuple);
+        return tuple;
+    }
+
+    @Override
+    public void end() throws HyracksDataException {
+        bulkLoader.end();
+    }
+
+    @Override
+    public void abort() throws HyracksDataException {
+        bulkLoader.abort();
+    }
+}

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/77f89525/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/LSMIndexDiskComponentBulkLoader.java
----------------------------------------------------------------------
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
new file mode 100644
index 0000000..8befee1
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/LSMIndexDiskComponentBulkLoader.java
@@ -0,0 +1,64 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.hyracks.storage.am.lsm.common.impls;
+
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+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.LSMOperationType;
+import org.apache.hyracks.storage.common.IIndexBulkLoader;
+
+public class LSMIndexDiskComponentBulkLoader implements IIndexBulkLoader {
+    private final AbstractLSMIndex lsmIndex;
+    private final ILSMDiskComponent component;
+    private final IIndexBulkLoader componentBulkLoader;
+
+    public LSMIndexDiskComponentBulkLoader(AbstractLSMIndex lsmIndex, float fillFactor, boolean verifyInput,
+            long numElementsHint) throws HyracksDataException {
+        this.lsmIndex = lsmIndex;
+        // Note that by using a flush target file name, we state that the
+        // new bulk loaded component is "newer" than any other merged component.
+        this.component = lsmIndex.createBulkLoadTarget();
+        this.componentBulkLoader =
+                component.createBulkLoader(fillFactor, verifyInput, numElementsHint, false, true, true);
+    }
+
+    @Override
+    public void add(ITupleReference tuple) throws HyracksDataException {
+        componentBulkLoader.add(tuple);
+    }
+
+    @Override
+    public void end() throws HyracksDataException {
+        componentBulkLoader.end();
+        if (component.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(LSMOperationType.FLUSH, null, component);
+            lsmIndex.getLsmHarness().addBulkLoadedComponent(component);
+        }
+    }
+
+    @Override
+    public void abort() throws HyracksDataException {
+        componentBulkLoader.abort();
+    }
+
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/77f89525/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/api/IInPlaceInvertedIndex.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/api/IInPlaceInvertedIndex.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/api/IInPlaceInvertedIndex.java
index a26e14c..6f583b4 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/api/IInPlaceInvertedIndex.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/api/IInPlaceInvertedIndex.java
@@ -41,13 +41,4 @@ public interface IInPlaceInvertedIndex extends IInvertedIndex {
      */
     void openInvertedListCursor(IInvertedListCursor listCursor, ITupleReference searchKey, IIndexOperationContext ictx)
             throws HyracksDataException;
-
-    /**
-     * Purge the index files out of the buffer cache.
-     * Can only be called if the caller is absolutely sure the files don't contain dirty pages
-     *
-     * @throws HyracksDataException
-     *             if the index is active
-     */
-    void purge() throws HyracksDataException;
 }


[6/6] asterixdb git commit: [NO ISSUE][STO][IDX] LSM storage cleanup

Posted by il...@apache.org.
[NO ISSUE][STO][IDX] LSM storage cleanup

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

- Replaced component lifecycle-related fabric methods in AbstractLSMIndex
  with direct method calls of lsmComponent functions.
- Extracted common lifecycle-related functionality from index-specific
  disk/memory lsmComponents to interfaces.
- Introduced composable disk component bulkloader design which assembles the
  proper bulkload pipeline from individual elements populating lsmFilters,
  bloomFilters, buddyBTrees\deletedKeysBTrees, bTress\rTrees\invIndexes.
- Changed methods to return index-specific versions of objects (accessors,
  components, index instances) to avoid nasty downcasting.

Change-Id: I6739d751b990e7a28e03e32a5de6e2b670d37a1e
Reviewed-on: https://asterix-gerrit.ics.uci.edu/2014
Sonar-Qube: Jenkins <je...@fulliautomatix.ics.uci.edu>
Integration-Tests: Jenkins <je...@fulliautomatix.ics.uci.edu>
Tested-by: Jenkins <je...@fulliautomatix.ics.uci.edu>
Contrib: Jenkins <je...@fulliautomatix.ics.uci.edu>
Reviewed-by: abdullah alamoudi <ba...@gmail.com>


Project: http://git-wip-us.apache.org/repos/asf/asterixdb/repo
Commit: http://git-wip-us.apache.org/repos/asf/asterixdb/commit/77f89525
Tree: http://git-wip-us.apache.org/repos/asf/asterixdb/tree/77f89525
Diff: http://git-wip-us.apache.org/repos/asf/asterixdb/diff/77f89525

Branch: refs/heads/master
Commit: 77f8952573443eaf81270f509ffd39fa84115411
Parents: 536aa3f
Author: Ildar Absalyamov <il...@gmail.com>
Authored: Fri Oct 13 12:22:52 2017 -0700
Committer: Ildar Absalyamov <il...@gmail.com>
Committed: Sun Oct 15 23:06:51 2017 -0700

----------------------------------------------------------------------
 asterixdb/asterix-common/pom.xml                |  10 +
 .../LSMBTreeIOOperationCallback.java            |   5 +-
 .../LSMBTreeWithBuddyIOOperationCallback.java   |   5 +-
 .../LSMInvertedIndexIOOperationCallback.java    |   2 +-
 .../LSMRTreeIOOperationCallback.java            |   5 +-
 .../LSMSecondaryIndexBulkLoadNodePushable.java  |   4 +-
 .../hyracks/storage/am/btree/impls/BTree.java   |   9 +-
 .../am/common/impls/AbstractTreeIndex.java      |   1 +
 .../am/lsm/btree/impls/ExternalBTree.java       |  97 +++----
 .../lsm/btree/impls/ExternalBTreeWithBuddy.java | 227 +++------------
 .../storage/am/lsm/btree/impls/LSMBTree.java    | 242 +++-------------
 .../am/lsm/btree/impls/LSMBTreeBulkLoader.java  |  62 -----
 .../lsm/btree/impls/LSMBTreeDiskComponent.java  |  68 ++---
 .../impls/LSMBTreeDiskComponentBulkLoader.java  |  59 ----
 .../impls/LSMBTreeDiskComponentFactory.java     |  21 +-
 .../impls/LSMBTreeDiskComponentScanCursor.java  |   6 +-
 .../btree/impls/LSMBTreeMemoryComponent.java    |  17 +-
 .../am/lsm/btree/impls/LSMBTreeOpContext.java   |  14 +-
 .../btree/impls/LSMBTreePointSearchCursor.java  |  12 +-
 .../btree/impls/LSMBTreeRangeSearchCursor.java  |   8 +-
 .../LSMBTreeWithBloomFilterDiskComponent.java   |  86 ++++++
 ...TreeWithBloomFilterDiskComponentFactory.java |  54 ++++
 .../impls/LSMBTreeWithBuddyAbstractCursor.java  |  19 +-
 .../impls/LSMBTreeWithBuddyDiskComponent.java   |  76 +++---
 ...SMBTreeWithBuddyDiskComponentBulkLoader.java |  65 -----
 .../LSMBTreeWithBuddyDiskComponentFactory.java  |  28 +-
 .../impls/LSMBTreeWithBuddyMemoryComponent.java |  26 +-
 .../btree/impls/LSMBuddyBTreeMergeCursor.java   |   2 +-
 .../am/lsm/btree/utils/LSMBTreeUtil.java        |  51 +++-
 ...AbstractLSMWithBloomFilterDiskComponent.java | 105 +++++++
 .../api/AbstractLSMWithBuddyDiskComponent.java  |  96 +++++++
 .../AbstractLSMWithBuddyMemoryComponent.java    |  61 +++++
 .../am/lsm/common/api/ILSMComponent.java        |   6 +
 .../am/lsm/common/api/ILSMDiskComponent.java    | 110 +++++++-
 .../common/api/ILSMDiskComponentBulkLoader.java |   7 +
 .../common/api/ILSMDiskComponentFactory.java    |   4 +-
 .../storage/am/lsm/common/api/ILSMIndex.java    |  17 --
 .../am/lsm/common/api/ILSMMemoryComponent.java  |  29 +-
 .../common/impls/AbstractLSMDiskComponent.java  |  89 +++++-
 .../AbstractLSMDiskComponentBulkLoader.java     | 202 --------------
 ...ractLSMDiskComponentWithBuddyBulkLoader.java | 156 -----------
 .../am/lsm/common/impls/AbstractLSMIndex.java   | 126 +++++----
 .../impls/AbstractLSMMemoryComponent.java       |  46 +++-
 .../lsm/common/impls/BloomFilterBulkLoader.java |  67 +++++
 .../ChainedLSMDiskComponentBulkLoader.java      | 110 ++++++++
 .../am/lsm/common/impls/EmptyComponent.java     |  67 +++++
 .../lsm/common/impls/ExternalIndexHarness.java  |   4 +-
 .../am/lsm/common/impls/FilterBulkLoader.java   |  82 ++++++
 .../impls/IChainedComponentBulkLoader.java      |  63 +++++
 .../common/impls/IndexWithBuddyBulkLoader.java  |  72 +++++
 .../storage/am/lsm/common/impls/LSMHarness.java |   4 +-
 .../am/lsm/common/impls/LSMIndexBulkLoader.java |  67 +++++
 .../impls/LSMIndexDiskComponentBulkLoader.java  |  64 +++++
 .../api/IInPlaceInvertedIndex.java              |   9 -
 .../invertedindex/impls/LSMInvertedIndex.java   | 273 +++----------------
 .../impls/LSMInvertedIndexDiskComponent.java    |  64 +++--
 ...LSMInvertedIndexDiskComponentBulkLoader.java |  71 -----
 .../LSMInvertedIndexDiskComponentFactory.java   |   6 +-
 .../impls/LSMInvertedIndexMemoryComponent.java  |  32 +--
 .../impls/LSMInvertedIndexOpContext.java        |  14 +-
 .../impls/PartitionedLSMInvertedIndex.java      |  32 +--
 .../inmemory/InMemoryInvertedIndex.java         |   6 +-
 .../inmemory/InMemoryInvertedIndexAccessor.java |   6 +-
 .../InMemoryInvertedIndexOpContext.java         |   4 +-
 .../PartitionedInMemoryInvertedIndex.java       |   6 +-
 .../ondisk/OnDiskInvertedIndex.java             |   4 +-
 .../ondisk/OnDiskInvertedIndexFactory.java      |   3 +-
 .../ondisk/PartitionedOnDiskInvertedIndex.java  |   3 +-
 .../PartitionedOnDiskInvertedIndexFactory.java  |   3 +-
 .../invertedindex/util/InvertedIndexUtils.java  |  26 +-
 .../am/lsm/rtree/impls/AbstractLSMRTree.java    | 117 ++------
 .../am/lsm/rtree/impls/ExternalRTree.java       | 111 +++-----
 .../storage/am/lsm/rtree/impls/LSMRTree.java    | 180 +++---------
 .../lsm/rtree/impls/LSMRTreeAbstractCursor.java |  16 +-
 .../am/lsm/rtree/impls/LSMRTreeBulkLoader.java  |  60 ----
 .../LSMRTreeDeletedKeysBTreeMergeCursor.java    |   2 +-
 .../lsm/rtree/impls/LSMRTreeDiskComponent.java  |  89 +++---
 .../impls/LSMRTreeDiskComponentBulkLoader.java  |  65 -----
 .../impls/LSMRTreeDiskComponentFactory.java     |   6 +-
 .../rtree/impls/LSMRTreeMemoryComponent.java    |  24 +-
 .../am/lsm/rtree/impls/LSMRTreeOpContext.java   |  12 +-
 .../impls/LSMRTreeWithAntiMatterTuples.java     | 155 ++---------
 ...AntiMatterTuplesDiskComponentBulkLoader.java |  59 ----
 ...ithAntiMatterTuplesDiskComponentFactory.java |   9 +-
 ...SMRTreeWithAntiMatterTuplesSearchCursor.java |   6 +-
 .../LSMRTreeWithAntimatterDiskComponent.java    |  71 +++++
 .../am/lsm/rtree/utils/LSMRTreeUtils.java       |  43 ++-
 .../hyracks/storage/am/rtree/impls/RTree.java   |   6 +-
 .../apache/hyracks/storage/common/IIndex.java   |  11 +
 .../storage/am/common/IndexTestContext.java     |   3 +-
 .../am/lsm/btree/LSMBTreeFileManagerTest.java   |   7 +-
 ...TreeUpdateInPlaceScanDiskComponentsTest.java |  12 +-
 .../storage/am/lsm/btree/impl/TestLsmBtree.java |  13 +-
 .../am/lsm/btree/impl/TestLsmBtreeUtil.java     |  21 +-
 94 files changed, 2156 insertions(+), 2439 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/asterixdb/blob/77f89525/asterixdb/asterix-common/pom.xml
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-common/pom.xml b/asterixdb/asterix-common/pom.xml
index 6df7049..42a8004 100644
--- a/asterixdb/asterix-common/pom.xml
+++ b/asterixdb/asterix-common/pom.xml
@@ -168,6 +168,16 @@
           </execution>
         </executions>
       </plugin>
+      <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-dependency-plugin</artifactId>
+        <configuration>
+          <ignoredUnusedDeclaredDependencies>
+            <ignoredUnusedDeclaredDependency>org.apache.hyracks:hyracks-storage-am-lsm-btree:*</ignoredUnusedDeclaredDependency>
+            <ignoredUnusedDeclaredDependency>org.apache.hyracks:hyracks-storage-am-lsm-rtree:*</ignoredUnusedDeclaredDependency>
+          </ignoredUnusedDeclaredDependencies>
+        </configuration>
+      </plugin>
     </plugins>
   </build>
   <dependencies>

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/77f89525/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/ioopcallbacks/LSMBTreeIOOperationCallback.java
----------------------------------------------------------------------
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
index 8c91879..147bc10 100644
--- 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
@@ -20,8 +20,8 @@
 package org.apache.asterix.common.ioopcallbacks;
 
 import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.storage.am.btree.impls.BTree;
 import org.apache.hyracks.storage.am.common.api.IMetadataPageManager;
-import org.apache.hyracks.storage.am.lsm.btree.impls.LSMBTreeDiskComponent;
 import org.apache.hyracks.storage.am.lsm.btree.impls.LSMBTreeFileManager;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMDiskComponent;
 
@@ -35,9 +35,8 @@ public class LSMBTreeIOOperationCallback extends AbstractLSMIOOperationCallback
     public long getComponentFileLSNOffset(ILSMDiskComponent diskComponent, String diskComponentFilePath)
             throws HyracksDataException {
         if (diskComponentFilePath.endsWith(LSMBTreeFileManager.BTREE_SUFFIX)) {
-            LSMBTreeDiskComponent btreeComponent = (LSMBTreeDiskComponent) diskComponent;
             IMetadataPageManager metadataPageManager =
-                    (IMetadataPageManager) btreeComponent.getBTree().getPageManager();
+                    (IMetadataPageManager) ((BTree) diskComponent.getIndex()).getPageManager();
             return metadataPageManager.getFileOffset(metadataPageManager.createMetadataFrame(), LSN_KEY);
         }
         return INVALID;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/77f89525/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/ioopcallbacks/LSMBTreeWithBuddyIOOperationCallback.java
----------------------------------------------------------------------
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
index 9f50f57..5b6de6c 100644
--- 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
@@ -19,8 +19,8 @@
 package org.apache.asterix.common.ioopcallbacks;
 
 import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.storage.am.btree.impls.BTree;
 import org.apache.hyracks.storage.am.common.api.IMetadataPageManager;
-import org.apache.hyracks.storage.am.lsm.btree.impls.LSMBTreeWithBuddyDiskComponent;
 import org.apache.hyracks.storage.am.lsm.btree.impls.LSMBTreeWithBuddyFileManager;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMDiskComponent;
 
@@ -30,9 +30,8 @@ public class LSMBTreeWithBuddyIOOperationCallback extends AbstractLSMIOOperation
     public long getComponentFileLSNOffset(ILSMDiskComponent diskComponent, String diskComponentFilePath)
             throws HyracksDataException {
         if (diskComponentFilePath.endsWith(LSMBTreeWithBuddyFileManager.BTREE_SUFFIX)) {
-            LSMBTreeWithBuddyDiskComponent btreeComponent = (LSMBTreeWithBuddyDiskComponent) diskComponent;
             IMetadataPageManager metadataPageManager =
-                    (IMetadataPageManager) btreeComponent.getBTree().getPageManager();
+                    (IMetadataPageManager) ((BTree) diskComponent.getIndex()).getPageManager();
             return metadataPageManager.getFileOffset(metadataPageManager.createMetadataFrame(), LSN_KEY);
         }
         return INVALID;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/77f89525/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/ioopcallbacks/LSMInvertedIndexIOOperationCallback.java
----------------------------------------------------------------------
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
index 69286d5..3c91a6a 100644
--- 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
@@ -37,7 +37,7 @@ public class LSMInvertedIndexIOOperationCallback extends AbstractLSMIOOperationC
         if (diskComponentFilePath.endsWith(LSMInvertedIndexFileManager.DELETED_KEYS_BTREE_SUFFIX)) {
             LSMInvertedIndexDiskComponent invIndexComponent = (LSMInvertedIndexDiskComponent) diskComponent;
             IMetadataPageManager metadataPageManager =
-                    (IMetadataPageManager) invIndexComponent.getDeletedKeysBTree().getPageManager();
+                    (IMetadataPageManager) invIndexComponent.getBuddyIndex().getPageManager();
             return metadataPageManager.getFileOffset(metadataPageManager.createMetadataFrame(), LSN_KEY);
         }
         return INVALID;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/77f89525/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/ioopcallbacks/LSMRTreeIOOperationCallback.java
----------------------------------------------------------------------
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
index 1001af9..063fe38 100644
--- 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
@@ -22,8 +22,8 @@ package org.apache.asterix.common.ioopcallbacks;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.storage.am.common.api.IMetadataPageManager;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMDiskComponent;
-import org.apache.hyracks.storage.am.lsm.rtree.impls.LSMRTreeDiskComponent;
 import org.apache.hyracks.storage.am.lsm.rtree.impls.LSMRTreeFileManager;
+import org.apache.hyracks.storage.am.rtree.impls.RTree;
 
 public class LSMRTreeIOOperationCallback extends AbstractLSMIOOperationCallback {
 
@@ -35,9 +35,8 @@ public class LSMRTreeIOOperationCallback extends AbstractLSMIOOperationCallback
     public long getComponentFileLSNOffset(ILSMDiskComponent diskComponent, String diskComponentFilePath)
             throws HyracksDataException {
         if (diskComponentFilePath.endsWith(LSMRTreeFileManager.RTREE_SUFFIX)) {
-            LSMRTreeDiskComponent rtreeComponent = (LSMRTreeDiskComponent) diskComponent;
             IMetadataPageManager metadataPageManager =
-                    (IMetadataPageManager) rtreeComponent.getRTree().getPageManager();
+                    (IMetadataPageManager) ((RTree) diskComponent.getIndex()).getPageManager();
             return metadataPageManager.getFileOffset(metadataPageManager.createMetadataFrame(), LSN_KEY);
         }
         return INVALID;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/77f89525/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/LSMSecondaryIndexBulkLoadNodePushable.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/LSMSecondaryIndexBulkLoadNodePushable.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/LSMSecondaryIndexBulkLoadNodePushable.java
index 12f49a8..cb81b64 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/LSMSecondaryIndexBulkLoadNodePushable.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/LSMSecondaryIndexBulkLoadNodePushable.java
@@ -199,8 +199,8 @@ public class LSMSecondaryIndexBulkLoadNodePushable extends AbstractLSMSecondaryI
         endCurrentComponent();
 
         component = secondaryIndex.createBulkLoadTarget();
-        componentBulkLoader = secondaryIndex.createComponentBulkLoader(component, 1.0f, false,
-                getNumDeletedTuples(componentPos), false, true, true);
+        int numTuples = getNumDeletedTuples(componentPos);
+        componentBulkLoader = component.createBulkLoader(1.0f, false, numTuples, false, true, true);
 
     }
 

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/77f89525/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/impls/BTree.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/impls/BTree.java b/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/impls/BTree.java
index 7206fee..5e3042e 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/impls/BTree.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/impls/BTree.java
@@ -118,8 +118,7 @@ public class BTree extends AbstractTreeIndex {
         // Stack validation protocol:
         //      * parent pushes the validation information onto the stack before validation
         //      * child pops the validation information off of the stack after validating
-        BTreeAccessor accessor =
-                (BTreeAccessor) createAccessor(NoOpOperationCallback.INSTANCE, NoOpOperationCallback.INSTANCE);
+        BTreeAccessor accessor = createAccessor(NoOpOperationCallback.INSTANCE, NoOpOperationCallback.INSTANCE);
         PageValidationInfo pvi = accessor.ctx.createPageValidationInfo(null);
         accessor.ctx.getValidationInfos().addFirst(pvi);
         if (isActive) {
@@ -819,12 +818,12 @@ public class BTree extends AbstractTreeIndex {
     }
 
     @Override
-    public ITreeIndexAccessor createAccessor(IModificationOperationCallback modificationCallback,
+    public BTreeAccessor createAccessor(IModificationOperationCallback modificationCallback,
             ISearchOperationCallback searchCallback) {
         return new BTreeAccessor(this, modificationCallback, searchCallback);
     }
 
-    public ITreeIndexAccessor createAccessor(IModificationOperationCallback modificationCallback,
+    public BTreeAccessor createAccessor(IModificationOperationCallback modificationCallback,
             ISearchOperationCallback searchCallback, int[] logTupleFields) {
         return new BTreeAccessor(this, modificationCallback, searchCallback, logTupleFields);
     }
@@ -892,7 +891,7 @@ public class BTree extends AbstractTreeIndex {
         }
 
         @Override
-        public ITreeIndexCursor createSearchCursor(boolean exclusive) {
+        public BTreeRangeSearchCursor createSearchCursor(boolean exclusive) {
             IBTreeLeafFrame leafFrame = (IBTreeLeafFrame) btree.getLeafFrameFactory().createFrame();
             return new BTreeRangeSearchCursor(leafFrame, exclusive);
         }

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/77f89525/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/impls/AbstractTreeIndex.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/impls/AbstractTreeIndex.java b/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/impls/AbstractTreeIndex.java
index 8ffd0eb..ae66402 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/impls/AbstractTreeIndex.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/impls/AbstractTreeIndex.java
@@ -121,6 +121,7 @@ public abstract class AbstractTreeIndex implements ITreeIndex {
         isActive = false;
     }
 
+    @Override
     public void purge() throws HyracksDataException {
         if (isActive) {
             throw HyracksDataException.create(ErrorCode.CANNOT_PURGE_ACTIVE_INDEX);

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/77f89525/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/ExternalBTree.java
----------------------------------------------------------------------
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 3775985..24f513a 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
@@ -28,8 +28,6 @@ import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.api.io.FileReference;
 import org.apache.hyracks.api.io.IIOManager;
 import org.apache.hyracks.dataflow.common.data.accessors.ITupleReference;
-import org.apache.hyracks.storage.am.bloomfilter.impls.BloomFilter;
-import org.apache.hyracks.storage.am.bloomfilter.impls.BloomFilterFactory;
 import org.apache.hyracks.storage.am.btree.impls.BTree;
 import org.apache.hyracks.storage.am.common.api.IIndexOperationContext;
 import org.apache.hyracks.storage.am.common.api.IMetadataPageManager;
@@ -41,6 +39,7 @@ import org.apache.hyracks.storage.am.common.ophelpers.IndexOperation;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponent;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMDiskComponent;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMDiskComponentBulkLoader;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMDiskComponentFactory;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperation;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallback;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationScheduler;
@@ -56,12 +55,12 @@ import org.apache.hyracks.storage.am.lsm.common.impls.ExternalIndexHarness;
 import org.apache.hyracks.storage.am.lsm.common.impls.LSMComponentFileReferences;
 import org.apache.hyracks.storage.am.lsm.common.impls.LSMTreeIndexAccessor;
 import org.apache.hyracks.storage.am.lsm.common.impls.LSMTreeIndexAccessor.ICursorFactory;
-import org.apache.hyracks.storage.am.lsm.common.impls.TreeIndexFactory;
 import org.apache.hyracks.storage.common.IIndexBulkLoader;
 import org.apache.hyracks.storage.common.IIndexCursor;
 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.buffercache.IBufferCache;
 
 /**
  * This is an lsm b-tree that does not have memory component and is modified
@@ -76,7 +75,7 @@ public class ExternalBTree extends LSMBTree implements ITwoPCIndex {
     // This component factory has to be different since it uses different tuple
     // writer in it's leaf frames to support inserting both
     // regular and delete tuples
-    private final LSMBTreeDiskComponentFactory transactionComponentFactory;
+    private final ILSMDiskComponentFactory transactionComponentFactory;
     // A second disk component list that will be used when a transaction is
     // committed and will be seen by subsequent accessors
     private final List<ILSMDiskComponent> secondDiskComponents;
@@ -90,16 +89,15 @@ public class ExternalBTree extends LSMBTree implements ITwoPCIndex {
     //TODO remove BloomFilter from external dataset's secondary LSMBTree index
     public ExternalBTree(IIOManager ioManager, ITreeIndexFrameFactory interiorFrameFactory,
             ITreeIndexFrameFactory insertLeafFrameFactory, ITreeIndexFrameFactory deleteLeafFrameFactory,
-            ILSMIndexFileManager fileManager, TreeIndexFactory<BTree> diskBTreeFactory,
-            TreeIndexFactory<BTree> bulkLoadBTreeFactory, BloomFilterFactory bloomFilterFactory,
+            IBufferCache bufferCache, ILSMIndexFileManager fileManager, ILSMDiskComponentFactory componentFactory,
+            ILSMDiskComponentFactory bulkLoadComponentFactory, ILSMDiskComponentFactory transactionComponentFactory,
             double bloomFilterFalsePositiveRate, IBinaryComparatorFactory[] cmpFactories, ILSMMergePolicy mergePolicy,
             ILSMOperationTracker opTracker, ILSMIOOperationScheduler ioScheduler, ILSMIOOperationCallback ioOpCallback,
-            TreeIndexFactory<BTree> transactionBTreeFactory, boolean durable) {
-        super(ioManager, insertLeafFrameFactory, deleteLeafFrameFactory, fileManager, diskBTreeFactory,
-                bulkLoadBTreeFactory, bloomFilterFactory, bloomFilterFalsePositiveRate, cmpFactories, mergePolicy,
-                opTracker, ioScheduler, ioOpCallback, false, durable);
-        this.transactionComponentFactory =
-                new LSMBTreeDiskComponentFactory(transactionBTreeFactory, bloomFilterFactory, null);
+            boolean durable) {
+        super(ioManager, insertLeafFrameFactory, deleteLeafFrameFactory, bufferCache, fileManager, componentFactory,
+                bulkLoadComponentFactory, bloomFilterFalsePositiveRate, cmpFactories, mergePolicy, opTracker,
+                ioScheduler, ioOpCallback, false, durable);
+        this.transactionComponentFactory = transactionComponentFactory;
         this.secondDiskComponents = new LinkedList<>();
         this.interiorFrameFactory = interiorFrameFactory;
     }
@@ -164,7 +162,8 @@ public class ExternalBTree extends LSMBTree implements ITwoPCIndex {
     // This method creates the appropriate opContext for the targeted version
     public ExternalBTreeOpContext createOpContext(ISearchOperationCallback searchCallback, int targetVersion) {
         return new ExternalBTreeOpContext(insertLeafFrameFactory, deleteLeafFrameFactory, searchCallback,
-                componentFactory.getBloomFilterKeyFields().length, cmpFactories, targetVersion, getLsmHarness());
+                ((LSMBTreeWithBloomFilterDiskComponentFactory) componentFactory).getBloomFilterKeyFields().length,
+                cmpFactories, targetVersion, getLsmHarness());
     }
 
     // The only reason to override the following method is that it uses a different context object
@@ -188,8 +187,8 @@ public class ExternalBTree extends LSMBTree implements ITwoPCIndex {
             }
         }
         ITreeIndexCursor cursor = new LSMBTreeRangeSearchCursor(opCtx, returnDeletedTuples);
-        BTree firstBTree = ((LSMBTreeDiskComponent) mergingComponents.get(0)).getBTree();
-        BTree lastBTree = ((LSMBTreeDiskComponent) mergingComponents.get(mergingComponents.size() - 1)).getBTree();
+        BTree firstBTree = ((LSMBTreeDiskComponent) mergingComponents.get(0)).getIndex();
+        BTree lastBTree = ((LSMBTreeDiskComponent) mergingComponents.get(mergingComponents.size() - 1)).getIndex();
         FileReference firstFile = firstBTree.getFileReference();
         FileReference lastFile = lastBTree.getFileReference();
         LSMComponentFileReferences relMergeFileRefs =
@@ -252,32 +251,23 @@ public class ExternalBTree extends LSMBTree implements ITwoPCIndex {
             //First time activation
             List<LSMComponentFileReferences> validFileReferences;
             validFileReferences = fileManager.cleanupAndGetValidFiles();
-            for (LSMComponentFileReferences lsmComonentFileReference : validFileReferences) {
-                LSMBTreeDiskComponent component;
-                component =
-                        createDiskComponent(componentFactory, lsmComonentFileReference.getInsertIndexFileReference(),
-                                lsmComonentFileReference.getBloomFilterFileReference(), false);
+            for (LSMComponentFileReferences lsmComponentFileReferences : validFileReferences) {
+                ILSMDiskComponent component =
+                        createDiskComponent(componentFactory, lsmComponentFileReferences.getInsertIndexFileReference(),
+                                null, lsmComponentFileReferences.getBloomFilterFileReference(), false);
                 diskComponents.add(component);
                 secondDiskComponents.add(component);
             }
             getLsmHarness().indexFirstTimeActivated();
         } else {
             // This index has been opened before
-            for (ILSMComponent c : diskComponents) {
-                LSMBTreeDiskComponent component = (LSMBTreeDiskComponent) c;
-                BTree btree = component.getBTree();
-                BloomFilter bloomFilter = component.getBloomFilter();
-                btree.activate();
-                bloomFilter.activate();
+            for (ILSMDiskComponent c : diskComponents) {
+                c.activate(false);
             }
-            for (ILSMComponent c : secondDiskComponents) {
+            for (ILSMDiskComponent c : secondDiskComponents) {
                 // Only activate non shared components
                 if (!diskComponents.contains(c)) {
-                    LSMBTreeDiskComponent component = (LSMBTreeDiskComponent) c;
-                    BTree btree = component.getBTree();
-                    BloomFilter bloomFilter = component.getBloomFilter();
-                    btree.activate();
-                    bloomFilter.activate();
+                    c.activate(false);
                 }
             }
         }
@@ -300,12 +290,12 @@ public class ExternalBTree extends LSMBTree implements ITwoPCIndex {
             cb.afterFinalize(LSMOperationType.FLUSH, null);
         }
         for (ILSMDiskComponent c : diskComponents) {
-            deactivateDiskComponent(c);
+            c.deactivateAndPurge();
         }
         for (ILSMDiskComponent c : secondDiskComponents) {
             // Only deactivate non shared components
             if (!diskComponents.contains(c)) {
-                deactivateDiskComponent(c);
+                c.deactivateAndPurge();
             }
         }
         isActive = false;
@@ -320,22 +310,14 @@ public class ExternalBTree extends LSMBTree implements ITwoPCIndex {
         }
         getLsmHarness().indexClear();
 
-        for (ILSMComponent c : diskComponents) {
-            LSMBTreeDiskComponent component = (LSMBTreeDiskComponent) c;
-            component.getBloomFilter().deactivate();
-            component.getBTree().deactivate();
-            component.getBloomFilter().destroy();
-            component.getBTree().destroy();
+        for (ILSMDiskComponent c : diskComponents) {
+            c.deactivateAndDestroy();
             // Remove from second list to avoid destroying twice
             secondDiskComponents.remove(c);
         }
 
-        for (ILSMComponent c : secondDiskComponents) {
-            LSMBTreeDiskComponent component = (LSMBTreeDiskComponent) c;
-            component.getBloomFilter().deactivate();
-            component.getBTree().deactivate();
-            component.getBloomFilter().destroy();
-            component.getBTree().destroy();
+        for (ILSMDiskComponent c : secondDiskComponents) {
+            c.deactivateAndDestroy();
         }
         diskComponents.clear();
         secondDiskComponents.clear();
@@ -348,12 +330,12 @@ public class ExternalBTree extends LSMBTree implements ITwoPCIndex {
             throw new HyracksDataException("Failed to destroy the index since it is activated.");
         }
         for (ILSMDiskComponent c : diskComponents) {
-            destroyDiskComponent(c);
+            c.destroy();
             // Remove from second list to avoid destroying twice
             secondDiskComponents.remove(c);
         }
         for (ILSMDiskComponent c : secondDiskComponents) {
-            destroyDiskComponent(c);
+            c.destroy();
         }
         diskComponents.clear();
         secondDiskComponents.clear();
@@ -448,7 +430,7 @@ public class ExternalBTree extends LSMBTree implements ITwoPCIndex {
             }
 
             componentBulkLoader =
-                    createComponentBulkLoader(component, fillFactor, verifyInput, numElementsHint, false, true, true);
+                    component.createBulkLoader(fillFactor, verifyInput, numElementsHint, false, true, true);
         }
 
         // It is expected that the mode was set to insert operation before
@@ -466,10 +448,7 @@ public class ExternalBTree extends LSMBTree implements ITwoPCIndex {
                     // Since this is a transaction component, validate and
                     // deactivate. it could later be added or deleted
                     component.markAsValid(durable);
-                    BTree btree = ((LSMBTreeDiskComponent) component).getBTree();
-                    BloomFilter bloomFilter = ((LSMBTreeDiskComponent) component).getBloomFilter();
-                    btree.deactivate();
-                    bloomFilter.deactivate();
+                    component.deactivate();
                 } else {
                     getLsmHarness().addBulkLoadedComponent(component);
                 }
@@ -502,7 +481,7 @@ public class ExternalBTree extends LSMBTree implements ITwoPCIndex {
                 throw HyracksDataException.create(e);
             }
             return createDiskComponent(transactionComponentFactory, componentFileRefs.getInsertIndexFileReference(),
-                    componentFileRefs.getBloomFilterFileReference(), true);
+                    null, componentFileRefs.getBloomFilterFileReference(), true);
         }
     }
 
@@ -568,11 +547,11 @@ public class ExternalBTree extends LSMBTree implements ITwoPCIndex {
 
     @Override
     public void commitTransaction() throws HyracksDataException {
-        LSMComponentFileReferences componentFileRefrences = fileManager.getTransactionFileReferenceForCommit();
-        LSMBTreeDiskComponent component = null;
-        if (componentFileRefrences != null) {
-            component = createDiskComponent(componentFactory, componentFileRefrences.getInsertIndexFileReference(),
-                    componentFileRefrences.getBloomFilterFileReference(), false);
+        LSMComponentFileReferences componentFileReferences = fileManager.getTransactionFileReferenceForCommit();
+        ILSMDiskComponent component = null;
+        if (componentFileReferences != null) {
+            component = createDiskComponent(componentFactory, componentFileReferences.getInsertIndexFileReference(),
+                    null, componentFileReferences.getBloomFilterFileReference(), false);
         }
         getLsmHarness().addTransactionComponents(component);
     }

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/77f89525/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/ExternalBTreeWithBuddy.java
----------------------------------------------------------------------
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 ff17905..55090ad 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
@@ -19,10 +19,8 @@
 package org.apache.hyracks.storage.am.lsm.btree.impls;
 
 import java.io.IOException;
-import java.util.HashSet;
 import java.util.LinkedList;
 import java.util.List;
-import java.util.Set;
 
 import org.apache.hyracks.api.dataflow.value.IBinaryComparatorFactory;
 import org.apache.hyracks.api.exceptions.ErrorCode;
@@ -30,10 +28,6 @@ import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.api.io.FileReference;
 import org.apache.hyracks.api.io.IIOManager;
 import org.apache.hyracks.dataflow.common.data.accessors.ITupleReference;
-import org.apache.hyracks.storage.am.bloomfilter.impls.BloomCalculations;
-import org.apache.hyracks.storage.am.bloomfilter.impls.BloomFilter;
-import org.apache.hyracks.storage.am.bloomfilter.impls.BloomFilterFactory;
-import org.apache.hyracks.storage.am.bloomfilter.impls.BloomFilterSpecification;
 import org.apache.hyracks.storage.am.btree.impls.BTree;
 import org.apache.hyracks.storage.am.btree.impls.RangePredicate;
 import org.apache.hyracks.storage.am.common.api.IIndexOperationContext;
@@ -44,6 +38,7 @@ import org.apache.hyracks.storage.am.common.api.ITreeIndexFrameFactory;
 import org.apache.hyracks.storage.am.common.api.ITwoPCIndexBulkLoader;
 import org.apache.hyracks.storage.am.common.impls.NoOpOperationCallback;
 import org.apache.hyracks.storage.am.common.ophelpers.IndexOperation;
+import org.apache.hyracks.storage.am.lsm.common.api.AbstractLSMWithBloomFilterDiskComponent;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponent;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMDiskComponent;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMDiskComponentBulkLoader;
@@ -54,7 +49,6 @@ import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationScheduler;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndexAccessor;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndexFileManager;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndexOperationContext;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMMemoryComponent;
 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;
@@ -64,22 +58,15 @@ import org.apache.hyracks.storage.am.lsm.common.impls.AbstractLSMIndexOperationC
 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.TreeIndexFactory;
 import org.apache.hyracks.storage.common.IIndexBulkLoader;
 import org.apache.hyracks.storage.common.IIndexCursor;
 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;
 import org.apache.hyracks.storage.common.buffercache.IBufferCache;
 
 public class ExternalBTreeWithBuddy extends AbstractLSMIndex implements ITreeIndex, ITwoPCIndex {
 
-    // For creating merge disk components
-    private final LSMBTreeWithBuddyDiskComponentFactory componentFactory;
-
-    private final LSMBTreeWithBuddyDiskComponentFactory bulkComponentFactory;
-
     private final IBinaryComparatorFactory[] btreeCmpFactories;
     private final IBinaryComparatorFactory[] buddyBtreeCmpFactories;
     private final int[] buddyBTreeFields;
@@ -96,25 +83,19 @@ public class ExternalBTreeWithBuddy extends AbstractLSMIndex implements ITreeInd
 
     public ExternalBTreeWithBuddy(IIOManager ioManager, ITreeIndexFrameFactory btreeInteriorFrameFactory,
             ITreeIndexFrameFactory btreeLeafFrameFactory, ITreeIndexFrameFactory buddyBtreeLeafFrameFactory,
-            IBufferCache diskBufferCache, ILSMIndexFileManager fileManager,
-            TreeIndexFactory<BTree> bulkLoadBTreeFactory, TreeIndexFactory<BTree> copyBtreeFactory,
-            TreeIndexFactory<BTree> buddyBtreeFactory, BloomFilterFactory bloomFilterFactory,
-            double bloomFilterFalsePositiveRate, ILSMMergePolicy mergePolicy, ILSMOperationTracker opTracker,
-            ILSMIOOperationScheduler ioScheduler, ILSMIOOperationCallback ioOpCallback,
-            IBinaryComparatorFactory[] btreeCmpFactories, IBinaryComparatorFactory[] buddyBtreeCmpFactories,
-            int[] buddyBTreeFields, boolean durable) {
+            IBufferCache diskBufferCache, ILSMIndexFileManager fileManager, ILSMDiskComponentFactory componentFactory,
+            ILSMDiskComponentFactory bulkLoadComponentFactory, double bloomFilterFalsePositiveRate,
+            ILSMMergePolicy mergePolicy, ILSMOperationTracker opTracker, ILSMIOOperationScheduler ioScheduler,
+            ILSMIOOperationCallback ioOpCallback, IBinaryComparatorFactory[] btreeCmpFactories,
+            IBinaryComparatorFactory[] buddyBtreeCmpFactories, int[] buddyBTreeFields, boolean durable) {
         super(ioManager, diskBufferCache, fileManager, bloomFilterFalsePositiveRate, mergePolicy, opTracker,
-                ioScheduler, ioOpCallback, durable);
+                ioScheduler, ioOpCallback, componentFactory, bulkLoadComponentFactory, durable);
         this.btreeCmpFactories = btreeCmpFactories;
         this.buddyBtreeCmpFactories = buddyBtreeCmpFactories;
         this.buddyBTreeFields = buddyBTreeFields;
         this.btreeInteriorFrameFactory = btreeInteriorFrameFactory;
         this.btreeLeafFrameFactory = btreeLeafFrameFactory;
         this.buddyBtreeLeafFrameFactory = buddyBtreeLeafFrameFactory;
-        this.componentFactory =
-                new LSMBTreeWithBuddyDiskComponentFactory(copyBtreeFactory, buddyBtreeFactory, bloomFilterFactory);
-        this.bulkComponentFactory =
-                new LSMBTreeWithBuddyDiskComponentFactory(bulkLoadBTreeFactory, buddyBtreeFactory, bloomFilterFactory);
         this.secondDiskComponents = new LinkedList<>();
     }
 
@@ -125,22 +106,16 @@ public class ExternalBTreeWithBuddy extends AbstractLSMIndex implements ITreeInd
     }
 
     @Override
-    protected ILSMDiskComponent loadComponent(LSMComponentFileReferences refs) throws HyracksDataException {
-        return null;
-    }
-
-    @Override
     public void activate() throws HyracksDataException {
         if (isActive) {
             throw new HyracksDataException("Failed to activate the index since it is already activated.");
         }
-
         if (diskComponents.size() == 0 && secondDiskComponents.size() == 0) {
             //First time activation
             List<LSMComponentFileReferences> validFileReferences;
             validFileReferences = fileManager.cleanupAndGetValidFiles();
             for (LSMComponentFileReferences lsmComonentFileReference : validFileReferences) {
-                LSMBTreeWithBuddyDiskComponent component;
+                ILSMDiskComponent component;
                 component =
                         createDiskComponent(componentFactory, lsmComonentFileReference.getInsertIndexFileReference(),
                                 lsmComonentFileReference.getDeleteIndexFileReference(),
@@ -152,25 +127,13 @@ public class ExternalBTreeWithBuddy extends AbstractLSMIndex implements ITreeInd
         } else {
             // This index has been opened before or is brand new with no
             // components. It should also maintain the version pointer
-            for (ILSMComponent c : diskComponents) {
-                LSMBTreeWithBuddyDiskComponent component = (LSMBTreeWithBuddyDiskComponent) c;
-                BTree btree = component.getBTree();
-                BTree buddyBtree = component.getBuddyBTree();
-                BloomFilter bloomFilter = component.getBloomFilter();
-                btree.activate();
-                buddyBtree.activate();
-                bloomFilter.activate();
+            for (ILSMDiskComponent c : diskComponents) {
+                c.activate(false);
             }
-            for (ILSMComponent c : secondDiskComponents) {
+            for (ILSMDiskComponent c : secondDiskComponents) {
                 // Only activate non shared components
                 if (!diskComponents.contains(c)) {
-                    LSMBTreeWithBuddyDiskComponent component = (LSMBTreeWithBuddyDiskComponent) c;
-                    BTree btree = component.getBTree();
-                    BTree buddyBtree = component.getBuddyBTree();
-                    BloomFilter bloomFilter = component.getBloomFilter();
-                    btree.activate();
-                    buddyBtree.activate();
-                    bloomFilter.activate();
+                    c.activate(false);
                 }
             }
         }
@@ -184,12 +147,12 @@ public class ExternalBTreeWithBuddy extends AbstractLSMIndex implements ITreeInd
         }
         ((ExternalIndexHarness) getLsmHarness()).indexClear();
         for (ILSMDiskComponent c : diskComponents) {
-            clearDiskComponent(c);
+            c.deactivateAndDestroy();
             // Remove from second list to avoid destroying twice
             secondDiskComponents.remove(c);
         }
         for (ILSMDiskComponent c : secondDiskComponents) {
-            clearDiskComponent(c);
+            c.deactivateAndDestroy();
         }
         diskComponents.clear();
         secondDiskComponents.clear();
@@ -197,28 +160,17 @@ public class ExternalBTreeWithBuddy extends AbstractLSMIndex implements ITreeInd
     }
 
     @Override
-    protected void clearDiskComponent(ILSMDiskComponent c) throws HyracksDataException {
-        LSMBTreeWithBuddyDiskComponent component = (LSMBTreeWithBuddyDiskComponent) c;
-        component.getBTree().deactivate();
-        component.getBuddyBTree().deactivate();
-        component.getBloomFilter().deactivate();
-        component.getBTree().destroy();
-        component.getBloomFilter().destroy();
-        component.getBuddyBTree().destroy();
-    }
-
-    @Override
     public void destroy() throws HyracksDataException {
         if (isActive) {
             throw new HyracksDataException("Failed to destroy the index since it is activated.");
         }
         for (ILSMDiskComponent c : diskComponents) {
-            destroyDiskComponent(c);
+            c.destroy();
             // Remove from second list to avoid destroying twice
             secondDiskComponents.remove(c);
         }
         for (ILSMDiskComponent c : secondDiskComponents) {
-            destroyDiskComponent(c);
+            c.destroy();
         }
         diskComponents.clear();
         secondDiskComponents.clear();
@@ -227,14 +179,6 @@ public class ExternalBTreeWithBuddy extends AbstractLSMIndex implements ITreeInd
     }
 
     @Override
-    protected void destroyDiskComponent(ILSMDiskComponent c) throws HyracksDataException {
-        LSMBTreeWithBuddyDiskComponent component = (LSMBTreeWithBuddyDiskComponent) c;
-        component.getBTree().destroy();
-        component.getBuddyBTree().destroy();
-        component.getBloomFilter().destroy();
-    }
-
-    @Override
     public ILSMIndexAccessor createAccessor(IModificationOperationCallback modificationCallback,
             ISearchOperationCallback searchCallback) throws HyracksDataException {
         return new LSMTreeIndexAccessor(getLsmHarness(), createOpContext(searchCallback, version),
@@ -269,27 +213,6 @@ public class ExternalBTreeWithBuddy extends AbstractLSMIndex implements ITreeInd
         newerList.add(swapIndex, newComponent);
     }
 
-    @Override
-    public ILSMDiskComponentBulkLoader createComponentBulkLoader(ILSMDiskComponent component, float fillFactor,
-            boolean verifyInput, long numElementsHint, boolean checkIfEmptyIndex, boolean withFilter,
-            boolean cleanupEmptyComponent) throws HyracksDataException {
-        BloomFilterSpecification bloomFilterSpec = null;
-        if (numElementsHint > 0) {
-            int maxBucketsPerElement = BloomCalculations.maxBucketsPerElement(numElementsHint);
-            bloomFilterSpec = BloomCalculations.computeBloomSpec(maxBucketsPerElement, bloomFilterFalsePositiveRate);
-        }
-        if (withFilter && filterFields != null) {
-            return new LSMBTreeWithBuddyDiskComponentBulkLoader((LSMBTreeWithBuddyDiskComponent) component,
-                    bloomFilterSpec, fillFactor, verifyInput, numElementsHint, checkIfEmptyIndex, cleanupEmptyComponent,
-                    filterManager, treeFields, filterFields,
-                    MultiComparator.create(component.getLSMComponentFilter().getFilterCmpFactories()));
-        } else {
-            return new LSMBTreeWithBuddyDiskComponentBulkLoader((LSMBTreeWithBuddyDiskComponent) component,
-                    bloomFilterSpec, fillFactor, verifyInput, numElementsHint, checkIfEmptyIndex,
-                    cleanupEmptyComponent);
-        }
-    }
-
     // For initial load
     @Override
     public IIndexBulkLoader createBulkLoader(float fillLevel, boolean verifyInput, long numElementsHint)
@@ -331,9 +254,9 @@ public class ExternalBTreeWithBuddy extends AbstractLSMIndex implements ITreeInd
 
     protected LSMComponentFileReferences getMergeTargetFileName(List<ILSMComponent> mergingDiskComponents)
             throws HyracksDataException {
-        BTree firstTree = ((LSMBTreeWithBuddyDiskComponent) mergingDiskComponents.get(0)).getBTree();
+        BTree firstTree = ((LSMBTreeWithBuddyDiskComponent) mergingDiskComponents.get(0)).getIndex();
         BTree lastTree = ((LSMBTreeWithBuddyDiskComponent) mergingDiskComponents.get(mergingDiskComponents.size() - 1))
-                .getBTree();
+                .getIndex();
         FileReference firstFile = firstTree.getFileReference();
         FileReference lastFile = lastTree.getFileReference();
         LSMComponentFileReferences fileRefs =
@@ -385,10 +308,10 @@ public class ExternalBTreeWithBuddy extends AbstractLSMIndex implements ITreeInd
         ILSMIndexOperationContext opCtx = ((LSMBTreeWithBuddySortedCursor) cursor).getOpCtx();
         search(opCtx, cursor, btreeSearchPred);
 
-        LSMBTreeWithBuddyDiskComponent mergedComponent = createDiskComponent(componentFactory, mergeOp.getTarget(),
+        ILSMDiskComponent mergedComponent = createDiskComponent(componentFactory, mergeOp.getTarget(),
                 mergeOp.getBuddyBTreeTarget(), mergeOp.getBloomFilterTarget(), true);
 
-        IIndexBulkLoader componentBulkLoader;
+        ILSMDiskComponentBulkLoader componentBulkLoader;
 
         // In case we must keep the deleted-keys BuddyBTrees, then they must be
         // merged *before* merging the b-trees so that
@@ -403,24 +326,23 @@ public class ExternalBTreeWithBuddy extends AbstractLSMIndex implements ITreeInd
 
             long numElements = 0L;
             for (int i = 0; i < mergeOp.getMergingComponents().size(); ++i) {
-                numElements += ((LSMBTreeWithBuddyDiskComponent) mergeOp.getMergingComponents().get(i)).getBloomFilter()
-                        .getNumElements();
+                numElements += ((AbstractLSMWithBloomFilterDiskComponent) mergeOp.getMergingComponents().get(i))
+                        .getBloomFilter().getNumElements();
             }
 
-            componentBulkLoader =
-                    createComponentBulkLoader(mergedComponent, 1.0f, false, numElements, false, false, false);
+            componentBulkLoader = mergedComponent.createBulkLoader(1.0f, false, numElements, false, false, false);
 
             try {
                 while (buddyBtreeCursor.hasNext()) {
                     buddyBtreeCursor.next();
                     ITupleReference tuple = buddyBtreeCursor.getTuple();
-                    ((LSMBTreeWithBuddyDiskComponentBulkLoader) componentBulkLoader).delete(tuple);
+                    componentBulkLoader.delete(tuple);
                 }
             } finally {
                 buddyBtreeCursor.close();
             }
         } else {
-            componentBulkLoader = createComponentBulkLoader(mergedComponent, 1.0f, false, 0L, false, false, false);
+            componentBulkLoader = mergedComponent.createBulkLoader(1.0f, false, 0L, false, false, false);
         }
 
         try {
@@ -515,12 +437,12 @@ public class ExternalBTreeWithBuddy extends AbstractLSMIndex implements ITreeInd
         // modify any of the lists to make sure they
         // are there if the index was opened again
         for (ILSMDiskComponent c : diskComponents) {
-            deactivateDiskComponent(c);
+            c.deactivateAndPurge();
         }
         for (ILSMDiskComponent c : secondDiskComponents) {
             // Only deactivate non shared components
             if (!diskComponents.contains(c)) {
-                deactivateDiskComponent(c);
+                c.deactivateAndPurge();
             }
         }
         isActive = false;
@@ -564,23 +486,6 @@ public class ExternalBTreeWithBuddy extends AbstractLSMIndex implements ITreeInd
         return btreeCmpFactories;
     }
 
-    private LSMBTreeWithBuddyDiskComponent createDiskComponent(ILSMDiskComponentFactory factory,
-            FileReference insertFileRef, FileReference deleteFileRef, FileReference bloomFilterFileRef,
-            boolean createComponent) throws HyracksDataException {
-        // Create new instance.
-        LSMBTreeWithBuddyDiskComponent component = (LSMBTreeWithBuddyDiskComponent) factory
-                .createComponent(new LSMComponentFileReferences(insertFileRef, deleteFileRef, bloomFilterFileRef));
-        if (createComponent) {
-            component.getBTree().create();
-            component.getBuddyBTree().create();
-            component.getBloomFilter().create();
-        }
-        component.getBTree().activate();
-        component.getBuddyBTree().activate();
-        component.getBloomFilter().activate();
-        return component;
-    }
-
     // even though the index doesn't support record level modification, the
     // accessor will try to do it
     // we could throw the exception here but we don't. it will eventually be
@@ -590,7 +495,7 @@ public class ExternalBTreeWithBuddy extends AbstractLSMIndex implements ITreeInd
     // modifications
     public class LSMTwoPCBTreeWithBuddyBulkLoader implements IIndexBulkLoader, ITwoPCIndexBulkLoader {
         private final ILSMDiskComponent component;
-        private final IIndexBulkLoader componentBulkLoader;
+        private final ILSMDiskComponentBulkLoader componentBulkLoader;
         private final boolean isTransaction;
 
         public LSMTwoPCBTreeWithBuddyBulkLoader(float fillFactor, boolean verifyInput, long numElementsHint,
@@ -604,7 +509,7 @@ public class ExternalBTreeWithBuddy extends AbstractLSMIndex implements ITreeInd
             }
 
             componentBulkLoader =
-                    createComponentBulkLoader(component, fillFactor, verifyInput, numElementsHint, false, true, false);
+                    component.createBulkLoader(fillFactor, verifyInput, numElementsHint, false, true, false);
         }
 
         @Override
@@ -620,12 +525,7 @@ public class ExternalBTreeWithBuddy extends AbstractLSMIndex implements ITreeInd
                     // Since this is a transaction component, validate and
                     // deactivate. it could later be added or deleted
                     component.markAsValid(durable);
-                    BTree btree = ((LSMBTreeWithBuddyDiskComponent) component).getBTree();
-                    BTree buddyBtree = ((LSMBTreeWithBuddyDiskComponent) component).getBuddyBTree();
-                    BloomFilter bloomFilter = ((LSMBTreeWithBuddyDiskComponent) component).getBloomFilter();
-                    btree.deactivate();
-                    buddyBtree.deactivate();
-                    bloomFilter.deactivate();
+                    component.deactivate();
                 } else {
                     getLsmHarness().addBulkLoadedComponent(component);
                 }
@@ -634,7 +534,7 @@ public class ExternalBTreeWithBuddy extends AbstractLSMIndex implements ITreeInd
 
         @Override
         public void delete(ITupleReference tuple) throws HyracksDataException {
-            ((LSMBTreeWithBuddyDiskComponentBulkLoader) componentBulkLoader).delete(tuple);
+            componentBulkLoader.delete(tuple);
         }
 
         @Override
@@ -654,20 +554,13 @@ public class ExternalBTreeWithBuddy extends AbstractLSMIndex implements ITreeInd
             } catch (IOException e) {
                 throw HyracksDataException.create(e);
             }
-            return createDiskComponent(bulkComponentFactory, componentFileRefs.getInsertIndexFileReference(),
+            return createDiskComponent(bulkLoadComponentFactory, componentFileRefs.getInsertIndexFileReference(),
                     componentFileRefs.getDeleteIndexFileReference(), componentFileRefs.getBloomFilterFileReference(),
                     true);
         }
     }
 
     @Override
-    public ILSMDiskComponent createBulkLoadTarget() throws HyracksDataException {
-        LSMComponentFileReferences componentFileRefs = fileManager.getRelFlushFileReference();
-        return createDiskComponent(bulkComponentFactory, componentFileRefs.getInsertIndexFileReference(),
-                componentFileRefs.getDeleteIndexFileReference(), componentFileRefs.getBloomFilterFileReference(), true);
-    }
-
-    @Override
     public ILSMIndexAccessor createAccessor(ISearchOperationCallback searchCallback, int targetIndexVersion)
             throws HyracksDataException {
         return new LSMTreeIndexAccessor(getLsmHarness(), createOpContext(searchCallback, targetIndexVersion),
@@ -709,7 +602,7 @@ public class ExternalBTreeWithBuddy extends AbstractLSMIndex implements ITreeInd
     @Override
     public void commitTransaction() throws HyracksDataException {
         LSMComponentFileReferences componentFileRefrences = fileManager.getTransactionFileReferenceForCommit();
-        LSMBTreeWithBuddyDiskComponent component = null;
+        ILSMDiskComponent component = null;
         if (componentFileRefrences != null) {
             component = createDiskComponent(componentFactory, componentFileRefrences.getInsertIndexFileReference(),
                     componentFileRefrences.getDeleteIndexFileReference(),
@@ -739,60 +632,6 @@ public class ExternalBTreeWithBuddy extends AbstractLSMIndex implements ITreeInd
     }
 
     @Override
-    public Set<String> getLSMComponentPhysicalFiles(ILSMComponent lsmComponent) {
-        Set<String> files = new HashSet<>();
-        LSMBTreeWithBuddyDiskComponent component = (LSMBTreeWithBuddyDiskComponent) lsmComponent;
-        files.add(component.getBTree().getFileReference().getFile().getAbsolutePath());
-        files.add(component.getBuddyBTree().getFileReference().getFile().getAbsolutePath());
-        files.add(component.getBloomFilter().getFileReference().getFile().getAbsolutePath());
-        return files;
-    }
-
-    @Override
-    protected void deactivateMemoryComponent(ILSMMemoryComponent c) throws HyracksDataException {
-        //do nothing since external index never use memory components
-    }
-
-    @Override
-    protected void deactivateDiskComponent(ILSMDiskComponent c) throws HyracksDataException {
-        LSMBTreeWithBuddyDiskComponent component = (LSMBTreeWithBuddyDiskComponent) c;
-        BTree btree = component.getBTree();
-        BTree buddyBtree = component.getBuddyBTree();
-        BloomFilter bloomFilter = component.getBloomFilter();
-        btree.deactivate();
-        buddyBtree.deactivate();
-        bloomFilter.deactivate();
-        btree.purge();
-        buddyBtree.purge();
-        bloomFilter.purge();
-    }
-
-    @Override
-    protected void clearMemoryComponent(ILSMMemoryComponent c) throws HyracksDataException {
-        //do nothing since external index never use memory components
-    }
-
-    @Override
-    protected long getMemoryComponentSize(ILSMMemoryComponent c) {
-        return 0;
-    }
-
-    @Override
-    protected void validateMemoryComponent(ILSMMemoryComponent c) throws HyracksDataException {
-        throw new UnsupportedOperationException("Validation not implemented for LSM B-Trees with Buddy B-Tree.");
-    }
-
-    @Override
-    protected void validateDiskComponent(ILSMDiskComponent c) throws HyracksDataException {
-        throw new UnsupportedOperationException("Validation not implemented for LSM B-Trees with Buddy B-Tree.");
-    }
-
-    @Override
-    protected void allocateMemoryComponent(ILSMMemoryComponent c) throws HyracksDataException {
-        //do nothing since external index never use memory components
-    }
-
-    @Override
     protected LSMComponentFileReferences getMergeFileReferences(ILSMDiskComponent firstComponent,
             ILSMDiskComponent lastComponent) throws HyracksDataException {
         return null;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/77f89525/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTree.java
----------------------------------------------------------------------
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 c7d45e1..f6942b0 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
@@ -20,9 +20,7 @@
 package org.apache.hyracks.storage.am.lsm.btree.impls;
 
 import java.util.ArrayList;
-import java.util.HashSet;
 import java.util.List;
-import java.util.Set;
 
 import org.apache.hyracks.api.dataflow.value.IBinaryComparatorFactory;
 import org.apache.hyracks.api.exceptions.ErrorCode;
@@ -31,9 +29,6 @@ import org.apache.hyracks.api.io.FileReference;
 import org.apache.hyracks.api.io.IIOManager;
 import org.apache.hyracks.data.std.primitive.IntegerPointable;
 import org.apache.hyracks.dataflow.common.data.accessors.ITupleReference;
-import org.apache.hyracks.storage.am.bloomfilter.impls.BloomCalculations;
-import org.apache.hyracks.storage.am.bloomfilter.impls.BloomFilterFactory;
-import org.apache.hyracks.storage.am.bloomfilter.impls.BloomFilterSpecification;
 import org.apache.hyracks.storage.am.btree.impls.BTree;
 import org.apache.hyracks.storage.am.btree.impls.BTree.BTreeAccessor;
 import org.apache.hyracks.storage.am.btree.impls.RangePredicate;
@@ -44,18 +39,19 @@ import org.apache.hyracks.storage.am.common.api.ITreeIndexCursor;
 import org.apache.hyracks.storage.am.common.api.ITreeIndexFrameFactory;
 import org.apache.hyracks.storage.am.common.impls.NoOpOperationCallback;
 import org.apache.hyracks.storage.am.lsm.btree.tuples.LSMBTreeTupleReference;
+import org.apache.hyracks.storage.am.lsm.common.api.AbstractLSMWithBloomFilterDiskComponent;
 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.ILSMComponentFilterFrameFactory;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMDiskComponent;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMDiskComponentBulkLoader;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMDiskComponentFactory;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperation;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallback;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationScheduler;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndexAccessor;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndexFileManager;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndexOperationContext;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMMemoryComponent;
 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;
@@ -66,9 +62,7 @@ import org.apache.hyracks.storage.am.lsm.common.impls.LSMComponentFileReferences
 import org.apache.hyracks.storage.am.lsm.common.impls.LSMComponentFilterManager;
 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.TreeIndexFactory;
 import org.apache.hyracks.storage.common.IIndexAccessor;
-import org.apache.hyracks.storage.common.IIndexBulkLoader;
 import org.apache.hyracks.storage.common.IIndexCursor;
 import org.apache.hyracks.storage.common.IModificationOperationCallback;
 import org.apache.hyracks.storage.common.ISearchOperationCallback;
@@ -80,12 +74,6 @@ import org.apache.hyracks.util.trace.ITracer;
 public class LSMBTree extends AbstractLSMIndex implements ITreeIndex {
 
     private static final ICursorFactory cursorFactory = opCtx -> new LSMBTreeSearchCursor(opCtx);
-    // For creating BTree's used in flush and merge.
-    protected final LSMBTreeDiskComponentFactory componentFactory;
-    // For creating BTree's used in bulk load. Different from diskBTreeFactory
-    // because it should have a different tuple writer in it's leaf frames.
-    protected final LSMBTreeDiskComponentFactory bulkLoadComponentFactory;
-
     // Common for in-memory and on-disk components.
     protected final ITreeIndexFrameFactory insertLeafFrameFactory;
     protected final ITreeIndexFrameFactory deleteLeafFrameFactory;
@@ -99,16 +87,16 @@ public class LSMBTree extends AbstractLSMIndex implements ITreeIndex {
 
     public LSMBTree(IIOManager ioManager, List<IVirtualBufferCache> virtualBufferCaches,
             ITreeIndexFrameFactory interiorFrameFactory, ITreeIndexFrameFactory insertLeafFrameFactory,
-            ITreeIndexFrameFactory deleteLeafFrameFactory, ILSMIndexFileManager fileManager,
-            TreeIndexFactory<BTree> diskBTreeFactory, TreeIndexFactory<BTree> bulkLoadBTreeFactory,
-            BloomFilterFactory bloomFilterFactory, IComponentFilterHelper filterHelper,
+            ITreeIndexFrameFactory deleteLeafFrameFactory, IBufferCache diskBufferCache,
+            ILSMIndexFileManager fileManager, ILSMDiskComponentFactory componentFactory,
+            ILSMDiskComponentFactory bulkLoadComponentFactory, IComponentFilterHelper filterHelper,
             ILSMComponentFilterFrameFactory filterFrameFactory, LSMComponentFilterManager filterManager,
             double bloomFilterFalsePositiveRate, int fieldCount, IBinaryComparatorFactory[] cmpFactories,
             ILSMMergePolicy mergePolicy, ILSMOperationTracker opTracker, ILSMIOOperationScheduler ioScheduler,
             ILSMIOOperationCallback ioOpCallback, boolean needKeyDupCheck, int[] btreeFields, int[] filterFields,
             boolean durable, boolean updateAware, ITracer tracer) throws HyracksDataException {
-        super(ioManager, virtualBufferCaches, diskBTreeFactory.getBufferCache(), fileManager,
-                bloomFilterFalsePositiveRate, mergePolicy, opTracker, ioScheduler, ioOpCallback, filterFrameFactory,
+        super(ioManager, virtualBufferCaches, diskBufferCache, fileManager, bloomFilterFalsePositiveRate, mergePolicy,
+                opTracker, ioScheduler, ioOpCallback, componentFactory, bulkLoadComponentFactory, filterFrameFactory,
                 filterManager, filterFields, durable, filterHelper, btreeFields, tracer);
         this.insertLeafFrameFactory = insertLeafFrameFactory;
         this.deleteLeafFrameFactory = deleteLeafFrameFactory;
@@ -125,35 +113,25 @@ public class LSMBTree extends AbstractLSMIndex implements ITreeIndex {
             memoryComponents.add(mutableComponent);
             ++i;
         }
-        componentFactory = new LSMBTreeDiskComponentFactory(diskBTreeFactory, bloomFilterFactory, filterHelper);
-        bulkLoadComponentFactory =
-                new LSMBTreeDiskComponentFactory(bulkLoadBTreeFactory, bloomFilterFactory, filterHelper);
         this.needKeyDupCheck = needKeyDupCheck;
         this.hasBloomFilter = needKeyDupCheck;
     }
 
     // Without memory components
     public LSMBTree(IIOManager ioManager, ITreeIndexFrameFactory insertLeafFrameFactory,
-            ITreeIndexFrameFactory deleteLeafFrameFactory, ILSMIndexFileManager fileManager,
-            TreeIndexFactory<BTree> diskBTreeFactory, TreeIndexFactory<BTree> bulkLoadBTreeFactory,
-            BloomFilterFactory bloomFilterFactory, double bloomFilterFalsePositiveRate,
-            IBinaryComparatorFactory[] cmpFactories, ILSMMergePolicy mergePolicy, ILSMOperationTracker opTracker,
-            ILSMIOOperationScheduler ioScheduler, ILSMIOOperationCallback ioOpCallback, boolean needKeyDupCheck,
-            boolean durable) {
-        super(ioManager, diskBTreeFactory.getBufferCache(), fileManager, bloomFilterFalsePositiveRate, mergePolicy,
-                opTracker, ioScheduler, ioOpCallback, durable);
+            ITreeIndexFrameFactory deleteLeafFrameFactory, IBufferCache bufferCache, ILSMIndexFileManager fileManager,
+            ILSMDiskComponentFactory componentFactory, ILSMDiskComponentFactory bulkLoadComponentFactory,
+            double bloomFilterFalsePositiveRate, IBinaryComparatorFactory[] cmpFactories, ILSMMergePolicy mergePolicy,
+            ILSMOperationTracker opTracker, ILSMIOOperationScheduler ioScheduler, ILSMIOOperationCallback ioOpCallback,
+            boolean needKeyDupCheck, boolean durable) {
+        super(ioManager, bufferCache, fileManager, bloomFilterFalsePositiveRate, mergePolicy, opTracker, ioScheduler,
+                ioOpCallback, componentFactory, bulkLoadComponentFactory, durable);
         this.insertLeafFrameFactory = insertLeafFrameFactory;
         this.deleteLeafFrameFactory = deleteLeafFrameFactory;
         this.cmpFactories = cmpFactories;
         this.needKeyDupCheck = needKeyDupCheck;
         this.hasBloomFilter = true;
         this.updateAware = false;
-        componentFactory = new LSMBTreeDiskComponentFactory(diskBTreeFactory, bloomFilterFactory, null);
-        bulkLoadComponentFactory = new LSMBTreeDiskComponentFactory(bulkLoadBTreeFactory, bloomFilterFactory, null);
-    }
-
-    public boolean hasBloomFilter() {
-        return hasBloomFilter;
     }
 
     @Override
@@ -167,35 +145,6 @@ public class LSMBTree extends AbstractLSMIndex implements ITreeIndex {
     }
 
     @Override
-    protected ILSMDiskComponent loadComponent(LSMComponentFileReferences lsmComonentFileReferences)
-            throws HyracksDataException {
-        return createDiskComponent(componentFactory, lsmComonentFileReferences.getInsertIndexFileReference(),
-                lsmComonentFileReferences.getBloomFilterFileReference(), false);
-    }
-
-    @Override
-    protected void destroyDiskComponent(ILSMDiskComponent c) throws HyracksDataException {
-        LSMBTreeDiskComponent component = (LSMBTreeDiskComponent) c;
-        component.getBTree().destroy();
-        if (hasBloomFilter) {
-            component.getBloomFilter().destroy();
-        }
-    }
-
-    @Override
-    protected void clearDiskComponent(ILSMDiskComponent c) throws HyracksDataException {
-        LSMBTreeDiskComponent component = (LSMBTreeDiskComponent) c;
-        if (hasBloomFilter) {
-            component.getBloomFilter().deactivate();
-        }
-        component.getBTree().deactivate();
-        if (hasBloomFilter) {
-            component.getBloomFilter().destroy();
-        }
-        component.getBTree().destroy();
-    }
-
-    @Override
     public void modify(IIndexOperationContext ictx, ITupleReference tuple) throws HyracksDataException {
         LSMBTreeOpContext ctx = (LSMBTreeOpContext) ictx;
         ITupleReference indexTuple;
@@ -297,7 +246,7 @@ public class LSMBTree extends AbstractLSMIndex implements ITreeIndex {
     public ILSMDiskComponent doFlush(ILSMIOOperation operation) throws HyracksDataException {
         LSMBTreeFlushOperation flushOp = (LSMBTreeFlushOperation) operation;
         LSMBTreeMemoryComponent flushingComponent = (LSMBTreeMemoryComponent) flushOp.getFlushingComponent();
-        IIndexAccessor accessor = flushingComponent.getBTree().createAccessor(NoOpOperationCallback.INSTANCE,
+        IIndexAccessor accessor = flushingComponent.getIndex().createAccessor(NoOpOperationCallback.INSTANCE,
                 NoOpOperationCallback.INSTANCE);
 
         RangePredicate nullPred = new RangePredicate(null, null, true, true, null, null);
@@ -317,11 +266,11 @@ public class LSMBTree extends AbstractLSMIndex implements ITreeIndex {
             }
         }
 
-        LSMBTreeDiskComponent component =
-                createDiskComponent(componentFactory, flushOp.getTarget(), flushOp.getBloomFilterTarget(), true);
+        ILSMDiskComponent component =
+                createDiskComponent(componentFactory, flushOp.getTarget(), null, flushOp.getBloomFilterTarget(), true);
 
         ILSMDiskComponentBulkLoader componentBulkLoader =
-                createComponentBulkLoader(component, 1.0f, false, numElements, false, false, false);
+                component.createBulkLoader(1.0f, false, numElements, false, false, false);
 
         IIndexCursor scanCursor = accessor.createSearchCursor(false);
         accessor.search(scanCursor, nullPred);
@@ -344,7 +293,7 @@ public class LSMBTree extends AbstractLSMIndex implements ITreeIndex {
             filterTuples.add(flushingComponent.getLSMComponentFilter().getMinTuple());
             filterTuples.add(flushingComponent.getLSMComponentFilter().getMaxTuple());
             getFilterManager().updateFilter(component.getLSMComponentFilter(), filterTuples);
-            getFilterManager().writeFilter(component.getLSMComponentFilter(), component.getBTree());
+            getFilterManager().writeFilter(component.getLSMComponentFilter(), component.getMetadataHolder());
         }
         // Write metadata from memory component to disk
         // Q. what about the merge operation? how do we resolve conflicts
@@ -370,14 +319,15 @@ public class LSMBTree extends AbstractLSMIndex implements ITreeIndex {
         if (hasBloomFilter) {
             //count elements in btree for creating Bloomfilter
             for (int i = 0; i < mergedComponents.size(); ++i) {
-                numElements += ((LSMBTreeDiskComponent) mergedComponents.get(i)).getBloomFilter().getNumElements();
+                numElements += ((AbstractLSMWithBloomFilterDiskComponent) mergedComponents.get(i)).getBloomFilter()
+                        .getNumElements();
             }
         }
-        LSMBTreeDiskComponent mergedComponent =
-                createDiskComponent(componentFactory, mergeOp.getTarget(), mergeOp.getBloomFilterTarget(), true);
+        ILSMDiskComponent mergedComponent =
+                createDiskComponent(componentFactory, mergeOp.getTarget(), null, mergeOp.getBloomFilterTarget(), true);
 
         ILSMDiskComponentBulkLoader componentBulkLoader =
-                createComponentBulkLoader(mergedComponent, 1.0f, false, numElements, false, false, false);
+                mergedComponent.createBulkLoader(1.0f, false, numElements, false, false, false);
         try {
             while (cursor.hasNext()) {
                 cursor.next();
@@ -394,69 +344,13 @@ public class LSMBTree extends AbstractLSMIndex implements ITreeIndex {
                 filterTuples.add(mergeOp.getMergingComponents().get(i).getLSMComponentFilter().getMaxTuple());
             }
             getFilterManager().updateFilter(mergedComponent.getLSMComponentFilter(), filterTuples);
-            getFilterManager().writeFilter(mergedComponent.getLSMComponentFilter(), mergedComponent.getBTree());
+            getFilterManager().writeFilter(mergedComponent.getLSMComponentFilter(),
+                    mergedComponent.getMetadataHolder());
         }
         componentBulkLoader.end();
         return mergedComponent;
     }
 
-    protected LSMBTreeDiskComponent createDiskComponent(LSMBTreeDiskComponentFactory factory,
-            FileReference btreeFileRef, FileReference bloomFilterFileRef, boolean createComponent)
-            throws HyracksDataException {
-        // Create new BTree instance.
-        LSMBTreeDiskComponent component =
-                factory.createComponent(new LSMComponentFileReferences(btreeFileRef, null, bloomFilterFileRef));
-        // BTree will be closed during cleanup of merge().
-        if (createComponent) {
-            component.getBTree().create();
-        }
-        component.getBTree().activate();
-        if (hasBloomFilter) {
-            if (createComponent) {
-                component.getBloomFilter().create();
-            }
-            component.getBloomFilter().activate();
-        }
-        if (component.getLSMComponentFilter() != null && !createComponent) {
-            getFilterManager().readFilter(component.getLSMComponentFilter(), component.getBTree());
-        }
-        return component;
-    }
-
-    @Override
-    public ILSMDiskComponentBulkLoader createComponentBulkLoader(ILSMDiskComponent component, float fillFactor,
-            boolean verifyInput, long numElementsHint, boolean checkIfEmptyIndex, boolean withFilter,
-            boolean cleanupEmptyComponent) throws HyracksDataException {
-        BloomFilterSpecification bloomFilterSpec = null;
-        if (hasBloomFilter) {
-            int maxBucketsPerElement = BloomCalculations.maxBucketsPerElement(numElementsHint);
-            bloomFilterSpec = BloomCalculations.computeBloomSpec(maxBucketsPerElement, bloomFilterFalsePositiveRate);
-        }
-
-        if (withFilter && filterFields != null) {
-            return new LSMBTreeDiskComponentBulkLoader((LSMBTreeDiskComponent) component, bloomFilterSpec, fillFactor,
-                    verifyInput, numElementsHint, checkIfEmptyIndex, cleanupEmptyComponent, filterManager, treeFields,
-                    filterFields, MultiComparator.create(component.getLSMComponentFilter().getFilterCmpFactories()));
-        } else {
-            return new LSMBTreeDiskComponentBulkLoader((LSMBTreeDiskComponent) component, bloomFilterSpec, fillFactor,
-                    verifyInput, numElementsHint, checkIfEmptyIndex, cleanupEmptyComponent);
-        }
-
-    }
-
-    @Override
-    public IIndexBulkLoader createBulkLoader(float fillLevel, boolean verifyInput, long numElementsHint)
-            throws HyracksDataException {
-        return new LSMBTreeBulkLoader(this, fillLevel, verifyInput, numElementsHint);
-    }
-
-    @Override
-    public ILSMDiskComponent createBulkLoadTarget() throws HyracksDataException {
-        LSMComponentFileReferences componentFileRefs = fileManager.getRelFlushFileReference();
-        return createDiskComponent(bulkLoadComponentFactory, componentFileRefs.getInsertIndexFileReference(),
-                componentFileRefs.getBloomFilterFileReference(), true);
-    }
-
     @Override
     protected ILSMIOOperation createFlushOperation(AbstractLSMIndexOperationContext opCtx,
             LSMComponentFileReferences componentFileRefs, ILSMIOOperationCallback callback) {
@@ -468,7 +362,9 @@ public class LSMBTree extends AbstractLSMIndex implements ITreeIndex {
     @Override
     public LSMBTreeOpContext createOpContext(IModificationOperationCallback modificationCallback,
             ISearchOperationCallback searchCallback) {
-        int numBloomFilterKeyFields = hasBloomFilter ? componentFactory.getBloomFilterKeyFields().length : 0;
+        int numBloomFilterKeyFields = hasBloomFilter
+                ? ((LSMBTreeWithBloomFilterDiskComponentFactory) componentFactory).getBloomFilterKeyFields().length
+                : 0;
         return new LSMBTreeOpContext(memoryComponents, insertLeafFrameFactory, deleteLeafFrameFactory,
                 modificationCallback, searchCallback, numBloomFilterKeyFields, getTreeFields(), getFilterFields(),
                 getLsmHarness(), getFilterCmpFactories());
@@ -488,113 +384,49 @@ public class LSMBTree extends AbstractLSMIndex implements ITreeIndex {
     public ITreeIndexFrameFactory getInteriorFrameFactory() {
         LSMBTreeMemoryComponent mutableComponent =
                 (LSMBTreeMemoryComponent) memoryComponents.get(currentMutableComponentId.get());
-        return mutableComponent.getBTree().getInteriorFrameFactory();
+        return mutableComponent.getIndex().getInteriorFrameFactory();
     }
 
     @Override
     public int getFieldCount() {
         LSMBTreeMemoryComponent mutableComponent =
                 (LSMBTreeMemoryComponent) memoryComponents.get(currentMutableComponentId.get());
-        return mutableComponent.getBTree().getFieldCount();
+        return mutableComponent.getIndex().getFieldCount();
     }
 
     @Override
     public int getFileId() {
         LSMBTreeMemoryComponent mutableComponent =
                 (LSMBTreeMemoryComponent) memoryComponents.get(currentMutableComponentId.get());
-        return mutableComponent.getBTree().getFileId();
+        return mutableComponent.getIndex().getFileId();
     }
 
     @Override
     public IPageManager getPageManager() {
         LSMBTreeMemoryComponent mutableComponent =
                 (LSMBTreeMemoryComponent) memoryComponents.get(currentMutableComponentId.get());
-        return mutableComponent.getBTree().getPageManager();
+        return mutableComponent.getIndex().getPageManager();
     }
 
     @Override
     public ITreeIndexFrameFactory getLeafFrameFactory() {
         LSMBTreeMemoryComponent mutableComponent =
                 (LSMBTreeMemoryComponent) memoryComponents.get(currentMutableComponentId.get());
-        return mutableComponent.getBTree().getLeafFrameFactory();
+        return mutableComponent.getIndex().getLeafFrameFactory();
     }
 
     @Override
     public int getRootPageId() {
         LSMBTreeMemoryComponent mutableComponent =
                 (LSMBTreeMemoryComponent) memoryComponents.get(currentMutableComponentId.get());
-        return mutableComponent.getBTree().getRootPageId();
-    }
-
-    @Override
-    protected long getMemoryComponentSize(ILSMMemoryComponent c) {
-        LSMBTreeMemoryComponent mutableComponent = (LSMBTreeMemoryComponent) c;
-        IBufferCache virtualBufferCache = mutableComponent.getBTree().getBufferCache();
-        return virtualBufferCache.getNumPages() * (long) virtualBufferCache.getPageSize();
-    }
-
-    @Override
-    public Set<String> getLSMComponentPhysicalFiles(ILSMComponent lsmComponent) {
-        Set<String> files = new HashSet<>();
-        LSMBTreeDiskComponent component = (LSMBTreeDiskComponent) lsmComponent;
-        files.add(component.getBTree().getFileReference().getFile().getAbsolutePath());
-        if (hasBloomFilter) {
-            files.add(component.getBloomFilter().getFileReference().getFile().getAbsolutePath());
-        }
-        return files;
-    }
-
-    @Override
-    protected void clearMemoryComponent(ILSMMemoryComponent c) throws HyracksDataException {
-        LSMBTreeMemoryComponent mutableComponent = (LSMBTreeMemoryComponent) c;
-        mutableComponent.getBTree().clear();
-        mutableComponent.reset();
-    }
-
-    @Override
-    protected void validateMemoryComponent(ILSMMemoryComponent c) throws HyracksDataException {
-        LSMBTreeMemoryComponent mutableComponent = (LSMBTreeMemoryComponent) c;
-        mutableComponent.getBTree().validate();
-    }
-
-    @Override
-    protected void validateDiskComponent(ILSMDiskComponent c) throws HyracksDataException {
-        BTree btree = ((LSMBTreeDiskComponent) c).getBTree();
-        btree.validate();
-    }
-
-    @Override
-    protected void deactivateDiskComponent(ILSMDiskComponent c) throws HyracksDataException {
-        LSMBTreeDiskComponent component = (LSMBTreeDiskComponent) c;
-        component.getBTree().deactivate();
-        component.getBTree().purge();
-        if (hasBloomFilter) {
-            component.getBloomFilter().deactivate();
-            component.getBloomFilter().purge();
-        }
-    }
-
-    @Override
-    protected void deactivateMemoryComponent(ILSMMemoryComponent c) throws HyracksDataException {
-        LSMBTreeMemoryComponent mutableComponent = (LSMBTreeMemoryComponent) c;
-        mutableComponent.getBTree().deactivate();
-        mutableComponent.getBTree().destroy();
-        ((IVirtualBufferCache) mutableComponent.getBTree().getBufferCache()).close();
-    }
-
-    @Override
-    protected void allocateMemoryComponent(ILSMMemoryComponent c) throws HyracksDataException {
-        LSMBTreeMemoryComponent mutableComponent = (LSMBTreeMemoryComponent) c;
-        ((IVirtualBufferCache) mutableComponent.getBTree().getBufferCache()).open();
-        mutableComponent.getBTree().create();
-        mutableComponent.getBTree().activate();
+        return mutableComponent.getIndex().getRootPageId();
     }
 
     @Override
     protected LSMComponentFileReferences getMergeFileReferences(ILSMDiskComponent firstComponent,
             ILSMDiskComponent lastComponent) throws HyracksDataException {
-        BTree firstBTree = ((LSMBTreeDiskComponent) firstComponent).getBTree();
-        BTree lastBTree = ((LSMBTreeDiskComponent) lastComponent).getBTree();
+        BTree firstBTree = (BTree) firstComponent.getIndex();
+        BTree lastBTree = (BTree) lastComponent.getIndex();
         FileReference firstFile = firstBTree.getFileReference();
         FileReference lastFile = lastBTree.getFileReference();
         return fileManager.getRelMergeFileReference(firstFile.getFile().getName(), lastFile.getFile().getName());