[ASTERIXDB-3026][STO][HYR] Part 2: Introduce LSMColumnBTree

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

Details:
This is the second part of introducing the columnar format in Hyracks.
This change contains the implementation of the ColumnBTree, its bulkloader,
and its cursors. This change also introduces an additional method to
IIndexAccessParameters to get a single parameter.

Change-Id: Icbb7346b84cabd3b5d30f4d3e1e24feae7880b9a
Reviewed-on: https://asterix-gerrit.ics.uci.edu/c/asterixdb/+/17225
Reviewed-by: Wail Alkowaileet <wael.y.k@gmail.com>
Reviewed-by: Murtadha Hubail <mhubail@apache.org>
Tested-by: Jenkins <jenkins@fulliautomatix.ics.uci.edu>
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/impls/IndexAccessParameters.java b/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/impls/IndexAccessParameters.java
index 11d3cd5..63902be 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/impls/IndexAccessParameters.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/impls/IndexAccessParameters.java
@@ -59,6 +59,11 @@
         return paramMap;
     }
 
+    @Override
+    public <T> T getParameter(String key, Class<T> clazz) {
+        return paramMap != null && clazz.isInstance(paramMap.get(key)) ? clazz.cast(paramMap.get(key)) : null;
+    }
+
     public static IIndexAccessParameters createNoOpParams(IIndexCursorStats stats) {
         if (stats == NoOpIndexCursorStats.INSTANCE) {
             return NoOpIndexAccessParameters.INSTANCE;
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/impls/NoOpIndexAccessParameters.java b/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/impls/NoOpIndexAccessParameters.java
index fa7811c..76a1930 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/impls/NoOpIndexAccessParameters.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/impls/NoOpIndexAccessParameters.java
@@ -47,4 +47,9 @@
     public Map<String, Object> getParameters() {
         return paramMap;
     }
+
+    @Override
+    public <T> T getParameter(String key, Class<T> clazz) {
+        return null;
+    }
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree-column/src/main/java/org/apache/hyracks/storage/am/lsm/btree/column/impls/btree/ColumnBTree.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree-column/src/main/java/org/apache/hyracks/storage/am/lsm/btree/column/impls/btree/ColumnBTree.java
new file mode 100644
index 0000000..fcee22c
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree-column/src/main/java/org/apache/hyracks/storage/am/lsm/btree/column/impls/btree/ColumnBTree.java
@@ -0,0 +1,94 @@
+/*
+ * 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.column.impls.btree;
+
+import org.apache.hyracks.api.dataflow.value.IBinaryComparatorFactory;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.io.FileReference;
+import org.apache.hyracks.api.util.HyracksConstants;
+import org.apache.hyracks.storage.am.btree.impls.DiskBTree;
+import org.apache.hyracks.storage.am.common.api.IPageManager;
+import org.apache.hyracks.storage.am.common.api.ITreeIndexCursor;
+import org.apache.hyracks.storage.am.common.api.ITreeIndexFrameFactory;
+import org.apache.hyracks.storage.am.lsm.btree.column.api.IColumnMetadata;
+import org.apache.hyracks.storage.am.lsm.btree.column.api.projection.IColumnProjectionInfo;
+import org.apache.hyracks.storage.common.IIndexAccessParameters;
+import org.apache.hyracks.storage.common.IIndexBulkLoader;
+import org.apache.hyracks.storage.common.IIndexCursorStats;
+import org.apache.hyracks.storage.common.NoOpIndexCursorStats;
+import org.apache.hyracks.storage.common.buffercache.IBufferCache;
+import org.apache.hyracks.storage.common.buffercache.IPageWriteCallback;
+
+public class ColumnBTree extends DiskBTree {
+    public ColumnBTree(IBufferCache bufferCache, IPageManager freePageManager,
+            ITreeIndexFrameFactory interiorFrameFactory, ITreeIndexFrameFactory leafFrameFactory,
+            IBinaryComparatorFactory[] cmpFactories, int fieldCount, FileReference file) {
+        super(bufferCache, freePageManager, interiorFrameFactory, leafFrameFactory, cmpFactories, fieldCount, file);
+    }
+
+    @Override
+    public IIndexBulkLoader createBulkLoader(float fillFactor, boolean verifyInput, long numElementsHint,
+            boolean checkIfEmptyIndex, IPageWriteCallback callback) {
+        throw new IllegalAccessError("Missing write column metadata");
+    }
+
+    public IIndexBulkLoader createBulkLoader(float fillFactor, boolean verifyInput, IPageWriteCallback callback,
+            IColumnMetadata columnMetadata) throws HyracksDataException {
+        ColumnBTreeLeafFrameFactory columnLeafFrameFactory = (ColumnBTreeLeafFrameFactory) leafFrameFactory;
+        ColumnBTreeWriteLeafFrame writeLeafFrame = columnLeafFrameFactory.createWriterFrame(columnMetadata);
+        return new ColumnBTreeBulkloader(fillFactor, verifyInput, callback, this, writeLeafFrame);
+    }
+
+    @Override
+    public BTreeAccessor createAccessor(IIndexAccessParameters iap) {
+        throw new IllegalArgumentException("Use createAccessor(IIndexAccessParameters, int, IColumnTupleProjector)");
+    }
+
+    public BTreeAccessor createAccessor(IIndexAccessParameters iap, int index, IColumnProjectionInfo projectionInfo) {
+        return new ColumnBTreeAccessor(this, iap, index, projectionInfo);
+    }
+
+    public class ColumnBTreeAccessor extends DiskBTreeAccessor {
+        private final int index;
+        private final IColumnProjectionInfo projectionInfo;
+
+        public ColumnBTreeAccessor(ColumnBTree btree, IIndexAccessParameters iap, int index,
+                IColumnProjectionInfo projectionInfo) {
+            super(btree, iap);
+            this.index = index;
+            this.projectionInfo = projectionInfo;
+        }
+
+        @Override
+        public ITreeIndexCursor createSearchCursor(boolean exclusive) {
+            ColumnBTreeLeafFrameFactory columnLeafFrameFactory = (ColumnBTreeLeafFrameFactory) leafFrameFactory;
+            ColumnBTreeReadLeafFrame readLeafFrame = columnLeafFrameFactory.createReadFrame(projectionInfo);
+            return new ColumnBTreeRangeSearchCursor(readLeafFrame, (IIndexCursorStats) iap.getParameters()
+                    .getOrDefault(HyracksConstants.INDEX_CURSOR_STATS, NoOpIndexCursorStats.INSTANCE), index);
+        }
+
+        @Override
+        public ITreeIndexCursor createPointCursor(boolean exclusive, boolean stateful) {
+            ColumnBTreeLeafFrameFactory columnLeafFrameFactory = (ColumnBTreeLeafFrameFactory) leafFrameFactory;
+            ColumnBTreeReadLeafFrame readLeafFrame = columnLeafFrameFactory.createReadFrame(projectionInfo);
+            return new ColumnBTreePointSearchCursor(readLeafFrame, (IIndexCursorStats) iap.getParameters()
+                    .getOrDefault(HyracksConstants.INDEX_CURSOR_STATS, NoOpIndexCursorStats.INSTANCE), index);
+        }
+    }
+}
\ No newline at end of file
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree-column/src/main/java/org/apache/hyracks/storage/am/lsm/btree/column/impls/btree/ColumnBTreeBulkloader.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree-column/src/main/java/org/apache/hyracks/storage/am/lsm/btree/column/impls/btree/ColumnBTreeBulkloader.java
new file mode 100644
index 0000000..48bd180
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree-column/src/main/java/org/apache/hyracks/storage/am/lsm/btree/column/impls/btree/ColumnBTreeBulkloader.java
@@ -0,0 +1,227 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.hyracks.storage.am.lsm.btree.column.impls.btree;
+
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+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.btree.impls.BTreeNSMBulkLoader;
+import org.apache.hyracks.storage.am.btree.impls.BTreeSplitKey;
+import org.apache.hyracks.storage.am.common.api.ISplitKey;
+import org.apache.hyracks.storage.am.common.api.ITreeIndex;
+import org.apache.hyracks.storage.am.common.api.ITreeIndexFrame;
+import org.apache.hyracks.storage.am.common.api.ITreeIndexTupleReference;
+import org.apache.hyracks.storage.am.common.impls.NodeFrontier;
+import org.apache.hyracks.storage.am.lsm.btree.column.api.AbstractColumnTupleWriter;
+import org.apache.hyracks.storage.am.lsm.btree.column.api.IColumnWriteMultiPageOp;
+import org.apache.hyracks.storage.common.buffercache.CachedPage;
+import org.apache.hyracks.storage.common.buffercache.IBufferCache;
+import org.apache.hyracks.storage.common.buffercache.ICachedPage;
+import org.apache.hyracks.storage.common.buffercache.IPageWriteCallback;
+import org.apache.hyracks.storage.common.file.BufferedFileHandle;
+
+public final class ColumnBTreeBulkloader extends BTreeNSMBulkLoader implements IColumnWriteMultiPageOp {
+    private final List<CachedPage> columnsPages;
+    private final List<CachedPage> tempConfiscatedPages;
+    private final ColumnBTreeWriteLeafFrame columnarFrame;
+    private final AbstractColumnTupleWriter columnWriter;
+    private final ISplitKey lowKey;
+    private boolean setLowKey;
+    private int tupleCount;
+
+    public ColumnBTreeBulkloader(float fillFactor, boolean verifyInput, IPageWriteCallback callback, ITreeIndex index,
+            ITreeIndexFrame leafFrame) throws HyracksDataException {
+        super(fillFactor, verifyInput, callback, index, leafFrame);
+        columnsPages = new ArrayList<>();
+        tempConfiscatedPages = new ArrayList<>();
+        columnarFrame = (ColumnBTreeWriteLeafFrame) leafFrame;
+        columnWriter = columnarFrame.getColumnTupleWriter();
+        columnWriter.init(this);
+        lowKey = new BTreeSplitKey(tupleWriter.createTupleReference());
+        lowKey.getTuple().setFieldCount(cmp.getKeyFieldCount());
+        setLowKey = true;
+    }
+
+    @Override
+    public void add(ITupleReference tuple) throws HyracksDataException {
+        if (isFull(tuple)) {
+            writeFullLeafPage();
+            confiscateNewLeafPage();
+        }
+        //Save the key of the last inserted tuple
+        setMinMaxKeys(tuple);
+        columnWriter.writeTuple(tuple);
+        tupleCount++;
+    }
+
+    @Override
+    protected ITreeIndexTupleReference createTupleReference() {
+        return tupleWriter.createTupleReference();
+    }
+
+    private boolean isFull(ITupleReference tuple) {
+        if (tupleCount == 0) {
+            return false;
+        } else if (tupleCount >= columnWriter.getMaxNumberOfTuples()) {
+            //We reached the maximum number of tuples
+            return true;
+        }
+        int requiredFreeSpace = AbstractColumnBTreeLeafFrame.HEADER_SIZE;
+        //Columns' Offsets
+        requiredFreeSpace += columnWriter.getColumnOffsetsSize();
+        //Occupied space from previous writes
+        requiredFreeSpace += columnWriter.getOccupiedSpace();
+        //min and max tuples' sizes
+        requiredFreeSpace += lowKey.getTuple().getTupleSize() + splitKey.getTuple().getTupleSize();
+        //New tuple required space
+        requiredFreeSpace += columnWriter.bytesRequired(tuple);
+        return bufferCache.getPageSize() <= requiredFreeSpace;
+    }
+
+    private void setMinMaxKeys(ITupleReference tuple) {
+        //Set max key
+        setSplitKey(splitKey, tuple);
+        if (setLowKey) {
+            setSplitKey(lowKey, tuple);
+            lowKey.getTuple().resetByTupleOffset(lowKey.getBuffer().array(), 0);
+            setLowKey = false;
+        }
+    }
+
+    @Override
+    public void end() throws HyracksDataException {
+        if (tupleCount > 0) {
+            splitKey.getTuple().resetByTupleOffset(splitKey.getBuffer().array(), 0);
+            columnarFrame.flush(columnWriter, tupleCount, this, lowKey.getTuple(), splitKey.getTuple());
+        }
+        columnWriter.close();
+        //We are done, return any temporary confiscated pages
+        for (ICachedPage page : tempConfiscatedPages) {
+            bufferCache.returnPage(page, false);
+        }
+        tempConfiscatedPages.clear();
+        //Where Page0 and columns pages will be written
+        super.end();
+    }
+
+    @Override
+    protected void writeFullLeafPage() throws HyracksDataException {
+        NodeFrontier leafFrontier = nodeFrontiers.get(0);
+        splitKey.getTuple().resetByTupleOffset(splitKey.getBuffer().array(), 0);
+        splitKey.setLeftPage(leafFrontier.pageId);
+        if (tupleCount > 0) {
+            //We need to flush columns to confiscate all columns pages first before calling propagateBulk
+            columnarFrame.flush(columnWriter, tupleCount, this, lowKey.getTuple(), splitKey.getTuple());
+        }
+
+        propagateBulk(1, pagesToWrite);
+
+        //Take a page for the next leaf
+        leafFrontier.pageId = freePageManager.takePage(metaFrame);
+        columnarFrame.setNextLeaf(leafFrontier.pageId);
+
+        /*
+         * Write columns' pages first to ensure they (columns' pages) are written before pageZero.
+         * It ensures pageZero does not land in between columns' pages if compression is enabled
+         */
+        writeColumnsPages();
+        //Then write page0
+        write(leafFrontier.page);
+
+        //Write interior nodes after writing columns pages
+        for (ICachedPage c : pagesToWrite) {
+            write(c);
+        }
+
+        pagesToWrite.clear();
+        splitKey.setRightPage(leafFrontier.pageId);
+        setLowKey = true;
+        tupleCount = 0;
+    }
+
+    @Override
+    protected void writeLastLeaf(ICachedPage page) throws HyracksDataException {
+        /*
+         * Write columns' pages first to ensure they (columns' pages) are written before pageZero.
+         * It ensures pageZero does not land in between columns' pages if compression is enabled
+         */
+        writeColumnsPages();
+        super.writeLastLeaf(page);
+    }
+
+    private void writeColumnsPages() throws HyracksDataException {
+        for (ICachedPage c : columnsPages) {
+            write(c);
+        }
+        columnsPages.clear();
+    }
+
+    @Override
+    public void abort() throws HyracksDataException {
+        for (ICachedPage page : columnsPages) {
+            bufferCache.returnPage(page, false);
+        }
+
+        for (ICachedPage page : tempConfiscatedPages) {
+            bufferCache.returnPage(page, false);
+        }
+        super.abort();
+    }
+
+    private void setSplitKey(ISplitKey splitKey, ITupleReference tuple) {
+        int splitKeySize = tupleWriter.bytesRequired(tuple, 0, cmp.getKeyFieldCount());
+        splitKey.initData(splitKeySize);
+        tupleWriter.writeTupleFields(tuple, 0, cmp.getKeyFieldCount(), splitKey.getBuffer().array(), 0);
+    }
+
+    /*
+     * ***********************************************************
+     * IColumnWriteMultiPageOp
+     * ***********************************************************
+     */
+
+    @Override
+    public ByteBuffer confiscatePersistent() throws HyracksDataException {
+        int pageId = freePageManager.takePage(metaFrame);
+        long dpid = BufferedFileHandle.getDiskPageId(fileId, pageId);
+        CachedPage page = (CachedPage) bufferCache.confiscatePage(dpid);
+        columnsPages.add(page);
+        return page.getBuffer();
+    }
+
+    @Override
+    public void persist() throws HyracksDataException {
+        writeColumnsPages();
+    }
+
+    @Override
+    public int getNumberOfPersistentBuffers() {
+        return columnsPages.size();
+    }
+
+    @Override
+    public ByteBuffer confiscateTemporary() throws HyracksDataException {
+        CachedPage page = (CachedPage) bufferCache.confiscatePage(IBufferCache.INVALID_DPID);
+        tempConfiscatedPages.add(page);
+        return page.getBuffer();
+    }
+}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree-column/src/main/java/org/apache/hyracks/storage/am/lsm/btree/column/impls/btree/ColumnBTreeFactory.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree-column/src/main/java/org/apache/hyracks/storage/am/lsm/btree/column/impls/btree/ColumnBTreeFactory.java
new file mode 100644
index 0000000..1b9e198
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree-column/src/main/java/org/apache/hyracks/storage/am/lsm/btree/column/impls/btree/ColumnBTreeFactory.java
@@ -0,0 +1,43 @@
+/*
+ * 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.column.impls.btree;
+
+import org.apache.hyracks.api.dataflow.value.IBinaryComparatorFactory;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.io.FileReference;
+import org.apache.hyracks.api.io.IIOManager;
+import org.apache.hyracks.storage.am.common.api.IPageManagerFactory;
+import org.apache.hyracks.storage.am.common.api.ITreeIndexFrameFactory;
+import org.apache.hyracks.storage.am.lsm.common.impls.TreeIndexFactory;
+import org.apache.hyracks.storage.common.buffercache.IBufferCache;
+
+public class ColumnBTreeFactory extends TreeIndexFactory<ColumnBTree> {
+    public ColumnBTreeFactory(IIOManager ioManager, IBufferCache bufferCache,
+            IPageManagerFactory freePageManagerFactory, ITreeIndexFrameFactory interiorFrameFactory,
+            ITreeIndexFrameFactory leafFrameFactory, IBinaryComparatorFactory[] cmpFactories, int fieldCount) {
+        super(ioManager, bufferCache, freePageManagerFactory, interiorFrameFactory, leafFrameFactory, cmpFactories,
+                fieldCount);
+    }
+
+    @Override
+    public ColumnBTree createIndexInstance(FileReference file) throws HyracksDataException {
+        return new ColumnBTree(bufferCache, freePageManagerFactory.createPageManager(bufferCache), interiorFrameFactory,
+                leafFrameFactory, cmpFactories, fieldCount, file);
+    }
+}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree-column/src/main/java/org/apache/hyracks/storage/am/lsm/btree/column/impls/btree/ColumnBTreePointSearchCursor.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree-column/src/main/java/org/apache/hyracks/storage/am/lsm/btree/column/impls/btree/ColumnBTreePointSearchCursor.java
new file mode 100644
index 0000000..c93e77e
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree-column/src/main/java/org/apache/hyracks/storage/am/lsm/btree/column/impls/btree/ColumnBTreePointSearchCursor.java
@@ -0,0 +1,55 @@
+/*
+ * 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.column.impls.btree;
+
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.storage.am.btree.api.IDiskBTreeStatefulPointSearchCursor;
+import org.apache.hyracks.storage.am.common.api.ITreeIndexFrame;
+import org.apache.hyracks.storage.common.IIndexCursorStats;
+import org.apache.hyracks.storage.common.ISearchPredicate;
+import org.apache.hyracks.storage.common.buffercache.IBufferCache;
+
+public class ColumnBTreePointSearchCursor extends ColumnBTreeRangeSearchCursor
+        implements IDiskBTreeStatefulPointSearchCursor {
+
+    public ColumnBTreePointSearchCursor(ColumnBTreeReadLeafFrame frame, IIndexCursorStats stats, int index) {
+        super(frame, stats, index);
+    }
+
+    @Override
+    public void doClose() throws HyracksDataException {
+        pageId = IBufferCache.INVALID_PAGEID;
+        super.doClose();
+    }
+
+    @Override
+    public int getLastPageId() {
+        return pageId;
+    }
+
+    @Override
+    public void setCursorToNextKey(ISearchPredicate searchPred) throws HyracksDataException {
+        initCursorPosition(searchPred);
+    }
+
+    @Override
+    public ITreeIndexFrame getFrame() {
+        return frame;
+    }
+}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree-column/src/main/java/org/apache/hyracks/storage/am/lsm/btree/column/impls/btree/ColumnBTreeRangeSearchCursor.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree-column/src/main/java/org/apache/hyracks/storage/am/lsm/btree/column/impls/btree/ColumnBTreeRangeSearchCursor.java
new file mode 100644
index 0000000..d0b7e2b
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree-column/src/main/java/org/apache/hyracks/storage/am/lsm/btree/column/impls/btree/ColumnBTreeRangeSearchCursor.java
@@ -0,0 +1,237 @@
+/*
+ * 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.column.impls.btree;
+
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.dataflow.common.data.accessors.ITupleReference;
+import org.apache.hyracks.storage.am.btree.impls.BTreeCursorInitialState;
+import org.apache.hyracks.storage.am.btree.impls.RangePredicate;
+import org.apache.hyracks.storage.am.common.api.ITreeIndexCursor;
+import org.apache.hyracks.storage.am.lsm.btree.column.api.IColumnReadMultiPageOp;
+import org.apache.hyracks.storage.am.lsm.btree.column.api.IColumnTupleIterator;
+import org.apache.hyracks.storage.common.EnforcedIndexCursor;
+import org.apache.hyracks.storage.common.ICursorInitialState;
+import org.apache.hyracks.storage.common.IIndexCursorStats;
+import org.apache.hyracks.storage.common.ISearchPredicate;
+import org.apache.hyracks.storage.common.MultiComparator;
+import org.apache.hyracks.storage.common.buffercache.IBufferCache;
+import org.apache.hyracks.storage.common.buffercache.ICachedPage;
+import org.apache.hyracks.storage.common.file.BufferedFileHandle;
+
+public class ColumnBTreeRangeSearchCursor extends EnforcedIndexCursor
+        implements ITreeIndexCursor, IColumnReadMultiPageOp {
+
+    protected final ColumnBTreeReadLeafFrame frame;
+    protected final IColumnTupleIterator frameTuple;
+
+    protected IBufferCache bufferCache = null;
+    protected int fileId;
+
+    protected int pageId;
+    protected ICachedPage page0 = null;
+
+    protected final RangePredicate reusablePredicate;
+    protected MultiComparator originalKeyCmp;
+
+    protected RangePredicate pred;
+    protected ITupleReference lowKey;
+    protected ITupleReference highKey;
+    protected boolean firstNextCall;
+
+    protected final IIndexCursorStats stats;
+
+    public ColumnBTreeRangeSearchCursor(ColumnBTreeReadLeafFrame frame, IIndexCursorStats stats, int index) {
+        this.frame = frame;
+        this.frameTuple = frame.createTupleReference(index, this);
+        this.reusablePredicate = new RangePredicate();
+        this.stats = stats;
+        fileId = -1;
+        pageId = IBufferCache.INVALID_PAGEID;
+    }
+
+    @Override
+    public void doDestroy() throws HyracksDataException {
+        // No Op all resources are released in the close call
+    }
+
+    @Override
+    public ITupleReference doGetTuple() {
+        return frameTuple;
+    }
+
+    private void fetchNextLeafPage(int leafPage) throws HyracksDataException {
+        int nextLeafPage = leafPage;
+        do {
+            ICachedPage nextLeaf = bufferCache.pin(BufferedFileHandle.getDiskPageId(fileId, nextLeafPage), false);
+            stats.getPageCounter().update(1);
+            bufferCache.unpin(page0);
+            page0 = nextLeaf;
+            frame.setPage(page0);
+            frameTuple.reset(0);
+            nextLeafPage = frame.getNextLeaf();
+        } while (frame.getTupleCount() == 0 && nextLeafPage > 0);
+    }
+
+    @Override
+    public boolean doHasNext() throws HyracksDataException {
+        int nextLeafPage;
+        if (frameTuple.isConsumed() && !firstNextCall) {
+            frameTuple.lastTupleReached();
+            nextLeafPage = frame.getNextLeaf();
+            if (nextLeafPage >= 0) {
+                fetchNextLeafPage(nextLeafPage);
+            } else {
+                return false;
+            }
+        }
+        return isNextIncluded();
+    }
+
+    @Override
+    public void doNext() throws HyracksDataException {
+        //NoOp
+    }
+
+    @Override
+    public void doOpen(ICursorInitialState initialState, ISearchPredicate searchPred) throws HyracksDataException {
+        // in case open is called multiple times without closing
+        if (page0 != null) {
+            releasePages();
+        }
+        originalKeyCmp = initialState.getOriginalKeyComparator();
+        page0 = initialState.getPage();
+        pageId = ((BTreeCursorInitialState) initialState).getPageId();
+        frame.setPage(page0);
+        frame.setMultiComparator(originalKeyCmp);
+        frameTuple.reset(0);
+        initCursorPosition(searchPred);
+    }
+
+    protected void initCursorPosition(ISearchPredicate searchPred) throws HyracksDataException {
+        pred = (RangePredicate) searchPred;
+        lowKey = pred.getLowKey();
+        highKey = pred.getHighKey();
+
+        reusablePredicate.setLowKeyComparator(originalKeyCmp);
+        reusablePredicate.setHighKeyComparator(pred.getHighKeyComparator());
+        reusablePredicate.setHighKey(pred.getHighKey(), pred.isHighKeyInclusive());
+        firstNextCall = true;
+        advanceTupleToLowKey();
+    }
+
+    protected boolean isNextIncluded() throws HyracksDataException {
+        if (firstNextCall) {
+            //The first call of frameTuple.next() was done during the opening of the cursor
+            firstNextCall = false;
+            return true;
+        } else if (frameTuple.isConsumed()) {
+            //All tuple were consumed
+            return false;
+        }
+        //Next tuple
+        frameTuple.next();
+        //Check whether the frameTuple is not consumed and also include the search key
+        return highKey == null || isLessOrEqual(frameTuple, highKey, pred.isHighKeyInclusive());
+    }
+
+    protected void advanceTupleToLowKey() throws HyracksDataException {
+        if (highKey != null && isLessOrEqual(highKey, frame.getLeftmostTuple(), !pred.isHighKeyInclusive())) {
+            /*
+             * Lowest key from the frame is greater than the requested highKey. No tuple will satisfy the search
+             * key. Consume the frameTuple to stop the search
+             */
+            firstNextCall = false;
+            frameTuple.consume();
+            return;
+        } else if (lowKey == null) {
+            //No range was specified.
+            frameTuple.next();
+            return;
+        }
+
+        //The lowKey is somewhere within the frame tuples
+        boolean stop = false;
+        int counter = 0;
+        while (!stop && !frameTuple.isConsumed()) {
+            frameTuple.next();
+            stop = isLessOrEqual(lowKey, frameTuple, pred.isLowKeyInclusive());
+            counter++;
+        }
+        //Advance all columns to the proper position
+        frameTuple.skip(counter - 1);
+    }
+
+    protected void releasePages() throws HyracksDataException {
+        //Unpin all column pages first
+        frameTuple.unpinColumnsPages();
+        if (page0 != null) {
+            bufferCache.unpin(page0);
+        }
+    }
+
+    private boolean isLessOrEqual(ITupleReference left, ITupleReference right, boolean inclusive)
+            throws HyracksDataException {
+        int cmp = originalKeyCmp.compare(left, right);
+        return cmp < 0 || cmp == 0 && inclusive;
+    }
+
+    @Override
+    public void doClose() throws HyracksDataException {
+        releasePages();
+        page0 = null;
+        pred = null;
+    }
+
+    @Override
+    public void setBufferCache(IBufferCache bufferCache) {
+        this.bufferCache = bufferCache;
+    }
+
+    @Override
+    public void setFileId(int fileId) {
+        this.fileId = fileId;
+    }
+
+    @Override
+    public boolean isExclusiveLatchNodes() {
+        return false;
+    }
+
+    /*
+     * ***********************************************************
+     * IColumnReadMultiPageOp
+     * ***********************************************************
+     */
+    @Override
+    public ICachedPage pin(int pageId) throws HyracksDataException {
+        stats.getPageCounter().update(1);
+        return bufferCache.pin(BufferedFileHandle.getDiskPageId(fileId, pageId), false);
+    }
+
+    @Override
+    public void unpin(ICachedPage page) throws HyracksDataException {
+        bufferCache.unpin(page);
+    }
+
+    @Override
+    public int getPageSize() {
+        return bufferCache.getPageSize();
+    }
+}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/IIndexAccessParameters.java b/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/IIndexAccessParameters.java
index a3745d0..8f553d9 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/IIndexAccessParameters.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/IIndexAccessParameters.java
@@ -40,4 +40,14 @@
      * Gets additional parameters.
      */
     Map<String, Object> getParameters();
+
+    /**
+     * Gets a parameter.
+     *
+     * @param key   of a parameter
+     * @param clazz used to explicitly cast the requested parameter to the required type
+     * @param <T>   the required type
+     * @return the requested parameter
+     */
+    <T> T getParameter(String key, Class<T> clazz);
 }