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/06/30 17:41:57 UTC

[10/50] [abbrv] incubator-carbondata git commit: [BUG]Column group with no dictionary and timestamp (#713)

[BUG]Column group with no dictionary and timestamp (#713)

While data loading, encoding is not proper for all dimension

Fixed query execution for no dictionary and column groups
Fixed test cases and added more test case for column group and dictionary exclude
Fixed review comment

Project: http://git-wip-us.apache.org/repos/asf/incubator-carbondata/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-carbondata/commit/e96de9f5
Tree: http://git-wip-us.apache.org/repos/asf/incubator-carbondata/tree/e96de9f5
Diff: http://git-wip-us.apache.org/repos/asf/incubator-carbondata/diff/e96de9f5

Branch: refs/heads/master
Commit: e96de9f5b9e0560a5e59cc5f941e450d4a7e00f8
Parents: 82332b0
Author: ashokblend <as...@gmail.com>
Authored: Sat Jun 25 14:05:56 2016 +0530
Committer: Kumar Vishal <ku...@gmail.com>
Committed: Sat Jun 25 16:35:56 2016 +0800

----------------------------------------------------------------------
 .../datastore/block/SegmentProperties.java      |  88 ++++-
 .../impl/ColumnGroupDimensionDataChunk.java     |   7 +-
 .../core/util/CarbonMetadataUtil.java           |  36 +-
 .../org/carbondata/core/util/CarbonUtil.java    |  49 ++-
 .../carbondata/core/vo/ColumnGroupModel.java    |  21 -
 .../carbon/executor/infos/KeyStructureInfo.java |  18 -
 .../query/carbon/executor/util/QueryUtil.java   |  35 +-
 .../executer/ColGroupFilterExecuterImpl.java    |  51 ++-
 .../core/writer/CarbonFooterWriterTest.java     |  58 ++-
 .../spark/merger/RowResultMerger.java           |  20 +-
 .../ColumnGroupDataTypesTestCase.scala          |  33 +-
 .../processing/mdkeygen/MDKeyGenStep.java       |  41 +-
 .../store/CarbonFactDataHandlerColumnar.java    | 101 ++---
 .../store/CarbonFactDataHandlerModel.java       |  63 ++-
 .../store/colgroup/ColGroupDataHolder.java      |   4 +-
 .../store/colgroup/ColGroupMinMax.java          |  30 +-
 .../store/writer/AbstractFactDataWriter.java    |   8 +-
 ...actDataWriterImplForIntIndexAndAggBlock.java |   6 +-
 .../store/colgroup/ColGroupMinMaxTest.java      | 380 ++++++++++---------
 19 files changed, 606 insertions(+), 443 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/e96de9f5/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
index 137468d..f6f369a 100644
--- 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
@@ -21,10 +21,12 @@ 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;
@@ -37,6 +39,7 @@ 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.carbondata.core.vo.ColumnGroupModel;
 
 import org.apache.commons.lang3.ArrayUtils;
 
@@ -85,6 +88,12 @@ public class SegmentProperties {
   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
    */
@@ -114,6 +123,12 @@ public class SegmentProperties {
   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
@@ -129,9 +144,9 @@ public class SegmentProperties {
   private int numberOfNoDictionaryDimension;
 
   /**
-   * column groups
+   * column group model
    */
-  private int[][] colGroups;
+  private ColumnGroupModel colGroupModel;
 
   public SegmentProperties(List<ColumnSchema> columnsInTable, int[] columnCardinality) {
     dimensions = new ArrayList<CarbonDimension>(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE);
@@ -141,6 +156,8 @@ public class SegmentProperties {
     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);
@@ -179,13 +196,14 @@ public class SegmentProperties {
       }
 
     }
-    colGroups = new int[colGrpList.size()][];
+    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);
   }
 
   /**
@@ -215,6 +233,24 @@ public class SegmentProperties {
       dimensionOrdinalToBlockMapping.put(dimension.getOrdinal(), ++blockOrdinal);
       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());
+    }
   }
 
   /**
@@ -457,6 +493,7 @@ public class SegmentProperties {
     // 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 =
@@ -464,14 +501,18 @@ public class SegmentProperties {
     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(cardinality, new int[] { cardinality.length })));
+          CarbonUtil.getDimensionBitLength(columnGroupCardinality,
+              new int[] { columnGroupCardinality.length })));
+      this.columnGroupOrdinalToMdkeymapping.put(next.getKey(), colGrpOrdinalMdkeyMapping);
     }
   }
 
@@ -586,11 +627,46 @@ public class SegmentProperties {
   }
 
   /**
-   *
    * @return
    */
   public int[][] getColumnGroups() {
-    return this.colGroups;
+    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/e96de9f5/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
index 55c38d3..7d0cc26 100644
--- 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
@@ -60,11 +60,8 @@ public class ColumnGroupDimensionDataChunk implements DimensionColumnDataChunk<b
    */
   @Override public int fillChunkData(byte[] data, int offset, int rowId,
       KeyStructureInfo restructuringInfo) {
-    byte[] rowData = new byte[restructuringInfo.getMaskedBytes().length];
-    System.arraycopy(dataChunk, rowId * chunkAttributes.getColumnValueSize(), rowData,
-        restructuringInfo.getBlockMdKeyStartOffset(), chunkAttributes.getColumnValueSize());
     byte[] maskedKey =
-        getMaskedKey(rowData, rowId * chunkAttributes.getColumnValueSize(), restructuringInfo);
+        getMaskedKey(dataChunk, rowId * chunkAttributes.getColumnValueSize(), restructuringInfo);
     System.arraycopy(maskedKey, 0, data, offset, maskedKey.length);
     return maskedKey.length;
   }
@@ -81,7 +78,7 @@ public class ColumnGroupDimensionDataChunk implements DimensionColumnDataChunk<b
     int byteRange = 0;
     for (int i = 0; i < info.getMaskByteRanges().length; i++) {
       byteRange = info.getMaskByteRanges()[i];
-      maskedKey[counter++] = (byte) (data[byteRange] & info.getMaxKey()[byteRange]);
+      maskedKey[counter++] = (byte) (data[byteRange + offset] & info.getMaxKey()[byteRange]);
     }
     return maskedKey;
   }

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/e96de9f5/core/src/main/java/org/carbondata/core/util/CarbonMetadataUtil.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/core/util/CarbonMetadataUtil.java b/core/src/main/java/org/carbondata/core/util/CarbonMetadataUtil.java
index 5e8e8a2..9fbcc47 100644
--- a/core/src/main/java/org/carbondata/core/util/CarbonMetadataUtil.java
+++ b/core/src/main/java/org/carbondata/core/util/CarbonMetadataUtil.java
@@ -8,9 +8,11 @@ import java.io.ObjectOutputStream;
 import java.nio.ByteBuffer;
 import java.util.ArrayList;
 import java.util.List;
+import java.util.Set;
 
 import org.carbondata.common.logging.LogService;
 import org.carbondata.common.logging.LogServiceFactory;
+import org.carbondata.core.carbon.datastore.block.SegmentProperties;
 import org.carbondata.core.carbon.metadata.index.BlockIndexInfo;
 import org.carbondata.core.constants.CarbonCommonConstants;
 import org.carbondata.core.datastorage.store.compression.ValueCompressionModel;
@@ -52,7 +54,8 @@ public class CarbonMetadataUtil {
    * @return FileFooter
    */
   public static FileFooter convertFileFooter(List<BlockletInfoColumnar> infoList, int numCols,
-      int[] cardinalities, List<ColumnSchema> columnSchemaList) throws IOException {
+      int[] cardinalities, List<ColumnSchema> columnSchemaList,
+      SegmentProperties segmentProperties) throws IOException {
 
     SegmentInfo segmentInfo = new SegmentInfo();
     segmentInfo.setNum_cols(columnSchemaList.size());
@@ -66,7 +69,7 @@ public class CarbonMetadataUtil {
     }
     footer.setTable_columns(columnSchemaList);
     for (BlockletInfoColumnar info : infoList) {
-      footer.addToBlocklet_info_list(getBlockletInfo(info, columnSchemaList));
+      footer.addToBlocklet_info_list(getBlockletInfo(info, columnSchemaList, segmentProperties));
     }
     return footer;
   }
@@ -122,7 +125,8 @@ public class CarbonMetadataUtil {
   }
 
   private static BlockletInfo getBlockletInfo(BlockletInfoColumnar blockletInfoColumnar,
-      List<ColumnSchema> columnSchenma) throws IOException {
+      List<ColumnSchema> columnSchenma,
+      SegmentProperties segmentProperties) throws IOException {
 
     BlockletInfo blockletInfo = new BlockletInfo();
     blockletInfo.setNum_rows(blockletInfoColumnar.getNumberOfKeys());
@@ -138,10 +142,10 @@ public class CarbonMetadataUtil {
       DataChunk dataChunk = new DataChunk();
       dataChunk.setChunk_meta(getChunkCompressionMeta());
       List<Encoding> encodings = new ArrayList<Encoding>();
-      if (columnSchenma.get(i).encoders.contains(Encoding.DICTIONARY)) {
+      if (containsEncoding(i, Encoding.DICTIONARY, columnSchenma, segmentProperties)) {
         encodings.add(Encoding.DICTIONARY);
       }
-      if (columnSchenma.get(i).encoders.contains(Encoding.DIRECT_DICTIONARY)) {
+      if (containsEncoding(i, Encoding.DIRECT_DICTIONARY, columnSchenma, segmentProperties)) {
         encodings.add(Encoding.DIRECT_DICTIONARY);
       }
       dataChunk.setRowMajor(colGrpblock[i]);
@@ -204,6 +208,28 @@ public class CarbonMetadataUtil {
     return blockletInfo;
   }
 
+  /**
+   * @param blockIndex
+   * @param encoding
+   * @param columnSchemas
+   * @param segmentProperties
+   * @return return true if given encoding is present in column
+   */
+  private static boolean containsEncoding(int blockIndex, Encoding encoding,
+      List<ColumnSchema> columnSchemas, SegmentProperties segmentProperties) {
+    Set<Integer> dimOrdinals = segmentProperties.getDimensionOrdinalForBlock(blockIndex);
+    //column groups will always have dictionary encoding
+    if (dimOrdinals.size() > 1 && Encoding.DICTIONARY == encoding) {
+      return true;
+    }
+    for (Integer dimOrdinal : dimOrdinals) {
+      if (columnSchemas.get(dimOrdinal).encoders.contains(encoding)) {
+        return true;
+      }
+    }
+    return false;
+  }
+
   private static byte[] serializeEncoderMeta(ValueEncoderMeta encoderMeta) throws IOException {
     // TODO : should remove the unnecessary fields.
     ByteArrayOutputStream aos = new ByteArrayOutputStream();

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/e96de9f5/core/src/main/java/org/carbondata/core/util/CarbonUtil.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/core/util/CarbonUtil.java b/core/src/main/java/org/carbondata/core/util/CarbonUtil.java
index 427fb97..2d50f07 100644
--- a/core/src/main/java/org/carbondata/core/util/CarbonUtil.java
+++ b/core/src/main/java/org/carbondata/core/util/CarbonUtil.java
@@ -56,6 +56,8 @@ import org.carbondata.core.carbon.metadata.blocklet.datachunk.DataChunk;
 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.carbon.path.CarbonStorePath;
 import org.carbondata.core.carbon.path.CarbonTablePath;
 import org.carbondata.core.constants.CarbonCommonConstants;
@@ -364,11 +366,10 @@ public final class CarbonUtil {
   /**
    * return ColumnGroupModel. check ColumnGroupModel for detail
    *
-   * @param dimLens      : dimension cardinality
    * @param columnGroups : column groups
    * @return ColumnGroupModel  model
    */
-  public static ColumnGroupModel getColGroupModel(int[] dimLens, int[][] columnGroups) {
+  public static ColumnGroupModel getColGroupModel(int[][] columnGroups) {
     int[] columnSplit = new int[columnGroups.length];
     int noOfColumnStore = columnSplit.length;
     boolean[] columnarStore = new boolean[noOfColumnStore];
@@ -380,7 +381,6 @@ public final class CarbonUtil {
     ColumnGroupModel colGroupModel = new ColumnGroupModel();
     colGroupModel.setNoOfColumnStore(noOfColumnStore);
     colGroupModel.setColumnSplit(columnSplit);
-    colGroupModel.setColumnGroupCardinality(dimLens);
     colGroupModel.setColumnarStore(columnarStore);
     colGroupModel.setColumnGroup(columnGroups);
     return colGroupModel;
@@ -1688,6 +1688,49 @@ public final class CarbonUtil {
   }
 
   /**
+   * @param dictionaryColumnCardinality
+   * @param wrapperColumnSchemaList
+   * @return It returns formatted cardinality by adding -1 value for NoDictionary columns
+   */
+  public static int[] getFormattedCardinality(int[] dictionaryColumnCardinality,
+      List<ColumnSchema> wrapperColumnSchemaList) {
+    List<Integer> cardinality = new ArrayList<>();
+    int counter = 0;
+    for (int i = 0; i < wrapperColumnSchemaList.size(); i++) {
+      if (CarbonUtil.hasEncoding(wrapperColumnSchemaList.get(i).getEncodingList(),
+          org.carbondata.core.carbon.metadata.encoder.Encoding.DICTIONARY)) {
+        cardinality.add(dictionaryColumnCardinality[counter]);
+        counter++;
+      } else if (!wrapperColumnSchemaList.get(i).isDimensionColumn()) {
+        continue;
+      } else {
+        cardinality.add(-1);
+      }
+    }
+    return ArrayUtils.toPrimitive(cardinality.toArray(new Integer[cardinality.size()]));
+  }
+
+  public static List<ColumnSchema> getColumnSchemaList(List<CarbonDimension> carbonDimensionsList,
+      List<CarbonMeasure> carbonMeasureList) {
+    List<ColumnSchema> wrapperColumnSchemaList = new ArrayList<ColumnSchema>();
+    fillCollumnSchemaListForComplexDims(carbonDimensionsList, wrapperColumnSchemaList);
+    for (CarbonMeasure carbonMeasure : carbonMeasureList) {
+      wrapperColumnSchemaList.add(carbonMeasure.getColumnSchema());
+    }
+    return wrapperColumnSchemaList;
+  }
+
+  private static void fillCollumnSchemaListForComplexDims(
+      List<CarbonDimension> carbonDimensionsList, List<ColumnSchema> wrapperColumnSchemaList) {
+    for (CarbonDimension carbonDimension : carbonDimensionsList) {
+      wrapperColumnSchemaList.add(carbonDimension.getColumnSchema());
+      List<CarbonDimension> childDims = carbonDimension.getListOfChildDimensions();
+      if (null != childDims && childDims.size() > 0) {
+        fillCollumnSchemaListForComplexDims(childDims, wrapperColumnSchemaList);
+      }
+    }
+  }
+  /**
    * Below method will be used to get all the block index info from index file
    *
    * @param taskId                  task id of the file

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/e96de9f5/core/src/main/java/org/carbondata/core/vo/ColumnGroupModel.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/core/vo/ColumnGroupModel.java b/core/src/main/java/org/carbondata/core/vo/ColumnGroupModel.java
index b3b9d37..a778598 100644
--- a/core/src/main/java/org/carbondata/core/vo/ColumnGroupModel.java
+++ b/core/src/main/java/org/carbondata/core/vo/ColumnGroupModel.java
@@ -21,11 +21,6 @@ package org.carbondata.core.vo;
 public class ColumnGroupModel {
 
   /**
-   * cardinality all dimension
-   */
-  private int[] colGroupCardinality;
-
-  /**
    * number of columns in columnar block
    */
   private int[] columnSplit;
@@ -50,22 +45,6 @@ public class ColumnGroupModel {
   private int[][] columnGroups;
 
   /**
-   * @return cardinality of column groups
-   */
-  public int[] getColumnGroupCardinality() {
-    return colGroupCardinality;
-  }
-
-  /**
-   * set columngroup cardinality
-   *
-   * @param columnGroupCardinality
-   */
-  public void setColumnGroupCardinality(int[] columnGroupCardinality) {
-    this.colGroupCardinality = columnGroupCardinality;
-  }
-
-  /**
    * return columnSplit
    *
    * @return

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/e96de9f5/core/src/main/java/org/carbondata/query/carbon/executor/infos/KeyStructureInfo.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/carbon/executor/infos/KeyStructureInfo.java b/core/src/main/java/org/carbondata/query/carbon/executor/infos/KeyStructureInfo.java
index 75e65a2..841184b 100644
--- a/core/src/main/java/org/carbondata/query/carbon/executor/infos/KeyStructureInfo.java
+++ b/core/src/main/java/org/carbondata/query/carbon/executor/infos/KeyStructureInfo.java
@@ -49,11 +49,6 @@ public class KeyStructureInfo {
   private byte[] maxKey;
 
   /**
-   * mdkey start index of block
-   */
-  private int blockMdKeyStartOffset;
-
-  /**
    * @return the keyGenerator
    */
   public KeyGenerator getKeyGenerator() {
@@ -109,17 +104,4 @@ public class KeyStructureInfo {
     this.maxKey = maxKey;
   }
 
-  /**
-   * @param startOffset
-   */
-  public void setBlockMdKeyStartOffset(int startOffset) {
-    this.blockMdKeyStartOffset = startOffset;
-  }
-
-  /**
-   * @return
-   */
-  public int getBlockMdKeyStartOffset() {
-    return this.blockMdKeyStartOffset;
-  }
 }

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/e96de9f5/core/src/main/java/org/carbondata/query/carbon/executor/util/QueryUtil.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/carbon/executor/util/QueryUtil.java b/core/src/main/java/org/carbondata/query/carbon/executor/util/QueryUtil.java
index a8eaa49..1d68872 100644
--- a/core/src/main/java/org/carbondata/query/carbon/executor/util/QueryUtil.java
+++ b/core/src/main/java/org/carbondata/query/carbon/executor/util/QueryUtil.java
@@ -528,29 +528,33 @@ public class QueryUtil {
     // get column group id and its ordinal mapping of column group
     Map<Integer, List<Integer>> columnGroupAndItsOrdinalMappingForQuery =
         getColumnGroupAndItsOrdinalMapping(queryDimensions);
-    KeyGenerator keyGenerator = segmentProperties.getDimensionKeyGenerator();
+    Map<Integer, KeyGenerator> columnGroupAndItsKeygenartor =
+        segmentProperties.getColumnGroupAndItsKeygenartor();
 
     Iterator<Entry<Integer, List<Integer>>> iterator =
         columnGroupAndItsOrdinalMappingForQuery.entrySet().iterator();
     KeyStructureInfo restructureInfos = null;
     while (iterator.hasNext()) {
       Entry<Integer, List<Integer>> next = iterator.next();
+      KeyGenerator keyGenerator = columnGroupAndItsKeygenartor.get(next.getKey());
       restructureInfos = new KeyStructureInfo();
       // sort the ordinal
       List<Integer> ordinal = next.getValue();
-      Collections.sort(ordinal);
+      List<Integer> mdKeyOrdinal = new ArrayList<Integer>();
+      for (Integer ord : ordinal) {
+        mdKeyOrdinal.add(segmentProperties.getColumnGroupMdKeyOrdinal(next.getKey(), ord));
+      }
+      Collections.sort(mdKeyOrdinal);
       // get the masked byte range for column group
-      int[] maskByteRanges = getMaskedByteRangeBasedOrdinal(ordinal, keyGenerator);
+      int[] maskByteRanges = getMaskedByteRangeBasedOrdinal(mdKeyOrdinal, keyGenerator);
       // max key for column group
-      byte[] maxKey = getMaxKeyBasedOnOrinal(ordinal, keyGenerator);
+      byte[] maxKey = getMaxKeyBasedOnOrinal(mdKeyOrdinal, keyGenerator);
       // get masked key for column group
       int[] maksedByte = getMaskedByte(keyGenerator.getKeySizeInBytes(), maskByteRanges);
       restructureInfos.setKeyGenerator(keyGenerator);
       restructureInfos.setMaskByteRanges(maskByteRanges);
       restructureInfos.setMaxKey(maxKey);
       restructureInfos.setMaskedBytes(maksedByte);
-      restructureInfos
-          .setBlockMdKeyStartOffset(getBlockMdKeyStartOffset(segmentProperties, ordinal));
       rowGroupToItsRSInfo
           .put(segmentProperties.getDimensionOrdinalToBlockMapping().get(ordinal.get(0)),
               restructureInfos);
@@ -559,25 +563,6 @@ public class QueryUtil {
   }
 
   /**
-   * It return mdkey start index of given column group
-   * @param segmentProperties
-   * @param ordinal : column group ordinal
-   * @return
-   */
-  public static int getBlockMdKeyStartOffset(SegmentProperties segmentProperties,
-      List<Integer> ordinal) {
-    int[][] colGroups = segmentProperties.getColumnGroups();
-    int blockMdkeyStartOffset = 0;
-    for (int i = 0; i < colGroups.length; i++) {
-      if (QueryUtil.searchInArray(colGroups[i], ordinal.get(0))) {
-        break;
-      }
-      blockMdkeyStartOffset += segmentProperties.getDimensionColumnsValueSize()[i];
-    }
-    return blockMdkeyStartOffset;
-  }
-
-  /**
    * return true if given key is found in array
    *
    * @param data

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/e96de9f5/core/src/main/java/org/carbondata/query/filter/executer/ColGroupFilterExecuterImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/filter/executer/ColGroupFilterExecuterImpl.java b/core/src/main/java/org/carbondata/query/filter/executer/ColGroupFilterExecuterImpl.java
index e4e3812..98b3ed4 100644
--- a/core/src/main/java/org/carbondata/query/filter/executer/ColGroupFilterExecuterImpl.java
+++ b/core/src/main/java/org/carbondata/query/filter/executer/ColGroupFilterExecuterImpl.java
@@ -27,6 +27,7 @@ import org.carbondata.common.logging.LogServiceFactory;
 import org.carbondata.core.carbon.datastore.block.SegmentProperties;
 import org.carbondata.core.carbon.datastore.chunk.DimensionColumnDataChunk;
 import org.carbondata.core.keygenerator.KeyGenException;
+import org.carbondata.core.keygenerator.KeyGenerator;
 import org.carbondata.core.util.ByteUtil;
 import org.carbondata.query.carbon.executor.infos.KeyStructureInfo;
 import org.carbondata.query.carbon.executor.util.QueryUtil;
@@ -88,24 +89,18 @@ public class ColGroupFilterExecuterImpl extends IncludeFilterExecuterImpl {
    * @throws KeyGenException
    */
   private KeyStructureInfo getKeyStructureInfo() throws KeyGenException {
-    List<Integer> ordinals = new ArrayList<Integer>();
-    ordinals.add(dimColumnEvaluatorInfo.getColumnIndex());
-
-    int[] maskByteRanges = QueryUtil
-        .getMaskedByteRangeBasedOrdinal(ordinals, segmentProperties.getDimensionKeyGenerator());
-    byte[] maxKey =
-        QueryUtil.getMaxKeyBasedOnOrinal(ordinals, segmentProperties.getDimensionKeyGenerator());
-    int[] maksedByte = QueryUtil
-        .getMaskedByte(segmentProperties.getDimensionKeyGenerator().getKeySizeInBytes(),
-            maskByteRanges);
-    int blockMdkeyStartOffset = QueryUtil.getBlockMdKeyStartOffset(segmentProperties, ordinals);
-
+    int colGrpId = getColumnGroupId(dimColumnEvaluatorInfo.getColumnIndex());
+    KeyGenerator keyGenerator = segmentProperties.getColumnGroupAndItsKeygenartor().get(colGrpId);
+    List<Integer> mdKeyOrdinal = new ArrayList<Integer>();
+    mdKeyOrdinal.add(getMdkeyOrdinal(dimColumnEvaluatorInfo.getColumnIndex(), colGrpId));
+    int[] maskByteRanges = QueryUtil.getMaskedByteRangeBasedOrdinal(mdKeyOrdinal, keyGenerator);
+    byte[] maxKey = QueryUtil.getMaxKeyBasedOnOrinal(mdKeyOrdinal, keyGenerator);
+    int[] maksedByte = QueryUtil.getMaskedByte(keyGenerator.getKeySizeInBytes(), maskByteRanges);
     KeyStructureInfo restructureInfos = new KeyStructureInfo();
-    restructureInfos.setKeyGenerator(segmentProperties.getDimensionKeyGenerator());
+    restructureInfos.setKeyGenerator(keyGenerator);
     restructureInfos.setMaskByteRanges(maskByteRanges);
     restructureInfos.setMaxKey(maxKey);
     restructureInfos.setMaskedBytes(maksedByte);
-    restructureInfos.setBlockMdKeyStartOffset(blockMdkeyStartOffset);
     return restructureInfos;
   }
 
@@ -155,8 +150,9 @@ public class ColGroupFilterExecuterImpl extends IncludeFilterExecuterImpl {
     int endIndex = 0;
     if (null != colGrpColumns) {
       for (int i = 0; i < colGrpColumns.length; i++) {
-        int[] byteRange =
-            segmentProperties.getDimensionKeyGenerator().getKeyByteOffsets(colGrpColumns[i]);
+        int colGrpId = getColumnGroupId(colGrpColumns[i]);
+        int mdKeyOrdinal = getMdkeyOrdinal(colGrpColumns[i], colGrpId);
+        int[] byteRange = getKeyGenerator(colGrpId).getKeyByteOffsets(mdKeyOrdinal);
         int colSize = 0;
         for (int j = byteRange[0]; j <= byteRange[1]; j++) {
           colSize++;
@@ -189,4 +185,25 @@ public class ColGroupFilterExecuterImpl extends IncludeFilterExecuterImpl {
     return null;
   }
 
-}
+  private int getMdkeyOrdinal(int ordinal, int colGrpId) {
+    return segmentProperties.getColumnGroupMdKeyOrdinal(colGrpId, ordinal);
+  }
+
+  private int getColumnGroupId(int ordinal) {
+    int[][] columnGroups = segmentProperties.getColumnGroups();
+    int colGrpId = -1;
+    for (int i = 0; i < columnGroups.length; i++) {
+      if (columnGroups[i].length > 1) {
+        colGrpId++;
+        if (QueryUtil.searchInArray(columnGroups[i], ordinal)) {
+          break;
+        }
+      }
+    }
+    return colGrpId;
+  }
+
+  public KeyGenerator getKeyGenerator(int colGrpId) {
+    return segmentProperties.getColumnGroupAndItsKeygenartor().get(colGrpId);
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/e96de9f5/core/src/test/java/org/carbondata/core/writer/CarbonFooterWriterTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/carbondata/core/writer/CarbonFooterWriterTest.java b/core/src/test/java/org/carbondata/core/writer/CarbonFooterWriterTest.java
index 14b019c..bfcbdde 100644
--- a/core/src/test/java/org/carbondata/core/writer/CarbonFooterWriterTest.java
+++ b/core/src/test/java/org/carbondata/core/writer/CarbonFooterWriterTest.java
@@ -28,6 +28,9 @@ import java.util.UUID;
 
 import junit.framework.TestCase;
 
+import org.carbondata.core.carbon.datastore.block.SegmentProperties;
+import org.carbondata.core.carbon.metadata.datatype.DataType;
+import org.carbondata.core.carbon.metadata.encoder.Encoding;
 import org.carbondata.core.constants.CarbonCommonConstants;
 import org.carbondata.core.datastorage.store.compression.ValueCompressionModel;
 import org.carbondata.core.datastorage.store.filesystem.CarbonFile;
@@ -35,6 +38,7 @@ import org.carbondata.core.datastorage.store.impl.FileFactory;
 import org.carbondata.core.metadata.BlockletInfoColumnar;
 import org.carbondata.core.reader.CarbonFooterReader;
 import org.carbondata.core.util.CarbonMetadataUtil;
+import org.carbondata.core.util.CarbonUtil;
 import org.carbondata.format.ColumnSchema;
 import org.junit.After;
 import org.junit.Before;
@@ -68,16 +72,26 @@ public class CarbonFooterWriterTest extends TestCase{
 
     List<BlockletInfoColumnar> infoColumnars = getBlockletInfoColumnars();
 
-		writer.writeFooter(CarbonMetadataUtil.convertFileFooter(
-				infoColumnars,
-				6,
-				new int[] { 2, 4, 5, 7 },
-				Arrays.asList(new ColumnSchema[]{getDimensionColumn("IMEI1"),
+    int[] cardinalities = new int[] { 2, 4, 5, 7, 9, 10 };
+    List<ColumnSchema> columnSchema = Arrays.asList(new ColumnSchema[]{getDimensionColumn("IMEI1"),
 						getDimensionColumn("IMEI2"),
 						getDimensionColumn("IMEI3"),
 						getDimensionColumn("IMEI4"),
 						getDimensionColumn("IMEI5"),
-						getDimensionColumn("IMEI6")})), 0);
+						getDimensionColumn("IMEI6")});
+    List<org.carbondata.core.carbon.metadata.schema.table.column.ColumnSchema> wrapperColumnSchema = Arrays.asList(new org.carbondata.core.carbon.metadata.schema.table.column.ColumnSchema[]{getWrapperDimensionColumn("IMEI1"),
+    	getWrapperDimensionColumn("IMEI2"),
+    	getWrapperDimensionColumn("IMEI3"),
+    	getWrapperDimensionColumn("IMEI4"),
+    	getWrapperDimensionColumn("IMEI5"),
+    	getWrapperDimensionColumn("IMEI6")});
+    int[] colCardinality = CarbonUtil.getFormattedCardinality(cardinalities, wrapperColumnSchema);
+    SegmentProperties segmentProperties = new SegmentProperties(wrapperColumnSchema, colCardinality);
+		writer.writeFooter(CarbonMetadataUtil.convertFileFooter(
+				infoColumnars,
+				6,
+				cardinalities,columnSchema, segmentProperties
+				), 0);
 
     CarbonFooterReader metaDataReader = new CarbonFooterReader(filePath, 0);
     assertTrue(metaDataReader.readFooter() != null);
@@ -97,6 +111,20 @@ public class CarbonFooterWriterTest extends TestCase{
 	    dimColumn.setNum_child(0);
 	    return dimColumn;
 	  }
+  public static org.carbondata.core.carbon.metadata.schema.table.column.ColumnSchema getWrapperDimensionColumn(String columnName) {
+   org.carbondata.core.carbon.metadata.schema.table.column.ColumnSchema dimColumn = new org.carbondata.core.carbon.metadata.schema.table.column.ColumnSchema();
+   dimColumn.setColumnar(true);
+   dimColumn.setColumnName(columnName);
+   dimColumn.setColumnUniqueId(UUID.randomUUID().toString());
+   dimColumn.setDataType(DataType.STRING);
+   dimColumn.setDimensionColumn(true);
+   List<Encoding> encodeList =
+        new ArrayList<>(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE);
+   encodeList.add(Encoding.DICTIONARY);
+   dimColumn.setEncodingList(encodeList);
+   dimColumn.setNumberOfChild(0);;
+   return dimColumn;
+ }
 
   /**
    * test writing fact metadata.
@@ -106,14 +134,24 @@ public class CarbonFooterWriterTest extends TestCase{
     createFile();
     CarbonFooterWriter writer = new CarbonFooterWriter(filePath);
     List<BlockletInfoColumnar> infoColumnars = getBlockletInfoColumnars();
-    writer.writeFooter(CarbonMetadataUtil
-        .convertFileFooter(infoColumnars, 6, new int[] { 2, 4, 5, 7 },
-        		Arrays.asList(new ColumnSchema[]{getDimensionColumn("IMEI1"),
+    int[] cardinalities = new int[] { 2, 4, 5, 7, 9, 10};
+    List<ColumnSchema> columnSchema = Arrays.asList(new ColumnSchema[]{getDimensionColumn("IMEI1"),
 						getDimensionColumn("IMEI2"),
 						getDimensionColumn("IMEI3"),
 						getDimensionColumn("IMEI4"),
 						getDimensionColumn("IMEI5"),
-						getDimensionColumn("IMEI6")})), 0);
+						getDimensionColumn("IMEI6")});
+    List<org.carbondata.core.carbon.metadata.schema.table.column.ColumnSchema> wrapperColumnSchema = Arrays.asList(new org.carbondata.core.carbon.metadata.schema.table.column.ColumnSchema[]{getWrapperDimensionColumn("IMEI1"),
+    	getWrapperDimensionColumn("IMEI2"),
+    	getWrapperDimensionColumn("IMEI3"),
+    	getWrapperDimensionColumn("IMEI4"),
+    	getWrapperDimensionColumn("IMEI5"),
+    	getWrapperDimensionColumn("IMEI6")});
+    int[] colCardinality = CarbonUtil.getFormattedCardinality(cardinalities, wrapperColumnSchema);
+    SegmentProperties segmentProperties = new SegmentProperties(wrapperColumnSchema, cardinalities);
+    writer.writeFooter(CarbonMetadataUtil
+        .convertFileFooter(infoColumnars, 6, colCardinality,
+        		columnSchema,segmentProperties), 0);
 
     CarbonFooterReader metaDataReader = new CarbonFooterReader(filePath, 0);
     List<BlockletInfoColumnar> nodeInfoColumnars =

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/e96de9f5/integration/spark/src/main/java/org/carbondata/integration/spark/merger/RowResultMerger.java
----------------------------------------------------------------------
diff --git a/integration/spark/src/main/java/org/carbondata/integration/spark/merger/RowResultMerger.java b/integration/spark/src/main/java/org/carbondata/integration/spark/merger/RowResultMerger.java
index dcdc7a4..71117de 100644
--- a/integration/spark/src/main/java/org/carbondata/integration/spark/merger/RowResultMerger.java
+++ b/integration/spark/src/main/java/org/carbondata/integration/spark/merger/RowResultMerger.java
@@ -35,6 +35,7 @@ import org.carbondata.core.carbon.datastore.block.SegmentProperties;
 import org.carbondata.core.carbon.metadata.CarbonMetadata;
 import org.carbondata.core.carbon.metadata.schema.table.CarbonTable;
 import org.carbondata.core.carbon.metadata.schema.table.column.CarbonMeasure;
+import org.carbondata.core.carbon.metadata.schema.table.column.ColumnSchema;
 import org.carbondata.core.carbon.path.CarbonStorePath;
 import org.carbondata.core.carbon.path.CarbonTablePath;
 import org.carbondata.core.constants.CarbonCommonConstants;
@@ -42,7 +43,6 @@ import org.carbondata.core.keygenerator.KeyGenException;
 import org.carbondata.core.util.ByteUtil;
 import org.carbondata.core.util.CarbonUtil;
 import org.carbondata.core.util.DataTypeUtil;
-import org.carbondata.core.vo.ColumnGroupModel;
 import org.carbondata.processing.datatypes.GenericDataType;
 import org.carbondata.processing.merger.exeception.SliceMergerException;
 import org.carbondata.processing.store.CarbonDataFileAttributes;
@@ -74,15 +74,13 @@ public class RowResultMerger {
   private AbstractQueue<RawResultIterator> recordHolderHeap;
 
   private TupleConversionAdapter tupleConvertor;
-  private ColumnGroupModel colGrpStoreModel;
 
   private static final LogService LOGGER =
       LogServiceFactory.getLogService(RowResultMerger.class.getName());
 
-  public RowResultMerger(List<RawResultIterator> iteratorList,
-      String schemaName, String tableName,
-      SegmentProperties segProp, String tempStoreLocation,
-      CarbonLoadModel loadModel, int[] colCardinality) {
+  public RowResultMerger(List<RawResultIterator> iteratorList, String schemaName, String tableName,
+      SegmentProperties segProp, String tempStoreLocation, CarbonLoadModel loadModel,
+      int[] colCardinality) {
 
     this.rawResultIteratorList = iteratorList;
     // create the List of RawResultIterator.
@@ -222,16 +220,20 @@ public class RowResultMerger {
         .setMdKeyLength(segprop.getDimensionKeyGenerator().getKeySizeInBytes());
     carbonFactDataHandlerModel.setStoreLocation(tempStoreLocation);
     carbonFactDataHandlerModel.setDimLens(segprop.getDimColumnsCardinality());
+    carbonFactDataHandlerModel.setSegmentProperties(segprop);
     carbonFactDataHandlerModel.setNoDictionaryCount(segprop.getNumberOfNoDictionaryDimension());
     carbonFactDataHandlerModel.setDimensionCount(
         segprop.getDimensions().size() - carbonFactDataHandlerModel.getNoDictionaryCount());
+    CarbonTable carbonTable = CarbonMetadata.getInstance()
+        .getCarbonTable(schemaName + CarbonCommonConstants.UNDERSCORE + tableName);
+    List<ColumnSchema> wrapperColumnSchema = CarbonUtil
+        .getColumnSchemaList(carbonTable.getDimensionByTableName(tableName),
+            carbonTable.getMeasureByTableName(tableName));
+    carbonFactDataHandlerModel.setWrapperColumnSchema(wrapperColumnSchema);
     //TO-DO Need to handle complex types here .
     Map<Integer, GenericDataType> complexIndexMap =
         new HashMap<Integer, GenericDataType>(segprop.getComplexDimensions().size());
     carbonFactDataHandlerModel.setComplexIndexMap(complexIndexMap);
-    this.colGrpStoreModel =
-        CarbonUtil.getColGroupModel(segprop.getDimColumnsCardinality(), segprop.getColumnGroups());
-    carbonFactDataHandlerModel.setColGrpModel(colGrpStoreModel);
     carbonFactDataHandlerModel.setDataWritingRequest(true);
 
     char[] aggType = new char[segprop.getMeasures().size()];

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/e96de9f5/integration/spark/src/test/scala/org/carbondata/spark/testsuite/detailquery/ColumnGroupDataTypesTestCase.scala
----------------------------------------------------------------------
diff --git a/integration/spark/src/test/scala/org/carbondata/spark/testsuite/detailquery/ColumnGroupDataTypesTestCase.scala b/integration/spark/src/test/scala/org/carbondata/spark/testsuite/detailquery/ColumnGroupDataTypesTestCase.scala
index 8371018..57f296a 100644
--- a/integration/spark/src/test/scala/org/carbondata/spark/testsuite/detailquery/ColumnGroupDataTypesTestCase.scala
+++ b/integration/spark/src/test/scala/org/carbondata/spark/testsuite/detailquery/ColumnGroupDataTypesTestCase.scala
@@ -23,7 +23,6 @@ import org.apache.spark.sql.common.util.CarbonHiveContext._
 import org.apache.spark.sql.common.util.QueryTest
 import org.scalatest.BeforeAndAfterAll
 
-
 /**
  * Test Class for aggregate query on multiple datatypes
  *
@@ -35,6 +34,12 @@ class ColumnGroupDataTypesTestCase extends QueryTest with BeforeAndAfterAll {
     sql("LOAD DATA LOCAL INPATH './src/test/resources/10dim_4msr.csv' INTO table colgrp options('FILEHEADER'='column1,column2,column3,column4,column5,column6,column7,column8,column9,column10,measure1,measure2,measure3,measure4')");
     sql("create table normal (column1 string,column2 string,column3 string,column4 string,column5 string,column6 string,column7 string,column8 string,column9 string,column10 string,measure1 int,measure2 int,measure3 int,measure4 int) STORED BY 'org.apache.carbondata.format'")
     sql("LOAD DATA LOCAL INPATH './src/test/resources/10dim_4msr.csv' INTO table normal options('FILEHEADER'='column1,column2,column3,column4,column5,column6,column7,column8,column9,column10,measure1,measure2,measure3,measure4')");
+    //column group with dictionary exclude before column group
+    sql("create table colgrp_dictexclude_before (column1 string,column2 string,column3 string,column4 string,column5 string,column6 string,column7 string,column8 string,column9 string,column10 string,measure1 int,measure2 int,measure3 int,measure4 int) STORED BY 'org.apache.carbondata.format' TBLPROPERTIES ('DICTIONARY_EXCLUDE'='column1',\"COLUMN_GROUPS\"=\"(column2,column3,column4),(column7,column8,column9)\")")
+    sql("LOAD DATA LOCAL INPATH './src/test/resources/10dim_4msr.csv' INTO table colgrp_dictexclude_before options('FILEHEADER'='column1,column2,column3,column4,column5,column6,column7,column8,column9,column10,measure1,measure2,measure3,measure4')");
+    //column group with dictionary exclude after column group
+    sql("create table colgrp_dictexclude_after (column1 string,column2 string,column3 string,column4 string,column5 string,column6 string,column7 string,column8 string,column9 string,column10 string,measure1 int,measure2 int,measure3 int,measure4 int) STORED BY 'org.apache.carbondata.format' TBLPROPERTIES ('DICTIONARY_EXCLUDE'='column10',\"COLUMN_GROUPS\"=\"(column2,column3,column4),(column7,column8,column9)\")")
+    sql("LOAD DATA LOCAL INPATH './src/test/resources/10dim_4msr.csv' INTO table colgrp_dictexclude_after options('FILEHEADER'='column1,column2,column3,column4,column5,column6,column7,column8,column9,column10,measure1,measure2,measure3,measure4')");
   }
 
   test("select all dimension query") {
@@ -79,8 +84,30 @@ class ColumnGroupDataTypesTestCase extends QueryTest with BeforeAndAfterAll {
       sql("select column1,column3,column4,column5,column6,column9,column10 from normal"))
   }
 
+   test("##ColumnGroup_DictionaryExcludeBefore select all dimension on column group and dictionary exclude table") {
+    checkAnswer(
+      sql("select * from colgrp_dictexclude_before"),
+      sql("select * from normal"))
+  }
+  test("##ColumnGroup_DictionaryExcludeBefore select all dimension query with filter on two dimension from same column group") {
+    checkAnswer(
+      sql("select * from colgrp_dictexclude_before where column3='column311' and column4='column42' "),
+      sql("select * from normal where column3='column311' and column4='column42'"))
+  }
+  test("##ColumnGroup_DictionaryExcludeAfter select all dimension on column group and dictionary exclude table") {
+    checkAnswer(
+      sql("select * from colgrp_dictexclude_after"),
+      sql("select * from normal"))
+  }
+  test("##ColumnGroup_DictionaryExcludeAfter select all dimension query with filter on two dimension from same column group") {
+    checkAnswer(
+      sql("select * from colgrp_dictexclude_after where column3='column311' and column4='column42' "),
+      sql("select * from normal where column3='column311' and column4='column42'"))
+  }
   override def afterAll {
-    sql("drop cube colgrp")
-    sql("drop cube normal")
+    sql("drop table colgrp")
+    sql("drop table normal")
+    sql("drop table colgrp_dictexclude_before")
+    sql("drop table colgrp_dictexclude_after")
   }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/e96de9f5/processing/src/main/java/org/carbondata/processing/mdkeygen/MDKeyGenStep.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/carbondata/processing/mdkeygen/MDKeyGenStep.java b/processing/src/main/java/org/carbondata/processing/mdkeygen/MDKeyGenStep.java
index d680265..473840b 100644
--- a/processing/src/main/java/org/carbondata/processing/mdkeygen/MDKeyGenStep.java
+++ b/processing/src/main/java/org/carbondata/processing/mdkeygen/MDKeyGenStep.java
@@ -31,9 +31,11 @@ import java.util.Map.Entry;
 import org.carbondata.common.logging.LogService;
 import org.carbondata.common.logging.LogServiceFactory;
 import org.carbondata.common.logging.impl.StandardLogService;
+import org.carbondata.core.carbon.datastore.block.SegmentProperties;
 import org.carbondata.core.carbon.metadata.CarbonMetadata;
 import org.carbondata.core.carbon.metadata.schema.table.CarbonTable;
 import org.carbondata.core.carbon.metadata.schema.table.column.CarbonMeasure;
+import org.carbondata.core.carbon.metadata.schema.table.column.ColumnSchema;
 import org.carbondata.core.carbon.path.CarbonStorePath;
 import org.carbondata.core.carbon.path.CarbonTablePath;
 import org.carbondata.core.constants.CarbonCommonConstants;
@@ -41,12 +43,10 @@ import org.carbondata.core.file.manager.composite.FileData;
 import org.carbondata.core.file.manager.composite.FileManager;
 import org.carbondata.core.file.manager.composite.IFileManagerComposite;
 import org.carbondata.core.keygenerator.KeyGenException;
-import org.carbondata.core.keygenerator.factory.KeyGeneratorFactory;
 import org.carbondata.core.util.CarbonProperties;
 import org.carbondata.core.util.CarbonUtil;
 import org.carbondata.core.util.CarbonUtilException;
 import org.carbondata.core.util.DataTypeUtil;
-import org.carbondata.core.vo.ColumnGroupModel;
 import org.carbondata.processing.datatypes.GenericDataType;
 import org.carbondata.processing.store.CarbonDataFileAttributes;
 import org.carbondata.processing.store.CarbonFactDataHandlerColumnar;
@@ -99,6 +99,12 @@ public class MDKeyGenStep extends BaseStep {
 
   private Map<Integer, GenericDataType> complexIndexMap;
 
+  private SegmentProperties segmentProperties;
+
+  private int[] colCardinality;
+
+  private List<ColumnSchema> wrapperColumnSchema;
+
   /**
    * readCounter
    */
@@ -126,7 +132,6 @@ public class MDKeyGenStep extends BaseStep {
 
   private int[] dimLens;
 
-  private ColumnGroupModel colGrpStoreModel;
   /**
    * to check whether dimension is of dictionary type
    * or not
@@ -287,23 +292,17 @@ public class MDKeyGenStep extends BaseStep {
       simpleDimsLen[i] = dimLens[i];
     }
 
-    String[] colStore = null != meta.getColumnGroupsString() ?
-        meta.getColumnGroupsString().split(",") :
-        new String[0];
-    int[][] colGroups = new int[colStore.length][];
-    for (int i = 0; i < colGroups.length; i++) {
-      String[] group = colStore[i].split("~");
-      colGroups[i] = new int[group.length];
-      for (int j = 0; j < colGroups[i].length; j++) {
-        colGroups[i][j] = Integer.parseInt(group[j]);
-      }
-    }
+    CarbonTable carbonTable = CarbonMetadata.getInstance()
+        .getCarbonTable(meta.getSchemaName() + CarbonCommonConstants.UNDERSCORE + tableName);
+    wrapperColumnSchema = CarbonUtil
+        .getColumnSchemaList(carbonTable.getDimensionByTableName(tableName),
+            carbonTable.getMeasureByTableName(tableName));
+    colCardinality =
+        CarbonUtil.getFormattedCardinality(dimLensWithComplex, wrapperColumnSchema);
+    segmentProperties = new SegmentProperties(wrapperColumnSchema, colCardinality);
     // Actual primitive dimension used to generate start & end key
 
-    this.colGrpStoreModel = CarbonUtil.getColGroupModel(simpleDimsLen, colGroups);
-    data.generator = KeyGeneratorFactory
-        .getKeyGenerator(colGrpStoreModel.getColumnGroupCardinality(),
-            colGrpStoreModel.getColumnSplit());
+    data.generator = segmentProperties.getDimensionKeyGenerator();
 
     //To Set MDKey Index of each primitive type in complex type
     int surrIndex = simpleDimsCount;
@@ -377,10 +376,12 @@ public class MDKeyGenStep extends BaseStep {
     carbonFactDataHandlerModel.setNoDictionaryCount(meta.getNoDictionaryCount());
     carbonFactDataHandlerModel.setDimensionCount(dimensionCount);
     carbonFactDataHandlerModel.setComplexIndexMap(complexIndexMap);
-    carbonFactDataHandlerModel.setColGrpModel(colGrpStoreModel);
+    carbonFactDataHandlerModel.setSegmentProperties(segmentProperties);
+    carbonFactDataHandlerModel.setColCardinality(colCardinality);
     carbonFactDataHandlerModel.setDataWritingRequest(true);
     carbonFactDataHandlerModel.setAggType(aggType);
     carbonFactDataHandlerModel.setFactDimLens(dimLens);
+    carbonFactDataHandlerModel.setWrapperColumnSchema(wrapperColumnSchema);
     return carbonFactDataHandlerModel;
   }
 
@@ -449,7 +450,7 @@ public class MDKeyGenStep extends BaseStep {
     }
     outputRow[l] = RemoveDictionaryUtil.getByteArrayForNoDictionaryCols(row);
 
-    int[] highCardExcludedRows = new int[colGrpStoreModel.getColumnGroupCardinality().length];
+    int[] highCardExcludedRows = new int[segmentProperties.getDimColumnsCardinality().length];
     for (int i = 0; i < highCardExcludedRows.length; i++) {
       Object key = RemoveDictionaryUtil.getDimension(i, row);
       highCardExcludedRows[i] = (Integer) key;

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/e96de9f5/processing/src/main/java/org/carbondata/processing/store/CarbonFactDataHandlerColumnar.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/carbondata/processing/store/CarbonFactDataHandlerColumnar.java b/processing/src/main/java/org/carbondata/processing/store/CarbonFactDataHandlerColumnar.java
index ed7bb23..d77f8b5 100644
--- a/processing/src/main/java/org/carbondata/processing/store/CarbonFactDataHandlerColumnar.java
+++ b/processing/src/main/java/org/carbondata/processing/store/CarbonFactDataHandlerColumnar.java
@@ -41,10 +41,9 @@ import java.util.concurrent.atomic.AtomicInteger;
 
 import org.carbondata.common.logging.LogService;
 import org.carbondata.common.logging.LogServiceFactory;
+import org.carbondata.core.carbon.datastore.block.SegmentProperties;
 import org.carbondata.core.carbon.metadata.CarbonMetadata;
 import org.carbondata.core.carbon.metadata.schema.table.CarbonTable;
-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.BlockIndexerStorageForInt;
@@ -58,7 +57,6 @@ import org.carbondata.core.keygenerator.KeyGenException;
 import org.carbondata.core.keygenerator.KeyGenerator;
 import org.carbondata.core.keygenerator.columnar.ColumnarSplitter;
 import org.carbondata.core.keygenerator.columnar.impl.MultiDimKeyVarLengthEquiSplitGenerator;
-import org.carbondata.core.keygenerator.columnar.impl.MultiDimKeyVarLengthVariableSplitGenerator;
 import org.carbondata.core.keygenerator.factory.KeyGeneratorFactory;
 import org.carbondata.core.util.CarbonProperties;
 import org.carbondata.core.util.CarbonUtil;
@@ -262,6 +260,11 @@ public class CarbonFactDataHandlerColumnar implements CarbonFactHandler {
   private int[] colCardinality;
 
   /**
+   * Segment properties
+   */
+  private SegmentProperties segmentProperties;
+
+  /**
    * CarbonFactDataHandler constructor
    */
   public CarbonFactDataHandlerColumnar(CarbonFactDataHandlerModel carbonFactDataHandlerModel) {
@@ -324,24 +327,24 @@ public class CarbonFactDataHandlerColumnar implements CarbonFactHandler {
   private void initParameters(CarbonFactDataHandlerModel carbonFactDataHandlerModel) {
     this.databaseName = carbonFactDataHandlerModel.getDatabaseName();
     this.tableName = carbonFactDataHandlerModel.getTableName();
+    this.type = carbonFactDataHandlerModel.getAggType();
+    this.segmentProperties = carbonFactDataHandlerModel.getSegmentProperties();
+    this.wrapperColumnSchemaList = carbonFactDataHandlerModel.getWrapperColumnSchema();
+    this.colCardinality = carbonFactDataHandlerModel.getColCardinality();
     this.storeLocation = carbonFactDataHandlerModel.getStoreLocation();
     this.measureCount = carbonFactDataHandlerModel.getMeasureCount();
     this.mdkeyLength = carbonFactDataHandlerModel.getMdKeyLength();
     this.mdKeyIndex = carbonFactDataHandlerModel.getMdKeyIndex();
     this.noDictionaryCount = carbonFactDataHandlerModel.getNoDictionaryCount();
-    this.colGrpModel = carbonFactDataHandlerModel.getColGrpModel();
+    this.colGrpModel = segmentProperties.getColumnGroupModel();
     this.completeDimLens = carbonFactDataHandlerModel.getDimLens();
-    this.dimLens = colGrpModel.getColumnGroupCardinality();
+    this.dimLens = this.segmentProperties.getDimColumnsCardinality();
     this.carbonDataFileAttributes = carbonFactDataHandlerModel.getCarbonDataFileAttributes();
-    this.type = carbonFactDataHandlerModel.getAggType();
     //TODO need to pass carbon table identifier to metadata
     CarbonTable carbonTable = CarbonMetadata.getInstance()
         .getCarbonTable(databaseName + CarbonCommonConstants.UNDERSCORE + tableName);
-    fillColumnSchemaList(carbonTable.getDimensionByTableName(tableName),
-        carbonTable.getMeasureByTableName(tableName));
     dimensionType =
         CarbonUtil.identifyDimensionType(carbonTable.getDimensionByTableName(tableName));
-    this.colCardinality = carbonFactDataHandlerModel.getColCardinality();
 
     if (carbonFactDataHandlerModel.isCompactionFlow()) {
       try {
@@ -393,26 +396,6 @@ public class CarbonFactDataHandlerColumnar implements CarbonFactHandler {
     return uniqueBlock;
   }
 
-  private void fillColumnSchemaList(List<CarbonDimension> carbonDimensionsList,
-      List<CarbonMeasure> carbonMeasureList) {
-    wrapperColumnSchemaList = new ArrayList<ColumnSchema>();
-    fillCollumnSchemaListForComplexDims(carbonDimensionsList, wrapperColumnSchemaList);
-    for (CarbonMeasure carbonMeasure : carbonMeasureList) {
-      wrapperColumnSchemaList.add(carbonMeasure.getColumnSchema());
-    }
-  }
-
-  private void fillCollumnSchemaListForComplexDims(List<CarbonDimension> carbonDimensionsList,
-      List<ColumnSchema> wrapperColumnSchemaList) {
-    for (CarbonDimension carbonDimension : carbonDimensionsList) {
-      wrapperColumnSchemaList.add(carbonDimension.getColumnSchema());
-      List<CarbonDimension> childDims = carbonDimension.getListOfChildDimensions();
-      if (null != childDims && childDims.size() > 0) {
-        fillCollumnSchemaListForComplexDims(childDims, wrapperColumnSchemaList);
-      }
-    }
-  }
-
   private void setComplexMapSurrogateIndex(int dimensionCount) {
     int surrIndex = 0;
     for (int i = 0; i < dimensionCount; i++) {
@@ -518,10 +501,7 @@ public class CarbonFactDataHandlerColumnar implements CarbonFactHandler {
     if ((noDictionaryCount + complexColCount) > 0) {
       noDictionaryKeyDataHolder = initialiseKeyBlockHolder(dataRows.size());
     }
-    /**
-     * It holds min max value of columns of column group
-     */
-    ColGroupMinMax[] colGrpMinMax = initializeColGrpMinMax();
+
     for (int count = 0; count < dataRows.size(); count++) {
       Object[] row = dataRows.get(count);
       byte[] mdKey = (byte[]) row[this.mdKeyIndex];
@@ -529,8 +509,6 @@ public class CarbonFactDataHandlerColumnar implements CarbonFactHandler {
       if (noDictionaryCount > 0 || complexIndexMap.size() > 0) {
         noDictionaryKey = (byte[]) row[this.mdKeyIndex - 1];
       }
-      //to evaluate min max value of columns in column group
-      evaluateColGrpMinMax(colGrpMinMax, mdKey);
       ByteBuffer byteBuffer = null;
       byte[] b = null;
       if (count == 0) {
@@ -601,8 +579,7 @@ public class CarbonFactDataHandlerColumnar implements CarbonFactHandler {
             .clone();
     NodeHolder nodeHolder =
         getNodeHolderObject(writableMeasureDataArray, byteArrayValues, dataRows.size(), startKey,
-            endKey, compressionModel, noDictionaryValueHolder, noDictStartKey, noDictEndKey,
-            colGrpMinMax);
+            endKey, compressionModel, noDictionaryValueHolder, noDictStartKey, noDictEndKey);
     nodeHolder.setMeasureNullValueIndex(nullValueIndexBitSet);
     LOGGER.info("Number Of records processed: " + dataRows.size());
     return nodeHolder;
@@ -611,7 +588,7 @@ public class CarbonFactDataHandlerColumnar implements CarbonFactHandler {
   private NodeHolder getNodeHolderObject(byte[][] dataHolderLocal, byte[][] byteArrayValues,
       int entryCountLocal, byte[] startkeyLocal, byte[] endKeyLocal,
       ValueCompressionModel compressionModel, byte[][] noDictionaryData,
-      byte[] noDictionaryStartKey, byte[] noDictionaryEndKey, ColGroupMinMax[] colGrpMinMax)
+      byte[] noDictionaryStartKey, byte[] noDictionaryEndKey)
       throws CarbonDataWriterException {
     byte[][][] noDictionaryColumnsData = null;
     List<ArrayList<byte[]>> colsAndValues = new ArrayList<ArrayList<byte[]>>();
@@ -621,7 +598,7 @@ public class CarbonFactDataHandlerColumnar implements CarbonFactHandler {
       colsAndValues.add(new ArrayList<byte[]>());
     }
     int noOfColumn = colGrpModel.getNoOfColumnStore();
-    DataHolder[] dataHolders = getDataHolders(noOfColumn, byteArrayValues.length, colGrpMinMax);
+    DataHolder[] dataHolders = getDataHolders(noOfColumn, byteArrayValues.length);
     for (int i = 0; i < byteArrayValues.length; i++) {
       byte[][] splitKey = columnarSplitter.splitKey(byteArrayValues[i]);
 
@@ -741,14 +718,17 @@ public class CarbonFactDataHandlerColumnar implements CarbonFactHandler {
    * @param noOfRow    : total no of row
    * @return : dataholder
    */
-  private DataHolder[] getDataHolders(int noOfColumn, int noOfRow, ColGroupMinMax[] colGrpMinMax) {
+  private DataHolder[] getDataHolders(int noOfColumn, int noOfRow) {
     DataHolder[] dataHolders = new DataHolder[noOfColumn];
+    int colGrpId = -1;
     for (int colGrp = 0; colGrp < noOfColumn; colGrp++) {
       if (colGrpModel.isColumnar(colGrp)) {
         dataHolders[colGrp] = new ColumnDataHolder(noOfRow);
       } else {
-        dataHolders[colGrp] = new ColGroupDataHolder(this.colGrpModel,
-            this.columnarSplitter.getBlockKeySize()[colGrp], noOfRow, colGrpMinMax[colGrp]);
+        ColGroupMinMax colGrpMinMax = new ColGroupMinMax(segmentProperties, ++colGrpId);
+        dataHolders[colGrp] =
+            new ColGroupDataHolder(this.columnarSplitter.getBlockKeySize()[colGrp], noOfRow,
+                colGrpMinMax);
       }
     }
     return dataHolders;
@@ -876,34 +856,6 @@ public class CarbonFactDataHandlerColumnar implements CarbonFactHandler {
   }
 
   /**
-   * initialize column group min max evaluator
-   */
-  private ColGroupMinMax[] initializeColGrpMinMax() {
-    int[][] colGrps = colGrpModel.getColumnGroup();
-    ColGroupMinMax[] colGrpMinMax = new ColGroupMinMax[colGrps.length];
-    for (int colGrp = 0; colGrp < colGrps.length; colGrp++) {
-      if (!colGrpModel.isColumnar(colGrp)) {
-        colGrpMinMax[colGrp] = new ColGroupMinMax(colGrpModel, columnarSplitter, colGrp);
-      }
-    }
-    return colGrpMinMax;
-  }
-
-  /**
-   * Evaluate min max of columns in columnn group
-   *
-   * @param mdkey -> mdkey of data
-   */
-  private void evaluateColGrpMinMax(ColGroupMinMax[] colGrpMinMax, byte[] mdkey) {
-
-    for (int colGrp = 0; colGrp < colGrpModel.getColumnGroup().length; colGrp++) {
-      if (!colGrpModel.isColumnar(colGrp)) {
-        colGrpMinMax[colGrp].add(mdkey);
-      }
-    }
-  }
-
-  /**
    * This method will be used to update the max value for each measure
    */
   private void calculateMaxMin(Object[] max, Object[] min, int[] decimal, int[] msrIndex,
@@ -983,9 +935,7 @@ public class CarbonFactDataHandlerColumnar implements CarbonFactHandler {
       //row store will be in single column store
       //e.g if {0,1,2,3,4,5} is dimension and {0,1,2) is row store dimension
       //than below splitter will return column as {0,1,2}{3}{4}{5}
-      this.columnarSplitter = new MultiDimKeyVarLengthVariableSplitGenerator(CarbonUtil
-          .getDimensionBitLength(colGrpModel.getColumnGroupCardinality(),
-              colGrpModel.getColumnSplit()), colGrpModel.getColumnSplit());
+      this.columnarSplitter = this.segmentProperties.getFixedLengthKeySplitter();
       System.arraycopy(columnarSplitter.getBlockKeySize(), 0, keyBlockSize, 0, noOfColStore);
       this.keyBlockHolder =
           new CarbonKeyBlockHolder[this.columnarSplitter.getBlockKeySize().length];
@@ -1036,7 +986,7 @@ public class CarbonFactDataHandlerColumnar implements CarbonFactHandler {
     this.dataWriter.setIsNoDictionary(isNoDictionary);
     // initialize the channel;
     this.dataWriter.initializeWriter();
-    initializeColGrpMinMax();
+    //initializeColGrpMinMax();
   }
 
   /**
@@ -1113,7 +1063,8 @@ public class CarbonFactDataHandlerColumnar implements CarbonFactHandler {
     return new CarbonFactDataWriterImplForIntIndexAndAggBlock(storeLocation, measureCount,
         mdKeyLength, tableName, fileManager, keyBlockSize, aggKeyBlock, isComplexTypes(),
         noDictionaryCount, carbonDataFileAttributes, databaseName, wrapperColumnSchemaList,
-        noDictionaryCount, dimensionType, carbonDataDirectoryPath, colCardinality);
+        noDictionaryCount, dimensionType, carbonDataDirectoryPath, colCardinality,
+        segmentProperties);
   }
 
   private boolean[] isComplexTypes() {

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/e96de9f5/processing/src/main/java/org/carbondata/processing/store/CarbonFactDataHandlerModel.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/carbondata/processing/store/CarbonFactDataHandlerModel.java b/processing/src/main/java/org/carbondata/processing/store/CarbonFactDataHandlerModel.java
index a3f7b4f..324d5fe 100644
--- a/processing/src/main/java/org/carbondata/processing/store/CarbonFactDataHandlerModel.java
+++ b/processing/src/main/java/org/carbondata/processing/store/CarbonFactDataHandlerModel.java
@@ -19,9 +19,11 @@
 
 package org.carbondata.processing.store;
 
+import java.util.List;
 import java.util.Map;
 
-import org.carbondata.core.vo.ColumnGroupModel;
+import org.carbondata.core.carbon.datastore.block.SegmentProperties;
+import org.carbondata.core.carbon.metadata.schema.table.column.ColumnSchema;
 import org.carbondata.processing.datatypes.GenericDataType;
 
 /**
@@ -105,14 +107,17 @@ public class CarbonFactDataHandlerModel {
    * map which maintains indexing of complex columns
    */
   private Map<Integer, GenericDataType> complexIndexMap;
+
   /**
-   * primitive dimensions cardinality
+   * Segment properties
    */
-  private int[] primitiveDimLens;
+  private SegmentProperties segmentProperties;
+
   /**
-   * column group model
+   * primitive dimensions cardinality
    */
-  private ColumnGroupModel colGrpModel;
+  private int[] primitiveDimLens;
+
   /**
    * array in which each character represents an aggregation type and
    * the array length will be equal to the number of measures in table
@@ -127,9 +132,18 @@ public class CarbonFactDataHandlerModel {
    */
   private String carbonDataDirectoryPath;
 
+  /**
+   * cardinality of dimension including no dictionary. no dictionary cardinality
+   * is set to -1
+   */
   private int[] colCardinality;
 
   /**
+   * wrapper column schema
+   */
+  private List<ColumnSchema> wrapperColumnSchema;
+
+  /**
    * This is the boolean which will determine whether the data handler call is from the compaction
    * or not.
    */
@@ -302,14 +316,6 @@ public class CarbonFactDataHandlerModel {
     this.primitiveDimLens = primitiveDimLens;
   }
 
-  public ColumnGroupModel getColGrpModel() {
-    return colGrpModel;
-  }
-
-  public void setColGrpModel(ColumnGroupModel colGrpModel) {
-    this.colGrpModel = colGrpModel;
-  }
-
   public char[] getAggType() {
     return aggType;
   }
@@ -341,5 +347,36 @@ public class CarbonFactDataHandlerModel {
   public void setCompactionFlow(boolean compactionFlow) {
     isCompactionFlow = compactionFlow;
   }
+
+  /**
+   *
+   * @return segmentProperties
+   */
+  public SegmentProperties getSegmentProperties() {
+    return segmentProperties;
+  }
+
+  /**
+   *
+   * @param segmentProperties
+   */
+  public void setSegmentProperties(SegmentProperties segmentProperties) {
+    this.segmentProperties = segmentProperties;
+  }
+
+  /**
+   * @return wrapperColumnSchema
+   */
+  public List<ColumnSchema> getWrapperColumnSchema() {
+    return wrapperColumnSchema;
+  }
+
+  /**
+   * @param wrapperColumnSchema
+   */
+  public void setWrapperColumnSchema(List<ColumnSchema> wrapperColumnSchema) {
+    this.wrapperColumnSchema = wrapperColumnSchema;
+  }
+
 }
 

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/e96de9f5/processing/src/main/java/org/carbondata/processing/store/colgroup/ColGroupDataHolder.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/carbondata/processing/store/colgroup/ColGroupDataHolder.java b/processing/src/main/java/org/carbondata/processing/store/colgroup/ColGroupDataHolder.java
index e62c8f2..9907ea2 100644
--- a/processing/src/main/java/org/carbondata/processing/store/colgroup/ColGroupDataHolder.java
+++ b/processing/src/main/java/org/carbondata/processing/store/colgroup/ColGroupDataHolder.java
@@ -18,7 +18,6 @@
  */
 package org.carbondata.processing.store.colgroup;
 
-import org.carbondata.core.vo.ColumnGroupModel;
 
 /**
  * This will hold column group data.
@@ -48,7 +47,7 @@ public class ColGroupDataHolder implements DataHolder {
    * @param colGroupId
    * @param noOfRecords
    */
-  public ColGroupDataHolder(ColumnGroupModel colGrpModel, int keyBlockSize,
+  public ColGroupDataHolder(int keyBlockSize,
        int noOfRecords,ColGroupMinMax colGrpMinMax) {
     this.noOfRecords = noOfRecords;
     this.keyBlockSize = keyBlockSize;
@@ -58,6 +57,7 @@ public class ColGroupDataHolder implements DataHolder {
 
   @Override public void addData(byte[] rowsData, int rowIndex) {
     colGrpData[rowIndex] = rowsData;
+    colGrpMinMax.add(rowsData);
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/e96de9f5/processing/src/main/java/org/carbondata/processing/store/colgroup/ColGroupMinMax.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/carbondata/processing/store/colgroup/ColGroupMinMax.java b/processing/src/main/java/org/carbondata/processing/store/colgroup/ColGroupMinMax.java
index a41bf08..768a128 100644
--- a/processing/src/main/java/org/carbondata/processing/store/colgroup/ColGroupMinMax.java
+++ b/processing/src/main/java/org/carbondata/processing/store/colgroup/ColGroupMinMax.java
@@ -25,11 +25,10 @@ import java.util.Set;
 
 import org.carbondata.common.logging.LogService;
 import org.carbondata.common.logging.LogServiceFactory;
+import org.carbondata.core.carbon.datastore.block.SegmentProperties;
 import org.carbondata.core.keygenerator.KeyGenException;
 import org.carbondata.core.keygenerator.KeyGenerator;
-import org.carbondata.core.keygenerator.columnar.ColumnarSplitter;
 import org.carbondata.core.util.ByteUtil;
-import org.carbondata.core.vo.ColumnGroupModel;
 
 /**
  * it gives min max of each column of column group
@@ -38,10 +37,6 @@ public class ColGroupMinMax {
 
   private static final LogService LOGGER =
       LogServiceFactory.getLogService(ColGroupMinMax.class.getName());
-  /**
-   * Column group model
-   */
-  private ColumnGroupModel colGrpModel;
 
   /**
    * key generator
@@ -49,11 +44,6 @@ public class ColGroupMinMax {
   private KeyGenerator keyGenerator;
 
   /**
-   * column group id
-   */
-  private int colGroupId;
-
-  /**
    * no of column in column group
    */
   private int noOfCol;
@@ -78,12 +68,9 @@ public class ColGroupMinMax {
    */
   private byte[][] maxKeys;
 
-  public ColGroupMinMax(ColumnGroupModel colGrpModel, ColumnarSplitter columnarSplitter,
-      int colGroupId) {
-    this.colGrpModel = colGrpModel;
-    this.keyGenerator = (KeyGenerator) columnarSplitter;
-    this.colGroupId = colGroupId;
-    this.noOfCol = colGrpModel.getColumnSplit()[colGroupId];
+  public ColGroupMinMax(SegmentProperties segmentProperties, int colGroupId) {
+    this.keyGenerator = segmentProperties.getColumnGroupAndItsKeygenartor().get(colGroupId);
+    this.noOfCol = segmentProperties.getNoOfColumnsInColumnGroup(colGroupId);
     min = new byte[noOfCol][];
     max = new byte[noOfCol][];
     initialise();
@@ -126,15 +113,14 @@ public class ColGroupMinMax {
       maskByteRange = new int[noOfCol][];
       maxKeys = new byte[noOfCol][];
       for (int i = 0; i < noOfCol; i++) {
-        maskByteRange[i] = getMaskByteRange(colGrpModel.getColumnGroup()[colGroupId][i]);
+        maskByteRange[i] = getMaskByteRange(i);
         // generating maxkey
-        long[] maxKey = new long[keyGenerator.getKeySizeInBytes()];
-        maxKey[colGrpModel.getColumnGroup()[colGroupId][i]] = Long.MAX_VALUE;
+        long[] maxKey = new long[noOfCol];
+        maxKey[i] = Long.MAX_VALUE;
         maxKeys[i] = keyGenerator.generateKey(maxKey);
       }
     } catch (KeyGenException e) {
-      LOGGER.error(e,
-          "Key generation failed while evaulating column group min max");
+      LOGGER.error(e, "Key generation failed while evaulating column group min max");
     }
 
   }

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/e96de9f5/processing/src/main/java/org/carbondata/processing/store/writer/AbstractFactDataWriter.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/carbondata/processing/store/writer/AbstractFactDataWriter.java b/processing/src/main/java/org/carbondata/processing/store/writer/AbstractFactDataWriter.java
index 4e49806..f1c7ad5 100644
--- a/processing/src/main/java/org/carbondata/processing/store/writer/AbstractFactDataWriter.java
+++ b/processing/src/main/java/org/carbondata/processing/store/writer/AbstractFactDataWriter.java
@@ -39,6 +39,7 @@ import java.util.concurrent.TimeUnit;
 
 import org.carbondata.common.logging.LogService;
 import org.carbondata.common.logging.LogServiceFactory;
+import org.carbondata.core.carbon.datastore.block.SegmentProperties;
 import org.carbondata.core.carbon.metadata.CarbonMetadata;
 import org.carbondata.core.carbon.metadata.blocklet.index.BlockletBTreeIndex;
 import org.carbondata.core.carbon.metadata.blocklet.index.BlockletIndex;
@@ -171,12 +172,14 @@ public abstract class AbstractFactDataWriter<T> implements CarbonFactDataWriter<
   private int spaceReservedForBlockMetaSize;
   private FileOutputStream fileOutputStream;
 
+  private SegmentProperties segmentProperties;
+
   private List<BlockIndexInfo> blockIndexInfoList;
 
   public AbstractFactDataWriter(String storeLocation, int measureCount, int mdKeyLength,
       String databaseName, String tableName, IFileManagerComposite fileManager, int[] keyBlockSize,
       CarbonDataFileAttributes carbonDataFileAttributes, List<ColumnSchema> columnSchema,
-      String carbonDataDirectoryPath, int[] colCardinality) {
+      String carbonDataDirectoryPath, int[] colCardinality, SegmentProperties segmentProperties) {
 
     // measure count
     this.measureCount = measureCount;
@@ -187,6 +190,7 @@ public abstract class AbstractFactDataWriter<T> implements CarbonFactDataWriter<
     this.databaseName = databaseName;
 
     this.storeLocation = storeLocation;
+    this.segmentProperties = segmentProperties;
     this.blockletInfoList =
         new ArrayList<BlockletInfoColumnar>(CarbonCommonConstants.CONSTANT_SIZE_TEN);
     blockIndexInfoList = new ArrayList<>();
@@ -356,7 +360,7 @@ public abstract class AbstractFactDataWriter<T> implements CarbonFactDataWriter<
       CarbonFooterWriter writer = new CarbonFooterWriter(filePath);
       FileFooter convertFileMeta = CarbonMetadataUtil
           .convertFileFooter(infoList, localCardinality.length, localCardinality,
-              thriftColumnSchemaList);
+              thriftColumnSchemaList, segmentProperties);
       fillBlockIndexInfoDetails(infoList, convertFileMeta.getNum_rows(), filePath, currentPosition);
       writer.writeFooter(convertFileMeta, currentPosition);
     } catch (IOException e) {

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/e96de9f5/processing/src/main/java/org/carbondata/processing/store/writer/CarbonFactDataWriterImplForIntIndexAndAggBlock.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/carbondata/processing/store/writer/CarbonFactDataWriterImplForIntIndexAndAggBlock.java b/processing/src/main/java/org/carbondata/processing/store/writer/CarbonFactDataWriterImplForIntIndexAndAggBlock.java
index 6177385..24fa88d 100644
--- a/processing/src/main/java/org/carbondata/processing/store/writer/CarbonFactDataWriterImplForIntIndexAndAggBlock.java
+++ b/processing/src/main/java/org/carbondata/processing/store/writer/CarbonFactDataWriterImplForIntIndexAndAggBlock.java
@@ -24,6 +24,7 @@ import java.nio.ByteBuffer;
 import java.nio.channels.FileChannel;
 import java.util.List;
 
+import org.carbondata.core.carbon.datastore.block.SegmentProperties;
 import org.carbondata.core.carbon.metadata.schema.table.column.ColumnSchema;
 import org.carbondata.core.constants.CarbonCommonConstants;
 import org.carbondata.core.datastorage.store.columnar.IndexStorage;
@@ -50,10 +51,11 @@ public class CarbonFactDataWriterImplForIntIndexAndAggBlock extends AbstractFact
       boolean[] aggBlocks, boolean[] isComplexType, int NoDictionaryCount,
       CarbonDataFileAttributes carbonDataFileAttributes, String databaseName,
       List<ColumnSchema> wrapperColumnSchemaList, int numberOfNoDictionaryColumn,
-      boolean[] isDictionaryColumn, String carbonDataDirectoryPath, int[] colCardinality) {
+      boolean[] isDictionaryColumn, String carbonDataDirectoryPath, int[] colCardinality,
+      SegmentProperties segmentProperties) {
     super(storeLocation, measureCount, mdKeyLength, databaseName, tableName, fileManager,
         keyBlockSize, carbonDataFileAttributes, wrapperColumnSchemaList, carbonDataDirectoryPath,
-        colCardinality);
+        colCardinality, segmentProperties);
     this.isComplexType = isComplexType;
     this.databaseName = databaseName;
     this.numberOfNoDictionaryColumn = numberOfNoDictionaryColumn;