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

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

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/cd6a4ff3/core/src/main/java/org/carbondata/core/carbon/datastore/block/SegmentProperties.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/core/carbon/datastore/block/SegmentProperties.java b/core/src/main/java/org/carbondata/core/carbon/datastore/block/SegmentProperties.java
deleted file mode 100644
index df99b4b..0000000
--- a/core/src/main/java/org/carbondata/core/carbon/datastore/block/SegmentProperties.java
+++ /dev/null
@@ -1,748 +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.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.carbondata.core.carbon.metadata.datatype.DataType;
-import org.carbondata.core.carbon.metadata.encoder.Encoding;
-import org.carbondata.core.carbon.metadata.schema.table.column.CarbonDimension;
-import org.carbondata.core.carbon.metadata.schema.table.column.CarbonMeasure;
-import org.carbondata.core.carbon.metadata.schema.table.column.ColumnSchema;
-import org.carbondata.core.constants.CarbonCommonConstants;
-import org.carbondata.core.datastorage.store.columnar.ColumnGroupModel;
-import org.carbondata.core.keygenerator.KeyGenerator;
-import org.carbondata.core.keygenerator.columnar.ColumnarSplitter;
-import org.carbondata.core.keygenerator.columnar.impl.MultiDimKeyVarLengthVariableSplitGenerator;
-import org.carbondata.core.keygenerator.mdkey.MultiDimKeyVarLengthGenerator;
-import org.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(columnsInTable);
-    fillOrdinalToBlockMappingForDimension();
-    fillOrdinalToBlockIndexMappingForMeasureColumns();
-    fillColumnGroupAndItsCardinality(columnCardinality);
-    fillKeyGeneratorDetails();
-  }
-
-  /**
-   * it fills column groups
-   * e.g {{1},{2,3,4},{5},{6},{7,8,9}}
-   *
-   * @param columnsInTable
-   */
-  private void intialiseColGroups(List<ColumnSchema> columnsInTable) {
-    // 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);
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/cd6a4ff3/core/src/main/java/org/carbondata/core/carbon/datastore/block/SegmentTaskIndex.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/core/carbon/datastore/block/SegmentTaskIndex.java b/core/src/main/java/org/carbondata/core/carbon/datastore/block/SegmentTaskIndex.java
deleted file mode 100644
index 7e415ac..0000000
--- a/core/src/main/java/org/carbondata/core/carbon/datastore/block/SegmentTaskIndex.java
+++ /dev/null
@@ -1,58 +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.carbondata.core.carbon.datastore.block;
-
-import java.util.List;
-
-import org.carbondata.core.carbon.datastore.BTreeBuilderInfo;
-import org.carbondata.core.carbon.datastore.BtreeBuilder;
-import org.carbondata.core.carbon.datastore.impl.btree.BlockBTreeBuilder;
-import org.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
-   *
-   * @param blockInfo block detail
-   */
-  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/cd6a4ff3/core/src/main/java/org/carbondata/core/carbon/datastore/block/TableBlockInfo.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/core/carbon/datastore/block/TableBlockInfo.java b/core/src/main/java/org/carbondata/core/carbon/datastore/block/TableBlockInfo.java
deleted file mode 100644
index cd0e831..0000000
--- a/core/src/main/java/org/carbondata/core/carbon/datastore/block/TableBlockInfo.java
+++ /dev/null
@@ -1,204 +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.carbondata.core.carbon.datastore.block;
-
-import java.io.Serializable;
-import java.util.Arrays;
-
-import org.carbondata.core.carbon.path.CarbonTablePath;
-import org.carbondata.core.carbon.path.CarbonTablePath.DataFileUtil;
-import org.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 extends Distributable
-    implements Serializable, Comparable<Distributable> {
-
-  /**
-   * 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;
-
-
-  public TableBlockInfo(String filePath, long blockOffset, String segmentId, String[] locations,
-      long blockLength) {
-    this.filePath = FileFactory.getUpdatedFilePath(filePath);
-    this.blockOffset = blockOffset;
-    this.segmentId = segmentId;
-    this.locations = locations;
-    this.blockLength = blockLength;
-  }
-
-  /**
-   * @return the filePath
-   */
-  public String getFilePath() {
-    return filePath;
-  }
-
-  /**
-   * @return the blockOffset
-   */
-  public long getBlockOffset() {
-    return 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) {
-      if (other.filePath != null) {
-        return false;
-      }
-    } else if (!filePath.equals(other.filePath)) {
-      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;
-    }
-    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 + Arrays.hashCode(locations);
-    return result;
-  }
-
-  @Override public String[] getLocations() {
-    return locations;
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/cd6a4ff3/core/src/main/java/org/carbondata/core/carbon/datastore/block/TableTaskInfo.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/core/carbon/datastore/block/TableTaskInfo.java b/core/src/main/java/org/carbondata/core/carbon/datastore/block/TableTaskInfo.java
deleted file mode 100644
index 0def27b..0000000
--- a/core/src/main/java/org/carbondata/core/carbon/datastore/block/TableTaskInfo.java
+++ /dev/null
@@ -1,114 +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.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.Set;
-import java.util.TreeMap;
-
-/**
- * This class is responsible for maintaining the mapping of tasks of a node.
- */
-public class TableTaskInfo extends 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 (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/cd6a4ff3/core/src/main/java/org/carbondata/core/carbon/datastore/block/TaskBlockInfo.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/core/carbon/datastore/block/TaskBlockInfo.java b/core/src/main/java/org/carbondata/core/carbon/datastore/block/TaskBlockInfo.java
deleted file mode 100644
index 3417f59..0000000
--- a/core/src/main/java/org/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.carbondata.core.carbon.datastore.block;
-
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-
-import org.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/cd6a4ff3/core/src/main/java/org/carbondata/core/carbon/datastore/chunk/DimensionChunkAttributes.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/core/carbon/datastore/chunk/DimensionChunkAttributes.java b/core/src/main/java/org/carbondata/core/carbon/datastore/chunk/DimensionChunkAttributes.java
deleted file mode 100644
index 997f54d..0000000
--- a/core/src/main/java/org/carbondata/core/carbon/datastore/chunk/DimensionChunkAttributes.java
+++ /dev/null
@@ -1,102 +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.carbondata.core.carbon.datastore.chunk;
-
-/**
- * Dimension chunk attributes which holds all the
- * property about the dimension chunk data
- */
-public class DimensionChunkAttributes {
-
-  /**
-   * inverted index of the data
-   */
-  private int[] invertedIndexes;
-
-  /**
-   * reverse index of the data
-   */
-  private int[] invertedIndexesReverse;
-
-  /**
-   * each row size
-   */
-  private int columnValueSize;
-
-  /**
-   * is no dictionary
-   */
-  private boolean isNoDictionary;
-
-  /**
-   * @return the invertedIndexes
-   */
-  public int[] getInvertedIndexes() {
-    return invertedIndexes;
-  }
-
-  /**
-   * @param invertedIndexes the invertedIndexes to set
-   */
-  public void setInvertedIndexes(int[] invertedIndexes) {
-    this.invertedIndexes = invertedIndexes;
-  }
-
-  /**
-   * @return the invertedIndexesReverse
-   */
-  public int[] getInvertedIndexesReverse() {
-    return invertedIndexesReverse;
-  }
-
-  /**
-   * @param invertedIndexesReverse the invertedIndexesReverse to set
-   */
-  public void setInvertedIndexesReverse(int[] invertedIndexesReverse) {
-    this.invertedIndexesReverse = invertedIndexesReverse;
-  }
-
-  /**
-   * @return the eachRowSize
-   */
-  public int getColumnValueSize() {
-    return columnValueSize;
-  }
-
-  /**
-   * @param eachRowSize the eachRowSize to set
-   */
-  public void setEachRowSize(int eachRowSize) {
-    this.columnValueSize = eachRowSize;
-  }
-
-  /**
-   * @return the isNoDictionary
-   */
-  public boolean isNoDictionary() {
-    return isNoDictionary;
-  }
-
-  /**
-   * @param isNoDictionary the isNoDictionary to set
-   */
-  public void setNoDictionary(boolean isNoDictionary) {
-    this.isNoDictionary = isNoDictionary;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/cd6a4ff3/core/src/main/java/org/carbondata/core/carbon/datastore/chunk/DimensionColumnDataChunk.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/core/carbon/datastore/chunk/DimensionColumnDataChunk.java b/core/src/main/java/org/carbondata/core/carbon/datastore/chunk/DimensionColumnDataChunk.java
deleted file mode 100644
index 6a30bc7..0000000
--- a/core/src/main/java/org/carbondata/core/carbon/datastore/chunk/DimensionColumnDataChunk.java
+++ /dev/null
@@ -1,71 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.carbondata.core.carbon.datastore.chunk;
-
-import org.carbondata.scan.executor.infos.KeyStructureInfo;
-
-/**
- * Interface for dimension column chunk.
- */
-public interface DimensionColumnDataChunk<T> {
-
-  /**
-   * 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
-   * @param rowId  row id of the chunk
-   * @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);
-
-  /**
-   * Below method to get  the data based in row id
-   *
-   * @param row id
-   *            row id of the data
-   * @return chunk
-   */
-  byte[] getChunkData(int columnIndex);
-
-  /**
-   * Below method will be used get the chunk attributes
-   *
-   * @return chunk attributes
-   */
-  DimensionChunkAttributes getAttributes();
-
-  /**
-   * Below method will be used to return the complete data chunk
-   * This will be required during filter query
-   *
-   * @return complete chunk
-   */
-  T getCompleteDataChunk();
-}

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

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/cd6a4ff3/core/src/main/java/org/carbondata/core/carbon/datastore/chunk/impl/ColumnGroupDimensionDataChunk.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/core/carbon/datastore/chunk/impl/ColumnGroupDimensionDataChunk.java b/core/src/main/java/org/carbondata/core/carbon/datastore/chunk/impl/ColumnGroupDimensionDataChunk.java
deleted file mode 100644
index d8536a1..0000000
--- a/core/src/main/java/org/carbondata/core/carbon/datastore/chunk/impl/ColumnGroupDimensionDataChunk.java
+++ /dev/null
@@ -1,128 +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.carbondata.core.carbon.datastore.chunk.impl;
-
-import org.carbondata.core.carbon.datastore.chunk.DimensionChunkAttributes;
-import org.carbondata.core.carbon.datastore.chunk.DimensionColumnDataChunk;
-import org.carbondata.scan.executor.infos.KeyStructureInfo;
-
-/**
- * This class is holder of the dimension column chunk data of the fixed length
- * key size
- */
-public class ColumnGroupDimensionDataChunk implements DimensionColumnDataChunk<byte[]> {
-
-  /**
-   * dimension chunk attributes
-   */
-  private DimensionChunkAttributes chunkAttributes;
-
-  /**
-   * data chunks
-   */
-  private byte[] dataChunk;
-
-  /**
-   * Constructor for this class
-   *
-   * @param dataChunk       data chunk
-   * @param chunkAttributes chunk attributes
-   */
-  public ColumnGroupDimensionDataChunk(byte[] dataChunk, DimensionChunkAttributes chunkAttributes) {
-    this.chunkAttributes = chunkAttributes;
-    this.dataChunk = dataChunk;
-  }
-
-  /**
-   * 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
-   * @param rowId            row id of the chunk
-   * @param restructuringInfo define the structure of the key
-   * @return how many bytes was copied
-   */
-  @Override public int fillChunkData(byte[] data, int offset, int rowId,
-      KeyStructureInfo restructuringInfo) {
-    byte[] maskedKey =
-        getMaskedKey(dataChunk, rowId * chunkAttributes.getColumnValueSize(), restructuringInfo);
-    System.arraycopy(maskedKey, 0, data, offset, maskedKey.length);
-    return maskedKey.length;
-  }
-
-  /**
-   * Converts to column dictionary integer value
-   */
-  @Override public int fillConvertedChunkData(int rowId, int columnIndex, int[] row,
-      KeyStructureInfo info) {
-    int start = rowId * chunkAttributes.getColumnValueSize();
-    long[] keyArray = info.getKeyGenerator().getKeyArray(dataChunk, start);
-    int[] ordinal = info.getMdkeyQueryDimensionOrdinal();
-    for (int i = 0; i < ordinal.length; i++) {
-      row[columnIndex++] = (int)keyArray[ordinal[i]];
-    }
-    return columnIndex;
-  }
-
-  /**
-   * Below method masks key
-   *
-   */
-  public byte[] getMaskedKey(byte[] data, int offset, KeyStructureInfo info) {
-    byte[] maskedKey = new byte[info.getMaskByteRanges().length];
-    int counter = 0;
-    int byteRange = 0;
-    for (int i = 0; i < info.getMaskByteRanges().length; i++) {
-      byteRange = info.getMaskByteRanges()[i];
-      maskedKey[counter++] = (byte) (data[byteRange + offset] & info.getMaxKey()[byteRange]);
-    }
-    return maskedKey;
-  }
-
-  /**
-   * Below method to get the data based in row id
-   *
-   * @param rowId row id of the data
-   * @return chunk
-   */
-  @Override public byte[] getChunkData(int rowId) {
-    byte[] data = new byte[chunkAttributes.getColumnValueSize()];
-    System.arraycopy(dataChunk, rowId * data.length, data, 0, data.length);
-    return data;
-  }
-
-  /**
-   * Below method will be used get the chunk attributes
-   *
-   * @return chunk attributes
-   */
-  @Override public DimensionChunkAttributes getAttributes() {
-    return chunkAttributes;
-  }
-
-  /**
-   * Below method will be used to return the complete data chunk
-   * This will be required during filter query
-   *
-   * @return complete chunk
-   */
-  @Override public byte[] getCompleteDataChunk() {
-    return dataChunk;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/cd6a4ff3/core/src/main/java/org/carbondata/core/carbon/datastore/chunk/impl/FixedLengthDimensionDataChunk.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/core/carbon/datastore/chunk/impl/FixedLengthDimensionDataChunk.java b/core/src/main/java/org/carbondata/core/carbon/datastore/chunk/impl/FixedLengthDimensionDataChunk.java
deleted file mode 100644
index cbe48b2..0000000
--- a/core/src/main/java/org/carbondata/core/carbon/datastore/chunk/impl/FixedLengthDimensionDataChunk.java
+++ /dev/null
@@ -1,123 +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.carbondata.core.carbon.datastore.chunk.impl;
-
-import org.carbondata.core.carbon.datastore.chunk.DimensionChunkAttributes;
-import org.carbondata.core.carbon.datastore.chunk.DimensionColumnDataChunk;
-import org.carbondata.scan.executor.infos.KeyStructureInfo;
-
-/**
- * This class is holder of the dimension column chunk data of the fixed length
- * key size
- */
-public class FixedLengthDimensionDataChunk implements DimensionColumnDataChunk<byte[]> {
-
-  /**
-   * dimension chunk attributes
-   */
-  private DimensionChunkAttributes chunkAttributes;
-
-  /**
-   * data chunks
-   */
-  private byte[] dataChunk;
-
-  /**
-   * Constructor for this class
-   *
-   * @param dataChunk       data chunk
-   * @param chunkAttributes chunk attributes
-   */
-  public FixedLengthDimensionDataChunk(byte[] dataChunk, DimensionChunkAttributes chunkAttributes) {
-    this.chunkAttributes = chunkAttributes;
-    this.dataChunk = dataChunk;
-  }
-
-  /**
-   * 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
-   * @param index            row id of the chunk
-   * @param keyStructureInfo define the structure of the key
-   * @return how many bytes was copied
-   */
-  @Override public int fillChunkData(byte[] data, int offset, int index,
-      KeyStructureInfo keyStructureInfo) {
-    if (chunkAttributes.getInvertedIndexes() != null) {
-      index = chunkAttributes.getInvertedIndexesReverse()[index];
-    }
-    System.arraycopy(dataChunk, index * chunkAttributes.getColumnValueSize(), data, offset,
-        chunkAttributes.getColumnValueSize());
-    return chunkAttributes.getColumnValueSize();
-  }
-
-  /**
-   * Converts to column dictionary integer value
-   */
-  @Override public int fillConvertedChunkData(int rowId, int columnIndex, int[] row,
-      KeyStructureInfo restructuringInfo) {
-    if (chunkAttributes.getInvertedIndexes() != null) {
-      rowId = chunkAttributes.getInvertedIndexesReverse()[rowId];
-    }
-    int start = rowId * chunkAttributes.getColumnValueSize();
-    int dict = 0;
-    for (int i = start; i < start + chunkAttributes.getColumnValueSize(); i++) {
-      dict <<= 8;
-      dict ^= dataChunk[i] & 0xFF;
-    }
-    row[columnIndex] = dict;
-    return columnIndex + 1;
-  }
-
-  /**
-   * 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) {
-    byte[] data = new byte[chunkAttributes.getColumnValueSize()];
-    if (chunkAttributes.getInvertedIndexes() != null) {
-      index = chunkAttributes.getInvertedIndexesReverse()[index];
-    }
-    System.arraycopy(dataChunk, index * chunkAttributes.getColumnValueSize(), data, 0,
-        chunkAttributes.getColumnValueSize());
-    return data;
-  }
-
-  /**
-   * Below method will be used get the chunk attributes
-   *
-   * @return chunk attributes
-   */
-  @Override public DimensionChunkAttributes getAttributes() {
-    return chunkAttributes;
-  }
-
-  /**
-   * Below method will be used to return the complete data chunk
-   * This will be required during filter query
-   *
-   * @return complete chunk
-   */
-  @Override public byte[] getCompleteDataChunk() {
-    return dataChunk;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/cd6a4ff3/core/src/main/java/org/carbondata/core/carbon/datastore/chunk/impl/VariableLengthDimensionDataChunk.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/core/carbon/datastore/chunk/impl/VariableLengthDimensionDataChunk.java b/core/src/main/java/org/carbondata/core/carbon/datastore/chunk/impl/VariableLengthDimensionDataChunk.java
deleted file mode 100644
index 7c69490..0000000
--- a/core/src/main/java/org/carbondata/core/carbon/datastore/chunk/impl/VariableLengthDimensionDataChunk.java
+++ /dev/null
@@ -1,114 +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.carbondata.core.carbon.datastore.chunk.impl;
-
-import java.util.List;
-
-import org.carbondata.core.carbon.datastore.chunk.DimensionChunkAttributes;
-import org.carbondata.core.carbon.datastore.chunk.DimensionColumnDataChunk;
-import org.carbondata.scan.executor.infos.KeyStructureInfo;
-
-/**
- * This class is holder of the dimension column chunk data of the variable
- * length key size
- */
-public class VariableLengthDimensionDataChunk implements DimensionColumnDataChunk<List<byte[]>> {
-
-  /**
-   * dimension chunk attributes
-   */
-  private DimensionChunkAttributes chunkAttributes;
-
-  /**
-   * data chunk
-   */
-  private List<byte[]> dataChunk;
-
-  /**
-   * Constructor for this class
-   *
-   * @param dataChunk       data chunk
-   * @param chunkAttributes chunk attributes
-   */
-  public VariableLengthDimensionDataChunk(List<byte[]> dataChunk,
-      DimensionChunkAttributes chunkAttributes) {
-    this.chunkAttributes = chunkAttributes;
-    this.dataChunk = dataChunk;
-  }
-
-  /**
-   * 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
-   * @param index            row id of the chunk
-   * @param restructuringInfo define the structure of the key
-   * @return how many bytes was copied
-   */
-  @Override public int fillChunkData(byte[] data, int offset, int index,
-      KeyStructureInfo restructuringInfo) {
-    // no required in this case because this column chunk is not the part if
-    // mdkey
-    return 0;
-  }
-
-  /**
-   * Converts to column dictionary integer value
-   * @param rowId
-   * @param columnIndex
-   * @param row
-   * @param restructuringInfo  @return
-   */
-  @Override public int fillConvertedChunkData(int rowId, int columnIndex, int[] row,
-      KeyStructureInfo restructuringInfo) {
-    return columnIndex + 1;
-  }
-
-  /**
-   * 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) {
-    if (null != chunkAttributes.getInvertedIndexes()) {
-      index = chunkAttributes.getInvertedIndexesReverse()[index];
-    }
-    return dataChunk.get(index);
-  }
-
-  /**
-   * Below method will be used get the chunk attributes
-   *
-   * @return chunk attributes
-   */
-  @Override public DimensionChunkAttributes getAttributes() {
-    return chunkAttributes;
-  }
-
-  /**
-   * Below method will be used to return the complete data chunk
-   * This will be required during filter query
-   *
-   * @return complete chunk
-   */
-  @Override public List<byte[]> getCompleteDataChunk() {
-    return dataChunk;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/cd6a4ff3/core/src/main/java/org/carbondata/core/carbon/datastore/chunk/reader/DimensionColumnChunkReader.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/core/carbon/datastore/chunk/reader/DimensionColumnChunkReader.java b/core/src/main/java/org/carbondata/core/carbon/datastore/chunk/reader/DimensionColumnChunkReader.java
deleted file mode 100644
index bf931a0..0000000
--- a/core/src/main/java/org/carbondata/core/carbon/datastore/chunk/reader/DimensionColumnChunkReader.java
+++ /dev/null
@@ -1,48 +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.carbondata.core.carbon.datastore.chunk.reader;
-
-import org.carbondata.core.carbon.datastore.chunk.DimensionColumnDataChunk;
-import org.carbondata.core.datastorage.store.FileHolder;
-
-/**
- * Interface for reading the data chunk
- * Its concrete implementation can be used to read the chunk.
- * compressed or uncompressed chunk
- */
-public interface DimensionColumnChunkReader {
-
-  /**
-   * Below method will be used to read the chunk based on block indexes
-   *
-   * @param fileReader   file reader to read the blocks from file
-   * @param blockIndexes blocks to be read
-   * @return dimension column chunks
-   */
-  DimensionColumnDataChunk[] readDimensionChunks(FileHolder fileReader, int... blockIndexes);
-
-  /**
-   * Below method will be used to read the chunk based on block index
-   *
-   * @param fileReader file reader to read the blocks from file
-   * @param blockIndex block to be read
-   * @return dimension column chunk
-   */
-  DimensionColumnDataChunk readDimensionChunk(FileHolder fileReader, int blockIndex);
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/cd6a4ff3/core/src/main/java/org/carbondata/core/carbon/datastore/chunk/reader/MeasureColumnChunkReader.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/core/carbon/datastore/chunk/reader/MeasureColumnChunkReader.java b/core/src/main/java/org/carbondata/core/carbon/datastore/chunk/reader/MeasureColumnChunkReader.java
deleted file mode 100644
index 37f20c9..0000000
--- a/core/src/main/java/org/carbondata/core/carbon/datastore/chunk/reader/MeasureColumnChunkReader.java
+++ /dev/null
@@ -1,47 +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.carbondata.core.carbon.datastore.chunk.reader;
-
-import org.carbondata.core.carbon.datastore.chunk.MeasureColumnDataChunk;
-import org.carbondata.core.datastorage.store.FileHolder;
-
-/**
- * Reader interface for reading the measure blocks from file
- */
-public interface MeasureColumnChunkReader {
-
-  /**
-   * Method to read the blocks data based on block indexes
-   *
-   * @param fileReader   file reader to read the blocks
-   * @param blockIndexes blocks to be read
-   * @return measure data chunks
-   */
-  MeasureColumnDataChunk[] readMeasureChunks(FileHolder fileReader, int... blockIndexes);
-
-  /**
-   * Method to read the blocks data based on block index
-   *
-   * @param fileReader file reader to read the blocks
-   * @param blockIndex block to be read
-   * @return measure data chunk
-   */
-  MeasureColumnDataChunk readMeasureChunk(FileHolder fileReader, int blockIndex);
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/cd6a4ff3/core/src/main/java/org/carbondata/core/carbon/datastore/chunk/reader/dimension/AbstractChunkReader.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/core/carbon/datastore/chunk/reader/dimension/AbstractChunkReader.java b/core/src/main/java/org/carbondata/core/carbon/datastore/chunk/reader/dimension/AbstractChunkReader.java
deleted file mode 100644
index f27a3e0..0000000
--- a/core/src/main/java/org/carbondata/core/carbon/datastore/chunk/reader/dimension/AbstractChunkReader.java
+++ /dev/null
@@ -1,143 +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.carbondata.core.carbon.datastore.chunk.reader.dimension;
-
-import java.nio.ByteBuffer;
-import java.util.ArrayList;
-import java.util.List;
-
-import org.carbondata.core.carbon.datastore.chunk.reader.DimensionColumnChunkReader;
-import org.carbondata.core.carbon.metadata.blocklet.datachunk.DataChunk;
-import org.carbondata.core.constants.CarbonCommonConstants;
-import org.carbondata.core.datastorage.store.compression.Compressor;
-import org.carbondata.core.datastorage.store.compression.SnappyCompression;
-import org.carbondata.core.keygenerator.mdkey.NumberCompressor;
-import org.carbondata.core.util.CarbonProperties;
-
-/**
- * Class which will have all the common properties and behavior among all type
- * of reader
- */
-public abstract class AbstractChunkReader implements DimensionColumnChunkReader {
-
-  /**
-   * compressor will be used to uncompress the data
-   */
-  protected static final Compressor<byte[]> COMPRESSOR =
-      SnappyCompression.SnappyByteCompression.INSTANCE;
-
-  /**
-   * data chunk list which holds the information
-   * about the data block metadata
-   */
-  protected List<DataChunk> dimensionColumnChunk;
-
-  /**
-   * size of the each column value
-   * for no dictionary column it will be -1
-   */
-  protected int[] eachColumnValueSize;
-
-  /**
-   * full qualified path of the data file from
-   * which data will be read
-   */
-  protected String filePath;
-
-  /**
-   * this will be used to uncompress the
-   * row id and rle chunk
-   */
-  protected NumberCompressor numberComressor;
-
-  /**
-   * number of element in each chunk
-   */
-  private int numberOfElement;
-
-  /**
-   * Constructor to get minimum parameter to create
-   * instance of this class
-   *
-   * @param dimensionColumnChunk dimension chunk metadata
-   * @param eachColumnValueSize  size of the each column value
-   * @param filePath             file from which data will be read
-   */
-  public AbstractChunkReader(List<DataChunk> dimensionColumnChunk, int[] eachColumnValueSize,
-      String filePath) {
-    this.dimensionColumnChunk = dimensionColumnChunk;
-    this.eachColumnValueSize = eachColumnValueSize;
-    this.filePath = filePath;
-    int numberOfElement = 0;
-    try {
-      numberOfElement = Integer.parseInt(CarbonProperties.getInstance()
-          .getProperty(CarbonCommonConstants.BLOCKLET_SIZE,
-              CarbonCommonConstants.BLOCKLET_SIZE_DEFAULT_VAL));
-    } catch (NumberFormatException exception) {
-      numberOfElement = Integer.parseInt(CarbonCommonConstants.BLOCKLET_SIZE_DEFAULT_VAL);
-    }
-    this.numberComressor = new NumberCompressor(numberOfElement);
-  }
-
-  /**
-   * Below method will be used to create the inverted index reverse
-   * this will be used to point to actual data in the chunk
-   *
-   * @param invertedIndex inverted index
-   * @return reverse inverted index
-   */
-  protected int[] getInvertedReverseIndex(int[] invertedIndex) {
-    int[] columnIndexTemp = new int[invertedIndex.length];
-
-    for (int i = 0; i < invertedIndex.length; i++) {
-      columnIndexTemp[invertedIndex[i]] = i;
-    }
-    return columnIndexTemp;
-  }
-
-  /**
-   * In case of no dictionary column size of the each column value
-   * will not be same, so in case of filter query we can not take
-   * advantage of binary search as length with each value will be also
-   * store with the data, so converting this data to two dimension
-   * array format filter query processing will be faster
-   *
-   * @param dataChunkWithLength no dictionary column chunk
-   *                            <Lenght><Data><Lenght><data>
-   *                            Length will store in 2 bytes
-   * @return list of data chuck, one value in list will represent one column value
-   */
-  protected List<byte[]> getNoDictionaryDataChunk(byte[] dataChunkWithLength) {
-    List<byte[]> dataChunk = new ArrayList<byte[]>(numberOfElement);
-    // wrapping the chunk to byte buffer
-    ByteBuffer buffer = ByteBuffer.wrap(dataChunkWithLength);
-    buffer.rewind();
-    byte[] data = null;
-    // iterating till all the elements are read
-    while (buffer.hasRemaining()) {
-      // as all the data is stored with length(2 bytes)
-      // first reading the size and then based on size
-      // we need to read the actual value
-      data = new byte[buffer.getShort()];
-      buffer.get(data);
-      dataChunk.add(data);
-    }
-    return dataChunk;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/cd6a4ff3/core/src/main/java/org/carbondata/core/carbon/datastore/chunk/reader/dimension/CompressedDimensionChunkFileBasedReader.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/core/carbon/datastore/chunk/reader/dimension/CompressedDimensionChunkFileBasedReader.java b/core/src/main/java/org/carbondata/core/carbon/datastore/chunk/reader/dimension/CompressedDimensionChunkFileBasedReader.java
deleted file mode 100644
index 6e6f566..0000000
--- a/core/src/main/java/org/carbondata/core/carbon/datastore/chunk/reader/dimension/CompressedDimensionChunkFileBasedReader.java
+++ /dev/null
@@ -1,135 +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.carbondata.core.carbon.datastore.chunk.reader.dimension;
-
-import java.util.List;
-
-import org.carbondata.core.carbon.datastore.chunk.DimensionChunkAttributes;
-import org.carbondata.core.carbon.datastore.chunk.DimensionColumnDataChunk;
-import org.carbondata.core.carbon.datastore.chunk.impl.ColumnGroupDimensionDataChunk;
-import org.carbondata.core.carbon.datastore.chunk.impl.FixedLengthDimensionDataChunk;
-import org.carbondata.core.carbon.datastore.chunk.impl.VariableLengthDimensionDataChunk;
-import org.carbondata.core.carbon.metadata.blocklet.datachunk.DataChunk;
-import org.carbondata.core.carbon.metadata.encoder.Encoding;
-import org.carbondata.core.datastorage.store.FileHolder;
-import org.carbondata.core.datastorage.store.columnar.UnBlockIndexer;
-import org.carbondata.core.util.CarbonUtil;
-
-/**
- * Compressed dimension chunk reader class
- */
-public class CompressedDimensionChunkFileBasedReader extends AbstractChunkReader {
-
-  /**
-   * Constructor to get minimum parameter to create instance of this class
-   *
-   * @param dimensionColumnChunk dimension chunk metadata
-   * @param eachColumnValueSize  size of the each column value
-   * @param filePath             file from which data will be read
-   */
-  public CompressedDimensionChunkFileBasedReader(List<DataChunk> dimensionColumnChunk,
-      int[] eachColumnValueSize, String filePath) {
-    super(dimensionColumnChunk, eachColumnValueSize, filePath);
-  }
-
-  /**
-   * Below method will be used to read the chunk based on block indexes
-   *
-   * @param fileReader   file reader to read the blocks from file
-   * @param blockIndexes blocks to be read
-   * @return dimension column chunks
-   */
-  @Override public DimensionColumnDataChunk[] readDimensionChunks(FileHolder fileReader,
-      int... blockIndexes) {
-    // read the column chunk based on block index and add
-    DimensionColumnDataChunk[] dataChunks =
-        new DimensionColumnDataChunk[dimensionColumnChunk.size()];
-    for (int i = 0; i < blockIndexes.length; i++) {
-      dataChunks[blockIndexes[i]] = readDimensionChunk(fileReader, blockIndexes[i]);
-    }
-    return dataChunks;
-  }
-
-  /**
-   * Below method will be used to read the chunk based on block index
-   *
-   * @param fileReader file reader to read the blocks from file
-   * @param blockIndex block to be read
-   * @return dimension column chunk
-   */
-  @Override public DimensionColumnDataChunk readDimensionChunk(FileHolder fileReader,
-      int blockIndex) {
-    byte[] dataPage = null;
-    int[] invertedIndexes = null;
-    int[] invertedIndexesReverse = null;
-    int[] rlePage = null;
-
-    // first read the data and uncompressed it
-    dataPage = COMPRESSOR.unCompress(fileReader
-        .readByteArray(filePath, dimensionColumnChunk.get(blockIndex).getDataPageOffset(),
-            dimensionColumnChunk.get(blockIndex).getDataPageLength()));
-    // if row id block is present then read the row id chunk and uncompress it
-    if (CarbonUtil.hasEncoding(dimensionColumnChunk.get(blockIndex).getEncodingList(),
-        Encoding.INVERTED_INDEX)) {
-      invertedIndexes = CarbonUtil
-          .getUnCompressColumnIndex(dimensionColumnChunk.get(blockIndex).getRowIdPageLength(),
-              fileReader.readByteArray(filePath,
-                  dimensionColumnChunk.get(blockIndex).getRowIdPageOffset(),
-                  dimensionColumnChunk.get(blockIndex).getRowIdPageLength()), numberComressor);
-      // get the reverse index
-      invertedIndexesReverse = getInvertedReverseIndex(invertedIndexes);
-    }
-    // if rle is applied then read the rle block chunk and then uncompress
-    //then actual data based on rle block
-    if (CarbonUtil
-        .hasEncoding(dimensionColumnChunk.get(blockIndex).getEncodingList(), Encoding.RLE)) {
-      // read and uncompress the rle block
-      rlePage = numberComressor.unCompress(fileReader
-          .readByteArray(filePath, dimensionColumnChunk.get(blockIndex).getRlePageOffset(),
-              dimensionColumnChunk.get(blockIndex).getRlePageLength()));
-      // uncompress the data with rle indexes
-      dataPage = UnBlockIndexer.uncompressData(dataPage, rlePage, eachColumnValueSize[blockIndex]);
-      rlePage = null;
-    }
-    // fill chunk attributes
-    DimensionChunkAttributes chunkAttributes = new DimensionChunkAttributes();
-    chunkAttributes.setEachRowSize(eachColumnValueSize[blockIndex]);
-    chunkAttributes.setInvertedIndexes(invertedIndexes);
-    chunkAttributes.setInvertedIndexesReverse(invertedIndexesReverse);
-    DimensionColumnDataChunk columnDataChunk = null;
-
-    if (dimensionColumnChunk.get(blockIndex).isRowMajor()) {
-      // to store fixed length column chunk values
-      columnDataChunk = new ColumnGroupDimensionDataChunk(dataPage, chunkAttributes);
-    }
-    // if no dictionary column then first create a no dictionary column chunk
-    // and set to data chunk instance
-    else if (!CarbonUtil
-        .hasEncoding(dimensionColumnChunk.get(blockIndex).getEncodingList(), Encoding.DICTIONARY)) {
-      columnDataChunk =
-          new VariableLengthDimensionDataChunk(getNoDictionaryDataChunk(dataPage), chunkAttributes);
-      chunkAttributes.setNoDictionary(true);
-    } else {
-      // to store fixed length column chunk values
-      columnDataChunk = new FixedLengthDimensionDataChunk(dataPage, chunkAttributes);
-    }
-    return columnDataChunk;
-  }
-
-}