http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/ce09aaaf/core/src/main/java/org/apache/carbondata/core/datastore/impl/FileHolderImpl.java
----------------------------------------------------------------------
diff --git 
a/core/src/main/java/org/apache/carbondata/core/datastore/impl/FileHolderImpl.java
 
b/core/src/main/java/org/apache/carbondata/core/datastore/impl/FileHolderImpl.java
new file mode 100644
index 0000000..48928c6
--- /dev/null
+++ 
b/core/src/main/java/org/apache/carbondata/core/datastore/impl/FileHolderImpl.java
@@ -0,0 +1,198 @@
+/*
+ * 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.datastore.impl;
+
+import java.io.FileInputStream;
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.FileChannel;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Map.Entry;
+
+import org.apache.carbondata.core.constants.CarbonCommonConstants;
+import org.apache.carbondata.core.datastore.FileHolder;
+
+public class FileHolderImpl implements FileHolder {
+  /**
+   * cache to hold filename and its stream
+   */
+  private Map<String, FileChannel> fileNameAndStreamCache;
+
+  /**
+   * FileHolderImpl Constructor
+   * It will create the cache
+   */
+  public FileHolderImpl() {
+    this.fileNameAndStreamCache =
+        new HashMap<String, 
FileChannel>(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE);
+  }
+
+  public FileHolderImpl(int capacity) {
+    this.fileNameAndStreamCache = new HashMap<String, FileChannel>(capacity);
+  }
+
+  /**
+   * This method will be used to read the byte array from file based on offset
+   * and length(number of bytes) need to read
+   *
+   * @param filePath fully qualified file path
+   * @param offset   reading start position,
+   * @param length   number of bytes to be read
+   * @return read byte array
+   */
+  @Override public byte[] readByteArray(String filePath, long offset, int 
length)
+      throws IOException {
+    FileChannel fileChannel = updateCache(filePath);
+    ByteBuffer byteBffer = read(fileChannel, length, offset);
+    return byteBffer.array();
+  }
+
+  /**
+   * This method will be used to close all the streams currently present in 
the cache
+   */
+  @Override public void finish() throws IOException {
+    for (Entry<String, FileChannel> entry : fileNameAndStreamCache.entrySet()) 
{
+      FileChannel channel = entry.getValue();
+      if (null != channel) {
+        channel.close();
+      }
+    }
+  }
+
+  /**
+   * This method will be used to read int from file from postion(offset), here
+   * length will be always 4 bacause int byte size if 4
+   *
+   * @param filePath fully qualified file path
+   * @param offset   reading start position,
+   * @return read int
+   */
+  @Override public int readInt(String filePath, long offset) throws 
IOException {
+    FileChannel fileChannel = updateCache(filePath);
+    ByteBuffer byteBffer = read(fileChannel, 
CarbonCommonConstants.INT_SIZE_IN_BYTE, offset);
+    return byteBffer.getInt();
+  }
+
+  /**
+   * This method will be used to read int from file from postion(offset), here
+   * length will be always 4 bacause int byte size if 4
+   *
+   * @param filePath fully qualified file path
+   * @return read int
+   */
+  @Override public int readInt(String filePath) throws IOException {
+    FileChannel fileChannel = updateCache(filePath);
+    ByteBuffer byteBffer = read(fileChannel, 
CarbonCommonConstants.INT_SIZE_IN_BYTE);
+    return byteBffer.getInt();
+  }
+
+  /**
+   * This method will be used to read int from file from postion(offset), here
+   * length will be always 4 bacause int byte size if 4
+   *
+   * @param filePath fully qualified file path
+   * @param offset   reading start position,
+   * @return read int
+   */
+  @Override public long readDouble(String filePath, long offset) throws 
IOException {
+    FileChannel fileChannel = updateCache(filePath);
+    ByteBuffer byteBffer = read(fileChannel, 
CarbonCommonConstants.LONG_SIZE_IN_BYTE, offset);
+    return byteBffer.getLong();
+  }
+
+  /**
+   * This method will be used to check whether stream is already present in
+   * cache or not for filepath if not present then create it and then add to
+   * cache, other wise get from cache
+   *
+   * @param filePath fully qualified file path
+   * @return channel
+   */
+  private FileChannel updateCache(String filePath) throws 
FileNotFoundException {
+    FileChannel fileChannel = fileNameAndStreamCache.get(filePath);
+    if (null == fileChannel) {
+      FileInputStream stream = new FileInputStream(filePath);
+      fileChannel = stream.getChannel();
+      fileNameAndStreamCache.put(filePath, fileChannel);
+    }
+    return fileChannel;
+  }
+
+  /**
+   * This method will be used to read from file based on number of bytes to be 
read and positon
+   *
+   * @param channel file channel
+   * @param size    number of bytes
+   * @param offset  position
+   * @return byte buffer
+   */
+  private ByteBuffer read(FileChannel channel, int size, long offset) throws 
IOException {
+    ByteBuffer byteBffer = ByteBuffer.allocate(size);
+    channel.position(offset);
+    channel.read(byteBffer);
+    byteBffer.rewind();
+    return byteBffer;
+  }
+
+  /**
+   * This method will be used to read from file based on number of bytes to be 
read and positon
+   *
+   * @param channel file channel
+   * @param size    number of bytes
+   * @return byte buffer
+   */
+  private ByteBuffer read(FileChannel channel, int size) throws IOException {
+    ByteBuffer byteBffer = ByteBuffer.allocate(size);
+    channel.read(byteBffer);
+    byteBffer.rewind();
+    return byteBffer;
+  }
+
+
+  /**
+   * This method will be used to read the byte array from file based on 
length(number of bytes)
+   *
+   * @param filePath fully qualified file path
+   * @param length   number of bytes to be read
+   * @return read byte array
+   */
+  @Override public byte[] readByteArray(String filePath, int length) throws 
IOException {
+    FileChannel fileChannel = updateCache(filePath);
+    ByteBuffer byteBffer = read(fileChannel, length);
+    return byteBffer.array();
+  }
+
+  /**
+   * This method will be used to read long from file from postion(offset), here
+   * length will be always 8 bacause int byte size is 8
+   *
+   * @param filePath fully qualified file path
+   * @param offset   reading start position,
+   * @return read long
+   */
+  @Override public long readLong(String filePath, long offset) throws 
IOException {
+    FileChannel fileChannel = updateCache(filePath);
+    ByteBuffer byteBffer = read(fileChannel, 
CarbonCommonConstants.LONG_SIZE_IN_BYTE, offset);
+    return byteBffer.getLong();
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/ce09aaaf/core/src/main/java/org/apache/carbondata/core/datastore/impl/btree/AbstractBTreeBuilder.java
----------------------------------------------------------------------
diff --git 
a/core/src/main/java/org/apache/carbondata/core/datastore/impl/btree/AbstractBTreeBuilder.java
 
b/core/src/main/java/org/apache/carbondata/core/datastore/impl/btree/AbstractBTreeBuilder.java
new file mode 100644
index 0000000..6b64410
--- /dev/null
+++ 
b/core/src/main/java/org/apache/carbondata/core/datastore/impl/btree/AbstractBTreeBuilder.java
@@ -0,0 +1,164 @@
+/*
+ * 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.datastore.impl.btree;
+
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.carbondata.core.constants.CarbonCommonConstants;
+import org.apache.carbondata.core.datastore.BtreeBuilder;
+import org.apache.carbondata.core.datastore.IndexKey;
+import org.apache.carbondata.core.util.CarbonProperties;
+
+/**
+ * Abstract Btree based builder
+ */
+public abstract class AbstractBTreeBuilder implements BtreeBuilder {
+
+  /**
+   * default Number of keys per page
+   */
+  private static final int DEFAULT_NUMBER_OF_ENTRIES_NONLEAF = 32;
+
+  /**
+   * Maximum number of entries in intermediate nodes
+   */
+  protected int maxNumberOfEntriesInNonLeafNodes;
+
+  /**
+   * Number of leaf nodes
+   */
+  protected int nLeaf;
+
+  /**
+   * root node of a btree
+   */
+  protected BTreeNode root;
+
+  public AbstractBTreeBuilder() {
+    maxNumberOfEntriesInNonLeafNodes = 
Integer.parseInt(CarbonProperties.getInstance()
+        .getProperty("com.huawei.datastore.internalnodesize",
+            DEFAULT_NUMBER_OF_ENTRIES_NONLEAF + ""));
+  }
+
+  /**
+   * Below method is to build the intermediate node of the btree
+   *
+   * @param curNode              current node
+   * @param childNodeGroups      children group which will have all the 
children for
+   *                             particular intermediate node
+   * @param currentGroup         current group
+   * @param interNSKeyList       list if keys
+   * @param numberOfInternalNode number of internal node
+   */
+  protected void addIntermediateNode(BTreeNode curNode, List<BTreeNode[]> 
childNodeGroups,
+      BTreeNode[] currentGroup, List<List<IndexKey>> interNSKeyList, int 
numberOfInternalNode) {
+
+    int groupCounter;
+    // Build internal nodes level by level. Each upper node can have
+    // upperMaxEntry keys and upperMaxEntry+1 children
+    int remainder;
+    int nHigh = numberOfInternalNode;
+    boolean bRootBuilt = false;
+    remainder = nLeaf % (maxNumberOfEntriesInNonLeafNodes);
+    List<IndexKey> interNSKeys = null;
+    while (nHigh > 1 || !bRootBuilt) {
+      List<BTreeNode[]> internalNodeGroups =
+          new ArrayList<BTreeNode[]>(CarbonCommonConstants.CONSTANT_SIZE_TEN);
+      List<List<IndexKey>> interNSKeyTmpList =
+          new 
ArrayList<List<IndexKey>>(CarbonCommonConstants.CONSTANT_SIZE_TEN);
+      numberOfInternalNode = 0;
+      for (int i = 0; i < nHigh; i++) {
+        // Create a new internal node
+        curNode = new BTreeNonLeafNode();
+        // Allocate a new node group if current node group is full
+        groupCounter = i % (maxNumberOfEntriesInNonLeafNodes);
+        if (groupCounter == 0) {
+          // Create new node group
+          currentGroup = new 
BTreeNonLeafNode[maxNumberOfEntriesInNonLeafNodes];
+          internalNodeGroups.add(currentGroup);
+          numberOfInternalNode++;
+          interNSKeys = new 
ArrayList<IndexKey>(CarbonCommonConstants.CONSTANT_SIZE_TEN);
+          interNSKeyTmpList.add(interNSKeys);
+        }
+
+        // Add the new internal node to current group
+        if (null != currentGroup) {
+          currentGroup[groupCounter] = curNode;
+        }
+        int nNodes;
+
+        if (i == nHigh - 1 && remainder != 0) {
+          nNodes = remainder;
+        } else {
+          nNodes = maxNumberOfEntriesInNonLeafNodes;
+        }
+        // Point the internal node to its children node group
+        curNode.setChildren(childNodeGroups.get(i));
+        // Fill the internal node with keys based on its child nodes
+        for (int j = 0; j < nNodes; j++) {
+          curNode.setKey(interNSKeyList.get(i).get(j));
+          if (j == 0 && null != interNSKeys) {
+            interNSKeys.add(interNSKeyList.get(i).get(j));
+
+          }
+        }
+      }
+      // If nHigh is 1, we have the root node
+      if (nHigh == 1) {
+        bRootBuilt = true;
+      }
+
+      remainder = nHigh % (maxNumberOfEntriesInNonLeafNodes);
+      nHigh = numberOfInternalNode;
+      childNodeGroups = internalNodeGroups;
+      interNSKeyList = interNSKeyTmpList;
+    }
+    root = curNode;
+  }
+
+  /**
+   * Below method is to convert the start key
+   * into fixed and variable length key.
+   * data format<lenght><fixed length key><length><variable length key>
+   *
+   * @param startKey
+   * @return Index key
+   */
+  protected IndexKey convertStartKeyToNodeEntry(byte[] startKey) {
+    ByteBuffer buffer = ByteBuffer.wrap(startKey);
+    buffer.rewind();
+    int dictonaryKeySize = buffer.getInt();
+    int nonDictonaryKeySize = buffer.getInt();
+    byte[] dictionaryKey = new byte[dictonaryKeySize];
+    buffer.get(dictionaryKey);
+    byte[] nonDictionaryKey = new byte[nonDictonaryKeySize];
+    buffer.get(nonDictionaryKey);
+    return new IndexKey(dictionaryKey, nonDictionaryKey);
+  }
+
+  /**
+   * Below method will be used to get the first data block
+   * in Btree case it will be root node
+   */
+  @Override public BTreeNode get() {
+    return root;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/ce09aaaf/core/src/main/java/org/apache/carbondata/core/datastore/impl/btree/AbstractBTreeLeafNode.java
----------------------------------------------------------------------
diff --git 
a/core/src/main/java/org/apache/carbondata/core/datastore/impl/btree/AbstractBTreeLeafNode.java
 
b/core/src/main/java/org/apache/carbondata/core/datastore/impl/btree/AbstractBTreeLeafNode.java
new file mode 100644
index 0000000..b9eb5ef
--- /dev/null
+++ 
b/core/src/main/java/org/apache/carbondata/core/datastore/impl/btree/AbstractBTreeLeafNode.java
@@ -0,0 +1,244 @@
+/*
+ * 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.datastore.impl.btree;
+
+import java.io.IOException;
+
+import 
org.apache.carbondata.core.cache.update.BlockletLevelDeleteDeltaDataCache;
+import org.apache.carbondata.core.datastore.DataRefNode;
+import org.apache.carbondata.core.datastore.FileHolder;
+import org.apache.carbondata.core.datastore.IndexKey;
+import org.apache.carbondata.core.datastore.chunk.DimensionColumnDataChunk;
+import org.apache.carbondata.core.datastore.chunk.MeasureColumnDataChunk;
+
+/**
+ * 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/datastore/impl/btree/BTreeDataRefNodeFinder.java
----------------------------------------------------------------------
diff --git 
a/core/src/main/java/org/apache/carbondata/core/datastore/impl/btree/BTreeDataRefNodeFinder.java
 
b/core/src/main/java/org/apache/carbondata/core/datastore/impl/btree/BTreeDataRefNodeFinder.java
new file mode 100644
index 0000000..af61a49
--- /dev/null
+++ 
b/core/src/main/java/org/apache/carbondata/core/datastore/impl/btree/BTreeDataRefNodeFinder.java
@@ -0,0 +1,265 @@
+/*
+ * 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.datastore.impl.btree;
+
+import java.nio.ByteBuffer;
+
+import org.apache.carbondata.core.datastore.DataRefNode;
+import org.apache.carbondata.core.datastore.DataRefNodeFinder;
+import org.apache.carbondata.core.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/datastore/impl/btree/BTreeNode.java
----------------------------------------------------------------------
diff --git 
a/core/src/main/java/org/apache/carbondata/core/datastore/impl/btree/BTreeNode.java
 
b/core/src/main/java/org/apache/carbondata/core/datastore/impl/btree/BTreeNode.java
new file mode 100644
index 0000000..a32fff5
--- /dev/null
+++ 
b/core/src/main/java/org/apache/carbondata/core/datastore/impl/btree/BTreeNode.java
@@ -0,0 +1,71 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.carbondata.core.datastore.impl.btree;
+
+import org.apache.carbondata.core.datastore.DataRefNode;
+import org.apache.carbondata.core.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/datastore/impl/btree/BTreeNonLeafNode.java
----------------------------------------------------------------------
diff --git 
a/core/src/main/java/org/apache/carbondata/core/datastore/impl/btree/BTreeNonLeafNode.java
 
b/core/src/main/java/org/apache/carbondata/core/datastore/impl/btree/BTreeNonLeafNode.java
new file mode 100644
index 0000000..db7a5a5
--- /dev/null
+++ 
b/core/src/main/java/org/apache/carbondata/core/datastore/impl/btree/BTreeNonLeafNode.java
@@ -0,0 +1,249 @@
+/*
+ * 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.datastore.impl.btree;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import 
org.apache.carbondata.core.cache.update.BlockletLevelDeleteDeltaDataCache;
+import org.apache.carbondata.core.constants.CarbonCommonConstants;
+import org.apache.carbondata.core.datastore.DataRefNode;
+import org.apache.carbondata.core.datastore.FileHolder;
+import org.apache.carbondata.core.datastore.IndexKey;
+import org.apache.carbondata.core.datastore.chunk.DimensionColumnDataChunk;
+import org.apache.carbondata.core.datastore.chunk.MeasureColumnDataChunk;
+
+/**
+ * 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/datastore/impl/btree/BlockBTreeBuilder.java
----------------------------------------------------------------------
diff --git 
a/core/src/main/java/org/apache/carbondata/core/datastore/impl/btree/BlockBTreeBuilder.java
 
b/core/src/main/java/org/apache/carbondata/core/datastore/impl/btree/BlockBTreeBuilder.java
new file mode 100644
index 0000000..60fb89f
--- /dev/null
+++ 
b/core/src/main/java/org/apache/carbondata/core/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.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.constants.CarbonCommonConstants;
+import org.apache.carbondata.core.datastore.BTreeBuilderInfo;
+import org.apache.carbondata.core.datastore.IndexKey;
+
+/**
+ * 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/datastore/impl/btree/BlockBTreeLeafNode.java
----------------------------------------------------------------------
diff --git 
a/core/src/main/java/org/apache/carbondata/core/datastore/impl/btree/BlockBTreeLeafNode.java
 
b/core/src/main/java/org/apache/carbondata/core/datastore/impl/btree/BlockBTreeLeafNode.java
new file mode 100644
index 0000000..41603aa
--- /dev/null
+++ 
b/core/src/main/java/org/apache/carbondata/core/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.datastore.impl.btree;
+
+import org.apache.carbondata.core.datastore.BTreeBuilderInfo;
+import org.apache.carbondata.core.datastore.block.BlockInfo;
+import org.apache.carbondata.core.datastore.block.TableBlockInfo;
+import org.apache.carbondata.core.metadata.blocklet.DataFileFooter;
+import org.apache.carbondata.core.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/datastore/impl/btree/BlockletBTreeBuilder.java
----------------------------------------------------------------------
diff --git 
a/core/src/main/java/org/apache/carbondata/core/datastore/impl/btree/BlockletBTreeBuilder.java
 
b/core/src/main/java/org/apache/carbondata/core/datastore/impl/btree/BlockletBTreeBuilder.java
new file mode 100644
index 0000000..6a37706
--- /dev/null
+++ 
b/core/src/main/java/org/apache/carbondata/core/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.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.constants.CarbonCommonConstants;
+import org.apache.carbondata.core.datastore.BTreeBuilderInfo;
+import org.apache.carbondata.core.datastore.IndexKey;
+
+/**
+ * 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/datastore/impl/btree/BlockletBTreeLeafNode.java
----------------------------------------------------------------------
diff --git 
a/core/src/main/java/org/apache/carbondata/core/datastore/impl/btree/BlockletBTreeLeafNode.java
 
b/core/src/main/java/org/apache/carbondata/core/datastore/impl/btree/BlockletBTreeLeafNode.java
new file mode 100644
index 0000000..cfa0d2c
--- /dev/null
+++ 
b/core/src/main/java/org/apache/carbondata/core/datastore/impl/btree/BlockletBTreeLeafNode.java
@@ -0,0 +1,129 @@
+/*
+ * 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.datastore.impl.btree;
+
+import java.io.IOException;
+
+import org.apache.carbondata.core.datastore.BTreeBuilderInfo;
+import org.apache.carbondata.core.datastore.FileHolder;
+import org.apache.carbondata.core.datastore.chunk.DimensionColumnDataChunk;
+import org.apache.carbondata.core.datastore.chunk.MeasureColumnDataChunk;
+import 
org.apache.carbondata.core.datastore.chunk.reader.CarbonDataReaderFactory;
+import 
org.apache.carbondata.core.datastore.chunk.reader.DimensionColumnChunkReader;
+import 
org.apache.carbondata.core.datastore.chunk.reader.MeasureColumnChunkReader;
+import org.apache.carbondata.core.metadata.blocklet.index.BlockletMinMaxIndex;
+
+/**
+ * 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/datastore/impl/data/compressed/AbstractHeavyCompressedDoubleArrayDataStore.java
----------------------------------------------------------------------
diff --git 
a/core/src/main/java/org/apache/carbondata/core/datastore/impl/data/compressed/AbstractHeavyCompressedDoubleArrayDataStore.java
 
b/core/src/main/java/org/apache/carbondata/core/datastore/impl/data/compressed/AbstractHeavyCompressedDoubleArrayDataStore.java
new file mode 100644
index 0000000..d74cf92
--- /dev/null
+++ 
b/core/src/main/java/org/apache/carbondata/core/datastore/impl/data/compressed/AbstractHeavyCompressedDoubleArrayDataStore.java
@@ -0,0 +1,90 @@
+/*
+ * 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.datastore.impl.data.compressed;
+
+import org.apache.carbondata.common.logging.LogService;
+import org.apache.carbondata.common.logging.LogServiceFactory;
+import org.apache.carbondata.core.constants.CarbonCommonConstants;
+import org.apache.carbondata.core.datastore.NodeMeasureDataStore;
+import org.apache.carbondata.core.datastore.compression.ValueCompressionHolder;
+import org.apache.carbondata.core.datastore.compression.WriterCompressModel;
+import org.apache.carbondata.core.datastore.dataholder.CarbonWriteDataHolder;
+import org.apache.carbondata.core.util.ValueCompressionUtil;
+
+public abstract class AbstractHeavyCompressedDoubleArrayDataStore
+    implements NodeMeasureDataStore //NodeMeasureDataStore<double[]>
+{
+
+  private LogService LOGGER =
+      
LogServiceFactory.getLogService(AbstractHeavyCompressedDoubleArrayDataStore.class.getName());
+
+  /**
+   * values.
+   */
+  protected ValueCompressionHolder[] values;
+
+  /**
+   * compressionModel.
+   */
+  protected WriterCompressModel compressionModel;
+
+  /**
+   * type
+   */
+  private char[] type;
+
+  /**
+   * AbstractHeavyCompressedDoubleArrayDataStore constructor.
+   *
+   * @param compressionModel
+   */
+  public AbstractHeavyCompressedDoubleArrayDataStore(WriterCompressModel 
compressionModel) {
+    this.compressionModel = compressionModel;
+    if (null != compressionModel) {
+      this.type = compressionModel.getType();
+      values =
+          new 
ValueCompressionHolder[compressionModel.getValueCompressionHolder().length];
+    }
+  }
+
+  // this method first invokes encoding routine to encode the data chunk,
+  // followed by invoking compression routine for preparing the data chunk for 
writing.
+  @Override public byte[][] 
getWritableMeasureDataArray(CarbonWriteDataHolder[] dataHolder) {
+    byte[][] returnValue = new byte[values.length][];
+    for (int i = 0; i < compressionModel.getValueCompressionHolder().length; 
i++) {
+      values[i] = compressionModel.getValueCompressionHolder()[i];
+      if (type[i] != CarbonCommonConstants.BYTE_VALUE_MEASURE) {
+        // first perform encoding of the data chunk
+        values[i].setValue(
+            
ValueCompressionUtil.getValueCompressor(compressionModel.getCompressionFinders()[i])
+                
.getCompressedValues(compressionModel.getCompressionFinders()[i], dataHolder[i],
+                    compressionModel.getMaxValue()[i],
+                    compressionModel.getMantissa()[i]));
+      } else {
+        values[i].setValue(dataHolder[i].getWritableByteArrayValues());
+      }
+      values[i].compress();
+      returnValue[i] = values[i].getCompressedData();
+    }
+
+    return returnValue;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/ce09aaaf/core/src/main/java/org/apache/carbondata/core/datastore/impl/data/compressed/HeavyCompressedDoubleArrayDataInMemoryStore.java
----------------------------------------------------------------------
diff --git 
a/core/src/main/java/org/apache/carbondata/core/datastore/impl/data/compressed/HeavyCompressedDoubleArrayDataInMemoryStore.java
 
b/core/src/main/java/org/apache/carbondata/core/datastore/impl/data/compressed/HeavyCompressedDoubleArrayDataInMemoryStore.java
new file mode 100644
index 0000000..793fc92
--- /dev/null
+++ 
b/core/src/main/java/org/apache/carbondata/core/datastore/impl/data/compressed/HeavyCompressedDoubleArrayDataInMemoryStore.java
@@ -0,0 +1,30 @@
+/*
+ * 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.datastore.impl.data.compressed;
+
+import org.apache.carbondata.core.datastore.compression.WriterCompressModel;
+
+public class HeavyCompressedDoubleArrayDataInMemoryStore
+    extends AbstractHeavyCompressedDoubleArrayDataStore {
+
+  public HeavyCompressedDoubleArrayDataInMemoryStore(WriterCompressModel 
compressionModel) {
+    super(compressionModel);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/ce09aaaf/core/src/main/java/org/apache/carbondata/core/dictionary/client/DictionaryClient.java
----------------------------------------------------------------------
diff --git 
a/core/src/main/java/org/apache/carbondata/core/dictionary/client/DictionaryClient.java
 
b/core/src/main/java/org/apache/carbondata/core/dictionary/client/DictionaryClient.java
index 92bc56a..802def5 100644
--- 
a/core/src/main/java/org/apache/carbondata/core/dictionary/client/DictionaryClient.java
+++ 
b/core/src/main/java/org/apache/carbondata/core/dictionary/client/DictionaryClient.java
@@ -27,7 +27,9 @@ import org.apache.carbondata.common.logging.LogServiceFactory;
 import org.apache.carbondata.core.dictionary.generator.key.DictionaryKey;
 
 import org.jboss.netty.bootstrap.ClientBootstrap;
-import org.jboss.netty.channel.*;
+import org.jboss.netty.channel.ChannelPipeline;
+import org.jboss.netty.channel.ChannelPipelineFactory;
+import org.jboss.netty.channel.Channels;
 import org.jboss.netty.channel.socket.nio.NioClientSocketChannelFactory;
 import org.jboss.netty.handler.codec.serialization.ClassResolvers;
 import org.jboss.netty.handler.codec.serialization.ObjectDecoder;

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/ce09aaaf/core/src/main/java/org/apache/carbondata/core/dictionary/client/DictionaryClientHandler.java
----------------------------------------------------------------------
diff --git 
a/core/src/main/java/org/apache/carbondata/core/dictionary/client/DictionaryClientHandler.java
 
b/core/src/main/java/org/apache/carbondata/core/dictionary/client/DictionaryClientHandler.java
index d5ca781..5dba202 100644
--- 
a/core/src/main/java/org/apache/carbondata/core/dictionary/client/DictionaryClientHandler.java
+++ 
b/core/src/main/java/org/apache/carbondata/core/dictionary/client/DictionaryClientHandler.java
@@ -28,7 +28,11 @@ import 
org.apache.carbondata.common.logging.LogServiceFactory;
 import org.apache.carbondata.core.dictionary.generator.key.DictionaryKey;
 import org.apache.carbondata.core.dictionary.generator.key.KryoRegister;
 
-import org.jboss.netty.channel.*;
+import org.jboss.netty.channel.ChannelHandlerContext;
+import org.jboss.netty.channel.ChannelStateEvent;
+import org.jboss.netty.channel.ExceptionEvent;
+import org.jboss.netty.channel.MessageEvent;
+import org.jboss.netty.channel.SimpleChannelHandler;
 
 /**
  * Client handler to get data.

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/ce09aaaf/core/src/main/java/org/apache/carbondata/core/dictionary/generator/IncrementalColumnDictionaryGenerator.java
----------------------------------------------------------------------
diff --git 
a/core/src/main/java/org/apache/carbondata/core/dictionary/generator/IncrementalColumnDictionaryGenerator.java
 
b/core/src/main/java/org/apache/carbondata/core/dictionary/generator/IncrementalColumnDictionaryGenerator.java
index f157fa7..3902746 100644
--- 
a/core/src/main/java/org/apache/carbondata/core/dictionary/generator/IncrementalColumnDictionaryGenerator.java
+++ 
b/core/src/main/java/org/apache/carbondata/core/dictionary/generator/IncrementalColumnDictionaryGenerator.java
@@ -19,10 +19,11 @@
 package org.apache.carbondata.core.dictionary.generator;
 
 import java.io.IOException;
-import java.util.*;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
 import java.util.concurrent.ConcurrentHashMap;
 
-import org.apache.carbondata.common.factory.CarbonCommonFactory;
 import org.apache.carbondata.common.logging.LogService;
 import org.apache.carbondata.common.logging.LogServiceFactory;
 import org.apache.carbondata.core.cache.Cache;
@@ -30,15 +31,16 @@ import org.apache.carbondata.core.cache.CacheProvider;
 import org.apache.carbondata.core.cache.CacheType;
 import org.apache.carbondata.core.cache.dictionary.Dictionary;
 import 
org.apache.carbondata.core.cache.dictionary.DictionaryColumnUniqueIdentifier;
-import org.apache.carbondata.core.carbon.CarbonTableIdentifier;
-import org.apache.carbondata.core.carbon.ColumnIdentifier;
-import org.apache.carbondata.core.carbon.metadata.CarbonMetadata;
-import org.apache.carbondata.core.carbon.metadata.schema.table.CarbonTable;
-import 
org.apache.carbondata.core.carbon.metadata.schema.table.column.CarbonDimension;
 import org.apache.carbondata.core.constants.CarbonCommonConstants;
 import org.apache.carbondata.core.devapi.BiDictionary;
 import org.apache.carbondata.core.devapi.DictionaryGenerationException;
 import org.apache.carbondata.core.devapi.DictionaryGenerator;
+import org.apache.carbondata.core.metadata.CarbonMetadata;
+import org.apache.carbondata.core.metadata.CarbonTableIdentifier;
+import org.apache.carbondata.core.metadata.ColumnIdentifier;
+import org.apache.carbondata.core.metadata.schema.table.CarbonTable;
+import org.apache.carbondata.core.metadata.schema.table.column.CarbonDimension;
+import org.apache.carbondata.core.service.CarbonCommonFactory;
 import org.apache.carbondata.core.service.DictionaryService;
 import org.apache.carbondata.core.util.CarbonUtil;
 import org.apache.carbondata.core.util.DataTypeUtil;

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/ce09aaaf/core/src/main/java/org/apache/carbondata/core/dictionary/generator/ServerDictionaryGenerator.java
----------------------------------------------------------------------
diff --git 
a/core/src/main/java/org/apache/carbondata/core/dictionary/generator/ServerDictionaryGenerator.java
 
b/core/src/main/java/org/apache/carbondata/core/dictionary/generator/ServerDictionaryGenerator.java
index bf88bb6..0da6f94 100644
--- 
a/core/src/main/java/org/apache/carbondata/core/dictionary/generator/ServerDictionaryGenerator.java
+++ 
b/core/src/main/java/org/apache/carbondata/core/dictionary/generator/ServerDictionaryGenerator.java
@@ -21,12 +21,12 @@ package org.apache.carbondata.core.dictionary.generator;
 import java.util.Map;
 import java.util.concurrent.ConcurrentHashMap;
 
-import org.apache.carbondata.core.carbon.metadata.CarbonMetadata;
-import org.apache.carbondata.core.carbon.metadata.schema.table.CarbonTable;
-import 
org.apache.carbondata.core.carbon.metadata.schema.table.column.CarbonDimension;
 import org.apache.carbondata.core.devapi.DictionaryGenerationException;
 import org.apache.carbondata.core.devapi.DictionaryGenerator;
 import org.apache.carbondata.core.dictionary.generator.key.DictionaryKey;
+import org.apache.carbondata.core.metadata.CarbonMetadata;
+import org.apache.carbondata.core.metadata.schema.table.CarbonTable;
+import org.apache.carbondata.core.metadata.schema.table.column.CarbonDimension;
 
 /**
  * This is the dictionary generator for all tables. It generates dictionary

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/ce09aaaf/core/src/main/java/org/apache/carbondata/core/dictionary/generator/TableDictionaryGenerator.java
----------------------------------------------------------------------
diff --git 
a/core/src/main/java/org/apache/carbondata/core/dictionary/generator/TableDictionaryGenerator.java
 
b/core/src/main/java/org/apache/carbondata/core/dictionary/generator/TableDictionaryGenerator.java
index 3e8b78e..b1a01f8 100644
--- 
a/core/src/main/java/org/apache/carbondata/core/dictionary/generator/TableDictionaryGenerator.java
+++ 
b/core/src/main/java/org/apache/carbondata/core/dictionary/generator/TableDictionaryGenerator.java
@@ -30,14 +30,14 @@ import java.util.concurrent.TimeUnit;
 
 import org.apache.carbondata.common.logging.LogService;
 import org.apache.carbondata.common.logging.LogServiceFactory;
-import org.apache.carbondata.core.carbon.metadata.CarbonMetadata;
-import org.apache.carbondata.core.carbon.metadata.schema.table.CarbonTable;
-import 
org.apache.carbondata.core.carbon.metadata.schema.table.column.CarbonDimension;
 import org.apache.carbondata.core.constants.CarbonCommonConstants;
 import org.apache.carbondata.core.devapi.BiDictionary;
 import org.apache.carbondata.core.devapi.DictionaryGenerationException;
 import org.apache.carbondata.core.devapi.DictionaryGenerator;
 import org.apache.carbondata.core.dictionary.generator.key.DictionaryKey;
+import org.apache.carbondata.core.metadata.CarbonMetadata;
+import org.apache.carbondata.core.metadata.schema.table.CarbonTable;
+import org.apache.carbondata.core.metadata.schema.table.column.CarbonDimension;
 import org.apache.carbondata.core.util.CarbonProperties;
 
 /**

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/ce09aaaf/core/src/main/java/org/apache/carbondata/core/dictionary/server/DictionaryServerHandler.java
----------------------------------------------------------------------
diff --git 
a/core/src/main/java/org/apache/carbondata/core/dictionary/server/DictionaryServerHandler.java
 
b/core/src/main/java/org/apache/carbondata/core/dictionary/server/DictionaryServerHandler.java
index 9160ffe..2ad15fd 100644
--- 
a/core/src/main/java/org/apache/carbondata/core/dictionary/server/DictionaryServerHandler.java
+++ 
b/core/src/main/java/org/apache/carbondata/core/dictionary/server/DictionaryServerHandler.java
@@ -24,7 +24,11 @@ import 
org.apache.carbondata.core.dictionary.generator.ServerDictionaryGenerator
 import org.apache.carbondata.core.dictionary.generator.key.DictionaryKey;
 import org.apache.carbondata.core.dictionary.generator.key.KryoRegister;
 
-import org.jboss.netty.channel.*;
+import org.jboss.netty.channel.ChannelHandlerContext;
+import org.jboss.netty.channel.ChannelStateEvent;
+import org.jboss.netty.channel.ExceptionEvent;
+import org.jboss.netty.channel.MessageEvent;
+import org.jboss.netty.channel.SimpleChannelHandler;
 
 /**
  * Handler for Dictionary server.

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/ce09aaaf/core/src/main/java/org/apache/carbondata/core/fileoperations/AtomicFileOperations.java
----------------------------------------------------------------------
diff --git 
a/core/src/main/java/org/apache/carbondata/core/fileoperations/AtomicFileOperations.java
 
b/core/src/main/java/org/apache/carbondata/core/fileoperations/AtomicFileOperations.java
new file mode 100644
index 0000000..72028ac
--- /dev/null
+++ 
b/core/src/main/java/org/apache/carbondata/core/fileoperations/AtomicFileOperations.java
@@ -0,0 +1,33 @@
+/*
+ * 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.fileoperations;
+
+import java.io.DataInputStream;
+import java.io.DataOutputStream;
+import java.io.IOException;
+
+public interface AtomicFileOperations {
+
+  DataInputStream openForRead() throws IOException;
+
+  void close() throws IOException;
+
+  DataOutputStream openForWrite(FileWriteOperation operation) throws 
IOException;
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/ce09aaaf/core/src/main/java/org/apache/carbondata/core/fileoperations/AtomicFileOperationsImpl.java
----------------------------------------------------------------------
diff --git 
a/core/src/main/java/org/apache/carbondata/core/fileoperations/AtomicFileOperationsImpl.java
 
b/core/src/main/java/org/apache/carbondata/core/fileoperations/AtomicFileOperationsImpl.java
new file mode 100644
index 0000000..27eb27e
--- /dev/null
+++ 
b/core/src/main/java/org/apache/carbondata/core/fileoperations/AtomicFileOperationsImpl.java
@@ -0,0 +1,87 @@
+/*
+ * 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.fileoperations;
+
+import java.io.DataInputStream;
+import java.io.DataOutputStream;
+import java.io.IOException;
+
+import org.apache.carbondata.core.constants.CarbonCommonConstants;
+import org.apache.carbondata.core.datastore.filesystem.CarbonFile;
+import org.apache.carbondata.core.datastore.impl.FileFactory;
+import org.apache.carbondata.core.datastore.impl.FileFactory.FileType;
+
+public class AtomicFileOperationsImpl implements AtomicFileOperations {
+
+  private String filePath;
+
+  private FileType fileType;
+
+  private String tempWriteFilePath;
+
+  private DataOutputStream dataOutStream;
+
+  public AtomicFileOperationsImpl(String filePath, FileType fileType) {
+    this.filePath = filePath;
+
+    this.fileType = fileType;
+  }
+
+  @Override public DataInputStream openForRead() throws IOException {
+    return FileFactory.getDataInputStream(filePath, fileType);
+  }
+
+  @Override public DataOutputStream openForWrite(FileWriteOperation operation) 
throws IOException {
+
+    filePath = filePath.replace("\\", "/");
+
+    tempWriteFilePath = filePath + 
CarbonCommonConstants.TEMPWRITEFILEEXTENSION;
+
+    if (FileFactory.isFileExist(tempWriteFilePath, fileType)) {
+      FileFactory.getCarbonFile(tempWriteFilePath, fileType).delete();
+    }
+
+    FileFactory.createNewFile(tempWriteFilePath, fileType);
+
+    dataOutStream = FileFactory.getDataOutputStream(tempWriteFilePath, 
fileType);
+
+    return dataOutStream;
+
+  }
+
+  /* (non-Javadoc)
+   * @see 
com.huawei.unibi.carbon.datastorage.store.fileperations.AtomicFileOperations#close()
+   */
+  @Override public void close() throws IOException {
+
+    if (null != dataOutStream) {
+      dataOutStream.close();
+
+      CarbonFile tempFile = FileFactory.getCarbonFile(tempWriteFilePath, 
fileType);
+
+      if (!tempFile.renameForce(filePath)) {
+        throw new IOException("temporary file renaming failed, src="
+            + tempFile.getPath() + ", dest=" + filePath);
+      }
+    }
+
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/ce09aaaf/core/src/main/java/org/apache/carbondata/core/fileoperations/FileWriteOperation.java
----------------------------------------------------------------------
diff --git 
a/core/src/main/java/org/apache/carbondata/core/fileoperations/FileWriteOperation.java
 
b/core/src/main/java/org/apache/carbondata/core/fileoperations/FileWriteOperation.java
new file mode 100644
index 0000000..49f58b7
--- /dev/null
+++ 
b/core/src/main/java/org/apache/carbondata/core/fileoperations/FileWriteOperation.java
@@ -0,0 +1,25 @@
+/*
+ * 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.fileoperations;
+
+public enum FileWriteOperation {
+
+  APPEND, OVERWRITE
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/ce09aaaf/core/src/main/java/org/apache/carbondata/core/keygenerator/directdictionary/DirectDictionaryGenerator.java
----------------------------------------------------------------------
diff --git 
a/core/src/main/java/org/apache/carbondata/core/keygenerator/directdictionary/DirectDictionaryGenerator.java
 
b/core/src/main/java/org/apache/carbondata/core/keygenerator/directdictionary/DirectDictionaryGenerator.java
index b773fde..50dbbed 100644
--- 
a/core/src/main/java/org/apache/carbondata/core/keygenerator/directdictionary/DirectDictionaryGenerator.java
+++ 
b/core/src/main/java/org/apache/carbondata/core/keygenerator/directdictionary/DirectDictionaryGenerator.java
@@ -18,7 +18,7 @@
  */
 package org.apache.carbondata.core.keygenerator.directdictionary;
 
-import org.apache.carbondata.core.carbon.metadata.datatype.DataType;
+import org.apache.carbondata.core.metadata.datatype.DataType;
 
 /**
  * The interface provides the method to generate dictionary key

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/ce09aaaf/core/src/main/java/org/apache/carbondata/core/keygenerator/directdictionary/DirectDictionaryKeyGeneratorFactory.java
----------------------------------------------------------------------
diff --git 
a/core/src/main/java/org/apache/carbondata/core/keygenerator/directdictionary/DirectDictionaryKeyGeneratorFactory.java
 
b/core/src/main/java/org/apache/carbondata/core/keygenerator/directdictionary/DirectDictionaryKeyGeneratorFactory.java
index 790fa2e..679e13b 100644
--- 
a/core/src/main/java/org/apache/carbondata/core/keygenerator/directdictionary/DirectDictionaryKeyGeneratorFactory.java
+++ 
b/core/src/main/java/org/apache/carbondata/core/keygenerator/directdictionary/DirectDictionaryKeyGeneratorFactory.java
@@ -18,10 +18,10 @@
  */
 package org.apache.carbondata.core.keygenerator.directdictionary;
 
-import org.apache.carbondata.core.carbon.metadata.datatype.DataType;
 import org.apache.carbondata.core.constants.CarbonCommonConstants;
 import 
org.apache.carbondata.core.keygenerator.directdictionary.timestamp.DateDirectDictionaryGenerator;
 import 
org.apache.carbondata.core.keygenerator.directdictionary.timestamp.TimeStampDirectDictionaryGenerator;
+import org.apache.carbondata.core.metadata.datatype.DataType;
 import org.apache.carbondata.core.util.CarbonProperties;
 
 /**


Reply via email to