http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/ce09aaaf/core/src/main/java/org/apache/carbondata/core/carbon/datastore/block/AbstractIndex.java
----------------------------------------------------------------------
diff --git 
a/core/src/main/java/org/apache/carbondata/core/carbon/datastore/block/AbstractIndex.java
 
b/core/src/main/java/org/apache/carbondata/core/carbon/datastore/block/AbstractIndex.java
deleted file mode 100644
index 4dd76c6..0000000
--- 
a/core/src/main/java/org/apache/carbondata/core/carbon/datastore/block/AbstractIndex.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.block;
-
-import java.util.List;
-import java.util.concurrent.atomic.AtomicInteger;
-
-import org.apache.carbondata.core.cache.Cacheable;
-import org.apache.carbondata.core.carbon.datastore.DataRefNode;
-import org.apache.carbondata.core.carbon.metadata.blocklet.DataFileFooter;
-
-public abstract class AbstractIndex implements Cacheable {
-
-  /**
-   * vo class which will hold the RS information of the block
-   */
-  protected SegmentProperties segmentProperties;
-
-  /**
-   * data block
-   */
-  protected DataRefNode dataRefNode;
-
-  /**
-   * total number of row present in the block
-   */
-  protected long totalNumberOfRows;
-
-  /**
-   * atomic integer to maintain the access count for a column access
-   */
-  protected AtomicInteger accessCount = new AtomicInteger();
-
-  /**
-   * Table block meta size.
-   */
-  protected long memorySize;
-
-  /**
-   * @return the segmentProperties
-   */
-  public SegmentProperties getSegmentProperties() {
-    return segmentProperties;
-  }
-
-  /**
-   * @return the dataBlock
-   */
-  public DataRefNode getDataRefNode() {
-    return dataRefNode;
-  }
-
-  @Override public long getFileTimeStamp() {
-    return 0;
-  }
-
-  /**
-   * Below method will be used to load the data block
-   *
-   * @param footerList footer list
-   */
-  public abstract void buildIndex(List<DataFileFooter> footerList);
-
-  /**
-   * the method returns the access count
-   *
-   * @return
-   */
-  @Override public int getAccessCount() {
-    return accessCount.get();
-  }
-
-  /**
-   * The method returns table block size
-   *
-   * @return
-   */
-  @Override public long getMemorySize() {
-    return this.memorySize;
-  }
-
-  /**
-   * The method is used to set the access count
-   */
-  public void incrementAccessCount() {
-    accessCount.incrementAndGet();
-  }
-
-  /**
-   * This method will release the objects and set default value for primitive 
types
-   */
-  public void clear() {
-    decrementAccessCount();
-  }
-
-  /**
-   * This method will decrement the access count for a column by 1
-   * whenever a column usage is complete
-   */
-  private void decrementAccessCount() {
-    if (accessCount.get() > 0) {
-      accessCount.decrementAndGet();
-    }
-  }
-
-  /**
-   * the method is used to set the memory size of the b-tree
-   * @param memorySize
-   */
-  public void setMemorySize(long memorySize) {
-    this.memorySize = memorySize;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/ce09aaaf/core/src/main/java/org/apache/carbondata/core/carbon/datastore/block/BlockIndex.java
----------------------------------------------------------------------
diff --git 
a/core/src/main/java/org/apache/carbondata/core/carbon/datastore/block/BlockIndex.java
 
b/core/src/main/java/org/apache/carbondata/core/carbon/datastore/block/BlockIndex.java
deleted file mode 100644
index 75ebb84..0000000
--- 
a/core/src/main/java/org/apache/carbondata/core/carbon/datastore/block/BlockIndex.java
+++ /dev/null
@@ -1,52 +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.block;
-
-import java.util.List;
-
-import org.apache.carbondata.core.carbon.datastore.BTreeBuilderInfo;
-import org.apache.carbondata.core.carbon.datastore.BtreeBuilder;
-import 
org.apache.carbondata.core.carbon.datastore.impl.btree.BlockletBTreeBuilder;
-import org.apache.carbondata.core.carbon.metadata.blocklet.DataFileFooter;
-
-/**
- * Class which is responsible for loading the b+ tree block. This class will
- * persist all the detail of a table block
- */
-public class BlockIndex extends AbstractIndex {
-
-  /**
-   * Below method will be used to load the data block
-   *
-   */
-  public void buildIndex(List<DataFileFooter> footerList) {
-    // create a metadata details
-    // this will be useful in query handling
-    segmentProperties = new 
SegmentProperties(footerList.get(0).getColumnInTable(),
-        footerList.get(0).getSegmentInfo().getColumnCardinality());
-    // create a segment builder info
-    BTreeBuilderInfo indexBuilderInfo =
-        new BTreeBuilderInfo(footerList, 
segmentProperties.getDimensionColumnsValueSize());
-    BtreeBuilder blocksBuilder = new BlockletBTreeBuilder();
-    // load the metadata
-    blocksBuilder.build(indexBuilderInfo);
-    dataRefNode = blocksBuilder.get();
-    totalNumberOfRows = footerList.get(0).getNumberOfRows();
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/ce09aaaf/core/src/main/java/org/apache/carbondata/core/carbon/datastore/block/BlockInfo.java
----------------------------------------------------------------------
diff --git 
a/core/src/main/java/org/apache/carbondata/core/carbon/datastore/block/BlockInfo.java
 
b/core/src/main/java/org/apache/carbondata/core/carbon/datastore/block/BlockInfo.java
deleted file mode 100644
index 2a49daf..0000000
--- 
a/core/src/main/java/org/apache/carbondata/core/carbon/datastore/block/BlockInfo.java
+++ /dev/null
@@ -1,134 +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.block;
-
-import org.apache.carbondata.core.carbon.path.CarbonTablePath;
-import org.apache.carbondata.core.constants.CarbonCommonConstants;
-
-/**
- * Below class will be used to store table block info
- * As in blocklet distribution we are dividing the same block
- * in parts but in case of block loading blocklets belongs to same
- * block will be loaded together. This class will be used to store table block 
info
- * and equals and hash code method is used to identify blocklet belongs to 
same block
- */
-public class BlockInfo {
-
-  /**
-   * table block info, stores all the details
-   * about the block
-   */
-  private TableBlockInfo info;
-  /**
-   * unique blockName
-   */
-  private String blockUniqueName;
-
-  /**
-   * Constructor
-   *
-   * @param info
-   */
-  public BlockInfo(TableBlockInfo info) {
-    this.info = info;
-    init();
-  }
-
-  /**
-   * init the block unique name
-   */
-  private void init() {
-    StringBuilder stringBuilder = new StringBuilder();
-    stringBuilder.append(this.info.getSegmentId());
-    stringBuilder.append(CarbonCommonConstants.FILE_SEPARATOR);
-    
stringBuilder.append(CarbonTablePath.getCarbonDataFileName(this.info.getFilePath()));
-    this.blockUniqueName = stringBuilder.toString();
-  }
-
-  /**
-   * @return table Block info
-   */
-  public TableBlockInfo getTableBlockInfo() {
-    return info;
-  }
-
-  /**
-   * To set the table block info
-   *
-   * @param info
-   */
-  public void setTableBlockInfo(TableBlockInfo info) {
-    this.info = info;
-  }
-
-  /**
-   * method to get the hash code
-   */
-  @Override public int hashCode() {
-    int result = info.getFilePath().hashCode();
-    result = 31 * result + (int) (info.getBlockOffset() ^ 
(info.getBlockOffset() >>> 32));
-    result = 31 * result + (int) (info.getBlockLength() ^ 
(info.getBlockLength() >>> 32));
-    result = 31 * result + info.getSegmentId().hashCode();
-    return result;
-  }
-
-  /**
-   * To check the equality
-   *
-   * @param obj
-   */
-  @Override public boolean equals(Object obj) {
-    if (this == obj) {
-      return true;
-    }
-    if (obj == null) {
-      return false;
-    }
-    if (!(obj instanceof BlockInfo)) {
-      return false;
-    }
-    BlockInfo other = (BlockInfo) obj;
-    if (!info.getSegmentId().equals(other.info.getSegmentId())) {
-      return false;
-    }
-    if (info.getBlockOffset() != other.info.getBlockOffset()) {
-      return false;
-    }
-    if (info.getBlockLength() != other.info.getBlockLength()) {
-      return false;
-    }
-
-    if (info.getFilePath() == null && other.info.getFilePath() != null) {
-      return false;
-    } else if (info.getFilePath() != null && other.info.getFilePath() == null) 
{
-      return false;
-    } else if (!info.getFilePath().equals(other.info.getFilePath())) {
-      return false;
-    }
-    return true;
-  }
-
-  /**
-   * returns unique blockname
-   * @return
-   */
-  public String getBlockUniqueName() {
-    return blockUniqueName;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/ce09aaaf/core/src/main/java/org/apache/carbondata/core/carbon/datastore/block/BlockletInfos.java
----------------------------------------------------------------------
diff --git 
a/core/src/main/java/org/apache/carbondata/core/carbon/datastore/block/BlockletInfos.java
 
b/core/src/main/java/org/apache/carbondata/core/carbon/datastore/block/BlockletInfos.java
deleted file mode 100644
index 4251888..0000000
--- 
a/core/src/main/java/org/apache/carbondata/core/carbon/datastore/block/BlockletInfos.java
+++ /dev/null
@@ -1,112 +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.block;
-
-import java.io.Serializable;
-
-/**
- * The class holds the blocks blocklets info
- */
-public class BlockletInfos implements Serializable {
-  /**
-   * no of blockLets
-   */
-  private int noOfBlockLets = 0;
-
-  /**
-   * start blocklet number
-   */
-  private int startBlockletNumber;
-  /**
-   * end blocklet number
-   */
-  private int numberOfBlockletToScan;
-  /**
-   * default constructor
-   */
-  public BlockletInfos(){
-  }
-  /**
-   * constructor to initialize the blockletinfo
-   * @param noOfBlockLets
-   * @param startBlockletNumber
-   * @param numberOfBlockletToScan
-   */
-  public BlockletInfos(int noOfBlockLets, int startBlockletNumber, int 
numberOfBlockletToScan) {
-    this.noOfBlockLets = noOfBlockLets;
-    this.startBlockletNumber = startBlockletNumber;
-    this.numberOfBlockletToScan = numberOfBlockletToScan;
-  }
-
-  /**
-   * returns the number of blockLets
-   *
-   * @return
-   */
-  public int getNoOfBlockLets() {
-    return noOfBlockLets;
-  }
-
-  /**
-   * sets the number of blockLets
-   *
-   * @param noOfBlockLets
-   */
-  public void setNoOfBlockLets(int noOfBlockLets) {
-    this.noOfBlockLets = noOfBlockLets;
-  }
-
-  /**
-   * returns start blocklet number
-   *
-   * @return
-   */
-  public int getStartBlockletNumber() {
-    return startBlockletNumber;
-  }
-
-  /**
-   * set start blocklet number
-   *
-   * @param startBlockletNumber
-   */
-  public void setStartBlockletNumber(int startBlockletNumber) {
-    this.startBlockletNumber = startBlockletNumber;
-  }
-
-  /**
-   * returns end blocklet number
-   *
-   * @return
-   */
-  public int getNumberOfBlockletToScan() {
-    return numberOfBlockletToScan;
-  }
-
-  /**
-   * set end blocklet number to be scaned
-   *
-   * @param numberOfBlockletToScan
-   */
-  public void setNumberOfBlockletToScan(int numberOfBlockletToScan) {
-    this.numberOfBlockletToScan = numberOfBlockletToScan;
-  }
-
-}
-

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/ce09aaaf/core/src/main/java/org/apache/carbondata/core/carbon/datastore/block/Distributable.java
----------------------------------------------------------------------
diff --git 
a/core/src/main/java/org/apache/carbondata/core/carbon/datastore/block/Distributable.java
 
b/core/src/main/java/org/apache/carbondata/core/carbon/datastore/block/Distributable.java
deleted file mode 100644
index 99d4459..0000000
--- 
a/core/src/main/java/org/apache/carbondata/core/carbon/datastore/block/Distributable.java
+++ /dev/null
@@ -1,27 +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.block;
-
-import java.io.IOException;
-
-/**
- * interface to get the locations of node. Used for making task distribution 
based on locality
- */
-public interface Distributable extends Comparable<Distributable> {
-
-  String[] getLocations() throws IOException;
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/ce09aaaf/core/src/main/java/org/apache/carbondata/core/carbon/datastore/block/SegmentProperties.java
----------------------------------------------------------------------
diff --git 
a/core/src/main/java/org/apache/carbondata/core/carbon/datastore/block/SegmentProperties.java
 
b/core/src/main/java/org/apache/carbondata/core/carbon/datastore/block/SegmentProperties.java
deleted file mode 100644
index 83d68ef..0000000
--- 
a/core/src/main/java/org/apache/carbondata/core/carbon/datastore/block/SegmentProperties.java
+++ /dev/null
@@ -1,754 +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.block;
-
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Map;
-import java.util.Map.Entry;
-import java.util.Set;
-
-import org.apache.carbondata.core.carbon.metadata.datatype.DataType;
-import org.apache.carbondata.core.carbon.metadata.encoder.Encoding;
-import 
org.apache.carbondata.core.carbon.metadata.schema.table.column.CarbonDimension;
-import 
org.apache.carbondata.core.carbon.metadata.schema.table.column.CarbonMeasure;
-import 
org.apache.carbondata.core.carbon.metadata.schema.table.column.ColumnSchema;
-import org.apache.carbondata.core.constants.CarbonCommonConstants;
-import org.apache.carbondata.core.datastorage.store.columnar.ColumnGroupModel;
-import org.apache.carbondata.core.keygenerator.KeyGenerator;
-import org.apache.carbondata.core.keygenerator.columnar.ColumnarSplitter;
-import 
org.apache.carbondata.core.keygenerator.columnar.impl.MultiDimKeyVarLengthVariableSplitGenerator;
-import 
org.apache.carbondata.core.keygenerator.mdkey.MultiDimKeyVarLengthGenerator;
-import org.apache.carbondata.core.util.CarbonUtil;
-
-import org.apache.commons.lang3.ArrayUtils;
-
-/**
- * This class contains all the details about the restructuring information of
- * the block. This will be used during query execution to handle restructure
- * information
- */
-public class SegmentProperties {
-
-  /**
-   * key generator of the block which was used to generate the mdkey for
-   * normal dimension. this will be required to
-   */
-  private KeyGenerator dimensionKeyGenerator;
-
-  /**
-   * list of dimension present in the block
-   */
-  private List<CarbonDimension> dimensions;
-
-  /**
-   * list of dimension present in the block
-   */
-  private List<CarbonDimension> complexDimensions;
-
-  /**
-   * list of measure present in the block
-   */
-  private List<CarbonMeasure> measures;
-
-  /**
-   * cardinality of dimension columns participated in key generator
-   */
-  private int[] dimColumnsCardinality;
-
-  /**
-   * cardinality of complex dimension
-   */
-  private int[] complexDimColumnCardinality;
-
-  /**
-   * mapping of dimension column to block in a file this will be used for
-   * reading the blocks from file
-   */
-  private Map<Integer, Integer> dimensionOrdinalToBlockMapping;
-
-  /**
-   * a block can have multiple columns. This will have block index as key
-   * and all dimension participated in that block as values
-   */
-  private Map<Integer, Set<Integer>> blockTodimensionOrdinalMapping;
-
-  /**
-   * mapping of measure column to block to in file this will be used while
-   * reading the block in a file
-   */
-  private Map<Integer, Integer> measuresOrdinalToBlockMapping;
-
-  /**
-   * size of the each dimension column value in a block this can be used when
-   * we need to do copy a cell value to create a tuple.for no dictionary
-   * column this value will be -1. for dictionary column we size of the value
-   * will be fixed.
-   */
-  private int[] eachDimColumnValueSize;
-
-  /**
-   * size of the each dimension column value in a block this can be used when
-   * we need to do copy a cell value to create a tuple.for no dictionary
-   * column this value will be -1. for dictionary column we size of the value
-   * will be fixed.
-   */
-  private int[] eachComplexDimColumnValueSize;
-
-  /**
-   * below mapping will have mapping of the column group to dimensions ordinal
-   * for example if 3 dimension present in the columngroupid 0 and its ordinal 
in
-   * 2,3,4 then map will contain 0,{2,3,4}
-   */
-  private Map<Integer, KeyGenerator> columnGroupAndItsKeygenartor;
-
-  /**
-   * column group key generator dimension index will not be same as dimension 
ordinal
-   * This will have mapping with ordinal and keygenerator or mdkey index
-   */
-  private Map<Integer, Map<Integer, Integer>> columnGroupOrdinalToMdkeymapping;
-
-  /**
-   * this will be used to split the fixed length key
-   * this will all the information about how key was created
-   * and how to split the key based on group
-   */
-  private ColumnarSplitter fixedLengthKeySplitter;
-
-  /**
-   * to store the number of no dictionary dimension
-   * this will be used during query execution for creating
-   * start and end key. Purpose of storing this value here is
-   * so during query execution no need to calculate every time
-   */
-  private int numberOfNoDictionaryDimension;
-
-  /**
-   * column group model
-   */
-  private ColumnGroupModel colGroupModel;
-
-  public SegmentProperties(List<ColumnSchema> columnsInTable, int[] 
columnCardinality) {
-    dimensions = new 
ArrayList<CarbonDimension>(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE);
-    complexDimensions =
-        new 
ArrayList<CarbonDimension>(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE);
-    measures = new 
ArrayList<CarbonMeasure>(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE);
-    fillDimensionAndMeasureDetails(columnsInTable, columnCardinality);
-    dimensionOrdinalToBlockMapping =
-        new HashMap<Integer, 
Integer>(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE);
-    blockTodimensionOrdinalMapping =
-        new HashMap<Integer, 
Set<Integer>>(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE);
-    measuresOrdinalToBlockMapping =
-        new HashMap<Integer, 
Integer>(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE);
-    intialiseColGroups();
-    fillOrdinalToBlockMappingForDimension();
-    fillOrdinalToBlockIndexMappingForMeasureColumns();
-    fillColumnGroupAndItsCardinality(columnCardinality);
-    fillKeyGeneratorDetails();
-  }
-
-  /**
-   * it fills column groups
-   * e.g {{1},{2,3,4},{5},{6},{7,8,9}}
-   *
-   */
-  private void intialiseColGroups() {
-    // StringBuffer columnGroups = new StringBuffer();
-    List<List<Integer>> colGrpList = new ArrayList<List<Integer>>();
-    List<Integer> group = new ArrayList<Integer>();
-    for (int i = 0; i < dimensions.size(); i++) {
-      CarbonDimension dimension = dimensions.get(i);
-      if (!dimension.hasEncoding(Encoding.DICTIONARY)) {
-        continue;
-      }
-      group.add(dimension.getOrdinal());
-      // columnGroups.append(dimension.getOrdinal());
-      if (i < dimensions.size() - 1) {
-        int currGroupOrdinal = dimension.columnGroupId();
-        int nextGroupOrdinal = dimensions.get(i + 1).columnGroupId();
-        if (!(currGroupOrdinal == nextGroupOrdinal && currGroupOrdinal != -1)) 
{
-          colGrpList.add(group);
-          group = new ArrayList<Integer>();
-        }
-      } else {
-        colGrpList.add(group);
-      }
-
-    }
-    int[][] colGroups = new int[colGrpList.size()][];
-    for (int i = 0; i < colGroups.length; i++) {
-      colGroups[i] = new int[colGrpList.get(i).size()];
-      for (int j = 0; j < colGroups[i].length; j++) {
-        colGroups[i][j] = colGrpList.get(i).get(j);
-      }
-    }
-    this.colGroupModel = CarbonUtil.getColGroupModel(colGroups);
-  }
-
-  /**
-   * below method is to fill the dimension and its mapping to file blocks all
-   * the column will point to same column group
-   */
-  private void fillOrdinalToBlockMappingForDimension() {
-    int blockOrdinal = -1;
-    CarbonDimension dimension = null;
-    int index = 0;
-    int prvcolumnGroupId = -1;
-    while (index < dimensions.size()) {
-      dimension = dimensions.get(index);
-      // if column id is same as previous one then block index will be
-      // same
-      if (dimension.isColumnar() || dimension.columnGroupId() != 
prvcolumnGroupId) {
-        blockOrdinal++;
-      }
-      dimensionOrdinalToBlockMapping.put(dimension.getOrdinal(), blockOrdinal);
-      prvcolumnGroupId = dimension.columnGroupId();
-      index++;
-    }
-    index = 0;
-    // complex dimension will be stored at last
-    while (index < complexDimensions.size()) {
-      dimension = complexDimensions.get(index);
-      dimensionOrdinalToBlockMapping.put(dimension.getOrdinal(), 
++blockOrdinal);
-      blockOrdinal = fillComplexDimensionChildBlockIndex(blockOrdinal, 
dimension);
-      index++;
-    }
-    fillBlockToDimensionOrdinalMapping();
-  }
-
-  /**
-   *
-   */
-  private void fillBlockToDimensionOrdinalMapping() {
-    Set<Entry<Integer, Integer>> blocks = 
dimensionOrdinalToBlockMapping.entrySet();
-    Iterator<Entry<Integer, Integer>> blockItr = blocks.iterator();
-    while (blockItr.hasNext()) {
-      Entry<Integer, Integer> block = blockItr.next();
-      Set<Integer> dimensionOrdinals = 
blockTodimensionOrdinalMapping.get(block.getValue());
-      if (dimensionOrdinals == null) {
-        dimensionOrdinals = new 
HashSet<Integer>(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE);
-        blockTodimensionOrdinalMapping.put(block.getValue(), 
dimensionOrdinals);
-      }
-      dimensionOrdinals.add(block.getKey());
-    }
-  }
-
-  /**
-   * Below method will be used to add the complex dimension child
-   * block index.It is a recursive method which will be get the children
-   * add the block index
-   *
-   * @param blockOrdinal start block ordinal
-   * @param dimension    parent dimension
-   * @return last block index
-   */
-  private int fillComplexDimensionChildBlockIndex(int blockOrdinal, 
CarbonDimension dimension) {
-    for (int i = 0; i < dimension.numberOfChild(); i++) {
-      dimensionOrdinalToBlockMapping
-          .put(dimension.getListOfChildDimensions().get(i).getOrdinal(), 
++blockOrdinal);
-      if (dimension.getListOfChildDimensions().get(i).numberOfChild() > 0) {
-        blockOrdinal = fillComplexDimensionChildBlockIndex(blockOrdinal,
-            dimension.getListOfChildDimensions().get(i));
-      }
-    }
-    return blockOrdinal;
-  }
-
-  /**
-   * Below method will be used to fill the mapping
-   * of measure ordinal to its block index mapping in
-   * file
-   */
-  private void fillOrdinalToBlockIndexMappingForMeasureColumns() {
-    int blockOrdinal = 0;
-    int index = 0;
-    while (index < measures.size()) {
-      measuresOrdinalToBlockMapping.put(measures.get(index).getOrdinal(), 
blockOrdinal);
-      blockOrdinal++;
-      index++;
-    }
-  }
-
-  /**
-   * below method will fill dimension and measure detail of the block.
-   *
-   * @param columnsInTable
-   * @param columnCardinality
-   */
-  private void fillDimensionAndMeasureDetails(List<ColumnSchema> 
columnsInTable,
-      int[] columnCardinality) {
-    ColumnSchema columnSchema = null;
-    // ordinal will be required to read the data from file block
-    int dimensonOrdinal = 0;
-    int measureOrdinal = -1;
-    // table ordinal is actually a schema ordinal this is required as
-    // cardinality array
-    // which is stored in segment info contains -1 if that particular column
-    // is n
-    int tableOrdinal = -1;
-    // creating a list as we do not know how many dimension not participated
-    // in the mdkey
-    List<Integer> cardinalityIndexForNormalDimensionColumn =
-        new ArrayList<Integer>(columnsInTable.size());
-    // creating a list as we do not know how many dimension not participated
-    // in the mdkey
-    List<Integer> cardinalityIndexForComplexDimensionColumn =
-        new ArrayList<Integer>(columnsInTable.size());
-    boolean isComplexDimensionStarted = false;
-    CarbonDimension carbonDimension = null;
-    // to store the position of dimension in surrogate key array which is
-    // participating in mdkey
-    int keyOrdinal = 0;
-    int previousColumnGroup = -1;
-    // to store the ordinal of the column group ordinal
-    int columnGroupOrdinal = 0;
-    int counter = 0;
-    int complexTypeOrdinal = 0;
-    while (counter < columnsInTable.size()) {
-      columnSchema = columnsInTable.get(counter);
-      if (columnSchema.isDimensionColumn()) {
-        tableOrdinal++;
-        // not adding the cardinality of the non dictionary
-        // column as it was not the part of mdkey
-        if (CarbonUtil.hasEncoding(columnSchema.getEncodingList(), 
Encoding.DICTIONARY)
-            && !isComplexDimensionStarted && columnSchema.getNumberOfChild() 
== 0) {
-          cardinalityIndexForNormalDimensionColumn.add(tableOrdinal);
-          if (columnSchema.isColumnar()) {
-            // if it is a columnar dimension participated in mdkey then added
-            // key ordinal and dimension ordinal
-            carbonDimension =
-                new CarbonDimension(columnSchema, dimensonOrdinal++, 
keyOrdinal++, -1, -1);
-          } else {
-            // if not columnnar then it is a column group dimension
-
-            // below code to handle first dimension of the column group
-            // in this case ordinal of the column group will be 0
-            if (previousColumnGroup != columnSchema.getColumnGroupId()) {
-              columnGroupOrdinal = 0;
-              carbonDimension = new CarbonDimension(columnSchema, 
dimensonOrdinal++, keyOrdinal++,
-                  columnGroupOrdinal++, -1);
-            }
-            // if previous dimension  column group id is same as current then
-            // then its belongs to same row group
-            else {
-              carbonDimension = new CarbonDimension(columnSchema, 
dimensonOrdinal++, keyOrdinal++,
-                  columnGroupOrdinal++, -1);
-            }
-            previousColumnGroup = columnSchema.getColumnGroupId();
-          }
-        }
-        // as complex type will be stored at last so once complex type started 
all the dimension
-        // will be added to complex type
-        else if (isComplexDimensionStarted || 
CarbonUtil.hasDataType(columnSchema.getDataType(),
-            new DataType[] { DataType.ARRAY, DataType.STRUCT })) {
-          cardinalityIndexForComplexDimensionColumn.add(tableOrdinal);
-          carbonDimension =
-              new CarbonDimension(columnSchema, dimensonOrdinal++, -1, -1, 
complexTypeOrdinal++);
-          
carbonDimension.initializeChildDimensionsList(columnSchema.getNumberOfChild());
-          complexDimensions.add(carbonDimension);
-          isComplexDimensionStarted = true;
-          int previouseOrdinal = dimensonOrdinal;
-          dimensonOrdinal =
-              readAllComplexTypeChildrens(dimensonOrdinal, 
columnSchema.getNumberOfChild(),
-                  columnsInTable, carbonDimension, complexTypeOrdinal);
-          int numberOfChildrenDimensionAdded = dimensonOrdinal - 
previouseOrdinal;
-          for (int i = 0; i < numberOfChildrenDimensionAdded; i++) {
-            cardinalityIndexForComplexDimensionColumn.add(++tableOrdinal);
-          }
-          counter = dimensonOrdinal;
-          complexTypeOrdinal = carbonDimension.getListOfChildDimensions()
-              .get(carbonDimension.getListOfChildDimensions().size() - 
1).getComplexTypeOrdinal();
-          complexTypeOrdinal++;
-          continue;
-        } else {
-          // for no dictionary dimension
-          carbonDimension = new CarbonDimension(columnSchema, 
dimensonOrdinal++, -1, -1, -1);
-          numberOfNoDictionaryDimension++;
-        }
-        dimensions.add(carbonDimension);
-      } else {
-        measures.add(new CarbonMeasure(columnSchema, ++measureOrdinal));
-      }
-      counter++;
-    }
-    dimColumnsCardinality = new 
int[cardinalityIndexForNormalDimensionColumn.size()];
-    complexDimColumnCardinality = new 
int[cardinalityIndexForComplexDimensionColumn.size()];
-    int index = 0;
-    // filling the cardinality of the dimension column to create the key
-    // generator
-    for (Integer cardinalityArrayIndex : 
cardinalityIndexForNormalDimensionColumn) {
-      dimColumnsCardinality[index++] = 
columnCardinality[cardinalityArrayIndex];
-    }
-    index = 0;
-    // filling the cardinality of the complex dimension column to create the
-    // key generator
-    for (Integer cardinalityArrayIndex : 
cardinalityIndexForComplexDimensionColumn) {
-      complexDimColumnCardinality[index++] = 
columnCardinality[cardinalityArrayIndex];
-    }
-  }
-
-  /**
-   * Read all primitive/complex children and set it as list of child carbon 
dimension to parent
-   * dimension
-   *
-   * @param dimensionOrdinal
-   * @param childCount
-   * @param listOfColumns
-   * @param parentDimension
-   * @return
-   */
-  private int readAllComplexTypeChildrens(int dimensionOrdinal, int childCount,
-      List<ColumnSchema> listOfColumns, CarbonDimension parentDimension,
-      int complexDimensionOrdianl) {
-    for (int i = 0; i < childCount; i++) {
-      ColumnSchema columnSchema = listOfColumns.get(dimensionOrdinal);
-      if (columnSchema.isDimensionColumn()) {
-        if (columnSchema.getNumberOfChild() > 0) {
-          CarbonDimension complexDimension =
-              new CarbonDimension(columnSchema, dimensionOrdinal++, -1, -1,
-                  complexDimensionOrdianl++);
-          
complexDimension.initializeChildDimensionsList(columnSchema.getNumberOfChild());
-          parentDimension.getListOfChildDimensions().add(complexDimension);
-          dimensionOrdinal =
-              readAllComplexTypeChildrens(dimensionOrdinal, 
columnSchema.getNumberOfChild(),
-                  listOfColumns, complexDimension, complexDimensionOrdianl);
-        } else {
-          parentDimension.getListOfChildDimensions().add(
-              new CarbonDimension(columnSchema, dimensionOrdinal++, -1, -1,
-                  complexDimensionOrdianl++));
-        }
-      }
-    }
-    return dimensionOrdinal;
-  }
-
-  /**
-   * Below method will fill the key generator detail of both the type of key
-   * generator. This will be required for during both query execution and data
-   * loading.
-   */
-  private void fillKeyGeneratorDetails() {
-    // create a dimension partitioner list
-    // this list will contain information about how dimension value are
-    // stored
-    // it is stored in group or individually
-    List<Integer> dimensionPartitionList =
-        new ArrayList<Integer>(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE);
-    List<Boolean> isDictionaryColumn =
-        new ArrayList<Boolean>(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE);
-    int prvcolumnGroupId = -1;
-    int counter = 0;
-    while (counter < dimensions.size()) {
-      CarbonDimension carbonDimension = dimensions.get(counter);
-      // if dimension is not a part of mdkey then no need to add
-      if (!carbonDimension.getEncoder().contains(Encoding.DICTIONARY)) {
-        isDictionaryColumn.add(false);
-        counter++;
-        continue;
-      }
-      // columnar column is stored individually
-      // so add one
-      if (carbonDimension.isColumnar()) {
-        dimensionPartitionList.add(1);
-        isDictionaryColumn.add(true);
-      }
-      // if in a group then need to add how many columns a selected in
-      // group
-      if (!carbonDimension.isColumnar() && carbonDimension.columnGroupId() == 
prvcolumnGroupId) {
-        // incrementing the previous value of the list as it is in same column 
group
-        dimensionPartitionList.set(dimensionPartitionList.size() - 1,
-            dimensionPartitionList.get(dimensionPartitionList.size() - 1) + 1);
-      } else if (!carbonDimension.isColumnar()) {
-        dimensionPartitionList.add(1);
-        isDictionaryColumn.add(true);
-      }
-      prvcolumnGroupId = carbonDimension.columnGroupId();
-      counter++;
-    }
-    // get the partitioner
-    int[] dimensionPartitions = ArrayUtils
-        .toPrimitive(dimensionPartitionList.toArray(new 
Integer[dimensionPartitionList.size()]));
-    // get the bit length of each column
-    int[] bitLength = CarbonUtil.getDimensionBitLength(dimColumnsCardinality, 
dimensionPartitions);
-    // create a key generator
-    this.dimensionKeyGenerator = new MultiDimKeyVarLengthGenerator(bitLength);
-    this.fixedLengthKeySplitter =
-        new MultiDimKeyVarLengthVariableSplitGenerator(bitLength, 
dimensionPartitions);
-    // get the size of each value in file block
-    int[] dictionayDimColumnValueSize = 
fixedLengthKeySplitter.getBlockKeySize();
-    int index = -1;
-    this.eachDimColumnValueSize = new int[isDictionaryColumn.size()];
-    for (int i = 0; i < eachDimColumnValueSize.length; i++) {
-      if (!isDictionaryColumn.get(i)) {
-        eachDimColumnValueSize[i] = -1;
-        continue;
-      }
-      eachDimColumnValueSize[i] = dictionayDimColumnValueSize[++index];
-    }
-    if (complexDimensions.size() > 0) {
-      int[] complexDimesionParition = new 
int[complexDimColumnCardinality.length];
-      // as complex dimension will be stored in column format add one
-      Arrays.fill(complexDimesionParition, 1);
-      bitLength =
-          CarbonUtil.getDimensionBitLength(complexDimColumnCardinality, 
complexDimesionParition);
-      for (int i = 0; i < bitLength.length; i++) {
-        if (complexDimColumnCardinality[i] == 0) {
-          bitLength[i] = 64;
-        }
-      }
-      ColumnarSplitter keySplitter =
-          new MultiDimKeyVarLengthVariableSplitGenerator(bitLength, 
complexDimesionParition);
-      eachComplexDimColumnValueSize = keySplitter.getBlockKeySize();
-    } else {
-      eachComplexDimColumnValueSize = new int[0];
-    }
-  }
-
-  /**
-   * Below method will be used to create a mapping of column group and its 
column cardinality this
-   * mapping will have column group id to cardinality of the dimension present 
in
-   * the column group.This mapping will be used during query execution, to 
create
-   * a mask key for the column group dimension which will be used in 
aggregation
-   * and filter query as column group dimension will be stored at the bit level
-   */
-  private void fillColumnGroupAndItsCardinality(int[] cardinality) {
-    // mapping of the column group and its ordinal
-    Map<Integer, List<Integer>> columnGroupAndOrdinalMapping =
-        new HashMap<Integer, 
List<Integer>>(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE);
-    // to store a column group
-    List<Integer> currentColumnGroup = null;
-    // current index
-    int index = 0;
-    // previous column group to check all the column of column id has bee 
selected
-    int prvColumnGroupId = -1;
-    while (index < dimensions.size()) {
-      // if dimension group id is not zero and it is same as the previous
-      // column id
-      // then we need to add ordinal of that column as it belongs to same
-      // column group
-      if (!dimensions.get(index).isColumnar()
-          && dimensions.get(index).columnGroupId() == prvColumnGroupId
-          && null != currentColumnGroup) {
-        currentColumnGroup.add(index);
-      }
-      // if column is not a columnar then new column group has come
-      // so we need to create a list of new column id group and add the
-      // ordinal
-      else if (!dimensions.get(index).isColumnar()) {
-        currentColumnGroup = new ArrayList<Integer>();
-        
columnGroupAndOrdinalMapping.put(dimensions.get(index).columnGroupId(), 
currentColumnGroup);
-        currentColumnGroup.add(index);
-      }
-      // update the column id every time,this is required to group the
-      // columns
-      // of the same column group
-      prvColumnGroupId = dimensions.get(index).columnGroupId();
-      index++;
-    }
-    // Initializing the map
-    this.columnGroupAndItsKeygenartor =
-        new HashMap<Integer, 
KeyGenerator>(columnGroupAndOrdinalMapping.size());
-    this.columnGroupOrdinalToMdkeymapping = new 
HashMap<>(columnGroupAndOrdinalMapping.size());
-    int[] columnGroupCardinality = null;
-    index = 0;
-    Iterator<Entry<Integer, List<Integer>>> iterator =
-        columnGroupAndOrdinalMapping.entrySet().iterator();
-    while (iterator.hasNext()) {
-      Entry<Integer, List<Integer>> next = iterator.next();
-      List<Integer> currentGroupOrdinal = next.getValue();
-      Map<Integer, Integer> colGrpOrdinalMdkeyMapping = new 
HashMap<>(currentGroupOrdinal.size());
-      // create the cardinality array
-      columnGroupCardinality = new int[currentGroupOrdinal.size()];
-      for (int i = 0; i < columnGroupCardinality.length; i++) {
-        // fill the cardinality
-        columnGroupCardinality[i] = cardinality[currentGroupOrdinal.get(i)];
-        colGrpOrdinalMdkeyMapping.put(currentGroupOrdinal.get(i), i);
-      }
-      this.columnGroupAndItsKeygenartor.put(next.getKey(), new 
MultiDimKeyVarLengthGenerator(
-          CarbonUtil.getDimensionBitLength(columnGroupCardinality,
-              new int[] { columnGroupCardinality.length })));
-      this.columnGroupOrdinalToMdkeymapping.put(next.getKey(), 
colGrpOrdinalMdkeyMapping);
-    }
-  }
-
-  /**
-   * Below method is to get the value of each dimension column. As this method
-   * will be used only once so we can merge both the dimension and complex
-   * dimension array. Complex dimension will be store at last so first copy
-   * the normal dimension the copy the complex dimension size. If we store
-   * this value as a class variable unnecessarily we will waste some space
-   *
-   * @return each dimension value size
-   */
-  public int[] getDimensionColumnsValueSize() {
-    int[] dimensionValueSize =
-        new int[eachDimColumnValueSize.length + 
eachComplexDimColumnValueSize.length];
-    System
-        .arraycopy(eachDimColumnValueSize, 0, dimensionValueSize, 0, 
eachDimColumnValueSize.length);
-    System.arraycopy(eachComplexDimColumnValueSize, 0, dimensionValueSize,
-        eachDimColumnValueSize.length, eachComplexDimColumnValueSize.length);
-    return dimensionValueSize;
-  }
-
-  /**
-   * @return the dimensionKeyGenerator
-   */
-  public KeyGenerator getDimensionKeyGenerator() {
-    return dimensionKeyGenerator;
-  }
-
-  /**
-   * @return the dimensions
-   */
-  public List<CarbonDimension> getDimensions() {
-    return dimensions;
-  }
-
-  /**
-   * @return the complexDimensions
-   */
-  public List<CarbonDimension> getComplexDimensions() {
-    return complexDimensions;
-  }
-
-  /**
-   * @return the measures
-   */
-  public List<CarbonMeasure> getMeasures() {
-    return measures;
-  }
-
-  /**
-   * @return the dimColumnsCardinality
-   */
-  public int[] getDimColumnsCardinality() {
-    return dimColumnsCardinality;
-  }
-
-  /**
-   * @return the complexDimColumnCardinality
-   */
-  public int[] getComplexDimColumnCardinality() {
-    return complexDimColumnCardinality;
-  }
-
-  /**
-   * @return the dimensionOrdinalToBlockMapping
-   */
-  public Map<Integer, Integer> getDimensionOrdinalToBlockMapping() {
-    return dimensionOrdinalToBlockMapping;
-  }
-
-  /**
-   * @return the measuresOrdinalToBlockMapping
-   */
-  public Map<Integer, Integer> getMeasuresOrdinalToBlockMapping() {
-    return measuresOrdinalToBlockMapping;
-  }
-
-  /**
-   * @return the eachDimColumnValueSize
-   */
-  public int[] getEachDimColumnValueSize() {
-    return eachDimColumnValueSize;
-  }
-
-  /**
-   * @return the eachComplexDimColumnValueSize
-   */
-  public int[] getEachComplexDimColumnValueSize() {
-    return eachComplexDimColumnValueSize;
-  }
-
-  /**
-   * @return the fixedLengthKeySplitter
-   */
-  public ColumnarSplitter getFixedLengthKeySplitter() {
-    return fixedLengthKeySplitter;
-  }
-
-  /**
-   * @return the columnGroupAndItsKeygenartor
-   */
-  public Map<Integer, KeyGenerator> getColumnGroupAndItsKeygenartor() {
-    return columnGroupAndItsKeygenartor;
-  }
-
-  /**
-   * @return the numberOfNoDictionaryDimension
-   */
-  public int getNumberOfNoDictionaryDimension() {
-    return numberOfNoDictionaryDimension;
-  }
-
-  /**
-   * @return
-   */
-  public int[][] getColumnGroups() {
-    return colGroupModel.getColumnGroup();
-  }
-
-  /**
-   * @return colGroupModel
-   */
-  public ColumnGroupModel getColumnGroupModel() {
-    return this.colGroupModel;
-  }
-
-  /**
-   * get mdkey ordinal for given dimension ordinal of given column group
-   *
-   * @param colGrpId
-   * @param ordinal
-   * @return mdkeyordinal
-   */
-  public int getColumnGroupMdKeyOrdinal(int colGrpId, int ordinal) {
-    return columnGroupOrdinalToMdkeymapping.get(colGrpId).get(ordinal);
-  }
-
-  /**
-   * It returns no of column availble in given column group
-   *
-   * @param colGrpId
-   * @return no of column in given column group
-   */
-  public int getNoOfColumnsInColumnGroup(int colGrpId) {
-    return columnGroupOrdinalToMdkeymapping.get(colGrpId).size();
-  }
-
-  /**
-   * @param blockIndex
-   * @return It returns all dimension present in given block index
-   */
-  public Set<Integer> getDimensionOrdinalForBlock(int blockIndex) {
-    return blockTodimensionOrdinalMapping.get(blockIndex);
-  }
-
-  /**
-   * @return It returns block index to dimension ordinal mapping
-   */
-  public Map<Integer, Set<Integer>> getBlockTodimensionOrdinalMapping() {
-    return blockTodimensionOrdinalMapping;
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/ce09aaaf/core/src/main/java/org/apache/carbondata/core/carbon/datastore/block/SegmentTaskIndex.java
----------------------------------------------------------------------
diff --git 
a/core/src/main/java/org/apache/carbondata/core/carbon/datastore/block/SegmentTaskIndex.java
 
b/core/src/main/java/org/apache/carbondata/core/carbon/datastore/block/SegmentTaskIndex.java
deleted file mode 100644
index 655226f..0000000
--- 
a/core/src/main/java/org/apache/carbondata/core/carbon/datastore/block/SegmentTaskIndex.java
+++ /dev/null
@@ -1,57 +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.block;
-
-import java.util.List;
-
-import org.apache.carbondata.core.carbon.datastore.BTreeBuilderInfo;
-import org.apache.carbondata.core.carbon.datastore.BtreeBuilder;
-import 
org.apache.carbondata.core.carbon.datastore.impl.btree.BlockBTreeBuilder;
-import org.apache.carbondata.core.carbon.metadata.blocklet.DataFileFooter;
-
-/**
- * Class which is responsible for loading the b+ tree block. This class will
- * persist all the detail of a table segment
- */
-public class SegmentTaskIndex extends AbstractIndex {
-
-  /**
-   * Below method is store the blocks in some data structure
-   *
-   */
-  public void buildIndex(List<DataFileFooter> footerList) {
-    // create a metadata details
-    // this will be useful in query handling
-    // all the data file metadata will have common segment properties we
-    // can use first one to get create the segment properties
-    segmentProperties = new 
SegmentProperties(footerList.get(0).getColumnInTable(),
-        footerList.get(0).getSegmentInfo().getColumnCardinality());
-    // create a segment builder info
-    // in case of segment create we do not need any file path and each column 
value size
-    // as Btree will be build as per min max and start key
-    BTreeBuilderInfo btreeBuilderInfo = new BTreeBuilderInfo(footerList, null);
-    BtreeBuilder blocksBuilder = new BlockBTreeBuilder();
-    // load the metadata
-    blocksBuilder.build(btreeBuilderInfo);
-    dataRefNode = blocksBuilder.get();
-    for (DataFileFooter footer : footerList) {
-      totalNumberOfRows += footer.getNumberOfRows();
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/ce09aaaf/core/src/main/java/org/apache/carbondata/core/carbon/datastore/block/SegmentTaskIndexWrapper.java
----------------------------------------------------------------------
diff --git 
a/core/src/main/java/org/apache/carbondata/core/carbon/datastore/block/SegmentTaskIndexWrapper.java
 
b/core/src/main/java/org/apache/carbondata/core/carbon/datastore/block/SegmentTaskIndexWrapper.java
deleted file mode 100644
index 6a0d31b..0000000
--- 
a/core/src/main/java/org/apache/carbondata/core/carbon/datastore/block/SegmentTaskIndexWrapper.java
+++ /dev/null
@@ -1,148 +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.block;
-
-import java.util.Map;
-import java.util.concurrent.atomic.AtomicInteger;
-import java.util.concurrent.atomic.AtomicLong;
-
-import org.apache.carbondata.core.cache.Cacheable;
-import org.apache.carbondata.core.carbon.datastore.SegmentTaskIndexStore;
-import org.apache.carbondata.core.update.UpdateVO;
-
-/**
- * SegmentTaskIndexWrapper class holds the  taskIdToTableSegmentMap
- */
-public class SegmentTaskIndexWrapper implements Cacheable {
-
-  /**
-   * task_id to table segment index map
-   */
-  private Map<SegmentTaskIndexStore.TaskBucketHolder, AbstractIndex> 
taskIdToTableSegmentMap;
-  /**
-   * atomic integer to maintain the access count for a column access
-   */
-  protected AtomicInteger accessCount = new AtomicInteger();
-
-  /**
-   * Table block meta size.
-   */
-  protected AtomicLong memorySize = new AtomicLong();
-
-  private Long refreshedTimeStamp;
-  private UpdateVO invalidTaskKey;
-  public SegmentTaskIndexWrapper(
-      Map<SegmentTaskIndexStore.TaskBucketHolder, AbstractIndex> 
taskIdToTableSegmentMap) {
-    this.taskIdToTableSegmentMap = taskIdToTableSegmentMap;
-  }
-
-  public Map<SegmentTaskIndexStore.TaskBucketHolder, AbstractIndex> 
getTaskIdToTableSegmentMap() {
-    return taskIdToTableSegmentMap;
-  }
-
-  public void setTaskIdToTableSegmentMap(
-      Map<SegmentTaskIndexStore.TaskBucketHolder, AbstractIndex> 
taskIdToTableSegmentMap) {
-    this.taskIdToTableSegmentMap = taskIdToTableSegmentMap;
-  }
-
-  /**
-   * return segment size
-   *
-   * @param memorySize
-   */
-  public void setMemorySize(long memorySize) {
-    this.memorySize.set(memorySize);
-  }
-
-  /**
-   * returns the timestamp
-   *
-   * @return
-   */
-  @Override public long getFileTimeStamp() {
-    return 0;
-  }
-
-  /**
-   * returns the access count
-   *
-   * @return
-   */
-  @Override public int getAccessCount() {
-    return accessCount.get();
-  }
-
-  /**
-   * returns the memory size
-   *
-   * @return
-   */
-  @Override public long getMemorySize() {
-    return memorySize.get();
-  }
-
-  /**
-   * The method is used to set the access count
-   */
-  public void incrementAccessCount() {
-    accessCount.incrementAndGet();
-  }
-
-  /**
-   * This method will release the objects and set default value for primitive 
types
-   */
-  public void clear() {
-    decrementAccessCount();
-  }
-
-  /**
-   * This method will decrement the access count for a column by 1
-   * whenever a column usage is complete
-   */
-  private void decrementAccessCount() {
-    if (accessCount.get() > 0) {
-      accessCount.decrementAndGet();
-    }
-  }
-
-  public Long getRefreshedTimeStamp() {
-    return refreshedTimeStamp;
-  }
-
-  public void setRefreshedTimeStamp(Long refreshedTimeStamp) {
-    this.refreshedTimeStamp = refreshedTimeStamp;
-  }
-
-  public void removeEntryFromCacheAndRefresh(String taskId) {
-    AbstractIndex blockEntry = 
this.getTaskIdToTableSegmentMap().remove(taskId);
-    if (null != blockEntry) {
-      memorySize.set(memorySize.get() - blockEntry.getMemorySize());
-    }
-  }
-
-  public void setLastUpdateVO(UpdateVO invalidTaskKey) {
-    this.invalidTaskKey = invalidTaskKey;
-  }
-
-  public UpdateVO getInvalidTaskKey() {
-    return invalidTaskKey;
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/ce09aaaf/core/src/main/java/org/apache/carbondata/core/carbon/datastore/block/TableBlockInfo.java
----------------------------------------------------------------------
diff --git 
a/core/src/main/java/org/apache/carbondata/core/carbon/datastore/block/TableBlockInfo.java
 
b/core/src/main/java/org/apache/carbondata/core/carbon/datastore/block/TableBlockInfo.java
deleted file mode 100644
index 86b7b58..0000000
--- 
a/core/src/main/java/org/apache/carbondata/core/carbon/datastore/block/TableBlockInfo.java
+++ /dev/null
@@ -1,311 +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.block;
-
-import java.io.Serializable;
-import java.util.HashMap;
-import java.util.Map;
-
-import org.apache.carbondata.core.carbon.ColumnarFormatVersion;
-import org.apache.carbondata.core.carbon.path.CarbonTablePath;
-import org.apache.carbondata.core.carbon.path.CarbonTablePath.DataFileUtil;
-import org.apache.carbondata.core.constants.CarbonCommonConstants;
-import org.apache.carbondata.core.datastorage.store.impl.FileFactory;
-
-/**
- * class will be used to pass the block detail detail will be passed form 
driver
- * to all the executor to load the b+ tree
- */
-public class TableBlockInfo implements Distributable, Serializable {
-
-  /**
-   * serialization id
-   */
-  private static final long serialVersionUID = -6502868998599821172L;
-
-  /**
-   * full qualified file path of the block
-   */
-  private String filePath;
-
-  /**
-   * block offset in the file
-   */
-  private long blockOffset;
-
-  /**
-   * length of the block
-   */
-  private long blockLength;
-
-  /**
-   * id of the segment this will be used to sort the blocks
-   */
-  private String segmentId;
-
-  private String[] locations;
-
-  private ColumnarFormatVersion version;
-  /**
-   * The class holds the blockletsinfo
-   */
-  private BlockletInfos blockletInfos = new BlockletInfos();
-
-  /**
-   * map of block location and storage id
-   */
-  private Map<String, String> blockStorageIdMap =
-          new HashMap<>(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE);
-
-  public TableBlockInfo(String filePath, long blockOffset, String segmentId, 
String[] locations,
-      long blockLength, ColumnarFormatVersion version) {
-    this.filePath = FileFactory.getUpdatedFilePath(filePath);
-    this.blockOffset = blockOffset;
-    this.segmentId = segmentId;
-    this.locations = locations;
-    this.blockLength = blockLength;
-    this.version = version;
-  }
-
-  /**
-   * constructor to initialize the TbaleBlockInfo with BlockletInfos
-   *
-   * @param filePath
-   * @param blockOffset
-   * @param segmentId
-   * @param locations
-   * @param blockLength
-   * @param blockletInfos
-   */
-  public TableBlockInfo(String filePath, long blockOffset, String segmentId, 
String[] locations,
-      long blockLength, BlockletInfos blockletInfos, ColumnarFormatVersion 
version) {
-    this(filePath, blockOffset, segmentId, locations, blockLength, version);
-    this.blockletInfos = blockletInfos;
-  }
-
-  /**
-   * constructor to initialize the TableBlockInfo with blockStorageIdMap
-   *
-   * @param filePath
-   * @param blockOffset
-   * @param segmentId
-   * @param locations
-   * @param blockLength
-   * @param blockletInfos
-   * @param version
-   * @param blockStorageIdMap
-   */
-  public TableBlockInfo(String filePath, long blockOffset, String segmentId, 
String[] locations,
-      long blockLength, BlockletInfos blockletInfos, ColumnarFormatVersion 
version,
-      Map<String, String> blockStorageIdMap) {
-    this(filePath, blockOffset, segmentId, locations, blockLength, 
blockletInfos, version);
-    this.blockStorageIdMap = blockStorageIdMap;
-  }
-
-  /**
-   * @return the filePath
-   */
-  public String getFilePath() {
-    return filePath;
-  }
-
-  /**
-   * @return the blockOffset
-   */
-  public long getBlockOffset() {
-    return blockOffset;
-  }
-
-  public void setBlockOffset(long blockOffset) {
-    this.blockOffset = blockOffset;
-  }
-
-  /**
-   * @return the segmentId
-   */
-  public String getSegmentId() {
-    return segmentId;
-  }
-
-  /**
-   * @return the blockLength
-   */
-  public long getBlockLength() {
-    return blockLength;
-  }
-
-  /*
-   * (non-Javadoc)
-   *
-   * @see java.lang.Object#equals(java.lang.Object)
-   */
-  @Override public boolean equals(Object obj) {
-    if (this == obj) {
-      return true;
-    }
-    if (obj == null) {
-      return false;
-    }
-    if (!(obj instanceof TableBlockInfo)) {
-      return false;
-    }
-    TableBlockInfo other = (TableBlockInfo) obj;
-    if (!segmentId.equals(other.segmentId)) {
-      return false;
-    }
-    if (blockOffset != other.blockOffset) {
-      return false;
-    }
-    if (blockLength != other.blockLength) {
-      return false;
-    }
-    if (filePath == null && other.filePath != null) {
-      return false;
-    } else if (filePath != null && other.filePath == null) {
-      return false;
-    } else if (!filePath.equals(other.filePath)) {
-      return false;
-    }
-    if (blockletInfos.getStartBlockletNumber() != 
other.blockletInfos.getStartBlockletNumber()) {
-      return false;
-    }
-    return true;
-  }
-
-  /**
-   * Below method will used to compare to TableBlockInfos object this will
-   * used for sorting Comparison logic is: 1. compare segment id if segment id
-   * is same 2. compare task id if task id is same 3. compare offsets of the
-   * block
-   */
-  @Override public int compareTo(Distributable other) {
-
-    int compareResult = 0;
-    // get the segment id
-    // converr seg ID to double.
-
-    double seg1 = Double.parseDouble(segmentId);
-    double seg2 = Double.parseDouble(((TableBlockInfo) other).segmentId);
-    if (seg1 - seg2 < 0) {
-      return -1;
-    }
-    if (seg1 - seg2 > 0) {
-      return 1;
-    }
-
-    // Comparing the time task id of the file to other
-    // if both the task id of the file is same then we need to compare the
-    // offset of
-    // the file
-    if (CarbonTablePath.isCarbonDataFile(filePath)) {
-      int firstTaskId = Integer.parseInt(DataFileUtil.getTaskNo(filePath));
-      int otherTaskId = 
Integer.parseInt(DataFileUtil.getTaskNo(((TableBlockInfo) other).filePath));
-      if (firstTaskId != otherTaskId) {
-        return firstTaskId - otherTaskId;
-      }
-      // compare the part no of both block info
-      int firstPartNo = Integer.parseInt(DataFileUtil.getPartNo(filePath));
-      int SecondPartNo =
-          Integer.parseInt(DataFileUtil.getPartNo(((TableBlockInfo) 
other).filePath));
-      compareResult = firstPartNo - SecondPartNo;
-    } else {
-      compareResult = filePath.compareTo(((TableBlockInfo) 
other).getFilePath());
-    }
-    if (compareResult != 0) {
-      return compareResult;
-    }
-    //compare result is not 0 then return
-    // if part no is also same then compare the offset and length of the block
-    if (blockOffset + blockLength
-        < ((TableBlockInfo) other).blockOffset + ((TableBlockInfo) 
other).blockLength) {
-      return -1;
-    } else if (blockOffset + blockLength
-        > ((TableBlockInfo) other).blockOffset + ((TableBlockInfo) 
other).blockLength) {
-      return 1;
-    }
-    //compare the startBlockLetNumber
-    int diffStartBlockLetNumber =
-        blockletInfos.getStartBlockletNumber() - ((TableBlockInfo) 
other).blockletInfos
-            .getStartBlockletNumber();
-    if (diffStartBlockLetNumber < 0) {
-      return -1;
-    }
-    if (diffStartBlockLetNumber > 0) {
-      return 1;
-    }
-    return 0;
-  }
-
-  @Override public int hashCode() {
-    int result = filePath.hashCode();
-    result = 31 * result + (int) (blockOffset ^ (blockOffset >>> 32));
-    result = 31 * result + (int) (blockLength ^ (blockLength >>> 32));
-    result = 31 * result + segmentId.hashCode();
-    result = 31 * result + blockletInfos.getStartBlockletNumber();
-    return result;
-  }
-
-  @Override public String[] getLocations() {
-    return locations;
-  }
-
-  /**
-   * returns BlockletInfos
-   *
-   * @return
-   */
-  public BlockletInfos getBlockletInfos() {
-    return blockletInfos;
-  }
-
-  /**
-   * set the blocklestinfos
-   *
-   * @param blockletInfos
-   */
-  public void setBlockletInfos(BlockletInfos blockletInfos) {
-    this.blockletInfos = blockletInfos;
-  }
-
-  public ColumnarFormatVersion getVersion() {
-    return version;
-  }
-
-  public void setVersion(ColumnarFormatVersion version) {
-    this.version = version;
-  }
-
-  /**
-   * returns the storage location vs storage id map
-   *
-   * @return
-   */
-  public Map<String, String> getBlockStorageIdMap() {
-    return this.blockStorageIdMap;
-  }
-
-  /**
-   * method to storage location vs storage id map
-   *
-   * @param blockStorageIdMap
-   */
-  public void setBlockStorageIdMap(Map<String, String> blockStorageIdMap) {
-    this.blockStorageIdMap = blockStorageIdMap;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/ce09aaaf/core/src/main/java/org/apache/carbondata/core/carbon/datastore/block/TableBlockUniqueIdentifier.java
----------------------------------------------------------------------
diff --git 
a/core/src/main/java/org/apache/carbondata/core/carbon/datastore/block/TableBlockUniqueIdentifier.java
 
b/core/src/main/java/org/apache/carbondata/core/carbon/datastore/block/TableBlockUniqueIdentifier.java
deleted file mode 100644
index 6e57e0f..0000000
--- 
a/core/src/main/java/org/apache/carbondata/core/carbon/datastore/block/TableBlockUniqueIdentifier.java
+++ /dev/null
@@ -1,72 +0,0 @@
-package org.apache.carbondata.core.carbon.datastore.block;
-
-import org.apache.carbondata.core.carbon.AbsoluteTableIdentifier;
-import org.apache.carbondata.core.carbon.CarbonTableIdentifier;
-import org.apache.carbondata.core.constants.CarbonCommonConstants;
-
-/**
- * Class : Holds the info to uniquely identify a blocks
- */
-public class TableBlockUniqueIdentifier {
-
-  /**
-   * table fully qualified name
-   */
-  private AbsoluteTableIdentifier absoluteTableIdentifier;
-
-  /**
-   * table block info
-   */
-  private TableBlockInfo tableBlockInfo;
-
-  public TableBlockUniqueIdentifier(AbsoluteTableIdentifier 
absoluteTableIdentifier,
-      TableBlockInfo tableBlockInfo) {
-    this.absoluteTableIdentifier = absoluteTableIdentifier;
-    this.tableBlockInfo = tableBlockInfo;
-  }
-
-  public AbsoluteTableIdentifier getAbsoluteTableIdentifier() {
-    return absoluteTableIdentifier;
-  }
-
-  public void setAbsoluteTableIdentifier(AbsoluteTableIdentifier 
absoluteTableIdentifier) {
-    this.absoluteTableIdentifier = absoluteTableIdentifier;
-  }
-
-  public TableBlockInfo getTableBlockInfo() {
-    return tableBlockInfo;
-  }
-
-  public void setTableBlockInfo(TableBlockInfo tableBlockInfo) {
-    this.tableBlockInfo = tableBlockInfo;
-  }
-
-  @Override public int hashCode() {
-    return this.absoluteTableIdentifier.hashCode() + 
this.tableBlockInfo.hashCode();
-  }
-
-  @Override public boolean equals(Object other) {
-    if (this == other) return true;
-    if (other == null || getClass() != other.getClass()) return false;
-    TableBlockUniqueIdentifier tableBlockUniqueIdentifier = 
(TableBlockUniqueIdentifier) other;
-    return 
this.absoluteTableIdentifier.equals(tableBlockUniqueIdentifier.absoluteTableIdentifier)
-        && 
this.tableBlockInfo.equals(tableBlockUniqueIdentifier.tableBlockInfo);
-  }
-
-  /**
-   * returns the String value to uniquely identify a block
-   *
-   * @return
-   */
-  public String getUniqueTableBlockName() {
-    BlockInfo blockInfo = new BlockInfo(this.tableBlockInfo);
-    CarbonTableIdentifier carbonTableIdentifier =
-        this.absoluteTableIdentifier.getCarbonTableIdentifier();
-    String uniqueTableBlockName = carbonTableIdentifier.getDatabaseName()
-        + CarbonCommonConstants.FILE_SEPARATOR + carbonTableIdentifier
-        .getDatabaseName() + CarbonCommonConstants.FILE_SEPARATOR
-        + this.tableBlockInfo.getSegmentId()
-        + CarbonCommonConstants.FILE_SEPARATOR + blockInfo.hashCode();
-    return uniqueTableBlockName;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/ce09aaaf/core/src/main/java/org/apache/carbondata/core/carbon/datastore/block/TableTaskInfo.java
----------------------------------------------------------------------
diff --git 
a/core/src/main/java/org/apache/carbondata/core/carbon/datastore/block/TableTaskInfo.java
 
b/core/src/main/java/org/apache/carbondata/core/carbon/datastore/block/TableTaskInfo.java
deleted file mode 100644
index da2cff8..0000000
--- 
a/core/src/main/java/org/apache/carbondata/core/carbon/datastore/block/TableTaskInfo.java
+++ /dev/null
@@ -1,115 +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.block;
-
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Map;
-import java.util.Objects;
-import java.util.Set;
-import java.util.TreeMap;
-
-/**
- * This class is responsible for maintaining the mapping of tasks of a node.
- */
-public class TableTaskInfo implements Distributable {
-
-  private final List<TableBlockInfo> tableBlockInfoList;
-  private final String taskId;
-  public String getTaskId() {
-    return taskId;
-  }
-
-  public List<TableBlockInfo> getTableBlockInfoList() {
-    return tableBlockInfoList;
-  }
-
-  public TableTaskInfo(String taskId, List<TableBlockInfo> tableBlockInfoList){
-    this.taskId = taskId;
-    this.tableBlockInfoList = tableBlockInfoList;
-  }
-
-  @Override public String[] getLocations() {
-    Set<String> locations = new HashSet<String>();
-    for(TableBlockInfo tableBlockInfo: tableBlockInfoList){
-      locations.addAll(Arrays.asList(tableBlockInfo.getLocations()));
-    }
-    locations.toArray(new String[locations.size()]);
-    List<String> nodes =  TableTaskInfo.maxNoNodes(tableBlockInfoList);
-    return nodes.toArray(new String[nodes.size()]);
-  }
-
-  @Override public int compareTo(Distributable o) {
-    return taskId.compareTo(((TableTaskInfo)o).getTaskId());
-  }
-
-  /**
-   * Finding which node has the maximum number of blocks for it.
-   * @param blockList
-   * @return
-   */
-  public static List<String> maxNoNodes(List<TableBlockInfo> blockList) {
-    boolean useIndex = true;
-    Integer maxOccurence = 0;
-    String maxNode = null;
-    Map<String, Integer> nodeAndOccurenceMapping = new TreeMap<>();
-
-    // populate the map of node and number of occurences of that node.
-    for (TableBlockInfo block : blockList) {
-      for (String node : block.getLocations()) {
-        Integer nodeOccurence = nodeAndOccurenceMapping.get(node);
-        if (null == nodeOccurence) {
-          nodeAndOccurenceMapping.put(node, 1);
-        } else {
-          nodeOccurence++;
-        }
-      }
-    }
-    Integer previousValueOccurence = null;
-
-    // check which node is occured maximum times.
-    for (Map.Entry<String, Integer> entry : 
nodeAndOccurenceMapping.entrySet()) {
-      // finding the maximum node.
-      if (entry.getValue() > maxOccurence) {
-        maxOccurence = entry.getValue();
-        maxNode = entry.getKey();
-      }
-      // first time scenario. initialzing the previous value.
-      if (null == previousValueOccurence) {
-        previousValueOccurence = entry.getValue();
-      } else {
-        // for the case where all the nodes have same number of blocks then
-        // we need to return complete list instead of max node.
-        if (!Objects.equals(previousValueOccurence, entry.getValue())) {
-          useIndex = false;
-        }
-      }
-    }
-
-    // if all the nodes have equal occurence then returning the complete key 
set.
-    if (useIndex) {
-      return new ArrayList<>(nodeAndOccurenceMapping.keySet());
-    }
-
-    // if any max node is found then returning the max node.
-    List<String> node =  new ArrayList<>(1);
-    node.add(maxNode);
-    return node;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/ce09aaaf/core/src/main/java/org/apache/carbondata/core/carbon/datastore/block/TaskBlockInfo.java
----------------------------------------------------------------------
diff --git 
a/core/src/main/java/org/apache/carbondata/core/carbon/datastore/block/TaskBlockInfo.java
 
b/core/src/main/java/org/apache/carbondata/core/carbon/datastore/block/TaskBlockInfo.java
deleted file mode 100644
index 1221cc1..0000000
--- 
a/core/src/main/java/org/apache/carbondata/core/carbon/datastore/block/TaskBlockInfo.java
+++ /dev/null
@@ -1,68 +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.block;
-
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-
-import org.apache.carbondata.core.constants.CarbonCommonConstants;
-
-/**
- * This class contains blocks info of each task
- */
-public class TaskBlockInfo {
-
-  // stores TableBlockInfo list of each task
-  private Map<String, List<TableBlockInfo>> taskBlockInfoMapping;
-
-  public TaskBlockInfo(){
-
-    taskBlockInfoMapping = new 
HashMap<>(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE);
-  }
-
-  /**
-   * returns task set
-   * @return
-   */
-  public Set<String> getTaskSet() {
-    return taskBlockInfoMapping.keySet();
-  }
-
-
-  /**
-   * returns TableBlockInfoList of given task
-   * @return
-   */
-  public List<TableBlockInfo> getTableBlockInfoList(String task) {
-    return taskBlockInfoMapping.get(task);
-  }
-
-  /**
-   *  maps TableBlockInfoList to respective task
-   * @param task
-   * @param tableBlockInfoList
-   */
-  public void addTableBlockInfoList(String task, List<TableBlockInfo> 
tableBlockInfoList) {
-    taskBlockInfoMapping.put(task, tableBlockInfoList);
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/ce09aaaf/core/src/main/java/org/apache/carbondata/core/carbon/datastore/chunk/DimensionColumnDataChunk.java
----------------------------------------------------------------------
diff --git 
a/core/src/main/java/org/apache/carbondata/core/carbon/datastore/chunk/DimensionColumnDataChunk.java
 
b/core/src/main/java/org/apache/carbondata/core/carbon/datastore/chunk/DimensionColumnDataChunk.java
deleted file mode 100644
index efaa48b..0000000
--- 
a/core/src/main/java/org/apache/carbondata/core/carbon/datastore/chunk/DimensionColumnDataChunk.java
+++ /dev/null
@@ -1,111 +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.chunk;
-
-import org.apache.carbondata.scan.executor.infos.KeyStructureInfo;
-import org.apache.carbondata.scan.result.vector.ColumnVectorInfo;
-
-/**
- * Interface for dimension column chunk.
- */
-public interface DimensionColumnDataChunk {
-
-  /**
-   * Below method will be used to fill the data based on offset and row id
-   *
-   * @param data   data to filed
-   * @param offset offset from which data need to be filed
-   * @return how many bytes was copied
-   */
-  int fillChunkData(byte[] data, int offset, int columnIndex, KeyStructureInfo 
restructuringInfo);
-
-  /**
-   * It uses to convert column data to dictionary integer value
-   *
-   * @param rowId
-   * @param columnIndex
-   * @param row
-   * @param restructuringInfo @return
-   */
-  int fillConvertedChunkData(int rowId, int columnIndex, int[] row,
-      KeyStructureInfo restructuringInfo);
-
-  /**
-   * Fill the data to vector
-   * @param vectorInfo
-   * @param column
-   * @param restructuringInfo
-   * @return next column index
-   */
-  int fillConvertedChunkData(ColumnVectorInfo[] vectorInfo, int column,
-      KeyStructureInfo restructuringInfo);
-
-  /**
-   * Fill the data to vector
-   * @param rowMapping
-   * @param vectorInfo
-   * @param column
-   * @param restructuringInfo
-   * @return next column index
-   */
-  int fillConvertedChunkData(int[] rowMapping, ColumnVectorInfo[] vectorInfo, 
int column,
-      KeyStructureInfo restructuringInfo);
-
-  /**
-   * Below method to get  the data based in row id
-   *
-   * @return chunk
-   */
-  byte[] getChunkData(int columnIndex);
-
-  /**
-   * @return inverted index
-   */
-  int getInvertedIndex(int index);
-
-  /**
-   * @return whether column is dictionary column or not
-   */
-  boolean isNoDicitionaryColumn();
-
-  /**
-   * @return length of each column
-   */
-  int getColumnValueSize();
-
-  /**
-   * @return whether columns where explictly sorted or not
-   */
-  boolean isExplicitSorted();
-
-  /**
-   * to compare the data
-   *
-   * @param index        row index to be compared
-   * @param compareValue value to compare
-   * @return compare result
-   */
-  int compareTo(int index, byte[] compareValue);
-
-  /**
-   * below method will be used to free the allocated memory
-   */
-  void freeMemory();
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/ce09aaaf/core/src/main/java/org/apache/carbondata/core/carbon/datastore/chunk/MeasureColumnDataChunk.java
----------------------------------------------------------------------
diff --git 
a/core/src/main/java/org/apache/carbondata/core/carbon/datastore/chunk/MeasureColumnDataChunk.java
 
b/core/src/main/java/org/apache/carbondata/core/carbon/datastore/chunk/MeasureColumnDataChunk.java
deleted file mode 100644
index e4e0d6b..0000000
--- 
a/core/src/main/java/org/apache/carbondata/core/carbon/datastore/chunk/MeasureColumnDataChunk.java
+++ /dev/null
@@ -1,74 +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.chunk;
-
-import 
org.apache.carbondata.core.carbon.metadata.blocklet.datachunk.PresenceMeta;
-import 
org.apache.carbondata.core.datastorage.store.dataholder.CarbonReadDataHolder;
-
-/**
- * Holder for measure column chunk
- * it will have data and its attributes which will
- * be required for processing
- */
-public class MeasureColumnDataChunk {
-
-  /**
-   * measure chunk
-   */
-  private CarbonReadDataHolder measureDataHolder;
-
-  /**
-   * below to hold null value holds this information
-   * about the null value index this will be helpful in case of
-   * to remove the null value while aggregation
-   */
-  private PresenceMeta nullValueIndexHolder;
-
-  /**
-   * @return the measureDataHolder
-   */
-  public CarbonReadDataHolder getMeasureDataHolder() {
-    return measureDataHolder;
-  }
-
-  /**
-   * @param measureDataHolder the measureDataHolder to set
-   */
-  public void setMeasureDataHolder(CarbonReadDataHolder measureDataHolder) {
-    this.measureDataHolder = measureDataHolder;
-  }
-
-  /**
-   * @return the nullValueIndexHolder
-   */
-  public PresenceMeta getNullValueIndexHolder() {
-    return nullValueIndexHolder;
-  }
-
-  /**
-   * @param nullValueIndexHolder the nullValueIndexHolder to set
-   */
-  public void setNullValueIndexHolder(PresenceMeta nullValueIndexHolder) {
-    this.nullValueIndexHolder = nullValueIndexHolder;
-  }
-
-  public void freeMemory() {
-    this.measureDataHolder.freeMemory();
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/ce09aaaf/core/src/main/java/org/apache/carbondata/core/carbon/datastore/chunk/impl/AbstractDimensionDataChunk.java
----------------------------------------------------------------------
diff --git 
a/core/src/main/java/org/apache/carbondata/core/carbon/datastore/chunk/impl/AbstractDimensionDataChunk.java
 
b/core/src/main/java/org/apache/carbondata/core/carbon/datastore/chunk/impl/AbstractDimensionDataChunk.java
deleted file mode 100644
index 8f6c284..0000000
--- 
a/core/src/main/java/org/apache/carbondata/core/carbon/datastore/chunk/impl/AbstractDimensionDataChunk.java
+++ /dev/null
@@ -1,90 +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.chunk.impl;
-
-import 
org.apache.carbondata.core.carbon.datastore.chunk.DimensionColumnDataChunk;
-import 
org.apache.carbondata.core.carbon.datastore.chunk.store.DimensionDataChunkStore;
-
-/**
- * Class responsibility is to give access to dimension column data chunk store
- */
-public abstract class AbstractDimensionDataChunk implements 
DimensionColumnDataChunk {
-
-  /**
-   * data chunks
-   */
-  protected DimensionDataChunkStore dataChunkStore;
-
-  /**
-   * @return whether columns where explicitly sorted or not
-   */
-  @Override public boolean isExplicitSorted() {
-    return dataChunkStore.isExplicitSorted();
-  }
-
-  /**
-   * Below method to get the data based in row id
-   *
-   * @param index row id of the data
-   * @return chunk
-   */
-  @Override public byte[] getChunkData(int index) {
-    return dataChunkStore.getRow(index);
-  }
-
-  /**
-   * @return inverted index
-   */
-  @Override public int getInvertedIndex(int index) {
-    return dataChunkStore.getInvertedIndex(index);
-  }
-
-  /**
-   * @return length of each column
-   */
-  @Override public int getColumnValueSize() {
-    return dataChunkStore.getColumnValueSize();
-  }
-
-  /**
-   * To compare the data
-   *
-   * @param index        row index to be compared
-   * @param compareValue value to compare
-   * @return compare result
-   */
-  @Override public int compareTo(int index, byte[] compareValue) {
-    // TODO Auto-generated method stub
-    return dataChunkStore.compareTo(index, compareValue);
-  }
-
-  /**
-   * below method will be used to free the allocated memory
-   */
-  @Override public void freeMemory() {
-    dataChunkStore.freeMemory();
-  }
-
-  /**
-   * @return column is dictionary column or not
-   */
-  @Override public boolean isNoDicitionaryColumn() {
-    return false;
-  }
-}

Reply via email to