You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@asterixdb.apache.org by wy...@apache.org on 2022/09/10 20:57:31 UTC

[asterixdb] branch master updated: [ASTERIXDB-3026][STO][HYR] Part 1: Introduce LSMColumnBTree

This is an automated email from the ASF dual-hosted git repository.

wyk pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/asterixdb.git


The following commit(s) were added to refs/heads/master by this push:
     new 511015e0fc [ASTERIXDB-3026][STO][HYR] Part 1: Introduce LSMColumnBTree
511015e0fc is described below

commit 511015e0fc91efbef8d6bd41a8d8b99dfef4c6ad
Author: Wail Alkowaileet <wa...@gmail.com>
AuthorDate: Thu Sep 8 16:42:57 2022 -0700

    [ASTERIXDB-3026][STO][HYR] Part 1: Introduce LSMColumnBTree
    
    - user model changes: no
    - storage format changes: no
    - interface changes: no
    
    Details:
    This the first part to introduce the columnar format in Hyracks.
    The first part contains all the necessary APIs and the leaf frame
    implementations for both read and write.
    
    Change-Id: I45e42d0021a199d39e33cc108566f0d4daf54359
    Reviewed-on: https://asterix-gerrit.ics.uci.edu/c/asterixdb/+/15525
    Tested-by: Jenkins <je...@fulliautomatix.ics.uci.edu>
    Reviewed-by: Wail Alkowaileet <wa...@gmail.com>
    Reviewed-by: Murtadha Hubail <mh...@apache.org>
---
 .../hyracks-storage-am-lsm-btree-column/pom.xml    |  76 ++++++
 .../column/api/AbstractColumnTupleReader.java      |  50 ++++
 .../AbstractColumnTupleReaderWriterFactory.java    |  66 +++++
 .../column/api/AbstractColumnTupleWriter.java      |  95 ++++++++
 .../api/AbstractTupleWriterDisabledMethods.java    |  74 ++++++
 .../btree/column/api/IColumnBufferProvider.java    |  65 +++++
 .../am/lsm/btree/column/api/IColumnManager.java    |  59 +++++
 .../btree/column/api/IColumnManagerFactory.java    |  47 ++++
 .../am/lsm/btree/column/api/IColumnMetadata.java   |  40 +++
 .../btree/column/api/IColumnReadMultiPageOp.java   |  48 ++++
 .../lsm/btree/column/api/IColumnTupleIterator.java |  78 ++++++
 .../btree/column/api/IColumnWriteMultiPageOp.java  |  55 +++++
 .../api/projection/IColumnProjectionInfo.java      |  40 +++
 .../api/projection/IColumnTupleProjector.java      |  37 +++
 .../impls/btree/AbstractColumnBTreeLeafFrame.java  | 267 +++++++++++++++++++++
 .../impls/btree/ColumnBTreeLeafFrameFactory.java   |  65 +++++
 .../impls/btree/ColumnBTreeReadLeafFrame.java      |  99 ++++++++
 .../impls/btree/ColumnBTreeWriteLeafFrame.java     |  78 ++++++
 hyracks-fullstack/hyracks/pom.xml                  | 158 ++++++------
 19 files changed, 1419 insertions(+), 78 deletions(-)

diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree-column/pom.xml b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree-column/pom.xml
new file mode 100644
index 0000000000..2949ab92b8
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree-column/pom.xml
@@ -0,0 +1,76 @@
+<!--
+ ! Licensed to the Apache Software Foundation (ASF) under one
+ ! or more contributor license agreements.  See the NOTICE file
+ ! distributed with this work for additional information
+ ! regarding copyright ownership.  The ASF licenses this file
+ ! to you under the Apache License, Version 2.0 (the
+ ! "License"); you may not use this file except in compliance
+ ! with the License.  You may obtain a copy of the License at
+ !
+ !   http://www.apache.org/licenses/LICENSE-2.0
+ !
+ ! Unless required by applicable law or agreed to in writing,
+ ! software distributed under the License is distributed on an
+ ! "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ ! KIND, either express or implied.  See the License for the
+ ! specific language governing permissions and limitations
+ ! under the License.
+ !-->
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+         xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/maven-v4_0_0.xsd">
+    <modelVersion>4.0.0</modelVersion>
+    <artifactId>hyracks-storage-am-lsm-btree-column</artifactId>
+    <parent>
+        <groupId>org.apache.hyracks</groupId>
+        <artifactId>hyracks</artifactId>
+        <version>0.3.8-SNAPSHOT</version>
+    </parent>
+    <licenses>
+        <license>
+            <name>Apache License, Version 2.0</name>
+            <url>http://www.apache.org/licenses/LICENSE-2.0.txt</url>
+            <distribution>repo</distribution>
+            <comments>A business-friendly OSS license</comments>
+        </license>
+    </licenses>
+    <properties>
+        <root.dir>${basedir}/../..</root.dir>
+    </properties>
+    <dependencies>
+        <dependency>
+            <groupId>org.apache.hyracks</groupId>
+            <artifactId>hyracks-storage-am-btree</artifactId>
+            <version>${project.version}</version>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.hyracks</groupId>
+            <artifactId>hyracks-storage-am-lsm-common</artifactId>
+            <version>${project.version}</version>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.hyracks</groupId>
+            <artifactId>hyracks-dataflow-common</artifactId>
+            <version>${project.version}</version>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.hyracks</groupId>
+            <artifactId>hyracks-storage-am-common</artifactId>
+            <version>${project.version}</version>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.hyracks</groupId>
+            <artifactId>hyracks-api</artifactId>
+            <version>${project.version}</version>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.hyracks</groupId>
+            <artifactId>hyracks-storage-common</artifactId>
+            <version>${project.version}</version>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.hyracks</groupId>
+            <artifactId>hyracks-data-std</artifactId>
+            <version>${project.version}</version>
+        </dependency>
+    </dependencies>
+</project>
\ 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/api/AbstractColumnTupleReader.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree-column/src/main/java/org/apache/hyracks/storage/am/lsm/btree/column/api/AbstractColumnTupleReader.java
new file mode 100644
index 0000000000..7db792b72a
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree-column/src/main/java/org/apache/hyracks/storage/am/lsm/btree/column/api/AbstractColumnTupleReader.java
@@ -0,0 +1,50 @@
+/*
+ * 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.api;
+
+import java.nio.ByteBuffer;
+
+import org.apache.hyracks.dataflow.common.data.accessors.ITupleReference;
+import org.apache.hyracks.storage.am.lsm.btree.column.impls.btree.AbstractColumnBTreeLeafFrame;
+import org.apache.hyracks.storage.am.lsm.btree.column.impls.btree.ColumnBTreeReadLeafFrame;
+
+/**
+ * Provided for columnar read tuple reference
+ */
+public abstract class AbstractColumnTupleReader extends AbstractTupleWriterDisabledMethods {
+    public abstract IColumnTupleIterator createTupleIterator(ColumnBTreeReadLeafFrame frame, int componentIndex,
+            IColumnReadMultiPageOp multiPageOp);
+
+    /**
+     * Currently fixed to 4-byte per offset
+     *
+     * @param buf         buffer of Page0
+     * @param columnIndex column index
+     * @return column offset
+     * @see AbstractColumnTupleWriter#getColumnOffsetsSize()
+     */
+    public final int getColumnOffset(ByteBuffer buf, int columnIndex) {
+        return buf.getInt(AbstractColumnBTreeLeafFrame.HEADER_SIZE + columnIndex * Integer.BYTES);
+    }
+
+    @Override
+    public final int bytesRequired(ITupleReference tuple) {
+        throw new UnsupportedOperationException(UNSUPPORTED_OPERATION_MSG);
+    }
+}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree-column/src/main/java/org/apache/hyracks/storage/am/lsm/btree/column/api/AbstractColumnTupleReaderWriterFactory.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree-column/src/main/java/org/apache/hyracks/storage/am/lsm/btree/column/api/AbstractColumnTupleReaderWriterFactory.java
new file mode 100644
index 0000000000..774bbb94b1
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree-column/src/main/java/org/apache/hyracks/storage/am/lsm/btree/column/api/AbstractColumnTupleReaderWriterFactory.java
@@ -0,0 +1,66 @@
+/*
+ * 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.api;
+
+import org.apache.hyracks.storage.am.common.api.ITreeIndexTupleWriter;
+import org.apache.hyracks.storage.am.common.api.ITreeIndexTupleWriterFactory;
+import org.apache.hyracks.storage.am.lsm.btree.column.api.projection.IColumnProjectionInfo;
+import org.apache.hyracks.storage.common.buffercache.IBufferCache;
+
+/**
+ * For columns, there are two types for {@link ITreeIndexTupleWriter} one used during write and another during read
+ */
+public abstract class AbstractColumnTupleReaderWriterFactory implements ITreeIndexTupleWriterFactory {
+    private static final long serialVersionUID = -2377235465942457248L;
+    protected final int pageSize;
+    protected final int maxNumberOfTuples;
+    protected final float tolerance;
+
+    /**
+     * Tuple reader/writer factory
+     *
+     * @param pageSize          {@link IBufferCache} page size
+     * @param maxNumberOfTuples maximum number of tuples stored per a mega leaf page
+     * @param tolerance         percentage of tolerated empty space
+     */
+    protected AbstractColumnTupleReaderWriterFactory(int pageSize, int maxNumberOfTuples, float tolerance) {
+        this.pageSize = pageSize;
+        this.maxNumberOfTuples = maxNumberOfTuples;
+        this.tolerance = tolerance;
+    }
+
+    /**
+     * Create columnar tuple writer
+     *
+     * @param columnMetadata writer column metadata
+     */
+    public abstract AbstractColumnTupleWriter createColumnWriter(IColumnMetadata columnMetadata);
+
+    /**
+     * Create columnar tuple reader
+     *
+     * @param columnProjectionInfo column projection info for either query or merge
+     */
+    public abstract AbstractColumnTupleReader createColumnReader(IColumnProjectionInfo columnProjectionInfo);
+
+    @Override
+    public final ITreeIndexTupleWriter createTupleWriter() {
+        throw new UnsupportedOperationException("Operation is not supported for " + getClass().getName());
+    }
+}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree-column/src/main/java/org/apache/hyracks/storage/am/lsm/btree/column/api/AbstractColumnTupleWriter.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree-column/src/main/java/org/apache/hyracks/storage/am/lsm/btree/column/api/AbstractColumnTupleWriter.java
new file mode 100644
index 0000000000..0c19ce79a6
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree-column/src/main/java/org/apache/hyracks/storage/am/lsm/btree/column/api/AbstractColumnTupleWriter.java
@@ -0,0 +1,95 @@
+/*
+ * 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.api;
+
+import java.nio.ByteBuffer;
+
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.dataflow.common.data.accessors.ITupleReference;
+
+/**
+ * Columnar Tuple Writer:
+ * <p>
+ * The writer does not write directly to the page(s) buffer but write to internal temporary buffers (provided by
+ * {@link IColumnWriteMultiPageOp} until the header page (or page0) is full or the number of tuples equals to the
+ * {@link #getMaxNumberOfTuples()}
+ * Then, the columns are flushed to disk.
+ * <p>
+ * Contract:
+ * - Initially, the writer has to set multiPageOp by calling {@link #init(IColumnWriteMultiPageOp)}
+ * - For each write, the caller should check if adding a tuple does not exceed the {@link #getMaxNumberOfTuples()} or
+ * the on-disk page size (called stopping condition)
+ * - If the stopping condition is reached, then {@link #flush(ByteBuffer)} needed to be called
+ * <p>
+ * Hyracks visibility:
+ * - Columns are written as blobs (i.e., not interpretable by Hyracks)
+ * - Hyracks only aware of where each column at
+ */
+public abstract class AbstractColumnTupleWriter extends AbstractTupleWriterDisabledMethods {
+    /**
+     * Set the writer with {@link IColumnWriteMultiPageOp} to allocate columns for their writers
+     *
+     * @param multiPageOp multiPageOp
+     */
+    public abstract void init(IColumnWriteMultiPageOp multiPageOp) throws HyracksDataException;
+
+    /**
+     * @return The current number of columns
+     */
+    public abstract int getNumberOfColumns();
+
+    /**
+     * Currently, a column offset takes 4-byte (fixed). But in the future, we can reformat the offsets. For example,
+     * we can store index-offset pairs if we encounter a sparse columns (i.e., most columns are just nulls).
+     *
+     * @return the size needed to store columns' offsets
+     */
+    public final int getColumnOffsetsSize() {
+        return Integer.BYTES * getNumberOfColumns();
+    }
+
+    /**
+     * @return maximum number of tuples to be stored per page (i.e., page0)
+     */
+    public abstract int getMaxNumberOfTuples();
+
+    /**
+     * @return page0 occupied space
+     */
+    public abstract int getOccupiedSpace();
+
+    /**
+     * Writes the tuple into a temporary internal buffers
+     *
+     * @param tuple The tuple to be written
+     */
+    public abstract void writeTuple(ITupleReference tuple) throws HyracksDataException;
+
+    /**
+     * Flush all columns from the internal buffers to the page buffer
+     *
+     * @return the allocated space used to write tuples
+     */
+    public abstract int flush(ByteBuffer pageZero) throws HyracksDataException;
+
+    /**
+     * Close the current writer and release all allocated temporary buffers
+     */
+    public abstract void close();
+}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree-column/src/main/java/org/apache/hyracks/storage/am/lsm/btree/column/api/AbstractTupleWriterDisabledMethods.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree-column/src/main/java/org/apache/hyracks/storage/am/lsm/btree/column/api/AbstractTupleWriterDisabledMethods.java
new file mode 100644
index 0000000000..abc5ef04e2
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree-column/src/main/java/org/apache/hyracks/storage/am/lsm/btree/column/api/AbstractTupleWriterDisabledMethods.java
@@ -0,0 +1,74 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.hyracks.storage.am.lsm.btree.column.api;
+
+import java.nio.ByteBuffer;
+
+import org.apache.hyracks.dataflow.common.data.accessors.ITupleReference;
+import org.apache.hyracks.storage.am.common.api.ITreeIndexTupleReference;
+import org.apache.hyracks.storage.am.common.api.ITreeIndexTupleWriter;
+
+/**
+ * Disable all row write methods
+ */
+public abstract class AbstractTupleWriterDisabledMethods implements ITreeIndexTupleWriter {
+    protected static final String UNSUPPORTED_OPERATION_MSG = "Operation is not supported for columnar tuple reader";
+
+    /* ***********************************************
+     * Disable write-related operations
+     * ***********************************************
+     */
+
+    @Override
+    public final ITreeIndexTupleReference createTupleReference() {
+        throw new UnsupportedOperationException(UNSUPPORTED_OPERATION_MSG);
+    }
+
+    @Override
+    public final int writeTuple(ITupleReference tuple, ByteBuffer targetBuf, int targetOff) {
+        throw new UnsupportedOperationException(UNSUPPORTED_OPERATION_MSG);
+    }
+
+    @Override
+    public final int writeTuple(ITupleReference tuple, byte[] targetBuf, int targetOff) {
+        throw new UnsupportedOperationException(UNSUPPORTED_OPERATION_MSG);
+    }
+
+    @Override
+    public final int writeTupleFields(ITupleReference tuple, int startField, int numFields, byte[] targetBuf,
+            int targetOff) {
+        throw new UnsupportedOperationException(UNSUPPORTED_OPERATION_MSG);
+    }
+
+    @Override
+    public final int bytesRequired(ITupleReference tuple, int startField, int numFields) {
+        throw new UnsupportedOperationException(UNSUPPORTED_OPERATION_MSG);
+    }
+
+    @Override
+    public final int getCopySpaceRequired(ITupleReference tuple) {
+        throw new UnsupportedOperationException(UNSUPPORTED_OPERATION_MSG);
+    }
+
+    @Override
+    public final void setUpdated(boolean isUpdated) {
+        throw new UnsupportedOperationException(UNSUPPORTED_OPERATION_MSG);
+    }
+
+}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree-column/src/main/java/org/apache/hyracks/storage/am/lsm/btree/column/api/IColumnBufferProvider.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree-column/src/main/java/org/apache/hyracks/storage/am/lsm/btree/column/api/IColumnBufferProvider.java
new file mode 100644
index 0000000000..d0b5e128dd
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree-column/src/main/java/org/apache/hyracks/storage/am/lsm/btree/column/api/IColumnBufferProvider.java
@@ -0,0 +1,65 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.hyracks.storage.am.lsm.btree.column.api;
+
+import java.nio.ByteBuffer;
+import java.util.Queue;
+
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.storage.am.lsm.btree.column.impls.btree.ColumnBTreeReadLeafFrame;
+import org.apache.hyracks.storage.common.buffercache.IBufferCache;
+
+/**
+ * A proxy to call {@link IBufferCache} operations. Each column should have its own buffer provider
+ */
+public interface IColumnBufferProvider {
+    /**
+     * Calling this method would pin all the pages of the requested columns from the buffer cache
+     *
+     * @param frame the frame for Page0
+     */
+    void reset(ColumnBTreeReadLeafFrame frame) throws HyracksDataException;
+
+    /**
+     * Return all the pages for a column
+     *
+     * @param buffers queue for all pages of a column
+     */
+    void readAll(Queue<ByteBuffer> buffers) throws HyracksDataException;
+
+    /**
+     * Release all the column pages (i.e., unpin all column pages)
+     */
+    void releaseAll() throws HyracksDataException;
+
+    /**
+     * @return a buffer of a column (in case there is only a single page for a column)
+     */
+    ByteBuffer getBuffer();
+
+    /**
+     * @return the actual length (in bytes) for all the column's pages
+     */
+    int getLength();
+
+    /**
+     * @return the column index
+     */
+    int getColumnIndex();
+}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree-column/src/main/java/org/apache/hyracks/storage/am/lsm/btree/column/api/IColumnManager.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree-column/src/main/java/org/apache/hyracks/storage/am/lsm/btree/column/api/IColumnManager.java
new file mode 100644
index 0000000000..278ea0325e
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree-column/src/main/java/org/apache/hyracks/storage/am/lsm/btree/column/api/IColumnManager.java
@@ -0,0 +1,59 @@
+/*
+ * 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.api;
+
+import java.util.List;
+
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.data.std.api.IValueReference;
+import org.apache.hyracks.storage.am.lsm.btree.column.api.projection.IColumnTupleProjector;
+
+public interface IColumnManager {
+    /**
+     * Activate the columnar manager for an empty dataset
+     *
+     * @return empty column metadata
+     */
+    IColumnMetadata activate() throws HyracksDataException;
+
+    /**
+     * Activate the column manager for a non-empty dataset
+     *
+     * @param metadata column metadata value from the latest component metadata
+     * @return latest column metadata
+     */
+    IColumnMetadata activate(IValueReference metadata) throws HyracksDataException;
+
+    /**
+     * Create merge column metadata for a newly created merge component
+     *
+     * @param metadata         latest column metadata value stored in the metadata page
+     * @param componentsTuples tuples of the merging components
+     * @return column metadata for a new merged component
+     */
+    IColumnMetadata createMergeColumnMetadata(IValueReference metadata, List<IColumnTupleIterator> componentsTuples)
+            throws HyracksDataException;
+
+    /**
+     * Create tuple projector for reading the merging components. The merge tuple projector will return all columns
+     *
+     * @return merge tuple projector
+     */
+    IColumnTupleProjector getMergeColumnProjector();
+}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree-column/src/main/java/org/apache/hyracks/storage/am/lsm/btree/column/api/IColumnManagerFactory.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree-column/src/main/java/org/apache/hyracks/storage/am/lsm/btree/column/api/IColumnManagerFactory.java
new file mode 100644
index 0000000000..a2dfbcf0da
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree-column/src/main/java/org/apache/hyracks/storage/am/lsm/btree/column/api/IColumnManagerFactory.java
@@ -0,0 +1,47 @@
+/*
+ * 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.api;
+
+import java.io.Serializable;
+
+import org.apache.hyracks.api.io.IJsonSerializable;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperation.LSMIOOperationType;
+
+public interface IColumnManagerFactory extends Serializable, IJsonSerializable {
+    /**
+     * @return a new instance of {@link IColumnManager}
+     */
+    IColumnManager createColumnManager();
+
+    /**
+     * Get column tuple reader/writer for the {@link LSMIOOperationType#LOAD}
+     */
+    AbstractColumnTupleReaderWriterFactory getLoadColumnTupleReaderWriterFactory();
+
+    /**
+     * Get column tuple reader/writer for the {@link LSMIOOperationType#FLUSH}
+     */
+    AbstractColumnTupleReaderWriterFactory getFlushColumnTupleReaderWriterFactory();
+
+    /**
+     * Get column tuple reader/writer for the {@link LSMIOOperationType#MERGE}
+     */
+    AbstractColumnTupleReaderWriterFactory createMergeColumnTupleReaderWriterFactory();
+
+}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree-column/src/main/java/org/apache/hyracks/storage/am/lsm/btree/column/api/IColumnMetadata.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree-column/src/main/java/org/apache/hyracks/storage/am/lsm/btree/column/api/IColumnMetadata.java
new file mode 100644
index 0000000000..4c23b97600
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree-column/src/main/java/org/apache/hyracks/storage/am/lsm/btree/column/api/IColumnMetadata.java
@@ -0,0 +1,40 @@
+/*
+ * 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.api;
+
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.data.std.api.IValueReference;
+import org.apache.hyracks.util.annotations.NotThreadSafe;
+
+/**
+ * A holder for the columnar metadata.
+ * Modifications on the columnar metadata are not thread safe.
+ */
+@NotThreadSafe
+public interface IColumnMetadata {
+    /**
+     * @return a serialized version of the columns metadata
+     */
+    IValueReference serializeColumnsMetadata() throws HyracksDataException;
+
+    /**
+     * abort in case of an error. This should clean up any artifact
+     */
+    void abort() throws HyracksDataException;
+}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree-column/src/main/java/org/apache/hyracks/storage/am/lsm/btree/column/api/IColumnReadMultiPageOp.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree-column/src/main/java/org/apache/hyracks/storage/am/lsm/btree/column/api/IColumnReadMultiPageOp.java
new file mode 100644
index 0000000000..f43a6e948c
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree-column/src/main/java/org/apache/hyracks/storage/am/lsm/btree/column/api/IColumnReadMultiPageOp.java
@@ -0,0 +1,48 @@
+/*
+ * 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.api;
+
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.storage.common.buffercache.IBufferCache;
+import org.apache.hyracks.storage.common.buffercache.ICachedPage;
+
+/**
+ * A proxy to call {@link IBufferCache} read columns' pages
+ * Implementer should be aware to unpin all pages in case of an error
+ */
+public interface IColumnReadMultiPageOp {
+    /**
+     * Pin a column page
+     *
+     * @return a page that belongs to a column
+     */
+    ICachedPage pin(int pageId) throws HyracksDataException;
+
+    /**
+     * Unpin a pinned column page
+     */
+    void unpin(ICachedPage page) throws HyracksDataException;
+
+    /**
+     * Return {@link IBufferCache} page size
+     *
+     * @see IBufferCache#getPageSize()
+     */
+    int getPageSize();
+}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree-column/src/main/java/org/apache/hyracks/storage/am/lsm/btree/column/api/IColumnTupleIterator.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree-column/src/main/java/org/apache/hyracks/storage/am/lsm/btree/column/api/IColumnTupleIterator.java
new file mode 100644
index 0000000000..2ffa1bbeb7
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree-column/src/main/java/org/apache/hyracks/storage/am/lsm/btree/column/api/IColumnTupleIterator.java
@@ -0,0 +1,78 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.hyracks.storage.am.lsm.btree.column.api;
+
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.storage.am.common.api.ILSMIndexCursor;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponent;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMTreeTupleReference;
+import org.apache.hyracks.storage.common.buffercache.IBufferCache;
+import org.apache.hyracks.storage.common.buffercache.ICachedPage;
+
+/**
+ * A tuple representation that combines all the columns. It simply provides a way to iterate over tuples for a given
+ * set that could span multiple pages.
+ */
+public interface IColumnTupleIterator extends ILSMTreeTupleReference, Comparable<IColumnTupleIterator> {
+    /**
+     * Reset the iterator starting at the provided index
+     *
+     * @param startIndex start from the tuple at this index
+     */
+    void reset(int startIndex) throws HyracksDataException;
+
+    /**
+     * Mark {@link IColumnTupleIterator} as consumed
+     */
+    void consume();
+
+    /**
+     * @return true if the {@link IColumnTupleIterator} is consumed, false otherwise
+     */
+    boolean isConsumed();
+
+    /**
+     * Skip a number of tuples
+     *
+     * @param count the number of tuples that needed to be skipped
+     */
+    void skip(int count) throws HyracksDataException;
+
+    /**
+     * Move to the next tuple
+     */
+    void next() throws HyracksDataException;
+
+    /**
+     * Notifies that the last tuple has been consumed
+     */
+    void lastTupleReached() throws HyracksDataException;
+
+    /**
+     * The component index is the same as the index of a component in an {@link ILSMIndexCursor}
+     *
+     * @return From which {@link ILSMComponent} this iterator is for
+     */
+    int getComponentIndex();
+
+    /**
+     * Calls {@link IBufferCache#unpin(ICachedPage)} for all columns' pages
+     */
+    void unpinColumnsPages() throws HyracksDataException;
+}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree-column/src/main/java/org/apache/hyracks/storage/am/lsm/btree/column/api/IColumnWriteMultiPageOp.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree-column/src/main/java/org/apache/hyracks/storage/am/lsm/btree/column/api/IColumnWriteMultiPageOp.java
new file mode 100644
index 0000000000..2309fe1912
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree-column/src/main/java/org/apache/hyracks/storage/am/lsm/btree/column/api/IColumnWriteMultiPageOp.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.api;
+
+import java.nio.ByteBuffer;
+
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.storage.common.buffercache.IBufferCache;
+
+/**
+ * A proxy to call {@link IBufferCache} writing methods
+ * <p>
+ * An instance of this interface is responsible for returning all confiscated pages back to {@link IBufferCache} upon
+ * failures. Temporary buffers should be returned to the {@link IBufferCache} once the multi-page operation is finished.
+ * <p>
+ * Users of an instance of this interface should not expect the temporary buffers will last after the multi-page
+ * operation is finished.
+ */
+public interface IColumnWriteMultiPageOp {
+    /**
+     * @return a buffer that correspond to a page in a file
+     */
+    ByteBuffer confiscatePersistent() throws HyracksDataException;
+
+    /**
+     * Persist all confiscated persistent buffers to disk
+     */
+    void persist() throws HyracksDataException;
+
+    /**
+     * @return the number confiscated persistent pages
+     */
+    int getNumberOfPersistentBuffers();
+
+    /**
+     * @return a {@link IBufferCache}-backed buffer for temporary use
+     */
+    ByteBuffer confiscateTemporary() throws HyracksDataException;
+}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree-column/src/main/java/org/apache/hyracks/storage/am/lsm/btree/column/api/projection/IColumnProjectionInfo.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree-column/src/main/java/org/apache/hyracks/storage/am/lsm/btree/column/api/projection/IColumnProjectionInfo.java
new file mode 100644
index 0000000000..15064338ea
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree-column/src/main/java/org/apache/hyracks/storage/am/lsm/btree/column/api/projection/IColumnProjectionInfo.java
@@ -0,0 +1,40 @@
+/*
+ * 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.api.projection;
+
+/**
+ * Gets information about the requested columns
+ */
+public interface IColumnProjectionInfo {
+    /**
+     * @param ordinal position of the requested column
+     * @return column index given the ordinal number of the requested column
+     */
+    int getColumnIndex(int ordinal);
+
+    /**
+     * @return total number of requested columns
+     */
+    int getNumberOfProjectedColumns();
+
+    /**
+     * @return number of primary keys
+     */
+    int getNumberOfPrimaryKeys();
+}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree-column/src/main/java/org/apache/hyracks/storage/am/lsm/btree/column/api/projection/IColumnTupleProjector.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree-column/src/main/java/org/apache/hyracks/storage/am/lsm/btree/column/api/projection/IColumnTupleProjector.java
new file mode 100644
index 0000000000..c1301da4ba
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree-column/src/main/java/org/apache/hyracks/storage/am/lsm/btree/column/api/projection/IColumnTupleProjector.java
@@ -0,0 +1,37 @@
+/*
+ * 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.api.projection;
+
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.data.std.api.IValueReference;
+import org.apache.hyracks.storage.am.lsm.common.api.IComponentMetadata;
+import org.apache.hyracks.storage.common.projection.ITupleProjector;
+
+/**
+ * A specialized {@link ITupleProjector} for columnar datasets
+ */
+public interface IColumnTupleProjector extends ITupleProjector {
+    /**
+     * Create projection information
+     *
+     * @param columnMetadata the latest component's raw column metadata as stored in {@link IComponentMetadata}
+     * @return projection information
+     */
+    IColumnProjectionInfo createProjectionInfo(IValueReference columnMetadata) throws HyracksDataException;
+}
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/AbstractColumnBTreeLeafFrame.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree-column/src/main/java/org/apache/hyracks/storage/am/lsm/btree/column/impls/btree/AbstractColumnBTreeLeafFrame.java
new file mode 100644
index 0000000000..9aeafa431c
--- /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/AbstractColumnBTreeLeafFrame.java
@@ -0,0 +1,267 @@
+/*
+ * 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 org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.dataflow.common.data.accessors.ITupleReference;
+import org.apache.hyracks.storage.am.common.api.ISlotManager;
+import org.apache.hyracks.storage.am.common.api.ISplitKey;
+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.api.ITreeIndexTupleWriter;
+import org.apache.hyracks.storage.am.common.frames.FrameOpSpaceStatus;
+import org.apache.hyracks.storage.am.lsm.btree.column.api.AbstractColumnTupleWriter;
+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.buffercache.IExtraPageBlockHelper;
+
+/**
+ * Disable all unsupported/unused operations
+ */
+public abstract class AbstractColumnBTreeLeafFrame implements ITreeIndexFrame {
+    private static final String UNSUPPORTED_OPERATION_MSG = "Operation is not supported";
+
+    /*
+     * Remap the BTreeNSMFrame pointers for columnar pages
+     */
+    //Same as before
+    public static final int TUPLE_COUNT_OFFSET = Constants.TUPLE_COUNT_OFFSET;
+    //Previously Renaming
+    public static final int NUMBER_OF_COLUMNS_OFFSET = Constants.FREE_SPACE_OFFSET;
+    //Previously first four byte of LSN.
+    public static final int LEFT_MOST_KEY_OFFSET = Constants.RESERVED_HEADER_SIZE;
+    //Previously last four byte of LSN.
+    public static final int RIGHT_MOST_KEY_OFFSET = LEFT_MOST_KEY_OFFSET + 4;
+    /**
+     * Currently, a column offset takes 4-byte (fixed). But in the future, we can reformat the offsets. For example,
+     * we can store index-offset pairs if we encounter a sparse columns (i.e., most columns are just nulls). This
+     * reformatting could be indicated by the FLAG byte.
+     *
+     * @see AbstractColumnTupleWriter#getColumnOffsetsSize()
+     */
+    public static final int SIZE_OF_COLUMNS_OFFSETS_OFFSET = RIGHT_MOST_KEY_OFFSET + 4;
+    //Total number of columns pages
+    public static final int NUMBER_OF_COLUMN_PAGES = SIZE_OF_COLUMNS_OFFSETS_OFFSET + 4;
+    //A flag (used in NSM to indicate small and large pages). We can reuse it as explained above
+    public static final int FLAG_OFFSET = NUMBER_OF_COLUMN_PAGES + 4;
+    public static final int NEXT_LEAF_OFFSET = FLAG_OFFSET + 1;
+    public static final int HEADER_SIZE = NEXT_LEAF_OFFSET + 4;
+
+    protected final ITreeIndexTupleWriter rowTupleWriter;
+
+    protected MultiComparator cmp;
+    protected ICachedPage page;
+    protected ByteBuffer buf;
+
+    AbstractColumnBTreeLeafFrame(ITreeIndexTupleWriter rowTupleWriter) {
+        this.rowTupleWriter = rowTupleWriter;
+    }
+
+    /* ****************************************************************************
+     * Needed by both read and write
+     * ****************************************************************************
+     */
+
+    @Override
+    public final ITreeIndexTupleWriter getTupleWriter() {
+        return rowTupleWriter;
+    }
+
+    @Override
+    public final void setMultiComparator(MultiComparator cmp) {
+        this.cmp = cmp;
+    }
+
+    @Override
+    public final void setPage(ICachedPage page) {
+        this.page = page;
+        this.buf = page.getBuffer();
+        buf.clear();
+        buf.position(HEADER_SIZE);
+    }
+
+    @Override
+    public final ICachedPage getPage() {
+        return page;
+    }
+
+    @Override
+    public final ByteBuffer getBuffer() {
+        return buf;
+    }
+
+    @Override
+    public final boolean isLeaf() {
+        return true;
+    }
+
+    @Override
+    public final boolean isInterior() {
+        return false;
+    }
+
+    @Override
+    public final int getPageHeaderSize() {
+        return HEADER_SIZE;
+    }
+
+    /* ****************************************************************************
+     * Operations that are needed by either read or write
+     * ****************************************************************************
+     */
+
+    @Override
+    public void initBuffer(byte level) {
+        throw new UnsupportedOperationException(UNSUPPORTED_OPERATION_MSG);
+    }
+
+    @Override
+    public int getTupleCount() {
+        throw new UnsupportedOperationException(UNSUPPORTED_OPERATION_MSG);
+    }
+
+    @Override
+    public ITreeIndexTupleReference createTupleReference() {
+        throw new UnsupportedOperationException(UNSUPPORTED_OPERATION_MSG);
+    }
+
+    @Override
+    public ITupleReference getLeftmostTuple() throws HyracksDataException {
+        throw new UnsupportedOperationException(UNSUPPORTED_OPERATION_MSG);
+    }
+
+    @Override
+    public ITupleReference getRightmostTuple() throws HyracksDataException {
+        throw new UnsupportedOperationException(UNSUPPORTED_OPERATION_MSG);
+    }
+
+    /* ****************************************************************************
+     * Unsupported Operations
+     * ****************************************************************************
+     */
+
+    @Override
+    public final String printHeader() {
+        throw new UnsupportedOperationException(UNSUPPORTED_OPERATION_MSG);
+    }
+
+    @Override
+    public final byte getLevel() {
+        throw new UnsupportedOperationException(UNSUPPORTED_OPERATION_MSG);
+    }
+
+    @Override
+    public final void setLevel(byte level) {
+        throw new UnsupportedOperationException(UNSUPPORTED_OPERATION_MSG);
+    }
+
+    @Override
+    public final int getBytesRequiredToWriteTuple(ITupleReference tuple) {
+        throw new UnsupportedOperationException(UNSUPPORTED_OPERATION_MSG);
+    }
+
+    @Override
+    public final FrameOpSpaceStatus hasSpaceInsert(ITupleReference tuple) throws HyracksDataException {
+        throw new UnsupportedOperationException(UNSUPPORTED_OPERATION_MSG);
+    }
+
+    @Override
+    public final void insert(ITupleReference tuple, int tupleIndex) {
+        throw new UnsupportedOperationException(UNSUPPORTED_OPERATION_MSG);
+    }
+
+    @Override
+    public final FrameOpSpaceStatus hasSpaceUpdate(ITupleReference newTuple, int oldTupleIndex) {
+        throw new UnsupportedOperationException(UNSUPPORTED_OPERATION_MSG);
+    }
+
+    @Override
+    public final void update(ITupleReference newTuple, int oldTupleIndex, boolean inPlace) {
+        throw new UnsupportedOperationException(UNSUPPORTED_OPERATION_MSG);
+    }
+
+    @Override
+    public final void delete(ITupleReference tuple, int tupleIndex) {
+        throw new UnsupportedOperationException(UNSUPPORTED_OPERATION_MSG);
+    }
+
+    @Override
+    public final boolean compact() {
+        throw new UnsupportedOperationException(UNSUPPORTED_OPERATION_MSG);
+    }
+
+    @Override
+    public final boolean compress() throws HyracksDataException {
+        throw new UnsupportedOperationException(UNSUPPORTED_OPERATION_MSG);
+    }
+
+    @Override
+    public final int getTupleOffset(int slotNum) {
+        throw new UnsupportedOperationException(UNSUPPORTED_OPERATION_MSG);
+    }
+
+    @Override
+    public final int getTotalFreeSpace() {
+        throw new UnsupportedOperationException(UNSUPPORTED_OPERATION_MSG);
+    }
+
+    @Override
+    public final void setPageLsn(long pageLsn) {
+        throw new UnsupportedOperationException(UNSUPPORTED_OPERATION_MSG);
+    }
+
+    @Override
+    public final long getPageLsn() {
+        throw new UnsupportedOperationException(UNSUPPORTED_OPERATION_MSG);
+    }
+
+    @Override
+    public final int getMaxTupleSize(int pageSize) {
+        throw new UnsupportedOperationException(UNSUPPORTED_OPERATION_MSG);
+    }
+
+    @Override
+    public final void split(ITreeIndexFrame rightFrame, ITupleReference tuple, ISplitKey splitKey,
+            IExtraPageBlockHelper extraPageBlockHelper, IBufferCache bufferCache) throws HyracksDataException {
+        throw new UnsupportedOperationException(UNSUPPORTED_OPERATION_MSG);
+    }
+
+    @Override
+    public final ISlotManager getSlotManager() {
+        throw new UnsupportedOperationException(UNSUPPORTED_OPERATION_MSG);
+    }
+
+    @Override
+    public final int getSlotSize() {
+        return 0;
+    }
+
+    @Override
+    public final int getFreeSpaceOff() {
+        throw new UnsupportedOperationException(UNSUPPORTED_OPERATION_MSG);
+    }
+
+    @Override
+    public final void setFreeSpaceOff(int freeSpace) {
+        throw new UnsupportedOperationException(UNSUPPORTED_OPERATION_MSG);
+    }
+}
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/ColumnBTreeLeafFrameFactory.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree-column/src/main/java/org/apache/hyracks/storage/am/lsm/btree/column/impls/btree/ColumnBTreeLeafFrameFactory.java
new file mode 100644
index 0000000000..31d85bd4f3
--- /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/ColumnBTreeLeafFrameFactory.java
@@ -0,0 +1,65 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.hyracks.storage.am.lsm.btree.column.impls.btree;
+
+import org.apache.hyracks.storage.am.btree.frames.BTreeNSMLeafFrame;
+import org.apache.hyracks.storage.am.common.api.ITreeIndexFrame;
+import org.apache.hyracks.storage.am.common.api.ITreeIndexFrameFactory;
+import org.apache.hyracks.storage.am.common.api.ITreeIndexTupleWriter;
+import org.apache.hyracks.storage.am.common.api.ITreeIndexTupleWriterFactory;
+import org.apache.hyracks.storage.am.lsm.btree.column.api.AbstractColumnTupleReader;
+import org.apache.hyracks.storage.am.lsm.btree.column.api.AbstractColumnTupleReaderWriterFactory;
+import org.apache.hyracks.storage.am.lsm.btree.column.api.AbstractColumnTupleWriter;
+import org.apache.hyracks.storage.am.lsm.btree.column.api.IColumnMetadata;
+import org.apache.hyracks.storage.am.lsm.btree.column.api.projection.IColumnProjectionInfo;
+
+public class ColumnBTreeLeafFrameFactory implements ITreeIndexFrameFactory {
+    private static final long serialVersionUID = 4136035898137820322L;
+    private final ITreeIndexTupleWriterFactory rowTupleWriterFactory;
+    private final AbstractColumnTupleReaderWriterFactory columnTupleWriterFactory;
+
+    public ColumnBTreeLeafFrameFactory(ITreeIndexTupleWriterFactory rowTupleWriterFactory,
+            AbstractColumnTupleReaderWriterFactory columnTupleWriterFactory) {
+        this.rowTupleWriterFactory = rowTupleWriterFactory;
+        this.columnTupleWriterFactory = columnTupleWriterFactory;
+    }
+
+    @Override
+    public ITreeIndexFrame createFrame() {
+        //Create a dummy leaf frame
+        return new BTreeNSMLeafFrame(rowTupleWriterFactory.createTupleWriter());
+    }
+
+    @Override
+    public ITreeIndexTupleWriterFactory getTupleWriterFactory() {
+        return rowTupleWriterFactory;
+    }
+
+    public ColumnBTreeWriteLeafFrame createWriterFrame(IColumnMetadata columnMetadata) {
+        ITreeIndexTupleWriter rowTupleWriter = rowTupleWriterFactory.createTupleWriter();
+        AbstractColumnTupleWriter columnTupleWriter = columnTupleWriterFactory.createColumnWriter(columnMetadata);
+        return new ColumnBTreeWriteLeafFrame(rowTupleWriter, columnTupleWriter);
+    }
+
+    public ColumnBTreeReadLeafFrame createReadFrame(IColumnProjectionInfo columnProjectionInfo) {
+        ITreeIndexTupleWriter rowTupleWriter = rowTupleWriterFactory.createTupleWriter();
+        AbstractColumnTupleReader columnTupleReader = columnTupleWriterFactory.createColumnReader(columnProjectionInfo);
+        return new ColumnBTreeReadLeafFrame(rowTupleWriter, columnTupleReader);
+    }
+}
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/ColumnBTreeReadLeafFrame.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree-column/src/main/java/org/apache/hyracks/storage/am/lsm/btree/column/impls/btree/ColumnBTreeReadLeafFrame.java
new file mode 100644
index 0000000000..88726131d4
--- /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/ColumnBTreeReadLeafFrame.java
@@ -0,0 +1,99 @@
+/*
+ * 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.dataflow.common.data.accessors.ITupleReference;
+import org.apache.hyracks.storage.am.common.api.ITreeIndexTupleReference;
+import org.apache.hyracks.storage.am.common.api.ITreeIndexTupleWriter;
+import org.apache.hyracks.storage.am.lsm.btree.column.api.AbstractColumnTupleReader;
+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.buffercache.CachedPage;
+import org.apache.hyracks.storage.common.file.BufferedFileHandle;
+
+public final class ColumnBTreeReadLeafFrame extends AbstractColumnBTreeLeafFrame {
+    private final AbstractColumnTupleReader columnarTupleReader;
+    private final ITreeIndexTupleReference leftMostTuple;
+    private final ITreeIndexTupleReference rightMostTuple;
+
+    public ColumnBTreeReadLeafFrame(ITreeIndexTupleWriter rowTupleWriter,
+            AbstractColumnTupleReader columnarTupleReader) {
+        super(rowTupleWriter);
+        this.columnarTupleReader = columnarTupleReader;
+        leftMostTuple = rowTupleWriter.createTupleReference();
+        rightMostTuple = rowTupleWriter.createTupleReference();
+    }
+
+    @Override
+    public ITupleReference getLeftmostTuple() {
+        if (getTupleCount() == 0) {
+            return null;
+        }
+        leftMostTuple.setFieldCount(cmp.getKeyFieldCount());
+        leftMostTuple.resetByTupleOffset(buf.array(), buf.getInt(LEFT_MOST_KEY_OFFSET));
+        return leftMostTuple;
+    }
+
+    @Override
+    public ITupleReference getRightmostTuple() {
+        if (getTupleCount() == 0) {
+            return null;
+        }
+        rightMostTuple.setFieldCount(cmp.getKeyFieldCount());
+        rightMostTuple.resetByTupleOffset(buf.array(), buf.getInt(RIGHT_MOST_KEY_OFFSET));
+        return rightMostTuple;
+    }
+
+    public IColumnTupleIterator createTupleReference(int index, IColumnReadMultiPageOp multiPageOp) {
+        return columnarTupleReader.createTupleIterator(this, index, multiPageOp);
+    }
+
+    @Override
+    public int getTupleCount() {
+        return buf.getInt(Constants.TUPLE_COUNT_OFFSET);
+    }
+
+    public int getPageId() {
+        return BufferedFileHandle.getPageId(((CachedPage) page).getDiskPageId());
+    }
+
+    public int getNumberOfColumns() {
+        return buf.getInt(NUMBER_OF_COLUMNS_OFFSET);
+    }
+
+    public int getColumnOffset(int columnIndex) {
+        if (columnIndex >= getNumberOfColumns()) {
+            throw new IndexOutOfBoundsException(columnIndex + " >= " + getNumberOfColumns());
+        }
+        return columnarTupleReader.getColumnOffset(buf, columnIndex);
+    }
+
+    AbstractColumnTupleReader getColumnarTupleReader() {
+        return columnarTupleReader;
+    }
+
+    int getNextLeaf() {
+        return buf.getInt(NEXT_LEAF_OFFSET);
+    }
+
+    @Override
+    public ITreeIndexTupleReference createTupleReference() {
+        throw new IllegalArgumentException("Use createTupleReference(int)");
+    }
+}
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/ColumnBTreeWriteLeafFrame.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree-column/src/main/java/org/apache/hyracks/storage/am/lsm/btree/column/impls/btree/ColumnBTreeWriteLeafFrame.java
new file mode 100644
index 0000000000..275fb0e5be
--- /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/ColumnBTreeWriteLeafFrame.java
@@ -0,0 +1,78 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.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.common.api.ITreeIndexTupleWriter;
+import org.apache.hyracks.storage.am.lsm.btree.column.api.AbstractColumnTupleWriter;
+import org.apache.hyracks.storage.am.lsm.btree.column.api.IColumnWriteMultiPageOp;
+
+public class ColumnBTreeWriteLeafFrame extends AbstractColumnBTreeLeafFrame {
+    private final AbstractColumnTupleWriter columnTupleWriter;
+
+    public ColumnBTreeWriteLeafFrame(ITreeIndexTupleWriter rowTupleWriter,
+            AbstractColumnTupleWriter columnTupleWriter) {
+        super(rowTupleWriter);
+        this.columnTupleWriter = columnTupleWriter;
+    }
+
+    @Override
+    public void initBuffer(byte level) {
+        buf.putInt(TUPLE_COUNT_OFFSET, 0);
+        buf.put(Constants.LEVEL_OFFSET, level);
+        buf.putInt(NUMBER_OF_COLUMNS_OFFSET, 0);
+        buf.putInt(LEFT_MOST_KEY_OFFSET, -1);
+        buf.putInt(RIGHT_MOST_KEY_OFFSET, -1);
+        buf.putInt(SIZE_OF_COLUMNS_OFFSETS_OFFSET, 0);
+        buf.putInt(NUMBER_OF_COLUMN_PAGES, 0);
+        buf.put(FLAG_OFFSET, (byte) 0);
+        buf.putInt(NEXT_LEAF_OFFSET, -1);
+    }
+
+    void flush(AbstractColumnTupleWriter columnWriter, int numberOfTuples, IColumnWriteMultiPageOp multiPageOp,
+            ITupleReference minKey, ITupleReference maxKey) throws HyracksDataException {
+        //Prepare the space for writing the columns' information such as the primary keys
+        buf.position(HEADER_SIZE);
+        //Write the columns' information including the columns' offsets and the primary keys
+        columnWriter.flush(buf);
+
+        //Write min and max keys
+        int offset = buf.position();
+        buf.putInt(LEFT_MOST_KEY_OFFSET, offset);
+        offset += rowTupleWriter.writeTuple(minKey, buf.array(), offset);
+        buf.putInt(RIGHT_MOST_KEY_OFFSET, offset);
+        rowTupleWriter.writeTuple(maxKey, buf.array(), offset);
+
+        //Write page information
+        int numberOfColumns = columnWriter.getNumberOfColumns();
+        buf.putInt(TUPLE_COUNT_OFFSET, numberOfTuples);
+        buf.putInt(NUMBER_OF_COLUMNS_OFFSET, numberOfColumns);
+        buf.putInt(SIZE_OF_COLUMNS_OFFSETS_OFFSET, columnWriter.getColumnOffsetsSize());
+        buf.putInt(NUMBER_OF_COLUMN_PAGES, multiPageOp.getNumberOfPersistentBuffers());
+    }
+
+    public AbstractColumnTupleWriter getColumnTupleWriter() {
+        return columnTupleWriter;
+    }
+
+    void setNextLeaf(int pageId) {
+        buf.putInt(NEXT_LEAF_OFFSET, pageId);
+    }
+}
diff --git a/hyracks-fullstack/hyracks/pom.xml b/hyracks-fullstack/hyracks/pom.xml
index 443e8c07ad..538957bc32 100644
--- a/hyracks-fullstack/hyracks/pom.xml
+++ b/hyracks-fullstack/hyracks/pom.xml
@@ -17,88 +17,90 @@
  ! under the License.
  !-->
 
-<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/maven-v4_0_0.xsd">
-  <modelVersion>4.0.0</modelVersion>
-  <artifactId>hyracks</artifactId>
-  <packaging>pom</packaging>
-  <name>hyracks</name>
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+         xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/maven-v4_0_0.xsd">
+    <modelVersion>4.0.0</modelVersion>
+    <artifactId>hyracks</artifactId>
+    <packaging>pom</packaging>
+    <name>hyracks</name>
 
-  <parent>
-    <groupId>org.apache.hyracks</groupId>
-    <artifactId>apache-hyracks</artifactId>
-    <version>0.3.8-SNAPSHOT</version>
-  </parent>
+    <parent>
+        <groupId>org.apache.hyracks</groupId>
+        <artifactId>apache-hyracks</artifactId>
+        <version>0.3.8-SNAPSHOT</version>
+    </parent>
 
-  <build>
-    <pluginManagement>
-      <plugins>
-        <plugin>
-          <groupId>org.apache.maven.plugins</groupId>
-          <artifactId>maven-plugin-plugin</artifactId>
-          <version>3.6.0</version>
-        </plugin>
-        <!--This plugin's configuration is used to store Eclipse m2e settings only. It has no influence on the Maven build itself.-->
-        <plugin>
-          <groupId>org.eclipse.m2e</groupId>
-          <artifactId>lifecycle-mapping</artifactId>
-          <version>1.0.0</version>
-          <configuration>
-            <lifecycleMappingMetadata>
-              <pluginExecutions>
-                <pluginExecution>
-                  <pluginExecutionFilter>
+    <build>
+        <pluginManagement>
+            <plugins>
+                <plugin>
                     <groupId>org.apache.maven.plugins</groupId>
                     <artifactId>maven-plugin-plugin</artifactId>
-                    <versionRange>[0.0,)</versionRange>
-                    <goals>
-                      <goal>descriptor</goal>
-                    </goals>
-                  </pluginExecutionFilter>
-                  <action>
-                    <ignore />
-                  </action>
-                </pluginExecution>
-              </pluginExecutions>
-            </lifecycleMappingMetadata>
-          </configuration>
-        </plugin>
-      </plugins>
-    </pluginManagement>
-  </build>
+                    <version>3.6.0</version>
+                </plugin>
+                <!--This plugin's configuration is used to store Eclipse m2e settings only. It has no influence on the Maven build itself.-->
+                <plugin>
+                    <groupId>org.eclipse.m2e</groupId>
+                    <artifactId>lifecycle-mapping</artifactId>
+                    <version>1.0.0</version>
+                    <configuration>
+                        <lifecycleMappingMetadata>
+                            <pluginExecutions>
+                                <pluginExecution>
+                                    <pluginExecutionFilter>
+                                        <groupId>org.apache.maven.plugins</groupId>
+                                        <artifactId>maven-plugin-plugin</artifactId>
+                                        <versionRange>[0.0,)</versionRange>
+                                        <goals>
+                                            <goal>descriptor</goal>
+                                        </goals>
+                                    </pluginExecutionFilter>
+                                    <action>
+                                        <ignore/>
+                                    </action>
+                                </pluginExecution>
+                            </pluginExecutions>
+                        </lifecycleMappingMetadata>
+                    </configuration>
+                </plugin>
+            </plugins>
+        </pluginManagement>
+    </build>
 
-  <properties>
-    <root.dir>${basedir}/..</root.dir>
-  </properties>
+    <properties>
+        <root.dir>${basedir}/..</root.dir>
+    </properties>
 
-  <modules>
-    <module>hyracks-util</module>
-    <module>hyracks-api</module>
-    <module>hyracks-ipc</module>
-    <module>hyracks-comm</module>
-    <module>hyracks-client</module>
-    <module>hyracks-dataflow-common</module>
-    <module>hyracks-dataflow-std</module>
-    <module>hyracks-control</module>
-    <module>hyracks-net</module>
-    <module>hyracks-data</module>
-    <module>hyracks-storage-common</module>
-    <module>hyracks-storage-am-common</module>
-    <module>hyracks-storage-am-bloomfilter</module>
-    <module>hyracks-storage-am-btree</module>
-    <module>hyracks-storage-am-lsm-invertedindex</module>
-    <module>hyracks-storage-am-lsm-common</module>
-    <module>hyracks-storage-am-lsm-btree</module>
-    <module>hyracks-storage-am-lsm-rtree</module>
-    <module>hyracks-storage-am-rtree</module>
-    <module>hyracks-test-support</module>
-    <module>hyracks-tests</module>
-    <module>hyracks-server</module>
-    <module>hyracks-examples</module>
-    <module>hyracks-documentation</module>
-    <!--module>hyracks-hadoop-compat</module-->
-    <module>hyracks-maven-plugins</module>
-    <module>hyracks-hdfs</module>
-    <module>hyracks-dist</module>
-    <module>hyracks-http</module>
-  </modules>
+    <modules>
+        <module>hyracks-util</module>
+        <module>hyracks-api</module>
+        <module>hyracks-ipc</module>
+        <module>hyracks-comm</module>
+        <module>hyracks-client</module>
+        <module>hyracks-dataflow-common</module>
+        <module>hyracks-dataflow-std</module>
+        <module>hyracks-control</module>
+        <module>hyracks-net</module>
+        <module>hyracks-data</module>
+        <module>hyracks-storage-common</module>
+        <module>hyracks-storage-am-common</module>
+        <module>hyracks-storage-am-bloomfilter</module>
+        <module>hyracks-storage-am-btree</module>
+        <module>hyracks-storage-am-lsm-invertedindex</module>
+        <module>hyracks-storage-am-lsm-common</module>
+        <module>hyracks-storage-am-lsm-btree</module>
+        <module>hyracks-storage-am-lsm-btree-column</module>
+        <module>hyracks-storage-am-lsm-rtree</module>
+        <module>hyracks-storage-am-rtree</module>
+        <module>hyracks-test-support</module>
+        <module>hyracks-tests</module>
+        <module>hyracks-server</module>
+        <module>hyracks-examples</module>
+        <module>hyracks-documentation</module>
+        <!--module>hyracks-hadoop-compat</module-->
+        <module>hyracks-maven-plugins</module>
+        <module>hyracks-hdfs</module>
+        <module>hyracks-dist</module>
+        <module>hyracks-http</module>
+    </modules>
 </project>