http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/ce09aaaf/core/src/main/java/org/apache/carbondata/core/carbon/datastore/impl/btree/AbstractBTreeLeafNode.java
----------------------------------------------------------------------
diff --git 
a/core/src/main/java/org/apache/carbondata/core/carbon/datastore/impl/btree/AbstractBTreeLeafNode.java
 
b/core/src/main/java/org/apache/carbondata/core/carbon/datastore/impl/btree/AbstractBTreeLeafNode.java
deleted file mode 100644
index eef239e..0000000
--- 
a/core/src/main/java/org/apache/carbondata/core/carbon/datastore/impl/btree/AbstractBTreeLeafNode.java
+++ /dev/null
@@ -1,244 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.carbondata.core.carbon.datastore.impl.btree;
-
-import java.io.IOException;
-
-import 
org.apache.carbondata.common.iudprocessor.cache.BlockletLevelDeleteDeltaDataCache;
-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.datastorage.store.FileHolder;
-
-/**
- * Non leaf node abstract class
- */
-public abstract class AbstractBTreeLeafNode implements BTreeNode {
-
-  /**
-   * Below method will be used to load the data block
-   *
-   * @param blockInfo block detail
-   */
-  protected BlockletLevelDeleteDeltaDataCache deleteDeltaDataCache;
-
-  /**
-   * number of keys in a btree
-   */
-  protected int numberOfKeys;
-
-  /**
-   * node number
-   */
-  protected long nodeNumber;
-
-  /**
-   * Next node of the leaf
-   */
-  protected BTreeNode nextNode;
-
-  /**
-   * max key of the column this will be used to check whether this leaf will
-   * be used for scanning or not
-   */
-  protected byte[][] maxKeyOfColumns;
-
-  /**
-   * min key of the column this will be used to check whether this leaf will
-   * be used for scanning or not
-   */
-  protected byte[][] minKeyOfColumns;
-
-  /**
-   * 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 int nodeSize() {
-    return this.numberOfKeys;
-  }
-
-  /**
-   * below method will used to set the next node
-   *
-   * @param nextNode
-   */
-  @Override public void setNextNode(BTreeNode nextNode) {
-    this.nextNode = nextNode;
-  }
-
-  /**
-   * Below method is to get the children based on index
-   *
-   * @param index children index
-   * @return btree node
-   */
-  @Override public BTreeNode getChild(int index) {
-    throw new UnsupportedOperationException("Operation not supported in case 
of leaf node");
-  }
-
-  /**
-   * below method to set the node entry
-   *
-   * @param key node entry
-   */
-  @Override public void setKey(IndexKey key) {
-    throw new UnsupportedOperationException("Operation not supported in case 
of leaf node");
-  }
-
-  /**
-   * 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() {
-    return nodeNumber;
-  }
-
-  /**
-   * This method will be used to get the max value of all the columns this can
-   * be used in case of filter query
-   *
-   */
-  @Override public byte[][] getColumnsMaxValue() {
-    return maxKeyOfColumns;
-  }
-
-  /**
-   * This method will be used to get the max value of all the columns this can
-   * be used in case of filter query
-   *
-   */
-  @Override public byte[][] getColumnsMinValue() {
-    return minKeyOfColumns;
-  }
-
-  /**
-   * to check whether node in a btree is a leaf node or not
-   *
-   * @return leaf node or not
-   */
-  @Override public boolean isLeafNode() {
-    return true;
-  }
-
-  /**
-   * 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() {
-    return nextNode;
-  }
-
-  /**
-   * below method will return the one node indexes
-   *
-   * @return node entry array
-   */
-  @Override public IndexKey[] getNodeKeys() {
-    // as this is a leaf node so this method implementation is not required
-    throw new UnsupportedOperationException("Operation not supported in case 
of leaf node");
-  }
-
-  /**
-   * below method will be used to set the children of intermediate node
-   *
-   * @param children array
-   */
-  @Override public void setChildren(BTreeNode[] children) {
-    // no required in case of leaf node as leaf node will not have any children
-    throw new UnsupportedOperationException("Operation not supported in case 
of leaf node");
-  }
-
-  /**
-   * 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) throws IOException {
-    // No required here as leaf which will will be use this class will 
implement its own get
-    // dimension chunks
-    return null;
-  }
-
-  /**
-   * 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) throws IOException {
-    // No required here as leaf which will will be use this class will 
implement
-    // its own get dimension chunks
-    return null;
-  }
-
-  /**
-   * 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) throws IOException {
-    // No required here as leaf which will will be use this class will 
implement its own get
-    // measure chunks
-    return null;
-  }
-
-  /**
-   * 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)
-      throws IOException {
-    // No required here as leaf which will will be use this class will 
implement its own get
-    // measure chunks
-    return null;
-  }
-
-  /**
-   * @param deleteDeltaDataCache
-   */
-  public void setDeleteDeltaDataCache(BlockletLevelDeleteDeltaDataCache 
deleteDeltaDataCache) {
-
-    this.deleteDeltaDataCache = deleteDeltaDataCache;
-  }
-  /**
-   * @return the segmentProperties
-   */
-  public BlockletLevelDeleteDeltaDataCache getDeleteDeltaDataCache() {
-    return deleteDeltaDataCache;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/ce09aaaf/core/src/main/java/org/apache/carbondata/core/carbon/datastore/impl/btree/BTreeDataRefNodeFinder.java
----------------------------------------------------------------------
diff --git 
a/core/src/main/java/org/apache/carbondata/core/carbon/datastore/impl/btree/BTreeDataRefNodeFinder.java
 
b/core/src/main/java/org/apache/carbondata/core/carbon/datastore/impl/btree/BTreeDataRefNodeFinder.java
deleted file mode 100644
index 911b643..0000000
--- 
a/core/src/main/java/org/apache/carbondata/core/carbon/datastore/impl/btree/BTreeDataRefNodeFinder.java
+++ /dev/null
@@ -1,265 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.carbondata.core.carbon.datastore.impl.btree;
-
-import java.nio.ByteBuffer;
-
-import org.apache.carbondata.core.carbon.datastore.DataRefNode;
-import org.apache.carbondata.core.carbon.datastore.DataRefNodeFinder;
-import org.apache.carbondata.core.carbon.datastore.IndexKey;
-import org.apache.carbondata.core.util.ByteUtil;
-
-/**
- * Below class will be used to find a block in a btree
- */
-public class BTreeDataRefNodeFinder implements DataRefNodeFinder {
-
-  /**
-   * no dictionary column value is of variable length so in each column value
-   * it will -1
-   */
-  private static final int NO_DCITIONARY_COLUMN_VALUE = -1;
-
-  /**
-   * sized of the short value in bytes
-   */
-  private static final short SHORT_SIZE_IN_BYTES = 2;
-  /**
-   * this will holds the information about the size of each value of a column,
-   * this will be used during Comparison of the btree node value and the
-   * search value if value is more than zero then its a fixed length column
-   * else its variable length column. So as data of both type of column store
-   * separately so this value size array will be used for both purpose
-   * comparison and jumping(which type value we need to compare)
-   */
-  private int[] eachColumnValueSize;
-
-  /**
-   * this will be used during search for no dictionary column
-   */
-  private int numberOfNoDictionaryColumns;
-
-  public BTreeDataRefNodeFinder(int[] eachColumnValueSize) {
-    this.eachColumnValueSize = eachColumnValueSize;
-
-    for (int i = 0; i < eachColumnValueSize.length; i++) {
-      if (eachColumnValueSize[i] == -1) {
-        numberOfNoDictionaryColumns++;
-      }
-    }
-  }
-
-  /**
-   * Below method will be used to get the first tentative data block based on
-   * search key
-   *
-   * @return data block
-   */
-  @Override public DataRefNode findFirstDataBlock(DataRefNode dataRefBlock, 
IndexKey searchKey) {
-    // as its for btree type cast it to btree interface
-    BTreeNode rootNode = (BTreeNode) dataRefBlock;
-    while (!rootNode.isLeafNode()) {
-      rootNode = findFirstLeafNode(searchKey, rootNode);
-    }
-    return rootNode;
-  }
-
-  /**
-   * Below method will be used to get the last data tentative block based on
-   * search key
-   *
-   * @return data block
-   */
-  @Override public DataRefNode findLastDataBlock(DataRefNode dataRefBlock, 
IndexKey searchKey) {
-    // as its for btree type cast it to btree interface
-    BTreeNode rootNode = (BTreeNode) dataRefBlock;
-    while (!rootNode.isLeafNode()) {
-      rootNode = findLastLeafNode(searchKey, rootNode);
-    }
-    return rootNode;
-  }
-
-  /**
-   * Binary search used to get the first tentative block of the btree based on
-   * search key
-   *
-   * @param key  search key
-   * @param node root node of btree
-   * @return first tentative block
-   */
-  private BTreeNode findFirstLeafNode(IndexKey key, BTreeNode node) {
-    int childNodeIndex;
-    int low = 0;
-    int high = node.nodeSize() - 1;
-    int mid = 0;
-    int compareRes = -1;
-    IndexKey[] nodeKeys = node.getNodeKeys();
-    //
-    while (low <= high) {
-      mid = (low + high) >>> 1;
-      // compare the entries
-      compareRes = compareIndexes(key, nodeKeys[mid]);
-      if (compareRes < 0) {
-        high = mid - 1;
-      } else if (compareRes > 0) {
-        low = mid + 1;
-      } else {
-        // if key is matched then get the first entry
-        int currentPos = mid;
-        while (currentPos - 1 >= 0 && compareIndexes(key, nodeKeys[currentPos 
- 1]) == 0) {
-          currentPos--;
-        }
-        mid = currentPos;
-        break;
-      }
-    }
-    // if compare result is less than zero then we
-    // and mid is more than 0 then we need to previous block as duplicates
-    // record can be present
-    if (compareRes < 0) {
-      if (mid > 0) {
-        mid--;
-      }
-      childNodeIndex = mid;
-    } else {
-      childNodeIndex = mid;
-    }
-    // get the leaf child
-    node = node.getChild(childNodeIndex);
-    return node;
-  }
-
-  /**
-   * Binary search used to get the last tentative block of the btree based on
-   * search key
-   *
-   * @param key  search key
-   * @param node root node of btree
-   * @return first tentative block
-   */
-  private BTreeNode findLastLeafNode(IndexKey key, BTreeNode node) {
-    int childNodeIndex;
-    int low = 0;
-    int high = node.nodeSize() - 1;
-    int mid = 0;
-    int compareRes = -1;
-    IndexKey[] nodeKeys = node.getNodeKeys();
-    //
-    while (low <= high) {
-      mid = (low + high) >>> 1;
-      // compare the entries
-      compareRes = compareIndexes(key, nodeKeys[mid]);
-      if (compareRes < 0) {
-        high = mid - 1;
-      } else if (compareRes > 0) {
-        low = mid + 1;
-      } else {
-        int currentPos = mid;
-        // if key is matched then get the first entry
-        while (currentPos + 1 < node.nodeSize()
-            && compareIndexes(key, nodeKeys[currentPos + 1]) == 0) {
-          currentPos++;
-        }
-        mid = currentPos;
-        break;
-      }
-    }
-    // if compare result is less than zero then we
-    // and mid is more than 0 then we need to previous block as duplicates
-    // record can be present
-    if (compareRes < 0) {
-      if (mid > 0) {
-        mid--;
-      }
-      childNodeIndex = mid;
-    } else {
-      childNodeIndex = mid;
-    }
-    node = node.getChild(childNodeIndex);
-    return node;
-  }
-
-  /**
-   * Comparison of index key will be following format of key <Dictionary> key
-   * will be in byte array No dictionary key Index of FirstKey (2
-   * bytes)><Index of SecondKey (2 bytes)><Index of NKey (2 bytes)> <First Key
-   * ByteArray><2nd Key ByteArray><N Key ByteArray> in each column value size
-   * of no dictionary column will be -1 if in each column value is not -1 then
-   * compare the byte array based on size and increment the offset to
-   * dictionary column size if size is -1 then its a no dictionary key so to
-   * get the length subtract the size of current with next key offset it will
-   * give the actual length if it is at last position or only one key is
-   * present then subtract with length
-   *
-   * @param first  key
-   * @param second key
-   * @return comparison value
-   */
-  private int compareIndexes(IndexKey first, IndexKey second) {
-    int dictionaryKeyOffset = 0;
-    int nonDictionaryKeyOffset = 0;
-    int compareResult = 0;
-    int processedNoDictionaryColumn = numberOfNoDictionaryColumns;
-    ByteBuffer firstNoDictionaryKeyBuffer = 
ByteBuffer.wrap(first.getNoDictionaryKeys());
-    ByteBuffer secondNoDictionaryKeyBuffer = 
ByteBuffer.wrap(second.getNoDictionaryKeys());
-    int actualOffset = 0;
-    int actualOffset1 = 0;
-    int firstNoDcitionaryLength = 0;
-    int secondNodeDictionaryLength = 0;
-
-    for (int i = 0; i < eachColumnValueSize.length; i++) {
-
-      if (eachColumnValueSize[i] != NO_DCITIONARY_COLUMN_VALUE) {
-        compareResult = ByteUtil.UnsafeComparer.INSTANCE
-            .compareTo(first.getDictionaryKeys(), dictionaryKeyOffset, 
eachColumnValueSize[i],
-                second.getDictionaryKeys(), dictionaryKeyOffset, 
eachColumnValueSize[i]);
-        dictionaryKeyOffset += eachColumnValueSize[i];
-      } else {
-        if (processedNoDictionaryColumn > 1) {
-          actualOffset = 
firstNoDictionaryKeyBuffer.getShort(nonDictionaryKeyOffset);
-          firstNoDcitionaryLength =
-              firstNoDictionaryKeyBuffer.getShort(nonDictionaryKeyOffset + 
SHORT_SIZE_IN_BYTES)
-                      - actualOffset;
-          actualOffset1 = 
secondNoDictionaryKeyBuffer.getShort(nonDictionaryKeyOffset);
-          secondNodeDictionaryLength =
-              secondNoDictionaryKeyBuffer.getShort(nonDictionaryKeyOffset + 
SHORT_SIZE_IN_BYTES)
-                      - actualOffset1;
-          compareResult = ByteUtil.UnsafeComparer.INSTANCE
-                  .compareTo(first.getNoDictionaryKeys(), actualOffset, 
firstNoDcitionaryLength,
-                          second.getNoDictionaryKeys(), actualOffset1, 
secondNodeDictionaryLength);
-          nonDictionaryKeyOffset += SHORT_SIZE_IN_BYTES;
-          processedNoDictionaryColumn--;
-        } else {
-          actualOffset = 
firstNoDictionaryKeyBuffer.getShort(nonDictionaryKeyOffset);
-          actualOffset1 = 
secondNoDictionaryKeyBuffer.getShort(nonDictionaryKeyOffset);
-          firstNoDcitionaryLength = first.getNoDictionaryKeys().length - 
actualOffset;
-          secondNodeDictionaryLength = second.getNoDictionaryKeys().length - 
actualOffset1;
-          compareResult = ByteUtil.UnsafeComparer.INSTANCE
-              .compareTo(first.getNoDictionaryKeys(), actualOffset, 
firstNoDcitionaryLength,
-                  second.getNoDictionaryKeys(), actualOffset1, 
secondNodeDictionaryLength);
-        }
-      }
-      if (compareResult != 0) {
-        return compareResult;
-      }
-    }
-
-    return 0;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/ce09aaaf/core/src/main/java/org/apache/carbondata/core/carbon/datastore/impl/btree/BTreeNode.java
----------------------------------------------------------------------
diff --git 
a/core/src/main/java/org/apache/carbondata/core/carbon/datastore/impl/btree/BTreeNode.java
 
b/core/src/main/java/org/apache/carbondata/core/carbon/datastore/impl/btree/BTreeNode.java
deleted file mode 100644
index 6b624c6..0000000
--- 
a/core/src/main/java/org/apache/carbondata/core/carbon/datastore/impl/btree/BTreeNode.java
+++ /dev/null
@@ -1,71 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.carbondata.core.carbon.datastore.impl.btree;
-
-import org.apache.carbondata.core.carbon.datastore.DataRefNode;
-import org.apache.carbondata.core.carbon.datastore.IndexKey;
-
-/**
- * Interface for btree node
- */
-public interface BTreeNode extends DataRefNode {
-
-  /**
-   * below method will return the one node indexes
-   *
-   * @return node entry array
-   */
-  IndexKey[] getNodeKeys();
-
-  /**
-   * to check whether node in a btree is a leaf node or not
-   *
-   * @return leaf node or not
-   */
-  boolean isLeafNode();
-
-  /**
-   * below method will be used to set the children of intermediate node
-   *
-   * @param children array
-   */
-  void setChildren(BTreeNode[] children);
-
-  /**
-   * below method will used to set the next node
-   *
-   * @param nextNode
-   */
-  void setNextNode(BTreeNode nextNode);
-
-  /**
-   * Below method is to get the children based on index
-   *
-   * @param index children index
-   * @return btree node
-   */
-  BTreeNode getChild(int index);
-
-  /**
-   * below method to set the node entry
-   *
-   * @param key node entry
-   */
-  void setKey(IndexKey key);
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/ce09aaaf/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
deleted file mode 100644
index 4449581..0000000
--- 
a/core/src/main/java/org/apache/carbondata/core/carbon/datastore/impl/btree/BTreeNonLeafNode.java
+++ /dev/null
@@ -1,249 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.carbondata.core.carbon.datastore.impl.btree;
-
-import java.util.ArrayList;
-import java.util.List;
-
-import 
org.apache.carbondata.common.iudprocessor.cache.BlockletLevelDeleteDeltaDataCache;
-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 {
-
-  /**
-   * Below method will be used to load the data block
-   *
-   * @param blockInfo block detail
-   */
-  protected BlockletLevelDeleteDeltaDataCache deleteDeltaDataCache;
-
-  /**
-   * 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
-   *
-   */
-  @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
-   *
-   */
-  @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
-   *
-   */
-  @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
-   *
-   */
-  @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
-   * @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");
-  }
-
-  /**
-   * @return the segmentProperties
-   */
-  public void setDeleteDeltaDataCache(BlockletLevelDeleteDeltaDataCache 
deleteDeltaDataCache) {
-
-    this.deleteDeltaDataCache = deleteDeltaDataCache;
-  }
-  /**
-   * @return the segmentProperties
-   */
-  public BlockletLevelDeleteDeltaDataCache getDeleteDeltaDataCache() {
-    return deleteDeltaDataCache;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/ce09aaaf/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
deleted file mode 100644
index 3828818..0000000
--- 
a/core/src/main/java/org/apache/carbondata/core/carbon/datastore/impl/btree/BlockBTreeBuilder.java
+++ /dev/null
@@ -1,118 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.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/ce09aaaf/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
deleted file mode 100644
index 494a2a3..0000000
--- 
a/core/src/main/java/org/apache/carbondata/core/carbon/datastore/impl/btree/BlockBTreeLeafNode.java
+++ /dev/null
@@ -1,64 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.carbondata.core.carbon.datastore.impl.btree;
-
-import org.apache.carbondata.core.carbon.datastore.BTreeBuilderInfo;
-import org.apache.carbondata.core.carbon.datastore.block.BlockInfo;
-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 BlockInfo blockInfo;
-
-  /**
-   * Create a leaf node
-   *
-   * @param builderInfos  builder infos which have required metadata to create 
a leaf
-   *                      node
-   * @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 = (int)footer.getNumberOfRows();
-    this.nodeNumber = nodeNumber;
-    this.blockInfo = footer.getBlockInfo();
-  }
-
-  /**
-   * 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.getTableBlockInfo();
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/ce09aaaf/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
deleted file mode 100644
index 3d6c11d..0000000
--- 
a/core/src/main/java/org/apache/carbondata/core/carbon/datastore/impl/btree/BlockletBTreeBuilder.java
+++ /dev/null
@@ -1,104 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.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/ce09aaaf/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
deleted file mode 100644
index e98eae1..0000000
--- 
a/core/src/main/java/org/apache/carbondata/core/carbon/datastore/impl/btree/BlockletBTreeLeafNode.java
+++ /dev/null
@@ -1,129 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.carbondata.core.carbon.datastore.impl.btree;
-
-import java.io.IOException;
-
-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.CarbonDataReaderFactory;
-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.metadata.blocklet.index.BlockletMinMaxIndex;
-import org.apache.carbondata.core.datastorage.store.FileHolder;
-
-/**
- * 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 = CarbonDataReaderFactory.getInstance()
-        
.getDimensionColumnChunkReader(builderInfos.getFooterList().get(0).getVersionId(),
-            
builderInfos.getFooterList().get(0).getBlockletList().get(leafIndex),
-            builderInfos.getDimensionColumnValueSize(),
-            
builderInfos.getFooterList().get(0).getBlockInfo().getTableBlockInfo().getFilePath());
-    // create a instance of measure column chunk reader
-    measureColumnChunkReader = CarbonDataReaderFactory.getInstance()
-        
.getMeasureColumnChunkReader(builderInfos.getFooterList().get(0).getVersionId(),
-            
builderInfos.getFooterList().get(0).getBlockletList().get(leafIndex),
-            
builderInfos.getFooterList().get(0).getBlockInfo().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) throws IOException {
-    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) throws IOException {
-    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) throws IOException {
-    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)
-      throws IOException {
-    return measureColumnChunkReader.readMeasureChunk(fileReader, blockIndex);
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/ce09aaaf/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
deleted file mode 100644
index ce5e457..0000000
--- 
a/core/src/main/java/org/apache/carbondata/core/carbon/metadata/CarbonMetadata.java
+++ /dev/null
@@ -1,165 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.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/ce09aaaf/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
deleted file mode 100644
index 314f7e2..0000000
--- 
a/core/src/main/java/org/apache/carbondata/core/carbon/metadata/blocklet/BlockletInfo.java
+++ /dev/null
@@ -1,154 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-package org.apache.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;
-
-  private List<Long> dimensionChunkOffsets;
-
-  private List<Short> dimensionChunksLength;
-
-  private List<Long> measureChunkOffsets;
-
-  private List<Short> measureChunksLength;
-
-  /**
-   * 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;
-  }
-
-  public List<Long> getDimensionChunkOffsets() {
-    return dimensionChunkOffsets;
-  }
-
-  public void setDimensionChunkOffsets(List<Long> dimensionChunkOffsets) {
-    this.dimensionChunkOffsets = dimensionChunkOffsets;
-  }
-
-  public List<Short> getDimensionChunksLength() {
-    return dimensionChunksLength;
-  }
-
-  public void setDimensionChunksLength(List<Short> dimensionChunksLength) {
-    this.dimensionChunksLength = dimensionChunksLength;
-  }
-
-  public List<Long> getMeasureChunkOffsets() {
-    return measureChunkOffsets;
-  }
-
-  public void setMeasureChunkOffsets(List<Long> measureChunkOffsets) {
-    this.measureChunkOffsets = measureChunkOffsets;
-  }
-
-  public List<Short> getMeasureChunksLength() {
-    return measureChunksLength;
-  }
-
-  public void setMeasureChunksLength(List<Short> measureChunksLength) {
-    this.measureChunksLength = measureChunksLength;
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/ce09aaaf/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
deleted file mode 100644
index a82bac9..0000000
--- 
a/core/src/main/java/org/apache/carbondata/core/carbon/metadata/blocklet/DataFileFooter.java
+++ /dev/null
@@ -1,171 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.carbondata.core.carbon.metadata.blocklet;
-
-import java.io.Serializable;
-import java.util.List;
-
-import org.apache.carbondata.core.carbon.ColumnarFormatVersion;
-import org.apache.carbondata.core.carbon.datastore.block.BlockInfo;
-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 ColumnarFormatVersion 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 BlockInfo blockInfo;
-
-  /**
-   * @return the versionId
-   */
-  public ColumnarFormatVersion getVersionId() {
-    return versionId;
-  }
-
-  /**
-   * @param versionId the versionId to set
-   */
-  public void setVersionId(ColumnarFormatVersion 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 BlockInfo getBlockInfo() {
-    return blockInfo;
-  }
-
-  /**
-   * @param tableBlockInfo the tableBlockInfo to set
-   */
-  public void setBlockInfo(BlockInfo tableBlockInfo) {
-    this.blockInfo = tableBlockInfo;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/ce09aaaf/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
deleted file mode 100644
index c0d6b55..0000000
--- 
a/core/src/main/java/org/apache/carbondata/core/carbon/metadata/blocklet/SegmentInfo.java
+++ /dev/null
@@ -1,72 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.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/ce09aaaf/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
deleted file mode 100644
index e977aca..0000000
--- 
a/core/src/main/java/org/apache/carbondata/core/carbon/metadata/blocklet/compressor/ChunkCompressorMeta.java
+++ /dev/null
@@ -1,36 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-package org.apache.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;
-
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/ce09aaaf/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
deleted file mode 100644
index f301fc6..0000000
--- 
a/core/src/main/java/org/apache/carbondata/core/carbon/metadata/blocklet/datachunk/DataChunk.java
+++ /dev/null
@@ -1,228 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.carbondata.core.carbon.metadata.blocklet.datachunk;
-
-import java.io.Serializable;
-import java.util.List;
-
-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;
-
-  /**
-   * whether this chunk is a row chunk or column chunk
-   */
-  private boolean isRowMajor;
-
-  /**
-   * 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;
-
-  /**
-   * 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 isRowMajor
-   */
-  public boolean isRowMajor() {
-    return isRowMajor;
-  }
-
-  /**
-   * @param isRowMajor the isRowMajor to set
-   */
-  public void setRowMajor(boolean isRowMajor) {
-    this.isRowMajor = isRowMajor;
-  }
-
-  /**
-   * @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 encoderList
-   */
-  public List<Encoding> getEncodingList() {
-    return encodingList;
-  }
-
-  /**
-   * @param encodingList the encoderList to set
-   */
-  public void setEncodingList(List<Encoding> encodingList) {
-    this.encodingList = encodingList;
-  }
-
-  /**
-   * @return the valueEncoderMeta
-   */
-  public List<ValueEncoderMeta> getValueEncoderMeta() {
-    return valueEncoderMetaList;
-  }
-
-  /**
-   * @param valueEncoderMetaList the valueEncoderMeta to set
-   */
-  public void setValueEncoderMeta(List<ValueEncoderMeta> valueEncoderMetaList) 
{
-    this.valueEncoderMetaList = valueEncoderMetaList;
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/ce09aaaf/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
deleted file mode 100644
index f73d3d0..0000000
--- 
a/core/src/main/java/org/apache/carbondata/core/carbon/metadata/blocklet/datachunk/PresenceMeta.java
+++ /dev/null
@@ -1,66 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-package org.apache.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/ce09aaaf/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
deleted file mode 100644
index 680b4eb..0000000
--- 
a/core/src/main/java/org/apache/carbondata/core/carbon/metadata/blocklet/index/BlockletBTreeIndex.java
+++ /dev/null
@@ -1,76 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * http://www.apache.org/licenses/LICENSE-2.0
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.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/ce09aaaf/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
deleted file mode 100644
index b78d2b9..0000000
--- 
a/core/src/main/java/org/apache/carbondata/core/carbon/metadata/blocklet/index/BlockletIndex.java
+++ /dev/null
@@ -1,77 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * http://www.apache.org/licenses/LICENSE-2.0
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.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/ce09aaaf/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
deleted file mode 100644
index 928884f..0000000
--- 
a/core/src/main/java/org/apache/carbondata/core/carbon/metadata/blocklet/index/BlockletMinMaxIndex.java
+++ /dev/null
@@ -1,83 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * http://www.apache.org/licenses/LICENSE-2.0
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.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;
-  }
-
-}

Reply via email to