You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@carbondata.apache.org by ch...@apache.org on 2016/08/15 07:09:30 UTC

[45/52] [partial] incubator-carbondata git commit: Renamed packages to org.apache.carbondata and fixed errors

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/cd6a4ff3/core/src/main/java/org/apache/carbondata/core/carbon/datastore/impl/btree/BTreeNonLeafNode.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/carbon/datastore/impl/btree/BTreeNonLeafNode.java b/core/src/main/java/org/apache/carbondata/core/carbon/datastore/impl/btree/BTreeNonLeafNode.java
new file mode 100644
index 0000000..cfbe06d
--- /dev/null
+++ b/core/src/main/java/org/apache/carbondata/core/carbon/datastore/impl/btree/BTreeNonLeafNode.java
@@ -0,0 +1,232 @@
+/*
+ * 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.carbondata.core.carbon.datastore.impl.btree;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.carbondata.core.carbon.datastore.DataRefNode;
+import org.apache.carbondata.core.carbon.datastore.IndexKey;
+import org.apache.carbondata.core.carbon.datastore.chunk.DimensionColumnDataChunk;
+import org.apache.carbondata.core.carbon.datastore.chunk.MeasureColumnDataChunk;
+import org.apache.carbondata.core.constants.CarbonCommonConstants;
+import org.apache.carbondata.core.datastorage.store.FileHolder;
+
+/**
+ * No leaf node of a b+tree class which will keep the matadata(start key) of the
+ * leaf node
+ */
+public class BTreeNonLeafNode implements BTreeNode {
+
+  /**
+   * Child nodes
+   */
+  private BTreeNode[] children;
+
+  /**
+   * list of keys in non leaf
+   */
+  private List<IndexKey> listOfKeys;
+
+  public BTreeNonLeafNode() {
+    // creating a list which will store all the indexes
+    listOfKeys = new ArrayList<IndexKey>(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE);
+  }
+
+  /**
+   * below method will return the one node indexes
+   *
+   * @return getting a complete leaf ]node keys
+   */
+  @Override public IndexKey[] getNodeKeys() {
+    return listOfKeys.toArray(new IndexKey[listOfKeys.size()]);
+  }
+
+  /**
+   * as it is a non leaf node it will have the reference of all the leaf node
+   * under it, setting all the children
+   *
+   * @param leaf nodes
+   */
+  @Override public void setChildren(BTreeNode[] children) {
+    this.children = children;
+  }
+
+  /**
+   * setting the next node
+   */
+  @Override public void setNextNode(BTreeNode nextNode) {
+    // no required in case of non leaf node
+  }
+
+  /**
+   * get the leaf node based on children
+   *
+   * @return leaf node
+   */
+  @Override public BTreeNode getChild(int index) {
+    return this.children[index];
+  }
+
+  /**
+   * add a key of a leaf node
+   *
+   * @param leaf node start keys
+   */
+  @Override public void setKey(IndexKey key) {
+    listOfKeys.add(key);
+
+  }
+
+  /**
+   * @return whether its a leaf node or not
+   */
+  @Override public boolean isLeafNode() {
+    return false;
+  }
+
+  /**
+   * Method to get the next block this can be used while scanning when
+   * iterator of this class can be used iterate over blocks
+   *
+   * @return next block
+   */
+  @Override public DataRefNode getNextDataRefNode() {
+    throw new UnsupportedOperationException("Unsupported operation");
+  }
+
+  /**
+   * to get the number of keys tuples present in the block
+   *
+   * @return number of keys in the block
+   */
+  @Override public int nodeSize() {
+    return listOfKeys.size();
+  }
+
+  /**
+   * Method can be used to get the block index .This can be used when multiple
+   * thread can be used scan group of blocks in that can we can assign the
+   * some of the blocks to one thread and some to other
+   *
+   * @return block number
+   */
+  @Override public long nodeNumber() {
+    throw new UnsupportedOperationException("Unsupported operation");
+  }
+
+  /**
+   * This method will be used to get the max value of all the columns this can
+   * be used in case of filter query
+   *
+   * @param max value of all the columns
+   */
+  @Override public byte[][] getColumnsMaxValue() {
+    // operation of getting the max value is not supported as its a non leaf
+    // node
+    // and in case of B+Tree data will be stored only in leaf node and
+    // intermediate
+    // node will be used only for searching the leaf node
+    throw new UnsupportedOperationException("Unsupported operation");
+  }
+
+  /**
+   * This method will be used to get the max value of all the columns this can
+   * be used in case of filter query
+   *
+   * @param min value of all the columns
+   */
+  @Override public byte[][] getColumnsMinValue() {
+    // operation of getting the min value is not supported as its a non leaf
+    // node
+    // and in case of B+Tree data will be stored only in leaf node and
+    // intermediate
+    // node will be used only for searching the leaf node
+    throw new UnsupportedOperationException("Unsupported operation");
+  }
+
+  /**
+   * Below method will be used to get the dimension chunks
+   *
+   * @param fileReader   file reader to read the chunks from file
+   * @param blockIndexes indexes of the blocks need to be read
+   * @return dimension data chunks
+   */
+  @Override public DimensionColumnDataChunk[] getDimensionChunks(FileHolder fileReader,
+      int[] blockIndexes) {
+
+    // operation of getting the dimension chunks is not supported as its a
+    // non leaf node
+    // and in case of B+Tree data will be stored only in leaf node and
+    // intermediate
+    // node will be used only for searching the leaf node
+    throw new UnsupportedOperationException("Unsupported operation");
+  }
+
+  /**
+   * Below method will be used to get the dimension chunk
+   *
+   * @param fileReader file reader to read the chunk from file
+   * @param blockIndex block index to be read
+   * @return dimension data chunk
+   */
+  @Override public DimensionColumnDataChunk getDimensionChunk(FileHolder fileReader,
+      int blockIndexes) {
+    // operation of getting the dimension chunk is not supported as its a
+    // non leaf node
+    // and in case of B+Tree data will be stored only in leaf node and
+    // intermediate
+    // node will be used only for searching the leaf node
+    throw new UnsupportedOperationException("Unsupported operation");
+  }
+
+  /**
+   * Below method will be used to get the measure chunk
+   *
+   * @param fileReader   file reader to read the chunk from file
+   * @param blockIndexes block indexes to be read from file
+   * @return measure column data chunk
+   */
+  @Override public MeasureColumnDataChunk[] getMeasureChunks(FileHolder fileReader,
+      int[] blockIndexes) {
+    // operation of getting the measure chunk is not supported as its a non
+    // leaf node
+    // and in case of B+Tree data will be stored only in leaf node and
+    // intermediate
+    // node will be used only for searching the leaf node
+    throw new UnsupportedOperationException("Unsupported operation");
+  }
+
+  /**
+   * Below method will be used to read the measure chunk
+   *
+   * @param fileReader file read to read the file chunk
+   * @param blockIndex block index to be read from file
+   * @return measure data chunk
+   */
+
+  @Override public MeasureColumnDataChunk getMeasureChunk(FileHolder fileReader, int blockIndex) {
+    // operation of getting the measure chunk is not supported as its a non
+    // leaf node
+    // and in case of B+Tree data will be stored only in leaf node and
+    // intermediate
+    // node will be used only for searching the leaf node
+    throw new UnsupportedOperationException("Unsupported operation");
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/cd6a4ff3/core/src/main/java/org/apache/carbondata/core/carbon/datastore/impl/btree/BlockBTreeBuilder.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/carbon/datastore/impl/btree/BlockBTreeBuilder.java b/core/src/main/java/org/apache/carbondata/core/carbon/datastore/impl/btree/BlockBTreeBuilder.java
new file mode 100644
index 0000000..3828818
--- /dev/null
+++ b/core/src/main/java/org/apache/carbondata/core/carbon/datastore/impl/btree/BlockBTreeBuilder.java
@@ -0,0 +1,118 @@
+/*
+ * 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.carbondata.core.carbon.datastore.impl.btree;
+/*
+ * 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.
+ */
+
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.carbondata.common.logging.LogService;
+import org.apache.carbondata.common.logging.LogServiceFactory;
+import org.apache.carbondata.core.carbon.datastore.BTreeBuilderInfo;
+import org.apache.carbondata.core.carbon.datastore.IndexKey;
+import org.apache.carbondata.core.constants.CarbonCommonConstants;
+
+/**
+ * Below class will be used to build the btree BTree will be built for all the
+ * blocks of a segment
+ */
+public class BlockBTreeBuilder extends AbstractBTreeBuilder {
+
+  /**
+   * Attribute for Carbon LOGGER
+   */
+  private static final LogService LOGGER =
+      LogServiceFactory.getLogService(BlockBTreeBuilder.class.getName());
+
+  /**
+   * Below method will be used to build the segment info bplus tree format
+   * Tree will be a read only tree, and it will be build on Bottoms up
+   * approach first all the leaf node will be built and then intermediate node
+   * in our case one leaf node will have not only one entry it will have group
+   * of entries
+   */
+  @Override public void build(BTreeBuilderInfo btreeBuilderInfo) {
+    int groupCounter;
+    int nInternal = 0;
+    BTreeNode curNode = null;
+    BTreeNode prevNode = null;
+    List<BTreeNode[]> nodeGroups =
+        new ArrayList<BTreeNode[]>(CarbonCommonConstants.CONSTANT_SIZE_TEN);
+    BTreeNode[] currentGroup = null;
+    List<List<IndexKey>> interNSKeyList =
+        new ArrayList<List<IndexKey>>(CarbonCommonConstants.CONSTANT_SIZE_TEN);
+    List<IndexKey> leafNSKeyList = null;
+    long nodeNumber = 0;
+    for (int metadataIndex = 0;
+         metadataIndex < btreeBuilderInfo.getFooterList().size(); metadataIndex++) {
+      // creating a leaf node
+      curNode = new BlockBTreeLeafNode(btreeBuilderInfo, metadataIndex, nodeNumber++);
+      nLeaf++;
+      // setting a next node as its a b+tree
+      // so all the leaf node will be chained
+      // will be stored in linked list
+      if (prevNode != null) {
+        prevNode.setNextNode(curNode);
+      }
+      prevNode = curNode;
+      // as intermediate node will have more than one leaf
+      // in cerating a group
+      groupCounter = (nLeaf - 1) % (maxNumberOfEntriesInNonLeafNodes);
+      if (groupCounter == 0) {
+        // Create new node group if current group is full
+        leafNSKeyList = new ArrayList<IndexKey>(CarbonCommonConstants.CONSTANT_SIZE_TEN);
+        currentGroup = new BTreeNode[maxNumberOfEntriesInNonLeafNodes];
+        nodeGroups.add(currentGroup);
+        nInternal++;
+        interNSKeyList.add(leafNSKeyList);
+      }
+      if (null != leafNSKeyList) {
+        leafNSKeyList.add(convertStartKeyToNodeEntry(
+            btreeBuilderInfo.getFooterList().get(metadataIndex).getBlockletIndex()
+                .getBtreeIndex().getStartKey()));
+      }
+      if (null != currentGroup) {
+        currentGroup[groupCounter] = curNode;
+      }
+    }
+    if (nLeaf == 0) {
+      return;
+    }
+    // adding a intermediate node
+    addIntermediateNode(curNode, nodeGroups, currentGroup, interNSKeyList, nInternal);
+    LOGGER.info("************************Total Number Rows In BTREE: " + nLeaf);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/cd6a4ff3/core/src/main/java/org/apache/carbondata/core/carbon/datastore/impl/btree/BlockBTreeLeafNode.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/carbon/datastore/impl/btree/BlockBTreeLeafNode.java b/core/src/main/java/org/apache/carbondata/core/carbon/datastore/impl/btree/BlockBTreeLeafNode.java
new file mode 100644
index 0000000..fcc98c2
--- /dev/null
+++ b/core/src/main/java/org/apache/carbondata/core/carbon/datastore/impl/btree/BlockBTreeLeafNode.java
@@ -0,0 +1,64 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.carbondata.core.carbon.datastore.impl.btree;
+
+import org.apache.carbondata.core.carbon.datastore.BTreeBuilderInfo;
+import org.apache.carbondata.core.carbon.datastore.block.TableBlockInfo;
+import org.apache.carbondata.core.carbon.metadata.blocklet.DataFileFooter;
+import org.apache.carbondata.core.carbon.metadata.blocklet.index.BlockletMinMaxIndex;
+
+/**
+ * Leaf node for btree where only min max will be store this can be used from
+ * driver when only we need to find whether particular block be selected for
+ * query execution
+ */
+public class BlockBTreeLeafNode extends AbstractBTreeLeafNode {
+
+  private TableBlockInfo blockInfo;
+
+  /**
+   * Create a leaf node
+   *
+   * @param builderInfos  builder infos which have required metadata to create a leaf
+   *                      node
+   * @param leafIndex     leaf node index
+   * @param metadataIndex metadata index
+   */
+  public BlockBTreeLeafNode(BTreeBuilderInfo builderInfos, int metadataIndex, long nodeNumber) {
+    DataFileFooter footer = builderInfos.getFooterList().get(metadataIndex);
+    BlockletMinMaxIndex minMaxIndex = footer.getBlockletIndex().getMinMaxIndex();
+    maxKeyOfColumns = minMaxIndex.getMaxValues();
+    minKeyOfColumns = minMaxIndex.getMinValues();
+    numberOfKeys = 1;
+    this.nodeNumber = nodeNumber;
+    this.blockInfo = footer.getTableBlockInfo();
+  }
+
+  /**
+   * Below method is to get the table block info
+   * This will be used only in case of BlockBtree leaf node which will
+   * be used to from driver
+   *
+   * @return TableBlockInfo
+   */
+  public TableBlockInfo getTableBlockInfo() {
+    return blockInfo;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/cd6a4ff3/core/src/main/java/org/apache/carbondata/core/carbon/datastore/impl/btree/BlockletBTreeBuilder.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/carbon/datastore/impl/btree/BlockletBTreeBuilder.java b/core/src/main/java/org/apache/carbondata/core/carbon/datastore/impl/btree/BlockletBTreeBuilder.java
new file mode 100644
index 0000000..3d6c11d
--- /dev/null
+++ b/core/src/main/java/org/apache/carbondata/core/carbon/datastore/impl/btree/BlockletBTreeBuilder.java
@@ -0,0 +1,104 @@
+/*
+ * 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.carbondata.core.carbon.datastore.impl.btree;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.carbondata.common.logging.LogService;
+import org.apache.carbondata.common.logging.LogServiceFactory;
+import org.apache.carbondata.core.carbon.datastore.BTreeBuilderInfo;
+import org.apache.carbondata.core.carbon.datastore.IndexKey;
+import org.apache.carbondata.core.constants.CarbonCommonConstants;
+
+/**
+ * Btree based builder which will build the leaf node in a b+ tree format
+ */
+public class BlockletBTreeBuilder extends AbstractBTreeBuilder {
+
+  /**
+   * Attribute for Carbon LOGGER
+   */
+  private static final LogService LOGGER =
+      LogServiceFactory.getLogService(BlockletBTreeBuilder.class.getName());
+
+  /**
+   * Below method will be used to build the segment info bplus tree format
+   * Tree will be a read only tree, and it will be build on Bottoms up approach
+   * first all the leaf node will be built and then intermediate node
+   * in our case one leaf node will have not only one entry it will have group of entries
+   */
+  @Override public void build(BTreeBuilderInfo segmentBuilderInfos) {
+    long totalNumberOfTuple = 0;
+    int groupCounter;
+    int nInternal = 0;
+    BTreeNode curNode = null;
+    BTreeNode prevNode = null;
+    List<BTreeNode[]> nodeGroups =
+        new ArrayList<BTreeNode[]>(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE);
+    BTreeNode[] currentGroup = null;
+    List<List<IndexKey>> interNSKeyList =
+        new ArrayList<List<IndexKey>>(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE);
+    List<IndexKey> leafNSKeyList = null;
+    long nodeNumber = 0;
+    for (int index = 0;
+         index < segmentBuilderInfos.getFooterList().get(0).getBlockletList()
+             .size(); index++) {
+      // creating a leaf node
+      curNode = new BlockletBTreeLeafNode(segmentBuilderInfos, index, nodeNumber++);
+      totalNumberOfTuple +=
+          segmentBuilderInfos.getFooterList().get(0).getBlockletList().get(index)
+              .getNumberOfRows();
+      nLeaf++;
+      // setting a next node as its a b+tree
+      // so all the leaf node will be chained
+      // will be stored in linked list
+      if (prevNode != null) {
+        prevNode.setNextNode(curNode);
+      }
+      prevNode = curNode;
+      // as intermediate node will have more than one leaf
+      // in cerating a group
+      groupCounter = (nLeaf - 1) % (maxNumberOfEntriesInNonLeafNodes);
+      if (groupCounter == 0) {
+        // Create new node group if current group is full
+        leafNSKeyList = new ArrayList<IndexKey>(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE);
+        currentGroup = new BTreeNode[maxNumberOfEntriesInNonLeafNodes];
+        nodeGroups.add(currentGroup);
+        nInternal++;
+        interNSKeyList.add(leafNSKeyList);
+      }
+      if (null != leafNSKeyList) {
+        leafNSKeyList.add(convertStartKeyToNodeEntry(
+            segmentBuilderInfos.getFooterList().get(0).getBlockletList().get(index)
+                .getBlockletIndex().getBtreeIndex().getStartKey()));
+      }
+      if (null != currentGroup) {
+        currentGroup[groupCounter] = curNode;
+      }
+    }
+    if (totalNumberOfTuple == 0) {
+      return;
+    }
+    // adding a intermediate node
+    addIntermediateNode(curNode, nodeGroups, currentGroup, interNSKeyList, nInternal);
+    LOGGER.info("****************************Total Number Rows In BTREE: " + totalNumberOfTuple);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/cd6a4ff3/core/src/main/java/org/apache/carbondata/core/carbon/datastore/impl/btree/BlockletBTreeLeafNode.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/carbon/datastore/impl/btree/BlockletBTreeLeafNode.java b/core/src/main/java/org/apache/carbondata/core/carbon/datastore/impl/btree/BlockletBTreeLeafNode.java
new file mode 100644
index 0000000..2bbddda
--- /dev/null
+++ b/core/src/main/java/org/apache/carbondata/core/carbon/datastore/impl/btree/BlockletBTreeLeafNode.java
@@ -0,0 +1,132 @@
+/*
+ * 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.carbondata.core.carbon.datastore.impl.btree;
+
+import org.apache.carbondata.core.carbon.datastore.BTreeBuilderInfo;
+import org.apache.carbondata.core.carbon.datastore.chunk.DimensionColumnDataChunk;
+import org.apache.carbondata.core.carbon.datastore.chunk.MeasureColumnDataChunk;
+import org.apache.carbondata.core.carbon.datastore.chunk.reader.DimensionColumnChunkReader;
+import org.apache.carbondata.core.carbon.datastore.chunk.reader.MeasureColumnChunkReader;
+import org.apache.carbondata.core.carbon.datastore.chunk.reader.dimension.CompressedDimensionChunkFileBasedReader;
+import org.apache.carbondata.core.carbon.datastore.chunk.reader.measure.CompressedMeasureChunkFileBasedReader;
+import org.apache.carbondata.core.carbon.metadata.blocklet.index.BlockletMinMaxIndex;
+import org.apache.carbondata.core.datastorage.store.FileHolder;
+import org.apache.carbondata.core.datastorage.store.compression.ValueCompressionModel;
+import org.apache.carbondata.core.util.CarbonUtil;
+
+/**
+ * Leaf node class of a Blocklet btree
+ */
+public class BlockletBTreeLeafNode extends AbstractBTreeLeafNode {
+
+  /**
+   * reader for dimension chunk
+   */
+  private DimensionColumnChunkReader dimensionChunksReader;
+
+  /**
+   * reader of measure chunk
+   */
+  private MeasureColumnChunkReader measureColumnChunkReader;
+
+  /**
+   * Create a leaf node
+   *
+   * @param builderInfos builder infos which have required metadata to create a leaf node
+   * @param leafIndex    leaf node index
+   * @param nodeNumber   node number of the node
+   *                     this will be used during query execution when we can
+   *                     give some leaf node of a btree to one executor some to other
+   */
+  public BlockletBTreeLeafNode(BTreeBuilderInfo builderInfos, int leafIndex, long nodeNumber) {
+    // get a lead node min max
+    BlockletMinMaxIndex minMaxIndex =
+        builderInfos.getFooterList().get(0).getBlockletList().get(leafIndex)
+            .getBlockletIndex().getMinMaxIndex();
+    // max key of the columns
+    maxKeyOfColumns = minMaxIndex.getMaxValues();
+    // min keys of the columns
+    minKeyOfColumns = minMaxIndex.getMinValues();
+    // number of keys present in the leaf
+    numberOfKeys = builderInfos.getFooterList().get(0).getBlockletList().get(leafIndex)
+        .getNumberOfRows();
+    // create a instance of dimension chunk
+    dimensionChunksReader = new CompressedDimensionChunkFileBasedReader(
+        builderInfos.getFooterList().get(0).getBlockletList().get(leafIndex)
+            .getDimensionColumnChunk(), builderInfos.getDimensionColumnValueSize(),
+        builderInfos.getFooterList().get(0).getTableBlockInfo().getFilePath());
+    // get the value compression model which was used to compress the measure values
+    ValueCompressionModel valueCompressionModel = CarbonUtil.getValueCompressionModel(
+        builderInfos.getFooterList().get(0).getBlockletList().get(leafIndex)
+            .getMeasureColumnChunk());
+    // create a instance of measure column chunk reader
+    measureColumnChunkReader = new CompressedMeasureChunkFileBasedReader(
+        builderInfos.getFooterList().get(0).getBlockletList().get(leafIndex)
+            .getMeasureColumnChunk(), valueCompressionModel,
+            builderInfos.getFooterList().get(0).getTableBlockInfo().getFilePath());
+    this.nodeNumber = nodeNumber;
+  }
+
+  /**
+   * Below method will be used to get the dimension chunks
+   *
+   * @param fileReader   file reader to read the chunks from file
+   * @param blockIndexes indexes of the blocks need to be read
+   * @return dimension data chunks
+   */
+  @Override public DimensionColumnDataChunk[] getDimensionChunks(FileHolder fileReader,
+      int[] blockIndexes) {
+    return dimensionChunksReader.readDimensionChunks(fileReader, blockIndexes);
+  }
+
+  /**
+   * Below method will be used to get the dimension chunk
+   *
+   * @param fileReader file reader to read the chunk from file
+   * @param blockIndex block index to be read
+   * @return dimension data chunk
+   */
+  @Override public DimensionColumnDataChunk getDimensionChunk(FileHolder fileReader,
+      int blockIndex) {
+    return dimensionChunksReader.readDimensionChunk(fileReader, blockIndex);
+  }
+
+  /**
+   * Below method will be used to get the measure chunk
+   *
+   * @param fileReader   file reader to read the chunk from file
+   * @param blockIndexes block indexes to be read from file
+   * @return measure column data chunk
+   */
+  @Override public MeasureColumnDataChunk[] getMeasureChunks(FileHolder fileReader,
+      int[] blockIndexes) {
+    return measureColumnChunkReader.readMeasureChunks(fileReader, blockIndexes);
+  }
+
+  /**
+   * Below method will be used to read the measure chunk
+   *
+   * @param fileReader file read to read the file chunk
+   * @param blockIndex block index to be read from file
+   * @return measure data chunk
+   */
+  @Override public MeasureColumnDataChunk getMeasureChunk(FileHolder fileReader, int blockIndex) {
+    return measureColumnChunkReader.readMeasureChunk(fileReader, blockIndex);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/cd6a4ff3/core/src/main/java/org/apache/carbondata/core/carbon/metadata/CarbonMetadata.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/carbon/metadata/CarbonMetadata.java b/core/src/main/java/org/apache/carbondata/core/carbon/metadata/CarbonMetadata.java
new file mode 100644
index 0000000..ce5e457
--- /dev/null
+++ b/core/src/main/java/org/apache/carbondata/core/carbon/metadata/CarbonMetadata.java
@@ -0,0 +1,165 @@
+/*
+ * 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.carbondata.core.carbon.metadata;
+
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+
+import org.apache.carbondata.core.carbon.metadata.schema.table.CarbonTable;
+import org.apache.carbondata.core.carbon.metadata.schema.table.TableInfo;
+import org.apache.carbondata.core.carbon.metadata.schema.table.column.CarbonDimension;
+
+/**
+ * Class which persist the information about the tables present the carbon schemas
+ */
+public final class CarbonMetadata {
+
+  /**
+   * meta data instance
+   */
+  private static final CarbonMetadata CARBONMETADATAINSTANCE = new CarbonMetadata();
+
+  /**
+   * holds the list of tableInfo currently present
+   */
+  private Map<String, CarbonTable> tableInfoMap;
+
+  private CarbonMetadata() {
+    // creating a concurrent map as it will be updated by multiple thread
+    tableInfoMap = new ConcurrentHashMap<String, CarbonTable>();
+  }
+
+  public static CarbonMetadata getInstance() {
+    return CARBONMETADATAINSTANCE;
+  }
+
+  /**
+   * removed the table information
+   *
+   * @param tableUniquName
+   */
+  public void removeTable(String tableUniquName) {
+    tableInfoMap.remove(convertToLowerCase(tableUniquName));
+  }
+
+  /**
+   * Below method will be used to set the carbon table
+   * This method will be used in executor side as driver will always have
+   * updated table so from driver during query execution and data loading
+   * we just need to add the table
+   *
+   * @param carbonTable
+   */
+  public void addCarbonTable(CarbonTable carbonTable) {
+    tableInfoMap.put(convertToLowerCase(carbonTable.getTableUniqueName()), carbonTable);
+  }
+
+  /**
+   * method load the table
+   *
+   * @param tableInfo
+   */
+  public void loadTableMetadata(TableInfo tableInfo) {
+    CarbonTable carbonTable = tableInfoMap.get(convertToLowerCase(tableInfo.getTableUniqueName()));
+    if (null == carbonTable || carbonTable.getTableLastUpdatedTime() < tableInfo
+        .getLastUpdatedTime()) {
+      carbonTable = new CarbonTable();
+      carbonTable.loadCarbonTable(tableInfo);
+      tableInfoMap.put(convertToLowerCase(tableInfo.getTableUniqueName()), carbonTable);
+    }
+  }
+
+  /**
+   * Below method to get the loaded carbon table
+   *
+   * @param tableUniqueName
+   * @return
+   */
+  public CarbonTable getCarbonTable(String tableUniqueName) {
+    return tableInfoMap.get(convertToLowerCase(tableUniqueName));
+  }
+
+  /**
+   * @return the number of tables present in the schema
+   */
+  public int getNumberOfTables() {
+    return tableInfoMap.size();
+  }
+
+  /**
+   * returns the given string in lowercase
+   * @param table
+   * @return
+   */
+  public String convertToLowerCase(String table) {
+    return table.toLowerCase();
+  }
+
+  /**
+   * method will return dimension instance based on the column identifier
+   * and table instance passed to it.
+   *
+   * @param carbonTable
+   * @param columnIdentifier
+   * @return CarbonDimension instance
+   */
+  public CarbonDimension getCarbonDimensionBasedOnColIdentifier(CarbonTable carbonTable,
+      String columnIdentifier) {
+    List<CarbonDimension> listOfCarbonDims =
+        carbonTable.getDimensionByTableName(carbonTable.getFactTableName());
+    for (CarbonDimension dimension : listOfCarbonDims) {
+      if (dimension.getColumnId().equals(columnIdentifier)) {
+        return dimension;
+      }
+      if (dimension.numberOfChild() > 0) {
+        CarbonDimension childDim =
+            getCarbonChildDimsBasedOnColIdentifier(columnIdentifier, dimension);
+        if (null != childDim) {
+          return childDim;
+        }
+      }
+    }
+    return null;
+  }
+
+  /**
+   * Below method will be used to get the dimension based on column identifier
+   * for complex dimension children
+   *
+   * @param columnIdentifier column identifier
+   * @param dimension        parent dimension
+   * @return children dimension
+   */
+  private CarbonDimension getCarbonChildDimsBasedOnColIdentifier(String columnIdentifier,
+      CarbonDimension dimension) {
+    for (int i = 0; i < dimension.numberOfChild(); i++) {
+      if (dimension.getListOfChildDimensions().get(i).getColumnId().equals(columnIdentifier)) {
+        return dimension.getListOfChildDimensions().get(i);
+      } else if (dimension.getListOfChildDimensions().get(i).numberOfChild() > 0) {
+        CarbonDimension childDim = getCarbonChildDimsBasedOnColIdentifier(columnIdentifier,
+            dimension.getListOfChildDimensions().get(i));
+        if (null != childDim) {
+          return childDim;
+        }
+      }
+    }
+    return null;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/cd6a4ff3/core/src/main/java/org/apache/carbondata/core/carbon/metadata/blocklet/BlockletInfo.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/carbon/metadata/blocklet/BlockletInfo.java b/core/src/main/java/org/apache/carbondata/core/carbon/metadata/blocklet/BlockletInfo.java
new file mode 100644
index 0000000..b2c72aa
--- /dev/null
+++ b/core/src/main/java/org/apache/carbondata/core/carbon/metadata/blocklet/BlockletInfo.java
@@ -0,0 +1,114 @@
+/*
+ * 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.carbondata.core.carbon.metadata.blocklet;
+
+import java.io.Serializable;
+import java.util.List;
+
+import org.apache.carbondata.core.carbon.metadata.blocklet.datachunk.DataChunk;
+import org.apache.carbondata.core.carbon.metadata.blocklet.index.BlockletIndex;
+
+/**
+ * class to store the information about the blocklet
+ */
+public class BlockletInfo implements Serializable {
+
+  /**
+   * serialization id
+   */
+  private static final long serialVersionUID = 1873135459695635381L;
+
+  /**
+   * Number of rows in this blocklet
+   */
+  private int numberOfRows;
+
+  /**
+   * Information about dimension chunk of all dimensions in this blocklet
+   */
+  private List<DataChunk> dimensionColumnChunk;
+
+  /**
+   * Information about measure chunk of all measures in this blocklet
+   */
+  private List<DataChunk> measureColumnChunk;
+
+  /**
+   * to store the index like min max and start and end key of each column of the blocklet
+   */
+  private BlockletIndex blockletIndex;
+
+  /**
+   * @return the numberOfRows
+   */
+  public int getNumberOfRows() {
+    return numberOfRows;
+  }
+
+  /**
+   * @param numberOfRows the numberOfRows to set
+   */
+  public void setNumberOfRows(int numberOfRows) {
+    this.numberOfRows = numberOfRows;
+  }
+
+  /**
+   * @return the dimensionColumnChunk
+   */
+  public List<DataChunk> getDimensionColumnChunk() {
+    return dimensionColumnChunk;
+  }
+
+  /**
+   * @param dimensionColumnChunk the dimensionColumnChunk to set
+   */
+  public void setDimensionColumnChunk(List<DataChunk> dimensionColumnChunk) {
+    this.dimensionColumnChunk = dimensionColumnChunk;
+  }
+
+  /**
+   * @return the measureColumnChunk
+   */
+  public List<DataChunk> getMeasureColumnChunk() {
+    return measureColumnChunk;
+  }
+
+  /**
+   * @param measureColumnChunk the measureColumnChunk to set
+   */
+  public void setMeasureColumnChunk(List<DataChunk> measureColumnChunk) {
+    this.measureColumnChunk = measureColumnChunk;
+  }
+
+  /**
+   * @return the blockletIndex
+   */
+  public BlockletIndex getBlockletIndex() {
+    return blockletIndex;
+  }
+
+  /**
+   * @param blockletIndex the blockletIndex to set
+   */
+  public void setBlockletIndex(BlockletIndex blockletIndex) {
+    this.blockletIndex = blockletIndex;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/cd6a4ff3/core/src/main/java/org/apache/carbondata/core/carbon/metadata/blocklet/DataFileFooter.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/carbon/metadata/blocklet/DataFileFooter.java b/core/src/main/java/org/apache/carbondata/core/carbon/metadata/blocklet/DataFileFooter.java
new file mode 100644
index 0000000..55587da
--- /dev/null
+++ b/core/src/main/java/org/apache/carbondata/core/carbon/metadata/blocklet/DataFileFooter.java
@@ -0,0 +1,170 @@
+/*
+ * 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.carbondata.core.carbon.metadata.blocklet;
+
+import java.io.Serializable;
+import java.util.List;
+
+import org.apache.carbondata.core.carbon.datastore.block.TableBlockInfo;
+import org.apache.carbondata.core.carbon.metadata.blocklet.index.BlockletIndex;
+import org.apache.carbondata.core.carbon.metadata.schema.table.column.ColumnSchema;
+
+/**
+ * Information of one data file
+ */
+public class DataFileFooter implements Serializable {
+
+  /**
+   * serialization id
+   */
+  private static final long serialVersionUID = -7284319972734500751L;
+
+  /**
+   * version used for data compatibility
+   */
+  private int versionId;
+
+  /**
+   * total number of rows in this file
+   */
+  private long numberOfRows;
+
+  /**
+   * Segment info (will be same/repeated for all block in this segment)
+   */
+  private SegmentInfo segmentInfo;
+
+  /**
+   * Information about leaf nodes of all columns in this file
+   */
+  private List<BlockletInfo> blockletList;
+
+  /**
+   * blocklet index of all blocklets in this file
+   */
+  private BlockletIndex blockletIndex;
+
+  /**
+   * Description of columns in this file
+   */
+  private List<ColumnSchema> columnInTable;
+
+  /**
+   * to store the block info detail like file name block index and locations
+   */
+  private TableBlockInfo tableBlockInfo;
+
+  /**
+   * @return the versionId
+   */
+  public int getVersionId() {
+    return versionId;
+  }
+
+  /**
+   * @param versionId the versionId to set
+   */
+  public void setVersionId(int versionId) {
+    this.versionId = versionId;
+  }
+
+  /**
+   * @return the numberOfRows
+   */
+  public long getNumberOfRows() {
+    return numberOfRows;
+  }
+
+  /**
+   * @param numberOfRows the numberOfRows to set
+   */
+  public void setNumberOfRows(long numberOfRows) {
+    this.numberOfRows = numberOfRows;
+  }
+
+  /**
+   * @return the segmentInfo
+   */
+  public SegmentInfo getSegmentInfo() {
+    return segmentInfo;
+  }
+
+  /**
+   * @param segmentInfo the segmentInfo to set
+   */
+  public void setSegmentInfo(SegmentInfo segmentInfo) {
+    this.segmentInfo = segmentInfo;
+  }
+
+  /**
+   * @return the List of Blocklet
+   */
+  public List<BlockletInfo> getBlockletList() {
+    return blockletList;
+  }
+
+  /**
+   * @param blockletList the blockletList to set
+   */
+  public void setBlockletList(List<BlockletInfo> blockletList) {
+    this.blockletList = blockletList;
+  }
+
+  /**
+   * @return the blockletIndex
+   */
+  public BlockletIndex getBlockletIndex() {
+    return blockletIndex;
+  }
+
+  /**
+   * @param blockletIndex the blockletIndex to set
+   */
+  public void setBlockletIndex(BlockletIndex blockletIndex) {
+    this.blockletIndex = blockletIndex;
+  }
+
+  /**
+   * @return the columnInTable
+   */
+  public List<ColumnSchema> getColumnInTable() {
+    return columnInTable;
+  }
+
+  /**
+   * @param columnInTable the columnInTable to set
+   */
+  public void setColumnInTable(List<ColumnSchema> columnInTable) {
+    this.columnInTable = columnInTable;
+  }
+
+  /**
+   * @return the tableBlockInfo
+   */
+  public TableBlockInfo getTableBlockInfo() {
+    return tableBlockInfo;
+  }
+
+  /**
+   * @param tableBlockInfo the tableBlockInfo to set
+   */
+  public void setTableBlockInfo(TableBlockInfo tableBlockInfo) {
+    this.tableBlockInfo = tableBlockInfo;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/cd6a4ff3/core/src/main/java/org/apache/carbondata/core/carbon/metadata/blocklet/SegmentInfo.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/carbon/metadata/blocklet/SegmentInfo.java b/core/src/main/java/org/apache/carbondata/core/carbon/metadata/blocklet/SegmentInfo.java
new file mode 100644
index 0000000..c0d6b55
--- /dev/null
+++ b/core/src/main/java/org/apache/carbondata/core/carbon/metadata/blocklet/SegmentInfo.java
@@ -0,0 +1,72 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.carbondata.core.carbon.metadata.blocklet;
+
+import java.io.Serializable;
+
+/**
+ * Class holds the information about the segment information
+ */
+public class SegmentInfo implements Serializable {
+
+  /**
+   * serialization version
+   */
+  private static final long serialVersionUID = -1749874611112709431L;
+
+  /**
+   * number of column in the segment
+   */
+  private int numberOfColumns;
+
+  /**
+   * cardinality of each columns
+   * column which is not participating in the multidimensional key cardinality will be -1;
+   */
+  private int[] columnCardinality;
+
+  /**
+   * @return the numberOfColumns
+   */
+  public int getNumberOfColumns() {
+    return numberOfColumns;
+  }
+
+  /**
+   * @param numberOfColumns the numberOfColumns to set
+   */
+  public void setNumberOfColumns(int numberOfColumns) {
+    this.numberOfColumns = numberOfColumns;
+  }
+
+  /**
+   * @return the columnCardinality
+   */
+  public int[] getColumnCardinality() {
+    return columnCardinality;
+  }
+
+  /**
+   * @param columnCardinality the columnCardinality to set
+   */
+  public void setColumnCardinality(int[] columnCardinality) {
+    this.columnCardinality = columnCardinality;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/cd6a4ff3/core/src/main/java/org/apache/carbondata/core/carbon/metadata/blocklet/compressor/ChunkCompressorMeta.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/carbon/metadata/blocklet/compressor/ChunkCompressorMeta.java b/core/src/main/java/org/apache/carbondata/core/carbon/metadata/blocklet/compressor/ChunkCompressorMeta.java
new file mode 100644
index 0000000..4f1ae37
--- /dev/null
+++ b/core/src/main/java/org/apache/carbondata/core/carbon/metadata/blocklet/compressor/ChunkCompressorMeta.java
@@ -0,0 +1,92 @@
+/*
+ * 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.carbondata.core.carbon.metadata.blocklet.compressor;
+
+import java.io.Serializable;
+
+/**
+ * Represents the compression information of data of one dimension
+ * one dimension group in one blocklet
+ */
+public class ChunkCompressorMeta implements Serializable {
+
+  /**
+   * serialization version
+   */
+  private static final long serialVersionUID = -6697087170420991140L;
+
+  /**
+   * data chunk compressor
+   */
+  private CompressionCodec compressor;
+
+  /**
+   * total byte size of all uncompressed pages in this column chunk (including the headers)
+   */
+  private long uncompressedSize;
+
+  /**
+   * total byte size of all compressed pages in this column chunk (including the headers)
+   */
+  private long compressedSize;
+
+  /**
+   * @return the compressor
+   */
+  public CompressionCodec getCompressorCodec() {
+    return compressor;
+  }
+
+  /**
+   * @param compressor the compressor to set
+   */
+  public void setCompressor(CompressionCodec compressor) {
+    this.compressor = compressor;
+  }
+
+  /**
+   * @return the uncompressedSize
+   */
+  public long getUncompressedSize() {
+    return uncompressedSize;
+  }
+
+  /**
+   * @param uncompressedSize the uncompressedSize to set
+   */
+  public void setUncompressedSize(long uncompressedSize) {
+    this.uncompressedSize = uncompressedSize;
+  }
+
+  /**
+   * @return the compressedSize
+   */
+  public long getCompressedSize() {
+    return compressedSize;
+  }
+
+  /**
+   * @param compressedSize the compressedSize to set
+   */
+  public void setCompressedSize(long compressedSize) {
+    this.compressedSize = compressedSize;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/cd6a4ff3/core/src/main/java/org/apache/carbondata/core/carbon/metadata/blocklet/compressor/CompressionCodec.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/carbon/metadata/blocklet/compressor/CompressionCodec.java b/core/src/main/java/org/apache/carbondata/core/carbon/metadata/blocklet/compressor/CompressionCodec.java
new file mode 100644
index 0000000..6f302ec
--- /dev/null
+++ b/core/src/main/java/org/apache/carbondata/core/carbon/metadata/blocklet/compressor/CompressionCodec.java
@@ -0,0 +1,31 @@
+/*
+ * 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.carbondata.core.carbon.metadata.blocklet.compressor;
+
+/**
+ * Compressions supported by Carbon Data.
+ */
+public enum CompressionCodec {
+
+  /**
+   * snappy compression
+   */
+  SNAPPY,
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/cd6a4ff3/core/src/main/java/org/apache/carbondata/core/carbon/metadata/blocklet/datachunk/DataChunk.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/carbon/metadata/blocklet/datachunk/DataChunk.java b/core/src/main/java/org/apache/carbondata/core/carbon/metadata/blocklet/datachunk/DataChunk.java
new file mode 100644
index 0000000..584e51f
--- /dev/null
+++ b/core/src/main/java/org/apache/carbondata/core/carbon/metadata/blocklet/datachunk/DataChunk.java
@@ -0,0 +1,327 @@
+/*
+ * 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.carbondata.core.carbon.metadata.blocklet.datachunk;
+
+import java.io.Serializable;
+import java.util.List;
+
+import org.apache.carbondata.core.carbon.metadata.blocklet.compressor.ChunkCompressorMeta;
+import org.apache.carbondata.core.carbon.metadata.blocklet.sort.SortState;
+import org.apache.carbondata.core.carbon.metadata.encoder.Encoding;
+import org.apache.carbondata.core.metadata.ValueEncoderMeta;
+
+/**
+ * Class holds the information about the data chunk metadata
+ */
+public class DataChunk implements Serializable {
+
+  /**
+   * serialization version
+   */
+  private static final long serialVersionUID = 1L;
+
+  /**
+   * the compression meta data of a chunk
+   */
+  private ChunkCompressorMeta chunkCompressionMeta;
+
+  /**
+   * whether this chunk is a row chunk or column chunk
+   */
+  private boolean isRowMajor;
+
+  /**
+   * the column IDs in this chunk, will have atleast
+   * one column ID for columnar format, many column ID for
+   * row major format
+   */
+  private List<Integer> columnUniqueIdList;
+
+  /**
+   * Offset of data page
+   */
+  private long dataPageOffset;
+
+  /**
+   * length of data page
+   */
+  private int dataPageLength;
+
+  /**
+   * information about presence of values in each row of this column chunk
+   */
+  private transient PresenceMeta nullValueIndexForColumn;
+
+  /**
+   * offset of row id page, only if encoded using inverted index
+   */
+  private long rowIdPageOffset;
+
+  /**
+   * length of row id page, only if encoded using inverted index
+   */
+  private int rowIdPageLength;
+
+  /**
+   * offset of rle page, only if RLE coded.
+   */
+  private long rlePageOffset;
+
+  /**
+   * length of rle page, only if RLE coded.
+   */
+  private int rlePageLength;
+
+  /**
+   * is rle is applied in the data chunk
+   */
+  private boolean isRleApplied;
+
+  /**
+   * is dictionary is applied in the column, only if it is a dimension column
+   */
+  private boolean isNoDictonaryColumn;
+
+  /**
+   * sorting type selected for chunk;
+   */
+  private SortState sortState;
+
+  /**
+   * The List of encoders overriden at node level
+   */
+  private List<Encoding> encodingList;
+
+  /**
+   * value encoder meta which will holds the information
+   * about max, min, decimal length, type
+   */
+  private List<ValueEncoderMeta> valueEncoderMetaList;
+
+  /**
+   * @return the chunkCompressionMeta
+   */
+  public ChunkCompressorMeta getChunkCompressionMeta() {
+    return chunkCompressionMeta;
+  }
+
+  /**
+   * @param chunkCompressionMeta the chunkCompressionMeta to set
+   */
+  public void setChunkCompressionMeta(ChunkCompressorMeta chunkCompressionMeta) {
+    this.chunkCompressionMeta = chunkCompressionMeta;
+  }
+
+  /**
+   * @return the isRowMajor
+   */
+  public boolean isRowMajor() {
+    return isRowMajor;
+  }
+
+  /**
+   * @param isRowMajor the isRowMajor to set
+   */
+  public void setRowMajor(boolean isRowMajor) {
+    this.isRowMajor = isRowMajor;
+  }
+
+  /**
+   * @return the columnUniqueIdList
+   */
+  public List<Integer> getColumnUniqueIdList() {
+    return columnUniqueIdList;
+  }
+
+  /**
+   * @param columnUniqueIdList the columnUniqueIdList to set
+   */
+  public void setColumnUniqueIdList(List<Integer> columnUniqueIdList) {
+    this.columnUniqueIdList = columnUniqueIdList;
+  }
+
+  /**
+   * @return the dataPageOffset
+   */
+  public long getDataPageOffset() {
+    return dataPageOffset;
+  }
+
+  /**
+   * @param dataPageOffset the dataPageOffset to set
+   */
+  public void setDataPageOffset(long dataPageOffset) {
+    this.dataPageOffset = dataPageOffset;
+  }
+
+  /**
+   * @return the dataPageLength
+   */
+  public int getDataPageLength() {
+    return dataPageLength;
+  }
+
+  /**
+   * @param dataPageLength the dataPageLength to set
+   */
+  public void setDataPageLength(int dataPageLength) {
+    this.dataPageLength = dataPageLength;
+  }
+
+  /**
+   * @return the nullValueIndexForColumn
+   */
+  public PresenceMeta getNullValueIndexForColumn() {
+    return nullValueIndexForColumn;
+  }
+
+  /**
+   * @param nullValueIndexForColumn the nullValueIndexForColumn to set
+   */
+  public void setNullValueIndexForColumn(PresenceMeta nullValueIndexForColumn) {
+    this.nullValueIndexForColumn = nullValueIndexForColumn;
+  }
+
+  /**
+   * @return the rowIdPageOffset
+   */
+  public long getRowIdPageOffset() {
+    return rowIdPageOffset;
+  }
+
+  /**
+   * @param rowIdPageOffset the rowIdPageOffset to set
+   */
+  public void setRowIdPageOffset(long rowIdPageOffset) {
+    this.rowIdPageOffset = rowIdPageOffset;
+  }
+
+  /**
+   * @return the rowIdPageLength
+   */
+  public int getRowIdPageLength() {
+    return rowIdPageLength;
+  }
+
+  /**
+   * @param rowIdPageLength the rowIdPageLength to set
+   */
+  public void setRowIdPageLength(int rowIdPageLength) {
+    this.rowIdPageLength = rowIdPageLength;
+  }
+
+  /**
+   * @return the rlePageOffset
+   */
+  public long getRlePageOffset() {
+    return rlePageOffset;
+  }
+
+  /**
+   * @param rlePageOffset the rlePageOffset to set
+   */
+  public void setRlePageOffset(long rlePageOffset) {
+    this.rlePageOffset = rlePageOffset;
+  }
+
+  /**
+   * @return the rlePageLength
+   */
+  public int getRlePageLength() {
+    return rlePageLength;
+  }
+
+  /**
+   * @param rlePageLength the rlePageLength to set
+   */
+  public void setRlePageLength(int rlePageLength) {
+    this.rlePageLength = rlePageLength;
+  }
+
+  /**
+   * @return the isRleApplied
+   */
+  public boolean isRleApplied() {
+    return isRleApplied;
+  }
+
+  /**
+   * @param isRleApplied the isRleApplied to set
+   */
+  public void setRleApplied(boolean isRleApplied) {
+    this.isRleApplied = isRleApplied;
+  }
+
+  /**
+   * @return the isNoDictonaryColumn
+   */
+  public boolean isNoDictonaryColumn() {
+    return isNoDictonaryColumn;
+  }
+
+  /**
+   * @param isNoDictonaryColumn the isNoDictonaryColumn to set
+   */
+  public void setNoDictonaryColumn(boolean isNoDictonaryColumn) {
+    this.isNoDictonaryColumn = isNoDictonaryColumn;
+  }
+
+  /**
+   * @return the sortState
+   */
+  public SortState getSortState() {
+    return sortState;
+  }
+
+  /**
+   * @param sortState the sortState to set
+   */
+  public void setSortState(SortState sortState) {
+    this.sortState = sortState;
+  }
+
+  /**
+   * @return the encoderList
+   */
+  public List<Encoding> getEncodingList() {
+    return encodingList;
+  }
+
+  /**
+   * @param encoderList the encoderList to set
+   */
+  public void setEncoderList(List<Encoding> encodingList) {
+    this.encodingList = encodingList;
+  }
+
+  /**
+   * @return the valueEncoderMeta
+   */
+  public List<ValueEncoderMeta> getValueEncoderMeta() {
+    return valueEncoderMetaList;
+  }
+
+  /**
+   * @param valueEncoderMeta the valueEncoderMeta to set
+   */
+  public void setValueEncoderMeta(List<ValueEncoderMeta> valueEncoderMetaList) {
+    this.valueEncoderMetaList = valueEncoderMetaList;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/cd6a4ff3/core/src/main/java/org/apache/carbondata/core/carbon/metadata/blocklet/datachunk/PresenceMeta.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/carbon/metadata/blocklet/datachunk/PresenceMeta.java b/core/src/main/java/org/apache/carbondata/core/carbon/metadata/blocklet/datachunk/PresenceMeta.java
new file mode 100644
index 0000000..f73d3d0
--- /dev/null
+++ b/core/src/main/java/org/apache/carbondata/core/carbon/metadata/blocklet/datachunk/PresenceMeta.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.carbondata.core.carbon.metadata.blocklet.datachunk;
+
+import java.util.BitSet;
+
+/**
+ * information about presence of values in each row of the column chunk
+ */
+public class PresenceMeta {
+
+  /**
+   * if true, ones in the bit stream reprents presence. otherwise represents absence
+   */
+  private boolean representNullValues;
+
+  /**
+   * Compressed bit stream representing the presence of null values
+   */
+  private BitSet bitSet;
+
+  /**
+   * @return the representNullValues
+   */
+  public boolean isRepresentNullValues() {
+    return representNullValues;
+  }
+
+  /**
+   * @param representNullValues the representNullValues to set
+   */
+  public void setRepresentNullValues(boolean representNullValues) {
+    this.representNullValues = representNullValues;
+  }
+
+  /**
+   * @return the bitSet
+   */
+  public BitSet getBitSet() {
+    return bitSet;
+  }
+
+  /**
+   * @param bitSet the bitSet to set
+   */
+  public void setBitSet(BitSet bitSet) {
+    this.bitSet = bitSet;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/cd6a4ff3/core/src/main/java/org/apache/carbondata/core/carbon/metadata/blocklet/index/BlockletBTreeIndex.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/carbon/metadata/blocklet/index/BlockletBTreeIndex.java b/core/src/main/java/org/apache/carbondata/core/carbon/metadata/blocklet/index/BlockletBTreeIndex.java
new file mode 100644
index 0000000..680b4eb
--- /dev/null
+++ b/core/src/main/java/org/apache/carbondata/core/carbon/metadata/blocklet/index/BlockletBTreeIndex.java
@@ -0,0 +1,76 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.carbondata.core.carbon.metadata.blocklet.index;
+
+import java.io.Serializable;
+
+/**
+ * Class hold the information about start and end key of one blocklet
+ */
+public class BlockletBTreeIndex implements Serializable {
+
+  /**
+   * serialization version
+   */
+  private static final long serialVersionUID = 6116185464700853045L;
+
+  /**
+   * Bit-packed start key of one blocklet
+   */
+  private byte[] startKey;
+
+  /**
+   * Bit-packed start key of one blocklet
+   */
+  private byte[] endKey;
+
+  public BlockletBTreeIndex() {
+  }
+
+  public BlockletBTreeIndex(byte[] startKey, byte[] endKey) {
+    this.startKey = startKey;
+    this.endKey = endKey;
+  }
+
+  /**
+   * @return the startKey
+   */
+  public byte[] getStartKey() {
+    return startKey;
+  }
+
+  /**
+   * @param startKey the startKey to set
+   */
+  public void setStartKey(byte[] startKey) {
+    this.startKey = startKey;
+  }
+
+  /**
+   * @return the endKey
+   */
+  public byte[] getEndKey() {
+    return endKey;
+  }
+
+  /**
+   * @param endKey the endKey to set
+   */
+  public void setEndKey(byte[] endKey) {
+    this.endKey = endKey;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/cd6a4ff3/core/src/main/java/org/apache/carbondata/core/carbon/metadata/blocklet/index/BlockletIndex.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/carbon/metadata/blocklet/index/BlockletIndex.java b/core/src/main/java/org/apache/carbondata/core/carbon/metadata/blocklet/index/BlockletIndex.java
new file mode 100644
index 0000000..b78d2b9
--- /dev/null
+++ b/core/src/main/java/org/apache/carbondata/core/carbon/metadata/blocklet/index/BlockletIndex.java
@@ -0,0 +1,77 @@
+/**
+ * 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.carbondata.core.carbon.metadata.blocklet.index;
+
+import java.io.Serializable;
+
+/**
+ * Persist Index of all blocklets in one file
+ */
+public class BlockletIndex implements Serializable {
+
+  /**
+   * serialization version
+   */
+  private static final long serialVersionUID = 1L;
+
+  /**
+   * list of btree index for all the leaf
+   */
+  private BlockletBTreeIndex btreeIndex;
+
+  /**
+   * list of max and min key of all leaf
+   */
+  private BlockletMinMaxIndex minMaxIndex;
+
+  public BlockletIndex() {
+  }
+
+  public BlockletIndex(BlockletBTreeIndex btree, BlockletMinMaxIndex minmax) {
+    this.btreeIndex = btree;
+    this.minMaxIndex = minmax;
+  }
+
+  /**
+   * @return the btreeIndex
+   */
+  public BlockletBTreeIndex getBtreeIndex() {
+    return btreeIndex;
+  }
+
+  /**
+   * @param btreeIndex the btreeIndex to set
+   */
+  public void setBtreeIndex(BlockletBTreeIndex btreeIndex) {
+    this.btreeIndex = btreeIndex;
+  }
+
+  /**
+   * @return the minMaxIndex
+   */
+  public BlockletMinMaxIndex getMinMaxIndex() {
+    return minMaxIndex;
+  }
+
+  /**
+   * @param minMaxIndex the minMaxIndex to set
+   */
+  public void setMinMaxIndex(BlockletMinMaxIndex minMaxIndex) {
+    this.minMaxIndex = minMaxIndex;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/cd6a4ff3/core/src/main/java/org/apache/carbondata/core/carbon/metadata/blocklet/index/BlockletMinMaxIndex.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/carbon/metadata/blocklet/index/BlockletMinMaxIndex.java b/core/src/main/java/org/apache/carbondata/core/carbon/metadata/blocklet/index/BlockletMinMaxIndex.java
new file mode 100644
index 0000000..928884f
--- /dev/null
+++ b/core/src/main/java/org/apache/carbondata/core/carbon/metadata/blocklet/index/BlockletMinMaxIndex.java
@@ -0,0 +1,83 @@
+/**
+ * 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.carbondata.core.carbon.metadata.blocklet.index;
+
+import java.io.Serializable;
+import java.nio.ByteBuffer;
+import java.util.List;
+
+/**
+ * Below class holds the information of max and min value of all the columns in a blocklet
+ */
+public class BlockletMinMaxIndex implements Serializable {
+
+  /**
+   * serialization version
+   */
+  private static final long serialVersionUID = -4311405145501302895L;
+
+  /**
+   * Min value of all columns of one blocklet Bit-Packed
+   */
+  private byte[][] minValues;
+
+  /**
+   * Max value of all columns of one blocklet Bit-Packed
+   */
+  private byte[][] maxValues;
+
+  public BlockletMinMaxIndex() {
+  }
+
+  public BlockletMinMaxIndex(List<ByteBuffer> minValues, List<ByteBuffer> maxValues) {
+    this.minValues = new byte[minValues.size()][];
+    this.maxValues = new byte[maxValues.size()][];
+    for (int i = 0; i < minValues.size(); i++) {
+      this.minValues[i] = minValues.get(i).array();
+      this.maxValues[i] = maxValues.get(i).array();
+    }
+  }
+
+  /**
+   * @return the minValues
+   */
+  public byte[][] getMinValues() {
+    return minValues;
+  }
+
+  /**
+   * @param minValues the minValues to set
+   */
+  public void setMinValues(byte[][] minValues) {
+    this.minValues = minValues;
+  }
+
+  /**
+   * @return the maxValues
+   */
+  public byte[][] getMaxValues() {
+    return maxValues;
+  }
+
+  /**
+   * @param maxValues the maxValues to set
+   */
+  public void setMaxValues(byte[][] maxValues) {
+    this.maxValues = maxValues;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/cd6a4ff3/core/src/main/java/org/apache/carbondata/core/carbon/metadata/blocklet/sort/SortState.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/carbon/metadata/blocklet/sort/SortState.java b/core/src/main/java/org/apache/carbondata/core/carbon/metadata/blocklet/sort/SortState.java
new file mode 100644
index 0000000..122198c
--- /dev/null
+++ b/core/src/main/java/org/apache/carbondata/core/carbon/metadata/blocklet/sort/SortState.java
@@ -0,0 +1,38 @@
+/**
+ * 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.carbondata.core.carbon.metadata.blocklet.sort;
+
+/**
+ * Enum for sort type information
+ */
+public enum SortState {
+
+  /**
+   * column is not sorted
+   */
+  SORT_NONE,
+
+  /**
+   * data from source was already in sorted order
+   */
+  SORT_NATIVE,
+
+  /**
+   * data from source was not sorted,so data is explicitly sorted
+   */
+  SORT_EXPLICT;
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/cd6a4ff3/core/src/main/java/org/apache/carbondata/core/carbon/metadata/converter/SchemaConverter.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/carbon/metadata/converter/SchemaConverter.java b/core/src/main/java/org/apache/carbondata/core/carbon/metadata/converter/SchemaConverter.java
new file mode 100644
index 0000000..c00c197
--- /dev/null
+++ b/core/src/main/java/org/apache/carbondata/core/carbon/metadata/converter/SchemaConverter.java
@@ -0,0 +1,105 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.carbondata.core.carbon.metadata.converter;
+
+import org.apache.carbondata.core.carbon.metadata.schema.SchemaEvolution;
+import org.apache.carbondata.core.carbon.metadata.schema.SchemaEvolutionEntry;
+import org.apache.carbondata.core.carbon.metadata.schema.table.TableInfo;
+import org.apache.carbondata.core.carbon.metadata.schema.table.TableSchema;
+import org.apache.carbondata.core.carbon.metadata.schema.table.column.ColumnSchema;
+
+/**
+ * Converter interface which will be implemented for external to carbon schema
+ */
+public interface SchemaConverter {
+  /**
+   * @param wrapperSchemaEvolutionEntry
+   * @return
+   */
+  org.apache.carbondata.format.SchemaEvolutionEntry fromWrapperToExternalSchemaEvolutionEntry(
+      SchemaEvolutionEntry wrapperSchemaEvolutionEntry);
+
+  /**
+   * @param wrapperSchemaEvolution
+   * @return
+   */
+  org.apache.carbondata.format.SchemaEvolution fromWrapperToExternalSchemaEvolution(
+      SchemaEvolution wrapperSchemaEvolution);
+
+  /**
+   * @param wrapperColumnSchema
+   * @return
+   */
+  org.apache.carbondata.format.ColumnSchema fromWrapperToExternalColumnSchema(
+      ColumnSchema wrapperColumnSchema);
+
+  /**
+   * @param wrapperTableSchema
+   * @return
+   */
+  org.apache.carbondata.format.TableSchema fromWrapperToExternalTableSchema(
+      TableSchema wrapperTableSchema);
+
+  /**
+   * @param wrapperTableInfo
+   * @param dbName
+   * @param tableName
+   * @return
+   */
+  org.apache.carbondata.format.TableInfo fromWrapperToExternalTableInfo(TableInfo wrapperTableInfo,
+      String dbName, String tableName);
+
+  /**
+   * @param externalSchemaEvolutionEntry
+   * @return
+   */
+  SchemaEvolutionEntry fromExternalToWrapperSchemaEvolutionEntry(
+      org.apache.carbondata.format.SchemaEvolutionEntry externalSchemaEvolutionEntry);
+
+  /**
+   * @param externalSchemaEvolution
+   * @return
+   */
+  SchemaEvolution fromExternalToWrapperSchemaEvolution(
+      org.apache.carbondata.format.SchemaEvolution externalSchemaEvolution);
+
+  /**
+   * @param externalColumnSchema
+   * @return
+   */
+  ColumnSchema fromExternalToWrapperColumnSchema(
+      org.apache.carbondata.format.ColumnSchema externalColumnSchema);
+
+  /**
+   * @param externalTableSchema
+   * @param tableNam
+   * @return
+   */
+  TableSchema fromExternalToWrapperTableSchema(
+      org.apache.carbondata.format.TableSchema externalTableSchema, String tableNam);
+
+  /**
+   * @param externalTableInfo
+   * @param dbName
+   * @param tableName
+   * @return
+   */
+  TableInfo fromExternalToWrapperTableInfo(org.apache.carbondata.format.TableInfo externalTableInfo,
+      String dbName, String tableName, String storePath);
+}