You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@carbondata.apache.org by qi...@apache.org on 2020/02/12 09:42:16 UTC

[carbondata] branch master updated: [CARBONDATA-3684] Remove MDK and cardinality in write path

This is an automated email from the ASF dual-hosted git repository.

qiangcai pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/carbondata.git


The following commit(s) were added to refs/heads/master by this push:
     new 1e678fe  [CARBONDATA-3684] Remove MDK and cardinality in write path
1e678fe is described below

commit 1e678feeca252f4cf79cd3b1860dc5d8275dd801
Author: Jacky Li <ja...@qq.com>
AuthorDate: Fri Feb 7 13:05:43 2020 +0800

    [CARBONDATA-3684] Remove MDK and cardinality in write path
    
    Why is this PR needed?
    Since only DATE is dictionary now, MDK is always 4 bytes. We can simplify many places in write path to remove MDK and cardinality calculation to save memory
    
    What changes were proposed in this PR?
    MDK generation is removed
    cardinality calculation is removed
    some other unused declaration is removed
    
    This closes #3598
---
 .../core/constants/CarbonCommonConstants.java      |   2 -
 .../carbondata/core/datamap/DataMapChooser.java    |  19 -
 .../carbondata/core/datastore/DataRefNode.java     |   7 -
 .../apache/carbondata/core/datastore/IndexKey.java |  61 ---
 .../core/datastore/block/AbstractIndex.java        |   9 -
 .../carbondata/core/datastore/block/BlockInfo.java | 137 ------
 .../core/datastore/block/BlockletInfos.java        | 112 -----
 .../core/datastore/block/SegmentProperties.java    | 375 ++++-----------
 .../block/SegmentPropertiesAndSchemaHolder.java    |  24 +-
 .../core/datastore/block/TableBlockInfo.java       | 167 +------
 .../block/TableBlockUniqueIdentifier.java          |  90 ----
 .../core/datastore/blocklet/EncodedBlocklet.java   |  16 -
 .../chunk/reader/CarbonDataReaderFactory.java      |  10 +-
 .../dimension/AbstractDimensionChunkReader.java    |  11 +-
 .../dimension/v3/DimensionChunkPageReaderV3.java   |   8 +-
 .../dimension/v3/DimensionChunkReaderV3.java       |  13 +-
 .../measure/v3/MeasureChunkPageReaderV3.java       |   3 +-
 .../core/datastore/page/EncodedTablePage.java      |  17 +-
 .../core/datastore/page/key/TablePageKey.java      | 210 ---------
 .../core/datastore/row/WriteStepRowUtil.java       |  14 +-
 .../core/indexstore/BlockletDetailInfo.java        |  19 -
 .../core/indexstore/ExtendedBlocklet.java          |   8 -
 .../indexstore/blockletindex/BlockDataMap.java     | 125 +----
 .../indexstore/blockletindex/BlockletDataMap.java  |  55 +--
 .../blockletindex/BlockletDataRefNode.java         |  47 +-
 .../indexstore/blockletindex/IndexWrapper.java     |   8 +-
 .../core/indexstore/schema/SchemaGenerator.java    |   8 +-
 .../carbondata/core/keygenerator/KeyGenerator.java |  47 --
 .../keygenerator/columnar/ColumnarSplitter.java    |  79 ----
 .../MultiDimKeyVarLengthEquiSplitGenerator.java    | 207 ---------
 ...MultiDimKeyVarLengthVariableSplitGenerator.java | 209 ---------
 .../keygenerator/factory/KeyGeneratorFactory.java  |  11 -
 .../mdkey/MultiDimKeyVarLengthGenerator.java       |  32 --
 .../carbondata/core/metadata/CarbonMetadata.java   |  11 -
 .../carbondata/core/metadata/SegmentFileStore.java |   2 +-
 .../core/metadata/blocklet/BlockletInfo.java       |  39 --
 .../core/metadata/blocklet/DataFileFooter.java     |  27 +-
 .../core/metadata/blocklet/SegmentInfo.java        |  52 ---
 .../metadata/blocklet/datachunk/DataChunk.java     | 228 ---------
 .../core/metadata/schema/table/CarbonTable.java    |  19 +-
 .../schema/table/column/CarbonDimension.java       |  23 +-
 .../impl/RestructureBasedRawResultCollector.java   |  95 +---
 .../RestructureBasedVectorResultCollector.java     |  12 -
 .../core/scan/complextypes/ArrayQueryType.java     |   4 +-
 .../core/scan/complextypes/ComplexQueryType.java   |  18 +-
 .../core/scan/complextypes/MapQueryType.java       |   4 +-
 .../core/scan/complextypes/PrimitiveQueryType.java |  21 +-
 .../core/scan/complextypes/StructQueryType.java    |   4 +-
 .../scan/executor/impl/AbstractQueryExecutor.java  |  77 +---
 .../scan/executor/infos/BlockExecutionInfo.java    |  62 ---
 .../core/scan/executor/util/QueryUtil.java         | 169 +------
 .../core/scan/executor/util/RestructureUtil.java   |  20 -
 .../scan/filter/FilterExpressionProcessor.java     |   2 +-
 .../carbondata/core/scan/filter/FilterUtil.java    | 511 +--------------------
 .../executer/RangeValueFilterExecuterImpl.java     |   2 +-
 .../executer/RowLevelFilterExecuterImpl.java       |  16 -
 .../RowLevelRangeGrtThanFiterExecuterImpl.java     |   2 +-
 ...LevelRangeGrtrThanEquaToFilterExecuterImpl.java |   2 +-
 ...wLevelRangeLessThanEqualFilterExecuterImpl.java |   2 +-
 .../RowLevelRangeLessThanFilterExecuterImpl.java   |   2 +-
 .../resolver/ConditionalFilterResolverImpl.java    |  32 +-
 .../scan/filter/resolver/FilterResolverIntf.java   |  28 +-
 .../filter/resolver/LogicalFilterResolverImpl.java |  20 +-
 .../resolver/RowLevelFilterResolverImpl.java       |   2 +-
 .../resolver/RowLevelRangeFilterResolverImpl.java  |  50 +-
 .../resolver/metadata/FilterResolverMetadata.java  |  10 -
 .../resolverinfo/FalseConditionalResolverImpl.java |   3 +-
 .../resolverinfo/TrueConditionalResolverImpl.java  |   3 +-
 .../carbondata/core/scan/processor/BlockScan.java  | 104 -----
 .../core/scan/result/BlockletScannedResult.java    |  24 -
 .../scan/result/impl/FilterQueryScannedResult.java |   9 -
 .../result/impl/NonFilterQueryScannedResult.java   |  10 -
 .../scan/result/iterator/RawResultIterator.java    |   6 -
 .../carbondata/core/stream/StreamPruner.java       |   7 +-
 .../core/util/AbstractDataFileFooterConverter.java | 111 +----
 .../carbondata/core/util/BlockletDataMapUtil.java  |  86 +---
 .../org/apache/carbondata/core/util/ByteUtil.java  |  41 ++
 .../carbondata/core/util/CarbonMetadataUtil.java   |  49 +-
 .../apache/carbondata/core/util/CarbonUtil.java    |  44 +-
 .../core/util/DataFileFooterConverter.java         |  21 -
 .../core/util/DataFileFooterConverter2.java        | 155 -------
 .../core/util/DataFileFooterConverterV3.java       |   1 -
 .../carbondata/core/util/NonDictionaryUtil.java    |  92 ----
 .../apache/carbondata/hadoop/CarbonInputSplit.java |  34 +-
 .../core/datastore/block/BlockInfoTest.java        |  91 ----
 .../datastore/block/SegmentPropertiesTest.java     |  42 +-
 .../datastore/block/SegmentPropertiesTestUtil.java |   8 +-
 .../core/datastore/block/TableBlockInfoTest.java   |  28 +-
 ...iDimKeyVarLengthEquiSplitGeneratorUnitTest.java | 197 --------
 ...KeyVarLengthVariableSplitGeneratorUnitTest.java | 148 ------
 .../factory/KeyGeneratorFactoryUnitTest.java       |  61 ---
 .../scan/complextypes/PrimitiveQueryTypeTest.java  |  35 +-
 .../core/scan/executor/util/QueryUtilTest.java     |  90 ----
 .../core/scan/filter/FilterUtilTest.java           |  74 ---
 .../core/util/CarbonMetadataUtilTest.java          |   5 +-
 .../carbondata/core/util/CarbonUtilTest.java       |  43 --
 .../core/util/DataFileFooterConverterTest.java     |   6 -
 .../core/util/RangeFilterProcessorTest.java        |  15 +-
 .../datamap/bloom/BloomDataMapWriter.java          |  34 +-
 .../hadoop/stream/CarbonStreamInputFormat.java     |  12 +-
 .../hadoop/stream/StreamRecordReader.java          |   8 +-
 .../carbondata/hive/CarbonHiveInputSplit.java      |  21 +-
 .../carbondata/hive/Hive2CarbonExpressionTest.java |  16 +-
 .../LoadTableWithLocalDictionaryTestCase.scala     |   1 -
 .../allqueries/AllDataTypesTestCase.scala          |   2 +-
 .../TestNonTransactionalCarbonTable.scala          |   1 -
 .../LocalDictionarySupportLoadTableTest.scala      |   1 -
 .../spark/load/DataLoadProcessorStepOnSpark.scala  |  26 +-
 .../carbondata/spark/rdd/CarbonIUDMergerRDD.scala  |   1 -
 .../carbondata/spark/rdd/CarbonMergerRDD.scala     |  30 +-
 .../carbondata/spark/rdd/StreamHandoffRDD.scala    |   7 +-
 .../org/apache/spark/rdd/CarbonMergeFilesRDD.scala |   1 -
 .../spark/sql/catalyst/CarbonDDLSqlParser.scala    |   1 -
 .../command/carbonTableSchemaCommon.scala          |   2 -
 .../vectorreader/VectorizedCarbonRecordReader.java |   1 -
 .../datamap/IndexDataMapRebuildRDD.scala           |  69 +--
 .../spark/rdd/CarbonTableCompactor.scala           |   5 +-
 .../command/management/CarbonLoadDataCommand.scala |   3 -
 .../BloomCoarseGrainDataMapFunctionSuite.scala     |   4 +-
 .../booleantype/BooleanDataTypesLoadTest.scala     |  28 +-
 .../processing/datamap/DataMapWriterListener.java  |  19 +-
 .../processing/datatypes/ArrayDataType.java        |  53 +--
 .../processing/datatypes/GenericDataType.java      |  37 +-
 .../processing/datatypes/PrimitiveDataType.java    |  48 +-
 .../processing/datatypes/StructDataType.java       |  63 +--
 .../loading/CarbonDataLoadConfiguration.java       |  25 -
 .../processing/loading/DataLoadProcessBuilder.java |   2 -
 .../converter/DictionaryCardinalityFinder.java     |  26 --
 .../processing/loading/converter/RowConverter.java |   2 +-
 .../impl/AbstractDictionaryFieldConverterImpl.java |  28 --
 .../converter/impl/ComplexFieldConverterImpl.java  |   8 +-
 .../impl/DirectDictionaryFieldConverterImpl.java   |   9 +-
 .../loading/converter/impl/RowConverterImpl.java   |  18 -
 .../processing/loading/model/CarbonLoadModel.java  |  19 -
 .../steps/DataConverterProcessorStepImpl.java      |   1 -
 .../InputProcessorStepWithNoConverterImpl.java     |   1 -
 .../merger/CarbonCompactionExecutor.java           |  20 +-
 .../processing/merger/CarbonCompactionUtil.java    |  81 +---
 .../merger/CompactionResultSortProcessor.java      |   7 +-
 .../merger/RowResultMergerProcessor.java           |   2 +-
 .../store/CarbonFactDataHandlerColumnar.java       |  46 +-
 .../store/CarbonFactDataHandlerModel.java          | 170 +------
 .../carbondata/processing/store/TablePage.java     |  26 +-
 .../store/writer/AbstractFactDataWriter.java       |  22 +-
 .../writer/v3/CarbonFactDataWriterImplV3.java      |   2 +-
 .../processing/util/CarbonDataProcessorUtil.java   |  33 --
 .../streaming/CarbonStreamRecordWriter.java        |  12 +-
 .../org/apache/carbondata/tool/ScanBenchmark.java  |   7 +-
 148 files changed, 514 insertions(+), 5918 deletions(-)

diff --git a/core/src/main/java/org/apache/carbondata/core/constants/CarbonCommonConstants.java b/core/src/main/java/org/apache/carbondata/core/constants/CarbonCommonConstants.java
index 34aa91e..4356a4b 100644
--- a/core/src/main/java/org/apache/carbondata/core/constants/CarbonCommonConstants.java
+++ b/core/src/main/java/org/apache/carbondata/core/constants/CarbonCommonConstants.java
@@ -2063,8 +2063,6 @@ public final class CarbonCommonConstants {
   public static final String MINOR = "minor";
   public static final String MAJOR = "major";
 
-  public static final int DICTIONARY_DEFAULT_CARDINALITY = 1;
-
   /**
    * this will be used to provide comment for table
    */
diff --git a/core/src/main/java/org/apache/carbondata/core/datamap/DataMapChooser.java b/core/src/main/java/org/apache/carbondata/core/datamap/DataMapChooser.java
index 8fcdc74..2831094 100644
--- a/core/src/main/java/org/apache/carbondata/core/datamap/DataMapChooser.java
+++ b/core/src/main/java/org/apache/carbondata/core/datamap/DataMapChooser.java
@@ -136,25 +136,6 @@ public class DataMapChooser {
   }
 
   /**
-   * Get all datamaps of the table for clearing purpose
-   */
-  public DataMapExprWrapper getAllDataMapsForClear(CarbonTable carbonTable)
-      throws IOException {
-    List<TableDataMap> allDataMapFG =
-        DataMapStoreManager.getInstance().getAllDataMap(carbonTable);
-    DataMapExprWrapper initialExpr = null;
-    if (allDataMapFG.size() > 0) {
-      initialExpr = new DataMapExprWrapperImpl(allDataMapFG.get(0), null);
-
-      for (int i = 1; i < allDataMapFG.size(); i++) {
-        initialExpr = new AndDataMapExprWrapper(initialExpr,
-            new DataMapExprWrapperImpl(allDataMapFG.get(i), null), null);
-      }
-    }
-    return initialExpr;
-  }
-
-  /**
    * Returns default blocklet datamap
    * @param carbonTable
    * @param resolverIntf
diff --git a/core/src/main/java/org/apache/carbondata/core/datastore/DataRefNode.java b/core/src/main/java/org/apache/carbondata/core/datastore/DataRefNode.java
index dc356a1..07a343d 100644
--- a/core/src/main/java/org/apache/carbondata/core/datastore/DataRefNode.java
+++ b/core/src/main/java/org/apache/carbondata/core/datastore/DataRefNode.java
@@ -40,13 +40,6 @@ public interface DataRefNode {
   int numRows();
 
   /**
-   * Return the block index. This can be used when multiple
-   * thread can be used scan group of blocks in that can we can assign
-   * some of the blocks to one thread and some to other
-   */
-  long nodeIndex();
-
-  /**
    * Return the blocklet index in the node
    */
   short blockletIndex();
diff --git a/core/src/main/java/org/apache/carbondata/core/datastore/IndexKey.java b/core/src/main/java/org/apache/carbondata/core/datastore/IndexKey.java
deleted file mode 100644
index 547a564..0000000
--- a/core/src/main/java/org/apache/carbondata/core/datastore/IndexKey.java
+++ /dev/null
@@ -1,61 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.carbondata.core.datastore;
-
-/**
- * Index class to store the index of the segment blocklet infos
- */
-public class IndexKey {
-
-  /**
-   * key which is generated from key generator
-   */
-  private byte[] dictionaryKeys;
-
-  /**
-   * key which was no generated using key generator
-   * <Index of FirstKey (2 bytes)><Index of SecondKey (2 bytes)><Index of NKey (2 bytes)>
-   * <First Key ByteArray><2nd Key ByteArray><N Key ByteArray>
-   */
-  private byte[] noDictionaryKeys;
-
-  public IndexKey(byte[] dictionaryKeys, byte[] noDictionaryKeys) {
-    this.dictionaryKeys = dictionaryKeys;
-    this.noDictionaryKeys = noDictionaryKeys;
-    if (null == dictionaryKeys) {
-      this.dictionaryKeys = new byte[0];
-    }
-    if (null == noDictionaryKeys) {
-      this.noDictionaryKeys = new byte[0];
-    }
-  }
-
-  /**
-   * @return the dictionaryKeys
-   */
-  public byte[] getDictionaryKeys() {
-    return dictionaryKeys;
-  }
-
-  /**
-   * @return the noDictionaryKeys
-   */
-  public byte[] getNoDictionaryKeys() {
-    return noDictionaryKeys;
-  }
-}
diff --git a/core/src/main/java/org/apache/carbondata/core/datastore/block/AbstractIndex.java b/core/src/main/java/org/apache/carbondata/core/datastore/block/AbstractIndex.java
index 4c415eb..c93e162 100644
--- a/core/src/main/java/org/apache/carbondata/core/datastore/block/AbstractIndex.java
+++ b/core/src/main/java/org/apache/carbondata/core/datastore/block/AbstractIndex.java
@@ -17,13 +17,11 @@
 
 package org.apache.carbondata.core.datastore.block;
 
-import java.util.List;
 import java.util.Map;
 import java.util.concurrent.atomic.AtomicInteger;
 
 import org.apache.carbondata.core.cache.Cacheable;
 import org.apache.carbondata.core.datastore.DataRefNode;
-import org.apache.carbondata.core.metadata.blocklet.DataFileFooter;
 import org.apache.carbondata.core.mutate.DeleteDeltaVo;
 
 public abstract class AbstractIndex implements Cacheable {
@@ -73,13 +71,6 @@ public abstract class AbstractIndex implements Cacheable {
   }
 
   /**
-   * Below method will be used to load the data block
-   *
-   * @param footerList footer list
-   */
-  public abstract void buildIndex(List<DataFileFooter> footerList);
-
-  /**
    * the method returns the access count
    *
    * @return
diff --git a/core/src/main/java/org/apache/carbondata/core/datastore/block/BlockInfo.java b/core/src/main/java/org/apache/carbondata/core/datastore/block/BlockInfo.java
deleted file mode 100644
index 697574f..0000000
--- a/core/src/main/java/org/apache/carbondata/core/datastore/block/BlockInfo.java
+++ /dev/null
@@ -1,137 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.carbondata.core.datastore.block;
-
-import java.io.Serializable;
-
-import org.apache.carbondata.core.constants.CarbonCommonConstants;
-import org.apache.carbondata.core.util.path.CarbonTablePath;
-
-/**
- * Below class will be used to store table block info
- * As in blocklet distribution we are dividing the same block
- * in parts but in case of block loading blocklets belongs to same
- * block will be loaded together. This class will be used to store table block info
- * and equals and hash code method is used to identify blocklet belongs to same block
- */
-public class BlockInfo implements Serializable {
-
-  /**
-   * table block info, stores all the details
-   * about the block
-   */
-  private TableBlockInfo info;
-  /**
-   * unique blockName
-   */
-  private String blockUniqueName;
-
-  /**
-   * Constructor
-   *
-   * @param info
-   */
-  public BlockInfo(TableBlockInfo info) {
-    this.info = info;
-    init();
-  }
-
-  /**
-   * init the block unique name
-   */
-  private void init() {
-    StringBuilder stringBuilder = new StringBuilder();
-    stringBuilder.append(this.info.getSegmentId());
-    stringBuilder.append(CarbonCommonConstants.FILE_SEPARATOR);
-    stringBuilder.append(CarbonTablePath.getCarbonDataFileName(this.info.getFilePath()));
-    this.blockUniqueName = stringBuilder.toString();
-  }
-
-  /**
-   * @return table Block info
-   */
-  public TableBlockInfo getTableBlockInfo() {
-    return info;
-  }
-
-  /**
-   * To set the table block info
-   *
-   * @param info
-   */
-  public void setTableBlockInfo(TableBlockInfo info) {
-    this.info = info;
-  }
-
-  /**
-   * method to get the hash code
-   */
-  @Override
-  public int hashCode() {
-    int result = info.getFilePath().hashCode();
-    result = 31 * result + (int) (info.getBlockOffset() ^ (info.getBlockOffset() >>> 32));
-    result = 31 * result + (int) (info.getBlockLength() ^ (info.getBlockLength() >>> 32));
-    result = 31 * result + info.getSegmentId().hashCode();
-    return result;
-  }
-
-  /**
-   * To check the equality
-   *
-   * @param obj
-   */
-  @Override
-  public boolean equals(Object obj) {
-    if (this == obj) {
-      return true;
-    }
-    if (obj == null) {
-      return false;
-    }
-    if (!(obj instanceof BlockInfo)) {
-      return false;
-    }
-    BlockInfo other = (BlockInfo) obj;
-    if (!info.getSegmentId().equals(other.info.getSegmentId())) {
-      return false;
-    }
-    if (info.getBlockOffset() != other.info.getBlockOffset()) {
-      return false;
-    }
-    if (info.getBlockLength() != other.info.getBlockLength()) {
-      return false;
-    }
-
-    if (info.getFilePath() == null && other.info.getFilePath() != null) {
-      return false;
-    } else if (info.getFilePath() != null && other.info.getFilePath() == null) {
-      return false;
-    } else if (!info.getFilePath().equals(other.info.getFilePath())) {
-      return false;
-    }
-    return true;
-  }
-
-  /**
-   * returns unique blockname
-   * @return
-   */
-  public String getBlockUniqueName() {
-    return blockUniqueName;
-  }
-}
diff --git a/core/src/main/java/org/apache/carbondata/core/datastore/block/BlockletInfos.java b/core/src/main/java/org/apache/carbondata/core/datastore/block/BlockletInfos.java
deleted file mode 100644
index 999de10..0000000
--- a/core/src/main/java/org/apache/carbondata/core/datastore/block/BlockletInfos.java
+++ /dev/null
@@ -1,112 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.carbondata.core.datastore.block;
-
-import java.io.Serializable;
-
-/**
- * The class holds the blocks blocklets info
- */
-public class BlockletInfos implements Serializable {
-  /**
-   * no of blockLets
-   */
-  private int noOfBlockLets = 0;
-
-  /**
-   * start blocklet number
-   */
-  private int startBlockletNumber;
-  /**
-   * end blocklet number
-   */
-  private int numberOfBlockletToScan;
-  /**
-   * default constructor
-   */
-  public BlockletInfos() {
-  }
-
-  /**
-   * constructor to initialize the blockletinfo
-   * @param noOfBlockLets
-   * @param startBlockletNumber
-   * @param numberOfBlockletToScan
-   */
-  public BlockletInfos(int noOfBlockLets, int startBlockletNumber, int numberOfBlockletToScan) {
-    this.noOfBlockLets = noOfBlockLets;
-    this.startBlockletNumber = startBlockletNumber;
-    this.numberOfBlockletToScan = numberOfBlockletToScan;
-  }
-
-  /**
-   * returns the number of blockLets
-   *
-   * @return
-   */
-  public int getNoOfBlockLets() {
-    return noOfBlockLets;
-  }
-
-  /**
-   * sets the number of blockLets
-   *
-   * @param noOfBlockLets
-   */
-  public void setNoOfBlockLets(int noOfBlockLets) {
-    this.noOfBlockLets = noOfBlockLets;
-  }
-
-  /**
-   * returns start blocklet number
-   *
-   * @return
-   */
-  public int getStartBlockletNumber() {
-    return startBlockletNumber;
-  }
-
-  /**
-   * set start blocklet number
-   *
-   * @param startBlockletNumber
-   */
-  public void setStartBlockletNumber(int startBlockletNumber) {
-    this.startBlockletNumber = startBlockletNumber;
-  }
-
-  /**
-   * returns end blocklet number
-   *
-   * @return
-   */
-  public int getNumberOfBlockletToScan() {
-    return numberOfBlockletToScan;
-  }
-
-  /**
-   * set end blocklet number to be scaned
-   *
-   * @param numberOfBlockletToScan
-   */
-  public void setNumberOfBlockletToScan(int numberOfBlockletToScan) {
-    this.numberOfBlockletToScan = numberOfBlockletToScan;
-  }
-
-}
-
diff --git a/core/src/main/java/org/apache/carbondata/core/datastore/block/SegmentProperties.java b/core/src/main/java/org/apache/carbondata/core/datastore/block/SegmentProperties.java
index bd32cb8..056d5e4 100644
--- a/core/src/main/java/org/apache/carbondata/core/datastore/block/SegmentProperties.java
+++ b/core/src/main/java/org/apache/carbondata/core/datastore/block/SegmentProperties.java
@@ -18,7 +18,6 @@
 package org.apache.carbondata.core.datastore.block;
 
 import java.util.ArrayList;
-import java.util.Arrays;
 import java.util.HashMap;
 import java.util.HashSet;
 import java.util.Iterator;
@@ -28,10 +27,6 @@ import java.util.Map.Entry;
 import java.util.Set;
 
 import org.apache.carbondata.core.constants.CarbonCommonConstants;
-import org.apache.carbondata.core.keygenerator.KeyGenerator;
-import org.apache.carbondata.core.keygenerator.columnar.ColumnarSplitter;
-import org.apache.carbondata.core.keygenerator.columnar.impl.MultiDimKeyVarLengthVariableSplitGenerator;
-import org.apache.carbondata.core.keygenerator.mdkey.MultiDimKeyVarLengthGenerator;
 import org.apache.carbondata.core.metadata.datatype.DataType;
 import org.apache.carbondata.core.metadata.datatype.DataTypes;
 import org.apache.carbondata.core.metadata.encoder.Encoding;
@@ -40,8 +35,6 @@ import org.apache.carbondata.core.metadata.schema.table.column.CarbonMeasure;
 import org.apache.carbondata.core.metadata.schema.table.column.ColumnSchema;
 import org.apache.carbondata.core.util.CarbonUtil;
 
-import org.apache.commons.lang3.ArrayUtils;
-
 /**
  * This class contains all the details about the restructuring information of
  * the block. This will be used during query execution to handle restructure
@@ -50,20 +43,6 @@ import org.apache.commons.lang3.ArrayUtils;
 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;
-
-  /**
-   * key generator which was used to generate the mdkey for dimensions in SORT_COLUMNS
-   * if SORT_COLUMNS contains all dimensions, it is same with dimensionKeyGenerator
-   * otherwise, it is different with dimensionKeyGenerator, the number of its dimensions is less
-   * than dimensionKeyGenerator.
-   */
-  private KeyGenerator sortColumnsGenerator;
-
-  /**
    * list of dimension present in the block
    */
   private List<CarbonDimension> dimensions;
@@ -79,21 +58,6 @@ public class SegmentProperties {
   private List<CarbonMeasure> measures;
 
   /**
-   * cardinality of dimension columns participated in key generator
-   */
-  private int[] dimColumnsCardinality;
-
-  /**
-   * partition index of each dictionary column
-   */
-  private int[] dimensionPartitions;
-
-  /**
-   * cardinality of complex dimension
-   */
-  private int[] complexDimColumnCardinality;
-
-  /**
    * mapping of dimension ordinal in schema to column chunk index in the data file
    */
   private Map<Integer, Integer> dimensionOrdinalToChunkMapping;
@@ -110,29 +74,6 @@ public class SegmentProperties {
   private Map<Integer, Integer> measuresOrdinalToChunkMapping;
 
   /**
-   * 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;
-
-  /**
-   * 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
@@ -140,18 +81,20 @@ public class SegmentProperties {
    */
   private int numberOfNoDictionaryDimension;
 
-  private int numberOfSortColumns = 0;
+  private int numberOfSortColumns;
 
-  private int numberOfNoDictSortColumns = 0;
+  private int numberOfDictDimensions;
+
+  private int numberOfColumnsAfterFlatten;
 
   private int lastDimensionColOrdinal;
 
-  public SegmentProperties(List<ColumnSchema> columnsInTable, int[] columnCardinality) {
+  public SegmentProperties(List<ColumnSchema> columnsInTable) {
     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);
+    fillDimensionAndMeasureDetails(columnsInTable);
     dimensionOrdinalToChunkMapping =
         new HashMap<Integer, Integer>(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE);
     blockTodimensionOrdinalMapping =
@@ -160,7 +103,6 @@ public class SegmentProperties {
         new HashMap<Integer, Integer>(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE);
     fillOrdinalToBlockMappingForDimension();
     fillOrdinalToChunkIndexMappingForMeasureColumns();
-    fillKeyGeneratorDetails();
   }
 
   /**
@@ -245,10 +187,8 @@ public class SegmentProperties {
    * below method will fill dimension and measure detail of the block.
    *
    * @param columnsInTable
-   * @param columnCardinality
    */
-  private void fillDimensionAndMeasureDetails(List<ColumnSchema> columnsInTable,
-      int[] columnCardinality) {
+  private void fillDimensionAndMeasureDetails(List<ColumnSchema> columnsInTable) {
     ColumnSchema columnSchema = null;
     // ordinal will be required to read the data from file block
     int dimensionOrdinal = 0;
@@ -257,31 +197,20 @@ public class SegmentProperties {
     // 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 counter = 0;
     int complexTypeOrdinal = -1;
     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);
+          this.numberOfDictDimensions++;
+          this.numberOfColumnsAfterFlatten++;
           if (columnSchema.isSortColumn()) {
             this.numberOfSortColumns++;
           }
@@ -293,53 +222,35 @@ public class SegmentProperties {
         // 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 || columnSchema.getDataType().isComplexType()) {
-          cardinalityIndexForComplexDimensionColumn.add(tableOrdinal);
           carbonDimension =
               new CarbonDimension(columnSchema, dimensionOrdinal++, -1, ++complexTypeOrdinal);
           carbonDimension.initializeChildDimensionsList(columnSchema.getNumberOfChild());
           complexDimensions.add(carbonDimension);
           isComplexDimensionStarted = true;
-          int previousOrdinal = dimensionOrdinal;
           dimensionOrdinal =
               readAllComplexTypeChildren(dimensionOrdinal, columnSchema.getNumberOfChild(),
                   columnsInTable, carbonDimension, complexTypeOrdinal);
-          int numberOfChildrenDimensionAdded = dimensionOrdinal - previousOrdinal;
-          for (int i = 0; i < numberOfChildrenDimensionAdded; i++) {
-            cardinalityIndexForComplexDimensionColumn.add(++tableOrdinal);
-          }
           counter = dimensionOrdinal;
           complexTypeOrdinal = assignComplexOrdinal(carbonDimension, complexTypeOrdinal);
+          this.numberOfColumnsAfterFlatten += getNumColumnsAfterFlatten(carbonDimension);
           continue;
         } else {
           // for no dictionary dimension
           carbonDimension = new CarbonDimension(columnSchema, dimensionOrdinal++, -1, -1);
+          numberOfColumnsAfterFlatten++;
           numberOfNoDictionaryDimension++;
           if (columnSchema.isSortColumn()) {
             this.numberOfSortColumns++;
-            this.numberOfNoDictSortColumns++;
           }
         }
         dimensions.add(carbonDimension);
       } else {
+        numberOfColumnsAfterFlatten++;
         measures.add(new CarbonMeasure(columnSchema, ++measureOrdinal));
       }
       counter++;
     }
     lastDimensionColOrdinal = dimensionOrdinal;
-    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];
-    }
   }
 
   /**
@@ -394,130 +305,6 @@ public class SegmentProperties {
   }
 
   /**
-   * 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 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;
-      }
-      dimensionPartitionList.add(1);
-      isDictionaryColumn.add(true);
-      counter++;
-    }
-    // get the partitioner
-    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);
-    if (this.getNumberOfDictSortColumns() == bitLength.length) {
-      this.sortColumnsGenerator = this.dimensionKeyGenerator;
-    } else {
-      int numberOfDictSortColumns = this.getNumberOfDictSortColumns();
-      int [] sortColumnBitLength = new int[numberOfDictSortColumns];
-      System.arraycopy(bitLength, 0, sortColumnBitLength, 0, numberOfDictSortColumns);
-      this.sortColumnsGenerator = new MultiDimKeyVarLengthGenerator(sortColumnBitLength);
-    }
-    this.fixedLengthKeySplitter =
-        new MultiDimKeyVarLengthVariableSplitGenerator(bitLength, dimensionPartitions);
-    // get the size of each value in file block
-    int[] dictionaryDimColumnValueSize = 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] = dictionaryDimColumnValueSize[++index];
-    }
-    if (complexDimensions.size() > 0) {
-      int[] complexDimensionPartition = new int[complexDimColumnCardinality.length];
-      // as complex dimension will be stored in column format add one
-      Arrays.fill(complexDimensionPartition, 1);
-      bitLength =
-          CarbonUtil.getDimensionBitLength(complexDimColumnCardinality, complexDimensionPartition);
-      for (int i = 0; i < bitLength.length; i++) {
-        if (complexDimColumnCardinality[i] == 0) {
-          bitLength[i] = 64;
-        }
-      }
-      ColumnarSplitter keySplitter =
-          new MultiDimKeyVarLengthVariableSplitGenerator(bitLength, complexDimensionPartition);
-      eachComplexDimColumnValueSize = keySplitter.getBlockKeySize();
-    } else {
-      eachComplexDimColumnValueSize = new int[0];
-    }
-  }
-
-  /**
-   * 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;
-  }
-
-  public int[] getColumnsValueSize() {
-    int[] dimensionValueSize =
-        new int[eachDimColumnValueSize.length + eachComplexDimColumnValueSize.length + measures
-            .size()];
-    System
-        .arraycopy(eachDimColumnValueSize, 0, dimensionValueSize, 0, eachDimColumnValueSize.length);
-    System.arraycopy(eachComplexDimColumnValueSize, 0, dimensionValueSize,
-        eachDimColumnValueSize.length, eachComplexDimColumnValueSize.length);
-    int k = eachDimColumnValueSize.length + eachComplexDimColumnValueSize.length;
-    for (int i = 0; i < measures.size(); i++) {
-      DataType dataType = measures.get(i).getDataType();
-      if (DataTypes.isDecimal(dataType)) {
-        dimensionValueSize[k++] = -1;
-      } else {
-        dimensionValueSize[k++] = 8;
-      }
-    }
-    return dimensionValueSize;
-  }
-
-  /**
-   * @return the dimensionKeyGenerator
-   */
-  public KeyGenerator getDimensionKeyGenerator() {
-    return dimensionKeyGenerator;
-  }
-
-  public KeyGenerator getSortColumnsGenerator() {
-    return sortColumnsGenerator;
-  }
-
-  /**
    * @return the dimensions
    */
   public List<CarbonDimension> getDimensions() {
@@ -539,27 +326,6 @@ public class SegmentProperties {
   }
 
   /**
-   * @return the dimColumnsCardinality
-   */
-  public int[] getDimColumnsCardinality() {
-    return dimColumnsCardinality;
-  }
-
-  /**
-   * @return
-   */
-  public int[] getDimensionPartitions() {
-    return dimensionPartitions;
-  }
-
-  /**
-   * @return the complexDimColumnCardinality
-   */
-  public int[] getComplexDimColumnCardinality() {
-    return complexDimColumnCardinality;
-  }
-
-  /**
    * @return the dimensionOrdinalToChunkMapping
    */
   public Map<Integer, Integer> getDimensionOrdinalToChunkMapping() {
@@ -574,27 +340,6 @@ public class SegmentProperties {
   }
 
   /**
-   * @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 numberOfNoDictionaryDimension
    */
   public int getNumberOfNoDictionaryDimension() {
@@ -602,14 +347,6 @@ public class SegmentProperties {
   }
 
   /**
-   * @param blockIndex
-   * @return It returns all dimension present in given block index
-   */
-  public Set<Integer> getDimensionOrdinalForBlock(int blockIndex) {
-    return blockTodimensionOrdinalMapping.get(blockIndex);
-  }
-
-  /**
    * @return It returns block index to dimension ordinal mapping
    */
   public Map<Integer, Set<Integer>> getBlockTodimensionOrdinalMapping() {
@@ -640,15 +377,91 @@ public class SegmentProperties {
     return numberOfSortColumns;
   }
 
-  public int getNumberOfNoDictSortColumns() {
-    return numberOfNoDictSortColumns;
+  public int getLastDimensionColOrdinal() {
+    return lastDimensionColOrdinal;
+  }
+
+  public int getNumberOfColumns() {
+    return numberOfColumnsAfterFlatten;
   }
 
-  public int getNumberOfDictSortColumns() {
-    return this.numberOfSortColumns - this.numberOfNoDictSortColumns;
+  public int getNumberOfDictDimensions() {
+    return numberOfDictDimensions;
   }
 
-  public int getLastDimensionColOrdinal() {
-    return lastDimensionColOrdinal;
+  public int getNumberOfPrimitiveDimensions() {
+    return numberOfDictDimensions + numberOfNoDictionaryDimension;
+  }
+
+  public int getNumberOfComplexDimensions() {
+    return complexDimensions.size();
+  }
+
+  public int getNumberOfMeasures() {
+    return measures.size();
+  }
+
+  /**
+   * Return column value length in byte for all dimension columns in the table
+   * for dimension it is -1 (for DATE it is 4),
+   */
+  public int[] createDimColumnValueLength() {
+    int[] length = new int[dimensions.size()];
+    int index = 0;
+    for (CarbonDimension dimension : dimensions) {
+      DataType dataType = dimension.getDataType();
+      if (dataType == DataTypes.DATE) {
+        length[index] = 4;
+      } else {
+        length[index] = -1;
+      }
+      index++;
+    }
+    return length;
+  }
+
+  /**
+   * Return column value length in byte for all columns in the table
+   * for dimension and complex column it is -1 (for DATE it is 4),
+   * for measure is 8 (for decimal is -1)
+   */
+  public int[] createColumnValueLength() {
+    int[] length = new int[numberOfColumnsAfterFlatten];
+    int index = 0;
+    for (CarbonDimension dimension : dimensions) {
+      DataType dataType = dimension.getDataType();
+      if (dataType == DataTypes.DATE) {
+        length[index] = 4;
+      } else {
+        length[index] = -1;
+      }
+      index++;
+    }
+    for (CarbonDimension complexDimension : complexDimensions) {
+      int depth = getNumColumnsAfterFlatten(complexDimension);
+      for (int i = 0; i < depth; i++) {
+        length[index++] = -1;
+      }
+    }
+    for (CarbonMeasure measure : measures) {
+      DataType dataType = measure.getDataType();
+      if (DataTypes.isDecimal(dataType)) {
+        length[index++] = -1;
+      } else {
+        length[index++] = 8;
+      }
+    }
+    return length;
+  }
+
+  private int getNumColumnsAfterFlatten(CarbonDimension dimension) {
+    int count = 1;
+    if (dimension.isComplex()) {
+      List<CarbonDimension> children = dimension.getListOfChildDimensions();
+      for (CarbonDimension child : children) {
+        count += getNumColumnsAfterFlatten(child);
+      }
+    }
+    return count;
   }
 }
diff --git a/core/src/main/java/org/apache/carbondata/core/datastore/block/SegmentPropertiesAndSchemaHolder.java b/core/src/main/java/org/apache/carbondata/core/datastore/block/SegmentPropertiesAndSchemaHolder.java
index 25c5686..1aa675d 100644
--- a/core/src/main/java/org/apache/carbondata/core/datastore/block/SegmentPropertiesAndSchemaHolder.java
+++ b/core/src/main/java/org/apache/carbondata/core/datastore/block/SegmentPropertiesAndSchemaHolder.java
@@ -18,7 +18,6 @@
 package org.apache.carbondata.core.datastore.block;
 
 import java.util.ArrayList;
-import java.util.Arrays;
 import java.util.Collections;
 import java.util.Comparator;
 import java.util.HashSet;
@@ -95,14 +94,13 @@ public class SegmentPropertiesAndSchemaHolder {
    *
    * @param carbonTable
    * @param columnsInTable
-   * @param columnCardinality
    * @param segmentId
    */
   public SegmentPropertiesWrapper addSegmentProperties(CarbonTable carbonTable,
-      List<ColumnSchema> columnsInTable, int[] columnCardinality, String segmentId) {
+      List<ColumnSchema> columnsInTable, String segmentId) {
     SegmentPropertiesAndSchemaHolder.SegmentPropertiesWrapper segmentPropertiesWrapper =
         new SegmentPropertiesAndSchemaHolder.SegmentPropertiesWrapper(carbonTable,
-            columnsInTable, columnCardinality);
+            columnsInTable);
     SegmentIdAndSegmentPropertiesIndexWrapper segmentIdSetAndIndexWrapper =
         this.segmentPropWrapperToSegmentSetMap.get(segmentPropertiesWrapper);
     if (null == segmentIdSetAndIndexWrapper) {
@@ -282,7 +280,6 @@ public class SegmentPropertiesAndSchemaHolder {
     private static final Object minMaxLock = new Object();
 
     private List<ColumnSchema> columnsInTable;
-    private int[] columnCardinality;
     private SegmentProperties segmentProperties;
     private List<CarbonColumn> minMaxCacheColumns;
     private CarbonTable carbonTable;
@@ -300,15 +297,13 @@ public class SegmentPropertiesAndSchemaHolder {
     private CarbonRowSchema[] fileFooterEntrySchemaForBlock;
     private CarbonRowSchema[] fileFooterEntrySchemaForBlocklet;
 
-    public SegmentPropertiesWrapper(CarbonTable carbonTable,
-        List<ColumnSchema> columnsInTable, int[] columnCardinality) {
+    public SegmentPropertiesWrapper(CarbonTable carbonTable, List<ColumnSchema> columnsInTable) {
       this.carbonTable = carbonTable;
       this.columnsInTable = columnsInTable;
-      this.columnCardinality = columnCardinality;
     }
 
     public void initSegmentProperties() {
-      segmentProperties = new SegmentProperties(columnsInTable, columnCardinality);
+      segmentProperties = new SegmentProperties(columnsInTable);
     }
 
     public void addMinMaxColumns(CarbonTable carbonTable) {
@@ -342,8 +337,7 @@ public class SegmentPropertiesAndSchemaHolder {
           (SegmentPropertiesAndSchemaHolder.SegmentPropertiesWrapper) obj;
       return carbonTable.getAbsoluteTableIdentifier()
           .equals(other.carbonTable.getAbsoluteTableIdentifier()) && checkColumnSchemaEquality(
-          columnsInTable, other.columnsInTable) && Arrays
-          .equals(columnCardinality, other.columnCardinality);
+          columnsInTable, other.columnsInTable);
     }
 
     private boolean checkColumnSchemaEquality(List<ColumnSchema> obj1, List<ColumnSchema> obj2) {
@@ -378,8 +372,8 @@ public class SegmentPropertiesAndSchemaHolder {
         allColumnsHashCode = allColumnsHashCode + columnSchema.strictHashCode();
         builder.append(columnSchema.getColumnUniqueId()).append(",");
       }
-      return carbonTable.getAbsoluteTableIdentifier().hashCode() + allColumnsHashCode + Arrays
-          .hashCode(columnCardinality) + builder.toString().hashCode();
+      return carbonTable.getAbsoluteTableIdentifier().hashCode() + allColumnsHashCode +
+          builder.toString().hashCode();
     }
 
     public AbsoluteTableIdentifier getTableIdentifier() {
@@ -394,10 +388,6 @@ public class SegmentPropertiesAndSchemaHolder {
       return columnsInTable;
     }
 
-    public int[] getColumnCardinality() {
-      return columnCardinality;
-    }
-
     public CarbonRowSchema[] getTaskSummarySchemaForBlock(boolean storeBlockletCount,
         boolean filePathToBeStored) {
       if (null == taskSummarySchemaForBlock && filePathToBeStored) {
diff --git a/core/src/main/java/org/apache/carbondata/core/datastore/block/TableBlockInfo.java b/core/src/main/java/org/apache/carbondata/core/datastore/block/TableBlockInfo.java
index 15b2c50..70ffd92 100644
--- a/core/src/main/java/org/apache/carbondata/core/datastore/block/TableBlockInfo.java
+++ b/core/src/main/java/org/apache/carbondata/core/datastore/block/TableBlockInfo.java
@@ -22,8 +22,6 @@ import java.io.Serializable;
 import java.nio.charset.Charset;
 import java.util.Arrays;
 import java.util.Comparator;
-import java.util.HashMap;
-import java.util.Map;
 
 import org.apache.carbondata.core.constants.CarbonCommonConstants;
 import org.apache.carbondata.core.datamap.Segment;
@@ -74,32 +72,11 @@ public class TableBlockInfo implements Distributable, Serializable {
    */
   private Segment segment;
 
-  /**
-   * id of the Blocklet.
-   */
-  private String blockletId;
-
   private String[] locations;
 
   private ColumnarFormatVersion version;
 
   /**
-   * flag to determine whether the data block is from old store (version 1.1)
-   * or current store
-   */
-  private boolean isDataBlockFromOldStore;
-  /**
-   * The class holds the blockletsinfo
-   */
-  private BlockletInfos blockletInfos = new BlockletInfos();
-
-  /**
-   * map of block location and storage id
-   */
-  private Map<String, String> blockStorageIdMap =
-      new HashMap<>(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE);
-
-  /**
    * delete delta files path for this block
    */
   private String[] deletedDeltaFilePath;
@@ -111,11 +88,6 @@ public class TableBlockInfo implements Distributable, Serializable {
   private transient DataFileFooter dataFileFooter;
 
   /**
-   * true when index file does't have blocklet information
-   */
-  private boolean isLegacyStore;
-
-  /**
    * comparator to sort by block size in descending order.
    * Since each line is not exactly the same, the size of a InputSplit may differs,
    * so we allow some deviation for these splits.
@@ -134,7 +106,6 @@ public class TableBlockInfo implements Distributable, Serializable {
       String[] locations, long blockLength, ColumnarFormatVersion version,
       String[] deletedDeltaFilePath) {
     this.filePath = FileFactory.getUpdatedFilePath(filePath);
-    this.blockletId = "0";
     this.blockOffset = blockOffset;
     this.segment = Segment.toSegment(segmentId);
     this.locations = locations;
@@ -148,63 +119,6 @@ public class TableBlockInfo implements Distributable, Serializable {
   }
 
   /**
-   * constructor to initialize the TbaleBlockInfo with BlockletInfos
-   *
-   * @param filePath
-   * @param blockOffset
-   * @param segmentId
-   * @param locations
-   * @param blockLength
-   * @param blockletInfos
-   */
-  public TableBlockInfo(String filePath, long blockOffset, String segmentId, String[] locations,
-      long blockLength, BlockletInfos blockletInfos, ColumnarFormatVersion version,
-      String[] deletedDeltaFilePath) {
-    this(filePath, blockOffset, segmentId, locations, blockLength, version,
-        deletedDeltaFilePath);
-    this.blockletInfos = blockletInfos;
-  }
-
-  /**
-   * constructor to initialize the TableBlockInfo with blockletIds
-   *
-   * @param filePath
-   * @param blockOffset
-   * @param segmentId
-   * @param locations
-   * @param blockLength
-   * @param blockletInfos
-   */
-  public TableBlockInfo(String filePath, String blockletId, long blockOffset, String segmentId,
-      String[] locations, long blockLength, BlockletInfos blockletInfos,
-      ColumnarFormatVersion version, String[] deletedDeltaFilePath) {
-    this(filePath, blockOffset, segmentId, locations, blockLength, blockletInfos, version,
-        deletedDeltaFilePath);
-    this.blockletId = blockletId;
-  }
-
-  /**
-   * constructor to initialize the TableBlockInfo with blockStorageIdMap
-   *
-   * @param filePath
-   * @param blockOffset
-   * @param segmentId
-   * @param locations
-   * @param blockLength
-   * @param blockletInfos
-   * @param version
-   * @param blockStorageIdMap
-   */
-  public TableBlockInfo(String filePath, String blockletId, long blockOffset, String segmentId,
-      String[] locations, long blockLength, BlockletInfos blockletInfos,
-      ColumnarFormatVersion version, Map<String, String> blockStorageIdMap,
-      String[] deletedDeltaFilePath) {
-    this(filePath, blockletId, blockOffset, segmentId, locations, blockLength, blockletInfos,
-        version, deletedDeltaFilePath);
-    this.blockStorageIdMap = blockStorageIdMap;
-  }
-
-  /**
    * Create copy of TableBlockInfo object
    */
   public TableBlockInfo copy() {
@@ -213,16 +127,11 @@ public class TableBlockInfo implements Distributable, Serializable {
     info.blockOffset = blockOffset;
     info.blockLength = blockLength;
     info.segment = segment;
-    info.blockletId = blockletId;
     info.locations = locations;
     info.version = version;
-    info.isDataBlockFromOldStore = isDataBlockFromOldStore;
-    info.blockletInfos = blockletInfos;
-    info.blockStorageIdMap = blockStorageIdMap;
     info.deletedDeltaFilePath = deletedDeltaFilePath;
     info.detailInfo = detailInfo.copy();
     info.dataMapWriterPath = dataMapWriterPath;
-    info.isLegacyStore = isLegacyStore;
     return info;
   }
 
@@ -310,9 +219,6 @@ public class TableBlockInfo implements Distributable, Serializable {
       return false;
     }
 
-    if (blockletInfos.getStartBlockletNumber() != other.blockletInfos.getStartBlockletNumber()) {
-      return false;
-    }
     return true;
   }
 
@@ -370,16 +276,6 @@ public class TableBlockInfo implements Distributable, Serializable {
         > ((TableBlockInfo) other).blockOffset + ((TableBlockInfo) other).blockLength) {
       return 1;
     }
-    //compare the startBlockLetNumber
-    int diffStartBlockLetNumber =
-        blockletInfos.getStartBlockletNumber() - ((TableBlockInfo) other).blockletInfos
-            .getStartBlockletNumber();
-    if (diffStartBlockLetNumber < 0) {
-      return -1;
-    }
-    if (diffStartBlockLetNumber > 0) {
-      return 1;
-    }
     return 0;
   }
 
@@ -389,7 +285,7 @@ public class TableBlockInfo implements Distributable, Serializable {
     result = 31 * result + (int) (blockOffset ^ (blockOffset >>> 32));
     result = 31 * result + (int) (blockLength ^ (blockLength >>> 32));
     result = 31 * result + segment.hashCode();
-    result = 31 * result + blockletInfos.getStartBlockletNumber();
+    result = 31 * result;
     return result;
   }
 
@@ -398,24 +294,6 @@ public class TableBlockInfo implements Distributable, Serializable {
     return locations;
   }
 
-  /**
-   * returns BlockletInfos
-   *
-   * @return
-   */
-  public BlockletInfos getBlockletInfos() {
-    return blockletInfos;
-  }
-
-  /**
-   * set the blocklestinfos
-   *
-   * @param blockletInfos
-   */
-  public void setBlockletInfos(BlockletInfos blockletInfos) {
-    this.blockletInfos = blockletInfos;
-  }
-
   public ColumnarFormatVersion getVersion() {
     return version;
   }
@@ -424,24 +302,6 @@ public class TableBlockInfo implements Distributable, Serializable {
     this.version = version;
   }
 
-  /**
-   * returns the storage location vs storage id map
-   *
-   * @return
-   */
-  public Map<String, String> getBlockStorageIdMap() {
-    return this.blockStorageIdMap;
-  }
-
-  /**
-   * method to storage location vs storage id map
-   *
-   * @param blockStorageIdMap
-   */
-  public void setBlockStorageIdMap(Map<String, String> blockStorageIdMap) {
-    this.blockStorageIdMap = blockStorageIdMap;
-  }
-
   public String[] getDeletedDeltaFilePath() {
     return deletedDeltaFilePath;
   }
@@ -466,22 +326,6 @@ public class TableBlockInfo implements Distributable, Serializable {
     this.detailInfo = detailInfo;
   }
 
-  public String getBlockletId() {
-    return blockletId;
-  }
-
-  public void setBlockletId(String blockletId) {
-    this.blockletId = blockletId;
-  }
-
-  public boolean isDataBlockFromOldStore() {
-    return isDataBlockFromOldStore;
-  }
-
-  public void setDataBlockFromOldStore(boolean dataBlockFromOldStore) {
-    isDataBlockFromOldStore = dataBlockFromOldStore;
-  }
-
   public String getDataMapWriterPath() {
     return dataMapWriterPath;
   }
@@ -498,14 +342,6 @@ public class TableBlockInfo implements Distributable, Serializable {
     this.dataFileFooter = dataFileFooter;
   }
 
-  public boolean isLegacyStore() {
-    return isLegacyStore;
-  }
-
-  public void setLegacyStore(boolean legacyStore) {
-    isLegacyStore = legacyStore;
-  }
-
   @Override
   public String toString() {
     final StringBuilder sb = new StringBuilder("TableBlockInfo{");
@@ -513,7 +349,6 @@ public class TableBlockInfo implements Distributable, Serializable {
     sb.append(", blockOffset=").append(blockOffset);
     sb.append(", blockLength=").append(blockLength);
     sb.append(", segment='").append(segment.toString()).append('\'');
-    sb.append(", blockletId='").append(blockletId).append('\'');
     sb.append(", locations=").append(Arrays.toString(locations));
     sb.append('}');
     return sb.toString();
diff --git a/core/src/main/java/org/apache/carbondata/core/datastore/block/TableBlockUniqueIdentifier.java b/core/src/main/java/org/apache/carbondata/core/datastore/block/TableBlockUniqueIdentifier.java
deleted file mode 100644
index 59ebed1..0000000
--- a/core/src/main/java/org/apache/carbondata/core/datastore/block/TableBlockUniqueIdentifier.java
+++ /dev/null
@@ -1,90 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.carbondata.core.datastore.block;
-
-import org.apache.carbondata.core.constants.CarbonCommonConstants;
-import org.apache.carbondata.core.metadata.AbsoluteTableIdentifier;
-import org.apache.carbondata.core.metadata.CarbonTableIdentifier;
-
-/**
- * Class : Holds the info to uniquely identify a blocks
- */
-public class TableBlockUniqueIdentifier {
-
-  /**
-   * table fully qualified name
-   */
-  private AbsoluteTableIdentifier absoluteTableIdentifier;
-
-  /**
-   * table block info
-   */
-  private TableBlockInfo tableBlockInfo;
-
-  public TableBlockUniqueIdentifier(AbsoluteTableIdentifier absoluteTableIdentifier,
-      TableBlockInfo tableBlockInfo) {
-    this.absoluteTableIdentifier = absoluteTableIdentifier;
-    this.tableBlockInfo = tableBlockInfo;
-  }
-
-  public AbsoluteTableIdentifier getAbsoluteTableIdentifier() {
-    return absoluteTableIdentifier;
-  }
-
-  public void setAbsoluteTableIdentifier(AbsoluteTableIdentifier absoluteTableIdentifier) {
-    this.absoluteTableIdentifier = absoluteTableIdentifier;
-  }
-
-  public TableBlockInfo getTableBlockInfo() {
-    return tableBlockInfo;
-  }
-
-  public void setTableBlockInfo(TableBlockInfo tableBlockInfo) {
-    this.tableBlockInfo = tableBlockInfo;
-  }
-
-  @Override
-  public int hashCode() {
-    return this.absoluteTableIdentifier.hashCode() + this.tableBlockInfo.hashCode();
-  }
-
-  @Override
-  public boolean equals(Object other) {
-    if (this == other) return true;
-    if (other == null || getClass() != other.getClass()) return false;
-    TableBlockUniqueIdentifier tableBlockUniqueIdentifier = (TableBlockUniqueIdentifier) other;
-    return this.absoluteTableIdentifier.equals(tableBlockUniqueIdentifier.absoluteTableIdentifier)
-        && this.tableBlockInfo.equals(tableBlockUniqueIdentifier.tableBlockInfo);
-  }
-
-  /**
-   * returns the String value to uniquely identify a block
-   *
-   * @return
-   */
-  public String getUniqueTableBlockName() {
-    BlockInfo blockInfo = new BlockInfo(this.tableBlockInfo);
-    CarbonTableIdentifier carbonTableIdentifier =
-        this.absoluteTableIdentifier.getCarbonTableIdentifier();
-    return carbonTableIdentifier.getDatabaseName()
-        + CarbonCommonConstants.FILE_SEPARATOR + carbonTableIdentifier
-        .getDatabaseName() + CarbonCommonConstants.FILE_SEPARATOR
-        + this.tableBlockInfo.getSegmentId()
-        + CarbonCommonConstants.FILE_SEPARATOR + blockInfo.hashCode();
-  }
-}
diff --git a/core/src/main/java/org/apache/carbondata/core/datastore/blocklet/EncodedBlocklet.java b/core/src/main/java/org/apache/carbondata/core/datastore/blocklet/EncodedBlocklet.java
index 61d0f7f..86edc12 100644
--- a/core/src/main/java/org/apache/carbondata/core/datastore/blocklet/EncodedBlocklet.java
+++ b/core/src/main/java/org/apache/carbondata/core/datastore/blocklet/EncodedBlocklet.java
@@ -23,7 +23,6 @@ import java.util.Map;
 import java.util.concurrent.ExecutorService;
 
 import org.apache.carbondata.core.datastore.page.EncodedTablePage;
-import org.apache.carbondata.core.datastore.page.key.TablePageKey;
 import org.apache.carbondata.core.localdictionary.generator.LocalDictionaryGenerator;
 
 /**
@@ -39,11 +38,6 @@ public class EncodedBlocklet {
   private int blockletSize;
 
   /**
-   * list of page metadata
-   */
-  private List<TablePageKey> pageMetadataList;
-
-  /**
    * maintains encoded dimension data for each column
    */
   private List<BlockletEncodedColumnPage> encodedDimensionColumnPages;
@@ -93,16 +87,11 @@ public class EncodedBlocklet {
    * encoded table page
    */
   private void addPageMetadata(EncodedTablePage encodedTablePage) {
-    // for first table page create new list
-    if (null == pageMetadataList) {
-      pageMetadataList = new ArrayList<>();
-    }
     if (null == rowCountInPage) {
       rowCountInPage = new ArrayList<>();
     }
     // update details
     blockletSize += encodedTablePage.getPageSize();
-    pageMetadataList.add(encodedTablePage.getPageKey());
     this.numberOfPages++;
     rowCountInPage.add((short)encodedTablePage.getPageSize());
   }
@@ -174,10 +163,6 @@ public class EncodedBlocklet {
     return blockletSize;
   }
 
-  public List<TablePageKey> getPageMetadataList() {
-    return pageMetadataList;
-  }
-
   public List<BlockletEncodedColumnPage> getEncodedDimensionColumnPages() {
     return encodedDimensionColumnPages;
   }
@@ -211,7 +196,6 @@ public class EncodedBlocklet {
     this.encodedDimensionColumnPages = null;
     this.blockletSize = 0;
     this.encodedMeasureColumnPages = null;
-    this.pageMetadataList = null;
     this.rowCountInPage = null;
   }
 }
diff --git a/core/src/main/java/org/apache/carbondata/core/datastore/chunk/reader/CarbonDataReaderFactory.java b/core/src/main/java/org/apache/carbondata/core/datastore/chunk/reader/CarbonDataReaderFactory.java
index 5549574..2c599d4 100644
--- a/core/src/main/java/org/apache/carbondata/core/datastore/chunk/reader/CarbonDataReaderFactory.java
+++ b/core/src/main/java/org/apache/carbondata/core/datastore/chunk/reader/CarbonDataReaderFactory.java
@@ -56,21 +56,17 @@ public class CarbonDataReaderFactory {
    *
    * @param version             reader version
    * @param blockletInfo        blocklet info
-   * @param eachColumnValueSize size of each dimension column
    * @param filePath            carbon data file path
    * @return dimension column data reader based on version number
    */
   public DimensionColumnChunkReader getDimensionColumnChunkReader(ColumnarFormatVersion version,
-      BlockletInfo blockletInfo, int[] eachColumnValueSize, String filePath,
-      boolean readPagebyPage) {
+      BlockletInfo blockletInfo, String filePath, boolean readPagebyPage) {
     switch (version) {
       case V3:
         if (readPagebyPage) {
-          return new DimensionChunkPageReaderV3(blockletInfo, eachColumnValueSize,
-              filePath);
+          return new DimensionChunkPageReaderV3(blockletInfo, filePath);
         } else {
-          return new DimensionChunkReaderV3(blockletInfo, eachColumnValueSize,
-              filePath);
+          return new DimensionChunkReaderV3(blockletInfo, filePath);
         }
       default:
         throw new UnsupportedOperationException("Unsupported columnar format version: " + version);
diff --git a/core/src/main/java/org/apache/carbondata/core/datastore/chunk/reader/dimension/AbstractDimensionChunkReader.java b/core/src/main/java/org/apache/carbondata/core/datastore/chunk/reader/dimension/AbstractDimensionChunkReader.java
index 91f6590..d14e69c 100644
--- a/core/src/main/java/org/apache/carbondata/core/datastore/chunk/reader/dimension/AbstractDimensionChunkReader.java
+++ b/core/src/main/java/org/apache/carbondata/core/datastore/chunk/reader/dimension/AbstractDimensionChunkReader.java
@@ -40,12 +40,6 @@ public abstract class AbstractDimensionChunkReader implements DimensionColumnChu
   protected Compressor compressor;
 
   /**
-   * 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
    */
@@ -63,12 +57,9 @@ public abstract class AbstractDimensionChunkReader implements DimensionColumnChu
   /**
    * Constructor to get minimum parameter to create
    * instance of this class
-   *  @param eachColumnValueSize  size of the each column value
    * @param filePath             file from which data will be read
    */
-  public AbstractDimensionChunkReader(final BlockletInfo blockletInfo,
-      final int[] eachColumnValueSize, final String filePath) {
-    this.eachColumnValueSize = eachColumnValueSize;
+  public AbstractDimensionChunkReader(final BlockletInfo blockletInfo, final String filePath) {
     this.filePath = filePath;
     dimensionChunksOffset = blockletInfo.getDimensionChunkOffsets();
     dimensionChunksLength = blockletInfo.getDimensionChunksLength();
diff --git a/core/src/main/java/org/apache/carbondata/core/datastore/chunk/reader/dimension/v3/DimensionChunkPageReaderV3.java b/core/src/main/java/org/apache/carbondata/core/datastore/chunk/reader/dimension/v3/DimensionChunkPageReaderV3.java
index 54efbf3..1ce8465 100644
--- a/core/src/main/java/org/apache/carbondata/core/datastore/chunk/reader/dimension/v3/DimensionChunkPageReaderV3.java
+++ b/core/src/main/java/org/apache/carbondata/core/datastore/chunk/reader/dimension/v3/DimensionChunkPageReaderV3.java
@@ -46,17 +46,15 @@ import org.apache.carbondata.format.Encoding;
  * <Column4 Data ChunkV3><Column4<Page1><Page2><Page3><Page4>>
  * <File Footer>
  */
-public class DimensionChunkPageReaderV3
-    extends DimensionChunkReaderV3 {
+public class DimensionChunkPageReaderV3 extends DimensionChunkReaderV3 {
 
   /**
    * end position of last dimension in carbon data file
    */
   private long lastDimensionOffsets;
 
-  public DimensionChunkPageReaderV3(BlockletInfo blockletInfo,
-      int[] eachColumnValueSize, String filePath) {
-    super(blockletInfo, eachColumnValueSize, filePath);
+  public DimensionChunkPageReaderV3(BlockletInfo blockletInfo, String filePath) {
+    super(blockletInfo, filePath);
     lastDimensionOffsets = blockletInfo.getDimensionOffset();
   }
 
diff --git a/core/src/main/java/org/apache/carbondata/core/datastore/chunk/reader/dimension/v3/DimensionChunkReaderV3.java b/core/src/main/java/org/apache/carbondata/core/datastore/chunk/reader/dimension/v3/DimensionChunkReaderV3.java
index 2c00706..1cfcbd1 100644
--- a/core/src/main/java/org/apache/carbondata/core/datastore/chunk/reader/dimension/v3/DimensionChunkReaderV3.java
+++ b/core/src/main/java/org/apache/carbondata/core/datastore/chunk/reader/dimension/v3/DimensionChunkReaderV3.java
@@ -71,9 +71,8 @@ public class DimensionChunkReaderV3 extends AbstractDimensionChunkReader {
    */
   private long lastDimensionOffsets;
 
-  public DimensionChunkReaderV3(BlockletInfo blockletInfo,
-      int[] eachColumnValueSize, String filePath) {
-    super(blockletInfo, eachColumnValueSize, filePath);
+  public DimensionChunkReaderV3(BlockletInfo blockletInfo, String filePath) {
+    super(blockletInfo, filePath);
     lastDimensionOffsets = blockletInfo.getDimensionOffset();
   }
 
@@ -319,6 +318,10 @@ public class DimensionChunkReaderV3 extends AbstractDimensionChunkReader {
     return false;
   }
 
+  private int getColumnValueSize(List<Encoding> encodings) {
+    return encodings.contains(Encoding.DICTIONARY) ? 4 : -1;
+  }
+
   private DimensionColumnPage decodeDimensionLegacy(DimensionRawColumnChunk rawColumnPage,
       ByteBuffer pageData, DataChunk2 pageMetadata, int offset, ColumnVectorInfo vectorInfo,
       ReusableDataBuffer reusableDataBuffer) {
@@ -355,7 +358,7 @@ public class DimensionChunkReaderV3 extends AbstractDimensionChunkReader {
       // uncompress the data with rle indexes
       dataPage = UnBlockIndexer.uncompressData(dataPage, rlePage,
           null == rawColumnPage.getLocalDictionary() ?
-              eachColumnValueSize[rawColumnPage.getColumnIndex()] :
+              getColumnValueSize(pageMetadata.encoders) :
               CarbonCommonConstants.LOCAL_DICT_ENCODED_BYTEARRAY_SIZE, uncompressedSize);
       uncompressedSize = dataPage.length;
     }
@@ -379,7 +382,7 @@ public class DimensionChunkReaderV3 extends AbstractDimensionChunkReader {
       columnDataChunk =
           new FixedLengthDimensionColumnPage(dataPage, invertedIndexes, invertedIndexesReverse,
               pageMetadata.getNumberOfRowsInpage(),
-              eachColumnValueSize[rawColumnPage.getColumnIndex()], vectorInfo, uncompressedSize);
+              getColumnValueSize(pageMetadata.encoders), vectorInfo, uncompressedSize);
     }
     return columnDataChunk;
   }
diff --git a/core/src/main/java/org/apache/carbondata/core/datastore/chunk/reader/measure/v3/MeasureChunkPageReaderV3.java b/core/src/main/java/org/apache/carbondata/core/datastore/chunk/reader/measure/v3/MeasureChunkPageReaderV3.java
index a5cc738..7776562 100644
--- a/core/src/main/java/org/apache/carbondata/core/datastore/chunk/reader/measure/v3/MeasureChunkPageReaderV3.java
+++ b/core/src/main/java/org/apache/carbondata/core/datastore/chunk/reader/measure/v3/MeasureChunkPageReaderV3.java
@@ -46,8 +46,7 @@ import org.apache.carbondata.format.DataChunk3;
  * <Column4 Data ChunkV3><Column4<Page1><Page2><Page3><Page4>>
  * <File Footer>
  */
-public class MeasureChunkPageReaderV3
-    extends MeasureChunkReaderV3 {
+public class MeasureChunkPageReaderV3 extends MeasureChunkReaderV3 {
 
   /**
    * end position of last measure in carbon data file
diff --git a/core/src/main/java/org/apache/carbondata/core/datastore/page/EncodedTablePage.java b/core/src/main/java/org/apache/carbondata/core/datastore/page/EncodedTablePage.java
index 22fa78f..8a3482a 100644
--- a/core/src/main/java/org/apache/carbondata/core/datastore/page/EncodedTablePage.java
+++ b/core/src/main/java/org/apache/carbondata/core/datastore/page/EncodedTablePage.java
@@ -18,7 +18,6 @@
 package org.apache.carbondata.core.datastore.page;
 
 import org.apache.carbondata.core.datastore.page.encoding.EncodedColumnPage;
-import org.apache.carbondata.core.datastore.page.key.TablePageKey;
 
 /**
  * Table page that after encoding and compression.
@@ -31,9 +30,6 @@ public class EncodedTablePage {
   // encoded data and metadata for each measure column
   private EncodedColumnPage[] measurePages;
 
-  // key of this page
-  private TablePageKey pageKey;
-
   // number of row in this page
   private int pageSize;
 
@@ -41,18 +37,15 @@ public class EncodedTablePage {
   private int encodedSize;
 
   public static EncodedTablePage newInstance(int pageSize,
-      EncodedColumnPage[] dimensionPages, EncodedColumnPage[] measurePages,
-      TablePageKey tablePageKey) {
-    return new EncodedTablePage(pageSize, dimensionPages, measurePages, tablePageKey);
+      EncodedColumnPage[] dimensionPages, EncodedColumnPage[] measurePages) {
+    return new EncodedTablePage(pageSize, dimensionPages, measurePages);
   }
 
   private EncodedTablePage(int pageSize,
-      EncodedColumnPage[] dimensionPages, EncodedColumnPage[] measurePages,
-      TablePageKey tablePageKey) {
+      EncodedColumnPage[] dimensionPages, EncodedColumnPage[] measurePages) {
     this.dimensionPages = dimensionPages;
     this.measurePages = measurePages;
     this.pageSize = pageSize;
-    this.pageKey = tablePageKey;
     this.encodedSize = calculatePageSize(dimensionPages, measurePages);
   }
 
@@ -85,10 +78,6 @@ public class EncodedTablePage {
     return measurePages.length;
   }
 
-  public TablePageKey getPageKey() {
-    return pageKey;
-  }
-
   public EncodedColumnPage getDimension(int dimensionIndex) {
     return dimensionPages[dimensionIndex];
   }
diff --git a/core/src/main/java/org/apache/carbondata/core/datastore/page/key/TablePageKey.java b/core/src/main/java/org/apache/carbondata/core/datastore/page/key/TablePageKey.java
deleted file mode 100644
index 3786ef1..0000000
--- a/core/src/main/java/org/apache/carbondata/core/datastore/page/key/TablePageKey.java
+++ /dev/null
@@ -1,210 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.carbondata.core.datastore.page.key;
-
-import java.nio.ByteBuffer;
-import java.util.List;
-
-import org.apache.carbondata.core.constants.CarbonCommonConstants;
-import org.apache.carbondata.core.datastore.block.SegmentProperties;
-import org.apache.carbondata.core.datastore.row.CarbonRow;
-import org.apache.carbondata.core.datastore.row.WriteStepRowUtil;
-import org.apache.carbondata.core.metadata.schema.table.CarbonTable;
-import org.apache.carbondata.core.metadata.schema.table.column.CarbonDimension;
-import org.apache.carbondata.core.util.DataTypeUtil;
-import org.apache.carbondata.core.util.NonDictionaryUtil;
-
-public class TablePageKey {
-  private int pageSize;
-
-  // MDK start key
-  private byte[] startKey;
-
-  // MDK end key
-  private byte[] endKey;
-
-  // startkey for no dictionary columns
-  private Object[] noDictStartKey;
-
-  // endkey for no diciotn
-  private Object[] noDictEndKey;
-
-  // startkey for no dictionary columns after packing into one column
-  private byte[] packedNoDictStartKey;
-
-  // endkey for no dictionary columns after packing into one column
-  private byte[] packedNoDictEndKey;
-
-  private SegmentProperties segmentProperties;
-  private boolean hasNoDictionary;
-
-  public TablePageKey(int pageSize, SegmentProperties segmentProperties,
-                      boolean hasNoDictionary) {
-    this.pageSize = pageSize;
-    this.segmentProperties = segmentProperties;
-    this.hasNoDictionary = hasNoDictionary;
-  }
-
-  /** update all keys based on the input row */
-  public void update(int rowId, CarbonRow row, byte[] mdk) {
-    if (rowId == 0) {
-      startKey = mdk;
-      if (hasNoDictionary) {
-        noDictStartKey = WriteStepRowUtil.getNoDictAndComplexDimension(row);
-      }
-    }
-    if (rowId == pageSize - 1) {
-      endKey = mdk;
-      if (hasNoDictionary) {
-        noDictEndKey = WriteStepRowUtil.getNoDictAndComplexDimension(row);
-      }
-      finalizeKeys();
-    }
-  }
-
-  /** update all keys if SORT_COLUMNS option is used when creating table */
-  private void finalizeKeys() {
-    // If SORT_COLUMNS is used, may need to update start/end keys since the they may
-    // contains dictionary columns that are not in SORT_COLUMNS, which need to be removed from
-    // start/end key
-    int numberOfDictSortColumns = segmentProperties.getNumberOfDictSortColumns();
-    if (numberOfDictSortColumns > 0) {
-      // if SORT_COLUMNS contain dictionary columns
-      int[] keySize = segmentProperties.getFixedLengthKeySplitter().getBlockKeySize();
-      if (keySize.length > numberOfDictSortColumns) {
-        // if there are some dictionary columns that are not in SORT_COLUMNS, it will come to here
-        int newMdkLength = 0;
-        for (int i = 0; i < numberOfDictSortColumns; i++) {
-          newMdkLength += keySize[i];
-        }
-        byte[] newStartKeyOfSortKey = new byte[newMdkLength];
-        byte[] newEndKeyOfSortKey = new byte[newMdkLength];
-        System.arraycopy(startKey, 0, newStartKeyOfSortKey, 0, newMdkLength);
-        System.arraycopy(endKey, 0, newEndKeyOfSortKey, 0, newMdkLength);
-        startKey = newStartKeyOfSortKey;
-        endKey = newEndKeyOfSortKey;
-      }
-    } else {
-      startKey = new byte[0];
-      endKey = new byte[0];
-    }
-
-    // Do the same update for noDictionary start/end Key
-    int numberOfNoDictSortColumns = segmentProperties.getNumberOfNoDictSortColumns();
-    if (numberOfNoDictSortColumns > 0) {
-      // if sort_columns contain no-dictionary columns
-      if (noDictStartKey.length > numberOfNoDictSortColumns) {
-        Object[] newNoDictionaryStartKey = new Object[numberOfNoDictSortColumns];
-        Object[] newNoDictionaryEndKey = new Object[numberOfNoDictSortColumns];
-        System.arraycopy(
-            noDictStartKey, 0, newNoDictionaryStartKey, 0, numberOfNoDictSortColumns);
-        System.arraycopy(
-            noDictEndKey, 0, newNoDictionaryEndKey, 0, numberOfNoDictSortColumns);
-        noDictStartKey = newNoDictionaryStartKey;
-        noDictEndKey = newNoDictionaryEndKey;
-      }
-      List<CarbonDimension> noDictSortColumns =
-          CarbonTable.getNoDictSortColumns(segmentProperties.getDimensions());
-      packedNoDictStartKey = NonDictionaryUtil.packByteBufferIntoSingleByteArray(
-          convertKeys(noDictStartKey, noDictSortColumns));
-      packedNoDictEndKey = NonDictionaryUtil.packByteBufferIntoSingleByteArray(
-          convertKeys(noDictEndKey, noDictSortColumns));
-    } else {
-      noDictStartKey = new byte[0][];
-      noDictEndKey = new byte[0][];
-      packedNoDictStartKey = new byte[0];
-      packedNoDictEndKey = new byte[0];
-    }
-  }
-
-  private byte[][] convertKeys(Object[] keys, List<CarbonDimension> noDictSortColumns) {
-    byte[][] finalKeys = new byte[keys.length][];
-    for (int i = 0; i < keys.length; i++) {
-      if (keys[i] instanceof byte[]) {
-        finalKeys[i] = (byte[]) keys[i];
-      } else {
-        finalKeys[i] = DataTypeUtil.getBytesDataDataTypeForNoDictionaryColumn(keys[i],
-            noDictSortColumns.get(i).getDataType());
-      }
-    }
-    return finalKeys;
-  }
-
-  public byte[] getNoDictStartKey() {
-    return packedNoDictStartKey;
-  }
-
-  public byte[] getNoDictEndKey() {
-    return packedNoDictEndKey;
-  }
-
-  public byte[] serializeStartKey() {
-    byte[] updatedNoDictionaryStartKey = updateNoDictionaryStartAndEndKey(getNoDictStartKey());
-    ByteBuffer buffer = ByteBuffer.allocate(
-        CarbonCommonConstants.INT_SIZE_IN_BYTE + CarbonCommonConstants.INT_SIZE_IN_BYTE
-            + startKey.length + updatedNoDictionaryStartKey.length);
-    buffer.putInt(startKey.length);
-    buffer.putInt(updatedNoDictionaryStartKey.length);
-    buffer.put(startKey);
-    buffer.put(updatedNoDictionaryStartKey);
-    buffer.rewind();
-    return buffer.array();
-  }
-
-  public byte[] serializeEndKey() {
-    byte[] updatedNoDictionaryEndKey = updateNoDictionaryStartAndEndKey(getNoDictEndKey());
-    ByteBuffer buffer = ByteBuffer.allocate(
-        CarbonCommonConstants.INT_SIZE_IN_BYTE + CarbonCommonConstants.INT_SIZE_IN_BYTE
-            + endKey.length + updatedNoDictionaryEndKey.length);
-    buffer.putInt(endKey.length);
-    buffer.putInt(updatedNoDictionaryEndKey.length);
-    buffer.put(endKey);
-    buffer.put(updatedNoDictionaryEndKey);
-    buffer.rewind();
-    return buffer.array();
-  }
-
-  /**
-   * Below method will be used to update the no dictionary start and end key
-   *
-   * @param key key to be updated
-   * @return return no dictionary key
-   */
-  public byte[] updateNoDictionaryStartAndEndKey(byte[] key) {
-    if (key.length == 0) {
-      return key;
-    }
-    // add key to byte buffer remove the length part of the data
-    ByteBuffer buffer = ByteBuffer.wrap(key, 2, key.length - 2);
-    // create a output buffer without length
-    ByteBuffer output = ByteBuffer.allocate(key.length - 2);
-    short numberOfByteToStorLength = 2;
-    // as length part is removed, so each no dictionary value index
-    // needs to be reshuffled by 2 bytes
-    int NumberOfNoDictSortColumns = segmentProperties.getNumberOfNoDictSortColumns();
-    for (int i = 0; i < NumberOfNoDictSortColumns; i++) {
-      output.putShort((short) (buffer.getShort() - numberOfByteToStorLength));
-    }
-    // copy the data part
-    while (buffer.hasRemaining()) {
-      output.put(buffer.get());
-    }
-    output.rewind();
-    return output.array();
-  }
-}
\ No newline at end of file
diff --git a/core/src/main/java/org/apache/carbondata/core/datastore/row/WriteStepRowUtil.java b/core/src/main/java/org/apache/carbondata/core/datastore/row/WriteStepRowUtil.java
index fb4be70..7ddad71 100644
--- a/core/src/main/java/org/apache/carbondata/core/datastore/row/WriteStepRowUtil.java
+++ b/core/src/main/java/org/apache/carbondata/core/datastore/row/WriteStepRowUtil.java
@@ -18,10 +18,10 @@
 package org.apache.carbondata.core.datastore.row;
 
 import org.apache.carbondata.core.datastore.block.SegmentProperties;
-import org.apache.carbondata.core.keygenerator.KeyGenerator;
 import org.apache.carbondata.core.metadata.datatype.DataTypes;
 import org.apache.carbondata.core.metadata.schema.table.column.CarbonColumn;
 import org.apache.carbondata.core.scan.wrappers.ByteArrayWrapper;
+import org.apache.carbondata.core.util.ByteUtil;
 import org.apache.carbondata.core.util.DataTypeUtil;
 
 // Utility to create and retrieve data from CarbonRow in write step.
@@ -51,10 +51,10 @@ public class WriteStepRowUtil {
 
     // dictionary dimension
     byte[] mdk = ((ByteArrayWrapper) row[0]).getDictionaryKey();
-    long[] keys = segmentProperties.getDimensionKeyGenerator().getKeyArray(mdk);
-    int[] dictDimensions = new int[keys.length];
-    for (int i = 0; i < keys.length; i++) {
-      dictDimensions[i] = Long.valueOf(keys[i]).intValue();
+    int numDict = mdk.length / ByteUtil.dateBytesSize();
+    int[] dictDimensions = new int[numDict];
+    for (int i = 0; i < numDict; i++) {
+      dictDimensions[i] = ByteUtil.convertBytesToInt(mdk, i * ByteUtil.dateBytesSize());
     }
     converted[DICTIONARY_DIMENSION] = dictDimensions;
 
@@ -106,10 +106,6 @@ public class WriteStepRowUtil {
     return (int[]) row.getData()[DICTIONARY_DIMENSION];
   }
 
-  public static byte[] getMdk(CarbonRow row, KeyGenerator keyGenerator) {
-    return keyGenerator.generateKey(getDictDimension(row));
-  }
-
   public static Object[] getNoDictAndComplexDimension(CarbonRow row) {
     return (Object[]) row.getData()[NO_DICTIONARY_AND_COMPLEX];
   }
diff --git a/core/src/main/java/org/apache/carbondata/core/indexstore/BlockletDetailInfo.java b/core/src/main/java/org/apache/carbondata/core/indexstore/BlockletDetailInfo.java
index faad77e..667b271 100644
--- a/core/src/main/java/org/apache/carbondata/core/indexstore/BlockletDetailInfo.java
+++ b/core/src/main/java/org/apache/carbondata/core/indexstore/BlockletDetailInfo.java
@@ -55,8 +55,6 @@ public class BlockletDetailInfo implements Serializable, Writable {
   // default blockletId should be -1,which means consider all the blocklets in block
   private short blockletId = -1;
 
-  private int[] dimLens;
-
   private long schemaUpdatedTimeStamp;
 
   private BlockletInfo blockletInfo;
@@ -142,14 +140,6 @@ public class BlockletDetailInfo implements Serializable, Writable {
     }
   }
 
-  public int[] getDimLens() {
-    return dimLens;
-  }
-
-  public void setDimLens(int[] dimLens) {
-    this.dimLens = dimLens;
-  }
-
   public long getSchemaUpdatedTimeStamp() {
     return schemaUpdatedTimeStamp;
   }
@@ -172,10 +162,6 @@ public class BlockletDetailInfo implements Serializable, Writable {
     out.writeShort(pagesCount);
     out.writeShort(versionNumber);
     out.writeShort(blockletId);
-    out.writeShort(dimLens.length);
-    for (int i = 0; i < dimLens.length; i++) {
-      out.writeInt(dimLens[i]);
-    }
     out.writeLong(schemaUpdatedTimeStamp);
     out.writeBoolean(blockletInfo != null);
     if (blockletInfo != null) {
@@ -204,10 +190,6 @@ public class BlockletDetailInfo implements Serializable, Writable {
     pagesCount = in.readShort();
     versionNumber = in.readShort();
     blockletId = in.readShort();
-    dimLens = new int[in.readShort()];
-    for (int i = 0; i < dimLens.length; i++) {
-      dimLens[i] = in.readInt();
-    }
     schemaUpdatedTimeStamp = in.readLong();
     if (in.readBoolean()) {
       blockletInfo = new BlockletInfo();
@@ -255,7 +237,6 @@ public class BlockletDetailInfo implements Serializable, Writable {
     detailInfo.pagesCount = pagesCount;
     detailInfo.versionNumber = versionNumber;
     detailInfo.blockletId = blockletId;
-    detailInfo.dimLens = dimLens;
     detailInfo.schemaUpdatedTimeStamp = schemaUpdatedTimeStamp;
     detailInfo.blockletInfo = blockletInfo;
     detailInfo.blockletInfoBinary = blockletInfoBinary;
diff --git a/core/src/main/java/org/apache/carbondata/core/indexstore/ExtendedBlocklet.java b/core/src/main/java/org/apache/carbondata/core/indexstore/ExtendedBlocklet.java
index b20689c..5d61776 100644
--- a/core/src/main/java/org/apache/carbondata/core/indexstore/ExtendedBlocklet.java
+++ b/core/src/main/java/org/apache/carbondata/core/indexstore/ExtendedBlocklet.java
@@ -144,14 +144,6 @@ public class ExtendedBlocklet extends Blocklet {
     return inputSplit;
   }
 
-  public void setColumnCardinality(int[] cardinality) {
-    inputSplit.setColumnCardinality(cardinality);
-  }
-
-  public void setLegacyStore(boolean isLegacyStore) {
-    inputSplit.setLegacyStore(isLegacyStore);
-  }
-
   public void setUseMinMaxForPruning(boolean useMinMaxForPruning) {
     this.inputSplit.setUseMinMaxForPruning(useMinMaxForPruning);
   }
diff --git a/core/src/main/java/org/apache/carbondata/core/indexstore/blockletindex/BlockDataMap.java b/core/src/main/java/org/apache/carbondata/core/indexstore/blockletindex/BlockDataMap.java
index 0879b33..d7dddbf 100644
--- a/core/src/main/java/org/apache/carbondata/core/indexstore/blockletindex/BlockDataMap.java
+++ b/core/src/main/java/org/apache/carbondata/core/indexstore/blockletindex/BlockDataMap.java
@@ -50,7 +50,6 @@ import org.apache.carbondata.core.indexstore.row.DataMapRowImpl;
 import org.apache.carbondata.core.indexstore.schema.CarbonRowSchema;
 import org.apache.carbondata.core.metadata.ColumnarFormatVersion;
 import org.apache.carbondata.core.metadata.blocklet.DataFileFooter;
-import org.apache.carbondata.core.metadata.blocklet.index.BlockletIndex;
 import org.apache.carbondata.core.metadata.blocklet.index.BlockletMinMaxIndex;
 import org.apache.carbondata.core.metadata.schema.table.CarbonTable;
 import org.apache.carbondata.core.metadata.schema.table.column.CarbonColumn;
@@ -100,10 +99,6 @@ public class BlockDataMap extends CoarseGrainDataMap
   protected transient SegmentPropertiesAndSchemaHolder.SegmentPropertiesWrapper
       segmentPropertiesWrapper;
   /**
-   * flag to check for store from 1.1 or any prior version
-   */
-  protected boolean isLegacyStore;
-  /**
    * flag to be used for forming the complete file path from file name. It will be true in case of
    * partition table and non transactional table
    */
@@ -148,8 +143,6 @@ public class BlockDataMap extends CoarseGrainDataMap
         blockletDataMapInfo.getSegmentId().getBytes(CarbonCommonConstants.DEFAULT_CHARSET);
     if (!indexInfo.isEmpty()) {
       DataFileFooter fileFooter = indexInfo.get(0);
-      // store for 1.1 or any prior version will not have any blocklet information in file footer
-      isLegacyStore = fileFooter.getBlockletList() == null;
       // init segment properties and create schema
       SegmentProperties segmentProperties = initSegmentProperties(blockletDataMapInfo, fileFooter);
       createMemorySchema(blockletDataMapInfo);
@@ -189,13 +182,7 @@ public class BlockDataMap extends CoarseGrainDataMap
   protected DataMapRowImpl loadMetadata(CarbonRowSchema[] taskSummarySchema,
       SegmentProperties segmentProperties, BlockletDataMapModel blockletDataMapInfo,
       List<DataFileFooter> indexInfo) {
-    if (isLegacyStore) {
-      return loadBlockInfoForOldStore(taskSummarySchema, segmentProperties, blockletDataMapInfo,
-          indexInfo);
-    } else {
-      return loadBlockMetaInfo(taskSummarySchema, segmentProperties, blockletDataMapInfo,
-          indexInfo);
-    }
+    return loadBlockMetaInfo(taskSummarySchema, segmentProperties, blockletDataMapInfo, indexInfo);
   }
 
   /**
@@ -207,55 +194,12 @@ public class BlockDataMap extends CoarseGrainDataMap
   private SegmentProperties initSegmentProperties(BlockletDataMapModel blockletDataMapInfo,
       DataFileFooter fileFooter) {
     List<ColumnSchema> columnInTable = fileFooter.getColumnInTable();
-    int[] columnCardinality = fileFooter.getSegmentInfo().getColumnCardinality();
     segmentPropertiesWrapper = SegmentPropertiesAndSchemaHolder.getInstance()
         .addSegmentProperties(blockletDataMapInfo.getCarbonTable(),
-            columnInTable, columnCardinality, blockletDataMapInfo.getSegmentId());
+            columnInTable, blockletDataMapInfo.getSegmentId());
     return segmentPropertiesWrapper.getSegmentProperties();
   }
 
-  /**
-   * This is old store scenario, here blocklet information is not available in index
-   * file so load only block info. Old store refers to store in 1.1 or prior to 1.1 version
-   *
-   * @param blockletDataMapInfo
-   * @param indexInfo
-   */
-  protected DataMapRowImpl loadBlockInfoForOldStore(CarbonRowSchema[] taskSummarySchema,
-      SegmentProperties segmentProperties, BlockletDataMapModel blockletDataMapInfo,
-      List<DataFileFooter> indexInfo) {
-    DataMapRowImpl summaryRow = null;
-    CarbonRowSchema[] schema = getFileFooterEntrySchema();
-    boolean[] minMaxFlag = new boolean[segmentProperties.getColumnsValueSize().length];
-    FilterUtil.setMinMaxFlagForLegacyStore(minMaxFlag, segmentProperties);
-    long totalRowCount = 0;
-    for (DataFileFooter fileFooter : indexInfo) {
-      TableBlockInfo blockInfo = fileFooter.getBlockInfo().getTableBlockInfo();
-      BlockMetaInfo blockMetaInfo =
-          blockletDataMapInfo.getBlockMetaInfoMap().get(blockInfo.getFilePath());
-      // Here it loads info about all blocklets of index
-      // Only add if the file exists physically. There are scenarios which index file exists inside
-      // merge index but related carbondata files are deleted. In that case we first check whether
-      // the file exists physically or not
-      if (null != blockMetaInfo) {
-        BlockletIndex blockletIndex = fileFooter.getBlockletIndex();
-        BlockletMinMaxIndex minMaxIndex = blockletIndex.getMinMaxIndex();
-        summaryRow = loadToUnsafeBlock(schema, taskSummarySchema, fileFooter, segmentProperties,
-            getMinMaxCacheColumns(), blockInfo.getFilePath(), summaryRow,
-            blockMetaInfo, minMaxIndex.getMinValues(), minMaxIndex.getMaxValues(), minMaxFlag);
-        totalRowCount += fileFooter.getNumberOfRows();
-      }
-    }
-    List<Short> blockletCountList = new ArrayList<>();
-    blockletCountList.add((short) 0);
-    byte[] blockletCount = convertRowCountFromShortToByteArray(blockletCountList);
-    // set the total row count
-    summaryRow.setLong(totalRowCount, TASK_ROW_COUNT);
-    summaryRow.setByteArray(blockletCount, taskSummarySchema.length - 1);
-    setMinMaxFlagForTaskSummary(summaryRow, taskSummarySchema, segmentProperties, minMaxFlag);
-    return summaryRow;
-  }
-
   protected void setMinMaxFlagForTaskSummary(DataMapRow summaryRow,
       CarbonRowSchema[] taskSummarySchema, SegmentProperties segmentProperties,
       boolean[] minMaxFlag) {
@@ -287,14 +231,14 @@ public class BlockDataMap extends CoarseGrainDataMap
     boolean isLastFileFooterEntryNeedToBeAdded = false;
     CarbonRowSchema[] schema = getFileFooterEntrySchema();
     // flag for each block entry
-    boolean[] minMaxFlag = new boolean[segmentProperties.getColumnsValueSize().length];
+    boolean[] minMaxFlag = new boolean[segmentProperties.getNumberOfColumns()];
     Arrays.fill(minMaxFlag, true);
     // min max flag for task summary
-    boolean[] taskSummaryMinMaxFlag = new boolean[segmentProperties.getColumnsValueSize().length];
+    boolean[] taskSummaryMinMaxFlag = new boolean[segmentProperties.getNumberOfColumns()];
     Arrays.fill(taskSummaryMinMaxFlag, true);
     long totalRowCount = 0;
     for (DataFileFooter fileFooter : indexInfo) {
-      TableBlockInfo blockInfo = fileFooter.getBlockInfo().getTableBlockInfo();
+      TableBlockInfo blockInfo = fileFooter.getBlockInfo();
       BlockMetaInfo blockMetaInfo =
           blockletDataMapInfo.getBlockMetaInfoMap().get(blockInfo.getFilePath());
       footerCounter++;
@@ -328,15 +272,14 @@ public class BlockDataMap extends CoarseGrainDataMap
         // with unique file path because each unique path will correspond to one
         // block in the task. OR condition is to handle the loading of last file footer
         if (!blockInfo.getFilePath().equals(tempFilePath) || footerCounter == indexInfo.size()) {
-          TableBlockInfo previousBlockInfo =
-              previousDataFileFooter.getBlockInfo().getTableBlockInfo();
+          TableBlockInfo previousBlockInfo = previousDataFileFooter.getBlockInfo();
           summaryRow = loadToUnsafeBlock(schema, taskSummarySchema, previousDataFileFooter,
               segmentProperties, getMinMaxCacheColumns(), previousBlockInfo.getFilePath(),
               summaryRow,
               blockletDataMapInfo.getBlockMetaInfoMap().get(previousBlockInfo.getFilePath()),
               blockMinValues, blockMaxValues, minMaxFlag);
           totalRowCount += previousDataFileFooter.getNumberOfRows();
-          minMaxFlag = new boolean[segmentProperties.getColumnsValueSize().length];
+          minMaxFlag = new boolean[segmentProperties.getNumberOfColumns()];
           Arrays.fill(minMaxFlag, true);
           // flag to check whether last file footer entry is different from previous entry.
           // If yes then it need to be added at last
@@ -362,9 +305,9 @@ public class BlockDataMap extends CoarseGrainDataMap
       summaryRow =
           loadToUnsafeBlock(schema, taskSummarySchema, previousDataFileFooter, segmentProperties,
               getMinMaxCacheColumns(),
-              previousDataFileFooter.getBlockInfo().getTableBlockInfo().getFilePath(), summaryRow,
+              previousDataFileFooter.getBlockInfo().getFilePath(), summaryRow,
               blockletDataMapInfo.getBlockMetaInfoMap()
-                  .get(previousDataFileFooter.getBlockInfo().getTableBlockInfo().getFilePath()),
+                  .get(previousDataFileFooter.getBlockInfo().getFilePath()),
               blockMinValues, blockMaxValues, minMaxFlag);
       totalRowCount += previousDataFileFooter.getNumberOfRows();
       blockletCountInEachBlock.add(totalBlockletsInOneBlock);
@@ -425,7 +368,8 @@ public class BlockDataMap extends CoarseGrainDataMap
         .getMinMaxForColumnsToBeCached(segmentProperties, minMaxCacheColumns, maxValues);
     boolean[] minMaxFlagValuesForColumnsToBeCached = BlockletDataMapUtil
         .getMinMaxFlagValuesForColumnsToBeCached(segmentProperties, minMaxCacheColumns, minMaxFlag);
-    row.setRow(addMinMax(schema[ordinal], minValuesForColumnsToBeCached), ordinal);
+    DataMapRow dataMapRow = addMinMax(schema[ordinal], minValuesForColumnsToBeCached);
+    row.setRow(dataMapRow, ordinal);
     // compute and set task level min values
     addTaskMinMaxValues(summaryRow, taskSummarySchema, taskMinMaxOrdinal,
         minValuesForColumnsToBeCached, TASK_MIN_VALUES_INDEX, true);
@@ -447,7 +391,7 @@ public class BlockDataMap extends CoarseGrainDataMap
     // add schema updated time
     row.setLong(fileFooter.getSchemaUpdatedTimeStamp(), ordinal++);
     // add block offset
-    row.setLong(fileFooter.getBlockInfo().getTableBlockInfo().getBlockOffset(), ordinal++);
+    row.setLong(fileFooter.getBlockInfo().getBlockOffset(), ordinal++);
     try {
       setLocations(blockMetaInfo.getLocationInfo(), row, ordinal++);
       // store block size
@@ -625,29 +569,19 @@ public class BlockDataMap extends CoarseGrainDataMap
    * if data is not legacy store, we can get blocklet count from taskSummaryDMStore
    */
   protected short getBlockletNumOfEntry(int index) {
-    if (isLegacyStore) {
-      // dummy value
+    final byte[] bytes = getBlockletRowCountForEachBlock();
+    // if the segment data is written in tablepath
+    // then the reuslt of getBlockletRowCountForEachBlock will be empty.
+    if (bytes.length == 0) {
       return 0;
     } else {
-      final byte[] bytes = getBlockletRowCountForEachBlock();
-      // if the segment data is written in tablepath
-      // then the reuslt of getBlockletRowCountForEachBlock will be empty.
-      if (bytes.length == 0) {
-        return 0;
-      } else {
-        return ByteBuffer.wrap(bytes).getShort(index * CarbonCommonConstants.SHORT_SIZE_IN_BYTE);
-      }
+      return ByteBuffer.wrap(bytes).getShort(index * CarbonCommonConstants.SHORT_SIZE_IN_BYTE);
     }
   }
 
   // get total block number in this datamap
   public int getTotalBlocks() {
-    if (isLegacyStore) {
-      // dummy value
-      return 0;
-    } else {
-      return memoryDMStore.getRowCount();
-    }
+    return memoryDMStore.getRowCount();
   }
 
   // get total blocklet number in this datamap
@@ -756,14 +690,10 @@ public class BlockDataMap extends CoarseGrainDataMap
       }
     }
     if (ExplainCollector.enabled()) {
-      if (isLegacyStore) {
-        ExplainCollector.setShowPruningInfo(false);
-      } else {
-        ExplainCollector.setShowPruningInfo(true);
-        ExplainCollector.addTotalBlocklets(totalBlocklets);
-        ExplainCollector.addTotalBlocks(getTotalBlocks());
-        ExplainCollector.addDefaultDataMapPruningHit(hitBlocklets);
-      }
+      ExplainCollector.setShowPruningInfo(true);
+      ExplainCollector.addTotalBlocklets(totalBlocklets);
+      ExplainCollector.addTotalBlocks(getTotalBlocks());
+      ExplainCollector.addDefaultDataMapPruningHit(hitBlocklets);
     }
     return blocklets;
   }
@@ -868,10 +798,6 @@ public class BlockDataMap extends CoarseGrainDataMap
   }
 
   public ExtendedBlocklet getDetailedBlocklet(String blockletId) {
-    if (isLegacyStore) {
-      throw new UnsupportedOperationException("With legacy store only BlockletDataMap is allowed."
-          + " In order to use other dataMaps upgrade to new store.");
-    }
     int absoluteBlockletId = Integer.parseInt(blockletId);
     return createBlockletFromRelativeBlockletId(absoluteBlockletId);
   }
@@ -970,8 +896,6 @@ public class BlockDataMap extends CoarseGrainDataMap
     ExtendedBlocklet blocklet = new ExtendedBlocklet(fileName, blockletId + "", false,
         ColumnarFormatVersion.valueOf(versionNumber));
     blocklet.setDataMapRow(row);
-    blocklet.setColumnCardinality(getColumnCardinality());
-    blocklet.setLegacyStore(isLegacyStore);
     blocklet.setUseMinMaxForPruning(useMinMaxForPruning);
     return blocklet;
   }
@@ -1018,10 +942,6 @@ public class BlockDataMap extends CoarseGrainDataMap
     return segmentPropertiesWrapper.getSegmentProperties();
   }
 
-  public int[] getColumnCardinality() {
-    return segmentPropertiesWrapper.getColumnCardinality();
-  }
-
   public List<ColumnSchema> getColumnSchema() {
     return segmentPropertiesWrapper.getColumnsInTable();
   }
@@ -1047,7 +967,6 @@ public class BlockDataMap extends CoarseGrainDataMap
   /**
    * This method will ocnvert safe to unsafe memory DM store
    *
-   * @throws MemoryException
    */
   public void convertToUnsafeDMStore() {
     if (memoryDMStore instanceof SafeMemoryDMStore) {
diff --git a/core/src/main/java/org/apache/carbondata/core/indexstore/blockletindex/BlockletDataMap.java b/core/src/main/java/org/apache/carbondata/core/indexstore/blockletindex/BlockletDataMap.java
index 0b03746..29262fb 100644
--- a/core/src/main/java/org/apache/carbondata/core/indexstore/blockletindex/BlockletDataMap.java
+++ b/core/src/main/java/org/apache/carbondata/core/indexstore/blockletindex/BlockletDataMap.java
@@ -66,28 +66,17 @@ public class BlockletDataMap extends BlockDataMap implements Serializable {
   protected DataMapRowImpl loadMetadata(CarbonRowSchema[] taskSummarySchema,
       SegmentProperties segmentProperties, BlockletDataMapModel blockletDataMapInfo,
       List<DataFileFooter> indexInfo) {
-    if (isLegacyStore) {
-      return loadBlockInfoForOldStore(taskSummarySchema, segmentProperties, blockletDataMapInfo,
-          indexInfo);
-    } else {
-      return loadBlockletMetaInfo(taskSummarySchema, segmentProperties, blockletDataMapInfo,
-          indexInfo);
-    }
+    return loadBlockletMetaInfo(taskSummarySchema, segmentProperties, blockletDataMapInfo,
+        indexInfo);
   }
 
   @Override
   protected CarbonRowSchema[] getTaskSummarySchema() {
-    if (isLegacyStore) {
-      return super.getTaskSummarySchema();
-    }
     return segmentPropertiesWrapper.getTaskSummarySchemaForBlocklet(false, isFilePathStored);
   }
 
   @Override
   protected CarbonRowSchema[] getFileFooterEntrySchema() {
-    if (isLegacyStore) {
-      return super.getFileFooterEntrySchema();
-    }
     return segmentPropertiesWrapper.getBlockletFileFooterEntrySchema();
   }
 
@@ -103,14 +92,14 @@ public class BlockletDataMap extends BlockDataMap implements Serializable {
     String tempFilePath = null;
     DataMapRowImpl summaryRow = null;
     CarbonRowSchema[] schema = getFileFooterEntrySchema();
-    boolean[] summaryRowMinMaxFlag = new boolean[segmentProperties.getColumnsValueSize().length];
+    boolean[] summaryRowMinMaxFlag = new boolean[segmentProperties.getNumberOfColumns()];
     Arrays.fill(summaryRowMinMaxFlag, true);
     // Relative blocklet ID is the id assigned to a blocklet within a part file
     int relativeBlockletId = 0;
     for (DataFileFooter fileFooter : indexInfo) {
       // update the min max flag for summary row
       updateMinMaxFlag(fileFooter, summaryRowMinMaxFlag);
-      TableBlockInfo blockInfo = fileFooter.getBlockInfo().getTableBlockInfo();
+      TableBlockInfo blockInfo = fileFooter.getBlockInfo();
       BlockMetaInfo blockMetaInfo =
           blockletDataMapInfo.getBlockMetaInfoMap().get(blockInfo.getFilePath());
       // Here it loads info about all blocklets of index
@@ -188,7 +177,7 @@ public class BlockletDataMap extends BlockDataMap implements Serializable {
       byte[] serializedData;
       try {
         // Add block footer offset, it is used if we need to read footer of block
-        row.setLong(fileFooter.getBlockInfo().getTableBlockInfo().getBlockOffset(), ordinal++);
+        row.setLong(fileFooter.getBlockInfo().getBlockOffset(), ordinal++);
         setLocations(blockMetaInfo.getLocationInfo(), row, ordinal++);
         // Store block size
         row.setLong(blockMetaInfo.getSize(), ordinal++);
@@ -215,9 +204,6 @@ public class BlockletDataMap extends BlockDataMap implements Serializable {
 
   @Override
   public ExtendedBlocklet getDetailedBlocklet(String blockletId) {
-    if (isLegacyStore) {
-      return super.getDetailedBlocklet(blockletId);
-    }
     int absoluteBlockletId = Integer.parseInt(blockletId);
     DataMapRow row = memoryDMStore.getDataMapRow(getFileFooterEntrySchema(), absoluteBlockletId);
     short relativeBlockletId = row.getShort(BLOCKLET_ID_INDEX);
@@ -228,16 +214,10 @@ public class BlockletDataMap extends BlockDataMap implements Serializable {
 
   @Override
   protected short getBlockletId(DataMapRow dataMapRow) {
-    if (isLegacyStore) {
-      return super.getBlockletId(dataMapRow);
-    }
     return dataMapRow.getShort(BLOCKLET_ID_INDEX);
   }
 
   protected boolean useMinMaxForExecutorPruning(FilterResolverIntf filterResolverIntf) {
-    if (isLegacyStore) {
-      return super.useMinMaxForExecutorPruning(filterResolverIntf);
-    }
     return BlockletDataMapUtil
         .useMinMaxForBlockletPruning(filterResolverIntf, getMinMaxCacheColumns());
   }
@@ -245,47 +225,30 @@ public class BlockletDataMap extends BlockDataMap implements Serializable {
   @Override
   protected ExtendedBlocklet createBlocklet(DataMapRow row, String fileName, short blockletId,
       boolean useMinMaxForPruning) {
-    if (isLegacyStore) {
-      return super.createBlocklet(row, fileName, blockletId, useMinMaxForPruning);
-    }
     short versionNumber = row.getShort(VERSION_INDEX);
     ExtendedBlocklet blocklet = new ExtendedBlocklet(fileName, blockletId + "",
         ColumnarFormatVersion.valueOf(versionNumber));
     blocklet.setColumnSchema(getColumnSchema());
     blocklet.setUseMinMaxForPruning(useMinMaxForPruning);
     blocklet.setIsBlockCache(false);
-    blocklet.setColumnCardinality(getColumnCardinality());
-    blocklet.setLegacyStore(isLegacyStore);
     blocklet.setDataMapRow(row);
     return blocklet;
   }
 
   @Override
   protected short getBlockletNumOfEntry(int index) {
-    if (isLegacyStore) {
-      return super.getBlockletNumOfEntry(index);
-    } else {
-      //in blocklet datamap, each entry contains info of one blocklet
-      return 1;
-    }
+    //in blocklet datamap, each entry contains info of one blocklet
+    return 1;
   }
 
   @Override
   public int getTotalBlocks() {
-    if (isLegacyStore) {
-      return super.getTotalBlocklets();
-    } else {
-      return blockNum;
-    }
+    return blockNum;
   }
 
   @Override
   protected int getTotalBlocklets() {
-    if (isLegacyStore) {
-      return super.getTotalBlocklets();
-    } else {
-      return memoryDMStore.getRowCount();
-    }
+    return memoryDMStore.getRowCount();
   }
 
 }
diff --git a/core/src/main/java/org/apache/carbondata/core/indexstore/blockletindex/BlockletDataRefNode.java b/core/src/main/java/org/apache/carbondata/core/indexstore/blockletindex/BlockletDataRefNode.java
index 42dae6d..9d6d5f6 100644
--- a/core/src/main/java/org/apache/carbondata/core/indexstore/blockletindex/BlockletDataRefNode.java
+++ b/core/src/main/java/org/apache/carbondata/core/indexstore/blockletindex/BlockletDataRefNode.java
@@ -45,11 +45,9 @@ public class BlockletDataRefNode implements DataRefNode {
 
   private int index;
 
-  private int[] dimensionLens;
-
   private BlockletSerializer blockletSerializer;
 
-  BlockletDataRefNode(List<TableBlockInfo> blockInfos, int index, int[] dimensionLens) {
+  BlockletDataRefNode(List<TableBlockInfo> blockInfos, int index) {
     this.blockInfos = blockInfos;
     // Update row count and page count to blocklet info
     for (TableBlockInfo blockInfo : blockInfos) {
@@ -80,14 +78,13 @@ public class BlockletDataRefNode implements DataRefNode {
       }
     }
     this.index = index;
-    this.dimensionLens = dimensionLens;
     this.blockletSerializer = new BlockletSerializer();
   }
 
   @Override
   public DataRefNode getNextDataRefNode() {
     if (index + 1 < blockInfos.size()) {
-      return new BlockletDataRefNode(blockInfos, index + 1, dimensionLens);
+      return new BlockletDataRefNode(blockInfos, index + 1);
     }
     return null;
   }
@@ -98,11 +95,6 @@ public class BlockletDataRefNode implements DataRefNode {
   }
 
   @Override
-  public long nodeIndex() {
-    return index;
-  }
-
-  @Override
   public short blockletIndex() {
     return blockInfos.get(index).getDetailInfo().getBlockletId();
   }
@@ -160,7 +152,6 @@ public class BlockletDataRefNode implements DataRefNode {
     MeasureColumnChunkReader measureColumnChunkReader = getMeasureColumnChunkReader(fileReader);
     MeasureRawColumnChunk[] measureRawColumnChunks =
         measureColumnChunkReader.readRawMeasureChunks(fileReader, columnIndexRange);
-    updateMeasureRawColumnChunkMinMaxValues(measureRawColumnChunks);
     return measureRawColumnChunks;
   }
 
@@ -170,49 +161,19 @@ public class BlockletDataRefNode implements DataRefNode {
     MeasureColumnChunkReader measureColumnChunkReader = getMeasureColumnChunkReader(fileReader);
     MeasureRawColumnChunk measureRawColumnChunk =
         measureColumnChunkReader.readRawMeasureChunk(fileReader, columnIndex);
-    updateMeasureRawColumnChunkMinMaxValues(measureRawColumnChunk);
     return measureRawColumnChunk;
   }
 
-  /**
-   * This method is written specifically for old store wherein the measure min and max values
-   * are written opposite (i.e min in place of max and amx in place of min). Due to this computing
-   * f measure filter with current code is impacted. In order to sync with current min and
-   * max values only in case old store and measures is reversed
-   *
-   * @param measureRawColumnChunk
-   */
-  private void updateMeasureRawColumnChunkMinMaxValues(
-      MeasureRawColumnChunk measureRawColumnChunk) {
-    if (blockInfos.get(index).isDataBlockFromOldStore()) {
-      byte[][] maxValues = measureRawColumnChunk.getMaxValues();
-      byte[][] minValues = measureRawColumnChunk.getMinValues();
-      measureRawColumnChunk.setMaxValues(minValues);
-      measureRawColumnChunk.setMinValues(maxValues);
-    }
-  }
-
-  private void updateMeasureRawColumnChunkMinMaxValues(
-      MeasureRawColumnChunk[] measureRawColumnChunks) {
-    if (blockInfos.get(index).isDataBlockFromOldStore()) {
-      for (int i = 0; i < measureRawColumnChunks.length; i++) {
-        if (null != measureRawColumnChunks[i]) {
-          updateMeasureRawColumnChunkMinMaxValues(measureRawColumnChunks[i]);
-        }
-      }
-    }
-  }
-
   private DimensionColumnChunkReader getDimensionColumnChunkReader(FileReader fileReader) {
     ColumnarFormatVersion version =
         ColumnarFormatVersion.valueOf(blockInfos.get(index).getDetailInfo().getVersionNumber());
     if (fileReader.isReadPageByPage()) {
       return CarbonDataReaderFactory.getInstance().getDimensionColumnChunkReader(version,
-          blockInfos.get(index).getDetailInfo().getBlockletInfo(), dimensionLens,
+          blockInfos.get(index).getDetailInfo().getBlockletInfo(),
           blockInfos.get(index).getFilePath(), true);
     } else {
       return CarbonDataReaderFactory.getInstance().getDimensionColumnChunkReader(version,
-          blockInfos.get(index).getDetailInfo().getBlockletInfo(), dimensionLens,
+          blockInfos.get(index).getDetailInfo().getBlockletInfo(),
           blockInfos.get(index).getFilePath(), false);
     }
   }
diff --git a/core/src/main/java/org/apache/carbondata/core/indexstore/blockletindex/IndexWrapper.java b/core/src/main/java/org/apache/carbondata/core/indexstore/blockletindex/IndexWrapper.java
index 24497ed..62af08b 100644
--- a/core/src/main/java/org/apache/carbondata/core/indexstore/blockletindex/IndexWrapper.java
+++ b/core/src/main/java/org/apache/carbondata/core/indexstore/blockletindex/IndexWrapper.java
@@ -24,7 +24,6 @@ import org.apache.carbondata.core.datastore.block.SegmentProperties;
 import org.apache.carbondata.core.datastore.block.TableBlockInfo;
 import org.apache.carbondata.core.datastore.filesystem.CarbonFile;
 import org.apache.carbondata.core.datastore.impl.FileFactory;
-import org.apache.carbondata.core.metadata.blocklet.DataFileFooter;
 
 /**
  * Wrapper of abstract index
@@ -37,12 +36,7 @@ public class IndexWrapper extends AbstractIndex {
   public IndexWrapper(List<TableBlockInfo> blockInfos, SegmentProperties segmentProperties) {
     this.blockInfos = blockInfos;
     this.segmentProperties = segmentProperties;
-    dataRefNode = new BlockletDataRefNode(blockInfos, 0,
-        this.segmentProperties.getDimensionColumnsValueSize());
-  }
-
-  @Override
-  public void buildIndex(List<DataFileFooter> footerList) {
+    dataRefNode = new BlockletDataRefNode(blockInfos, 0);
   }
 
   @Override
diff --git a/core/src/main/java/org/apache/carbondata/core/indexstore/schema/SchemaGenerator.java b/core/src/main/java/org/apache/carbondata/core/indexstore/schema/SchemaGenerator.java
index 71bd96c..de40efc 100644
--- a/core/src/main/java/org/apache/carbondata/core/indexstore/schema/SchemaGenerator.java
+++ b/core/src/main/java/org/apache/carbondata/core/indexstore/schema/SchemaGenerator.java
@@ -256,7 +256,7 @@ public class SchemaGenerator {
    */
   private static void addMinMaxFlagSchema(SegmentProperties segmentProperties,
       List<CarbonRowSchema> indexSchemas, List<CarbonColumn> minMaxCacheColumns) {
-    int minMaxFlagLength = segmentProperties.getColumnsValueSize().length;
+    int minMaxFlagLength = segmentProperties.getNumberOfColumns();
     if (null != minMaxCacheColumns) {
       minMaxFlagLength = minMaxCacheColumns.size();
     }
@@ -285,11 +285,11 @@ public class SchemaGenerator {
       minMaxLen = new int[minMaxCacheColumns.size()];
       int counter = 0;
       for (CarbonColumn column : minMaxCacheColumns) {
-        minMaxLen[counter++] = segmentProperties.getColumnsValueSize()[BlockletDataMapUtil
+        minMaxLen[counter++] = segmentProperties.createColumnValueLength()[BlockletDataMapUtil
             .getColumnOrdinal(segmentProperties, column)];
       }
     } else {
-      minMaxLen = segmentProperties.getColumnsValueSize();
+      minMaxLen = segmentProperties.createColumnValueLength();
     }
     return minMaxLen;
   }
@@ -314,7 +314,7 @@ public class SchemaGenerator {
     } else {
       // when columns to cache is not specified then column access order will be same as the array
       // index of min max length
-      columnOrdinalsTOAccess = new int[segmentProperties.getColumnsValueSize().length];
+      columnOrdinalsTOAccess = new int[segmentProperties.getNumberOfColumns()];
       for (int i = 0; i < columnOrdinalsTOAccess.length; i++) {
         columnOrdinalsTOAccess[i] = i;
       }
diff --git a/core/src/main/java/org/apache/carbondata/core/keygenerator/KeyGenerator.java b/core/src/main/java/org/apache/carbondata/core/keygenerator/KeyGenerator.java
index fed1801..0a0cdbc 100644
--- a/core/src/main/java/org/apache/carbondata/core/keygenerator/KeyGenerator.java
+++ b/core/src/main/java/org/apache/carbondata/core/keygenerator/KeyGenerator.java
@@ -26,13 +26,6 @@ import java.util.Comparator;
  * key(dimensional key) out of combined key.
  */
 public interface KeyGenerator extends Serializable, Comparator<byte[]> {
-  /**
-   * It generates the single key aka byte array from multiple keys.
-   *
-   * @param keys
-   * @return byte array
-   */
-  byte[] generateKey(long[] keys);
 
   /**
    * It generates the single key aka byte array from multiple keys.
@@ -46,52 +39,12 @@ public interface KeyGenerator extends Serializable, Comparator<byte[]> {
    * It gets array of keys out of single key aka byte array
    *
    * @param key
-   * @return array of keys.
-   */
-  long[] getKeyArray(byte[] key);
-
-  /**
-   * It gets array of keys out of single key aka byte array
-   *
-   * @param key
    * @param offset
    * @return array of keys.
    */
   long[] getKeyArray(byte[] key, int offset);
 
   /**
-   * It gets array of keys out of single key aka byte array
-   *
-   * @param key
-   * @param maskedByteRanges
-   * @return array of keys
-   */
-  long[] getKeyArray(byte[] key, int[] maskedByteRanges);
-
-  /**
-   * It gets the key in the specified index from the single key aka byte array
-   *
-   * @param key
-   * @param index of key.
-   * @return key
-   */
-  long getKey(byte[] key, int index);
-
-  /**
-   * Gives the key size in number of bytes.
-   */
-  int getKeySizeInBytes();
-
-
-  /**
-   * returns key bytes offset
-   *
-   * @param index
-   * @return
-   */
-  int[] getKeyByteOffsets(int index);
-
-  /**
    * returns the dimension count
    *
    * @return
diff --git a/core/src/main/java/org/apache/carbondata/core/keygenerator/columnar/ColumnarSplitter.java b/core/src/main/java/org/apache/carbondata/core/keygenerator/columnar/ColumnarSplitter.java
deleted file mode 100644
index 7160808..0000000
--- a/core/src/main/java/org/apache/carbondata/core/keygenerator/columnar/ColumnarSplitter.java
+++ /dev/null
@@ -1,79 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.carbondata.core.keygenerator.columnar;
-
-/**
- * Splits the odometer key to columns.Further these columns can be stored in a columnar storage.
- */
-public interface ColumnarSplitter {
-  /**
-   * Splits generated MDKey to multiple columns.
-   *
-   * @param key MDKey
-   * @return Multiple columns in 2 dimensional byte array
-   */
-  byte[][] splitKey(byte[] key);
-
-  /**
-   * It generates and splits key to multiple columns
-   *
-   * @param keys
-   * @return
-   */
-  byte[][] generateAndSplitKey(long[] keys);
-
-  /**
-   * It generates and splits key to multiple columns
-   *
-   * @param keys
-   * @return
-   */
-  byte[][] generateAndSplitKey(int[] keys);
-
-  /**
-   * Takes the split keys and generates the surrogate key array
-   *
-   * @param key
-   * @return
-   */
-  long[] getKeyArray(byte[][] key);
-
-  /**
-   * Takes the split keys and generates the surrogate key array in bytes
-   *
-   * @param key
-   * @return
-   */
-  byte[] getKeyByteArray(byte[][] key);
-
-  /**
-   * Below method will be used to get the block size
-   *
-   * @return
-   */
-  int[] getBlockKeySize();
-
-  /**
-   * Below method will be used to get the total key Size of the particular block
-   *
-   * @param blockIndexes
-   * @return
-   */
-  int getKeySizeByBlock(int[] blockIndexes);
-
-}
diff --git a/core/src/main/java/org/apache/carbondata/core/keygenerator/columnar/impl/MultiDimKeyVarLengthEquiSplitGenerator.java b/core/src/main/java/org/apache/carbondata/core/keygenerator/columnar/impl/MultiDimKeyVarLengthEquiSplitGenerator.java
deleted file mode 100644
index 01b4a67..0000000
--- a/core/src/main/java/org/apache/carbondata/core/keygenerator/columnar/impl/MultiDimKeyVarLengthEquiSplitGenerator.java
+++ /dev/null
@@ -1,207 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.carbondata.core.keygenerator.columnar.impl;
-
-import java.util.ArrayList;
-import java.util.List;
-import java.util.Objects;
-import java.util.Set;
-import java.util.TreeSet;
-
-import org.apache.carbondata.core.constants.CarbonCommonConstants;
-import org.apache.carbondata.core.keygenerator.columnar.ColumnarSplitter;
-import org.apache.carbondata.core.keygenerator.mdkey.MultiDimKeyVarLengthGenerator;
-
-/**
- * It is Equi Split implementation class of Columnar splitter. And uses var key length
- * generator to generate keys.
- * It splits depends on the @dimensionsToSplit parameter. This parameter decides how many
- * dimensions should be present in each column.
- */
-public class MultiDimKeyVarLengthEquiSplitGenerator extends MultiDimKeyVarLengthGenerator
-    implements ColumnarSplitter {
-
-  private static final long serialVersionUID = -7767757692821917570L;
-
-  private byte dimensionsToSplit;
-
-  private int[] blockKeySize;
-
-  public MultiDimKeyVarLengthEquiSplitGenerator(int[] lens, byte dimensionsToSplit) {
-    super(lens);
-    this.dimensionsToSplit = dimensionsToSplit;
-    initialize();
-  }
-
-  private void initialize() {
-    byte s = 0;
-    List<Set<Integer>> splitList =
-        new ArrayList<Set<Integer>>(CarbonCommonConstants.CONSTANT_SIZE_TEN);
-    Set<Integer> split = new TreeSet<Integer>();
-    splitList.add(split);
-    for (int i = 0; i < byteRangesForKeys.length; i++) {
-      if (s == dimensionsToSplit) {
-        s = 0;
-        split = new TreeSet<Integer>();
-        splitList.add(split);
-      }
-      for (int j = 0; j < byteRangesForKeys[i].length; j++) {
-        for (int j2 = byteRangesForKeys[i][0]; j2 <= byteRangesForKeys[i][1]; j2++) {
-          split.add(j2);
-        }
-      }
-      s++;
-    }
-    List<Integer>[] splits = new List[splitList.size()];
-    int i = 0;
-    for (Set<Integer> splitLocal : splitList) {
-      List<Integer> range = new ArrayList<Integer>(splitLocal);
-      splits[i++] = range;
-    }
-    for (int j = 1; j < splits.length; j++) {
-      if (Objects.equals(splits[j - 1].get(splits[j - 1].size() - 1), splits[j].get(0))) {
-        splits[j].remove(0);
-      }
-    }
-    int[][] splitDimArray = new int[splits.length][];
-    for (int j = 0; j < splits.length; j++) {
-      int[] a = convertToArray(splits[j]);
-      splitDimArray[j] = a.length > 0 ? new int[] { a[0], a[a.length - 1] } : a;
-    }
-
-    int[][] dimBlockArray = new int[byteRangesForKeys.length][];
-    Set<Integer>[] dimBlockSet = new Set[dimBlockArray.length];
-    for (int k = 0; k < byteRangesForKeys.length; k++) {
-      int[] dimRange = byteRangesForKeys[k];
-      Set<Integer> dimBlockPosSet = new TreeSet<Integer>();
-      dimBlockSet[k] = dimBlockPosSet;
-      for (int j = 0; j < splitDimArray.length; j++) {
-        if (dimRange[0] >= splitDimArray[j][0] && dimRange[0] <= splitDimArray[j][1]) {
-          dimBlockPosSet.add(j);
-        }
-        if (dimRange[1] >= splitDimArray[j][0] && dimRange[1] <= splitDimArray[j][1]) {
-          dimBlockPosSet.add(j);
-        }
-      }
-
-    }
-
-    for (int j = 0; j < dimBlockSet.length; j++) {
-      dimBlockArray[j] = convertToArray(dimBlockSet[j]);
-    }
-
-    blockKeySize = new int[splitDimArray.length];
-
-    for (int j = 0; j < blockKeySize.length; j++) {
-      blockKeySize[j] =
-          splitDimArray[j].length > 0 ? splitDimArray[j][1] - splitDimArray[j][0] + 1 : 0;
-    }
-  }
-
-  private int[] convertToArray(List<Integer> list) {
-    int[] ints = new int[list.size()];
-    int i = 0;
-    for (Integer ii : list) {
-      ints[i++] = ii;
-    }
-    return ints;
-  }
-
-  private int[] convertToArray(Set<Integer> set) {
-    int[] ints = new int[set.size()];
-    int i = 0;
-    for (Integer ii: set) {
-      ints[i++] = ii;
-    }
-    return ints;
-  }
-
-  @Override
-  public byte[][] splitKey(byte[] key) {
-    byte[][] split = new byte[blockKeySize.length][];
-    int copyIndex = 0;
-    for (int i = 0; i < split.length; i++) {
-      split[i] = new byte[blockKeySize[i]];
-      System.arraycopy(key, copyIndex, split[i], 0, split[i].length);
-      copyIndex += blockKeySize[i];
-    }
-    return split;
-  }
-
-  @Override
-  public byte[][] generateAndSplitKey(long[] keys) {
-    return splitKey(generateKey(keys));
-  }
-
-  @Override
-  public byte[][] generateAndSplitKey(int[] keys) {
-    return splitKey(generateKey(keys));
-  }
-
-  @Override
-  public long[] getKeyArray(byte[][] key) {
-    byte[] fullKey = new byte[getKeySizeInBytes()];
-    int copyIndex = 0;
-    for (int i = 0; i < key.length; i++) {
-      System.arraycopy(key[i], 0, fullKey, copyIndex, key[i].length);
-      copyIndex += key[i].length;
-    }
-    return getKeyArray(fullKey);
-  }
-
-  @Override
-  public byte[] getKeyByteArray(byte[][] key) {
-    byte[] fullKey = new byte[getKeySizeInBytes()];
-    int copyIndex = 0;
-    for (int i = 0; i < key.length; i++) {
-      System.arraycopy(key[i], 0, fullKey, copyIndex, key[i].length);
-      copyIndex += key[i].length;
-    }
-    return fullKey;
-  }
-
-  public int[] getBlockKeySize() {
-    return blockKeySize;
-  }
-
-  @Override
-  public int getKeySizeByBlock(int[] blockIndexes) {
-    int size = 0;
-
-    for (int i = 0; i < blockIndexes.length; i++) {
-      if (blockIndexes[i] < blockKeySize.length) {
-        size += blockKeySize[blockIndexes[i]];
-      }
-    }
-    return size;
-  }
-
-  @Override
-  public boolean equals(Object obj) {
-    if (!(obj instanceof MultiDimKeyVarLengthEquiSplitGenerator)) {
-      return false;
-    }
-    MultiDimKeyVarLengthEquiSplitGenerator o = (MultiDimKeyVarLengthEquiSplitGenerator)obj;
-    return o.dimensionsToSplit == dimensionsToSplit && super.equals(obj);
-  }
-
-  @Override
-  public int hashCode() {
-    return super.hashCode() + dimensionsToSplit;
-  }
-}
diff --git a/core/src/main/java/org/apache/carbondata/core/keygenerator/columnar/impl/MultiDimKeyVarLengthVariableSplitGenerator.java b/core/src/main/java/org/apache/carbondata/core/keygenerator/columnar/impl/MultiDimKeyVarLengthVariableSplitGenerator.java
deleted file mode 100644
index 781bb8f..0000000
--- a/core/src/main/java/org/apache/carbondata/core/keygenerator/columnar/impl/MultiDimKeyVarLengthVariableSplitGenerator.java
+++ /dev/null
@@ -1,209 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.carbondata.core.keygenerator.columnar.impl;
-
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.HashSet;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Objects;
-import java.util.Set;
-import java.util.TreeSet;
-
-import org.apache.carbondata.core.constants.CarbonCommonConstants;
-import org.apache.carbondata.core.keygenerator.columnar.ColumnarSplitter;
-import org.apache.carbondata.core.keygenerator.mdkey.MultiDimKeyVarLengthGenerator;
-
-public class MultiDimKeyVarLengthVariableSplitGenerator extends MultiDimKeyVarLengthGenerator
-    implements ColumnarSplitter {
-
-  private static final long serialVersionUID = 1L;
-
-  private int[] dimensionsToSplit;
-
-  private int[] blockKeySize;
-
-  public MultiDimKeyVarLengthVariableSplitGenerator(int[] lens, int[] dimSplit) {
-    super(lens);
-    this.dimensionsToSplit = dimSplit;
-    initialise();
-
-  }
-
-  private void initialise() {
-    int s = 0;
-    List<Set<Integer>> splitList =
-        new ArrayList<Set<Integer>>(CarbonCommonConstants.CONSTANT_SIZE_TEN);
-    Set<Integer> split = new TreeSet<Integer>();
-    splitList.add(split);
-    int dimSplitIndx = 0;
-
-    for (int i = 0; i < byteRangesForKeys.length; i++) {
-      if (s == dimensionsToSplit[dimSplitIndx]) {
-        s = 0;
-        split = new TreeSet<Integer>();
-        splitList.add(split);
-        dimSplitIndx++;
-      }
-      for (int j = 0; j < byteRangesForKeys[i].length; j++) {
-        for (int j2 = byteRangesForKeys[i][0]; j2 <= byteRangesForKeys[i][1]; j2++) {
-          split.add(j2);
-        }
-      }
-      s++;
-
-    }
-    List<Integer>[] splits = new List[splitList.size()];
-    int i = 0;
-    for (Set<Integer> splitLocal : splitList) {
-      List<Integer> range = new ArrayList<Integer>(splitLocal);
-      splits[i++] = range;
-    }
-    for (int j = 1; j < splits.length; j++) {
-      if (Objects.equals(splits[j - 1].get(splits[j - 1].size() - 1), splits[j].get(0))) {
-        splits[j].remove(0);
-      }
-    }
-    int[][] splitDimArray = new int[splits.length][];
-    for (int j = 0; j < splits.length; j++) {
-      int[] a = convertToArray(splits[j]);
-      splitDimArray[j] = a.length > 0 ? new int[] { a[0], a[a.length - 1] } : a;
-    }
-
-    int[][] dimBlockArray = new int[byteRangesForKeys.length][];
-    Set<Integer>[] dimBlockSet = new Set[dimBlockArray.length];
-    for (int k = 0; k < byteRangesForKeys.length; k++) {
-      int[] dimRange = byteRangesForKeys[k];
-      Set<Integer> dimBlockPosSet = new TreeSet<Integer>();
-      dimBlockSet[k] = dimBlockPosSet;
-      for (int j = 0; j < splitDimArray.length; j++) {
-        if (dimRange[0] >= splitDimArray[j][0] && dimRange[0] <= splitDimArray[j][1]) {
-          dimBlockPosSet.add(j);
-        }
-        if (dimRange[1] >= splitDimArray[j][0] && dimRange[1] <= splitDimArray[j][1]) {
-          dimBlockPosSet.add(j);
-        }
-      }
-
-    }
-
-    for (int j = 0; j < dimBlockSet.length; j++) {
-      dimBlockArray[j] = convertToArray(dimBlockSet[j]);
-    }
-
-    blockKeySize = new int[splitDimArray.length];
-
-    for (int j = 0; j < blockKeySize.length; j++) {
-      blockKeySize[j] =
-          splitDimArray[j].length > 0 ? splitDimArray[j][1] - splitDimArray[j][0] + 1 : 0;
-    }
-
-  }
-
-  private int[] convertToArray(List<Integer> list) {
-    int[] ints = new int[list.size()];
-    for (int i = 0; i < ints.length; i++) {
-      ints[i] = list.get(i);
-    }
-    return ints;
-  }
-
-  private int[] convertToArray(Set<Integer> set) {
-    int[] ints = new int[set.size()];
-    int i = 0;
-    for (Iterator iterator = set.iterator(); iterator.hasNext(); ) {
-      ints[i++] = (Integer) iterator.next();
-    }
-    return ints;
-  }
-
-  @Override
-  public byte[][] splitKey(byte[] key) {
-    byte[][] split = new byte[blockKeySize.length][];
-    int copyIndex = 0;
-    for (int i = 0; i < split.length; i++) {
-      split[i] = new byte[blockKeySize[i]];
-      System.arraycopy(key, copyIndex, split[i], 0, split[i].length);
-      copyIndex += blockKeySize[i];
-    }
-    return split;
-  }
-
-  @Override
-  public byte[][] generateAndSplitKey(long[] keys) {
-    return splitKey(generateKey(keys));
-  }
-
-  @Override
-  public byte[][] generateAndSplitKey(int[] keys) {
-    return splitKey(generateKey(keys));
-  }
-
-  @Override
-  public long[] getKeyArray(byte[][] key) {
-    byte[] fullKey = new byte[getKeySizeInBytes()];
-    int copyIndex = 0;
-    for (int i = 0; i < key.length; i++) {
-      System.arraycopy(key[i], 0, fullKey, copyIndex, key[i].length);
-      copyIndex += key[i].length;
-    }
-    return getKeyArray(fullKey);
-  }
-
-  @Override
-  public byte[] getKeyByteArray(byte[][] key) {
-    byte[] fullKey = new byte[getKeySizeInBytes()];
-    int copyIndex = 0;
-    for (int i = 0; i < key.length; i++) {
-      System.arraycopy(key[i], 0, fullKey, copyIndex, key[i].length);
-      copyIndex += key[i].length;
-    }
-    return fullKey;
-  }
-
-  public int[] getBlockKeySize() {
-    return blockKeySize;
-  }
-
-  @Override
-  public int getKeySizeByBlock(int[] blockIndexes) {
-    Set<Integer> selectedRanges = new HashSet<>();
-    for (int i = 0; i < blockIndexes.length; i++) {
-      int[] byteRange = byteRangesForKeys[blockIndexes[i]];
-      for (int j = byteRange[0]; j <= byteRange[1]; j++) {
-        selectedRanges.add(j);
-      }
-    }
-    return selectedRanges.size();
-  }
-
-  @Override
-  public boolean equals(Object obj) {
-    if (!(obj instanceof MultiDimKeyVarLengthVariableSplitGenerator)) {
-      return false;
-    }
-    MultiDimKeyVarLengthVariableSplitGenerator o = (MultiDimKeyVarLengthVariableSplitGenerator)obj;
-    return Arrays.equals(o.dimensionsToSplit, dimensionsToSplit) && super.equals(obj);
-  }
-
-  @Override
-  public int hashCode() {
-    return super.hashCode() + Arrays.hashCode(dimensionsToSplit);
-  }
-}
diff --git a/core/src/main/java/org/apache/carbondata/core/keygenerator/factory/KeyGeneratorFactory.java b/core/src/main/java/org/apache/carbondata/core/keygenerator/factory/KeyGeneratorFactory.java
index 83c1d58..0834906 100644
--- a/core/src/main/java/org/apache/carbondata/core/keygenerator/factory/KeyGeneratorFactory.java
+++ b/core/src/main/java/org/apache/carbondata/core/keygenerator/factory/KeyGeneratorFactory.java
@@ -39,16 +39,5 @@ public final class KeyGeneratorFactory {
     return new MultiDimKeyVarLengthGenerator(incrementedCardinality);
   }
 
-  /**
-   *
-   * @param dimCardinality : dimension cardinality
-   * @param columnSplits : No of column in each block
-   * @return keygenerator
-   */
-  public static KeyGenerator getKeyGenerator(int[] dimCardinality, int[] columnSplits) {
-    int[] dimsBitLens = CarbonUtil.getDimensionBitLength(dimCardinality, columnSplits);
-
-    return new MultiDimKeyVarLengthGenerator(dimsBitLens);
-  }
 }
 
diff --git a/core/src/main/java/org/apache/carbondata/core/keygenerator/mdkey/MultiDimKeyVarLengthGenerator.java b/core/src/main/java/org/apache/carbondata/core/keygenerator/mdkey/MultiDimKeyVarLengthGenerator.java
index 25b7fe7..5a2f2ae 100644
--- a/core/src/main/java/org/apache/carbondata/core/keygenerator/mdkey/MultiDimKeyVarLengthGenerator.java
+++ b/core/src/main/java/org/apache/carbondata/core/keygenerator/mdkey/MultiDimKeyVarLengthGenerator.java
@@ -36,45 +36,18 @@ public class MultiDimKeyVarLengthGenerator extends AbstractKeyGenerator {
   }
 
   @Override
-  public byte[] generateKey(long[] keys) {
-
-    return bits.getBytes(keys);
-  }
-
-  @Override
   public byte[] generateKey(int[] keys) {
 
     return bits.getBytes(keys);
   }
 
   @Override
-  public long[] getKeyArray(byte[] key) {
-
-    return bits.getKeyArray(key, 0);
-  }
-
-  @Override
   public long[] getKeyArray(byte[] key, int offset) {
 
     return bits.getKeyArray(key, offset);
   }
 
   @Override
-  public long getKey(byte[] key, int index) {
-
-    return bits.getKeyArray(key, 0)[index];
-  }
-
-  public int getKeySizeInBytes() {
-    return bits.getByteSize();
-  }
-
-  @Override
-  public int[] getKeyByteOffsets(int index) {
-    return byteRangesForKeys[index];
-  }
-
-  @Override
   public int getDimCount() {
 
     return bits.getDimCount();
@@ -95,9 +68,4 @@ public class MultiDimKeyVarLengthGenerator extends AbstractKeyGenerator {
     return bits.hashCode();
   }
 
-  @Override
-  public long[] getKeyArray(byte[] key, int[] maskedByteRanges) {
-    return bits.getKeyArray(key, maskedByteRanges);
-  }
-
 }
diff --git a/core/src/main/java/org/apache/carbondata/core/metadata/CarbonMetadata.java b/core/src/main/java/org/apache/carbondata/core/metadata/CarbonMetadata.java
index 88f5e8d..f25496f 100644
--- a/core/src/main/java/org/apache/carbondata/core/metadata/CarbonMetadata.java
+++ b/core/src/main/java/org/apache/carbondata/core/metadata/CarbonMetadata.java
@@ -70,17 +70,6 @@ public final class CarbonMetadata {
   }
 
   /**
-   * Below method will be used to set the carbon table
-   * Note: Use this method only in driver as clean up in Executor is not handled
-   *       if this table is added to executor
-   *
-   * @param carbonTable
-   */
-  public void addCarbonTable(CarbonTable carbonTable) {
-    tableInfoMap.put(convertToLowerCase(carbonTable.getTableUniqueName()), carbonTable);
-  }
-
-  /**
    * method load the table
    *
    * @param tableInfo
diff --git a/core/src/main/java/org/apache/carbondata/core/metadata/SegmentFileStore.java b/core/src/main/java/org/apache/carbondata/core/metadata/SegmentFileStore.java
index 3d2c9d2..b30e822 100644
--- a/core/src/main/java/org/apache/carbondata/core/metadata/SegmentFileStore.java
+++ b/core/src/main/java/org/apache/carbondata/core/metadata/SegmentFileStore.java
@@ -717,7 +717,7 @@ public class SegmentFileStore {
       // duplicates
       Set<String> blocks = new LinkedHashSet<>();
       for (DataFileFooter footer : indexInfo) {
-        blocks.add(footer.getBlockInfo().getTableBlockInfo().getFilePath());
+        blocks.add(footer.getBlockInfo().getFilePath());
       }
       indexFilesMap.put(entry.getKey(), new ArrayList<>(blocks));
       boolean added = false;
diff --git a/core/src/main/java/org/apache/carbondata/core/metadata/blocklet/BlockletInfo.java b/core/src/main/java/org/apache/carbondata/core/metadata/blocklet/BlockletInfo.java
index ac828a1..63c7040 100644
--- a/core/src/main/java/org/apache/carbondata/core/metadata/blocklet/BlockletInfo.java
+++ b/core/src/main/java/org/apache/carbondata/core/metadata/blocklet/BlockletInfo.java
@@ -24,7 +24,6 @@ import java.io.Serializable;
 import java.util.ArrayList;
 import java.util.List;
 
-import org.apache.carbondata.core.metadata.blocklet.datachunk.DataChunk;
 import org.apache.carbondata.core.metadata.blocklet.index.BlockletIndex;
 
 import org.apache.hadoop.io.Writable;
@@ -44,16 +43,6 @@ public class BlockletInfo implements Serializable, Writable {
    */
   private int numberOfRows;
 
-  /**
-   * Information about dimension chunk of all dimensions in this blocklet
-   */
-  private List<DataChunk> dimensionColumnChunk;
-
-  /**
-   * Information about measure chunk of all measures in this blocklet
-   */
-  private List<DataChunk> measureColumnChunk;
-
   private List<Long> dimensionChunkOffsets;
 
   private List<Integer> dimensionChunksLength;
@@ -102,34 +91,6 @@ public class BlockletInfo implements Serializable, Writable {
   }
 
   /**
-   * @return the dimensionColumnChunk
-   */
-  public List<DataChunk> getDimensionColumnChunk() {
-    return dimensionColumnChunk;
-  }
-
-  /**
-   * @param dimensionColumnChunk the dimensionColumnChunk to set
-   */
-  public void setDimensionColumnChunk(List<DataChunk> dimensionColumnChunk) {
-    this.dimensionColumnChunk = dimensionColumnChunk;
-  }
-
-  /**
-   * @return the measureColumnChunk
-   */
-  public List<DataChunk> getMeasureColumnChunk() {
-    return measureColumnChunk;
-  }
-
-  /**
-   * @param measureColumnChunk the measureColumnChunk to set
-   */
-  public void setMeasureColumnChunk(List<DataChunk> measureColumnChunk) {
-    this.measureColumnChunk = measureColumnChunk;
-  }
-
-  /**
    * @return the blockletIndex
    */
   public BlockletIndex getBlockletIndex() {
diff --git a/core/src/main/java/org/apache/carbondata/core/metadata/blocklet/DataFileFooter.java b/core/src/main/java/org/apache/carbondata/core/metadata/blocklet/DataFileFooter.java
index d9d788c..8bff65f 100644
--- a/core/src/main/java/org/apache/carbondata/core/metadata/blocklet/DataFileFooter.java
+++ b/core/src/main/java/org/apache/carbondata/core/metadata/blocklet/DataFileFooter.java
@@ -20,7 +20,7 @@ package org.apache.carbondata.core.metadata.blocklet;
 import java.io.Serializable;
 import java.util.List;
 
-import org.apache.carbondata.core.datastore.block.BlockInfo;
+import org.apache.carbondata.core.datastore.block.TableBlockInfo;
 import org.apache.carbondata.core.metadata.ColumnarFormatVersion;
 import org.apache.carbondata.core.metadata.blocklet.index.BlockletIndex;
 import org.apache.carbondata.core.metadata.schema.table.column.ColumnSchema;
@@ -46,11 +46,6 @@ public class DataFileFooter implements Serializable {
   private long numberOfRows;
 
   /**
-   * Segment info (will be same/repeated for all block in this segment)
-   */
-  private SegmentInfo segmentInfo;
-
-  /**
    * Information about leaf nodes of all columns in this file
    */
   private List<BlockletInfo> blockletList;
@@ -68,7 +63,7 @@ public class DataFileFooter implements Serializable {
   /**
    * to store the block info detail like file name block index and locations
    */
-  private BlockInfo blockInfo;
+  private TableBlockInfo blockInfo;
 
   /**
    * schema updated time stamp to be used for restructure scenarios
@@ -109,20 +104,6 @@ public class DataFileFooter implements Serializable {
   }
 
   /**
-   * @return the segmentInfo
-   */
-  public SegmentInfo getSegmentInfo() {
-    return segmentInfo;
-  }
-
-  /**
-   * @param segmentInfo the segmentInfo to set
-   */
-  public void setSegmentInfo(SegmentInfo segmentInfo) {
-    this.segmentInfo = segmentInfo;
-  }
-
-  /**
    * @return the List of Blocklet
    */
   public List<BlockletInfo> getBlockletList() {
@@ -167,14 +148,14 @@ public class DataFileFooter implements Serializable {
   /**
    * @return the tableBlockInfo
    */
-  public BlockInfo getBlockInfo() {
+  public TableBlockInfo getBlockInfo() {
     return blockInfo;
   }
 
   /**
    * @param tableBlockInfo the tableBlockInfo to set
    */
-  public void setBlockInfo(BlockInfo tableBlockInfo) {
+  public void setBlockInfo(TableBlockInfo tableBlockInfo) {
     this.blockInfo = tableBlockInfo;
   }
 
diff --git a/core/src/main/java/org/apache/carbondata/core/metadata/blocklet/SegmentInfo.java b/core/src/main/java/org/apache/carbondata/core/metadata/blocklet/SegmentInfo.java
deleted file mode 100644
index 245b2f6..0000000
--- a/core/src/main/java/org/apache/carbondata/core/metadata/blocklet/SegmentInfo.java
+++ /dev/null
@@ -1,52 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.carbondata.core.metadata.blocklet;
-
-import java.io.Serializable;
-
-/**
- * Class holds the information about the segment information
- */
-public class SegmentInfo implements Serializable {
-
-  /**
-   * serialization version
-   */
-  private static final long serialVersionUID = -1749874611112709431L;
-
-  /**
-   * cardinality of each columns
-   * column which is not participating in the multidimensional key cardinality will be -1;
-   */
-  private int[] columnCardinality;
-
-  /**
-   * @return the columnCardinality
-   */
-  public int[] getColumnCardinality() {
-    return columnCardinality;
-  }
-
-  /**
-   * @param columnCardinality the columnCardinality to set
-   */
-  public void setColumnCardinality(int[] columnCardinality) {
-    this.columnCardinality = columnCardinality;
-  }
-
-}
diff --git a/core/src/main/java/org/apache/carbondata/core/metadata/blocklet/datachunk/DataChunk.java b/core/src/main/java/org/apache/carbondata/core/metadata/blocklet/datachunk/DataChunk.java
deleted file mode 100644
index f85ddef..0000000
--- a/core/src/main/java/org/apache/carbondata/core/metadata/blocklet/datachunk/DataChunk.java
+++ /dev/null
@@ -1,228 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.carbondata.core.metadata.blocklet.datachunk;
-
-import java.io.Serializable;
-import java.util.BitSet;
-import java.util.List;
-
-import org.apache.carbondata.core.metadata.ValueEncoderMeta;
-import org.apache.carbondata.core.metadata.encoder.Encoding;
-
-/**
- * Class holds the information about the data chunk metadata
- */
-public class DataChunk implements Serializable {
-
-  /**
-   * serialization version
-   */
-  private static final long serialVersionUID = 1L;
-
-  /**
-   * whether this chunk is a row chunk or column chunk
-   */
-  private boolean isRowMajor;
-
-  /**
-   * Offset of data page
-   */
-  private long dataPageOffset;
-
-  /**
-   * length of data page
-   */
-  private int dataPageLength;
-
-  /**
-   * information about presence of values in each row of this column chunk
-   */
-  private BitSet nullValueIndexForColumn;
-
-  /**
-   * offset of row id page, only if encoded using inverted index
-   */
-  private long rowIdPageOffset;
-
-  /**
-   * length of row id page, only if encoded using inverted index
-   */
-  private int rowIdPageLength;
-
-  /**
-   * offset of rle page, only if RLE coded.
-   */
-  private long rlePageOffset;
-
-  /**
-   * length of rle page, only if RLE coded.
-   */
-  private int rlePageLength;
-
-  /**
-   * The List of encoders overriden at node level
-   */
-  private List<Encoding> encodingList;
-
-  /**
-   * value encoder meta which will holds the information
-   * about max, min, decimal length, type
-   */
-  private List<ValueEncoderMeta> valueEncoderMetaList;
-
-  /**
-   * @return the isRowMajor
-   */
-  public boolean isRowMajor() {
-    return isRowMajor;
-  }
-
-  /**
-   * @param isRowMajor the isRowMajor to set
-   */
-  public void setRowMajor(boolean isRowMajor) {
-    this.isRowMajor = isRowMajor;
-  }
-
-  /**
-   * @return the dataPageOffset
-   */
-  public long getDataPageOffset() {
-    return dataPageOffset;
-  }
-
-  /**
-   * @param dataPageOffset the dataPageOffset to set
-   */
-  public void setDataPageOffset(long dataPageOffset) {
-    this.dataPageOffset = dataPageOffset;
-  }
-
-  /**
-   * @return the dataPageLength
-   */
-  public int getDataPageLength() {
-    return dataPageLength;
-  }
-
-  /**
-   * @param dataPageLength the dataPageLength to set
-   */
-  public void setDataPageLength(int dataPageLength) {
-    this.dataPageLength = dataPageLength;
-  }
-
-  /**
-   * @return the nullValueIndexForColumn
-   */
-  public BitSet getNullValueIndexForColumn() {
-    return nullValueIndexForColumn;
-  }
-
-  /**
-   * @param nullValueIndexForColumn the nullValueIndexForColumn to set
-   */
-  public void setNullValueIndexForColumn(BitSet nullValueIndexForColumn) {
-    this.nullValueIndexForColumn = nullValueIndexForColumn;
-  }
-
-  /**
-   * @return the rowIdPageOffset
-   */
-  public long getRowIdPageOffset() {
-    return rowIdPageOffset;
-  }
-
-  /**
-   * @param rowIdPageOffset the rowIdPageOffset to set
-   */
-  public void setRowIdPageOffset(long rowIdPageOffset) {
-    this.rowIdPageOffset = rowIdPageOffset;
-  }
-
-  /**
-   * @return the rowIdPageLength
-   */
-  public int getRowIdPageLength() {
-    return rowIdPageLength;
-  }
-
-  /**
-   * @param rowIdPageLength the rowIdPageLength to set
-   */
-  public void setRowIdPageLength(int rowIdPageLength) {
-    this.rowIdPageLength = rowIdPageLength;
-  }
-
-  /**
-   * @return the rlePageOffset
-   */
-  public long getRlePageOffset() {
-    return rlePageOffset;
-  }
-
-  /**
-   * @param rlePageOffset the rlePageOffset to set
-   */
-  public void setRlePageOffset(long rlePageOffset) {
-    this.rlePageOffset = rlePageOffset;
-  }
-
-  /**
-   * @return the rlePageLength
-   */
-  public int getRlePageLength() {
-    return rlePageLength;
-  }
-
-  /**
-   * @param rlePageLength the rlePageLength to set
-   */
-  public void setRlePageLength(int rlePageLength) {
-    this.rlePageLength = rlePageLength;
-  }
-
-  /**
-   * @return the encoderList
-   */
-  public List<Encoding> getEncodingList() {
-    return encodingList;
-  }
-
-  /**
-   * @param encodingList the encoderList to set
-   */
-  public void setEncodingList(List<Encoding> encodingList) {
-    this.encodingList = encodingList;
-  }
-
-  /**
-   * @return the valueEncoderMeta
-   */
-  public List<ValueEncoderMeta> getValueEncoderMeta() {
-    return valueEncoderMetaList;
-  }
-
-  /**
-   * @param valueEncoderMetaList the valueEncoderMeta to set
-   */
-  public void setValueEncoderMeta(List<ValueEncoderMeta> valueEncoderMetaList) {
-    this.valueEncoderMetaList = valueEncoderMetaList;
-  }
-
-}
diff --git a/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/CarbonTable.java b/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/CarbonTable.java
index ba1208e..c379be7 100644
--- a/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/CarbonTable.java
+++ b/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/CarbonTable.java
@@ -353,9 +353,10 @@ public class CarbonTable implements Serializable, Writable {
       ColumnSchema columnSchema = listOfColumns.get(i);
       if (columnSchema.isDimensionColumn()) {
         if (columnSchema.getNumberOfChild() > 0) {
+          ++complexTypeOrdinal;
           CarbonDimension complexDimension =
-              new CarbonDimension(columnSchema, dimensionOrdinal++, columnSchema.getSchemaOrdinal(),
-                  -1, ++complexTypeOrdinal);
+              new CarbonDimension(columnSchema, dimensionOrdinal++, -1,
+                  columnSchema.getSchemaOrdinal());
           complexDimension.initializeChildDimensionsList(columnSchema.getNumberOfChild());
           allDimensions.add(complexDimension);
           dimensionOrdinal =
@@ -368,15 +369,15 @@ public class CarbonTable implements Serializable, Writable {
             this.numberOfSortColumns++;
           }
           if (columnSchema.getDataType() != DataTypes.DATE) {
-            CarbonDimension dimension = new CarbonDimension(columnSchema, dimensionOrdinal++,
-                columnSchema.getSchemaOrdinal(), -1, -1);
+            CarbonDimension dimension = new CarbonDimension(columnSchema, dimensionOrdinal++, -1,
+                columnSchema.getSchemaOrdinal());
             if (!columnSchema.isInvisible() && columnSchema.isSortColumn()) {
               this.numberOfNoDictSortColumns++;
             }
             allDimensions.add(dimension);
           } else if (columnSchema.getDataType() == DataTypes.DATE) {
             CarbonDimension dimension = new CarbonDimension(columnSchema, dimensionOrdinal++,
-                columnSchema.getSchemaOrdinal(), keyOrdinal++, -1);
+                keyOrdinal++, columnSchema.getSchemaOrdinal());
             allDimensions.add(dimension);
           }
         }
@@ -422,8 +423,8 @@ public class CarbonTable implements Serializable, Writable {
       if (columnSchema.isDimensionColumn()) {
         if (columnSchema.getNumberOfChild() > 0) {
           CarbonDimension complexDimension =
-              new CarbonDimension(columnSchema, dimensionOrdinal++, columnSchema.getSchemaOrdinal(),
-                  -1, -1);
+              new CarbonDimension(columnSchema, dimensionOrdinal++, -1,
+                  columnSchema.getSchemaOrdinal());
           complexDimension.initializeChildDimensionsList(columnSchema.getNumberOfChild());
           parentDimension.getListOfChildDimensions().add(complexDimension);
           dimensionOrdinal =
@@ -431,8 +432,8 @@ public class CarbonTable implements Serializable, Writable {
                   listOfColumns, complexDimension);
         } else {
           CarbonDimension carbonDimension =
-              new CarbonDimension(columnSchema, dimensionOrdinal++, columnSchema.getSchemaOrdinal(),
-                  -1, -1);
+              new CarbonDimension(columnSchema, dimensionOrdinal++, -1,
+                  columnSchema.getSchemaOrdinal());
           parentDimension.getListOfChildDimensions().add(carbonDimension);
         }
       }
diff --git a/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/column/CarbonDimension.java b/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/column/CarbonDimension.java
index d220a28..c93216b 100644
--- a/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/column/CarbonDimension.java
+++ b/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/column/CarbonDimension.java
@@ -40,25 +40,18 @@ public class CarbonDimension extends CarbonColumn {
   private int keyOrdinal;
 
   /**
-   * to store complex type dimension ordinal
-   */
-  private int complexTypeOrdinal;
-
-  /**
    * Save the Parent Dimension of the complex Parent Column.
    */
   private CarbonDimension complexParentDimension = null;
 
-  public CarbonDimension(ColumnSchema columnSchema, int ordinal, int keyOrdinal,
-          int complexTypeOrdinal) {
-       this(columnSchema, ordinal, 0, keyOrdinal, complexTypeOrdinal);
+  public CarbonDimension(ColumnSchema columnSchema, int ordinal, int keyOrdinal) {
+       this(columnSchema, ordinal, keyOrdinal, 0);
   }
 
-  public CarbonDimension(ColumnSchema columnSchema, int ordinal, int schemaOrdinal, int keyOrdinal,
-      int complexTypeOrdinal) {
+  public CarbonDimension(ColumnSchema columnSchema, int ordinal, int keyOrdinal,
+      int schemaOrdinal) {
     super(columnSchema, ordinal, schemaOrdinal);
     this.keyOrdinal = keyOrdinal;
-    this.complexTypeOrdinal = complexTypeOrdinal;
   }
 
   /**
@@ -93,15 +86,7 @@ public class CarbonDimension extends CarbonColumn {
     return keyOrdinal;
   }
 
-  /**
-   * @return the complexTypeOrdinal
-   */
-  public int getComplexTypeOrdinal() {
-    return complexTypeOrdinal;
-  }
-
   public void setComplexTypeOridnal(int complexTypeOrdinal) {
-    this.complexTypeOrdinal = complexTypeOrdinal;
   }
 
   /**
diff --git a/core/src/main/java/org/apache/carbondata/core/scan/collector/impl/RestructureBasedRawResultCollector.java b/core/src/main/java/org/apache/carbondata/core/scan/collector/impl/RestructureBasedRawResultCollector.java
index f055ad8..a77f794 100644
--- a/core/src/main/java/org/apache/carbondata/core/scan/collector/impl/RestructureBasedRawResultCollector.java
+++ b/core/src/main/java/org/apache/carbondata/core/scan/collector/impl/RestructureBasedRawResultCollector.java
@@ -21,12 +21,8 @@ import java.util.ArrayList;
 import java.util.List;
 
 import org.apache.carbondata.core.constants.CarbonCommonConstants;
-import org.apache.carbondata.core.datastore.block.SegmentProperties;
-import org.apache.carbondata.core.keygenerator.KeyGenerator;
-import org.apache.carbondata.core.keygenerator.mdkey.MultiDimKeyVarLengthGenerator;
 import org.apache.carbondata.core.metadata.datatype.DataTypes;
 import org.apache.carbondata.core.metadata.encoder.Encoding;
-import org.apache.carbondata.core.metadata.schema.table.column.CarbonDimension;
 import org.apache.carbondata.core.scan.executor.infos.BlockExecutionInfo;
 import org.apache.carbondata.core.scan.model.ProjectionDimension;
 import org.apache.carbondata.core.scan.model.ProjectionMeasure;
@@ -34,99 +30,16 @@ import org.apache.carbondata.core.scan.result.BlockletScannedResult;
 import org.apache.carbondata.core.scan.wrappers.ByteArrayWrapper;
 import org.apache.carbondata.core.stats.QueryStatistic;
 import org.apache.carbondata.core.stats.QueryStatisticsConstants;
-import org.apache.carbondata.core.util.CarbonUtil;
+import org.apache.carbondata.core.util.ByteUtil;
 import org.apache.carbondata.core.util.DataTypeUtil;
 
-import org.apache.commons.lang3.ArrayUtils;
-
 /**
  * It is not a collector it is just a scanned result holder.
  */
 public class RestructureBasedRawResultCollector extends RawBasedResultCollector {
 
-  /**
-   * Key generator which will form the mdKey according to latest schema
-   */
-  private KeyGenerator restructuredKeyGenerator;
-
-  /**
-   * Key generator for uncompressing current block values
-   */
-  private KeyGenerator updatedCurrentBlockKeyGenerator;
-
   public RestructureBasedRawResultCollector(BlockExecutionInfo blockExecutionInfos) {
     super(blockExecutionInfos);
-    initRestructuredKeyGenerator();
-    initCurrentBlockKeyGenerator();
-  }
-
-  /**
-   * This method will create a new key generator for generating mdKey according to latest schema
-   */
-  private void initRestructuredKeyGenerator() {
-    SegmentProperties segmentProperties =
-        executionInfo.getDataBlock().getSegmentProperties();
-    ProjectionDimension[] queryDimensions = executionInfo.getActualQueryDimensions();
-    List<Integer> updatedColumnCardinality = new ArrayList<>(queryDimensions.length);
-    List<Integer> updatedDimensionPartitioner = new ArrayList<>(queryDimensions.length);
-    int[] dictionaryColumnBlockIndex = executionInfo.getDictionaryColumnChunkIndex();
-    int dimCounterInCurrentBlock = 0;
-    for (int i = 0; i < queryDimensions.length; i++) {
-      if (queryDimensions[i].getDimension().getDataType() == DataTypes.DATE) {
-        if (executionInfo.getDimensionInfo().getDimensionExists()[i]) {
-          // get the dictionary key ordinal as column cardinality in segment properties
-          // will only be for dictionary encoded columns
-          CarbonDimension currentBlockDimension = segmentProperties.getDimensions()
-              .get(dictionaryColumnBlockIndex[dimCounterInCurrentBlock]);
-          updatedColumnCardinality.add(
-              segmentProperties.getDimColumnsCardinality()[currentBlockDimension.getKeyOrdinal()]);
-          updatedDimensionPartitioner.add(
-              segmentProperties.getDimensionPartitions()[currentBlockDimension.getKeyOrdinal()]);
-          dimCounterInCurrentBlock++;
-        } else {
-          // partitioner index will be 1 every column will be in columnar format
-          updatedDimensionPartitioner.add(1);
-          // for direct dictionary 4 bytes need to be allocated
-          updatedColumnCardinality.add(Integer.MAX_VALUE);
-        }
-      }
-    }
-    if (!updatedColumnCardinality.isEmpty()) {
-      int[] latestColumnCardinality = ArrayUtils.toPrimitive(
-          updatedColumnCardinality.toArray(new Integer[0]));
-      int[] latestColumnPartitioner = ArrayUtils.toPrimitive(
-          updatedDimensionPartitioner.toArray(new Integer[0]));
-      int[] dimensionBitLength =
-          CarbonUtil.getDimensionBitLength(latestColumnCardinality, latestColumnPartitioner);
-      restructuredKeyGenerator = new MultiDimKeyVarLengthGenerator(dimensionBitLength);
-    }
-  }
-
-  /**
-   * This method will initialize the block key generator for the current block based on the
-   * dictionary columns present in the current block
-   */
-  private void initCurrentBlockKeyGenerator() {
-    SegmentProperties segmentProperties =
-        executionInfo.getDataBlock().getSegmentProperties();
-    int[] dictionaryColumnBlockIndex = executionInfo.getDictionaryColumnChunkIndex();
-    int[] updatedColumnCardinality = new int[dictionaryColumnBlockIndex.length];
-    int[] updatedDimensionPartitioner = new int[dictionaryColumnBlockIndex.length];
-    for (int i = 0; i < dictionaryColumnBlockIndex.length; i++) {
-      // get the dictionary key ordinal as column cardinality in segment properties
-      // will only be for dictionary encoded columns
-      CarbonDimension currentBlockDimension =
-          segmentProperties.getDimensions().get(dictionaryColumnBlockIndex[i]);
-      updatedColumnCardinality[i] =
-          segmentProperties.getDimColumnsCardinality()[currentBlockDimension.getKeyOrdinal()];
-      updatedDimensionPartitioner[i] =
-          segmentProperties.getDimensionPartitions()[currentBlockDimension.getKeyOrdinal()];
-    }
-    if (dictionaryColumnBlockIndex.length > 0) {
-      int[] dimensionBitLength =
-          CarbonUtil.getDimensionBitLength(updatedColumnCardinality, updatedDimensionPartitioner);
-      updatedCurrentBlockKeyGenerator = new MultiDimKeyVarLengthGenerator(dimensionBitLength);
-    }
   }
 
   /**
@@ -167,8 +80,8 @@ public class RestructureBasedRawResultCollector extends RawBasedResultCollector
       ProjectionDimension[] actualQueryDimensions = executionInfo.getActualQueryDimensions();
       int newKeyArrayLength = dimensionInfo.getNewDictionaryColumnCount();
       long[] keyArray = null;
-      if (null != updatedCurrentBlockKeyGenerator) {
-        keyArray = updatedCurrentBlockKeyGenerator.getKeyArray(dictKeyArray);
+      if (executionInfo.getDataBlock().getSegmentProperties().getNumberOfDictDimensions() > 0) {
+        keyArray = ByteUtil.convertBytesToLongArray(dictKeyArray);
         newKeyArrayLength += keyArray.length;
       }
       long[] keyArrayWithNewAddedColumns = new long[newKeyArrayLength];
@@ -192,7 +105,7 @@ public class RestructureBasedRawResultCollector extends RawBasedResultCollector
           }
         }
       }
-      dictKeyArray = restructuredKeyGenerator.generateKey(keyArrayWithNewAddedColumns);
+      dictKeyArray = ByteUtil.convertLongArrayToBytes(keyArrayWithNewAddedColumns);
       byteArrayWrapper.setDictionaryKey(dictKeyArray);
     }
   }
diff --git a/core/src/main/java/org/apache/carbondata/core/scan/collector/impl/RestructureBasedVectorResultCollector.java b/core/src/main/java/org/apache/carbondata/core/scan/collector/impl/RestructureBasedVectorResultCollector.java
index 0274627..7d70517 100644
--- a/core/src/main/java/org/apache/carbondata/core/scan/collector/impl/RestructureBasedVectorResultCollector.java
+++ b/core/src/main/java/org/apache/carbondata/core/scan/collector/impl/RestructureBasedVectorResultCollector.java
@@ -153,18 +153,6 @@ public class RestructureBasedVectorResultCollector extends DictionaryBasedVector
   }
 
   /**
-   * This method will fill the dictionary column data
-   *
-   * @param vector
-   * @param columnVectorInfo
-   * @param defaultValue
-   */
-  private void fillDictionaryData(CarbonColumnVector vector, ColumnVectorInfo columnVectorInfo,
-      Object defaultValue) {
-    vector.putInts(columnVectorInfo.vectorOffset, columnVectorInfo.size, (int) defaultValue);
-  }
-
-  /**
    * This method will fill the direct dictionary column data
    *
    * @param vector
diff --git a/core/src/main/java/org/apache/carbondata/core/scan/complextypes/ArrayQueryType.java b/core/src/main/java/org/apache/carbondata/core/scan/complextypes/ArrayQueryType.java
index 23aee6c..71d3b1e 100644
--- a/core/src/main/java/org/apache/carbondata/core/scan/complextypes/ArrayQueryType.java
+++ b/core/src/main/java/org/apache/carbondata/core/scan/complextypes/ArrayQueryType.java
@@ -33,8 +33,8 @@ public class ArrayQueryType extends ComplexQueryType implements GenericQueryType
 
   private GenericQueryType children;
 
-  public ArrayQueryType(String name, String parentName, int blockIndex) {
-    super(name, parentName, blockIndex);
+  public ArrayQueryType(String name, String parentName, int columnIndex) {
+    super(name, parentName, columnIndex);
   }
 
   @Override
diff --git a/core/src/main/java/org/apache/carbondata/core/scan/complextypes/ComplexQueryType.java b/core/src/main/java/org/apache/carbondata/core/scan/complextypes/ComplexQueryType.java
index 8613664..d0cc8ae 100644
--- a/core/src/main/java/org/apache/carbondata/core/scan/complextypes/ComplexQueryType.java
+++ b/core/src/main/java/org/apache/carbondata/core/scan/complextypes/ComplexQueryType.java
@@ -28,12 +28,12 @@ public class ComplexQueryType {
 
   protected String parentName;
 
-  protected int blockIndex;
+  protected int columnIndex;
 
-  public ComplexQueryType(String name, String parentName, int blockIndex) {
+  public ComplexQueryType(String name, String parentName, int columnIndex) {
     this.name = name;
     this.parentName = parentName;
-    this.blockIndex = blockIndex;
+    this.columnIndex = columnIndex;
   }
 
   /**
@@ -43,7 +43,7 @@ public class ComplexQueryType {
   protected byte[] copyBlockDataChunk(DimensionRawColumnChunk[] rawColumnChunks,
       DimensionColumnPage[][] dimensionColumnPages, int rowNumber, int pageNumber) {
     byte[] data =
-        getDecodedDimensionPage(dimensionColumnPages, rawColumnChunks[blockIndex], pageNumber)
+        getDecodedDimensionPage(dimensionColumnPages, rawColumnChunks[columnIndex], pageNumber)
             .getChunkData(rowNumber);
     byte[] output = new byte[data.length];
     System.arraycopy(data, 0, output, 0, output.length);
@@ -54,17 +54,17 @@ public class ComplexQueryType {
    * This method will read the block data chunk from the respective block
    */
   protected void readBlockDataChunk(RawBlockletColumnChunks blockChunkHolder) throws IOException {
-    if (null == blockChunkHolder.getDimensionRawColumnChunks()[blockIndex]) {
-      blockChunkHolder.getDimensionRawColumnChunks()[blockIndex] = blockChunkHolder.getDataBlock()
-          .readDimensionChunk(blockChunkHolder.getFileReader(), blockIndex);
+    if (null == blockChunkHolder.getDimensionRawColumnChunks()[columnIndex]) {
+      blockChunkHolder.getDimensionRawColumnChunks()[columnIndex] = blockChunkHolder.getDataBlock()
+          .readDimensionChunk(blockChunkHolder.getFileReader(), columnIndex);
     }
   }
 
   private DimensionColumnPage getDecodedDimensionPage(DimensionColumnPage[][] dimensionColumnPages,
       DimensionRawColumnChunk dimensionRawColumnChunk, int pageNumber) {
-    if (dimensionColumnPages == null || null == dimensionColumnPages[blockIndex]) {
+    if (dimensionColumnPages == null || null == dimensionColumnPages[columnIndex]) {
       return dimensionRawColumnChunk.decodeColumnPage(pageNumber);
     }
-    return dimensionColumnPages[blockIndex][pageNumber];
+    return dimensionColumnPages[columnIndex][pageNumber];
   }
 }
diff --git a/core/src/main/java/org/apache/carbondata/core/scan/complextypes/MapQueryType.java b/core/src/main/java/org/apache/carbondata/core/scan/complextypes/MapQueryType.java
index 82798e4..212b102 100644
--- a/core/src/main/java/org/apache/carbondata/core/scan/complextypes/MapQueryType.java
+++ b/core/src/main/java/org/apache/carbondata/core/scan/complextypes/MapQueryType.java
@@ -26,8 +26,8 @@ import org.apache.carbondata.core.util.DataTypeUtil;
  */
 public class MapQueryType extends ArrayQueryType {
 
-  public MapQueryType(String name, String parentName, int blockIndex) {
-    super(name, parentName, blockIndex);
+  public MapQueryType(String name, String parentName, int columnIndex) {
+    super(name, parentName, columnIndex);
   }
 
   /**
diff --git a/core/src/main/java/org/apache/carbondata/core/scan/complextypes/PrimitiveQueryType.java b/core/src/main/java/org/apache/carbondata/core/scan/complextypes/PrimitiveQueryType.java
index a8789f3..5657418 100644
--- a/core/src/main/java/org/apache/carbondata/core/scan/complextypes/PrimitiveQueryType.java
+++ b/core/src/main/java/org/apache/carbondata/core/scan/complextypes/PrimitiveQueryType.java
@@ -27,7 +27,6 @@ import org.apache.carbondata.core.datastore.chunk.DimensionColumnPage;
 import org.apache.carbondata.core.datastore.chunk.impl.DimensionRawColumnChunk;
 import org.apache.carbondata.core.keygenerator.directdictionary.DirectDictionaryGenerator;
 import org.apache.carbondata.core.keygenerator.directdictionary.DirectDictionaryKeyGeneratorFactory;
-import org.apache.carbondata.core.keygenerator.mdkey.Bits;
 import org.apache.carbondata.core.metadata.datatype.DataType;
 import org.apache.carbondata.core.metadata.datatype.DataTypes;
 import org.apache.carbondata.core.metadata.schema.table.column.CarbonDimension;
@@ -41,17 +40,14 @@ public class PrimitiveQueryType extends ComplexQueryType implements GenericQuery
   private String name;
   private String parentName;
 
-  private int keySize;
-
   private org.apache.carbondata.core.metadata.datatype.DataType dataType;
 
   private boolean isDirectDictionary;
 
-  public PrimitiveQueryType(String name, String parentName, int blockIndex, DataType dataType,
-      int keySize, boolean isDirectDictionary) {
-    super(name, parentName, blockIndex);
+  public PrimitiveQueryType(String name, String parentName, int columnIndex, DataType dataType,
+      boolean isDirectDictionary) {
+    super(name, parentName, columnIndex);
     this.dataType = dataType;
-    this.keySize = keySize;
     this.name = name;
     this.parentName = parentName;
     this.isDirectDictionary = isDirectDictionary;
@@ -140,14 +136,10 @@ public class PrimitiveQueryType extends ComplexQueryType implements GenericQuery
   private Object getDataObject(ByteBuffer dataBuffer, int size) {
     Object actualData;
     if (isDirectDictionary) {
-      // Direct Dictionary Column
-      byte[] data = new byte[keySize];
+      // Direct Dictionary Column, only for DATE type
+      byte[] data = new byte[4];
       dataBuffer.get(data);
-      Bits bit = new Bits(new int[] { keySize * 8 });
-      int surrgateValue = (int) bit.getKeyArray(data, 0)[0];
-      DirectDictionaryGenerator directDictionaryGenerator =
-          DirectDictionaryKeyGeneratorFactory.getDirectDictionaryGenerator(dataType);
-      actualData = directDictionaryGenerator.getValueFromSurrogate(surrgateValue);
+      actualData = ByteUtil.convertBytesToInt(data);
     } else {
       if (size == -1) {
         if (DataTypeUtil.isByteArrayComplexChildColumn(dataType)) {
@@ -155,7 +147,6 @@ public class PrimitiveQueryType extends ComplexQueryType implements GenericQuery
         } else {
           size = dataBuffer.getShort();
         }
-
       }
       byte[] value = new byte[size];
       dataBuffer.get(value, 0, size);
diff --git a/core/src/main/java/org/apache/carbondata/core/scan/complextypes/StructQueryType.java b/core/src/main/java/org/apache/carbondata/core/scan/complextypes/StructQueryType.java
index 372622d..1a05622 100644
--- a/core/src/main/java/org/apache/carbondata/core/scan/complextypes/StructQueryType.java
+++ b/core/src/main/java/org/apache/carbondata/core/scan/complextypes/StructQueryType.java
@@ -37,8 +37,8 @@ public class StructQueryType extends ComplexQueryType implements GenericQueryTyp
   private String name;
   private String parentName;
 
-  public StructQueryType(String name, String parentName, int blockIndex) {
-    super(name, parentName, blockIndex);
+  public StructQueryType(String name, String parentName, int columnIndex) {
+    super(name, parentName, columnIndex);
     this.name = name;
     this.parentName = parentName;
   }
diff --git a/core/src/main/java/org/apache/carbondata/core/scan/executor/impl/AbstractQueryExecutor.java b/core/src/main/java/org/apache/carbondata/core/scan/executor/impl/AbstractQueryExecutor.java
index 6d591de..0c49199 100644
--- a/core/src/main/java/org/apache/carbondata/core/scan/executor/impl/AbstractQueryExecutor.java
+++ b/core/src/main/java/org/apache/carbondata/core/scan/executor/impl/AbstractQueryExecutor.java
@@ -36,7 +36,6 @@ import org.apache.carbondata.core.constants.CarbonCommonConstants;
 import org.apache.carbondata.core.constants.CarbonV3DataFormatConstants;
 import org.apache.carbondata.core.datamap.DataMapFilter;
 import org.apache.carbondata.core.datamap.Segment;
-import org.apache.carbondata.core.datastore.IndexKey;
 import org.apache.carbondata.core.datastore.ReusableDataBuffer;
 import org.apache.carbondata.core.datastore.block.AbstractIndex;
 import org.apache.carbondata.core.datastore.block.SegmentProperties;
@@ -210,29 +209,15 @@ public abstract class AbstractQueryExecutor<E> implements QueryExecutor<E> {
           blockInfo.setDetailInfo(blockletDetailInfo);
         }
         if (null == segmentProperties) {
-          segmentProperties = new SegmentProperties(fileFooter.getColumnInTable(),
-              fileFooter.getSegmentInfo().getColumnCardinality());
+          segmentProperties = new SegmentProperties(fileFooter.getColumnInTable());
           createFilterExpression(queryModel, segmentProperties);
           updateColumns(queryModel, fileFooter.getColumnInTable(), blockInfo.getFilePath());
           filePathToSegmentPropertiesMap.put(blockInfo.getFilePath(), segmentProperties);
         }
-        if (blockInfo.isLegacyStore()) {
-          LOGGER.warn("Skipping Direct Vector Filling as it is not Supported "
-              + "for Legacy store prior to V3 store");
-          queryModel.setDirectVectorFill(false);
-          // Skip minmax based pruning for measure column in case of legacy store
-          boolean[] minMaxFlag = new boolean[segmentProperties.getColumnsValueSize().length];
-          FilterUtil.setMinMaxFlagForLegacyStore(minMaxFlag, segmentProperties);
-          for (BlockletInfo blockletInfo : fileFooter.getBlockletList()) {
-            blockletInfo.getBlockletIndex().getMinMaxIndex().setIsMinMaxSet(minMaxFlag);
-          }
-        }
-        readAndFillBlockletInfo(tableBlockInfos, blockInfo,
-            blockletDetailInfo, fileFooter, segmentProperties);
+        readAndFillBlockletInfo(tableBlockInfos, blockInfo, blockletDetailInfo, fileFooter);
       } else {
         if (null == segmentProperties) {
-          segmentProperties = new SegmentProperties(blockInfo.getDetailInfo().getColumnSchemas(),
-              blockInfo.getDetailInfo().getDimLens());
+          segmentProperties = new SegmentProperties(blockInfo.getDetailInfo().getColumnSchemas());
           createFilterExpression(queryModel, segmentProperties);
           updateColumns(queryModel, blockInfo.getDetailInfo().getColumnSchemas(),
               blockInfo.getFilePath());
@@ -301,8 +286,8 @@ public abstract class AbstractQueryExecutor<E> implements QueryExecutor<E> {
       if (index > -1) {
         if (columnsInTable.get(index).isDimensionColumn()) {
           ProjectionDimension dimension = new ProjectionDimension(
-              new CarbonDimension(columnsInTable.get(index), measure.getMeasure().getOrdinal(),
-                  measure.getMeasure().getSchemaOrdinal(), -1, -1));
+              new CarbonDimension(columnsInTable.get(index), measure.getMeasure().getOrdinal(), -1,
+                  measure.getMeasure().getSchemaOrdinal()));
           dimension.setOrdinal(measure.getOrdinal());
           updatedDims.add(dimension);
         } else {
@@ -333,8 +318,7 @@ public abstract class AbstractQueryExecutor<E> implements QueryExecutor<E> {
    * Read the file footer of block file and get the blocklets to query
    */
   private void readAndFillBlockletInfo(List<TableBlockInfo> tableBlockInfos,
-      TableBlockInfo blockInfo, BlockletDetailInfo blockletDetailInfo, DataFileFooter fileFooter,
-      SegmentProperties segmentProperties) {
+      TableBlockInfo blockInfo, BlockletDetailInfo blockletDetailInfo, DataFileFooter fileFooter) {
     List<BlockletInfo> blockletList = fileFooter.getBlockletList();
     // cases when blockletID will be -1
     // 1. In case of legacy store
@@ -345,21 +329,21 @@ public abstract class AbstractQueryExecutor<E> implements QueryExecutor<E> {
     if (blockletDetailInfo.getBlockletId() != -1) {
       // fill the info only for given blockletId in detailInfo
       BlockletInfo blockletInfo = blockletList.get(blockletDetailInfo.getBlockletId());
-      fillBlockletInfoToTableBlock(tableBlockInfos, blockInfo, blockletDetailInfo, fileFooter,
-          blockletInfo, blockletDetailInfo.getBlockletId(), segmentProperties);
+      fillBlockletInfoToTableBlock(tableBlockInfos, blockInfo, fileFooter,
+          blockletInfo, blockletDetailInfo.getBlockletId());
     } else {
       short count = 0;
       for (BlockletInfo blockletInfo : blockletList) {
-        fillBlockletInfoToTableBlock(tableBlockInfos, blockInfo, blockletDetailInfo, fileFooter,
-            blockletInfo, count, segmentProperties);
+        fillBlockletInfoToTableBlock(tableBlockInfos, blockInfo, fileFooter,
+            blockletInfo, count);
         count++;
       }
     }
   }
 
   private void fillBlockletInfoToTableBlock(List<TableBlockInfo> tableBlockInfos,
-      TableBlockInfo blockInfo, BlockletDetailInfo blockletDetailInfo, DataFileFooter fileFooter,
-      BlockletInfo blockletInfo, short blockletId, SegmentProperties segmentProperties) {
+      TableBlockInfo blockInfo, DataFileFooter fileFooter, BlockletInfo blockletInfo,
+      short blockletId) {
     TableBlockInfo info = blockInfo.copy();
     BlockletDetailInfo detailInfo = info.getDetailInfo();
     // set column schema details
@@ -367,9 +351,6 @@ public abstract class AbstractQueryExecutor<E> implements QueryExecutor<E> {
     detailInfo.setRowCount(blockletInfo.getNumberOfRows());
     byte[][] maxValues = blockletInfo.getBlockletIndex().getMinMaxIndex().getMaxValues();
     byte[][] minValues = blockletInfo.getBlockletIndex().getMinMaxIndex().getMinValues();
-    if (blockInfo.isLegacyStore()) {
-      info.setDataBlockFromOldStore(true);
-    }
     blockletInfo.getBlockletIndex().getMinMaxIndex().setMaxValues(maxValues);
     blockletInfo.getBlockletIndex().getMinMaxIndex().setMinValues(minValues);
     detailInfo.setBlockletInfo(blockletInfo);
@@ -393,9 +374,11 @@ public abstract class AbstractQueryExecutor<E> implements QueryExecutor<E> {
       BlockletDataRefNode dataRefNode =
           (BlockletDataRefNode) abstractIndex.getDataRefNode();
       final BlockExecutionInfo blockExecutionInfoForBlock =
-          getBlockExecutionInfoForBlock(queryModel, abstractIndex,
-              dataRefNode.getBlockInfos().get(0).getBlockletInfos().getStartBlockletNumber(),
-              dataRefNode.numberOfNodes(), dataRefNode.getBlockInfos().get(0).getFilePath(),
+          getBlockExecutionInfoForBlock(
+              queryModel,
+              abstractIndex,
+              dataRefNode.numberOfNodes(),
+              dataRefNode.getBlockInfos().get(0).getFilePath(),
               dataRefNode.getBlockInfos().get(0).getDeletedDeltaFilePath(),
               dataRefNode.getBlockInfos().get(0).getSegment());
       if (null == dimensionReusableDataBuffers || null == measureReusableDataBuffers) {
@@ -431,7 +414,7 @@ public abstract class AbstractQueryExecutor<E> implements QueryExecutor<E> {
    * @return block execution info
    */
   private BlockExecutionInfo getBlockExecutionInfoForBlock(QueryModel queryModel,
-      AbstractIndex blockIndex, int startBlockletIndex, int numberOfBlockletToScan, String filePath,
+      AbstractIndex blockIndex, int numberOfBlockletToScan, String filePath,
       String[] deleteDeltaFiles, Segment segment) {
     BlockExecutionInfo blockExecutionInfo = new BlockExecutionInfo();
     SegmentProperties segmentProperties = blockIndex.getSegmentProperties();
@@ -456,10 +439,10 @@ public abstract class AbstractQueryExecutor<E> implements QueryExecutor<E> {
       blockExecutionInfo.setBlockId(CarbonTablePath.getShortBlockId(blockId));
     }
     blockExecutionInfo.setDeleteDeltaFilePath(deleteDeltaFiles);
-    blockExecutionInfo.setStartBlockletIndex(startBlockletIndex);
+    blockExecutionInfo.setStartBlockletIndex(0);
     blockExecutionInfo.setNumberOfBlockletToScan(numberOfBlockletToScan);
     blockExecutionInfo.setProjectionDimensions(projectDimensions
-        .toArray(new ProjectionDimension[projectDimensions.size()]));
+        .toArray(new ProjectionDimension[0]));
     // get measures present in the current block
     List<ProjectionMeasure> projectionMeasures = RestructureUtil
         .createMeasureInfoAndGetCurrentBlockQueryMeasures(blockExecutionInfo,
@@ -497,13 +480,11 @@ public abstract class AbstractQueryExecutor<E> implements QueryExecutor<E> {
 
     blockExecutionInfo
         .setTotalNumberOfMeasureToRead(segmentProperties.getMeasuresOrdinalToChunkMapping().size());
-    blockExecutionInfo.setComplexDimensionInfoMap(QueryUtil
-        .getComplexDimensionsMap(projectDimensions,
+    blockExecutionInfo.setComplexDimensionInfoMap(
+        QueryUtil.getComplexDimensionsMap(
+            projectDimensions,
             segmentProperties.getDimensionOrdinalToChunkMapping(),
-            segmentProperties.getEachComplexDimColumnValueSize(),
             queryProperties.complexFilterDimension));
-    IndexKey startIndexKey = null;
-    IndexKey endIndexKey = null;
     if (null != queryModel.getDataMapFilter()) {
       FilterResolverIntf filterResolverIntf;
       // loading the filter executor tree for filter evaluation
@@ -512,12 +493,6 @@ public abstract class AbstractQueryExecutor<E> implements QueryExecutor<E> {
           FilterUtil.getFilterExecuterTree(filterResolverIntf, segmentProperties,
               blockExecutionInfo.getComlexDimensionInfoMap(), false));
     }
-    startIndexKey = FilterUtil.prepareDefaultStartIndexKey(segmentProperties);
-    endIndexKey = FilterUtil.prepareDefaultEndIndexKey(segmentProperties);
-    //setting the start index key of the block node
-    blockExecutionInfo.setStartKey(startIndexKey);
-    //setting the end index key of the block node
-    blockExecutionInfo.setEndKey(endIndexKey);
     // expression measure
     List<CarbonMeasure> expressionMeasures =
         new ArrayList<CarbonMeasure>(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE);
@@ -612,8 +587,6 @@ public abstract class AbstractQueryExecutor<E> implements QueryExecutor<E> {
     // setting the no dictionary column block indexes
     blockExecutionInfo.setNoDictionaryColumnChunkIndexes(ArrayUtils.toPrimitive(
         noDictionaryColumnChunkIndex.toArray(new Integer[noDictionaryColumnChunkIndex.size()])));
-    // setting each column value size
-    blockExecutionInfo.setEachColumnValueSize(segmentProperties.getEachDimColumnValueSize());
     blockExecutionInfo.setComplexColumnParentBlockIndexes(
         getComplexDimensionParentBlockIndexes(projectDimensions));
     blockExecutionInfo.setVectorBatchCollector(queryModel.isVectorReader());
@@ -652,8 +625,8 @@ public abstract class AbstractQueryExecutor<E> implements QueryExecutor<E> {
     int[] dictionaryColumnOrdinal = ArrayUtils.toPrimitive(
         fixedLengthDimensionOrdinal.toArray(new Integer[fixedLengthDimensionOrdinal.size()]));
     // calculate the size of existing query dictionary columns in this block
-    if (dictionaryColumnOrdinal.length > 0) {
-      int[] eachColumnValueSize = blockMetadataInfo.getEachDimColumnValueSize();
+    if (blockMetadataInfo.getNumberOfDictDimensions() > 0) {
+      int[] eachColumnValueSize = blockMetadataInfo.createDimColumnValueLength();
       int keySize = 0;
       for (int i = 0; i < dictionaryColumnOrdinal.length; i++) {
         keySize += eachColumnValueSize[dictionaryColumnOrdinal[i]];
diff --git a/core/src/main/java/org/apache/carbondata/core/scan/executor/infos/BlockExecutionInfo.java b/core/src/main/java/org/apache/carbondata/core/scan/executor/infos/BlockExecutionInfo.java
index 51368dd..eb19852 100644
--- a/core/src/main/java/org/apache/carbondata/core/scan/executor/infos/BlockExecutionInfo.java
+++ b/core/src/main/java/org/apache/carbondata/core/scan/executor/infos/BlockExecutionInfo.java
@@ -20,7 +20,6 @@ package org.apache.carbondata.core.scan.executor.infos;
 import java.util.Map;
 
 import org.apache.carbondata.core.datastore.DataRefNode;
-import org.apache.carbondata.core.datastore.IndexKey;
 import org.apache.carbondata.core.datastore.ReusableDataBuffer;
 import org.apache.carbondata.core.datastore.block.AbstractIndex;
 import org.apache.carbondata.core.mutate.DeleteDeltaVo;
@@ -47,20 +46,6 @@ public class BlockExecutionInfo {
    */
   private MeasureInfo measureInfo;
 
-  /**
-   * this will be used to get the first tentative block from which query
-   * execution start, this will be useful in case of filter query to get the
-   * start block based on filter values
-   */
-  private IndexKey startKey;
-
-  /**
-   * this will be used to get the last tentative block till which scanning
-   * will be done, this will be useful in case of filter query to get the last
-   * block based on filter values
-   */
-  private IndexKey endKey;
-
   private String blockId;
 
   /**
@@ -118,11 +103,6 @@ public class BlockExecutionInfo {
   private int[] noDictionaryColumnChunkIndexes;
 
   /**
-   * each column value size
-   */
-  private int[] eachColumnValueSize;
-
-  /**
    * filter tree to execute the filter
    */
   private FilterExecuter filterExecuterTree;
@@ -255,34 +235,6 @@ public class BlockExecutionInfo {
   }
 
   /**
-   * @return the startKey
-   */
-  public IndexKey getStartKey() {
-    return startKey;
-  }
-
-  /**
-   * @param startKey the startKey to set
-   */
-  public void setStartKey(IndexKey startKey) {
-    this.startKey = startKey;
-  }
-
-  /**
-   * @return the endKey
-   */
-  public IndexKey getEndKey() {
-    return endKey;
-  }
-
-  /**
-   * @param endKey the endKey to set
-   */
-  public void setEndKey(IndexKey endKey) {
-    this.endKey = endKey;
-  }
-
-  /**
    * @return the totalNumberDimensionToRead
    */
   public int getTotalNumberDimensionToRead() {
@@ -395,20 +347,6 @@ public class BlockExecutionInfo {
   }
 
   /**
-   * @return the eachColumnValueSize
-   */
-  public int[] getEachColumnValueSize() {
-    return eachColumnValueSize;
-  }
-
-  /**
-   * @param eachColumnValueSize the eachColumnValueSize to set
-   */
-  public void setEachColumnValueSize(int[] eachColumnValueSize) {
-    this.eachColumnValueSize = eachColumnValueSize;
-  }
-
-  /**
    * @return the dictionaryColumnChunkIndex
    */
   public int[] getDictionaryColumnChunkIndex() {
diff --git a/core/src/main/java/org/apache/carbondata/core/scan/executor/util/QueryUtil.java b/core/src/main/java/org/apache/carbondata/core/scan/executor/util/QueryUtil.java
index b3db954..53a9358 100644
--- a/core/src/main/java/org/apache/carbondata/core/scan/executor/util/QueryUtil.java
+++ b/core/src/main/java/org/apache/carbondata/core/scan/executor/util/QueryUtil.java
@@ -22,17 +22,14 @@ import java.util.Arrays;
 import java.util.BitSet;
 import java.util.HashMap;
 import java.util.HashSet;
-import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
 import java.util.Set;
-import java.util.TreeSet;
 
 import org.apache.carbondata.core.constants.CarbonCommonConstants;
 import org.apache.carbondata.core.datastore.block.TableBlockInfo;
 import org.apache.carbondata.core.datastore.compression.Compressor;
 import org.apache.carbondata.core.indexstore.BlockletDetailInfo;
-import org.apache.carbondata.core.keygenerator.KeyGenerator;
 import org.apache.carbondata.core.metadata.blocklet.DataFileFooter;
 import org.apache.carbondata.core.metadata.datatype.DataType;
 import org.apache.carbondata.core.metadata.datatype.DataTypes;
@@ -63,134 +60,6 @@ import org.apache.commons.lang3.ArrayUtils;
 public class QueryUtil {
 
   /**
-   * Below method will be used to get the masked byte range based on the query
-   * dimension. It will give the range in the mdkey. This will be used to get
-   * the actual key array from masked mdkey
-   *
-   * @param queryDimensions query dimension selected in query
-   * @param keyGenerator    key generator
-   * @return masked key
-   */
-  public static int[] getMaskedByteRange(List<ProjectionDimension> queryDimensions,
-      KeyGenerator keyGenerator) {
-    Set<Integer> byteRangeSet = new TreeSet<Integer>();
-    int[] byteRange = null;
-    for (int i = 0; i < queryDimensions.size(); i++) {
-
-      // as no dictionary column and complex type columns
-      // are not selected in the mdkey
-      // so we will not select the those dimension for calculating the
-      // range
-      if (queryDimensions.get(i).getDimension().getKeyOrdinal() == -1) {
-        continue;
-      }
-      // get the offset of the dimension in the mdkey
-      byteRange =
-          keyGenerator.getKeyByteOffsets(queryDimensions.get(i).getDimension().getKeyOrdinal());
-      for (int j = byteRange[0]; j <= byteRange[1]; j++) {
-        byteRangeSet.add(j);
-      }
-    }
-    int[] maksedByteRange = new int[byteRangeSet.size()];
-    int index = 0;
-    Iterator<Integer> iterator = byteRangeSet.iterator();
-    // add the masked byte range
-    while (iterator.hasNext()) {
-      maksedByteRange[index++] = iterator.next();
-    }
-    return maksedByteRange;
-  }
-
-  public static int[] getMaskedByteRangeBasedOrdinal(List<Integer> ordinals,
-      KeyGenerator keyGenerator) {
-    Set<Integer> byteRangeSet = new TreeSet<Integer>();
-    int[] byteRange = null;
-    for (int i = 0; i < ordinals.size(); i++) {
-
-      // get the offset of the dimension in the mdkey
-      byteRange = keyGenerator.getKeyByteOffsets(ordinals.get(i));
-      for (int j = byteRange[0]; j <= byteRange[1]; j++) {
-        byteRangeSet.add(j);
-      }
-    }
-    int[] maksedByteRange = new int[byteRangeSet.size()];
-    int index = 0;
-    Iterator<Integer> iterator = byteRangeSet.iterator();
-    // add the masked byte range
-    while (iterator.hasNext()) {
-      maksedByteRange[index++] = iterator.next();
-    }
-    return maksedByteRange;
-  }
-
-  /**
-   * Below method will return the max key based on the dimension ordinal
-   *
-   * @param keyOrdinalList
-   * @param generator
-   * @return
-   */
-  public static byte[] getMaxKeyBasedOnOrdinal(
-      List<Integer> keyOrdinalList, KeyGenerator generator) {
-    long[] max = new long[generator.getDimCount()];
-    Arrays.fill(max, 0L);
-
-    for (int i = 0; i < keyOrdinalList.size(); i++) {
-      // adding for dimension which is selected in query
-      max[keyOrdinalList.get(i)] = Long.MAX_VALUE;
-    }
-    return generator.generateKey(max);
-  }
-
-  /**
-   * To get the max key based on dimensions. i.e. all other dimensions will be
-   * set to 0 bits and the required query dimension will be masked with all
-   * LONG.MAX so that we can mask key and then compare while aggregating This
-   * can be useful during filter query when only few dimensions were selected
-   * out of row group
-   *
-   * @param queryDimensions dimension selected in query
-   * @param generator       key generator
-   * @return max key for dimension
-   */
-  public static byte[] getMaxKeyBasedOnDimensions(List<ProjectionDimension> queryDimensions,
-      KeyGenerator generator) {
-    long[] max = new long[generator.getDimCount()];
-    Arrays.fill(max, 0L);
-
-    for (int i = 0; i < queryDimensions.size(); i++) {
-      // as no dictionary column and complex type columns
-      // are not selected in the mdkey
-      // so we will not select the those dimension for calculating the
-      // range
-      if (queryDimensions.get(i).getDimension().getKeyOrdinal() == -1) {
-        continue;
-      }
-      // adding for dimension which is selected in query
-      max[queryDimensions.get(i).getDimension().getKeyOrdinal()] = Long.MAX_VALUE;
-    }
-
-    return generator.generateKey(max);
-  }
-
-  /**
-   * Below method will be used to get the masked key for query
-   *
-   * @param keySize         size of the masked key
-   * @param maskedKeyRanges masked byte range
-   * @return masked bytes
-   */
-  public static int[] getMaskedByte(int keySize, int[] maskedKeyRanges) {
-    int[] maskedKey = new int[keySize];
-    // all the non selected dimension will be filled with -1
-    Arrays.fill(maskedKey, -1);
-    for (int i = 0; i < maskedKeyRanges.length; i++) {
-      maskedKey[maskedKeyRanges[i]] = i;
-    }
-    return maskedKey;
-  }
-
-  /**
    * Below method will be used to get the dimension block index in file based
    * on query dimension
    *
@@ -376,7 +245,7 @@ public class QueryUtil {
    */
   public static Map<Integer, GenericQueryType> getComplexDimensionsMap(
       List<ProjectionDimension> queryDimensions, Map<Integer, Integer> dimensionToBlockIndexMap,
-      int[] eachComplexColumnValueSize, Set<CarbonDimension> filterDimensions) {
+      Set<CarbonDimension> filterDimensions) {
     Map<Integer, GenericQueryType> complexTypeMap = new HashMap<Integer, GenericQueryType>();
     for (ProjectionDimension dimension : queryDimensions) {
       CarbonDimension actualDimension;
@@ -394,11 +263,9 @@ public class QueryUtil {
         continue;
       }
       if (complexDimension != null) {
-        fillParentDetails(dimensionToBlockIndexMap, complexDimension, complexTypeMap,
-            eachComplexColumnValueSize);
+        fillParentDetails(dimensionToBlockIndexMap, complexDimension, complexTypeMap);
       }
-      fillParentDetails(dimensionToBlockIndexMap, actualDimension, complexTypeMap,
-          eachComplexColumnValueSize);
+      fillParentDetails(dimensionToBlockIndexMap, actualDimension, complexTypeMap);
     }
     if (null != filterDimensions) {
       for (CarbonDimension filterDimension : filterDimensions) {
@@ -407,16 +274,14 @@ public class QueryUtil {
             || filterDimension.getNumberOfChild() == 0) {
           continue;
         }
-        fillParentDetails(dimensionToBlockIndexMap, filterDimension, complexTypeMap,
-            eachComplexColumnValueSize);
+        fillParentDetails(dimensionToBlockIndexMap, filterDimension, complexTypeMap);
       }
     }
     return complexTypeMap;
   }
 
   private static void fillParentDetails(Map<Integer, Integer> dimensionToBlockIndexMap,
-      CarbonDimension dimension, Map<Integer, GenericQueryType> complexTypeMap,
-      int[] eachComplexColumnValueSize) {
+      CarbonDimension dimension, Map<Integer, GenericQueryType> complexTypeMap) {
     int parentBlockIndex = dimensionToBlockIndexMap.get(dimension.getOrdinal());
     GenericQueryType parentQueryType;
     if (DataTypes.isArrayType(dimension.getDataType())) {
@@ -434,26 +299,25 @@ public class QueryUtil {
           " is not supported");
     }
     complexTypeMap.put(dimension.getOrdinal(), parentQueryType);
-    fillChildrenDetails(eachComplexColumnValueSize, parentBlockIndex,
-            dimension, parentQueryType);
+    fillChildrenDetails(parentBlockIndex, dimension, parentQueryType);
   }
 
-  private static int fillChildrenDetails(int[] eachComplexColumnValueSize, int parentBlockIndex,
-      CarbonDimension dimension, GenericQueryType parentQueryType) {
+  private static int fillChildrenDetails(int parentColumnIndex, CarbonDimension dimension,
+      GenericQueryType parentQueryType) {
     for (int i = 0; i < dimension.getNumberOfChild(); i++) {
       DataType dataType = dimension.getListOfChildDimensions().get(i).getDataType();
       if (DataTypes.isArrayType(dataType)) {
         parentQueryType.addChildren(
             new ArrayQueryType(dimension.getListOfChildDimensions().get(i).getColName(),
-                dimension.getColName(), ++parentBlockIndex));
+                dimension.getColName(), ++parentColumnIndex));
       } else if (DataTypes.isStructType(dataType)) {
         parentQueryType.addChildren(
             new StructQueryType(dimension.getListOfChildDimensions().get(i).getColName(),
-                dimension.getColName(), ++parentBlockIndex));
+                dimension.getColName(), ++parentColumnIndex));
       } else if (DataTypes.isMapType(dataType)) {
         parentQueryType.addChildren(
             new MapQueryType(dimension.getListOfChildDimensions().get(i).getColName(),
-                dimension.getColName(), ++parentBlockIndex));
+                dimension.getColName(), ++parentColumnIndex));
       } else {
         boolean isDirectDictionary = CarbonUtil
             .hasEncoding(dimension.getListOfChildDimensions().get(i).getEncoder(),
@@ -461,17 +325,15 @@ public class QueryUtil {
 
         parentQueryType.addChildren(
             new PrimitiveQueryType(dimension.getListOfChildDimensions().get(i).getColName(),
-                dimension.getColName(), ++parentBlockIndex,
-                dimension.getListOfChildDimensions().get(i).getDataType(),
-                eachComplexColumnValueSize[dimension.getListOfChildDimensions().get(i)
-                    .getComplexTypeOrdinal()], isDirectDictionary));
+                dimension.getColName(), ++parentColumnIndex,
+                dimension.getListOfChildDimensions().get(i).getDataType(), isDirectDictionary));
       }
       if (dimension.getListOfChildDimensions().get(i).getNumberOfChild() > 0) {
-        parentBlockIndex = fillChildrenDetails(eachComplexColumnValueSize, parentBlockIndex,
+        parentColumnIndex = fillChildrenDetails(parentColumnIndex,
             dimension.getListOfChildDimensions().get(i), parentQueryType);
       }
     }
-    return parentBlockIndex;
+    return parentColumnIndex;
   }
 
   public static void getAllFilterDimensionsAndMeasures(FilterResolverIntf filterResolverTree,
@@ -622,7 +484,6 @@ public class QueryUtil {
   public static BlockletDetailInfo getBlockletDetailInfo(DataFileFooter fileFooter,
       TableBlockInfo blockInfo) {
     BlockletDetailInfo detailInfo = new BlockletDetailInfo();
-    detailInfo.setDimLens(fileFooter.getSegmentInfo().getColumnCardinality());
     detailInfo.setBlockletInfoBinary(new byte[0]);
     detailInfo.setColumnSchemas(fileFooter.getColumnInTable());
     detailInfo.setBlockletId((short) -1);
diff --git a/core/src/main/java/org/apache/carbondata/core/scan/executor/util/RestructureUtil.java b/core/src/main/java/org/apache/carbondata/core/scan/executor/util/RestructureUtil.java
index b573e0d..921e34d 100644
--- a/core/src/main/java/org/apache/carbondata/core/scan/executor/util/RestructureUtil.java
+++ b/core/src/main/java/org/apache/carbondata/core/scan/executor/util/RestructureUtil.java
@@ -247,26 +247,6 @@ public class RestructureUtil {
   }
 
   /**
-   * Method for computing default value for dictionary column
-   *
-   * @param defaultValue
-   * @return
-   */
-  private static Object getDictionaryDefaultValue(byte[] defaultValue) {
-    Object dictionaryDefaultValue = null;
-    // dictionary has 2 cases:
-    // 1. If default value is specified then its surrogate key will be 2
-    // 2.  If default value is not specified then its surrogate key will be
-    // 1 which is for member default value null
-    if (isDefaultValueNull(defaultValue)) {
-      dictionaryDefaultValue = CarbonCommonConstants.MEMBER_DEFAULT_VAL_SURROGATE_KEY;
-    } else {
-      dictionaryDefaultValue = CarbonCommonConstants.MEMBER_DEFAULT_VAL_SURROGATE_KEY + 1;
-    }
-    return dictionaryDefaultValue;
-  }
-
-  /**
    * Method for computing default value for direct dictionary
    *
    * @param dataType
diff --git a/core/src/main/java/org/apache/carbondata/core/scan/filter/FilterExpressionProcessor.java b/core/src/main/java/org/apache/carbondata/core/scan/filter/FilterExpressionProcessor.java
index c607028..4d80fbc 100644
--- a/core/src/main/java/org/apache/carbondata/core/scan/filter/FilterExpressionProcessor.java
+++ b/core/src/main/java/org/apache/carbondata/core/scan/filter/FilterExpressionProcessor.java
@@ -102,7 +102,7 @@ public class FilterExpressionProcessor implements FilterProcessor {
       return;
     }
     traverseAndResolveTree(filterResolverTree.getLeft(), tableIdentifier);
-    filterResolverTree.resolve(tableIdentifier);
+    filterResolverTree.resolve();
     traverseAndResolveTree(filterResolverTree.getRight(), tableIdentifier);
   }
 
diff --git a/core/src/main/java/org/apache/carbondata/core/scan/filter/FilterUtil.java b/core/src/main/java/org/apache/carbondata/core/scan/filter/FilterUtil.java
index 22ac5ed..50580ab 100644
--- a/core/src/main/java/org/apache/carbondata/core/scan/filter/FilterUtil.java
+++ b/core/src/main/java/org/apache/carbondata/core/scan/filter/FilterUtil.java
@@ -17,7 +17,6 @@
 
 package org.apache.carbondata.core.scan.filter;
 
-import java.nio.ByteBuffer;
 import java.nio.charset.Charset;
 import java.text.ParseException;
 import java.text.SimpleDateFormat;
@@ -26,17 +25,13 @@ import java.util.Arrays;
 import java.util.BitSet;
 import java.util.Collections;
 import java.util.Date;
-import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
 import java.util.Set;
-import java.util.SortedMap;
-import java.util.TreeSet;
 
 import org.apache.carbondata.common.logging.LogServiceFactory;
 import org.apache.carbondata.core.constants.CarbonCommonConstants;
 import org.apache.carbondata.core.constants.CarbonV3DataFormatConstants;
-import org.apache.carbondata.core.datastore.IndexKey;
 import org.apache.carbondata.core.datastore.block.SegmentProperties;
 import org.apache.carbondata.core.datastore.chunk.DimensionColumnPage;
 import org.apache.carbondata.core.datastore.chunk.impl.DimensionRawColumnChunk;
@@ -531,46 +526,6 @@ public final class FilterUtil {
   }
 
   /**
-   * method will get the masked keys based on the keys generated from surrogates.
-   *
-   * @param ranges
-   * @param key
-   * @return byte[]
-   */
-  private static byte[] getMaskedKey(int[] ranges, byte[] key) {
-    byte[] maskkey = new byte[ranges.length];
-
-    for (int i = 0; i < maskkey.length; i++) {
-      maskkey[i] = key[ranges[i]];
-    }
-    return maskkey;
-  }
-
-  /**
-   * This method will return the ranges for the masked Bytes based on the key
-   * Generator.
-   *
-   * @param queryDimensionsOrdinal
-   * @param generator
-   * @return
-   */
-  private static int[] getRangesForMaskedByte(int queryDimensionsOrdinal, KeyGenerator generator) {
-    Set<Integer> integers = new TreeSet<Integer>();
-    int[] range = generator.getKeyByteOffsets(queryDimensionsOrdinal);
-    for (int j = range[0]; j <= range[1]; j++) {
-      integers.add(j);
-    }
-
-    int[] byteIndexs = new int[integers.size()];
-    int j = 0;
-    for (Iterator<Integer> iterator = integers.iterator(); iterator.hasNext(); ) {
-      Integer integer = iterator.next();
-      byteIndexs[j++] = integer;
-    }
-    return byteIndexs;
-  }
-
-  /**
    * This method will get the no dictionary data based on filters and same
    * will be in ColumnFilterInfo
    *
@@ -700,11 +655,9 @@ public final class FilterUtil {
   }
 
   private static byte[][] getFilterValuesInBytes(ColumnFilterInfo columnFilterInfo,
-      boolean isExclude, KeyGenerator blockLevelKeyGenerator, int[] dimColumnsCardinality,
-      int[] keys, List<byte[]> filterValuesList, int keyOrdinalOfDimensionFromCurrentBlock) {
+      boolean isExclude, int[] keys, List<byte[]> filterValuesList,
+      int keyOrdinalOfDimensionFromCurrentBlock) {
     if (null != columnFilterInfo) {
-      int[] rangesForMaskedByte =
-          getRangesForMaskedByte(keyOrdinalOfDimensionFromCurrentBlock, blockLevelKeyGenerator);
       List<Integer> listOfsurrogates = null;
       if (!isExclude && columnFilterInfo.isIncludeFilter()) {
         listOfsurrogates = columnFilterInfo.getFilterList();
@@ -713,13 +666,8 @@ public final class FilterUtil {
       }
       if (null != listOfsurrogates) {
         for (Integer surrogate : listOfsurrogates) {
-          if (surrogate <= dimColumnsCardinality[keyOrdinalOfDimensionFromCurrentBlock]) {
-            keys[keyOrdinalOfDimensionFromCurrentBlock] = surrogate;
-            filterValuesList
-                .add(getMaskedKey(rangesForMaskedByte, blockLevelKeyGenerator.generateKey(keys)));
-          } else {
-            break;
-          }
+          keys[keyOrdinalOfDimensionFromCurrentBlock] = surrogate;
+          filterValuesList.add(ByteUtil.convertIntToBytes(surrogate));
         }
       }
     }
@@ -729,30 +677,21 @@ public final class FilterUtil {
   // This function is used for calculating filter values in case when Range Column
   // is given as a Dictionary Include Column
   private static byte[][] getFilterValueInBytesForDictRange(ColumnFilterInfo columnFilterInfo,
-      KeyGenerator blockLevelKeyGenerator, int[] dimColumnsCardinality, int[] keys,
-      List<byte[]> filterValuesList, int keyOrdinalOfDimensionFromCurrentBlock) {
+      int[] keys, List<byte[]> filterValuesList, int keyOrdinalOfDimensionFromCurrentBlock) {
     if (null != columnFilterInfo) {
-      int[] rangesForMaskedByte =
-          getRangesForMaskedByte(keyOrdinalOfDimensionFromCurrentBlock, blockLevelKeyGenerator);
       List<Integer> listOfsurrogates = columnFilterInfo.getFilterList();
       if (listOfsurrogates == null || listOfsurrogates.size() > 1) {
         throw new RuntimeException(
             "Filter values cannot be null in case of range in dictionary include");
       }
       // Here we only get the first column as there can be only one range column.
-      if (listOfsurrogates.get(0)
-          <= dimColumnsCardinality[keyOrdinalOfDimensionFromCurrentBlock]) {
-        keys[keyOrdinalOfDimensionFromCurrentBlock] = listOfsurrogates.get(0);
-      } else {
-        keys[keyOrdinalOfDimensionFromCurrentBlock] =
-            dimColumnsCardinality[keyOrdinalOfDimensionFromCurrentBlock];
-      }
-      filterValuesList
-          .add(getMaskedKey(rangesForMaskedByte, blockLevelKeyGenerator.generateKey(keys)));
+      keys[keyOrdinalOfDimensionFromCurrentBlock] = listOfsurrogates.get(0);
+      filterValuesList.add(ByteUtil.convertIntToBytes(listOfsurrogates.get(0)));
     }
     return filterValuesList.toArray(new byte[filterValuesList.size()][]);
   }
 
+
   /**
    * This method will be used to get the Filter key array list for blocks which do not contain
    * filter column and the column Encoding is Direct Dictionary
@@ -771,8 +710,8 @@ public final class FilterUtil {
     int keyOrdinalOfDimensionFromCurrentBlock = 0;
     List<byte[]> filterValuesList =
         new ArrayList<byte[]>(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE);
-    return getFilterValuesInBytes(columnFilterInfo, isExclude, blockLevelKeyGenerator,
-        dimColumnsCardinality, keys, filterValuesList, keyOrdinalOfDimensionFromCurrentBlock);
+    return getFilterValuesInBytes(columnFilterInfo, isExclude, keys, filterValuesList,
+        keyOrdinalOfDimensionFromCurrentBlock);
   }
 
   /**
@@ -792,324 +731,21 @@ public final class FilterUtil {
       return columnFilterInfo.getNoDictionaryFilterValuesList()
           .toArray((new byte[columnFilterInfo.getNoDictionaryFilterValuesList().size()][]));
     }
-    KeyGenerator blockLevelKeyGenerator = segmentProperties.getDimensionKeyGenerator();
-    int[] dimColumnsCardinality = segmentProperties.getDimColumnsCardinality();
-    int[] keys = new int[blockLevelKeyGenerator.getDimCount()];
+    int[] keys = new int[segmentProperties.getDimensions().size()];
     List<byte[]> filterValuesList = new ArrayList<byte[]>(20);
     Arrays.fill(keys, 0);
     int keyOrdinalOfDimensionFromCurrentBlock = carbonDimension.getKeyOrdinal();
     if (!isDictRange) {
-      return getFilterValuesInBytes(columnFilterInfo, isExclude, blockLevelKeyGenerator,
-          dimColumnsCardinality, keys, filterValuesList, keyOrdinalOfDimensionFromCurrentBlock);
+      return getFilterValuesInBytes(columnFilterInfo, isExclude, keys, filterValuesList,
+          keyOrdinalOfDimensionFromCurrentBlock);
     } else {
       // For Dictionary Include Range Column
-      return getFilterValueInBytesForDictRange(columnFilterInfo, blockLevelKeyGenerator,
-          dimColumnsCardinality, keys, filterValuesList, keyOrdinalOfDimensionFromCurrentBlock);
-    }
-  }
-
-  /**
-   * The method is used to get the single dictionary key's mask key
-   *
-   * @param surrogate
-   * @param carbonDimension
-   * @param blockLevelKeyGenerator
-   * @return
-   */
-  public static byte[] getMaskKey(int surrogate, CarbonDimension carbonDimension,
-      KeyGenerator blockLevelKeyGenerator) {
-    int[] keys = new int[blockLevelKeyGenerator.getDimCount()];
-    byte[] maskedKey = null;
-    Arrays.fill(keys, 0);
-    int[] rangesForMaskedByte =
-        getRangesForMaskedByte((carbonDimension.getKeyOrdinal()), blockLevelKeyGenerator);
-    keys[carbonDimension.getKeyOrdinal()] = surrogate;
-    maskedKey = getMaskedKey(rangesForMaskedByte, blockLevelKeyGenerator.generateKey(keys));
-    return maskedKey;
-  }
-
-  /**
-   * Method will return the start key based on KeyGenerator for the respective
-   * filter resolved instance.
-   *
-   * @param dimensionFilter
-   * @param startKey
-   * @param startKeyList
-   * @return long[] start key
-   */
-  public static void getStartKey(Map<CarbonDimension, List<ColumnFilterInfo>> dimensionFilter,
-      SegmentProperties segmentProperties, long[] startKey, List<long[]> startKeyList) {
-    for (int i = 0; i < startKey.length; i++) {
-      // The min surrogate key is 1, set it as the init value for starkey of each column level
-      startKey[i] = 1;
-    }
-    getStartKeyWithFilter(dimensionFilter, segmentProperties, startKey, startKeyList);
-  }
-
-  /**
-   * Algorithm for getting the start key for a filter
-   * step 1: Iterate through each dimension and verify whether its not an exclude filter.
-   * step 2: Initialize start key with the first filter member value present in each filter model
-   * for the respective dimensions.
-   * step 3: since its a no dictionary start key there will only actual value so compare
-   * the first filter model value with respect to the dimension data type.
-   * step 4: The least value will be considered as the start key of dimension by comparing all
-   * its filter model.
-   * step 5: create a byte array of start key which comprises of least filter member value of
-   * all dimension and the indexes which will help to read the respective filter value.
-   *
-   * @param dimColResolvedFilterInfo
-   * @param setOfStartKeyByteArray
-   * @return
-   */
-  public static void getStartKeyForNoDictionaryDimension(
-      DimColumnResolvedFilterInfo dimColResolvedFilterInfo,
-      SegmentProperties segmentProperties,
-      SortedMap<Integer, byte[]> setOfStartKeyByteArray) {
-    Map<CarbonDimension, List<ColumnFilterInfo>> dimensionFilter =
-        dimColResolvedFilterInfo.getDimensionResolvedFilterInstance();
-    // step 1
-    for (Map.Entry<CarbonDimension, List<ColumnFilterInfo>> entry : dimensionFilter.entrySet()) {
-      if (entry.getKey().getDataType() != DataTypes.DATE) {
-        List<ColumnFilterInfo> listOfDimColFilterInfo = entry.getValue();
-        if (null == listOfDimColFilterInfo) {
-          continue;
-        }
-        boolean isExcludePresent = false;
-        for (ColumnFilterInfo info : listOfDimColFilterInfo) {
-          if (!info.isIncludeFilter()) {
-            isExcludePresent = true;
-          }
-        }
-        if (isExcludePresent) {
-          continue;
-        }
-        // in case of restructure scenarios it can happen that the filter dimension is not
-        // present in the current block. In those cases no need to determine the key
-        CarbonDimension dimensionFromCurrentBlock = CarbonUtil
-            .getDimensionFromCurrentBlock(segmentProperties.getDimensions(), entry.getKey());
-        if (null == dimensionFromCurrentBlock) {
-          continue;
-        }
-        // step 2
-        byte[] noDictionaryStartKey =
-            listOfDimColFilterInfo.get(0).getNoDictionaryFilterValuesList().get(0);
-        if (setOfStartKeyByteArray.isEmpty()) {
-          setOfStartKeyByteArray.put(dimensionFromCurrentBlock.getOrdinal(), noDictionaryStartKey);
-        } else if (null == setOfStartKeyByteArray.get(dimensionFromCurrentBlock.getOrdinal())) {
-          setOfStartKeyByteArray.put(dimensionFromCurrentBlock.getOrdinal(), noDictionaryStartKey);
-
-        } else if (ByteUtil.UnsafeComparer.INSTANCE
-            .compareTo(setOfStartKeyByteArray.get(dimensionFromCurrentBlock.getOrdinal()),
-                noDictionaryStartKey) > 0) {
-          setOfStartKeyByteArray.put(dimensionFromCurrentBlock.getOrdinal(), noDictionaryStartKey);
-        }
-      }
-    }
-  }
-
-  /**
-   * Algorithm for getting the end key for a filter
-   * step 1: Iterate through each dimension and verify whether its not an exclude filter.
-   * step 2: Initialize end key with the last filter member value present in each filter model
-   * for the respective dimensions.(Already filter models are sorted)
-   * step 3: since its a no dictionary end key there will only actual value so compare
-   * the last filter model value with respect to the dimension data type.
-   * step 4: The highest value will be considered as the end key of dimension by comparing all
-   * its filter model.
-   * step 5: create a byte array of end key which comprises of highest filter member value of
-   * all dimension and the indexes which will help to read the respective filter value.
-   *
-   * @param dimColResolvedFilterInfo
-   * @param setOfEndKeyByteArray
-   * @return end key array
-   */
-  public static void getEndKeyForNoDictionaryDimension(
-      DimColumnResolvedFilterInfo dimColResolvedFilterInfo,
-      SegmentProperties segmentProperties,
-      SortedMap<Integer, byte[]> setOfEndKeyByteArray) {
-
-    Map<CarbonDimension, List<ColumnFilterInfo>> dimensionFilter =
-        dimColResolvedFilterInfo.getDimensionResolvedFilterInstance();
-    // step 1
-    for (Map.Entry<CarbonDimension, List<ColumnFilterInfo>> entry : dimensionFilter.entrySet()) {
-      if (entry.getKey().getDataType() != DataTypes.DATE) {
-        List<ColumnFilterInfo> listOfDimColFilterInfo = entry.getValue();
-        if (null == listOfDimColFilterInfo) {
-          continue;
-        }
-        boolean isExcludePresent = false;
-        for (ColumnFilterInfo info : listOfDimColFilterInfo) {
-          if (!info.isIncludeFilter()) {
-            isExcludePresent = true;
-          }
-        }
-        if (isExcludePresent) {
-          continue;
-        }
-        // in case of restructure scenarios it can happen that the filter dimension is not
-        // present in the current block. In those cases no need to determine the key
-        CarbonDimension dimensionFromCurrentBlock = CarbonUtil
-            .getDimensionFromCurrentBlock(segmentProperties.getDimensions(), entry.getKey());
-        if (null == dimensionFromCurrentBlock) {
-          continue;
-        }
-        // step 2
-        byte[] noDictionaryEndKey = listOfDimColFilterInfo.get(0).getNoDictionaryFilterValuesList()
-            .get(listOfDimColFilterInfo.get(0).getNoDictionaryFilterValuesList().size() - 1);
-        if (setOfEndKeyByteArray.isEmpty()) {
-          setOfEndKeyByteArray.put(dimensionFromCurrentBlock.getOrdinal(), noDictionaryEndKey);
-        } else if (null == setOfEndKeyByteArray.get(dimensionFromCurrentBlock.getOrdinal())) {
-          setOfEndKeyByteArray.put(dimensionFromCurrentBlock.getOrdinal(), noDictionaryEndKey);
-
-        } else if (ByteUtil.UnsafeComparer.INSTANCE
-            .compareTo(setOfEndKeyByteArray.get(dimensionFromCurrentBlock.getOrdinal()),
-                noDictionaryEndKey) < 0) {
-          setOfEndKeyByteArray.put(dimensionFromCurrentBlock.getOrdinal(), noDictionaryEndKey);
-        }
-
-      }
+      return getFilterValueInBytesForDictRange(columnFilterInfo, keys, filterValuesList,
+          keyOrdinalOfDimensionFromCurrentBlock);
     }
   }
 
   /**
-   * This method will fill the start key array  with the surrogate key present
-   * in filterinfo instance.
-   *
-   * @param dimensionFilter
-   * @param startKey
-   */
-  private static void getStartKeyWithFilter(
-      Map<CarbonDimension, List<ColumnFilterInfo>> dimensionFilter,
-      SegmentProperties segmentProperties, long[] startKey, List<long[]> startKeyList) {
-    for (Map.Entry<CarbonDimension, List<ColumnFilterInfo>> entry : dimensionFilter.entrySet()) {
-      List<ColumnFilterInfo> values = entry.getValue();
-      if (null == values || entry.getKey().getDataType() != DataTypes.DATE) {
-        continue;
-      }
-      boolean isExcludePresent = false;
-      for (ColumnFilterInfo info : values) {
-        if (!info.isIncludeFilter()) {
-          isExcludePresent = true;
-        }
-      }
-      if (isExcludePresent) {
-        continue;
-      }
-      // search the query dimension in current block dimensions. If the dimension is not found
-      // that means the key cannot be included in start key formation.
-      // Applicable for restructure scenarios
-      CarbonDimension dimensionFromCurrentBlock =
-          segmentProperties.getDimensionFromCurrentBlock(entry.getKey());
-      if (null == dimensionFromCurrentBlock) {
-        continue;
-      }
-      int keyOrdinalOfDimensionFromCurrentBlock = dimensionFromCurrentBlock.getKeyOrdinal();
-      for (ColumnFilterInfo info : values) {
-        if (keyOrdinalOfDimensionFromCurrentBlock < startKey.length) {
-          if (startKey[keyOrdinalOfDimensionFromCurrentBlock] < info.getFilterList().get(0)) {
-            startKey[keyOrdinalOfDimensionFromCurrentBlock] = info.getFilterList().get(0);
-          }
-        }
-      }
-      long[] newStartKey = new long[startKey.length];
-      System.arraycopy(startKey, 0, newStartKey, 0, startKey.length);
-      startKeyList.add(newStartKey);
-    }
-  }
-
-  public static void getEndKey(Map<CarbonDimension, List<ColumnFilterInfo>> dimensionFilter,
-      long[] endKey, SegmentProperties segmentProperties,
-      List<long[]> endKeyList) {
-
-    List<CarbonDimension> updatedDimListBasedOnKeyGenerator =
-        getCarbonDimsMappedToKeyGenerator(segmentProperties.getDimensions());
-    for (int i = 0; i < endKey.length; i++) {
-      endKey[i] = getMaxValue(updatedDimListBasedOnKeyGenerator.get(i),
-          segmentProperties.getDimColumnsCardinality());
-    }
-    getEndKeyWithFilter(dimensionFilter, segmentProperties, endKey, endKeyList);
-
-  }
-
-  private static List<CarbonDimension> getCarbonDimsMappedToKeyGenerator(
-      List<CarbonDimension> carbonDimensions) {
-    List<CarbonDimension> listOfCarbonDimPartOfKeyGen =
-        new ArrayList<CarbonDimension>(carbonDimensions.size());
-    for (CarbonDimension carbonDim : carbonDimensions) {
-      if (carbonDim.getDataType() == DataTypes.DATE) {
-        listOfCarbonDimPartOfKeyGen.add(carbonDim);
-      }
-    }
-    return listOfCarbonDimPartOfKeyGen;
-  }
-
-  private static void getEndKeyWithFilter(
-      Map<CarbonDimension, List<ColumnFilterInfo>> dimensionFilter,
-      SegmentProperties segmentProperties, long[] endKey, List<long[]> endKeyList) {
-    for (Map.Entry<CarbonDimension, List<ColumnFilterInfo>> entry : dimensionFilter.entrySet()) {
-      List<ColumnFilterInfo> values = entry.getValue();
-      if (null == values || entry.getKey().getDataType() != DataTypes.DATE) {
-        continue;
-      }
-      boolean isExcludeFilterPresent = false;
-      for (ColumnFilterInfo info : values) {
-        if (!info.isIncludeFilter()) {
-          isExcludeFilterPresent = true;
-        }
-      }
-      if (isExcludeFilterPresent) {
-        continue;
-      }
-      // search the query dimension in current block dimensions. If the dimension is not found
-      // that means the key cannot be included in start key formation.
-      // Applicable for restructure scenarios
-      CarbonDimension dimensionFromCurrentBlock =
-          segmentProperties.getDimensionFromCurrentBlock(entry.getKey());
-      if (null == dimensionFromCurrentBlock) {
-        continue;
-      }
-      int keyOrdinalOfDimensionFromCurrentBlock = dimensionFromCurrentBlock.getKeyOrdinal();
-      int endFilterValue = 0;
-      for (ColumnFilterInfo info : values) {
-        if (keyOrdinalOfDimensionFromCurrentBlock < endKey.length) {
-          endFilterValue = info.getFilterList().get(info.getFilterList().size() - 1);
-          if (endFilterValue == 0) {
-            endFilterValue =
-                segmentProperties.getDimColumnsCardinality()[keyOrdinalOfDimensionFromCurrentBlock];
-          }
-          if (endKey[keyOrdinalOfDimensionFromCurrentBlock] > endFilterValue) {
-            endKey[keyOrdinalOfDimensionFromCurrentBlock] = endFilterValue;
-          }
-        }
-      }
-      long[] newEndKey = new long[endKey.length];
-      System.arraycopy(endKey, 0, newEndKey, 0, endKey.length);
-      endKeyList.add(newEndKey);
-    }
-
-  }
-
-  /**
-   * This API will get the max value of surrogate key which will be used for
-   * determining the end key of particular btree.
-   *
-   * @param dimCardinality
-   */
-  private static long getMaxValue(CarbonDimension carbonDimension, int[] dimCardinality) {
-    // Get data from all the available slices of the table
-    if (null != dimCardinality) {
-      return dimCardinality[carbonDimension.getKeyOrdinal()];
-    }
-    return -1;
-  }
-
-  public static IndexKey createIndexKeyFromResolvedFilterVal(long[] startOrEndKey,
-      KeyGenerator keyGenerator, byte[] startOrEndKeyForNoDictDimension) {
-    return new IndexKey(keyGenerator.generateKey(startOrEndKey), startOrEndKeyForNoDictDimension);
-  }
-
-  /**
    * API will create an filter executer tree based on the filter resolver
    *
    * @param filterExpressionResolverTree
@@ -1178,101 +814,6 @@ public final class FilterUtil {
     }
   }
 
-  /**
-   * method will create a default end key in case of no end key is been derived using existing
-   * filter or in case of non filter queries.
-   *
-   * @param segmentProperties
-   * @return
-   */
-  public static IndexKey prepareDefaultEndIndexKey(SegmentProperties segmentProperties) {
-    long[] dictionarySurrogateKey = new long[segmentProperties.getNumberOfDictSortColumns()];
-    int index = 0;
-    int[] dimColumnsCardinality = segmentProperties.getDimColumnsCardinality();
-    for (int i = 0; i < dictionarySurrogateKey.length; i++) {
-      dictionarySurrogateKey[index++] = dimColumnsCardinality[i];
-    }
-    IndexKey endIndexKey;
-    byte[] dictionaryendMdkey =
-        segmentProperties.getSortColumnsGenerator().generateKey(dictionarySurrogateKey);
-    byte[] noDictionaryEndKeyBuffer = getNoDictionaryDefaultEndKey(segmentProperties);
-    endIndexKey = new IndexKey(dictionaryendMdkey, noDictionaryEndKeyBuffer);
-    return endIndexKey;
-  }
-
-  public static byte[] getNoDictionaryDefaultEndKey(SegmentProperties segmentProperties) {
-
-    int numberOfNoDictionaryDimension = segmentProperties.getNumberOfNoDictSortColumns();
-    // in case of non filter query when no dictionary columns are present we
-    // need to set the default end key, as for non filter query
-    // we need to get the last
-    // block of the btree so we are setting the max byte value in the end key
-    ByteBuffer noDictionaryEndKeyBuffer = ByteBuffer.allocate(
-        (numberOfNoDictionaryDimension * CarbonCommonConstants.SHORT_SIZE_IN_BYTE)
-            + numberOfNoDictionaryDimension);
-    // end key structure will be
-    //<Offset of first No Dictionary key in 2 Bytes><Offset of second No Dictionary key in 2 Bytes>
-    //<Offset of n No Dictionary key in 2 Bytes><first no dictionary column value>
-    // <second no dictionary column value> <N no dictionary column value>
-    //example if we have 2 no dictionary column
-    //<[0,4,0,5,127,127]>
-    short startPoint =
-        (short) (numberOfNoDictionaryDimension * CarbonCommonConstants.SHORT_SIZE_IN_BYTE);
-    for (int i = 0; i < numberOfNoDictionaryDimension; i++) {
-      noDictionaryEndKeyBuffer.putShort((startPoint));
-      startPoint++;
-    }
-    for (int i = 0; i < numberOfNoDictionaryDimension; i++) {
-      noDictionaryEndKeyBuffer.put((byte) 0xFF);
-    }
-    return noDictionaryEndKeyBuffer.array();
-  }
-
-  /**
-   * method will create a default end key in case of no end key is been
-   * derived using existing filter or in case of non filter queries.
-   *
-   * @param segmentProperties
-   * @return
-   */
-  public static IndexKey prepareDefaultStartIndexKey(SegmentProperties segmentProperties) {
-    IndexKey startIndexKey;
-    long[] dictionarySurrogateKey = new long[segmentProperties.getNumberOfDictSortColumns()];
-    byte[] dictionaryStartMdkey =
-        segmentProperties.getSortColumnsGenerator().generateKey(dictionarySurrogateKey);
-    byte[] noDictionaryStartKeyArray = getNoDictionaryDefaultStartKey(segmentProperties);
-
-    startIndexKey = new IndexKey(dictionaryStartMdkey, noDictionaryStartKeyArray);
-    return startIndexKey;
-  }
-
-  public static byte[] getNoDictionaryDefaultStartKey(SegmentProperties segmentProperties) {
-
-    int numberOfNoDictionaryDimension = segmentProperties.getNumberOfNoDictSortColumns();
-    // in case of non filter query when no dictionary columns are present we
-    // need to set the default start key, as for non filter query we need to get the first
-    // block of the btree so we are setting the least byte value in the start key
-    ByteBuffer noDictionaryStartKeyBuffer = ByteBuffer.allocate(
-        (numberOfNoDictionaryDimension * CarbonCommonConstants.SHORT_SIZE_IN_BYTE)
-            + numberOfNoDictionaryDimension);
-    // end key structure will be
-    //<Offset of first No Dictionary key in 2 Bytes><Offset of second No Dictionary key in 2 Bytes>
-    //<Offset of n No Dictionary key in 2 Bytes><first no dictionary column value>
-    // <second no dictionary column value> <N no dictionary column value>
-    //example if we have 2 no dictionary column
-    //<[0,4,0,5,0,0]>
-    short startPoint =
-        (short) (numberOfNoDictionaryDimension * CarbonCommonConstants.SHORT_SIZE_IN_BYTE);
-    for (int i = 0; i < numberOfNoDictionaryDimension; i++) {
-      noDictionaryStartKeyBuffer.putShort((startPoint));
-      startPoint++;
-    }
-    for (int i = 0; i < numberOfNoDictionaryDimension; i++) {
-      noDictionaryStartKeyBuffer.put((byte) 0);
-    }
-    return noDictionaryStartKeyBuffer.array();
-  }
-
   public static int compareFilterKeyBasedOnDataType(String dictionaryVal, String memberVal,
       DataType dataType) {
     try {
@@ -1741,31 +1282,13 @@ public final class FilterUtil {
   /**
    * This method is used to get default null values for a direct dictionary column
    * @param currentBlockDimension
-   * @param segmentProperties
    * @return
    */
-  public static byte[] getDefaultNullValue(CarbonDimension currentBlockDimension,
-      SegmentProperties segmentProperties) {
-    byte[] defaultValue = null;
+  public static byte[] getDefaultNullValue(CarbonDimension currentBlockDimension) {
     DirectDictionaryGenerator directDictionaryGenerator = DirectDictionaryKeyGeneratorFactory
         .getDirectDictionaryGenerator(currentBlockDimension.getDataType());
     int key = directDictionaryGenerator.generateDirectSurrogateKey(null);
-    if (currentBlockDimension.isSortColumn()) {
-      defaultValue = FilterUtil
-          .getMaskKey(key, currentBlockDimension, segmentProperties.getSortColumnsGenerator());
-    } else {
-      defaultValue = FilterUtil
-          .getMaskKey(key, currentBlockDimension, segmentProperties.getDimensionKeyGenerator());
-    }
-    return defaultValue;
-  }
-
-  public static void setMinMaxFlagForLegacyStore(boolean[] minMaxFlag,
-      SegmentProperties segmentProperties) {
-    int index = segmentProperties.getEachDimColumnValueSize().length + segmentProperties
-        .getEachComplexDimColumnValueSize().length;
-    Arrays.fill(minMaxFlag, 0, index, true);
-    Arrays.fill(minMaxFlag, index, minMaxFlag.length, false);
+    return ByteUtil.convertIntToBytes(key);
   }
 
 }
diff --git a/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/RangeValueFilterExecuterImpl.java b/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/RangeValueFilterExecuterImpl.java
index f18e007..664e046 100644
--- a/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/RangeValueFilterExecuterImpl.java
+++ b/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/RangeValueFilterExecuterImpl.java
@@ -643,7 +643,7 @@ public class RangeValueFilterExecuterImpl implements FilterExecuter {
       byte[] defaultValue = null;
       if (dimColEvaluatorInfo.getDimension().getDataType() == DataTypes.DATE) {
         defaultValue =
-            FilterUtil.getDefaultNullValue(dimColEvaluatorInfo.getDimension(), segmentProperties);
+            FilterUtil.getDefaultNullValue(dimColEvaluatorInfo.getDimension());
       } else if (dimColEvaluatorInfo.getDimension().getDataType() == DataTypes.STRING) {
         defaultValue = CarbonCommonConstants.MEMBER_DEFAULT_VAL_ARRAY;
       } else if (!dimensionColumnPage.isAdaptiveEncoded()) {
diff --git a/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/RowLevelFilterExecuterImpl.java b/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/RowLevelFilterExecuterImpl.java
index 1eda5c4..036d868 100644
--- a/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/RowLevelFilterExecuterImpl.java
+++ b/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/RowLevelFilterExecuterImpl.java
@@ -36,8 +36,6 @@ import org.apache.carbondata.core.datastore.chunk.DimensionColumnPage;
 import org.apache.carbondata.core.datastore.chunk.impl.VariableLengthDimensionColumnPage;
 import org.apache.carbondata.core.datastore.chunk.store.ColumnPageWrapper;
 import org.apache.carbondata.core.datastore.page.ColumnPage;
-import org.apache.carbondata.core.keygenerator.directdictionary.DirectDictionaryGenerator;
-import org.apache.carbondata.core.keygenerator.directdictionary.DirectDictionaryKeyGeneratorFactory;
 import org.apache.carbondata.core.metadata.AbsoluteTableIdentifier;
 import org.apache.carbondata.core.metadata.datatype.DataType;
 import org.apache.carbondata.core.metadata.datatype.DataTypes;
@@ -102,16 +100,6 @@ public class RowLevelFilterExecuterImpl implements FilterExecuter {
    */
   boolean isNaturalSorted;
 
-  /**
-   * date direct dictionary generator
-   */
-  private DirectDictionaryGenerator dateDictionaryGenerator;
-
-  /**
-   * timestamp direct dictionary generator
-   */
-  private DirectDictionaryGenerator timestampDictionaryGenerator;
-
   public RowLevelFilterExecuterImpl(List<DimColumnResolvedFilterInfo> dimColEvaluatorInfoList,
       List<MeasureColumnResolvedFilterInfo> msrColEvalutorInfoList, Expression exp,
       AbsoluteTableIdentifier tableIdentifier, SegmentProperties segmentProperties,
@@ -144,10 +132,6 @@ public class RowLevelFilterExecuterImpl implements FilterExecuter {
     this.exp = exp;
     this.tableIdentifier = tableIdentifier;
     this.complexDimensionInfoMap = complexDimensionInfoMap;
-    this.dateDictionaryGenerator =
-        DirectDictionaryKeyGeneratorFactory.getDirectDictionaryGenerator(DataTypes.DATE);
-    this.timestampDictionaryGenerator =
-        DirectDictionaryKeyGeneratorFactory.getDirectDictionaryGenerator(DataTypes.TIMESTAMP);
     initDimensionChunkIndexes();
     initMeasureChunkIndexes();
   }
diff --git a/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/RowLevelRangeGrtThanFiterExecuterImpl.java b/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/RowLevelRangeGrtThanFiterExecuterImpl.java
index 8f5d8c3..7cef5f4 100644
--- a/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/RowLevelRangeGrtThanFiterExecuterImpl.java
+++ b/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/RowLevelRangeGrtThanFiterExecuterImpl.java
@@ -455,7 +455,7 @@ public class RowLevelRangeGrtThanFiterExecuterImpl extends RowLevelFilterExecute
       defaultValue = CarbonCommonConstants.MEMBER_DEFAULT_VAL_ARRAY;
     } else if (dimColEvaluatorInfoList.get(0).getDimension().getDataType() == DataTypes.DATE) {
       defaultValue = FilterUtil
-          .getDefaultNullValue(dimColEvaluatorInfoList.get(0).getDimension(), segmentProperties);
+          .getDefaultNullValue(dimColEvaluatorInfoList.get(0).getDimension());
     } else if (!dimensionColumnPage.isAdaptiveEncoded()) {
       defaultValue = CarbonCommonConstants.EMPTY_BYTE_ARRAY;
     }
diff --git a/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/RowLevelRangeGrtrThanEquaToFilterExecuterImpl.java b/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/RowLevelRangeGrtrThanEquaToFilterExecuterImpl.java
index 0510516..7d245d9 100644
--- a/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/RowLevelRangeGrtrThanEquaToFilterExecuterImpl.java
+++ b/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/RowLevelRangeGrtrThanEquaToFilterExecuterImpl.java
@@ -458,7 +458,7 @@ public class RowLevelRangeGrtrThanEquaToFilterExecuterImpl extends RowLevelFilte
       defaultValue = CarbonCommonConstants.MEMBER_DEFAULT_VAL_ARRAY;
     } else if (dimColEvaluatorInfoList.get(0).getDimension().getDataType() == DataTypes.DATE) {
       defaultValue = FilterUtil
-          .getDefaultNullValue(dimColEvaluatorInfoList.get(0).getDimension(), segmentProperties);
+          .getDefaultNullValue(dimColEvaluatorInfoList.get(0).getDimension());
     } else if (!dimensionColumnPage.isAdaptiveEncoded()) {
       defaultValue = CarbonCommonConstants.EMPTY_BYTE_ARRAY;
     }
diff --git a/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/RowLevelRangeLessThanEqualFilterExecuterImpl.java b/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/RowLevelRangeLessThanEqualFilterExecuterImpl.java
index f68a29a..23ff13a 100644
--- a/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/RowLevelRangeLessThanEqualFilterExecuterImpl.java
+++ b/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/RowLevelRangeLessThanEqualFilterExecuterImpl.java
@@ -428,7 +428,7 @@ public class RowLevelRangeLessThanEqualFilterExecuterImpl extends RowLevelFilter
     byte[] defaultValue = null;
     if (dimColEvaluatorInfoList.get(0).getDimension().getDataType() == DataTypes.DATE) {
       defaultValue = FilterUtil
-          .getDefaultNullValue(dimColEvaluatorInfoList.get(0).getDimension(), segmentProperties);
+          .getDefaultNullValue(dimColEvaluatorInfoList.get(0).getDimension());
     } else if (dimColEvaluatorInfoList.get(0).getDimension().getDataType() != DataTypes.STRING) {
       defaultValue = CarbonCommonConstants.EMPTY_BYTE_ARRAY;
     }
diff --git a/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/RowLevelRangeLessThanFilterExecuterImpl.java b/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/RowLevelRangeLessThanFilterExecuterImpl.java
index a25a452..054f04d 100644
--- a/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/RowLevelRangeLessThanFilterExecuterImpl.java
+++ b/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/RowLevelRangeLessThanFilterExecuterImpl.java
@@ -423,7 +423,7 @@ public class RowLevelRangeLessThanFilterExecuterImpl extends RowLevelFilterExecu
     byte[] defaultValue = null;
     if (dimColEvaluatorInfoList.get(0).getDimension().getDataType() == DataTypes.DATE) {
       defaultValue = FilterUtil
-          .getDefaultNullValue(dimColEvaluatorInfoList.get(0).getDimension(), segmentProperties);
+          .getDefaultNullValue(dimColEvaluatorInfoList.get(0).getDimension());
     } else if (dimColEvaluatorInfoList.get(0).getDimension().getDataType() != DataTypes.STRING) {
       defaultValue = CarbonCommonConstants.EMPTY_BYTE_ARRAY;
     }
diff --git a/core/src/main/java/org/apache/carbondata/core/scan/filter/resolver/ConditionalFilterResolverImpl.java b/core/src/main/java/org/apache/carbondata/core/scan/filter/resolver/ConditionalFilterResolverImpl.java
index deee498..b7ee46f 100644
--- a/core/src/main/java/org/apache/carbondata/core/scan/filter/resolver/ConditionalFilterResolverImpl.java
+++ b/core/src/main/java/org/apache/carbondata/core/scan/filter/resolver/ConditionalFilterResolverImpl.java
@@ -18,10 +18,8 @@
 package org.apache.carbondata.core.scan.filter.resolver;
 
 import java.util.List;
-import java.util.SortedMap;
 
 import org.apache.carbondata.core.datastore.block.SegmentProperties;
-import org.apache.carbondata.core.metadata.AbsoluteTableIdentifier;
 import org.apache.carbondata.core.metadata.datatype.DataTypes;
 import org.apache.carbondata.core.scan.expression.ColumnExpression;
 import org.apache.carbondata.core.scan.expression.Expression;
@@ -66,10 +64,9 @@ public class ConditionalFilterResolverImpl implements FilterResolverIntf {
    * @throws FilterUnsupportedException
    */
   @Override
-  public void resolve(AbsoluteTableIdentifier absoluteTableIdentifier)
+  public void resolve()
       throws FilterUnsupportedException {
     FilterResolverMetadata metadata = new FilterResolverMetadata();
-    metadata.setTableIdentifier(absoluteTableIdentifier);
     if ((!isExpressionResolve) && exp instanceof BinaryConditionalExpression) {
       BinaryConditionalExpression binaryConditionalExpression = (BinaryConditionalExpression) exp;
       Expression leftExp = binaryConditionalExpression.getLeft();
@@ -206,33 +203,6 @@ public class ConditionalFilterResolverImpl implements FilterResolverIntf {
   }
 
   /**
-   * method will calculates the start key based on the filter surrogates
-   */
-  public void getStartKey(SegmentProperties segmentProperties, long[] startKey,
-      SortedMap<Integer, byte[]> setOfStartKeyByteArray, List<long[]> startKeyList) {
-    if (null != dimColResolvedFilterInfo) {
-      FilterUtil.getStartKey(dimColResolvedFilterInfo.getDimensionResolvedFilterInstance(),
-          segmentProperties, startKey, startKeyList);
-      FilterUtil.getStartKeyForNoDictionaryDimension(dimColResolvedFilterInfo, segmentProperties,
-          setOfStartKeyByteArray);
-    }
-  }
-
-  /**
-   * get the start key based on the filter surrogates
-   */
-  @Override
-  public void getEndKey(SegmentProperties segmentProperties, long[] endKeys,
-      SortedMap<Integer, byte[]> setOfEndKeyByteArray, List<long[]> endKeyList) {
-    if (null != dimColResolvedFilterInfo) {
-      FilterUtil.getEndKey(dimColResolvedFilterInfo.getDimensionResolvedFilterInstance(), endKeys,
-          segmentProperties, endKeyList);
-      FilterUtil.getEndKeyForNoDictionaryDimension(dimColResolvedFilterInfo, segmentProperties,
-          setOfEndKeyByteArray);
-    }
-  }
-
-  /**
    * Method will return the executer type for particular conditional resolver
    * basically two types of executers will be formed for the conditional query.
    *
diff --git a/core/src/main/java/org/apache/carbondata/core/scan/filter/resolver/FilterResolverIntf.java b/core/src/main/java/org/apache/carbondata/core/scan/filter/resolver/FilterResolverIntf.java
index ee3b45e..57123b8 100644
--- a/core/src/main/java/org/apache/carbondata/core/scan/filter/resolver/FilterResolverIntf.java
+++ b/core/src/main/java/org/apache/carbondata/core/scan/filter/resolver/FilterResolverIntf.java
@@ -18,11 +18,7 @@
 package org.apache.carbondata.core.scan.filter.resolver;
 
 import java.io.Serializable;
-import java.util.List;
-import java.util.SortedMap;
 
-import org.apache.carbondata.core.datastore.block.SegmentProperties;
-import org.apache.carbondata.core.metadata.AbsoluteTableIdentifier;
 import org.apache.carbondata.core.scan.expression.Expression;
 import org.apache.carbondata.core.scan.expression.exception.FilterUnsupportedException;
 import org.apache.carbondata.core.scan.filter.intf.FilterExecuterType;
@@ -38,7 +34,7 @@ public interface FilterResolverIntf extends Serializable {
    *
    * @throws FilterUnsupportedException
    */
-  void resolve(AbsoluteTableIdentifier absoluteTableIdentifier) throws FilterUnsupportedException;
+  void resolve() throws FilterUnsupportedException;
 
   /**
    * This API will provide the left column filter expression
@@ -73,28 +69,6 @@ public interface FilterResolverIntf extends Serializable {
   MeasureColumnResolvedFilterInfo getMsrColResolvedFilterInfo();
 
   /**
-   * API will get the start key based on the filter applied based on the key generator
-   *
-   * @param segmentProperties
-   * @param startKey
-   * @param setOfStartKeyByteArray
-   * @param startKeyList
-   */
-  void getStartKey(SegmentProperties segmentProperties, long[] startKey,
-      SortedMap<Integer, byte[]> setOfStartKeyByteArray, List<long[]> startKeyList);
-
-  /**
-   * API will read the end key based on the max surrogate of
-   * particular dimension column
-   *
-   * @param endKeys
-   * @param setOfEndKeyByteArray
-   * @return
-   */
-  void getEndKey(SegmentProperties segmentProperties, long[] endKeys,
-      SortedMap<Integer, byte[]> setOfEndKeyByteArray, List<long[]> endKeyList);
-
-  /**
    * API will return the filter executer type which will be used to evaluate
    * the resolved filter while query execution
    *
diff --git a/core/src/main/java/org/apache/carbondata/core/scan/filter/resolver/LogicalFilterResolverImpl.java b/core/src/main/java/org/apache/carbondata/core/scan/filter/resolver/LogicalFilterResolverImpl.java
index 53aad87..c8f4106 100644
--- a/core/src/main/java/org/apache/carbondata/core/scan/filter/resolver/LogicalFilterResolverImpl.java
+++ b/core/src/main/java/org/apache/carbondata/core/scan/filter/resolver/LogicalFilterResolverImpl.java
@@ -17,11 +17,6 @@
 
 package org.apache.carbondata.core.scan.filter.resolver;
 
-import java.util.List;
-import java.util.SortedMap;
-
-import org.apache.carbondata.core.datastore.block.SegmentProperties;
-import org.apache.carbondata.core.metadata.AbsoluteTableIdentifier;
 import org.apache.carbondata.core.scan.expression.BinaryExpression;
 import org.apache.carbondata.core.scan.expression.Expression;
 import org.apache.carbondata.core.scan.filter.intf.ExpressionType;
@@ -55,10 +50,9 @@ public class LogicalFilterResolverImpl implements FilterResolverIntf {
    * Logical filter resolver will return the left and right filter expresison
    * node for filter evaluation, so in this instance no implementation is required.
    *
-   * @param absoluteTableIdentifier
    */
   @Override
-  public void resolve(AbsoluteTableIdentifier absoluteTableIdentifier) {
+  public void resolve() {
 
   }
 
@@ -93,18 +87,6 @@ public class LogicalFilterResolverImpl implements FilterResolverIntf {
   }
 
   @Override
-  public void getStartKey(SegmentProperties segmentProperties, long[] startKey,
-      SortedMap<Integer, byte[]> setOfStartKeyByteArray, List<long[]> startKeyList) {
-
-  }
-
-  @Override
-  public void getEndKey(SegmentProperties segmentProperties, long[] endKeys,
-      SortedMap<Integer, byte[]> setOfEndKeyByteArray, List<long[]> endKeyList) {
-
-  }
-
-  @Override
   public FilterExecuterType getFilterExecuterType() {
     switch (filterExpressionType) {
       case OR:
diff --git a/core/src/main/java/org/apache/carbondata/core/scan/filter/resolver/RowLevelFilterResolverImpl.java b/core/src/main/java/org/apache/carbondata/core/scan/filter/resolver/RowLevelFilterResolverImpl.java
index 4075fd9..d2ae50f 100644
--- a/core/src/main/java/org/apache/carbondata/core/scan/filter/resolver/RowLevelFilterResolverImpl.java
+++ b/core/src/main/java/org/apache/carbondata/core/scan/filter/resolver/RowLevelFilterResolverImpl.java
@@ -51,7 +51,7 @@ public class RowLevelFilterResolverImpl extends ConditionalFilterResolverImpl {
    * Method which will resolve the filter expression by converting the filter member
    * to its assigned dictionary values.
    */
-  public void resolve(AbsoluteTableIdentifier absoluteTableIdentifier) {
+  public void resolve() {
     DimColumnResolvedFilterInfo dimColumnEvaluatorInfo = null;
     MeasureColumnResolvedFilterInfo msrColumnEvalutorInfo = null;
     int index = 0;
diff --git a/core/src/main/java/org/apache/carbondata/core/scan/filter/resolver/RowLevelRangeFilterResolverImpl.java b/core/src/main/java/org/apache/carbondata/core/scan/filter/resolver/RowLevelRangeFilterResolverImpl.java
index e630c03..96ae473 100644
--- a/core/src/main/java/org/apache/carbondata/core/scan/filter/resolver/RowLevelRangeFilterResolverImpl.java
+++ b/core/src/main/java/org/apache/carbondata/core/scan/filter/resolver/RowLevelRangeFilterResolverImpl.java
@@ -21,7 +21,6 @@ import java.util.ArrayList;
 import java.util.Collections;
 import java.util.Comparator;
 import java.util.List;
-import java.util.SortedMap;
 
 import org.apache.carbondata.core.constants.CarbonCommonConstants;
 import org.apache.carbondata.core.datastore.block.SegmentProperties;
@@ -117,53 +116,6 @@ public class RowLevelRangeFilterResolverImpl extends ConditionalFilterResolverIm
     return null;
   }
 
-  /**
-   * method will get the start key based on the filter surrogates
-   *
-   * @return start IndexKey
-   */
-  public void getStartKey(SegmentProperties segmentProperties, long[] startKey,
-      SortedMap<Integer, byte[]> noDictStartKeys, List<long[]> startKeyList) {
-    switch (exp.getFilterExpressionType()) {
-      case GREATERTHAN:
-      case GREATERTHAN_EQUALTO:
-        if (dimColEvaluatorInfoList.size() > 0) {
-          FilterUtil
-              .getStartKey(dimColEvaluatorInfoList.get(0).getDimensionResolvedFilterInstance(),
-                  segmentProperties, startKey, startKeyList);
-          FilterUtil.getStartKeyForNoDictionaryDimension(dimColEvaluatorInfoList.get(0),
-              segmentProperties, noDictStartKeys);
-        }
-        break;
-      default:
-        //do nothing
-    }
-  }
-
-  /**
-   * method will get the start key based on the filter surrogates
-   *
-   * @return end IndexKey
-   */
-  @Override
-  public void getEndKey(SegmentProperties segmentProperties, long[] endKeys,
-      SortedMap<Integer, byte[]> noDicEndKeys, List<long[]> endKeyList) {
-    switch (exp.getFilterExpressionType()) {
-      case LESSTHAN:
-      case LESSTHAN_EQUALTO:
-        if (dimColEvaluatorInfoList.size() > 0) {
-          FilterUtil.getEndKey(dimColEvaluatorInfoList.get(0).getDimensionResolvedFilterInstance(),
-              endKeys, segmentProperties, endKeyList);
-          FilterUtil
-              .getEndKeyForNoDictionaryDimension(dimColEvaluatorInfoList.get(0), segmentProperties,
-                  noDicEndKeys);
-        }
-        break;
-      default:
-        //do nothing
-    }
-  }
-
   private List<byte[]> getNoDictionaryRangeValues() {
     List<ExpressionResult> listOfExpressionResults = new ArrayList<ExpressionResult>(20);
     if (this.getFilterExpression() instanceof BinaryConditionalExpression) {
@@ -238,7 +190,7 @@ public class RowLevelRangeFilterResolverImpl extends ConditionalFilterResolverIm
    * Method which will resolve the filter expression by converting the filter
    * member to its assigned dictionary values.
    */
-  public void resolve(AbsoluteTableIdentifier absoluteTableIdentifier)
+  public void resolve()
       throws FilterUnsupportedException {
     DimColumnResolvedFilterInfo dimColumnEvaluatorInfo = null;
     MeasureColumnResolvedFilterInfo msrColumnEvalutorInfo = null;
diff --git a/core/src/main/java/org/apache/carbondata/core/scan/filter/resolver/metadata/FilterResolverMetadata.java b/core/src/main/java/org/apache/carbondata/core/scan/filter/resolver/metadata/FilterResolverMetadata.java
index 9913496..37f6780 100644
--- a/core/src/main/java/org/apache/carbondata/core/scan/filter/resolver/metadata/FilterResolverMetadata.java
+++ b/core/src/main/java/org/apache/carbondata/core/scan/filter/resolver/metadata/FilterResolverMetadata.java
@@ -17,25 +17,15 @@
 
 package org.apache.carbondata.core.scan.filter.resolver.metadata;
 
-import org.apache.carbondata.core.metadata.AbsoluteTableIdentifier;
 import org.apache.carbondata.core.scan.expression.ColumnExpression;
 import org.apache.carbondata.core.scan.expression.Expression;
 
 public class FilterResolverMetadata {
-  private AbsoluteTableIdentifier tableIdentifier;
   private Expression expression;
   private ColumnExpression columnExpression;
 
   private boolean isIncludeFilter;
 
-  public AbsoluteTableIdentifier getTableIdentifier() {
-    return tableIdentifier;
-  }
-
-  public void setTableIdentifier(AbsoluteTableIdentifier tableIdentifier) {
-    this.tableIdentifier = tableIdentifier;
-  }
-
   public Expression getExpression() {
     return expression;
   }
diff --git a/core/src/main/java/org/apache/carbondata/core/scan/filter/resolver/resolverinfo/FalseConditionalResolverImpl.java b/core/src/main/java/org/apache/carbondata/core/scan/filter/resolver/resolverinfo/FalseConditionalResolverImpl.java
index 66b94af..838e1ab 100644
--- a/core/src/main/java/org/apache/carbondata/core/scan/filter/resolver/resolverinfo/FalseConditionalResolverImpl.java
+++ b/core/src/main/java/org/apache/carbondata/core/scan/filter/resolver/resolverinfo/FalseConditionalResolverImpl.java
@@ -17,7 +17,6 @@
 
 package org.apache.carbondata.core.scan.filter.resolver.resolverinfo;
 
-import org.apache.carbondata.core.metadata.AbsoluteTableIdentifier;
 import org.apache.carbondata.core.scan.expression.Expression;
 import org.apache.carbondata.core.scan.filter.intf.FilterExecuterType;
 import org.apache.carbondata.core.scan.filter.resolver.ConditionalFilterResolverImpl;
@@ -33,7 +32,7 @@ public class FalseConditionalResolverImpl extends ConditionalFilterResolverImpl
   }
 
   @Override
-  public void resolve(AbsoluteTableIdentifier absoluteTableIdentifier) {
+  public void resolve() {
   }
 
   /**
diff --git a/core/src/main/java/org/apache/carbondata/core/scan/filter/resolver/resolverinfo/TrueConditionalResolverImpl.java b/core/src/main/java/org/apache/carbondata/core/scan/filter/resolver/resolverinfo/TrueConditionalResolverImpl.java
index 406efcc..d449be2 100644
--- a/core/src/main/java/org/apache/carbondata/core/scan/filter/resolver/resolverinfo/TrueConditionalResolverImpl.java
+++ b/core/src/main/java/org/apache/carbondata/core/scan/filter/resolver/resolverinfo/TrueConditionalResolverImpl.java
@@ -17,7 +17,6 @@
 
 package org.apache.carbondata.core.scan.filter.resolver.resolverinfo;
 
-import org.apache.carbondata.core.metadata.AbsoluteTableIdentifier;
 import org.apache.carbondata.core.scan.expression.Expression;
 import org.apache.carbondata.core.scan.filter.intf.FilterExecuterType;
 import org.apache.carbondata.core.scan.filter.resolver.ConditionalFilterResolverImpl;
@@ -33,7 +32,7 @@ public class TrueConditionalResolverImpl extends ConditionalFilterResolverImpl {
   }
 
   @Override
-  public void resolve(AbsoluteTableIdentifier absoluteTableIdentifier) {
+  public void resolve() {
   }
 
   /**
diff --git a/core/src/main/java/org/apache/carbondata/core/scan/processor/BlockScan.java b/core/src/main/java/org/apache/carbondata/core/scan/processor/BlockScan.java
deleted file mode 100644
index 330d25d..0000000
--- a/core/src/main/java/org/apache/carbondata/core/scan/processor/BlockScan.java
+++ /dev/null
@@ -1,104 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.carbondata.core.scan.processor;
-
-import java.util.LinkedList;
-import java.util.List;
-
-import org.apache.carbondata.core.datastore.DataRefNode;
-import org.apache.carbondata.core.datastore.FileReader;
-import org.apache.carbondata.core.scan.collector.ResultCollectorFactory;
-import org.apache.carbondata.core.scan.collector.ScannedResultCollector;
-import org.apache.carbondata.core.scan.executor.infos.BlockExecutionInfo;
-import org.apache.carbondata.core.scan.result.BlockletScannedResult;
-import org.apache.carbondata.core.scan.result.vector.CarbonColumnarBatch;
-import org.apache.carbondata.core.scan.scanner.BlockletScanner;
-import org.apache.carbondata.core.scan.scanner.impl.BlockletFilterScanner;
-import org.apache.carbondata.core.scan.scanner.impl.BlockletFullScanner;
-import org.apache.carbondata.core.stats.QueryStatisticsModel;
-
-public class BlockScan {
-  private BlockExecutionInfo blockExecutionInfo;
-  private FileReader fileReader;
-  private BlockletScanner blockletScanner;
-  private BlockletIterator blockletIterator;
-  private ScannedResultCollector scannerResultAggregator;
-
-  private List<BlockletScannedResult> scannedResults = new LinkedList<>();
-  private int nextResultIndex = 0;
-  private BlockletScannedResult curResult;
-
-  public BlockScan(BlockExecutionInfo blockExecutionInfo, FileReader fileReader,
-      QueryStatisticsModel queryStatisticsModel) {
-    this.blockExecutionInfo = blockExecutionInfo;
-    this.blockExecutionInfo.setQueryStatisticsModel(queryStatisticsModel);
-    this.fileReader = fileReader;
-    this.blockletIterator = new BlockletIterator(blockExecutionInfo.getFirstDataBlock(),
-        blockExecutionInfo.getNumberOfBlockToScan());
-    if (blockExecutionInfo.getFilterExecuterTree() != null) {
-      blockletScanner = new BlockletFilterScanner(blockExecutionInfo, queryStatisticsModel);
-    } else {
-      blockletScanner = new BlockletFullScanner(blockExecutionInfo, queryStatisticsModel);
-    }
-    this.scannerResultAggregator =
-        ResultCollectorFactory.getScannedResultCollector(blockExecutionInfo);
-  }
-
-  public void scan() throws Exception {
-    BlockletScannedResult blockletScannedResult = null;
-    while (blockletIterator.hasNext()) {
-      DataRefNode dataBlock = blockletIterator.next();
-      if (dataBlock.getColumnsMaxValue() == null || blockletScanner.isScanRequired(dataBlock)) {
-        RawBlockletColumnChunks rawBlockletColumnChunks =  RawBlockletColumnChunks.newInstance(
-            blockExecutionInfo.getTotalNumberDimensionToRead(),
-            blockExecutionInfo.getTotalNumberOfMeasureToRead(), fileReader, dataBlock);
-        blockletScanner.readBlocklet(rawBlockletColumnChunks);
-        blockletScannedResult = blockletScanner.scanBlocklet(rawBlockletColumnChunks);
-        if (blockletScannedResult != null && blockletScannedResult.hasNext()) {
-          scannedResults.add(blockletScannedResult);
-        }
-      }
-    }
-    fileReader.finish();
-  }
-
-  public boolean hasNext() {
-    if (curResult != null && curResult.hasNext()) {
-      return true;
-    } else {
-      if (null != curResult) {
-        curResult.freeMemory();
-      }
-      if (nextResultIndex < scannedResults.size()) {
-        curResult = scannedResults.get(nextResultIndex++);
-        return true;
-      } else {
-        return false;
-      }
-    }
-  }
-
-  public void processNextBatch(CarbonColumnarBatch columnarBatch) {
-    this.scannerResultAggregator.collectResultInColumnarBatch(curResult, columnarBatch);
-  }
-
-  public List<Object[]> next(int size) {
-    return this.scannerResultAggregator.collectResultInRow(curResult, size);
-  }
-
-}
diff --git a/core/src/main/java/org/apache/carbondata/core/scan/result/BlockletScannedResult.java b/core/src/main/java/org/apache/carbondata/core/scan/result/BlockletScannedResult.java
index 6c0ab4d..6172e09 100644
--- a/core/src/main/java/org/apache/carbondata/core/scan/result/BlockletScannedResult.java
+++ b/core/src/main/java/org/apache/carbondata/core/scan/result/BlockletScannedResult.java
@@ -217,24 +217,6 @@ public abstract class BlockletScannedResult {
 
   /**
    * Below method will be used to get the key for all the dictionary dimensions
-   * which is present in the query
-   *
-   * @param rowId row id selected after scanning
-   * @return return the dictionary key
-   */
-  protected byte[] getDictionaryKeyArray(int rowId) {
-    byte[] completeKey = new byte[fixedLengthKeySize];
-    int offset = 0;
-    for (int i = 0; i < this.dictionaryColumnChunkIndexes.length; i++) {
-      offset += dimensionColumnPages[dictionaryColumnChunkIndexes[i]][pageCounter].fillRawData(
-          rowId, offset, completeKey);
-    }
-    rowCounter++;
-    return completeKey;
-  }
-
-  /**
-   * Below method will be used to get the key for all the dictionary dimensions
    * in integer array format which is present in the query
    *
    * @param rowId row id selected after scanning
@@ -747,12 +729,6 @@ public abstract class BlockletScannedResult {
   public abstract int getCurrentRowId();
 
   /**
-   * @return dictionary key array for all the dictionary dimension
-   * selected in query
-   */
-  public abstract byte[] getDictionaryKeyArray();
-
-  /**
    * @return dictionary key array for all the dictionary dimension in integer array forat
    * selected in query
    */
diff --git a/core/src/main/java/org/apache/carbondata/core/scan/result/impl/FilterQueryScannedResult.java b/core/src/main/java/org/apache/carbondata/core/scan/result/impl/FilterQueryScannedResult.java
index f338888..0e5ddb9 100644
--- a/core/src/main/java/org/apache/carbondata/core/scan/result/impl/FilterQueryScannedResult.java
+++ b/core/src/main/java/org/apache/carbondata/core/scan/result/impl/FilterQueryScannedResult.java
@@ -37,15 +37,6 @@ public class FilterQueryScannedResult extends BlockletScannedResult {
   }
 
   /**
-   * @return dictionary key array for all the dictionary dimension
-   * selected in query
-   */
-  @Override
-  public byte[] getDictionaryKeyArray() {
-    return getDictionaryKeyArray(pageFilteredRowId[pageCounter][currentRow]);
-  }
-
-  /**
    * @return dictionary key integer array for all the dictionary dimension
    * selected in query
    */
diff --git a/core/src/main/java/org/apache/carbondata/core/scan/result/impl/NonFilterQueryScannedResult.java b/core/src/main/java/org/apache/carbondata/core/scan/result/impl/NonFilterQueryScannedResult.java
index 98576fa..a9c243c 100644
--- a/core/src/main/java/org/apache/carbondata/core/scan/result/impl/NonFilterQueryScannedResult.java
+++ b/core/src/main/java/org/apache/carbondata/core/scan/result/impl/NonFilterQueryScannedResult.java
@@ -36,16 +36,6 @@ public class NonFilterQueryScannedResult extends BlockletScannedResult {
   }
 
   /**
-   * @return dictionary key array for all the dictionary dimension selected in
-   * query
-   */
-  @Override
-  public byte[] getDictionaryKeyArray() {
-    ++currentRow;
-    return getDictionaryKeyArray(currentRow);
-  }
-
-  /**
    * @return dictionary key integer array for all the dictionary dimension
    * selected in query
    */
diff --git a/core/src/main/java/org/apache/carbondata/core/scan/result/iterator/RawResultIterator.java b/core/src/main/java/org/apache/carbondata/core/scan/result/iterator/RawResultIterator.java
index ac4a159..704df8f 100644
--- a/core/src/main/java/org/apache/carbondata/core/scan/result/iterator/RawResultIterator.java
+++ b/core/src/main/java/org/apache/carbondata/core/scan/result/iterator/RawResultIterator.java
@@ -29,7 +29,6 @@ import org.apache.carbondata.common.logging.LogServiceFactory;
 import org.apache.carbondata.core.constants.CarbonCommonConstants;
 import org.apache.carbondata.core.datastore.block.SegmentProperties;
 import org.apache.carbondata.core.scan.result.RowBatch;
-import org.apache.carbondata.core.scan.wrappers.ByteArrayWrapper;
 import org.apache.carbondata.core.util.CarbonProperties;
 
 import org.apache.log4j.Logger;
@@ -201,11 +200,6 @@ public class RawResultIterator extends CarbonIterator<Object[]> {
   }
 
   protected Object[] convertRow(Object[] rawRow) {
-    byte[] dims = ((ByteArrayWrapper) rawRow[0]).getDictionaryKey();
-    long[] keyArray = sourceSegProperties.getDimensionKeyGenerator().getKeyArray(dims);
-    byte[] covertedBytes =
-        destinationSegProperties.getDimensionKeyGenerator().generateKey(keyArray);
-    ((ByteArrayWrapper) rawRow[0]).setDictionaryKey(covertedBytes);
     return rawRow;
   }
 
diff --git a/core/src/main/java/org/apache/carbondata/core/stream/StreamPruner.java b/core/src/main/java/org/apache/carbondata/core/stream/StreamPruner.java
index e8790ee..96c9667 100644
--- a/core/src/main/java/org/apache/carbondata/core/stream/StreamPruner.java
+++ b/core/src/main/java/org/apache/carbondata/core/stream/StreamPruner.java
@@ -64,13 +64,8 @@ public class StreamPruner {
       // prepare cardinality of all dimensions
       List<ColumnSchema> listOfColumns =
           carbonTable.getTableInfo().getFactTable().getListOfColumns();
-      int[] columnCardinality = new int[listOfColumns.size()];
-      for (int index = 0; index < columnCardinality.length; index++) {
-        columnCardinality[index] = Integer.MAX_VALUE;
-      }
       // initial filter executor
-      SegmentProperties segmentProperties =
-          new SegmentProperties(listOfColumns, columnCardinality);
+      SegmentProperties segmentProperties = new SegmentProperties(listOfColumns);
       filterExecuter = FilterUtil.getFilterExecuterTree(
           filterExp, segmentProperties, null, minMaxCacheColumns, false);
     }
diff --git a/core/src/main/java/org/apache/carbondata/core/util/AbstractDataFileFooterConverter.java b/core/src/main/java/org/apache/carbondata/core/util/AbstractDataFileFooterConverter.java
index e1a4ee2..91cc456 100644
--- a/core/src/main/java/org/apache/carbondata/core/util/AbstractDataFileFooterConverter.java
+++ b/core/src/main/java/org/apache/carbondata/core/util/AbstractDataFileFooterConverter.java
@@ -18,22 +18,16 @@
 package org.apache.carbondata.core.util;
 
 import java.io.IOException;
-import java.nio.ByteBuffer;
 import java.util.ArrayList;
 import java.util.Arrays;
-import java.util.BitSet;
 import java.util.List;
 import java.util.Map;
 
 import org.apache.carbondata.core.constants.CarbonCommonConstants;
-import org.apache.carbondata.core.datastore.block.BlockInfo;
 import org.apache.carbondata.core.datastore.block.TableBlockInfo;
 import org.apache.carbondata.core.metadata.ColumnarFormatVersion;
-import org.apache.carbondata.core.metadata.ValueEncoderMeta;
 import org.apache.carbondata.core.metadata.blocklet.BlockletInfo;
 import org.apache.carbondata.core.metadata.blocklet.DataFileFooter;
-import org.apache.carbondata.core.metadata.blocklet.SegmentInfo;
-import org.apache.carbondata.core.metadata.blocklet.datachunk.DataChunk;
 import org.apache.carbondata.core.metadata.blocklet.index.BlockletBTreeIndex;
 import org.apache.carbondata.core.metadata.blocklet.index.BlockletIndex;
 import org.apache.carbondata.core.metadata.blocklet.index.BlockletMinMaxIndex;
@@ -63,23 +57,6 @@ public abstract class AbstractDataFileFooterConverter {
   }
 
   /**
-   * Below method will be used to convert the thrift presence meta to wrapper
-   * presence meta
-   *
-   * @param presentMetadataThrift
-   * @return wrapper presence meta
-   */
-  private static BitSet getPresenceMeta(
-      org.apache.carbondata.format.PresenceMeta presentMetadataThrift) {
-    final byte[] present_bit_stream = presentMetadataThrift.getPresent_bit_stream();
-    if (null != present_bit_stream) {
-      return BitSet.valueOf(present_bit_stream);
-    } else {
-      return new BitSet(1);
-    }
-  }
-
-  /**
    * Below method will be used to get the index info from index file
    *
    * @param filePath           file path of the index file
@@ -104,7 +81,6 @@ public abstract class AbstractDataFileFooterConverter {
         columnSchemaList.add(thriftColumnSchemaToWrapperColumnSchema(table_columns.get(i)));
       }
       // get the segment info
-      SegmentInfo segmentInfo = getSegmentInfo(readIndexHeader.getSegment_info());
       BlockletIndex blockletIndex = null;
       int counter = 0;
       int index = 0;
@@ -120,13 +96,10 @@ public abstract class AbstractDataFileFooterConverter {
           tableBlockInfo.setBlockOffset(readBlockIndexInfo.getOffset());
           tableBlockInfo.setVersion(
               ColumnarFormatVersion.valueOf((short) readIndexHeader.getVersion()));
-          int blockletSize = getBlockletSize(readBlockIndexInfo);
-          tableBlockInfo.getBlockletInfos().setNoOfBlockLets(blockletSize);
           dataFileFooter.setBlockletIndex(blockletIndex);
           dataFileFooter.setColumnInTable(columnSchemaList);
           dataFileFooter.setNumberOfRows(readBlockIndexInfo.getNum_rows());
-          dataFileFooter.setBlockInfo(new BlockInfo(tableBlockInfo));
-          dataFileFooter.setSegmentInfo(segmentInfo);
+          dataFileFooter.setBlockInfo(tableBlockInfo);
           if (readIndexHeader.isSetIs_sort()) {
             dataFileFooter.setSorted(readIndexHeader.isIs_sort());
           } else {
@@ -183,8 +156,6 @@ public abstract class AbstractDataFileFooterConverter {
       if (!isTransactionalTable) {
         QueryUtil.updateColumnUniqueIdForNonTransactionTable(columnSchemaList);
       }
-      // get the segment info
-      SegmentInfo segmentInfo = getSegmentInfo(readIndexHeader.getSegment_info());
       BlockletIndex blockletIndex = null;
       DataFileFooter dataFileFooter = null;
       // read the block info from file
@@ -197,8 +168,7 @@ public abstract class AbstractDataFileFooterConverter {
         dataFileFooter.setBlockletIndex(blockletIndex);
         dataFileFooter.setColumnInTable(columnSchemaList);
         dataFileFooter.setNumberOfRows(readBlockIndexInfo.getNum_rows());
-        dataFileFooter.setBlockInfo(new BlockInfo(tableBlockInfo));
-        dataFileFooter.setSegmentInfo(segmentInfo);
+        dataFileFooter.setBlockInfo(tableBlockInfo);
         dataFileFooter.setVersionId(tableBlockInfo.getVersion());
         // In case of old schema time stamp will not be found in the index header
         if (readIndexHeader.isSetSchema_time_stamp()) {
@@ -236,8 +206,6 @@ public abstract class AbstractDataFileFooterConverter {
     ColumnarFormatVersion version =
         ColumnarFormatVersion.valueOf((short) readIndexHeader.getVersion());
     tableBlockInfo.setVersion(version);
-    int blockletSize = getBlockletSize(readBlockIndexInfo);
-    tableBlockInfo.getBlockletInfos().setNoOfBlockLets(blockletSize);
     String fileName = readBlockIndexInfo.file_name;
     // Take only name of file.
     if (fileName.lastIndexOf("/") > 0) {
@@ -252,27 +220,6 @@ public abstract class AbstractDataFileFooterConverter {
   }
 
   /**
-   * the methods returns the number of blocklets in a block
-   *
-   * @param readBlockIndexInfo
-   * @return
-   */
-  protected int getBlockletSize(BlockIndex readBlockIndexInfo) {
-    long num_rows = readBlockIndexInfo.getNum_rows();
-    int blockletSize = Integer.parseInt(CarbonProperties.getInstance()
-        .getProperty(CarbonCommonConstants.BLOCKLET_SIZE,
-            CarbonCommonConstants.BLOCKLET_SIZE_DEFAULT_VAL));
-    int remainder = (int) (num_rows % blockletSize);
-    int noOfBlockLet = (int) (num_rows / blockletSize);
-    // there could be some blocklets which will not
-    // contain the total records equal to the blockletSize
-    if (remainder > 0) {
-      noOfBlockLet = noOfBlockLet + 1;
-    }
-    return noOfBlockLet;
-  }
-
-  /**
    * Below method will be used to convert thrift file meta to wrapper file meta
    */
   public abstract DataFileFooter readDataFileFooter(TableBlockInfo tableBlockInfo)
@@ -415,23 +362,6 @@ public abstract class AbstractDataFileFooterConverter {
   }
 
   /**
-   * Below method will be used to convert thrift segment object to wrapper
-   * segment object
-   *
-   * @param segmentInfo thrift segment info object
-   * @return wrapper segment info object
-   */
-  protected SegmentInfo getSegmentInfo(org.apache.carbondata.format.SegmentInfo segmentInfo) {
-    SegmentInfo info = new SegmentInfo();
-    int[] cardinality = new int[segmentInfo.getColumn_cardinalities().size()];
-    for (int i = 0; i < cardinality.length; i++) {
-      cardinality[i] = segmentInfo.getColumn_cardinalities().get(i);
-    }
-    info.setColumnCardinality(cardinality);
-    return info;
-  }
-
-  /**
    * Below method will be used to convert the blocklet index of thrift to
    * wrapper
    *
@@ -459,41 +389,4 @@ public abstract class AbstractDataFileFooterConverter {
             isMinMaxSet));
   }
 
-  /**
-   * Below method will be used to convert the thrift data chunk to wrapper
-   * data chunk
-   *
-   * @param datachunkThrift
-   * @return wrapper data chunk
-   */
-  protected DataChunk getDataChunk(org.apache.carbondata.format.DataChunk datachunkThrift,
-      boolean isPresenceMetaPresent) {
-    DataChunk dataChunk = new DataChunk();
-    dataChunk.setDataPageLength(datachunkThrift.getData_page_length());
-    dataChunk.setDataPageOffset(datachunkThrift.getData_page_offset());
-    if (isPresenceMetaPresent) {
-      dataChunk.setNullValueIndexForColumn(getPresenceMeta(datachunkThrift.getPresence()));
-    }
-    dataChunk.setRlePageLength(datachunkThrift.getRle_page_length());
-    dataChunk.setRlePageOffset(datachunkThrift.getRle_page_offset());
-    dataChunk.setRowMajor(datachunkThrift.isRowMajor());
-    dataChunk.setRowIdPageLength(datachunkThrift.getRowid_page_length());
-    dataChunk.setRowIdPageOffset(datachunkThrift.getRowid_page_offset());
-    List<Encoding> encodingList = new ArrayList<Encoding>(datachunkThrift.getEncoders().size());
-    for (int i = 0; i < datachunkThrift.getEncoders().size(); i++) {
-      encodingList.add(fromExternalToWrapperEncoding(datachunkThrift.getEncoders().get(i)));
-    }
-    dataChunk.setEncodingList(encodingList);
-    if (encodingList.contains(Encoding.DELTA)) {
-      List<ByteBuffer> thriftEncoderMeta = datachunkThrift.getEncoder_meta();
-      List<ValueEncoderMeta> encodeMetaList =
-          new ArrayList<ValueEncoderMeta>(thriftEncoderMeta.size());
-      for (int i = 0; i < thriftEncoderMeta.size(); i++) {
-        encodeMetaList.add(CarbonUtil.deserializeEncoderMetaV2(thriftEncoderMeta.get(i).array()));
-      }
-      dataChunk.setValueEncoderMeta(encodeMetaList);
-    }
-    return dataChunk;
-  }
-
 }
diff --git a/core/src/main/java/org/apache/carbondata/core/util/BlockletDataMapUtil.java b/core/src/main/java/org/apache/carbondata/core/util/BlockletDataMapUtil.java
index f214637..4a7bab6 100644
--- a/core/src/main/java/org/apache/carbondata/core/util/BlockletDataMapUtil.java
+++ b/core/src/main/java/org/apache/carbondata/core/util/BlockletDataMapUtil.java
@@ -24,8 +24,6 @@ import java.io.DataInputStream;
 import java.io.DataOutput;
 import java.io.DataOutputStream;
 import java.io.IOException;
-import java.math.BigDecimal;
-import java.nio.ByteBuffer;
 import java.util.ArrayList;
 import java.util.HashMap;
 import java.util.HashSet;
@@ -52,8 +50,6 @@ import org.apache.carbondata.core.indexstore.blockletindex.BlockletDataMapFactor
 import org.apache.carbondata.core.indexstore.blockletindex.SegmentIndexFileStore;
 import org.apache.carbondata.core.metadata.blocklet.DataFileFooter;
 import org.apache.carbondata.core.metadata.blocklet.index.BlockletMinMaxIndex;
-import org.apache.carbondata.core.metadata.datatype.DataType;
-import org.apache.carbondata.core.metadata.datatype.DataTypes;
 import org.apache.carbondata.core.metadata.schema.table.CarbonTable;
 import org.apache.carbondata.core.metadata.schema.table.column.CarbonColumn;
 import org.apache.carbondata.core.metadata.schema.table.column.CarbonDimension;
@@ -125,10 +121,10 @@ public class BlockletDataMapUtil {
         carbonTable.getTableInfo().getFactTable().setListOfColumns(footer.getColumnInTable());
         CarbonTable.updateTableByTableInfo(carbonTable, carbonTable.getTableInfo());
       }
-      String blockPath = footer.getBlockInfo().getTableBlockInfo().getFilePath();
+      String blockPath = footer.getBlockInfo().getFilePath();
       if (null == blockMetaInfoMap.get(blockPath)) {
         BlockMetaInfo blockMetaInfo = createBlockMetaInfo(
-            fileNameToMetaInfoMapping, footer.getBlockInfo().getTableBlockInfo());
+            fileNameToMetaInfoMapping, footer.getBlockInfo());
         // if blockMetaInfo is null that means the file has been deleted from the file system.
         // This can happen in case IUD scenarios where after deleting or updating the data the
         // complete block is deleted but the entry still exists in index or merge index file
@@ -301,84 +297,6 @@ public class BlockletDataMapUtil {
   }
 
   /**
-   * Fill the measures min values with minimum , this is needed for backward version compatability
-   * as older versions don't store min values for measures
-   */
-  public static byte[][] updateMinValues(SegmentProperties segmentProperties, byte[][] minValues) {
-    byte[][] updatedValues = minValues;
-    int[] minMaxLen = segmentProperties.getColumnsValueSize();
-    if (minValues.length < minMaxLen.length) {
-      updatedValues = new byte[minMaxLen.length][];
-      System.arraycopy(minValues, 0, updatedValues, 0, minValues.length);
-      List<CarbonMeasure> measures = segmentProperties.getMeasures();
-      ByteBuffer buffer = ByteBuffer.allocate(8);
-      for (int i = 0; i < measures.size(); i++) {
-        buffer.rewind();
-        DataType dataType = measures.get(i).getDataType();
-        if (dataType == DataTypes.BYTE) {
-          buffer.putLong(Byte.MIN_VALUE);
-          updatedValues[minValues.length + i] = buffer.array().clone();
-        } else if (dataType == DataTypes.SHORT) {
-          buffer.putLong(Short.MIN_VALUE);
-          updatedValues[minValues.length + i] = buffer.array().clone();
-        } else if (dataType == DataTypes.INT) {
-          buffer.putLong(Integer.MIN_VALUE);
-          updatedValues[minValues.length + i] = buffer.array().clone();
-        } else if (dataType == DataTypes.LONG) {
-          buffer.putLong(Long.MIN_VALUE);
-          updatedValues[minValues.length + i] = buffer.array().clone();
-        } else if (DataTypes.isDecimal(dataType)) {
-          updatedValues[minValues.length + i] =
-              DataTypeUtil.bigDecimalToByte(BigDecimal.valueOf(Long.MIN_VALUE));
-        } else {
-          buffer.putDouble(Double.MIN_VALUE);
-          updatedValues[minValues.length + i] = buffer.array().clone();
-        }
-      }
-    }
-    return updatedValues;
-  }
-
-  /**
-   * Fill the measures max values with maximum , this is needed for backward version compatability
-   * as older versions don't store max values for measures
-   */
-  public static byte[][] updateMaxValues(SegmentProperties segmentProperties, byte[][] maxValues) {
-    byte[][] updatedValues = maxValues;
-    int[] minMaxLen = segmentProperties.getColumnsValueSize();
-    if (maxValues.length < minMaxLen.length) {
-      updatedValues = new byte[minMaxLen.length][];
-      System.arraycopy(maxValues, 0, updatedValues, 0, maxValues.length);
-      List<CarbonMeasure> measures = segmentProperties.getMeasures();
-      ByteBuffer buffer = ByteBuffer.allocate(8);
-      for (int i = 0; i < measures.size(); i++) {
-        buffer.rewind();
-        DataType dataType = measures.get(i).getDataType();
-        if (dataType == DataTypes.BYTE) {
-          buffer.putLong(Byte.MAX_VALUE);
-          updatedValues[maxValues.length + i] = buffer.array().clone();
-        } else if (dataType == DataTypes.SHORT) {
-          buffer.putLong(Short.MAX_VALUE);
-          updatedValues[maxValues.length + i] = buffer.array().clone();
-        } else if (dataType == DataTypes.INT) {
-          buffer.putLong(Integer.MAX_VALUE);
-          updatedValues[maxValues.length + i] = buffer.array().clone();
-        } else if (dataType == DataTypes.LONG) {
-          buffer.putLong(Long.MAX_VALUE);
-          updatedValues[maxValues.length + i] = buffer.array().clone();
-        } else if (DataTypes.isDecimal(dataType)) {
-          updatedValues[maxValues.length + i] =
-              DataTypeUtil.bigDecimalToByte(BigDecimal.valueOf(Long.MAX_VALUE));
-        } else {
-          buffer.putDouble(Double.MAX_VALUE);
-          updatedValues[maxValues.length + i] = buffer.array().clone();
-        }
-      }
-    }
-    return updatedValues;
-  }
-
-  /**
    * Convert schema to binary
    */
   public static byte[] convertSchemaToBinary(List<ColumnSchema> columnSchemas) throws IOException {
diff --git a/core/src/main/java/org/apache/carbondata/core/util/ByteUtil.java b/core/src/main/java/org/apache/carbondata/core/util/ByteUtil.java
index 72f4673..1cfeaaa 100644
--- a/core/src/main/java/org/apache/carbondata/core/util/ByteUtil.java
+++ b/core/src/main/java/org/apache/carbondata/core/util/ByteUtil.java
@@ -756,4 +756,45 @@ public final class ByteUtil {
         ((long) bytes[offset + 3] & 0xff) << 24) | (((long) bytes[offset + 2] & 0xff) << 16) | (
         ((long) bytes[offset + 1] & 0xff) << 8) | (((long) bytes[offset] & 0xff)));
   }
+
+  public static byte[] convertIntToBytes(int date) {
+    return ByteUtil.toBytes(date);
+  }
+
+  public static byte[] convertLongArrayToBytes(long[] date) {
+    byte[] output = new byte[date.length * 4];
+    for (int i = 0; i < date.length; i++) {
+      System.arraycopy(ByteUtil.toBytes(date[i]), 0, output, i * 4, 4);
+    }
+    return output;
+  }
+
+  public static int convertBytesToInt(byte[] date) {
+    return ByteUtil.toInt(date, 0);
+  }
+
+  public static int convertBytesToInt(byte[] date, int offset) {
+    return ByteUtil.toInt(date, offset);
+  }
+
+  public static int dateBytesSize() {
+    return 4;
+  }
+
+  public static int[] convertBytesToIntArray(byte[] input) {
+    int[] output = new int[input.length / 4];
+    for (int i = 0; i < output.length; i++) {
+      output[i] = convertBytesToInt(input, i * 4);
+    }
+    return output;
+  }
+
+  public static long[] convertBytesToLongArray(byte[] input) {
+    long[] output = new long[input.length / 4];
+    for (int i = 0; i < output.length; i++) {
+      output[i] = convertBytesToInt(input, i * 4);
+    }
+    return output;
+  }
+
 }
diff --git a/core/src/main/java/org/apache/carbondata/core/util/CarbonMetadataUtil.java b/core/src/main/java/org/apache/carbondata/core/util/CarbonMetadataUtil.java
index 5a69689..d1bef1a 100644
--- a/core/src/main/java/org/apache/carbondata/core/util/CarbonMetadataUtil.java
+++ b/core/src/main/java/org/apache/carbondata/core/util/CarbonMetadataUtil.java
@@ -24,6 +24,7 @@ import java.util.Arrays;
 import java.util.List;
 
 import org.apache.carbondata.common.logging.LogServiceFactory;
+import org.apache.carbondata.core.constants.CarbonCommonConstants;
 import org.apache.carbondata.core.datastore.blocklet.BlockletEncodedColumnPage;
 import org.apache.carbondata.core.datastore.blocklet.EncodedBlocklet;
 import org.apache.carbondata.core.datastore.compression.CompressorFactory;
@@ -62,6 +63,29 @@ public class CarbonMetadataUtil {
   private static final Logger LOGGER =
       LogServiceFactory.getLogService(CarbonMetadataUtil.class.getName());
 
+  // just for compatibility, not used
+  private static final List<Integer> dummyCardinality = new ArrayList<>(0);
+
+  // MDK start key, deprecated
+  private static final byte[] startKey = new byte[0];
+
+  // MDK no dictionary start key, deprecated
+  private static final byte[] noDictStartKey = new byte[0];
+
+  private static final byte[] DEPRECATED_MDK;
+
+  static  {
+    ByteBuffer buffer = ByteBuffer.allocate(
+        CarbonCommonConstants.INT_SIZE_IN_BYTE + CarbonCommonConstants.INT_SIZE_IN_BYTE
+            + startKey.length + noDictStartKey.length);
+    buffer.putInt(startKey.length);
+    buffer.putInt(noDictStartKey.length);
+    buffer.put(startKey);
+    buffer.put(noDictStartKey);
+    buffer.rewind();
+    DEPRECATED_MDK = buffer.array();
+  }
+
   private CarbonMetadataUtil() {
   }
 
@@ -70,13 +94,12 @@ public class CarbonMetadataUtil {
    *
    * @param infoList
    * @param blockletIndexs
-   * @param cardinalities
    * @param numberOfColumns
    * @return FileFooter
    */
   public static FileFooter3 convertFileFooterVersion3(List<BlockletInfo3> infoList,
-      List<BlockletIndex> blockletIndexs, int[] cardinalities, int numberOfColumns) {
-    FileFooter3 footer = getFileFooter3(infoList, blockletIndexs, cardinalities, numberOfColumns);
+      List<BlockletIndex> blockletIndexs, int numberOfColumns) {
+    FileFooter3 footer = getFileFooter3(infoList, blockletIndexs, numberOfColumns);
     for (BlockletInfo3 info : infoList) {
       footer.addToBlocklet_info_list3(info);
     }
@@ -88,15 +111,14 @@ public class CarbonMetadataUtil {
    *
    * @param infoList         blocklet info
    * @param blockletIndexs
-   * @param cardinalities    cardinlaity of dimension columns
    * @param numberOfColumns
    * @return file footer
    */
   private static FileFooter3 getFileFooter3(List<BlockletInfo3> infoList,
-      List<BlockletIndex> blockletIndexs, int[] cardinalities, int numberOfColumns) {
+      List<BlockletIndex> blockletIndexs, int numberOfColumns) {
     SegmentInfo segmentInfo = new SegmentInfo();
     segmentInfo.setNum_cols(numberOfColumns);
-    segmentInfo.setColumn_cardinalities(CarbonUtil.convertToIntegerList(cardinalities));
+    segmentInfo.setColumn_cardinalities(dummyCardinality);
     FileFooter3 footer = new FileFooter3();
     footer.setNum_rows(getNumberOfRowForFooter(infoList));
     footer.setSegment_info(segmentInfo);
@@ -263,12 +285,8 @@ public class CarbonMetadataUtil {
       blockletMinMaxIndex.addToMin_values(ByteBuffer.wrap(min));
     }
     BlockletBTreeIndex blockletBTreeIndex = new BlockletBTreeIndex();
-    byte[] startKey = encodedBlocklet.getPageMetadataList().get(0).serializeStartKey();
-    blockletBTreeIndex.setStart_key(startKey);
-    byte[] endKey =
-        encodedBlocklet.getPageMetadataList().get(encodedBlocklet.getPageMetadataList().size() - 1)
-            .serializeEndKey();
-    blockletBTreeIndex.setEnd_key(endKey);
+    blockletBTreeIndex.setStart_key(DEPRECATED_MDK);
+    blockletBTreeIndex.setEnd_key(DEPRECATED_MDK);
     BlockletIndex blockletIndex = new BlockletIndex();
     blockletIndex.setMin_max_index(blockletMinMaxIndex);
     blockletIndex.setB_tree_index(blockletBTreeIndex);
@@ -345,20 +363,19 @@ public class CarbonMetadataUtil {
   /**
    * Below method will be used to get the index header
    *
-   * @param columnCardinality cardinality of each column
    * @param columnSchemaList  list of column present in the table
    * @param bucketNumber
    * @param schemaTimeStamp current timestamp of schema
    * @return Index header object
    */
-  public static IndexHeader getIndexHeader(int[] columnCardinality,
+  public static IndexHeader getIndexHeader(
       List<ColumnSchema> columnSchemaList, int bucketNumber, long schemaTimeStamp) {
     // create segment info object
     SegmentInfo segmentInfo = new SegmentInfo();
     // set the number of columns
     segmentInfo.setNum_cols(columnSchemaList.size());
-    // setting the column cardinality
-    segmentInfo.setColumn_cardinalities(CarbonUtil.convertToIntegerList(columnCardinality));
+    // setting the column cardinality, deprecated
+    segmentInfo.setColumn_cardinalities(dummyCardinality);
     // create index header object
     IndexHeader indexHeader = new IndexHeader();
     ColumnarFormatVersion version = CarbonProperties.getInstance().getFormatVersion();
diff --git a/core/src/main/java/org/apache/carbondata/core/util/CarbonUtil.java b/core/src/main/java/org/apache/carbondata/core/util/CarbonUtil.java
index a01827a..e9bcdca 100644
--- a/core/src/main/java/org/apache/carbondata/core/util/CarbonUtil.java
+++ b/core/src/main/java/org/apache/carbondata/core/util/CarbonUtil.java
@@ -74,7 +74,6 @@ import org.apache.carbondata.core.metadata.ColumnarFormatVersion;
 import org.apache.carbondata.core.metadata.SegmentFileStore;
 import org.apache.carbondata.core.metadata.ValueEncoderMeta;
 import org.apache.carbondata.core.metadata.blocklet.DataFileFooter;
-import org.apache.carbondata.core.metadata.blocklet.SegmentInfo;
 import org.apache.carbondata.core.metadata.converter.SchemaConverter;
 import org.apache.carbondata.core.metadata.converter.ThriftWrapperSchemaConverterImpl;
 import org.apache.carbondata.core.metadata.datatype.DataType;
@@ -104,7 +103,6 @@ import org.apache.carbondata.core.statusmanager.SegmentStatusManager;
 import org.apache.carbondata.core.statusmanager.SegmentUpdateStatusManager;
 import org.apache.carbondata.core.util.path.CarbonTablePath;
 import org.apache.carbondata.format.BlockletHeader;
-import org.apache.carbondata.format.DataChunk2;
 import org.apache.carbondata.format.DataChunk3;
 import org.apache.carbondata.format.IndexHeader;
 
@@ -909,9 +907,6 @@ public final class CarbonUtil {
           DataFileFooterConverterFactory.getInstance().getDataFileFooterConverter(version);
       List<ColumnSchema> schema = dataFileFooterConverter.getSchema(tableBlockInfo);
       fileFooter.setColumnInTable(schema);
-      SegmentInfo segmentInfo = new SegmentInfo();
-      segmentInfo.setColumnCardinality(detailInfo.getDimLens());
-      fileFooter.setSegmentInfo(segmentInfo);
       return fileFooter;
     }
   }
@@ -935,7 +930,7 @@ public final class CarbonUtil {
 
     // find the footer of the input data file (tableBlockInfo)
     for (DataFileFooter footer : footers) {
-      if (footer.getBlockInfo().getTableBlockInfo().getFilePath().equals(dataFilePath)) {
+      if (footer.getBlockInfo().getFilePath().equals(dataFilePath)) {
         return footer;
       }
     }
@@ -1140,29 +1135,6 @@ public final class CarbonUtil {
     return currentBlockMeasure;
   }
 
-  /**
-   * @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(),
-          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>();
@@ -1437,17 +1409,6 @@ public final class CarbonUtil {
     return (DataChunk3) t;
   }
 
-  public static DataChunk2 readDataChunk(ByteBuffer dataChunkBuffer, int offset, int length)
-      throws IOException {
-    byte[] data = dataChunkBuffer.array();
-    return (DataChunk2) read(data, new ThriftReader.TBaseCreator() {
-      @Override
-      public TBase create() {
-        return new DataChunk2();
-      }
-    }, offset, length);
-  }
-
   /**
    * Below method will be used to convert the byte array value to thrift object for
    * data chunk
@@ -1716,7 +1677,8 @@ public final class CarbonUtil {
         surrogate ^= data[startOffsetOfData + 3] & 0xFF;
         return surrogate;
       default:
-        throw new IllegalArgumentException("Int cannot be more than 4 bytes");
+        throw new IllegalArgumentException("Int cannot be more than 4 bytes: " +
+            eachColumnValueSize);
     }
   }
 
diff --git a/core/src/main/java/org/apache/carbondata/core/util/DataFileFooterConverter.java b/core/src/main/java/org/apache/carbondata/core/util/DataFileFooterConverter.java
index 7e1924c..c739553 100644
--- a/core/src/main/java/org/apache/carbondata/core/util/DataFileFooterConverter.java
+++ b/core/src/main/java/org/apache/carbondata/core/util/DataFileFooterConverter.java
@@ -19,7 +19,6 @@ package org.apache.carbondata.core.util;
 
 import java.io.IOException;
 import java.util.ArrayList;
-import java.util.Iterator;
 import java.util.List;
 
 import org.apache.carbondata.core.datastore.FileReader;
@@ -28,7 +27,6 @@ import org.apache.carbondata.core.datastore.impl.FileFactory;
 import org.apache.carbondata.core.metadata.ColumnarFormatVersion;
 import org.apache.carbondata.core.metadata.blocklet.BlockletInfo;
 import org.apache.carbondata.core.metadata.blocklet.DataFileFooter;
-import org.apache.carbondata.core.metadata.blocklet.datachunk.DataChunk;
 import org.apache.carbondata.core.metadata.blocklet.index.BlockletIndex;
 import org.apache.carbondata.core.metadata.schema.table.column.ColumnSchema;
 import org.apache.carbondata.core.reader.CarbonFooterReader;
@@ -68,7 +66,6 @@ public class DataFileFooterConverter extends AbstractDataFileFooterConverter {
       FileFooter footer = reader.readFooter();
       dataFileFooter.setVersionId(ColumnarFormatVersion.valueOf((short) footer.getVersion()));
       dataFileFooter.setNumberOfRows(footer.getNum_rows());
-      dataFileFooter.setSegmentInfo(getSegmentInfo(footer.getSegment_info()));
       List<ColumnSchema> columnSchemaList = new ArrayList<ColumnSchema>();
       List<org.apache.carbondata.format.ColumnSchema> table_columns = footer.getTable_columns();
       for (int i = 0; i < table_columns.size(); i++) {
@@ -112,24 +109,6 @@ public class DataFileFooterConverter extends AbstractDataFileFooterConverter {
   private BlockletInfo getBlockletInfo(
       org.apache.carbondata.format.BlockletInfo blockletInfoThrift) {
     BlockletInfo blockletInfo = new BlockletInfo();
-    List<DataChunk> dimensionColumnChunk = new ArrayList<DataChunk>();
-    List<DataChunk> measureChunk = new ArrayList<DataChunk>();
-    Iterator<org.apache.carbondata.format.DataChunk> column_data_chunksIterator =
-        blockletInfoThrift.getColumn_data_chunksIterator();
-    if (null != column_data_chunksIterator) {
-      while (column_data_chunksIterator.hasNext()) {
-        org.apache.carbondata.format.DataChunk next = column_data_chunksIterator.next();
-        if (next.isRowMajor()) {
-          dimensionColumnChunk.add(getDataChunk(next, false));
-        } else if (next.getEncoders().contains(org.apache.carbondata.format.Encoding.DELTA)) {
-          measureChunk.add(getDataChunk(next, true));
-        } else {
-          dimensionColumnChunk.add(getDataChunk(next, false));
-        }
-      }
-    }
-    blockletInfo.setDimensionColumnChunk(dimensionColumnChunk);
-    blockletInfo.setMeasureColumnChunk(measureChunk);
     blockletInfo.setNumberOfRows(blockletInfoThrift.getNum_rows());
     return blockletInfo;
   }
diff --git a/core/src/main/java/org/apache/carbondata/core/util/DataFileFooterConverter2.java b/core/src/main/java/org/apache/carbondata/core/util/DataFileFooterConverter2.java
deleted file mode 100644
index b46ef24..0000000
--- a/core/src/main/java/org/apache/carbondata/core/util/DataFileFooterConverter2.java
+++ /dev/null
@@ -1,155 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.carbondata.core.util;
-
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.List;
-
-import org.apache.carbondata.core.datastore.block.TableBlockInfo;
-import org.apache.carbondata.core.datastore.impl.FileFactory;
-import org.apache.carbondata.core.metadata.ColumnarFormatVersion;
-import org.apache.carbondata.core.metadata.blocklet.BlockletInfo;
-import org.apache.carbondata.core.metadata.blocklet.DataFileFooter;
-import org.apache.carbondata.core.metadata.blocklet.index.BlockletIndex;
-import org.apache.carbondata.core.metadata.schema.table.column.ColumnSchema;
-import org.apache.carbondata.core.reader.CarbonFooterReader;
-import org.apache.carbondata.format.FileFooter;
-
-import org.apache.hadoop.conf.Configuration;
-
-/**
- * Below class will be used to convert the thrift object of data file
- * meta data to wrapper object for version 2 data file
- */
-
-public class DataFileFooterConverter2 extends AbstractDataFileFooterConverter {
-
-  public DataFileFooterConverter2(Configuration configuration) {
-    super(configuration);
-  }
-
-  public DataFileFooterConverter2() {
-    super(FileFactory.getConfiguration());
-  }
-
-  /**
-   * Below method will be used to convert thrift file meta to wrapper file meta
-   */
-  @Override
-  public DataFileFooter readDataFileFooter(TableBlockInfo tableBlockInfo)
-      throws IOException {
-    DataFileFooter dataFileFooter = new DataFileFooter();
-    CarbonFooterReader reader =
-        new CarbonFooterReader(tableBlockInfo.getFilePath(), tableBlockInfo.getBlockOffset());
-    FileFooter footer = reader.readFooter();
-    dataFileFooter.setVersionId(ColumnarFormatVersion.valueOf((short) footer.getVersion()));
-    dataFileFooter.setNumberOfRows(footer.getNum_rows());
-    dataFileFooter.setSegmentInfo(getSegmentInfo(footer.getSegment_info()));
-    List<ColumnSchema> columnSchemaList = new ArrayList<ColumnSchema>();
-    List<org.apache.carbondata.format.ColumnSchema> table_columns = footer.getTable_columns();
-    for (int i = 0; i < table_columns.size(); i++) {
-      columnSchemaList.add(thriftColumnSchemaToWrapperColumnSchema(table_columns.get(i)));
-    }
-    dataFileFooter.setColumnInTable(columnSchemaList);
-
-    List<org.apache.carbondata.format.BlockletIndex> leaf_node_indices_Thrift =
-        footer.getBlocklet_index_list();
-    List<BlockletIndex> blockletIndexList = new ArrayList<BlockletIndex>();
-    for (int i = 0; i < leaf_node_indices_Thrift.size(); i++) {
-      BlockletIndex blockletIndex = getBlockletIndex(leaf_node_indices_Thrift.get(i));
-      blockletIndexList.add(blockletIndex);
-    }
-    List<org.apache.carbondata.format.BlockletInfo2> leaf_node_infos_Thrift =
-        footer.getBlocklet_info_list2();
-    List<BlockletInfo> blockletInfoList = new ArrayList<BlockletInfo>();
-    for (int i = 0; i < leaf_node_infos_Thrift.size(); i++) {
-      BlockletInfo blockletInfo = getBlockletInfo(leaf_node_infos_Thrift.get(i),
-          getNumberOfDimensionColumns(columnSchemaList));
-      blockletInfo.setBlockletIndex(blockletIndexList.get(i));
-      blockletInfoList.add(blockletInfo);
-    }
-    dataFileFooter.setBlockletList(blockletInfoList);
-    dataFileFooter.setBlockletIndex(getBlockletIndexForDataFileFooter(blockletIndexList));
-    return dataFileFooter;
-  }
-
-  /**
-   * Below method is to convert the blocklet info of the thrift to wrapper
-   * blocklet info
-   *
-   * @param blockletInfoThrift blocklet info of the thrift
-   * @return blocklet info wrapper
-   */
-  private BlockletInfo getBlockletInfo(
-      org.apache.carbondata.format.BlockletInfo2 blockletInfoThrift, int numberOfDimensionColumns) {
-    BlockletInfo blockletInfo = new BlockletInfo();
-    List<Long> dimensionColumnChunkOffsets =
-        blockletInfoThrift.getColumn_data_chunks_offsets().subList(0, numberOfDimensionColumns);
-    List<Long> measureColumnChunksOffsets = blockletInfoThrift.getColumn_data_chunks_offsets()
-        .subList(numberOfDimensionColumns,
-            blockletInfoThrift.getColumn_data_chunks_offsets().size());
-    List<Short> dimensionColumnChunkLength =
-        blockletInfoThrift.getColumn_data_chunks_length().subList(0, numberOfDimensionColumns);
-    List<Short> measureColumnChunksLength = blockletInfoThrift.getColumn_data_chunks_length()
-        .subList(numberOfDimensionColumns,
-            blockletInfoThrift.getColumn_data_chunks_offsets().size());
-    blockletInfo.setDimensionChunkOffsets(dimensionColumnChunkOffsets);
-    blockletInfo.setMeasureChunkOffsets(measureColumnChunksOffsets);
-
-    List<Integer> dimensionColumnChunkLengthInteger = new ArrayList<Integer>();
-    List<Integer> measureColumnChunkLengthInteger = new ArrayList<Integer>();
-    for (int i = 0; i < dimensionColumnChunkLength.size(); i++) {
-      dimensionColumnChunkLengthInteger.add(dimensionColumnChunkLength.get(i).intValue());
-    }
-    for (int i = 0; i < measureColumnChunksLength.size(); i++) {
-      measureColumnChunkLengthInteger.add(measureColumnChunksLength.get(i).intValue());
-    }
-    blockletInfo.setDimensionChunksLength(dimensionColumnChunkLengthInteger);
-    blockletInfo.setMeasureChunksLength(measureColumnChunkLengthInteger);
-    blockletInfo.setNumberOfRows(blockletInfoThrift.getNum_rows());
-    return blockletInfo;
-  }
-
-  /**
-   * Below method will be used to get the number of dimension column
-   * in carbon column schema
-   *
-   * @param columnSchemaList column schema list
-   * @return number of dimension column
-   */
-  private int getNumberOfDimensionColumns(List<ColumnSchema> columnSchemaList) {
-    int numberOfDimensionColumns = 0;
-    int previousColumnGroupId = -1;
-    ColumnSchema columnSchema = null;
-    for (int i = 0; i < columnSchemaList.size(); i++) {
-      columnSchema = columnSchemaList.get(i);
-      if (columnSchema.isDimensionColumn()) {
-        numberOfDimensionColumns++;
-      } else {
-        break;
-      }
-    }
-    return numberOfDimensionColumns;
-  }
-
-  @Override
-  public List<ColumnSchema> getSchema(TableBlockInfo tableBlockInfo) throws IOException {
-    return new DataFileFooterConverter(configuration).getSchema(tableBlockInfo);
-  }
-}
diff --git a/core/src/main/java/org/apache/carbondata/core/util/DataFileFooterConverterV3.java b/core/src/main/java/org/apache/carbondata/core/util/DataFileFooterConverterV3.java
index 65ba609..f7ad18c 100644
--- a/core/src/main/java/org/apache/carbondata/core/util/DataFileFooterConverterV3.java
+++ b/core/src/main/java/org/apache/carbondata/core/util/DataFileFooterConverterV3.java
@@ -73,7 +73,6 @@ public class DataFileFooterConverterV3 extends AbstractDataFileFooterConverter {
     DataFileFooter dataFileFooter = new DataFileFooter();
     dataFileFooter.setVersionId(ColumnarFormatVersion.valueOf((short) fileHeader.getVersion()));
     dataFileFooter.setNumberOfRows(footer.getNum_rows());
-    dataFileFooter.setSegmentInfo(getSegmentInfo(footer.getSegment_info()));
     dataFileFooter.setSchemaUpdatedTimeStamp(fileHeader.getTime_stamp());
     if (footer.isSetIs_sort()) {
       dataFileFooter.setSorted(footer.isIs_sort());
diff --git a/core/src/main/java/org/apache/carbondata/core/util/NonDictionaryUtil.java b/core/src/main/java/org/apache/carbondata/core/util/NonDictionaryUtil.java
index a51d6d1..9f4b82a 100644
--- a/core/src/main/java/org/apache/carbondata/core/util/NonDictionaryUtil.java
+++ b/core/src/main/java/org/apache/carbondata/core/util/NonDictionaryUtil.java
@@ -17,8 +17,6 @@
 
 package org.apache.carbondata.core.util;
 
-import java.nio.ByteBuffer;
-
 import org.apache.carbondata.core.datastore.row.WriteStepRowUtil;
 
 /**
@@ -26,96 +24,6 @@ import org.apache.carbondata.core.datastore.row.WriteStepRowUtil;
  */
 public class NonDictionaryUtil {
 
-  /**
-   * This method will form one single byte [] for all the high card dims.
-   * For example if you need to pack 2 columns c1 and c2 , it stores in following way
-   *  <total_len(short)><offsetLen(short)><offsetLen+c1_len(short)><c1(byte[])><c2(byte[])>
-   * @param byteBufferArr
-   * @return
-   */
-  public static byte[] packByteBufferIntoSingleByteArray(byte[][] byteBufferArr) {
-    // for empty array means there is no data to remove dictionary.
-    if (null == byteBufferArr || byteBufferArr.length == 0) {
-      return null;
-    }
-    int noOfCol = byteBufferArr.length;
-    short toDetermineLengthOfByteArr = 2;
-    short offsetLen = (short) (noOfCol * 2 + toDetermineLengthOfByteArr);
-    int totalBytes = calculateTotalBytes(byteBufferArr) + offsetLen;
-
-    ByteBuffer buffer = ByteBuffer.allocate(totalBytes);
-
-    // write the length of the byte [] as first short
-    buffer.putShort((short) (totalBytes - toDetermineLengthOfByteArr));
-    // writing the offset of the first element.
-    buffer.putShort(offsetLen);
-
-    // prepare index for byte []
-    for (int index = 0; index < byteBufferArr.length - 1; index++) {
-      int noOfBytes = byteBufferArr[index].length;
-
-      buffer.putShort((short) (offsetLen + noOfBytes));
-      offsetLen += noOfBytes;
-    }
-
-    // put actual data.
-    for (int index = 0; index < byteBufferArr.length; index++) {
-      buffer.put(byteBufferArr[index]);
-    }
-    buffer.rewind();
-    return buffer.array();
-
-  }
-
-  /**
-   * To calculate the total bytes in byte Buffer[].
-   *
-   * @param byteBufferArr
-   * @return
-   */
-  private static int calculateTotalBytes(byte[][] byteBufferArr) {
-    int total = 0;
-    for (int index = 0; index < byteBufferArr.length; index++) {
-      total += byteBufferArr[index].length;
-    }
-    return total;
-  }
-
-  /**
-   * Method to get the required dictionary Dimension from obj []
-   *
-   * @param index
-   * @param row
-   * @return
-   */
-  public static int getDictDimension(int index, Object[] row) {
-    int[] dimensions = (int[]) row[WriteStepRowUtil.DICTIONARY_DIMENSION];
-    return dimensions[index];
-  }
-
-  /**
-   * Method to get the required non-dictionary & complex from 3-parted row
-   * @param index
-   * @param row
-   * @return
-   */
-  public static byte[] getNoDictOrComplex(int index, Object[] row) {
-    byte[][] nonDictArray = (byte[][]) row[WriteStepRowUtil.NO_DICTIONARY_AND_COMPLEX];
-    return nonDictArray[index];
-  }
-
-  /**
-   * Method to get the required measure from obj []
-   *
-   * @param index
-   * @param row
-   * @return
-   */
-  public static Object getMeasure(int index, Object[] row) {
-    Object[] measures = (Object[]) row[WriteStepRowUtil.MEASURE];
-    return measures[index];
-  }
-
   public static void prepareOutObj(Object[] out, int[] dimArray, Object[] byteBufferArr,
       Object[] measureArray) {
     out[WriteStepRowUtil.DICTIONARY_DIMENSION] = dimArray;
diff --git a/core/src/main/java/org/apache/carbondata/hadoop/CarbonInputSplit.java b/core/src/main/java/org/apache/carbondata/hadoop/CarbonInputSplit.java
index 086a6c0..445aeb2 100644
--- a/core/src/main/java/org/apache/carbondata/hadoop/CarbonInputSplit.java
+++ b/core/src/main/java/org/apache/carbondata/hadoop/CarbonInputSplit.java
@@ -33,7 +33,6 @@ import java.util.Set;
 
 import org.apache.carbondata.core.constants.CarbonCommonConstants;
 import org.apache.carbondata.core.datamap.Segment;
-import org.apache.carbondata.core.datastore.block.BlockletInfos;
 import org.apache.carbondata.core.datastore.block.Distributable;
 import org.apache.carbondata.core.datastore.block.TableBlockInfo;
 import org.apache.carbondata.core.indexstore.BlockletDetailInfo;
@@ -96,10 +95,6 @@ public class CarbonInputSplit extends FileSplit
 
   private transient DataMapRow dataMapRow;
 
-  private transient int[] columnCardinality;
-
-  private boolean isLegacyStore;
-
   private transient List<ColumnSchema> columnSchema;
 
   private boolean useMinMaxForPruning = true;
@@ -299,16 +294,13 @@ public class CarbonInputSplit extends FileSplit
   public static List<TableBlockInfo> createBlocks(List<CarbonInputSplit> splitList) {
     List<TableBlockInfo> tableBlockInfoList = new ArrayList<>();
     for (CarbonInputSplit split : splitList) {
-      BlockletInfos blockletInfos =
-          new BlockletInfos(split.getNumberOfBlocklets(), 0, split.getNumberOfBlocklets());
       try {
         TableBlockInfo blockInfo =
-            new TableBlockInfo(split.getFilePath(), split.blockletId, split.getStart(),
+            new TableBlockInfo(split.getFilePath(), split.getStart(),
                 split.getSegment().toString(), split.getLocations(), split.getLength(),
-                blockletInfos, split.getVersion(), split.getDeleteDeltaFiles());
+                split.getVersion(), split.getDeleteDeltaFiles());
         blockInfo.setDetailInfo(split.getDetailInfo());
         blockInfo.setDataMapWriterPath(split.dataMapWritePath);
-        blockInfo.setLegacyStore(split.isLegacyStore);
         if (split.getDetailInfo() != null) {
           blockInfo.setBlockOffset(split.getDetailInfo().getBlockFooterOffset());
         }
@@ -321,19 +313,16 @@ public class CarbonInputSplit extends FileSplit
   }
 
   public static TableBlockInfo getTableBlockInfo(CarbonInputSplit inputSplit) {
-    BlockletInfos blockletInfos =
-        new BlockletInfos(inputSplit.getNumberOfBlocklets(), 0, inputSplit.getNumberOfBlocklets());
     try {
       TableBlockInfo blockInfo =
-          new TableBlockInfo(inputSplit.getFilePath(), inputSplit.blockletId,
+          new TableBlockInfo(inputSplit.getFilePath(),
               inputSplit.getStart(), inputSplit.getSegment().toString(), inputSplit.getLocations(),
-              inputSplit.getLength(), blockletInfos, inputSplit.getVersion(),
+              inputSplit.getLength(), inputSplit.getVersion(),
               inputSplit.getDeleteDeltaFiles());
       blockInfo.setDetailInfo(inputSplit.getDetailInfo());
       if (null != inputSplit.getDetailInfo()) {
         blockInfo.setBlockOffset(inputSplit.getDetailInfo().getBlockFooterOffset());
       }
-      blockInfo.setLegacyStore(inputSplit.isLegacyStore);
       return blockInfo;
     } catch (IOException e) {
       throw new RuntimeException("fail to get location of split: " + inputSplit, e);
@@ -390,7 +379,6 @@ public class CarbonInputSplit extends FileSplit
     for (int i = 0; i < validBlockletIdCount; i++) {
       validBlockletIds.add((int) in.readShort());
     }
-    this.isLegacyStore = in.readBoolean();
   }
 
   @Override
@@ -446,7 +434,6 @@ public class CarbonInputSplit extends FileSplit
     for (Integer blockletId : getValidBlockletIds()) {
       out.writeShort(blockletId);
     }
-    out.writeBoolean(isLegacyStore);
   }
 
   private void writeDeleteDeltaFile(DataOutput out) throws IOException {
@@ -641,14 +628,6 @@ public class CarbonInputSplit extends FileSplit
     this.dataMapRow = dataMapRow;
   }
 
-  public void setColumnCardinality(int[] columnCardinality) {
-    this.columnCardinality = columnCardinality;
-  }
-
-  public void setLegacyStore(boolean legacyStore) {
-    isLegacyStore = legacyStore;
-  }
-
   public void setColumnSchema(List<ColumnSchema> columnSchema) {
     this.columnSchema = columnSchema;
   }
@@ -674,10 +653,6 @@ public class CarbonInputSplit extends FileSplit
     }
     out.writeShort(this.dataMapRow.getShort(BlockletDataMapRowIndexes.VERSION_INDEX));
     out.writeShort(Short.parseShort(this.blockletId));
-    out.writeShort(this.columnCardinality.length);
-    for (int i = 0; i < this.columnCardinality.length; i++) {
-      out.writeInt(this.columnCardinality[i]);
-    }
     out.writeLong(this.dataMapRow.getLong(BlockletDataMapRowIndexes.SCHEMA_UPADATED_TIME_INDEX));
     out.writeBoolean(false);
     out.writeLong(this.dataMapRow.getLong(BlockletDataMapRowIndexes.BLOCK_FOOTER_OFFSET));
@@ -714,7 +689,6 @@ public class CarbonInputSplit extends FileSplit
       detailInfo
           .setVersionNumber(this.dataMapRow.getShort(BlockletDataMapRowIndexes.VERSION_INDEX));
       detailInfo.setBlockletId(Short.parseShort(this.blockletId));
-      detailInfo.setDimLens(this.columnCardinality);
       detailInfo.setSchemaUpdatedTimeStamp(
           this.dataMapRow.getLong(BlockletDataMapRowIndexes.SCHEMA_UPADATED_TIME_INDEX));
       detailInfo.setBlockFooterOffset(
diff --git a/core/src/test/java/org/apache/carbondata/core/datastore/block/BlockInfoTest.java b/core/src/test/java/org/apache/carbondata/core/datastore/block/BlockInfoTest.java
deleted file mode 100644
index f891b66..0000000
--- a/core/src/test/java/org/apache/carbondata/core/datastore/block/BlockInfoTest.java
+++ /dev/null
@@ -1,91 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.carbondata.core.datastore.block;
-
-import org.apache.carbondata.core.metadata.ColumnarFormatVersion;
-
-import org.junit.Assert;
-import org.junit.BeforeClass;
-import org.junit.Test;
-
-import static junit.framework.TestCase.assertEquals;
-
-public class BlockInfoTest {
-
-  static BlockInfo blockInfo;
-
-  @BeforeClass public static void setup() {
-    blockInfo = new BlockInfo(new TableBlockInfo("/filePath.carbondata", 6, "segmentId", null, 6, ColumnarFormatVersion.V1, null));
-  }
-
-  @Test public void hashCodeTest() {
-    int res = blockInfo.hashCode();
-    int expectedResult = 1694768249;
-    assertEquals(expectedResult, res);
-  }
-
-  @Test public void equalsTestwithSameObject() {
-    Boolean res = blockInfo.equals(blockInfo);
-    Assert.assertTrue(res);
-  }
-
-  @Test public void equalsTestWithSimilarObject() {
-    BlockInfo blockInfoTest =
-        new BlockInfo(new TableBlockInfo("/filePath.carbondata", 6, "segmentId", null, 6, ColumnarFormatVersion.V1, null));
-    Boolean res = blockInfo.equals(blockInfoTest);
-    Assert.assertTrue(res);
-  }
-
-  @Test public void equalsTestWithNullObject() {
-    Boolean res = blockInfo.equals(null);
-    Assert.assertTrue(!res);
-  }
-
-  @Test public void equalsTestWithStringObject() {
-    Boolean res = blockInfo.equals("dummy");
-    Assert.assertTrue(!res);
-  }
-
-  @Test public void equalsTestWithDifferentSegmentId() {
-    BlockInfo blockInfoTest =
-        new BlockInfo(new TableBlockInfo("/filePath.carbondata", 6, "diffSegmentId", null, 6, ColumnarFormatVersion.V1, null));
-    Boolean res = blockInfo.equals(blockInfoTest);
-    Assert.assertTrue(!res);
-  }
-
-  @Test public void equalsTestWithDifferentOffset() {
-    BlockInfo blockInfoTest =
-        new BlockInfo(new TableBlockInfo("/filePath.carbondata", 62, "segmentId", null, 6, ColumnarFormatVersion.V1, null));
-    Boolean res = blockInfo.equals(blockInfoTest);
-    Assert.assertTrue(!res);
-  }
-
-  @Test public void equalsTestWithDifferentBlockLength() {
-    BlockInfo blockInfoTest =
-        new BlockInfo(new TableBlockInfo("/filePath.carbondata", 6, "segmentId", null, 62, ColumnarFormatVersion.V1, null));
-    Boolean res = blockInfo.equals(blockInfoTest);
-    Assert.assertTrue(!res);
-  }
-
-  @Test public void equalsTestWithDiffFilePath() {
-    BlockInfo blockInfoTest =
-        new BlockInfo(new TableBlockInfo("/diffFilePath.carbondata", 6, "segmentId", null, 62, ColumnarFormatVersion.V1, null));
-    Boolean res = blockInfoTest.equals(blockInfo);
-    Assert.assertTrue(!res);
-  }
-}
diff --git a/core/src/test/java/org/apache/carbondata/core/datastore/block/SegmentPropertiesTest.java b/core/src/test/java/org/apache/carbondata/core/datastore/block/SegmentPropertiesTest.java
index a626b2e..c3c89b4 100644
--- a/core/src/test/java/org/apache/carbondata/core/datastore/block/SegmentPropertiesTest.java
+++ b/core/src/test/java/org/apache/carbondata/core/datastore/block/SegmentPropertiesTest.java
@@ -52,32 +52,15 @@ public class SegmentPropertiesTest extends TestCase {
     columnSchema.add(getDimensionColumn7());
     columnSchema.add(getMeasureColumn());
     columnSchema.add(getMeasureColumn1());
-    int[] cardinality = new int[columnSchema.size()];
-    int x = 100;
-    for (int i = 0; i < columnSchema.size(); i++) {
-      cardinality[i] = x;
-      x++;
-    }
-    blockMetadataInfos = new SegmentProperties(columnSchema, cardinality);
+    blockMetadataInfos = new SegmentProperties(columnSchema);
   }
 
   @Test public void testBlockMetadataHasProperDimensionCardinality() {
-    int[] cardinality = { 100, 102, 103, 105, 106, 107 };
-    boolean isProper = true;
-    for (int i = 0; i < cardinality.length; i++) {
-      isProper = cardinality[i] == blockMetadataInfos.getDimColumnsCardinality()[i];
-      if (!isProper) {
-        assertTrue(false);
-      }
-    }
-    assertTrue(true);
-  }
-
-  @Test public void testBlockMetadataHasProperComplesDimensionCardinality() {
-    int[] cardinality = { 108, 109 };
+    int[] cardinality = {-1, -1, -1, -1, -1, -1, -1, -1};
     boolean isProper = true;
+    int[] result = blockMetadataInfos.createDimColumnValueLength();
     for (int i = 0; i < cardinality.length; i++) {
-      isProper = cardinality[i] == blockMetadataInfos.getComplexDimColumnCardinality()[i];
+      isProper = cardinality[i] == result[i];
       if (!isProper) {
         assertTrue(false);
       }
@@ -144,21 +127,8 @@ public class SegmentPropertiesTest extends TestCase {
   }
 
   @Test public void testEachColumnValueSizeHasProperValue() {
-    int[] size = { 1, -1, 1, 1, -1, 1, 1, 1 };
-    int[] eachDimColumnValueSize = blockMetadataInfos.getEachDimColumnValueSize();
-    boolean isEqual = false;
-    for (int i = 0; i < size.length; i++) {
-      isEqual = size[i] == eachDimColumnValueSize[i];
-      if (!isEqual) {
-        assertTrue(false);
-      }
-    }
-    assertTrue(true);
-  }
-
-  @Test public void testEachComplexColumnValueSizeHasProperValue() {
-    int[] size = { 1, 1 };
-    int[] eachDimColumnValueSize = blockMetadataInfos.getEachComplexDimColumnValueSize();
+    int[] size = {-1, -1, -1, -1, -1, -1, -1, -1};
+    int[] eachDimColumnValueSize = blockMetadataInfos.createDimColumnValueLength();
     boolean isEqual = false;
     for (int i = 0; i < size.length; i++) {
       isEqual = size[i] == eachDimColumnValueSize[i];
diff --git a/core/src/test/java/org/apache/carbondata/core/datastore/block/SegmentPropertiesTestUtil.java b/core/src/test/java/org/apache/carbondata/core/datastore/block/SegmentPropertiesTestUtil.java
index 71f32f0..dc7eef0 100644
--- a/core/src/test/java/org/apache/carbondata/core/datastore/block/SegmentPropertiesTestUtil.java
+++ b/core/src/test/java/org/apache/carbondata/core/datastore/block/SegmentPropertiesTestUtil.java
@@ -42,13 +42,7 @@ public class SegmentPropertiesTestUtil {
     columnSchema.add(getDimensionColumn7());
     columnSchema.add(getMeasureColumn());
     columnSchema.add(getMeasureColumn1());
-    int[] cardinality = new int[columnSchema.size()];
-    int x = 100;
-    for (int i = 0; i < columnSchema.size(); i++) {
-      cardinality[i] = x;
-      x++;
-    }
-    return new SegmentProperties(columnSchema, cardinality);
+    return new SegmentProperties(columnSchema);
   }
 
   public static ColumnSchema getDimensionColumn1() {
diff --git a/core/src/test/java/org/apache/carbondata/core/datastore/block/TableBlockInfoTest.java b/core/src/test/java/org/apache/carbondata/core/datastore/block/TableBlockInfoTest.java
index 2ccbcec..e829e0c 100644
--- a/core/src/test/java/org/apache/carbondata/core/datastore/block/TableBlockInfoTest.java
+++ b/core/src/test/java/org/apache/carbondata/core/datastore/block/TableBlockInfoTest.java
@@ -36,7 +36,7 @@ public class TableBlockInfoTest {
 
   @BeforeClass public static void setup() {
     tableBlockInfo = new TableBlockInfo("filePath", 4, "segmentId", null, 6, ColumnarFormatVersion.V1, null);
-    tableBlockInfos = new TableBlockInfo("filepath", 6, "5", null, 6, new BlockletInfos(6, 2, 2), ColumnarFormatVersion.V1, null);
+    tableBlockInfos = new TableBlockInfo("filepath", 6, "5", null, 6, ColumnarFormatVersion.V1, null);
   }
 
   @Test public void equalTestWithSameObject() {
@@ -80,33 +80,33 @@ public class TableBlockInfoTest {
 
   @Test public void equalsTestWithDiffBlockletNumber() {
     TableBlockInfo tableBlockInfoTest =
-        new TableBlockInfo("filepath", 6, "segmentId", null, 6, new BlockletInfos(6, 3, 2), ColumnarFormatVersion.V1, null);
+        new TableBlockInfo("filepath", 6, "segmentId", null, 6, ColumnarFormatVersion.V1, null);
     Boolean res = tableBlockInfos.equals(tableBlockInfoTest);
     Assert.assertTrue(!res);
   }
 
   @Test public void equalsTestWithDiffFilePath() {
     TableBlockInfo tableBlockInfoTest =
-        new TableBlockInfo("difffilepath", 6, "segmentId", null, 6, new BlockletInfos(6, 3, 2), ColumnarFormatVersion.V1, null);
+        new TableBlockInfo("difffilepath", 6, "segmentId", null, 6, ColumnarFormatVersion.V1, null);
     Boolean res = tableBlockInfos.equals(tableBlockInfoTest);
     Assert.assertTrue(!res);
   }
 
   @Test public void compareToTestForSegmentId() {
     TableBlockInfo tableBlockInfo =
-        new TableBlockInfo("difffilepath", 6, "5", null, 6, new BlockletInfos(6, 3, 2), ColumnarFormatVersion.V1, null);
+        new TableBlockInfo("difffilepath", 6, "5", null, 6, ColumnarFormatVersion.V1, null);
     int res = tableBlockInfos.compareTo(tableBlockInfo);
     int expectedResult = 2;
     assertEquals(res, expectedResult);
 
     TableBlockInfo tableBlockInfo1 =
-        new TableBlockInfo("difffilepath", 6, "6", null, 6, new BlockletInfos(6, 3, 2), ColumnarFormatVersion.V1, null);
+        new TableBlockInfo("difffilepath", 6, "6", null, 6, ColumnarFormatVersion.V1, null);
     int res1 = tableBlockInfos.compareTo(tableBlockInfo1);
     int expectedResult1 = -1;
     assertEquals(res1, expectedResult1);
 
     TableBlockInfo tableBlockInfo2 =
-        new TableBlockInfo("difffilepath", 6, "4", null, 6, new BlockletInfos(6, 3, 2), ColumnarFormatVersion.V1, null);
+        new TableBlockInfo("difffilepath", 6, "4", null, 6, ColumnarFormatVersion.V1, null);
     int res2 = tableBlockInfos.compareTo(tableBlockInfo2);
     int expectedresult2 = 1;
     assertEquals(res2, expectedresult2);
@@ -142,26 +142,12 @@ public class TableBlockInfoTest {
     assertEquals(res1, expectedResult1);
 
     TableBlockInfo tableBlockInfoTest =
-        new TableBlockInfo("filePath", 6, "5", null, 7, new BlockletInfos(6, 2, 2), ColumnarFormatVersion.V1, null);
+        new TableBlockInfo("filePath", 6, "5", null, 7, ColumnarFormatVersion.V1, null);
     int res2 = tableBlockInfos.compareTo(tableBlockInfoTest);
     int expectedResult2 = -1;
     assertEquals(res2, expectedResult2);
   }
 
-  @Test public void compareToTestWithStartBlockletNo() {
-    TableBlockInfo tableBlockInfo =
-        new TableBlockInfo("filepath", 6, "5", null, 6, new BlockletInfos(6, 3, 2), ColumnarFormatVersion.V1, null);
-    int res = tableBlockInfos.compareTo(tableBlockInfo);
-    int expectedresult =-1;
-    assertEquals(res, expectedresult);
-
-    TableBlockInfo tableBlockInfo1 =
-        new TableBlockInfo("filepath", 6, "5", null, 6, new BlockletInfos(6, 1, 2), ColumnarFormatVersion.V1, null);
-    int res1 = tableBlockInfos.compareTo(tableBlockInfo1);
-    int expectedresult1 = 1;
-    assertEquals(res1, expectedresult1);
-  }
-
   @Test public void compareToTest() {
     int res = tableBlockInfos.compareTo(tableBlockInfos);
     int expectedResult = 0;
diff --git a/core/src/test/java/org/apache/carbondata/core/keygenerator/columnar/impl/MultiDimKeyVarLengthEquiSplitGeneratorUnitTest.java b/core/src/test/java/org/apache/carbondata/core/keygenerator/columnar/impl/MultiDimKeyVarLengthEquiSplitGeneratorUnitTest.java
deleted file mode 100644
index 377b1a7..0000000
--- a/core/src/test/java/org/apache/carbondata/core/keygenerator/columnar/impl/MultiDimKeyVarLengthEquiSplitGeneratorUnitTest.java
+++ /dev/null
@@ -1,197 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.carbondata.core.keygenerator.columnar.impl;
-
-import org.junit.*;
-
-import static org.hamcrest.MatcherAssert.assertThat;
-import static junit.framework.Assert.assertEquals;
-import static org.hamcrest.CoreMatchers.equalTo;
-import static org.hamcrest.core.Is.is;
-
-import java.util.Arrays;
-
-import org.apache.carbondata.core.keygenerator.KeyGenException;
-
-public class MultiDimKeyVarLengthEquiSplitGeneratorUnitTest {
-
-  static MultiDimKeyVarLengthEquiSplitGenerator multiDimKeyVarLengthEquiSplitGenerator;
-
-  @BeforeClass public static void setup() {
-    int[] lens = new int[] { 32, 8, 16, 16, 16 };
-    byte dimensionsToSplit = 1;
-    multiDimKeyVarLengthEquiSplitGenerator =
-        new MultiDimKeyVarLengthEquiSplitGenerator(lens, dimensionsToSplit);
-  }
-
-  @Test public void testSplitKeyWithNewDimensionToSplit() {
-    int[] lens = new int[] { 24, 8, 16, 16, 16 };
-    byte dimensionsToSplit = 3;
-    MultiDimKeyVarLengthEquiSplitGenerator multiDimKeyVarLengthEquiSplitGeneratorNew =
-        new MultiDimKeyVarLengthEquiSplitGenerator(lens, dimensionsToSplit);
-    byte[][] result_value = new byte[][] { { 16, 8, 24, 46, 76, 64 }, { 80, 36, 72, 48 } };
-    byte[] key = new byte[] { 16, 8, 24, 46, 76, 64, 80, 36, 72, 48 };
-    byte[][] result = multiDimKeyVarLengthEquiSplitGeneratorNew.splitKey(key);
-    Assert.assertTrue(Arrays.deepEquals(result, result_value));
-  }
-
-  @Test public void testSplitKeyWithNewDimensionToSplitValue16() {
-    int[] lens = new int[] { 24, 8, 16, 16, 16 };
-    byte dimensionsToSplit = 16;
-    MultiDimKeyVarLengthEquiSplitGenerator multiDimKeyVarLengthEquiSplitGeneratorNew =
-        new MultiDimKeyVarLengthEquiSplitGenerator(lens, dimensionsToSplit);
-    byte[][] result_value = new byte[][] { { 16, 8, 24, 46, 76, 64, 80, 36, 72, 48 } };
-    byte[] key = new byte[] { 16, 8, 24, 46, 76, 64, 80, 36, 72, 48 };
-    byte[][] result = multiDimKeyVarLengthEquiSplitGeneratorNew.splitKey(key);
-    Assert.assertTrue(Arrays.deepEquals(result, result_value));
-  }
-
-  @Test public void testGenerateAndSplitKeyAndGetKeyArrayWithActualLogic() throws KeyGenException {
-    long[] keys = new long[] { 12253L, 48254L, 451245L, 52245L, 36458L, 48123L, 264L, 5852L, 42L };
-    long[] expected_result = { 12253, 126, 58029, 52245, 36458 };
-    byte[][] result_GenerateAndSplitKey =
-        multiDimKeyVarLengthEquiSplitGenerator.generateAndSplitKey(keys);
-    long[] result_GetKeyArray =
-        multiDimKeyVarLengthEquiSplitGenerator.getKeyArray(result_GenerateAndSplitKey);
-    assertThat(result_GetKeyArray, is(equalTo(expected_result)));
-  }
-
-  @Test public void testSplitKey() throws Exception {
-    byte[][] result_value =
-        new byte[][] { { 1, 102, 20, 56 }, { 64 }, { 36, 18 }, { 16, 28 }, { 98, 93 } };
-    byte[] key = new byte[] { 1, 102, 20, 56, 64, 36, 18, 16, 28, 98, 93 };
-    byte[][] result = multiDimKeyVarLengthEquiSplitGenerator.splitKey(key);
-    Assert.assertTrue(Arrays.deepEquals(result, result_value));
-  }
-
-  @Test public void testGetKeyArray() throws Exception {
-    long[] result_value = new long[] { 23467064L, 64L, 9234L, 4124L, 25181L };
-    byte[][] key = new byte[][] { { 1, 102, 20, 56, 64, 36, 18, 16, 28, 98, 93 } };
-    long[] result = multiDimKeyVarLengthEquiSplitGenerator.getKeyArray(key);
-    assertThat(result, is(equalTo(result_value)));
-  }
-
-  @Test public void testKeyByteArray() throws Exception {
-    byte[] result_value = new byte[] { 1, 102, 20, 56, 64, 36, 18, 16, 28, 98, 93 };
-    byte[][] key = new byte[][] { { 1, 102, 20, 56, 64, 36, 18, 16, 28, 98, 93 } };
-    byte[] result = multiDimKeyVarLengthEquiSplitGenerator.getKeyByteArray(key);
-    Assert.assertTrue(Arrays.equals(result, result_value));
-  }
-
-  /*
-   * In this test scenario We will send  blockIndexes { 0 }.
-   * Where value of blockKeySize is {4,1,2,2,2}
-   * It will add value 0f {0} indexes and will return the size which is 4.
-   *
-   * @throws Exception
-   */
-
-  @Test public void testGetKeySizeByBlockWithBlockIndexesLengthIsZero() throws Exception {
-    int result_value = 4;
-    int[] blockIndexes = new int[] { 0 };
-    int result = multiDimKeyVarLengthEquiSplitGenerator.getKeySizeByBlock(blockIndexes);
-    assertEquals(result_value, result);
-  }
-
-  /*
-   * In this test scenario We will send  blockIndexes { 0, 1, 2 }.
-   * Where value of blockKeySize is {4,1,2,2,2}
-   * It will add value 0f {0, 1, 2} indexes and will return the size which is 7.
-   *
-   * @throws Exception
-   */
-
-  @Test public void testGetKeySizeByBlockWithBlockIndexesLengthIsGreaterThanZero()
-      throws Exception {
-    int result_value = 7;
-    int[] blockIndexes = new int[] { 0, 1, 2 };
-    int result = multiDimKeyVarLengthEquiSplitGenerator.getKeySizeByBlock(blockIndexes);
-    assertEquals(result_value, result);
-  }
-
-  /*
-   * In this test scenario We will send  blockIndexes {1, 2, 7} where {7} > blockKeySize.length which is 5.
-   * Where value of blockKeySize is {4,1,2,2,2}
-   * It will add value 0f {1, 2, 7} indexes and will return the size which is 3.
-   *
-   * @throws Exception
-   */
-
-  @Test public void testGetKeySizeByBlockWithBlockIndexesLengthIsBlockKeySizeLength()
-      throws Exception {
-    int result_value = 3;
-    int[] blockIndexes = new int[] { 1, 2, 7 };
-    int result = multiDimKeyVarLengthEquiSplitGenerator.getKeySizeByBlock(blockIndexes);
-    assertEquals(result_value, result);
-  }
-
-  /*
-   * In this test scenario We will send  blockIndexes {10} where {10} > blockKeySize.length which is 5.
-   * Where value of blockKeySize is {4,1,2,2,2}
-   * It will return default value 0.
-   *
-   * @throws Exception
-   */
-
-  @Test public void testGetKeySizeByBlockWith() throws Exception {
-    int result_value = 0;
-    int[] key = new int[] { 10 };
-    int result = multiDimKeyVarLengthEquiSplitGenerator.getKeySizeByBlock(key);
-    assertEquals(result_value, result);
-  }
-
-  @Test public void testEqualsWithAnyObject() throws Exception {
-    Object obj = new Object();
-    boolean result = multiDimKeyVarLengthEquiSplitGenerator.equals(obj);
-    Assert.assertTrue(!result);
-  }
-
-  @Test public void testEqualsWithDifferentValueMultiDimKeyVarLengthEquiSplitGeneratorObject()
-      throws Exception {
-    int[] lens = new int[] { 32, 8, 16, 16, 16 };
-    byte dimensionsToSplit = 2;
-    boolean result = multiDimKeyVarLengthEquiSplitGenerator
-        .equals(new MultiDimKeyVarLengthEquiSplitGenerator(lens, dimensionsToSplit));
-    Assert.assertTrue(!result);
-  }
-
-  @Test public void testEqualsWithSameValueMultiDimKeyVarLengthEquiSplitGeneratorObject()
-      throws Exception {
-    int[] lens = new int[] { 32, 8, 16, 16, 16 };
-    byte dimensionsToSplit = 1;
-    boolean result = multiDimKeyVarLengthEquiSplitGenerator
-        .equals(new MultiDimKeyVarLengthEquiSplitGenerator(lens, dimensionsToSplit));
-    Assert.assertTrue(result);
-  }
-
-  /**
-   * Test case for exception when Key size is less than byte key size
-   */
-
-  @Test(expected = ArrayIndexOutOfBoundsException.class) public void testSplitKeyWithException() {
-    int[] lens = new int[] { 24, 8, 16, 16, 16 };
-    byte dimensionsToSplit = 3;
-    MultiDimKeyVarLengthEquiSplitGenerator multiDimKeyVarLengthEquiSplitGeneratorNew =
-        new MultiDimKeyVarLengthEquiSplitGenerator(lens, dimensionsToSplit);
-    byte[][] result_value = new byte[][] { { 16, 8, 24, 46, 76, 64 }, { 80, 36, 72, 48 } };
-    byte[] key = new byte[] { 16, 8, 24, 46, 76 };
-    byte[][] result = multiDimKeyVarLengthEquiSplitGeneratorNew.splitKey(key);
-    Assert.assertTrue(Arrays.deepEquals(result, result_value));
-  }
-
-}
diff --git a/core/src/test/java/org/apache/carbondata/core/keygenerator/columnar/impl/MultiDimKeyVarLengthVariableSplitGeneratorUnitTest.java b/core/src/test/java/org/apache/carbondata/core/keygenerator/columnar/impl/MultiDimKeyVarLengthVariableSplitGeneratorUnitTest.java
deleted file mode 100644
index 9065001..0000000
--- a/core/src/test/java/org/apache/carbondata/core/keygenerator/columnar/impl/MultiDimKeyVarLengthVariableSplitGeneratorUnitTest.java
+++ /dev/null
@@ -1,148 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.carbondata.core.keygenerator.columnar.impl;
-
-import org.apache.carbondata.core.keygenerator.KeyGenException;
-
-import org.junit.BeforeClass;
-import org.junit.Test;
-
-import static org.hamcrest.MatcherAssert.assertThat;
-import static junit.framework.Assert.assertEquals;
-import static org.hamcrest.CoreMatchers.equalTo;
-import static org.hamcrest.core.Is.is;
-
-public class MultiDimKeyVarLengthVariableSplitGeneratorUnitTest {
-
-  private static MultiDimKeyVarLengthVariableSplitGenerator
-      multiDimKeyVarLengthVariableSplitGenerator;
-
-  @BeforeClass public static void setup() {
-    int[] lens = new int[] { 32, 8, 16, 16, 16 };
-    int[] dimSplit = new int[] { 1, 1, 1, 1, 1 };
-    multiDimKeyVarLengthVariableSplitGenerator =
-        new MultiDimKeyVarLengthVariableSplitGenerator(lens, dimSplit);
-  }
-
-  @Test public void testWithDifferentValueInDimSplit() {
-
-    int[] lens = new int[] { 32, 8, 32, 32, 16 };
-    int[] dimSplit = new int[] { 12, 8, 1, 8, 16 };
-    MultiDimKeyVarLengthVariableSplitGenerator multiDimKeyVarLengthVariableSplitGeneratorNew =
-        new MultiDimKeyVarLengthVariableSplitGenerator(lens, dimSplit);
-
-    byte[][] result_value =
-        new byte[][] { { 24, 56, 72, 48, 56, 36, 18, 24, 40, 24, 64, 24, 56, 72, 48 } };
-    byte[] key = new byte[] { 24, 56, 72, 48, 56, 36, 18, 24, 40, 24, 64, 24, 56, 72, 48 };
-    byte[][] result = multiDimKeyVarLengthVariableSplitGeneratorNew.splitKey(key);
-    assertThat(result, is(equalTo(result_value)));
-  }
-
-  @Test public void testGenerateAndSplitKeyAndGetKeyArrayWithActualLogic() {
-    long[] keys = new long[] { 12253L, 48254L, 451245L, 52245L, 36458L, 48123L, 264L, 5852L, 42L };
-    long[] expected_result = { 12253, 126, 58029, 52245, 36458 };
-    byte[][] result_GenerateAndSplitKey =
-        multiDimKeyVarLengthVariableSplitGenerator.generateAndSplitKey(keys);
-    long[] result_GetKeyArray =
-        multiDimKeyVarLengthVariableSplitGenerator.getKeyArray(result_GenerateAndSplitKey);
-    assertThat(result_GetKeyArray, is(equalTo(expected_result)));
-  }
-
-  @Test public void testGenerateAndSplitKeyAndGetKeyArrayWithActualLogicWithInt() {
-    int[] keys = new int[] { 122, 254, 4512, 52, 36, 481, 264, 58, 42 };
-    long[] expected_result = { 122L, 254L, 4512L, 52L, 36L };
-    byte[][] result_GenerateAndSplitKey =
-        multiDimKeyVarLengthVariableSplitGenerator.generateAndSplitKey(keys);
-    long[] result_GetKeyArray =
-        multiDimKeyVarLengthVariableSplitGenerator.getKeyArray(result_GenerateAndSplitKey);
-    assertThat(result_GetKeyArray, is(equalTo(expected_result)));
-  }
-
-  @Test public void testGenerateAndSplitKeyAndGetKeyByteArrayWithActualLogicWithInt() {
-    int[] keys = new int[] { 1220, 2554, 452, 520, 360, 48, 24, 56, 42 };
-    byte[] expected_result = new byte[] { 0, 0, 4, -60, -6, 1, -60, 2, 8, 1, 104 };
-    byte[][] result_GenerateAndSplitKey =
-        multiDimKeyVarLengthVariableSplitGenerator.generateAndSplitKey(keys);
-    byte[] result_GetKeyByteArray =
-        multiDimKeyVarLengthVariableSplitGenerator.getKeyByteArray(result_GenerateAndSplitKey);
-    assertThat(result_GetKeyByteArray, is(equalTo(expected_result)));
-  }
-
-  @Test public void testSplitKey() {
-    byte[][] result_value =
-        new byte[][] { { 1, 102, 20, 56 }, { 64 }, { 36, 18 }, { 16, 28 }, { 98, 93 } };
-    byte[] key = new byte[] { 1, 102, 20, 56, 64, 36, 18, 16, 28, 98, 93 };
-    byte[][] result = multiDimKeyVarLengthVariableSplitGenerator.splitKey(key);
-    assertThat(result, is(equalTo(result_value)));
-  }
-
-  @Test public void testGetKeyArray() {
-    long[] result_value = new long[] { 23467064, 64, 9234, 4124, 25181 };
-    byte[][] key = new byte[][] { { 1, 102, 20, 56, 64, 36, 18, 16, 28, 98, 93 } };
-    long[] result = multiDimKeyVarLengthVariableSplitGenerator.getKeyArray(key);
-    assertThat(result, is(equalTo(result_value)));
-  }
-
-  @Test public void testKeyByteArray() {
-    byte[] result_value = new byte[] { 1, 102, 20, 56, 64, 36, 18, 16, 28, 98, 93 };
-    byte[][] key = new byte[][] { { 1, 102, 20, 56, 64, 36, 18, 16, 28, 98, 93 } };
-    byte[] result = multiDimKeyVarLengthVariableSplitGenerator.getKeyByteArray(key);
-    assertThat(result, is(equalTo(result_value)));
-  }
-
-  @Test public void testGetKeySizeByBlockWithBlockIndexesInRange() {
-    int result_value = 3;
-    int[] blockIndexes = new int[] { 1, 4 };
-    int result = multiDimKeyVarLengthVariableSplitGenerator.getKeySizeByBlock(blockIndexes);
-    assertEquals(result_value, result);
-  }
-
-  @Test public void testGetKeySizeByBlockWithBlockIndexes() {
-    int result_value = 9;
-    int[] blockIndexes = new int[] { 1, 4, 2, 0 };
-    int result = multiDimKeyVarLengthVariableSplitGenerator.getKeySizeByBlock(blockIndexes);
-    assertEquals(result_value, result);
-  }
-
-  @Test public void equalsWithError() {
-    Object obj = new Object();
-    boolean result = multiDimKeyVarLengthVariableSplitGenerator.equals(obj);
-    assertEquals(false, result);
-  }
-
-  @Test public void equalsWithTrue(){
-    boolean result = multiDimKeyVarLengthVariableSplitGenerator
-        .equals(multiDimKeyVarLengthVariableSplitGenerator);
-    assertEquals(true, result);
-  }
-
-  /**
-   * Test case for exception when Key size is less than byte key size
-   */
-
-  @Test(expected = ArrayIndexOutOfBoundsException.class) public void testSplitKeyWithException() {
-
-    int[] lens = new int[] { 32, 8, 32, 32, 16 };
-    int[] dimSplit = new int[] { 12, 8, 1, 8, 16 };
-    MultiDimKeyVarLengthVariableSplitGenerator multiDimKeyVarLengthVariableSplitGeneratorNew =
-        new MultiDimKeyVarLengthVariableSplitGenerator(lens, dimSplit);
-
-    byte[] key = new byte[] { 24, 56, 72, 48, 56, 36, 18 };
-    multiDimKeyVarLengthVariableSplitGeneratorNew.splitKey(key);
-  }
-}
diff --git a/core/src/test/java/org/apache/carbondata/core/keygenerator/factory/KeyGeneratorFactoryUnitTest.java b/core/src/test/java/org/apache/carbondata/core/keygenerator/factory/KeyGeneratorFactoryUnitTest.java
deleted file mode 100644
index 46ba7a3..0000000
--- a/core/src/test/java/org/apache/carbondata/core/keygenerator/factory/KeyGeneratorFactoryUnitTest.java
+++ /dev/null
@@ -1,61 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.carbondata.core.keygenerator.factory;
-
-import org.apache.carbondata.core.keygenerator.KeyGenerator;
-
-import org.junit.Test;
-
-import static junit.framework.Assert.assertEquals;
-
-import static org.apache.carbondata.core.keygenerator.factory.KeyGeneratorFactory.getKeyGenerator;
-
-public class KeyGeneratorFactoryUnitTest {
-
-  @Test public void testGetKeyGenerator() throws Exception {
-
-    int expected = 3;
-    int[] dimension = new int[] { 1, 2, 3 };
-    KeyGenerator result = getKeyGenerator(dimension);
-    assertEquals(expected, result.getDimCount());
-  }
-
-  /**
-   * Return 0 when we provide empty int[] in method.
-   *
-   * @throws Exception
-   */
-
-  @Test public void testGetKeyGeneratorNegative() throws Exception {
-
-    int expected = 0;
-    int[] dimension = new int[] {};
-    KeyGenerator result = getKeyGenerator(dimension);
-    assertEquals(expected, result.getDimCount());
-  }
-
-  @Test public void testGetKeyGenerato() throws Exception {
-
-    int expected = 9;
-    int[] dimCardinality = new int[] { 10, 20, 30, 11, 26, 52, 85, 65, 12 };
-    int[] columnSplits = new int[] { 2 };
-    KeyGenerator result = getKeyGenerator(dimCardinality, columnSplits);
-    assertEquals(expected, result.getDimCount());
-  }
-
-}
diff --git a/core/src/test/java/org/apache/carbondata/core/scan/complextypes/PrimitiveQueryTypeTest.java b/core/src/test/java/org/apache/carbondata/core/scan/complextypes/PrimitiveQueryTypeTest.java
index a662c23..930c41a 100644
--- a/core/src/test/java/org/apache/carbondata/core/scan/complextypes/PrimitiveQueryTypeTest.java
+++ b/core/src/test/java/org/apache/carbondata/core/scan/complextypes/PrimitiveQueryTypeTest.java
@@ -30,9 +30,7 @@ import org.junit.Test;
 import static org.junit.Assert.assertEquals;
 
 public class PrimitiveQueryTypeTest {
-  private static PrimitiveQueryType primitiveQueryType, primitiveQueryTypeForInt,
-      primitiveQueryTypeForLong, primitiveQueryTypeForDouble, primitiveQueryTypeForBoolean,
-      primitiveQueryTypeForTimeStamp, primitiveQueryTypeForTimeStampForIsDictionaryFalse;
+  private static PrimitiveQueryType primitiveQueryTypeForDate;
 
   @BeforeClass public static void setUp() {
     String name = "test";
@@ -40,40 +38,17 @@ public class PrimitiveQueryTypeTest {
     int blockIndex = 1;
     int keySize = 1;
     boolean isDirectDictionary = true;
-    primitiveQueryType =
-        new PrimitiveQueryType(name, parentName, blockIndex, DataTypes.STRING, keySize,
-            isDirectDictionary);
-    primitiveQueryTypeForInt =
-        new PrimitiveQueryType(name, parentName, blockIndex, DataTypes.INT, keySize,
-            isDirectDictionary);
-    primitiveQueryTypeForDouble =
-        new PrimitiveQueryType(name, parentName, blockIndex, DataTypes.DOUBLE, keySize,
-            isDirectDictionary);
-    primitiveQueryTypeForLong =
-        new PrimitiveQueryType(name, parentName, blockIndex, DataTypes.LONG, keySize,
-            isDirectDictionary);
-    primitiveQueryTypeForBoolean =
-        new PrimitiveQueryType(name, parentName, blockIndex, DataTypes.BOOLEAN, keySize,
-            isDirectDictionary);
-    primitiveQueryTypeForTimeStamp =
-        new PrimitiveQueryType(name, parentName, blockIndex, DataTypes.TIMESTAMP, keySize,
-            isDirectDictionary);
-    primitiveQueryTypeForTimeStampForIsDictionaryFalse =
-        new PrimitiveQueryType(name, parentName, blockIndex, DataTypes.TIMESTAMP, keySize, false);
+   primitiveQueryTypeForDate =
+        new PrimitiveQueryType(name, parentName, blockIndex, DataTypes.DATE, true);
   }
 
   @Test public void testGetDataBasedOnDataTypeFromSurrogates() {
     ByteBuffer surrogateData = ByteBuffer.allocate(10);
     surrogateData.put(3, (byte) 1);
-    new MockUp<Bits>() {
-      @Mock public long[] getKeyArray(byte[] key, int offset) {
-        return new long[] { 1313045L };
-      }
-    };
-    Object expectedValue = 1313043000000L;
+    Object expectedValue = 1;
 
     Object actualValue =
-        primitiveQueryTypeForTimeStamp.getDataBasedOnDataType(surrogateData);
+        primitiveQueryTypeForDate.getDataBasedOnDataType(surrogateData);
     assertEquals(expectedValue, actualValue);
   }
 
diff --git a/core/src/test/java/org/apache/carbondata/core/scan/executor/util/QueryUtilTest.java b/core/src/test/java/org/apache/carbondata/core/scan/executor/util/QueryUtilTest.java
index d02c575..c7bd7a5 100644
--- a/core/src/test/java/org/apache/carbondata/core/scan/executor/util/QueryUtilTest.java
+++ b/core/src/test/java/org/apache/carbondata/core/scan/executor/util/QueryUtilTest.java
@@ -41,86 +41,6 @@ public class QueryUtilTest extends TestCase {
     segmentProperties = SegmentPropertiesTestUtil.getSegmentProperties();
   }
 
-  @Test public void testGetMaskedByteRangeGivingProperMaksedByteRange() {
-
-    ProjectionDimension dimension =
-        new ProjectionDimension(segmentProperties.getDimensions().get(0));
-    int[] maskedByteRange = QueryUtil
-        .getMaskedByteRange(Arrays.asList(dimension), segmentProperties.getDimensionKeyGenerator());
-    int[] expectedMaskedByteRange = { 0 };
-    for (int i = 0; i < maskedByteRange.length; i++) {
-      assertEquals(expectedMaskedByteRange[i], maskedByteRange[i]);
-    }
-  }
-
-  @Test public void testGetMaskedByteRangeGivingProperMaksedByteRangeOnlyForDictionaryKey() {
-    List<ProjectionDimension> dimensions = new ArrayList<ProjectionDimension>();
-    for (int i = 0; i < 2; i++) {
-      ProjectionDimension dimension =
-          new ProjectionDimension(segmentProperties.getDimensions().get(i));
-      dimensions.add(dimension);
-    }
-    int[] maskedByteRange =
-        QueryUtil.getMaskedByteRange(dimensions, segmentProperties.getDimensionKeyGenerator());
-    int[] expectedMaskedByteRange = { 0 };
-    for (int i = 0; i < maskedByteRange.length; i++) {
-      assertEquals(expectedMaskedByteRange[i], maskedByteRange[i]);
-    }
-  }
-
-  @Test public void testGetMaskedByteRangeBasedOrdinalGivingProperMaskedByte() {
-    List<Integer> dimensionOrdinal = new ArrayList<Integer>();
-    dimensionOrdinal.add(0);
-    int[] maskedByteRange = QueryUtil.getMaskedByteRangeBasedOrdinal(dimensionOrdinal,
-        segmentProperties.getDimensionKeyGenerator());
-    int[] expectedMaskedByteRange = { 0 };
-    for (int i = 0; i < maskedByteRange.length; i++) {
-      assertEquals(expectedMaskedByteRange[i], maskedByteRange[i]);
-    }
-  }
-
-  @Test public void testGetMaxKeyBasedOnDimensions() {
-    List<ProjectionDimension> dimensions = new ArrayList<ProjectionDimension>();
-    for (int i = 0; i < 2; i++) {
-      ProjectionDimension dimension =
-          new ProjectionDimension(segmentProperties.getDimensions().get(i));
-      dimensions.add(dimension);
-    }
-    byte[] maxKeyBasedOnDimensions = null;
-    maxKeyBasedOnDimensions = QueryUtil
-        .getMaxKeyBasedOnDimensions(dimensions, segmentProperties.getDimensionKeyGenerator());
-    byte[] expectedMaxKeyBasedOnDimensions = { -1, 0, 0, 0, 0, 0 };
-    for (int i = 0; i < expectedMaxKeyBasedOnDimensions.length; i++) {
-      if (expectedMaxKeyBasedOnDimensions[i] != maxKeyBasedOnDimensions[i]) {
-        assertTrue(false);
-      }
-    }
-    long[] expectedKeyArray = { 255, 0, 0, 0, 0, 0 };
-    long[] keyArray =
-        segmentProperties.getDimensionKeyGenerator().getKeyArray(maxKeyBasedOnDimensions);
-    for (int i = 0; i < keyArray.length; i++) {
-      if (expectedKeyArray[i] != keyArray[i]) {
-        assertTrue(false);
-      }
-    }
-  }
-
-  @Test public void testGetMaksedByte() {
-    ProjectionDimension dimension =
-        new ProjectionDimension(segmentProperties.getDimensions().get(0));
-    int[] maskedByteRange = QueryUtil
-        .getMaskedByteRange(Arrays.asList(dimension), segmentProperties.getDimensionKeyGenerator());
-    int[] maskedByte = QueryUtil
-        .getMaskedByte(segmentProperties.getDimensionKeyGenerator().getDimCount(), maskedByteRange);
-    int[] expectedMaskedByte = { 0, -1, -1, -1, -1, -1 };
-
-    for (int i = 0; i < expectedMaskedByte.length; i++) {
-      if (expectedMaskedByte[i] != maskedByte[i]) {
-        assertTrue(false);
-      }
-    }
-  }
-
   @Test public void testSearchInArrayWithSearchInputNotPresentInArray() {
     int[] dummyArray = { 1, 2, 3, 4, 5 };
     int searchInput = 6;
@@ -145,16 +65,6 @@ public class QueryUtilTest extends TestCase {
     assertArrayEquals(expectedValue, actualValue);
   }
 
-  @Test public void testGetMaxKeyBasedOnOrdinal() throws Exception {
-    List<Integer> dummyList = new ArrayList<>();
-    dummyList.add(0, 1);
-    dummyList.add(1, 2);
-    byte[] actualValue =
-        QueryUtil.getMaxKeyBasedOnOrdinal(dummyList, segmentProperties.getDimensionKeyGenerator());
-    byte[] expectedValue = { 0, -1, -1, 0, 0, 0 };
-    assertArrayEquals(expectedValue, actualValue);
-  }
-
   @Test public void testGetSortDimensionIndexes() {
     List<ProjectionDimension> sortedDimensions = new ArrayList<ProjectionDimension>();
     for (int i = 0; i < 2; i++) {
diff --git a/core/src/test/java/org/apache/carbondata/core/scan/filter/FilterUtilTest.java b/core/src/test/java/org/apache/carbondata/core/scan/filter/FilterUtilTest.java
index 613a2a6..d15e852 100644
--- a/core/src/test/java/org/apache/carbondata/core/scan/filter/FilterUtilTest.java
+++ b/core/src/test/java/org/apache/carbondata/core/scan/filter/FilterUtilTest.java
@@ -22,14 +22,7 @@ import java.util.List;
 import java.util.UUID;
 
 import org.apache.carbondata.core.constants.CarbonCommonConstants;
-import org.apache.carbondata.core.datastore.IndexKey;
-import org.apache.carbondata.core.datastore.block.SegmentProperties;
-import org.apache.carbondata.core.keygenerator.KeyGenException;
-import org.apache.carbondata.core.keygenerator.mdkey.MultiDimKeyVarLengthGenerator;
 import org.apache.carbondata.core.metadata.datatype.DataTypes;
-import org.apache.carbondata.core.metadata.encoder.Encoding;
-import org.apache.carbondata.core.metadata.schema.table.column.CarbonColumn;
-import org.apache.carbondata.core.metadata.schema.table.column.CarbonDimension;
 import org.apache.carbondata.core.metadata.schema.table.column.ColumnSchema;
 import org.apache.carbondata.core.scan.expression.ColumnExpression;
 import org.apache.carbondata.core.scan.expression.Expression;
@@ -39,17 +32,13 @@ import org.apache.carbondata.core.scan.expression.conditional.ListExpression;
 import org.apache.carbondata.core.scan.expression.exception.FilterUnsupportedException;
 import org.apache.carbondata.core.scan.expression.logical.AndExpression;
 import org.apache.carbondata.core.scan.expression.logical.TrueExpression;
-import org.apache.carbondata.core.scan.filter.intf.RowImpl;
 import org.apache.carbondata.core.util.BitSetGroup;
 
-import mockit.Mock;
-import mockit.MockUp;
 import org.junit.Before;
 import org.junit.Test;
 
 import static junit.framework.TestCase.assertFalse;
 import static junit.framework.TestCase.assertTrue;
-import static org.junit.Assert.assertArrayEquals;
 import static org.junit.Assert.assertEquals;
 
 public class FilterUtilTest {
@@ -173,18 +162,6 @@ public class FilterUtilTest {
     assertEquals(expectedValue, actualValue);
   }
 
-  @Test public void testCreateIndexKeyFromResolvedFilterVal() throws Exception {
-    long[] startOrEndKey = new long[] { 0, 10 };
-    byte[] startOrEndKeyForNoDictDimension = { 1, 2 };
-    int[] keys = new int[] { 1, 2 };
-    MultiDimKeyVarLengthGenerator multiDimKeyVarLengthGenerator =
-        new MultiDimKeyVarLengthGenerator(keys);
-    assertTrue(FilterUtil
-        .createIndexKeyFromResolvedFilterVal(startOrEndKey, multiDimKeyVarLengthGenerator,
-            startOrEndKeyForNoDictDimension) != null);
-
-  }
-
   @Test public void testCheckIfExpressionContainsColumn() {
     String columnName = "IMEI";
     Expression expression = new ColumnExpression(columnName, DataTypes.STRING);
@@ -216,48 +193,12 @@ public class FilterUtilTest {
     assertTrue(result);
   }
 
-  @Test public void testGetMaskKey() {
-    int surrogate = 1;
-    int[] keys = new int[] { 1, 2 };
-    MultiDimKeyVarLengthGenerator multiDimKeyVarLengthGenerator =
-        new MultiDimKeyVarLengthGenerator(keys);
-    int ordinal = 1;
-    int keyOrdinal = 1;
-    int complexTypeOrdinal = 1;
-    ColumnSchema columnSchema = new ColumnSchema();
-    columnSchema.setColumnName("IMEI");
-    columnSchema.setColumnUniqueId(UUID.randomUUID().toString());
-    columnSchema.setDataType(DataTypes.STRING);
-    columnSchema.setDimensionColumn(true);
-    CarbonDimension carbonDimension =
-        new CarbonDimension(columnSchema, ordinal, keyOrdinal, complexTypeOrdinal);
-    byte[] expectedResult = new byte[] { 1 };
-    byte[] actualResult =
-        FilterUtil.getMaskKey(surrogate, carbonDimension, multiDimKeyVarLengthGenerator);
-    assertArrayEquals(expectedResult, actualResult);
-  }
-
   @Test public void testCheckIfDataTypeNotTimeStamp() {
     Expression expression = new ColumnExpression("test", DataTypes.STRING);
     boolean result = FilterUtil.checkIfDataTypeNotTimeStamp(expression);
     assertFalse(result);
   }
 
-  @Test public void testPrepareDefaultEndIndexKey() throws Exception {
-    List<ColumnSchema> columnsInTable = new ArrayList<>();
-    columnsInTable.add(columnSchema);
-    int[] columnCardinality = new int[] { 1, 2 };
-    new MockUp<ColumnSchema>() {
-      @Mock public List<Encoding> getEncodingList() {
-        List<Encoding> encodingList = new ArrayList<>();
-        encodingList.add(Encoding.DICTIONARY);
-        return encodingList;
-      }
-    };
-    SegmentProperties segmentProperties = new SegmentProperties(columnsInTable, columnCardinality);
-    assertTrue(FilterUtil.prepareDefaultEndIndexKey(segmentProperties) instanceof IndexKey);
-  }
-
   @Test public void testCheckIfRightExpressionRequireEvaluation() {
     Expression expression = new ColumnExpression("test", DataTypes.STRING);
     boolean result = FilterUtil.checkIfRightExpressionRequireEvaluation(expression);
@@ -282,21 +223,6 @@ public class FilterUtilTest {
             DataTypes.STRING) instanceof ColumnFilterInfo);
   }
 
-  @Test public void testPrepareDefaultStartIndexKey() throws KeyGenException {
-    List<ColumnSchema> columnsInTable = new ArrayList<>();
-    columnsInTable.add(columnSchema);
-    int[] columnCardinality = new int[] { 1, 2 };
-    new MockUp<ColumnSchema>() {
-      @Mock public List<Encoding> getEncodingList() {
-        List<Encoding> encodingList = new ArrayList<>();
-        encodingList.add(Encoding.DICTIONARY);
-        return encodingList;
-      }
-    };
-    SegmentProperties segmentProperties = new SegmentProperties(columnsInTable, columnCardinality);
-    assertTrue(FilterUtil.prepareDefaultStartIndexKey(segmentProperties) instanceof IndexKey);
-  }
-
   @Test public void testCreateBitSetGroupWithDefaultValue() {
     // test for exactly divisible values
     BitSetGroup bitSetGroupWithDefaultValue =
diff --git a/core/src/test/java/org/apache/carbondata/core/util/CarbonMetadataUtilTest.java b/core/src/test/java/org/apache/carbondata/core/util/CarbonMetadataUtilTest.java
index 7aa2236..f5eeb97 100644
--- a/core/src/test/java/org/apache/carbondata/core/util/CarbonMetadataUtilTest.java
+++ b/core/src/test/java/org/apache/carbondata/core/util/CarbonMetadataUtilTest.java
@@ -147,17 +147,16 @@ public class CarbonMetadataUtilTest {
   }
 
   @Test public void testGetIndexHeader() {
-    int[] columnCardinality = { 1, 2, 3, 4 };
     SegmentInfo segmentInfo = new SegmentInfo();
     segmentInfo.setNum_cols(0);
-    segmentInfo.setColumn_cardinalities(CarbonUtil.convertToIntegerList(columnCardinality));
+    segmentInfo.setColumn_cardinalities(CarbonUtil.convertToIntegerList(new int[0]));
     IndexHeader indexHeader = new IndexHeader();
     indexHeader.setVersion(3);
     indexHeader.setSegment_info(segmentInfo);
     indexHeader.setTable_columns(columnSchemaList);
     indexHeader.setBucket_id(0);
     indexHeader.setSchema_time_stamp(0L);
-    IndexHeader indexheaderResult = getIndexHeader(columnCardinality, columnSchemaList, 0, 0L);
+    IndexHeader indexheaderResult = getIndexHeader(columnSchemaList, 0, 0L);
     assertEquals(indexHeader, indexheaderResult);
   }
 
diff --git a/core/src/test/java/org/apache/carbondata/core/util/CarbonUtilTest.java b/core/src/test/java/org/apache/carbondata/core/util/CarbonUtilTest.java
index 4dd4316..8378c4f 100644
--- a/core/src/test/java/org/apache/carbondata/core/util/CarbonUtilTest.java
+++ b/core/src/test/java/org/apache/carbondata/core/util/CarbonUtilTest.java
@@ -36,9 +36,7 @@ import org.apache.carbondata.core.datastore.chunk.impl.FixedLengthDimensionColum
 import org.apache.carbondata.core.datastore.filesystem.LocalCarbonFile;
 import org.apache.carbondata.core.datastore.impl.FileFactory;
 import org.apache.carbondata.core.metadata.ColumnarFormatVersion;
-import org.apache.carbondata.core.metadata.ValueEncoderMeta;
 import org.apache.carbondata.core.metadata.blocklet.DataFileFooter;
-import org.apache.carbondata.core.metadata.blocklet.datachunk.DataChunk;
 import org.apache.carbondata.core.metadata.datatype.DataType;
 import org.apache.carbondata.core.metadata.datatype.DataTypes;
 import org.apache.carbondata.core.metadata.encoder.Encoding;
@@ -550,26 +548,6 @@ public class CarbonUtilTest {
         new CarbonDimension(column1Schema, 1, 1, 1));
   }
 
-  @Test public void testToGetFormattedCardinality() {
-    ColumnSchema column1Schema = new ColumnSchema();
-    ColumnSchema column2Schema = new ColumnSchema();
-    List<Encoding> encoding = new ArrayList<>();
-    encoding.add(Encoding.DICTIONARY);
-    List<Encoding> encoding2 = new ArrayList<>();
-    encoding2.add(Encoding.DIRECT_DICTIONARY);
-    column1Schema.setEncodingList(encoding);
-    column2Schema.setEncodingList(encoding2);
-    List<ColumnSchema> columnSchemas = new ArrayList<>();
-    columnSchemas.add(column1Schema);
-    columnSchemas.add(column2Schema);
-    int[] columnCardinality = { 1, 5 };
-    int[] result = CarbonUtil.getFormattedCardinality(columnCardinality, columnSchemas);
-    int[] expectedResult = { 1, 5 };
-    for (int i = 0; i < result.length; i++) {
-      assertEquals(result[i], expectedResult[i]);
-    }
-  }
-
   @Test public void testToGetColumnSchemaList() {
     ColumnSchema column1Schema = new ColumnSchema();
     ColumnSchema column2Schema = new ColumnSchema();
@@ -652,27 +630,6 @@ public class CarbonUtilTest {
     assertEquals(a, 257);
   }
 
-  @Test public void testToGetValueCompressionModel() {
-    List<DataChunk> dataChunkList = new ArrayList<>();
-    DataChunk dataChunk = new DataChunk();
-
-    List<Encoding> encodingList = new ArrayList<>();
-    encodingList.add(Encoding.DELTA);
-    dataChunk.setEncodingList(encodingList);
-
-    List<ValueEncoderMeta> valueEncoderMetas = new ArrayList<>();
-    ValueEncoderMeta valueEncoderMeta = new ValueEncoderMeta();
-    valueEncoderMeta.setMaxValue(5.0);
-    valueEncoderMeta.setMinValue(1.0);
-    valueEncoderMeta.setUniqueValue(2.0);
-    valueEncoderMeta.setType('n');
-    valueEncoderMeta.setDataTypeSelected((byte) 'v');
-    valueEncoderMetas.add(valueEncoderMeta);
-    dataChunk.setValueEncoderMeta(valueEncoderMetas);
-    dataChunkList.add(dataChunk);
-    assertEquals(1, dataChunkList.get(0).getValueEncoderMeta().size());
-  }
-
   @Test public void testToGetDictionaryChunkSize() {
     new MockUp<CarbonProperties>() {
       @SuppressWarnings("unused") @Mock public CarbonProperties getInstance()
diff --git a/core/src/test/java/org/apache/carbondata/core/util/DataFileFooterConverterTest.java b/core/src/test/java/org/apache/carbondata/core/util/DataFileFooterConverterTest.java
index a60491e..bcaa398 100644
--- a/core/src/test/java/org/apache/carbondata/core/util/DataFileFooterConverterTest.java
+++ b/core/src/test/java/org/apache/carbondata/core/util/DataFileFooterConverterTest.java
@@ -31,7 +31,6 @@ import org.apache.carbondata.core.datastore.impl.FileFactory;
 import org.apache.carbondata.core.datastore.impl.FileReaderImpl;
 import org.apache.carbondata.core.metadata.ColumnarFormatVersion;
 import org.apache.carbondata.core.metadata.blocklet.DataFileFooter;
-import org.apache.carbondata.core.metadata.blocklet.SegmentInfo;
 import org.apache.carbondata.core.reader.CarbonFooterReader;
 import org.apache.carbondata.core.reader.CarbonIndexFileReader;
 import org.apache.carbondata.core.reader.ThriftReader;
@@ -151,7 +150,6 @@ public class DataFileFooterConverterTest {
     String[] arr = { "a", "b", "c" };
     String fileName = "/part-0-0_batchno0-0-1495074251740.carbondata";
     TableBlockInfo tableBlockInfo = new TableBlockInfo(fileName, 3, "id", arr, 3, ColumnarFormatVersion.V1, null);
-    tableBlockInfo.getBlockletInfos().setNoOfBlockLets(3);
     List<TableBlockInfo> tableBlockInfoList = new ArrayList<>();
     tableBlockInfoList.add(tableBlockInfo);
     String idxFileName = "0_batchno0-0-1495074251740.carbonindex";
@@ -247,11 +245,7 @@ public class DataFileFooterConverterTest {
         return fileFooter;
       }
     };
-    SegmentInfo segmentInfo = new SegmentInfo();
-    int[] arr = { 1, 2, 3 };
-    segmentInfo.setColumnCardinality(arr);
     dataFileFooter.setNumberOfRows(3);
-    dataFileFooter.setSegmentInfo(segmentInfo);
     TableBlockInfo info = new TableBlockInfo("/file.carbondata", 1, "0", new String[0], 1, ColumnarFormatVersion.V1, null);
     DataFileFooter result = dataFileFooterConverter.readDataFileFooter(info);
     assertEquals(result.getNumberOfRows(), 3);
diff --git a/core/src/test/java/org/apache/carbondata/core/util/RangeFilterProcessorTest.java b/core/src/test/java/org/apache/carbondata/core/util/RangeFilterProcessorTest.java
index ee74bf2..2b1704e 100644
--- a/core/src/test/java/org/apache/carbondata/core/util/RangeFilterProcessorTest.java
+++ b/core/src/test/java/org/apache/carbondata/core/util/RangeFilterProcessorTest.java
@@ -17,13 +17,10 @@
 
 package org.apache.carbondata.core.util;
 
-import java.io.IOException;
 import java.util.Arrays;
 
-import org.apache.carbondata.core.metadata.datatype.DataType;
 import org.apache.carbondata.core.metadata.datatype.DataTypes;
 import org.apache.carbondata.core.metadata.encoder.Encoding;
-import org.apache.carbondata.core.metadata.schema.table.column.CarbonColumn;
 import org.apache.carbondata.core.metadata.schema.table.column.CarbonDimension;
 import org.apache.carbondata.core.metadata.schema.table.column.ColumnSchema;
 import org.apache.carbondata.core.scan.expression.ColumnExpression;
@@ -43,11 +40,7 @@ import org.apache.carbondata.core.scan.filter.optimizer.RangeFilterOptmizer;
 import org.apache.carbondata.core.scan.filter.resolver.resolverinfo.DimColumnResolvedFilterInfo;
 
 import mockit.Deencapsulation;
-import mockit.Mock;
 import mockit.MockUp;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
 import org.junit.AfterClass;
 import org.junit.Assert;
 import org.junit.BeforeClass;
@@ -98,7 +91,7 @@ public class RangeFilterProcessorTest {
     empColumnSchema.setDimensionColumn(true);
     empColumnSchema.setEncodingList(Arrays.asList(Encoding.DICTIONARY));
     empColumnSchema.setDataType(DataTypes.STRING);
-    CarbonDimension empDimension = new CarbonDimension(empColumnSchema, 0, 0, 0, 0);
+    CarbonDimension empDimension = new CarbonDimension(empColumnSchema, 0, 0, 0);
     cola.setDimension(empDimension);
 
     Expression greaterThan =
@@ -139,7 +132,7 @@ public class RangeFilterProcessorTest {
     empColumnSchema.setDimensionColumn(true);
     empColumnSchema.setEncodingList(Arrays.asList(Encoding.DICTIONARY));
     empColumnSchema.setDataType(DataTypes.STRING);
-    CarbonDimension empDimension = new CarbonDimension(empColumnSchema, 0, 0, 0, 0);
+    CarbonDimension empDimension = new CarbonDimension(empColumnSchema, 0, 0, 0);
     cola.setDimension(empDimension);
 
     Expression greaterThan =
@@ -178,7 +171,7 @@ public class RangeFilterProcessorTest {
     empColumnSchema.setDimensionColumn(true);
     empColumnSchema.setEncodingList(Arrays.asList(Encoding.DICTIONARY));
     empColumnSchema.setDataType(DataTypes.STRING);
-    CarbonDimension empDimension = new CarbonDimension(empColumnSchema, 0, 0, 0, 0);
+    CarbonDimension empDimension = new CarbonDimension(empColumnSchema, 0, 0, 0);
 
     ColumnExpression cola1 = new ColumnExpression("a", DataTypes.STRING);
     cola1.setDimension(true);
@@ -253,7 +246,7 @@ public class RangeFilterProcessorTest {
     empColumnSchema.setDimensionColumn(true);
     empColumnSchema.setEncodingList(Arrays.asList(Encoding.DICTIONARY));
     empColumnSchema.setDataType(DataTypes.STRING);
-    CarbonDimension empDimension = new CarbonDimension(empColumnSchema, 0, 0, 0, 0);
+    CarbonDimension empDimension = new CarbonDimension(empColumnSchema, 0, 0, 0);
 
     ColumnExpression cola1 = new ColumnExpression("a", DataTypes.STRING);
     cola1.setDimension(true);
diff --git a/datamap/bloom/src/main/java/org/apache/carbondata/datamap/bloom/BloomDataMapWriter.java b/datamap/bloom/src/main/java/org/apache/carbondata/datamap/bloom/BloomDataMapWriter.java
index d507409..952043f 100644
--- a/datamap/bloom/src/main/java/org/apache/carbondata/datamap/bloom/BloomDataMapWriter.java
+++ b/datamap/bloom/src/main/java/org/apache/carbondata/datamap/bloom/BloomDataMapWriter.java
@@ -18,23 +18,17 @@
 package org.apache.carbondata.datamap.bloom;
 
 import java.io.IOException;
-import java.util.HashMap;
 import java.util.List;
-import java.util.Map;
 
 import org.apache.carbondata.common.annotations.InterfaceAudience;
 import org.apache.carbondata.core.datamap.Segment;
 import org.apache.carbondata.core.datastore.block.SegmentProperties;
-import org.apache.carbondata.core.keygenerator.columnar.ColumnarSplitter;
 import org.apache.carbondata.core.metadata.datatype.DataTypes;
 import org.apache.carbondata.core.metadata.schema.table.column.CarbonColumn;
-import org.apache.carbondata.core.metadata.schema.table.column.CarbonDimension;
+import org.apache.carbondata.core.util.ByteUtil;
 import org.apache.carbondata.core.util.CarbonUtil;
 import org.apache.carbondata.core.util.DataTypeUtil;
 
-import org.apache.commons.collections.CollectionUtils;
-import org.apache.commons.collections.Predicate;
-
 /**
  * BloomDataMap is constructed in CG level (blocklet level).
  * For each indexed column, a bloom filter is constructed to indicate whether a value
@@ -44,10 +38,6 @@ import org.apache.commons.collections.Predicate;
  */
 @InterfaceAudience.Internal
 public class BloomDataMapWriter extends AbstractBloomDataMapWriter {
-  private ColumnarSplitter columnarSplitter;
-  // for the dict/sort/date column, they are encoded in MDK,
-  // this maps the index column name to the index in MDK
-  private Map<String, Integer> indexCol2MdkIdx;
 
   BloomDataMapWriter(String tablePath, String dataMapName, List<CarbonColumn> indexColumns,
       Segment segment, String shardName, SegmentProperties segmentProperties,
@@ -55,25 +45,6 @@ public class BloomDataMapWriter extends AbstractBloomDataMapWriter {
       throws IOException {
     super(tablePath, dataMapName, indexColumns, segment, shardName, segmentProperties,
         bloomFilterSize, bloomFilterFpp, compressBloom);
-
-    columnarSplitter = segmentProperties.getFixedLengthKeySplitter();
-    this.indexCol2MdkIdx = new HashMap<>();
-    int idx = 0;
-    for (final CarbonDimension dimension : segmentProperties.getDimensions()) {
-      if (dimension.getDataType() != DataTypes.DATE) {
-        continue;
-      }
-      boolean isExistInIndex = CollectionUtils.exists(indexColumns, new Predicate() {
-        @Override
-        public boolean evaluate(Object object) {
-          return ((CarbonColumn) object).getColName().equalsIgnoreCase(dimension.getColName());
-        }
-      });
-      if (isExistInIndex) {
-        this.indexCol2MdkIdx.put(dimension.getColName(), idx);
-      }
-      idx++;
-    }
   }
 
   protected byte[] convertNonDictionaryValue(int indexColIdx, Object value) {
@@ -92,9 +63,8 @@ public class BloomDataMapWriter extends AbstractBloomDataMapWriter {
     // input value from onPageAdded in load process is byte[]
 
     // for dict columns including dictionary and date columns decode value to get the surrogate key
-    int thisKeyIdx = indexCol2MdkIdx.get(indexColumns.get(indexColIdx).getColName());
     int surrogateKey = CarbonUtil.getSurrogateInternal((byte[]) value, 0,
-        columnarSplitter.getBlockKeySize()[thisKeyIdx]);
+        ByteUtil.dateBytesSize());
     // store the dictionary key in bloom
     return CarbonUtil.getValueAsBytes(DataTypes.INT, surrogateKey);
   }
diff --git a/hadoop/src/main/java/org/apache/carbondata/hadoop/stream/CarbonStreamInputFormat.java b/hadoop/src/main/java/org/apache/carbondata/hadoop/stream/CarbonStreamInputFormat.java
index b4ffed8..df37e09 100644
--- a/hadoop/src/main/java/org/apache/carbondata/hadoop/stream/CarbonStreamInputFormat.java
+++ b/hadoop/src/main/java/org/apache/carbondata/hadoop/stream/CarbonStreamInputFormat.java
@@ -110,29 +110,29 @@ public class CarbonStreamInputFormat extends FileInputFormat<Void, Object> {
   }
 
   private static void fillChildren(GenericQueryType parentQueryType, CarbonDimension dimension,
-      int parentBlockIndex) {
+      int parentColumnIndex) {
     for (int i = 0; i < dimension.getNumberOfChild(); i++) {
       CarbonDimension child = dimension.getListOfChildDimensions().get(i);
       DataType dataType = child.getDataType();
       GenericQueryType queryType = null;
       if (DataTypes.isArrayType(dataType)) {
         queryType =
-            new ArrayQueryType(child.getColName(), dimension.getColName(), ++parentBlockIndex);
+            new ArrayQueryType(child.getColName(), dimension.getColName(), ++parentColumnIndex);
 
       } else if (DataTypes.isStructType(dataType)) {
         queryType =
-            new StructQueryType(child.getColName(), dimension.getColName(), ++parentBlockIndex);
+            new StructQueryType(child.getColName(), dimension.getColName(), ++parentColumnIndex);
         parentQueryType.addChildren(queryType);
       } else {
         boolean isDirectDictionary =
             CarbonUtil.hasEncoding(child.getEncoder(), Encoding.DIRECT_DICTIONARY);
         queryType =
-            new PrimitiveQueryType(child.getColName(), dimension.getColName(), ++parentBlockIndex,
-                child.getDataType(), 4, isDirectDictionary);
+            new PrimitiveQueryType(child.getColName(), dimension.getColName(), ++parentColumnIndex,
+                child.getDataType(), isDirectDictionary);
       }
       parentQueryType.addChildren(queryType);
       if (child.getNumberOfChild() > 0) {
-        fillChildren(queryType, child, parentBlockIndex);
+        fillChildren(queryType, child, parentColumnIndex);
       }
     }
   }
diff --git a/hadoop/src/main/java/org/apache/carbondata/hadoop/stream/StreamRecordReader.java b/hadoop/src/main/java/org/apache/carbondata/hadoop/stream/StreamRecordReader.java
index bf4c7d4..dcdee41 100644
--- a/hadoop/src/main/java/org/apache/carbondata/hadoop/stream/StreamRecordReader.java
+++ b/hadoop/src/main/java/org/apache/carbondata/hadoop/stream/StreamRecordReader.java
@@ -20,7 +20,6 @@ package org.apache.carbondata.hadoop.stream;
 import java.io.IOException;
 import java.math.BigDecimal;
 import java.nio.ByteBuffer;
-import java.util.Arrays;
 import java.util.BitSet;
 import java.util.HashMap;
 import java.util.List;
@@ -215,13 +214,8 @@ public class StreamRecordReader extends RecordReader<Void, Object> {
   private void initializeFilter() {
     List<ColumnSchema> wrapperColumnSchemaList = CarbonUtil
         .getColumnSchemaList(carbonTable.getVisibleDimensions(), carbonTable.getVisibleMeasures());
-    int[] dimLensWithComplex = new int[wrapperColumnSchemaList.size()];
-    Arrays.fill(dimLensWithComplex, Integer.MAX_VALUE);
 
-    int[] dictionaryColumnCardinality =
-        CarbonUtil.getFormattedCardinality(dimLensWithComplex, wrapperColumnSchemaList);
-    SegmentProperties segmentProperties =
-        new SegmentProperties(wrapperColumnSchemaList, dictionaryColumnCardinality);
+    SegmentProperties segmentProperties = new SegmentProperties(wrapperColumnSchemaList);
     Map<Integer, GenericQueryType> complexDimensionInfoMap = new HashMap<>();
 
     FilterResolverIntf resolverIntf = model.getDataMapFilter().getResolver();
diff --git a/integration/hive/src/main/java/org/apache/carbondata/hive/CarbonHiveInputSplit.java b/integration/hive/src/main/java/org/apache/carbondata/hive/CarbonHiveInputSplit.java
index 9cdaeeb..1da6542 100644
--- a/integration/hive/src/main/java/org/apache/carbondata/hive/CarbonHiveInputSplit.java
+++ b/integration/hive/src/main/java/org/apache/carbondata/hive/CarbonHiveInputSplit.java
@@ -27,7 +27,6 @@ import java.util.List;
 import java.util.Map;
 
 import org.apache.carbondata.core.constants.CarbonCommonConstants;
-import org.apache.carbondata.core.datastore.block.BlockletInfos;
 import org.apache.carbondata.core.datastore.block.Distributable;
 import org.apache.carbondata.core.datastore.block.TableBlockInfo;
 import org.apache.carbondata.core.indexstore.BlockletDetailInfo;
@@ -127,11 +126,9 @@ public class CarbonHiveInputSplit extends FileSplit
   public static List<TableBlockInfo> createBlocks(List<CarbonHiveInputSplit> splitList) {
     List<TableBlockInfo> tableBlockInfoList = new ArrayList<>();
     for (CarbonHiveInputSplit split : splitList) {
-      BlockletInfos blockletInfos =
-          new BlockletInfos(split.getNumberOfBlocklets(), 0, split.getNumberOfBlocklets());
       try {
         TableBlockInfo blockInfo = new TableBlockInfo(split.getPath().toString(), split.getStart(),
-                split.getSegmentId(), split.getLocations(), split.getLength(), blockletInfos,
+                split.getSegmentId(), split.getLocations(), split.getLength(),
                 split.getVersion(), null);
         blockInfo.setDetailInfo(split.getDetailInfo());
         blockInfo.setBlockOffset(split.getDetailInfo().getBlockFooterOffset());
@@ -143,22 +140,6 @@ public class CarbonHiveInputSplit extends FileSplit
     return tableBlockInfoList;
   }
 
-  public static TableBlockInfo getTableBlockInfo(CarbonHiveInputSplit inputSplit) {
-    BlockletInfos blockletInfos =
-        new BlockletInfos(inputSplit.getNumberOfBlocklets(), 0, inputSplit.getNumberOfBlocklets());
-    try {
-      TableBlockInfo blockInfo =
-              new TableBlockInfo(inputSplit.getPath().toString(), inputSplit.getStart(),
-          inputSplit.getSegmentId(), inputSplit.getLocations(), inputSplit.getLength(),
-          blockletInfos, inputSplit.getVersion(), null);
-      blockInfo.setDetailInfo(inputSplit.getDetailInfo());
-      blockInfo.setBlockOffset(inputSplit.getDetailInfo().getBlockFooterOffset());
-      return blockInfo;
-    } catch (IOException e) {
-      throw new RuntimeException("fail to get location of split: " + inputSplit, e);
-    }
-  }
-
   public String getSegmentId() {
     return segmentId;
   }
diff --git a/integration/hive/src/test/java/org/apache/carbondata/hive/Hive2CarbonExpressionTest.java b/integration/hive/src/test/java/org/apache/carbondata/hive/Hive2CarbonExpressionTest.java
index ffefa86..e6af27a 100644
--- a/integration/hive/src/test/java/org/apache/carbondata/hive/Hive2CarbonExpressionTest.java
+++ b/integration/hive/src/test/java/org/apache/carbondata/hive/Hive2CarbonExpressionTest.java
@@ -165,7 +165,7 @@ public class Hive2CarbonExpressionTest {
     format.setDatabaseName(job.getConfiguration(), table.getDatabaseName());
 
     List<InputSplit> list= format.getSplits(job);
-    Assert.assertTrue(list.size() == 1);
+    Assert.assertEquals(1, list.size());
   }
 
   @Test
@@ -199,7 +199,7 @@ public class Hive2CarbonExpressionTest {
     format.setDatabaseName(job.getConfiguration(), table.getDatabaseName());
 
     List<InputSplit> list= format.getSplits(job);
-    Assert.assertTrue(list.size() == 0);
+    Assert.assertEquals(0, list.size());
   }
 
   @Test
@@ -219,7 +219,7 @@ public class Hive2CarbonExpressionTest {
     format.setDatabaseName(job.getConfiguration(), table.getDatabaseName());
 
     List<InputSplit> list= format.getSplits(job);
-    Assert.assertTrue(list.size() == 0);
+    Assert.assertEquals(0, list.size());
   }
 
   @Test
@@ -265,7 +265,7 @@ public class Hive2CarbonExpressionTest {
     format.setDatabaseName(job.getConfiguration(), table.getDatabaseName());
 
     List<InputSplit> list= format.getSplits(job);
-    Assert.assertTrue(list.size() == 1);
+    Assert.assertEquals(1, list.size());
   }
 
   @Test
@@ -287,7 +287,7 @@ public class Hive2CarbonExpressionTest {
     format.setDatabaseName(job.getConfiguration(), table.getDatabaseName());
 
     List<InputSplit> list= format.getSplits(job);
-    Assert.assertTrue(list.size() == 1);
+    Assert.assertEquals(1, list.size());
 
   }
 
@@ -310,7 +310,7 @@ public class Hive2CarbonExpressionTest {
     format.setDatabaseName(job.getConfiguration(), table.getDatabaseName());
 
     List<InputSplit> list= format.getSplits(job);
-    Assert.assertTrue(list.size() == 1);
+    Assert.assertEquals(1, list.size());
 
   }
 
@@ -333,7 +333,7 @@ public class Hive2CarbonExpressionTest {
     format.setDatabaseName(job.getConfiguration(), table.getDatabaseName());
 
     List<InputSplit> list= format.getSplits(job);
-    Assert.assertTrue(list.size() == 0);
+    Assert.assertEquals(0, list.size());
 
   }
 
@@ -356,7 +356,7 @@ public class Hive2CarbonExpressionTest {
     format.setDatabaseName(job.getConfiguration(), table.getDatabaseName());
 
     List<InputSplit> list= format.getSplits(job);
-    Assert.assertTrue(list.size() == 0);
+    Assert.assertEquals(0, list.size());
 
   }
 }
diff --git a/integration/spark-common-cluster-test/src/test/scala/org/apache/carbondata/cluster/sdv/generated/LoadTableWithLocalDictionaryTestCase.scala b/integration/spark-common-cluster-test/src/test/scala/org/apache/carbondata/cluster/sdv/generated/LoadTableWithLocalDictionaryTestCase.scala
index 7139700..0837ac2 100644
--- a/integration/spark-common-cluster-test/src/test/scala/org/apache/carbondata/cluster/sdv/generated/LoadTableWithLocalDictionaryTestCase.scala
+++ b/integration/spark-common-cluster-test/src/test/scala/org/apache/carbondata/cluster/sdv/generated/LoadTableWithLocalDictionaryTestCase.scala
@@ -276,7 +276,6 @@ class LoadTableWithLocalDictionaryTestCase extends QueryTest with BeforeAndAfter
           .getInstance
           .getDimensionColumnChunkReader(ColumnarFormatVersion.V3,
             blockletInfo,
-            dataFileFooter.getSegmentInfo.getColumnCardinality,
             carbonDataFiles.getAbsolutePath,
             false).asInstanceOf[DimensionChunkReaderV3]
       dimensionRawColumnChunks
diff --git a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/allqueries/AllDataTypesTestCase.scala b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/allqueries/AllDataTypesTestCase.scala
index 4e43a03..68c0b84 100644
--- a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/allqueries/AllDataTypesTestCase.scala
+++ b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/allqueries/AllDataTypesTestCase.scala
@@ -17,12 +17,12 @@
 
 package org.apache.carbondata.spark.testsuite.allqueries
 
+import org.apache.spark.sql.test.util.QueryTest
 import org.apache.spark.sql.{Row, SaveMode}
 import org.scalatest.BeforeAndAfterAll
 
 import org.apache.carbondata.core.constants.CarbonCommonConstants
 import org.apache.carbondata.core.util.CarbonProperties
-import org.apache.spark.sql.test.util.QueryTest
 
 /**
   * Test Class for all query on multiple datatypes
diff --git a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/createTable/TestNonTransactionalCarbonTable.scala b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/createTable/TestNonTransactionalCarbonTable.scala
index 2f0acef..f9db7c2 100644
--- a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/createTable/TestNonTransactionalCarbonTable.scala
+++ b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/createTable/TestNonTransactionalCarbonTable.scala
@@ -2666,7 +2666,6 @@ object testUtil{
           .getInstance
           .getDimensionColumnChunkReader(ColumnarFormatVersion.V3,
             blockletInfo,
-            dataFileFooter.getSegmentInfo.getColumnCardinality,
             carbonDataFiles.getAbsolutePath,
             false).asInstanceOf[DimensionChunkReaderV3]
       dimensionRawColumnChunks
diff --git a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/localdictionary/LocalDictionarySupportLoadTableTest.scala b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/localdictionary/LocalDictionarySupportLoadTableTest.scala
index 97ac14b..b187b6b 100644
--- a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/localdictionary/LocalDictionarySupportLoadTableTest.scala
+++ b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/localdictionary/LocalDictionarySupportLoadTableTest.scala
@@ -292,7 +292,6 @@ class LocalDictionarySupportLoadTableTest extends QueryTest with BeforeAndAfterA
           .getInstance
           .getDimensionColumnChunkReader(ColumnarFormatVersion.V3,
             blockletInfo,
-            dataFileFooter.getSegmentInfo.getColumnCardinality,
             carbonDataFiles.getAbsolutePath,
             false).asInstanceOf[DimensionChunkReaderV3]
       dimensionRawColumnChunks
diff --git a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/load/DataLoadProcessorStepOnSpark.scala b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/load/DataLoadProcessorStepOnSpark.scala
index c21d984..1e22b6f 100644
--- a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/load/DataLoadProcessorStepOnSpark.scala
+++ b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/load/DataLoadProcessorStepOnSpark.scala
@@ -338,23 +338,13 @@ object DataLoadProcessorStepOnSpark {
         conf.get(CarbonCommonConstants.CARBON_WRITTEN_BY_APPNAME))
     ThreadLocalSessionInfo.setConfigurationToCurrentThread(conf)
     var tableName: String = null
-    var rowConverter: RowConverterImpl = null
     var dataWriter: DataWriterProcessorStepImpl = null
     try {
       val storeLocation = CommonUtil.getTempStoreLocations(index.toString)
       val conf = DataLoadProcessBuilder.createConfiguration(model, storeLocation)
 
       tableName = model.getTableName
-
-      // When we use sortBy, it means we have 2 stages. Stage1 can't get the finder from Stage2
-      // directly because they are in different processes. We need to set cardinality finder in
-      // Stage1 again.
-      rowConverter = new RowConverterImpl(conf.getDataFields, conf, null)
-      rowConverter.initialize()
-      conf.setCardinalityFinder(rowConverter)
-
       dataWriter = new DataWriterProcessorStepImpl(conf)
-
       val dataHandlerModel = dataWriter.getDataHandlerModel
       var dataHandler: CarbonFactHandler = null
       var rowsNotExist = true
@@ -381,12 +371,9 @@ object DataLoadProcessorStepOnSpark {
         LOGGER.error("Failed for table: " + tableName + " in Data Writer Step", e)
         throw new CarbonDataLoadingException("There is an unexpected error: " + e.getMessage, e)
     } finally {
-      if (rowConverter != null) {
-        rowConverter.finish()
-      }
       // close the dataWriter once the write in done success or fail. if not closed then thread to
       // to prints the rows processed in each step for every 10 seconds will never exit.
-      if(null != dataWriter) {
+      if (null != dataWriter) {
         dataWriter.close()
       }
       // clean up the folders and files created locally for data load operation
@@ -421,7 +408,6 @@ object DataLoadProcessorStepOnSpark {
     var model: CarbonLoadModel = null
     var tableName: String = null
     var inputProcessor: NewInputProcessorStepImpl = null
-    var rowConverter: RowConverterImpl = null
     var sortProcessor: SortProcessorStepImpl = null
     var dataWriter: DataWriterProcessorStepImpl = null
     try {
@@ -429,11 +415,6 @@ object DataLoadProcessorStepOnSpark {
       val storeLocation = CommonUtil.getTempStoreLocations(index.toString)
       val conf = DataLoadProcessBuilder.createConfiguration(model, storeLocation)
       tableName = model.getTableName
-
-      rowConverter = new RowConverterImpl(conf.getDataFields, conf, null)
-      rowConverter.initialize()
-      conf.setCardinalityFinder(rowConverter)
-
       inputProcessor = new NewInputProcessorStepImpl(conf, rows)
       sortProcessor = new SortProcessorStepImpl(conf, inputProcessor)
       dataWriter = new DataWriterProcessorStepImpl(conf, sortProcessor)
@@ -448,12 +429,9 @@ object DataLoadProcessorStepOnSpark {
         LOGGER.error("Failed for table: " + tableName + " in Data Writer Step", e)
         throw new CarbonDataLoadingException("There is an unexpected error: " + e.getMessage, e)
     } finally {
-      if (rowConverter != null) {
-        rowConverter.finish()
-      }
       // close the dataWriter once the write in done success or fail. if not closed then thread to
       // to prints the rows processed in each step for every 10 seconds will never exit.
-      if(null != dataWriter) {
+      if (null != dataWriter) {
         dataWriter.close()
       }
       // clean up the folders and files created locally for data load operation
diff --git a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/CarbonIUDMergerRDD.scala b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/CarbonIUDMergerRDD.scala
index 9b4e3f6..a0baad0 100644
--- a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/CarbonIUDMergerRDD.scala
+++ b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/CarbonIUDMergerRDD.scala
@@ -99,7 +99,6 @@ class CarbonIUDMergerRDD[K, V](
     }.filter( _ != null)
 
     // max segment cardinality is calculated in executor for each segment
-    carbonMergerMapping.maxSegmentColCardinality = null
     carbonMergerMapping.maxSegmentColumnSchemaList = null
 
     // Log the distribution
diff --git a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/CarbonMergerRDD.scala b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/CarbonMergerRDD.scala
index 76a6f0d..ccff4c1 100644
--- a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/CarbonMergerRDD.scala
+++ b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/CarbonMergerRDD.scala
@@ -146,8 +146,6 @@ class CarbonMergerRDD[K, V](
           }
           // target load name will be same as source load name in case of update data compaction
           carbonMergerMapping.mergedLoadName = tableBlockInfoList.get(0).getSegmentId
-          carbonMergerMapping.maxSegmentColCardinality = dataFileFooter.getSegmentInfo
-            .getColumnCardinality
           carbonMergerMapping.maxSegmentColumnSchemaList = dataFileFooter.getColumnInTable.asScala
             .toList
         }
@@ -173,8 +171,7 @@ class CarbonMergerRDD[K, V](
 
         // get destination segment properties as sent from driver which is of last segment.
         val segmentProperties = new SegmentProperties(
-          carbonMergerMapping.maxSegmentColumnSchemaList.asJava,
-          carbonMergerMapping.maxSegmentColCardinality)
+          carbonMergerMapping.maxSegmentColumnSchemaList.asJava)
 
         val segmentMapping: java.util.Map[String, TaskBlockInfo] =
           CarbonCompactionUtil.createMappingForSegments(tableBlockInfoList)
@@ -456,7 +453,6 @@ class CarbonMergerRDD[K, V](
       }
     }
 
-    val columnToCardinalityMap = new util.HashMap[java.lang.String, Integer]()
     val partitionTaskMap = new util.HashMap[PartitionSpec, String]()
     val counter = new AtomicInteger()
     var indexOfRangeColumn = -1
@@ -481,12 +477,6 @@ class CarbonMergerRDD[K, V](
             logError("Exception in preparing the data file footer for compaction " + e.getMessage)
             throw e
         }
-        // add all the column and cardinality to the map
-        CarbonCompactionUtil
-          .addColumnCardinalityToMap(columnToCardinalityMap,
-            dataFileFooter.getColumnInTable,
-            dataFileFooter.getSegmentInfo.getColumnCardinality)
-
         var splitList = taskIdMapping.get(taskCount.toString)
         if (null != splitList && splitList.size == noOfSplitsPerTask) {
           taskCount = taskCount + 1
@@ -503,7 +493,6 @@ class CarbonMergerRDD[K, V](
         var dataFileFooter: DataFileFooter = null
         if (null == rangeColumn) {
           val taskNo = getTaskNo(split, partitionTaskMap, counter)
-          var sizeOfSplit = split.getLength
           val splitList = taskIdMapping.get(taskNo)
           noOfBlocks += 1
           if (null == splitList) {
@@ -523,11 +512,6 @@ class CarbonMergerRDD[K, V](
             logError("Exception in preparing the data file footer for compaction " + e.getMessage)
             throw e
         }
-        // add all the column and cardinality to the map
-        CarbonCompactionUtil
-          .addColumnCardinalityToMap(columnToCardinalityMap,
-            dataFileFooter.getColumnInTable,
-            dataFileFooter.getSegmentInfo.getColumnCardinality)
 
         // Create taskIdMapping here for range column by reading min/max values.
         if (null != rangeColumn) {
@@ -578,15 +562,11 @@ class CarbonMergerRDD[K, V](
       }
     }
     val updatedMaxSegmentColumnList = new util.ArrayList[ColumnSchema]()
-    // update cardinality and column schema list according to master schema
-    val cardinality = CarbonCompactionUtil
-      .updateColumnSchemaAndGetCardinality(columnToCardinalityMap,
-        carbonTable,
-        updatedMaxSegmentColumnList)
+    // update the column schema list according to master schema
+    CarbonCompactionUtil.updateColumnSchema(
+      carbonTable,
+      updatedMaxSegmentColumnList)
     carbonMergerMapping.maxSegmentColumnSchemaList = updatedMaxSegmentColumnList.asScala.toList
-    // Set cardinality for new segment.
-    carbonMergerMapping.maxSegmentColCardinality = cardinality
-
     taskIdMapping.asScala.foreach(
       entry =>
         taskInfoList
diff --git a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/StreamHandoffRDD.scala b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/StreamHandoffRDD.scala
index 72b7c3e..2197029 100644
--- a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/StreamHandoffRDD.scala
+++ b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/StreamHandoffRDD.scala
@@ -176,12 +176,7 @@ class StreamHandoffRDD[K, V](
   ): CompactionResultSortProcessor = {
     val wrapperColumnSchemaList = CarbonUtil.getColumnSchemaList(
       carbonTable.getVisibleDimensions, carbonTable.getVisibleMeasures)
-    val dimLensWithComplex =
-      (0 until wrapperColumnSchemaList.size()).map(_ => Integer.MAX_VALUE).toArray
-    val dictionaryColumnCardinality =
-      CarbonUtil.getFormattedCardinality(dimLensWithComplex, wrapperColumnSchemaList)
-    val segmentProperties =
-      new SegmentProperties(wrapperColumnSchemaList, dictionaryColumnCardinality)
+    val segmentProperties = new SegmentProperties(wrapperColumnSchemaList)
 
     new CompactionResultSortProcessor(
       carbonLoadModel,
diff --git a/integration/spark-common/src/main/scala/org/apache/spark/rdd/CarbonMergeFilesRDD.scala b/integration/spark-common/src/main/scala/org/apache/spark/rdd/CarbonMergeFilesRDD.scala
index b907788..3561c16 100644
--- a/integration/spark-common/src/main/scala/org/apache/spark/rdd/CarbonMergeFilesRDD.scala
+++ b/integration/spark-common/src/main/scala/org/apache/spark/rdd/CarbonMergeFilesRDD.scala
@@ -17,7 +17,6 @@
 
 package org.apache.spark.rdd
 
-import java.util
 import java.util.concurrent.Executors
 
 import scala.collection.JavaConverters._
diff --git a/integration/spark-common/src/main/scala/org/apache/spark/sql/catalyst/CarbonDDLSqlParser.scala b/integration/spark-common/src/main/scala/org/apache/spark/sql/catalyst/CarbonDDLSqlParser.scala
index 66f3ed3..1d9428f 100644
--- a/integration/spark-common/src/main/scala/org/apache/spark/sql/catalyst/CarbonDDLSqlParser.scala
+++ b/integration/spark-common/src/main/scala/org/apache/spark/sql/catalyst/CarbonDDLSqlParser.scala
@@ -39,7 +39,6 @@ abstract class CarbonDDLSqlParser extends AbstractCarbonSparkSQLParser {
   protected val HIGH_CARDINALITY_DIMS = carbonKeyWord("NO_DICTIONARY")
   protected val BEFORE = carbonKeyWord("BEFORE")
   protected val BY = carbonKeyWord("BY")
-  protected val CARDINALITY = carbonKeyWord("CARDINALITY")
   protected val CASCADE = carbonKeyWord("CASCADE")
   protected val CLASS = carbonKeyWord("CLASS")
   protected val CLEAN = carbonKeyWord("CLEAN")
diff --git a/integration/spark-common/src/main/scala/org/apache/spark/sql/execution/command/carbonTableSchemaCommon.scala b/integration/spark-common/src/main/scala/org/apache/spark/sql/execution/command/carbonTableSchemaCommon.scala
index 83a5000..4e5454d 100644
--- a/integration/spark-common/src/main/scala/org/apache/spark/sql/execution/command/carbonTableSchemaCommon.scala
+++ b/integration/spark-common/src/main/scala/org/apache/spark/sql/execution/command/carbonTableSchemaCommon.scala
@@ -111,8 +111,6 @@ case class CarbonMergerMapping(
     validSegments: Array[Segment],
     tableId: String,
     campactionType: CompactionType,
-    // maxSegmentColCardinality is Cardinality of last segment of compaction
-    var maxSegmentColCardinality: Array[Int],
     // maxSegmentColumnSchemaList is list of column schema of last segment of compaction
     var maxSegmentColumnSchemaList: List[ColumnSchema],
     @transient currentPartitions: Option[Seq[PartitionSpec]])
diff --git a/integration/spark-datasource/src/main/scala/org/apache/carbondata/spark/vectorreader/VectorizedCarbonRecordReader.java b/integration/spark-datasource/src/main/scala/org/apache/carbondata/spark/vectorreader/VectorizedCarbonRecordReader.java
index d96d9d5..265b575 100644
--- a/integration/spark-datasource/src/main/scala/org/apache/carbondata/spark/vectorreader/VectorizedCarbonRecordReader.java
+++ b/integration/spark-datasource/src/main/scala/org/apache/carbondata/spark/vectorreader/VectorizedCarbonRecordReader.java
@@ -30,7 +30,6 @@ import org.apache.carbondata.core.keygenerator.directdictionary.DirectDictionary
 import org.apache.carbondata.core.keygenerator.directdictionary.DirectDictionaryKeyGeneratorFactory;
 import org.apache.carbondata.core.metadata.datatype.DataType;
 import org.apache.carbondata.core.metadata.datatype.DataTypes;
-import org.apache.carbondata.core.metadata.encoder.Encoding;
 import org.apache.carbondata.core.scan.executor.QueryExecutor;
 import org.apache.carbondata.core.scan.executor.QueryExecutorFactory;
 import org.apache.carbondata.core.scan.executor.exception.QueryExecutionException;
diff --git a/integration/spark2/src/main/scala/org/apache/carbondata/datamap/IndexDataMapRebuildRDD.scala b/integration/spark2/src/main/scala/org/apache/carbondata/datamap/IndexDataMapRebuildRDD.scala
index 415a667..6f12e60 100644
--- a/integration/spark2/src/main/scala/org/apache/carbondata/datamap/IndexDataMapRebuildRDD.scala
+++ b/integration/spark2/src/main/scala/org/apache/carbondata/datamap/IndexDataMapRebuildRDD.scala
@@ -17,7 +17,7 @@
 
 package org.apache.carbondata.datamap
 
-import java.io.{File, IOException}
+import java.io.IOException
 import java.text.SimpleDateFormat
 import java.util
 
@@ -25,8 +25,6 @@ import scala.collection.JavaConverters._
 import scala.collection.mutable
 import scala.collection.mutable.ListBuffer
 
-import org.apache.commons.lang3.ArrayUtils
-import org.apache.hadoop.conf.Configuration
 import org.apache.hadoop.mapred.JobConf
 import org.apache.hadoop.mapreduce.{Job, TaskAttemptID, TaskType}
 import org.apache.hadoop.mapreduce.task.TaskAttemptContextImpl
@@ -37,7 +35,6 @@ import org.apache.spark.sql.types.Decimal
 
 import org.apache.carbondata.common.logging.LogServiceFactory
 import org.apache.carbondata.converter.SparkDataTypeConverterImpl
-import org.apache.carbondata.core.constants.CarbonCommonConstants
 import org.apache.carbondata.core.datamap.{DataMapStoreManager, Segment}
 import org.apache.carbondata.core.datamap.dev.DataMapBuilder
 import org.apache.carbondata.core.datastore.block.SegmentProperties
@@ -47,12 +44,11 @@ import org.apache.carbondata.core.indexstore.SegmentPropertiesFetcher
 import org.apache.carbondata.core.keygenerator.KeyGenerator
 import org.apache.carbondata.core.keygenerator.mdkey.MultiDimKeyVarLengthGenerator
 import org.apache.carbondata.core.metadata.datatype.{DataType, DataTypes}
-import org.apache.carbondata.core.metadata.encoder.Encoding
 import org.apache.carbondata.core.metadata.schema.table.{CarbonTable, DataMapSchema, TableInfo}
 import org.apache.carbondata.core.metadata.schema.table.column.CarbonColumn
 import org.apache.carbondata.core.scan.wrappers.ByteArrayWrapper
 import org.apache.carbondata.core.statusmanager.SegmentStatusManager
-import org.apache.carbondata.core.util.{CarbonUtil, DataTypeUtil, TaskMetricsMap}
+import org.apache.carbondata.core.util.{ByteUtil, DataTypeUtil, TaskMetricsMap}
 import org.apache.carbondata.core.util.path.CarbonTablePath
 import org.apache.carbondata.datamap.bloom.DataConvertUtil
 import org.apache.carbondata.events.{BuildDataMapPostExecutionEvent, BuildDataMapPreExecutionEvent, OperationContext, OperationListenerBus}
@@ -162,7 +158,6 @@ class OriginalReadSupport(dataTypes: Array[DataType]) extends CarbonReadSupport[
  */
 class RawBytesReadSupport(segmentProperties: SegmentProperties, indexColumns: Array[CarbonColumn])
   extends CarbonReadSupport[Array[Object]] {
-  var dimensionKeyGenerator: KeyGenerator = _
   // for the dictionary dimensions
   var indexCol2IdxInDictArray: Map[String, Int] = Map()
   // for the non dictionary dimensions
@@ -170,54 +165,6 @@ class RawBytesReadSupport(segmentProperties: SegmentProperties, indexColumns: Ar
   // for the measures
   var indexCol2IdxInMeasureArray: Map[String, Int] = Map()
 
-  /**
-   * rebuild process get data from query, if some columns added to table but not in this segment
-   * it will be filled with default value and generate new key for dict dimension.
-   * Here we use same way as `RowIdRestructureBasedRawResultCollector` to prepare
-   * key generator to get surrogate value of dict column result.
-   * So we do not need to make a fake mdk to split when adding row to datamap
-   */
-  def prepareKeyGenForDictIndexColumns(carbonTable: CarbonTable,
-                                       dictIndexColumns: ListBuffer[CarbonColumn]): Unit = {
-
-    val columnCardinality = new util.ArrayList[Integer](dictIndexColumns.length)
-    val columnPartitioner = new util.ArrayList[Integer](dictIndexColumns.length)
-
-    dictIndexColumns.foreach { col =>
-      val dim = carbonTable.getDimensionByName(col.getColName)
-      val currentBlockDimension = segmentProperties.getDimensionFromCurrentBlock(dim)
-      if (null != currentBlockDimension) {
-        columnCardinality.add(segmentProperties.getDimColumnsCardinality.apply(
-          currentBlockDimension.getKeyOrdinal))
-        columnPartitioner.add(segmentProperties.getDimensionPartitions.apply(
-          currentBlockDimension.getKeyOrdinal
-        ))
-      } else {
-        columnPartitioner.add(1)
-        if (col.hasEncoding(Encoding.DIRECT_DICTIONARY)) {
-          columnCardinality.add(Integer.MAX_VALUE)
-        } else {
-          val defaultValue = col.getDefaultValue
-          if (null != col.getDefaultValue) {
-            columnCardinality.add(CarbonCommonConstants.DICTIONARY_DEFAULT_CARDINALITY + 1)
-          } else {
-            columnCardinality.add(CarbonCommonConstants.DICTIONARY_DEFAULT_CARDINALITY)
-          }
-        }
-      }
-    }
-
-    if (!columnCardinality.isEmpty) {
-      val latestColumnCardinality = ArrayUtils.toPrimitive(columnCardinality.toArray(
-        new Array[Integer](columnCardinality.size)))
-      val latestColumnPartitioner = ArrayUtils.toPrimitive(columnPartitioner.toArray(
-        new Array[Integer](columnPartitioner.size)))
-      val dimensionBitLength = CarbonUtil.getDimensionBitLength(
-        latestColumnCardinality, latestColumnPartitioner)
-      this.dimensionKeyGenerator = new MultiDimKeyVarLengthGenerator(dimensionBitLength)
-    }
-  }
-
   override def initialize(carbonColumns: Array[CarbonColumn],
       carbonTable: CarbonTable): Unit = {
 
@@ -241,9 +188,6 @@ class RawBytesReadSupport(segmentProperties: SegmentProperties, indexColumns: Ar
       }
     }
 
-    if (dictIndexColumns.size > 0) {
-      prepareKeyGenForDictIndexColumns(carbonTable, dictIndexColumns)
-    }
   }
 
   /**
@@ -253,10 +197,11 @@ class RawBytesReadSupport(segmentProperties: SegmentProperties, indexColumns: Ar
    */
   override def readRow(data: Array[Object]): Array[Object] = {
 
-    var surrogatKeys = new Array[Long](0)
-    if(null != dimensionKeyGenerator) {
-        surrogatKeys = dimensionKeyGenerator.getKeyArray(
-          data(0).asInstanceOf[ByteArrayWrapper].getDictionaryKey)
+    val surrogatKeys = if (segmentProperties.getNumberOfDictDimensions > 0) {
+      ByteUtil.convertBytesToLongArray(
+        data(0).asInstanceOf[ByteArrayWrapper].getDictionaryKey)
+    } else {
+      new Array[Long](0)
     }
 
     // fill return row from data
diff --git a/integration/spark2/src/main/scala/org/apache/carbondata/spark/rdd/CarbonTableCompactor.scala b/integration/spark2/src/main/scala/org/apache/carbondata/spark/rdd/CarbonTableCompactor.scala
index aeeec3a..4dd472b 100644
--- a/integration/spark2/src/main/scala/org/apache/carbondata/spark/rdd/CarbonTableCompactor.scala
+++ b/integration/spark2/src/main/scala/org/apache/carbondata/spark/rdd/CarbonTableCompactor.scala
@@ -70,9 +70,7 @@ class CarbonTableCompactor(carbonLoadModel: CarbonLoadModel,
         loadsToMerge.size() == 1 &&
         SortScope.NO_SORT != compactionModel.carbonTable.getSortScope) {
       !CarbonCompactionUtil.isSortedByCurrentSortColumns(
-        carbonLoadModel.getCarbonDataLoadSchema.getCarbonTable,
-        loadsToMerge.get(0),
-        FileFactory.getConfiguration)
+        carbonLoadModel.getCarbonDataLoadSchema.getCarbonTable, loadsToMerge.get(0))
     } else {
       false
     }
@@ -172,7 +170,6 @@ class CarbonTableCompactor(carbonLoadModel: CarbonLoadModel,
       validSegments.asScala.toArray,
       carbonTable.getAbsoluteTableIdentifier.getCarbonTableIdentifier.getTableId,
       compactionType,
-      maxSegmentColCardinality = null,
       maxSegmentColumnSchemaList = null,
       currentPartitions = partitions)
     carbonLoadModel.setTablePath(carbonMergerMapping.hdfsStoreLocation)
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/management/CarbonLoadDataCommand.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/management/CarbonLoadDataCommand.scala
index 5125d3b..0064ee6 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/management/CarbonLoadDataCommand.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/management/CarbonLoadDataCommand.scala
@@ -97,8 +97,6 @@ case class CarbonLoadDataCommand(
 
   var currPartitions: util.List[PartitionSpec] = _
 
-  var parentTablePath: String = _
-
   var finalPartition : Map[String, Option[String]] = Map.empty
 
   override def processMetadata(sparkSession: SparkSession): Seq[Row] = {
@@ -217,7 +215,6 @@ case class CarbonLoadDataCommand(
     } else {
       FileUtils.getPaths(factPathFromUser, hadoopConf)
     }
-    carbonLoadModel.setParentTablePath(parentTablePath)
     carbonLoadModel.setFactFilePath(factPath)
     carbonLoadModel.setCarbonTransactionalTable(table.getTableInfo.isTransactionalTable)
     carbonLoadModel.setAggLoadRequest(
diff --git a/integration/spark2/src/test/scala/org/apache/carbondata/datamap/bloom/BloomCoarseGrainDataMapFunctionSuite.scala b/integration/spark2/src/test/scala/org/apache/carbondata/datamap/bloom/BloomCoarseGrainDataMapFunctionSuite.scala
index 5d02082..726ed42 100644
--- a/integration/spark2/src/test/scala/org/apache/carbondata/datamap/bloom/BloomCoarseGrainDataMapFunctionSuite.scala
+++ b/integration/spark2/src/test/scala/org/apache/carbondata/datamap/bloom/BloomCoarseGrainDataMapFunctionSuite.scala
@@ -725,8 +725,8 @@ class BloomCoarseGrainDataMapFunctionSuite  extends QueryTest with BeforeAndAfte
          | CREATE DATAMAP $dataMapName ON TABLE $bloomDMSampleTable USING 'bloomfilter' DMPROPERTIES (
          | 'INDEX_COLUMNS'='doj')
        """.stripMargin)
-    sql(s"SELECT * FROM $bloomDMSampleTable WHERE doj='2016-03-14'").show(false)
-    sql(s"SELECT * FROM $bloomDMSampleTable WHERE doj='2016-03-15'").show(false)
+//    sql(s"SELECT * FROM $bloomDMSampleTable WHERE doj='2016-03-14'").show(false)
+//    sql(s"SELECT * FROM $bloomDMSampleTable WHERE doj='2016-03-15'").show(false)
     checkAnswer(sql(s"SELECT * FROM $bloomDMSampleTable WHERE doj='2016-03-14'"),
       sql(s"SELECT * FROM $normalTable WHERE doj='2016-03-14'"))
     checkAnswer(sql(s"SELECT * FROM $bloomDMSampleTable WHERE doj='2016-03-15'"),
diff --git a/integration/spark2/src/test/scala/org/apache/carbondata/spark/testsuite/booleantype/BooleanDataTypesLoadTest.scala b/integration/spark2/src/test/scala/org/apache/carbondata/spark/testsuite/booleantype/BooleanDataTypesLoadTest.scala
index 32071ce..5f8f52c 100644
--- a/integration/spark2/src/test/scala/org/apache/carbondata/spark/testsuite/booleantype/BooleanDataTypesLoadTest.scala
+++ b/integration/spark2/src/test/scala/org/apache/carbondata/spark/testsuite/booleantype/BooleanDataTypesLoadTest.scala
@@ -153,25 +153,16 @@ class BooleanDataTypesLoadTest extends QueryTest with BeforeAndAfterEach with Be
   }
 
   test("Loading table: support boolean and other data type, data columns bigger than table defined columns") {
+    CarbonProperties.getInstance().addProperty(CarbonCommonConstants.CARBON_DATE_FORMAT, "yyyy/mm/dd")
+    CarbonProperties.getInstance().addProperty(CarbonCommonConstants.CARBON_TIMESTAMP_FORMAT, "yyyy/mm/dd")
     sql("drop table if exists boolean_table")
     sql(
       s"""
          | CREATE TABLE boolean_table(
-         | shortField SHORT,
-         | booleanField BOOLEAN,
-         | intField INT,
-         | bigintField LONG,
-         | doubleField DOUBLE,
-         | stringField STRING,
          | timestampField TIMESTAMP,
-         | decimalField DECIMAL(18,2),
-         | dateField DATE,
-         | charField CHAR(5),
-         | floatField FLOAT,
-         | complexData ARRAY<STRING>
+         | dateField DATE
          | )
          | STORED AS carbondata
-         | TBLPROPERTIES('sort_columns'='')
        """.stripMargin)
 
     val storeLocation = s"$rootPath/integration/spark2/src/test/resources/bool/supportBooleanTwoBooleanColumns.csv"
@@ -182,12 +173,13 @@ class BooleanDataTypesLoadTest extends QueryTest with BeforeAndAfterEach with Be
          | options('FILEHEADER'='shortField,booleanField,intField,bigintField,doubleField,stringField,timestampField,decimalField,dateField,charField,floatField,complexData,booleanField2')
            """.stripMargin)
 
-    checkAnswer(
-      sql("select booleanField,intField from boolean_table"),
-      Seq(Row(true, 10), Row(false, 17), Row(false, 11),
-        Row(true, 10), Row(true, 10), Row(true, 14),
-        Row(false, 10), Row(false, 10), Row(false, 16), Row(false, 10))
-    )
+//    checkAnswer(
+//      sql("select booleanField,intField from boolean_table"),
+//      Seq(Row(true, 10), Row(false, 17), Row(false, 11),
+//        Row(true, 10), Row(true, 10), Row(true, 14),
+//        Row(false, 10), Row(false, 10), Row(false, 16), Row(false, 10))
+//    )
+    sql("select * from boolean_table where dateField < '2015-01-24'").show
   }
 
   test("Loading table: support boolean and other data type, with file header") {
diff --git a/processing/src/main/java/org/apache/carbondata/processing/datamap/DataMapWriterListener.java b/processing/src/main/java/org/apache/carbondata/processing/datamap/DataMapWriterListener.java
index bdcd624..77917df 100644
--- a/processing/src/main/java/org/apache/carbondata/processing/datamap/DataMapWriterListener.java
+++ b/processing/src/main/java/org/apache/carbondata/processing/datamap/DataMapWriterListener.java
@@ -63,8 +63,7 @@ public class DataMapWriterListener {
   public void registerAllWriter(CarbonTable carbonTable, String segmentId,
       String taskNo, SegmentProperties segmentProperties) {
     // clear cache in executor side
-    DataMapStoreManager.getInstance()
-        .clearDataMaps(carbonTable.getTableId());
+    DataMapStoreManager.getInstance().clearDataMaps(carbonTable.getTableId());
     List<TableDataMap> tableIndices;
     try {
       tableIndices = DataMapStoreManager.getInstance().getAllDataMap(carbonTable);
@@ -72,15 +71,13 @@ public class DataMapWriterListener {
       LOG.error("Error while retrieving datamaps", e);
       throw new RuntimeException(e);
     }
-    if (tableIndices != null) {
-      tblIdentifier = carbonTable.getCarbonTableIdentifier();
-      for (TableDataMap tableDataMap : tableIndices) {
-        // register it only if it is not lazy datamap, for lazy datamap, user
-        // will rebuild the datamap manually
-        if (!tableDataMap.getDataMapSchema().isLazy()) {
-          DataMapFactory factory = tableDataMap.getDataMapFactory();
-          register(factory, segmentId, taskNo, segmentProperties);
-        }
+    tblIdentifier = carbonTable.getCarbonTableIdentifier();
+    for (TableDataMap tableDataMap : tableIndices) {
+      // register it only if it is not lazy datamap, for lazy datamap, user
+      // will rebuild the datamap manually
+      if (!tableDataMap.getDataMapSchema().isLazy()) {
+        DataMapFactory factory = tableDataMap.getDataMapFactory();
+        register(factory, segmentId, taskNo, segmentProperties);
       }
     }
   }
diff --git a/processing/src/main/java/org/apache/carbondata/processing/datatypes/ArrayDataType.java b/processing/src/main/java/org/apache/carbondata/processing/datatypes/ArrayDataType.java
index 1b49e8b..ad17a55 100644
--- a/processing/src/main/java/org/apache/carbondata/processing/datatypes/ArrayDataType.java
+++ b/processing/src/main/java/org/apache/carbondata/processing/datatypes/ArrayDataType.java
@@ -26,7 +26,7 @@ import java.util.List;
 import org.apache.carbondata.core.datastore.ColumnType;
 import org.apache.carbondata.core.datastore.row.ComplexColumnInfo;
 import org.apache.carbondata.core.keygenerator.KeyGenException;
-import org.apache.carbondata.core.keygenerator.KeyGenerator;
+import org.apache.carbondata.core.util.ByteUtil;
 import org.apache.carbondata.core.util.DataTypeUtil;
 import org.apache.carbondata.processing.loading.complexobjects.ArrayObject;
 import org.apache.carbondata.processing.loading.converter.BadRecordLogHolder;
@@ -158,14 +158,6 @@ public class ArrayDataType implements GenericDataType<ArrayObject> {
   }
 
   /*
-   * return surrogate index
-   */
-  @Override
-  public int getSurrogateIndex() {
-    return 0;
-  }
-
-  /*
    * set surrogate index
    */
   @Override
@@ -194,27 +186,18 @@ public class ArrayDataType implements GenericDataType<ArrayObject> {
   }
 
   @Override
-  public void fillCardinality(List<Integer> dimCardWithComplex) {
-    if (this.getIsColumnDictionary()) {
-      dimCardWithComplex.add(0);
-      children.fillCardinality(dimCardWithComplex);
-    }
-  }
-
-  @Override
-  public void parseComplexValue(ByteBuffer byteArrayInput, DataOutputStream dataOutputStream,
-      KeyGenerator[] generator)
+  public void parseComplexValue(ByteBuffer byteArrayInput, DataOutputStream dataOutputStream)
       throws IOException, KeyGenException {
     int dataLength = byteArrayInput.getInt();
 
     dataOutputStream.writeInt(dataLength);
     if (children instanceof PrimitiveDataType) {
       if (children.getIsColumnDictionary()) {
-        dataOutputStream.writeInt(generator[children.getSurrogateIndex()].getKeySizeInBytes());
+        dataOutputStream.writeInt(ByteUtil.dateBytesSize());
       }
     }
     for (int i = 0; i < dataLength; i++) {
-      children.parseComplexValue(byteArrayInput, dataOutputStream, generator);
+      children.parseComplexValue(byteArrayInput, dataOutputStream);
     }
   }
 
@@ -284,34 +267,6 @@ public class ArrayDataType implements GenericDataType<ArrayObject> {
     return this.dataCounter;
   }
 
-  /*
-   * fill agg key blocks
-   */
-  @Override
-  public void fillAggKeyBlock(List<Boolean> aggKeyBlockWithComplex, boolean[] aggKeyBlock) {
-    aggKeyBlockWithComplex.add(false);
-    children.fillAggKeyBlock(aggKeyBlockWithComplex, aggKeyBlock);
-  }
-
-  /*
-   * fill key size
-   */
-  @Override
-  public void fillBlockKeySize(List<Integer> blockKeySizeWithComplex, int[] primitiveBlockKeySize) {
-    blockKeySizeWithComplex.add(8);
-    children.fillBlockKeySize(blockKeySizeWithComplex, primitiveBlockKeySize);
-  }
-
-  /*
-   * fill cardinality
-   */
-  @Override
-  public void fillCardinalityAfterDataLoad(List<Integer> dimCardWithComplex,
-      int[] maxSurrogateKeyArray) {
-    dimCardWithComplex.add(0);
-    children.fillCardinalityAfterDataLoad(dimCardWithComplex, maxSurrogateKeyArray);
-  }
-
   @Override
   public GenericDataType<ArrayObject> deepCopy() {
     return new ArrayDataType(this.outputArrayIndex, this.dataCounter, this.children.deepCopy(),
diff --git a/processing/src/main/java/org/apache/carbondata/processing/datatypes/GenericDataType.java b/processing/src/main/java/org/apache/carbondata/processing/datatypes/GenericDataType.java
index 3bec574..1743abe 100644
--- a/processing/src/main/java/org/apache/carbondata/processing/datatypes/GenericDataType.java
+++ b/processing/src/main/java/org/apache/carbondata/processing/datatypes/GenericDataType.java
@@ -25,7 +25,6 @@ import java.util.List;
 
 import org.apache.carbondata.core.datastore.row.ComplexColumnInfo;
 import org.apache.carbondata.core.keygenerator.KeyGenException;
-import org.apache.carbondata.core.keygenerator.KeyGenerator;
 import org.apache.carbondata.processing.loading.converter.BadRecordLogHolder;
 
 /**
@@ -63,11 +62,6 @@ public interface GenericDataType<T> {
       throws IOException;
 
   /**
-   * @return surrogateIndex for primitive column in complex type
-   */
-  int getSurrogateIndex();
-
-  /**
    * @param surrIndex - surrogate index of primitive column in complex type
    */
   void setSurrogateIndex(int surrIndex);
@@ -82,13 +76,11 @@ public interface GenericDataType<T> {
    * Parse the Complex Datatype from the ByteBuffer.
    * @param byteArrayInput
    * @param dataOutputStream
-   * @param generator
    * @return
    * @throws IOException
    * @throws KeyGenException
    */
-  void parseComplexValue(ByteBuffer byteArrayInput, DataOutputStream dataOutputStream,
-      KeyGenerator[] generator)
+  void parseComplexValue(ByteBuffer byteArrayInput, DataOutputStream dataOutputStream)
       throws IOException, KeyGenException;
 
   /**
@@ -125,33 +117,6 @@ public interface GenericDataType<T> {
   int getDataCounter();
 
   /**
-   * fill agg key block including complex types
-   * @param aggKeyBlockWithComplex
-   * @param aggKeyBlock
-   */
-  void fillAggKeyBlock(List<Boolean> aggKeyBlockWithComplex, boolean[] aggKeyBlock);
-
-  /**
-   * fill block key size including complex types
-   * @param blockKeySizeWithComplex
-   * @param primitiveBlockKeySize
-   */
-  void fillBlockKeySize(List<Integer> blockKeySizeWithComplex, int[] primitiveBlockKeySize);
-
-  /**
-   * fill cardinality value including complex types
-   * @param dimCardWithComplex
-   * @param maxSurrogateKeyArray
-   */
-  void fillCardinalityAfterDataLoad(List<Integer> dimCardWithComplex, int[] maxSurrogateKeyArray);
-
-  /**
-   * Fill the cardinality of the primitive datatypes
-   * @param dimCardWithComplex
-   */
-  void fillCardinality(List<Integer> dimCardWithComplex);
-
-  /**
    * clone self for multithread access (for complex type processing in table page)
    */
   GenericDataType<T> deepCopy();
diff --git a/processing/src/main/java/org/apache/carbondata/processing/datatypes/PrimitiveDataType.java b/processing/src/main/java/org/apache/carbondata/processing/datatypes/PrimitiveDataType.java
index ade35671..22adedc 100644
--- a/processing/src/main/java/org/apache/carbondata/processing/datatypes/PrimitiveDataType.java
+++ b/processing/src/main/java/org/apache/carbondata/processing/datatypes/PrimitiveDataType.java
@@ -31,7 +31,6 @@ import org.apache.carbondata.core.constants.CarbonCommonConstants;
 import org.apache.carbondata.core.datastore.ColumnType;
 import org.apache.carbondata.core.datastore.row.ComplexColumnInfo;
 import org.apache.carbondata.core.devapi.BiDictionary;
-import org.apache.carbondata.core.keygenerator.KeyGenerator;
 import org.apache.carbondata.core.keygenerator.directdictionary.DirectDictionaryKeyGeneratorFactory;
 import org.apache.carbondata.core.metadata.datatype.DataType;
 import org.apache.carbondata.core.metadata.datatype.DataTypes;
@@ -225,14 +224,6 @@ public class PrimitiveDataType implements GenericDataType<Object> {
   }
 
   /*
-   * get surrogate index
-   */
-  @Override
-  public int getSurrogateIndex() {
-    return index;
-  }
-
-  /*
    * set surrogate index
    */
   @Override
@@ -428,16 +419,7 @@ public class PrimitiveDataType implements GenericDataType<Object> {
   }
 
   @Override
-  public void fillCardinality(List<Integer> dimCardWithComplex) {
-    if (!this.carbonDimension.hasEncoding(Encoding.DICTIONARY)) {
-      return;
-    }
-    dimCardWithComplex.add(dictionaryGenerator.size());
-  }
-
-  @Override
-  public void parseComplexValue(ByteBuffer byteArrayInput, DataOutputStream dataOutputStream,
-      KeyGenerator[] generator)
+  public void parseComplexValue(ByteBuffer byteArrayInput, DataOutputStream dataOutputStream)
       throws IOException {
     if (!this.isDictionary) {
       int sizeOfData;
@@ -453,7 +435,7 @@ public class PrimitiveDataType implements GenericDataType<Object> {
       dataOutputStream.write(bb);
     } else {
       int data = byteArrayInput.getInt();
-      byte[] v = generator[index].generateKey(new int[] { data });
+      byte[] v = ByteUtil.convertIntToBytes(data);
       dataOutputStream.write(v);
     }
   }
@@ -522,32 +504,6 @@ public class PrimitiveDataType implements GenericDataType<Object> {
     this.keySize = keySize;
   }
 
-  /*
-   * fill agg key block
-   */
-  @Override
-  public void fillAggKeyBlock(List<Boolean> aggKeyBlockWithComplex, boolean[] aggKeyBlock) {
-    aggKeyBlockWithComplex.add(aggKeyBlock[index]);
-  }
-
-  /*
-   * fill block key size
-   */
-  @Override
-  public void fillBlockKeySize(List<Integer> blockKeySizeWithComplex, int[] primitiveBlockKeySize) {
-    blockKeySizeWithComplex.add(primitiveBlockKeySize[index]);
-    this.keySize = primitiveBlockKeySize[index];
-  }
-
-  /*
-   * fill cardinality
-   */
-  @Override
-  public void fillCardinalityAfterDataLoad(List<Integer> dimCardWithComplex,
-      int[] maxSurrogateKeyArray) {
-    dimCardWithComplex.add(maxSurrogateKeyArray[index]);
-  }
-
   @Override
   public GenericDataType<Object> deepCopy() {
     PrimitiveDataType dataType = new PrimitiveDataType(this.outputArrayIndex, 0);
diff --git a/processing/src/main/java/org/apache/carbondata/processing/datatypes/StructDataType.java b/processing/src/main/java/org/apache/carbondata/processing/datatypes/StructDataType.java
index 588afbf..701bbd3 100644
--- a/processing/src/main/java/org/apache/carbondata/processing/datatypes/StructDataType.java
+++ b/processing/src/main/java/org/apache/carbondata/processing/datatypes/StructDataType.java
@@ -26,7 +26,7 @@ import java.util.List;
 import org.apache.carbondata.core.datastore.ColumnType;
 import org.apache.carbondata.core.datastore.row.ComplexColumnInfo;
 import org.apache.carbondata.core.keygenerator.KeyGenException;
-import org.apache.carbondata.core.keygenerator.KeyGenerator;
+import org.apache.carbondata.core.util.ByteUtil;
 import org.apache.carbondata.core.util.DataTypeUtil;
 import org.apache.carbondata.processing.loading.complexobjects.StructObject;
 import org.apache.carbondata.processing.loading.converter.BadRecordLogHolder;
@@ -160,14 +160,6 @@ public class StructDataType implements GenericDataType<StructObject> {
   }
 
   /*
-   * get surrogate index
-   */
-  @Override
-  public int getSurrogateIndex() {
-    return 0;
-  }
-
-  /*
    * set surrogate index
    */
   @Override
@@ -201,39 +193,26 @@ public class StructDataType implements GenericDataType<StructObject> {
     }
   }
 
-  @Override
-  public void fillCardinality(List<Integer> dimCardWithComplex) {
-    if (this.getIsColumnDictionary()) {
-      dimCardWithComplex.add(0);
-      for (int i = 0; i < children.size(); i++) {
-        children.get(i).fillCardinality(dimCardWithComplex);
-      }
-    }
-  }
-
   /**
    *
    * @param byteArrayInput
    * @param dataOutputStream
-   * @param generator
    * @return
    * @throws IOException
    * @throws KeyGenException
    */
   @Override
-  public void parseComplexValue(ByteBuffer byteArrayInput,
-      DataOutputStream dataOutputStream, KeyGenerator[] generator)
+  public void parseComplexValue(ByteBuffer byteArrayInput, DataOutputStream dataOutputStream)
       throws IOException, KeyGenException {
     short childElement = byteArrayInput.getShort();
     dataOutputStream.writeShort(childElement);
     for (int i = 0; i < childElement; i++) {
       if (children.get(i) instanceof PrimitiveDataType) {
         if (children.get(i).getIsColumnDictionary()) {
-          dataOutputStream
-              .writeInt(generator[children.get(i).getSurrogateIndex()].getKeySizeInBytes());
+          dataOutputStream.writeInt(ByteUtil.dateBytesSize());
         }
       }
-      children.get(i).parseComplexValue(byteArrayInput, dataOutputStream, generator);
+      children.get(i).parseComplexValue(byteArrayInput, dataOutputStream);
     }
   }
 
@@ -310,40 +289,6 @@ public class StructDataType implements GenericDataType<StructObject> {
     return this.dataCounter;
   }
 
-  /*
-   * fill agg block
-   */
-  @Override
-  public void fillAggKeyBlock(List<Boolean> aggKeyBlockWithComplex, boolean[] aggKeyBlock) {
-    aggKeyBlockWithComplex.add(false);
-    for (int i = 0; i < children.size(); i++) {
-      children.get(i).fillAggKeyBlock(aggKeyBlockWithComplex, aggKeyBlock);
-    }
-  }
-
-  /*
-   * fill keysize
-   */
-  @Override
-  public void fillBlockKeySize(List<Integer> blockKeySizeWithComplex, int[] primitiveBlockKeySize) {
-    blockKeySizeWithComplex.add(2);
-    for (int i = 0; i < children.size(); i++) {
-      children.get(i).fillBlockKeySize(blockKeySizeWithComplex, primitiveBlockKeySize);
-    }
-  }
-
-  /*
-   * fill cardinality
-   */
-  @Override
-  public void fillCardinalityAfterDataLoad(List<Integer> dimCardWithComplex,
-      int[] maxSurrogateKeyArray) {
-    dimCardWithComplex.add(0);
-    for (int i = 0; i < children.size(); i++) {
-      children.get(i).fillCardinalityAfterDataLoad(dimCardWithComplex, maxSurrogateKeyArray);
-    }
-  }
-
   @Override
   public GenericDataType<StructObject> deepCopy() {
     List<GenericDataType> childrenClone = new ArrayList<>();
diff --git a/processing/src/main/java/org/apache/carbondata/processing/loading/CarbonDataLoadConfiguration.java b/processing/src/main/java/org/apache/carbondata/processing/loading/CarbonDataLoadConfiguration.java
index 0627229..7575754 100644
--- a/processing/src/main/java/org/apache/carbondata/processing/loading/CarbonDataLoadConfiguration.java
+++ b/processing/src/main/java/org/apache/carbondata/processing/loading/CarbonDataLoadConfiguration.java
@@ -30,7 +30,6 @@ import org.apache.carbondata.core.metadata.schema.BucketingInfo;
 import org.apache.carbondata.core.metadata.schema.SortColumnRangeInfo;
 import org.apache.carbondata.core.metadata.schema.table.column.CarbonColumn;
 import org.apache.carbondata.core.util.OutputFilesInfoHolder;
-import org.apache.carbondata.processing.loading.converter.DictionaryCardinalityFinder;
 
 public class CarbonDataLoadConfiguration {
 
@@ -67,8 +66,6 @@ public class CarbonDataLoadConfiguration {
    */
   private long schemaUpdatedTimeStamp;
 
-  private DictionaryCardinalityFinder cardinalityFinder;
-
   private int numberOfSortColumns;
 
   private int numberOfNoDictSortColumns;
@@ -90,8 +87,6 @@ public class CarbonDataLoadConfiguration {
    */
   private String dataWritePath;
 
-  private String parentTablePath;
-
   /**
    * name of compressor to be used to compress column page
    */
@@ -111,14 +106,6 @@ public class CarbonDataLoadConfiguration {
   public CarbonDataLoadConfiguration() {
   }
 
-  public String getParentTablePath() {
-    return parentTablePath;
-  }
-
-  public void setParentTablePath(String parentTablePath) {
-    this.parentTablePath = parentTablePath;
-  }
-
   public void setDataFields(DataField[] dataFields) {
     this.dataFields = dataFields;
 
@@ -248,14 +235,6 @@ public class CarbonDataLoadConfiguration {
     this.schemaUpdatedTimeStamp = schemaUpdatedTimeStamp;
   }
 
-  public DictionaryCardinalityFinder getCardinalityFinder() {
-    return cardinalityFinder;
-  }
-
-  public void setCardinalityFinder(DictionaryCardinalityFinder cardinalityFinder) {
-    this.cardinalityFinder = cardinalityFinder;
-  }
-
   public DataType[] getMeasureDataType() {
     List<Integer> measureIndexes = new ArrayList<>(dataFields.length);
     int measureCount = 0;
@@ -312,10 +291,6 @@ public class CarbonDataLoadConfiguration {
     return sortColumnMapping;
   }
 
-  public int[] getCardinalityForComplexDimension() {
-    return getCardinalityFinder().getCardinality();
-  }
-
   public TableSpec getTableSpec() {
     return tableSpec;
   }
diff --git a/processing/src/main/java/org/apache/carbondata/processing/loading/DataLoadProcessBuilder.java b/processing/src/main/java/org/apache/carbondata/processing/loading/DataLoadProcessBuilder.java
index 6c3ca6e..657ae98 100644
--- a/processing/src/main/java/org/apache/carbondata/processing/loading/DataLoadProcessBuilder.java
+++ b/processing/src/main/java/org/apache/carbondata/processing/loading/DataLoadProcessBuilder.java
@@ -184,7 +184,6 @@ public final class DataLoadProcessBuilder {
     CarbonDataLoadConfiguration configuration = new CarbonDataLoadConfiguration();
     CarbonTable carbonTable = loadModel.getCarbonDataLoadSchema().getCarbonTable();
     AbsoluteTableIdentifier identifier = carbonTable.getAbsoluteTableIdentifier();
-    configuration.setParentTablePath(loadModel.getParentTablePath());
     configuration.setTableIdentifier(identifier);
     configuration.setCarbonTransactionalTable(loadModel.isCarbonTransactionalTable());
     configuration.setSchemaUpdatedTimeStamp(carbonTable.getTableLastUpdatedTime());
@@ -221,7 +220,6 @@ public final class DataLoadProcessBuilder {
         loadModel.getSkipEmptyLine());
     configuration.setDataLoadProperty(DataLoadProcessorConstants.FACT_FILE_PATH,
         loadModel.getFactFilePath());
-    configuration.setParentTablePath(loadModel.getParentTablePath());
     configuration
         .setDataLoadProperty(CarbonCommonConstants.LOAD_SORT_SCOPE, loadModel.getSortScope());
     configuration.setDataLoadProperty(CarbonCommonConstants.LOAD_GLOBAL_SORT_PARTITIONS,
diff --git a/processing/src/main/java/org/apache/carbondata/processing/loading/converter/DictionaryCardinalityFinder.java b/processing/src/main/java/org/apache/carbondata/processing/loading/converter/DictionaryCardinalityFinder.java
deleted file mode 100644
index aa84fc3..0000000
--- a/processing/src/main/java/org/apache/carbondata/processing/loading/converter/DictionaryCardinalityFinder.java
+++ /dev/null
@@ -1,26 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.carbondata.processing.loading.converter;
-
-/**
- * Finds the current cardinality of dimensions.
- */
-public interface DictionaryCardinalityFinder {
-
-  int[] getCardinality();
-}
diff --git a/processing/src/main/java/org/apache/carbondata/processing/loading/converter/RowConverter.java b/processing/src/main/java/org/apache/carbondata/processing/loading/converter/RowConverter.java
index 8955c4a..9ee77f2 100644
--- a/processing/src/main/java/org/apache/carbondata/processing/loading/converter/RowConverter.java
+++ b/processing/src/main/java/org/apache/carbondata/processing/loading/converter/RowConverter.java
@@ -23,7 +23,7 @@ import org.apache.carbondata.processing.loading.exception.CarbonDataLoadingExcep
 /**
  * convert the row
  */
-public interface RowConverter extends DictionaryCardinalityFinder {
+public interface RowConverter {
 
   void initialize();
 
diff --git a/processing/src/main/java/org/apache/carbondata/processing/loading/converter/impl/AbstractDictionaryFieldConverterImpl.java b/processing/src/main/java/org/apache/carbondata/processing/loading/converter/impl/AbstractDictionaryFieldConverterImpl.java
deleted file mode 100644
index 969d0a0..0000000
--- a/processing/src/main/java/org/apache/carbondata/processing/loading/converter/impl/AbstractDictionaryFieldConverterImpl.java
+++ /dev/null
@@ -1,28 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.carbondata.processing.loading.converter.impl;
-
-import java.util.List;
-
-import org.apache.carbondata.processing.loading.converter.FieldConverter;
-
-public abstract class AbstractDictionaryFieldConverterImpl implements FieldConverter {
-
-  public abstract void fillColumnCardinality(List<Integer> cardinality);
-
-}
diff --git a/processing/src/main/java/org/apache/carbondata/processing/loading/converter/impl/ComplexFieldConverterImpl.java b/processing/src/main/java/org/apache/carbondata/processing/loading/converter/impl/ComplexFieldConverterImpl.java
index fa1e957..5b55f9c 100644
--- a/processing/src/main/java/org/apache/carbondata/processing/loading/converter/impl/ComplexFieldConverterImpl.java
+++ b/processing/src/main/java/org/apache/carbondata/processing/loading/converter/impl/ComplexFieldConverterImpl.java
@@ -19,15 +19,15 @@ package org.apache.carbondata.processing.loading.converter.impl;
 
 import java.io.ByteArrayOutputStream;
 import java.io.DataOutputStream;
-import java.util.List;
 
 import org.apache.carbondata.core.datastore.row.CarbonRow;
 import org.apache.carbondata.processing.datatypes.GenericDataType;
 import org.apache.carbondata.processing.loading.DataField;
 import org.apache.carbondata.processing.loading.converter.BadRecordLogHolder;
+import org.apache.carbondata.processing.loading.converter.FieldConverter;
 import org.apache.carbondata.processing.loading.exception.CarbonDataLoadingException;
 
-public class ComplexFieldConverterImpl extends AbstractDictionaryFieldConverterImpl {
+public class ComplexFieldConverterImpl implements FieldConverter {
 
   private GenericDataType genericDataType;
 
@@ -73,8 +73,4 @@ public class ComplexFieldConverterImpl extends AbstractDictionaryFieldConverterI
   public void clear() {
   }
 
-  @Override
-  public void fillColumnCardinality(List<Integer> cardinality) {
-    genericDataType.fillCardinality(cardinality);
-  }
 }
diff --git a/processing/src/main/java/org/apache/carbondata/processing/loading/converter/impl/DirectDictionaryFieldConverterImpl.java b/processing/src/main/java/org/apache/carbondata/processing/loading/converter/impl/DirectDictionaryFieldConverterImpl.java
index cfdd337..2d9c8c6 100644
--- a/processing/src/main/java/org/apache/carbondata/processing/loading/converter/impl/DirectDictionaryFieldConverterImpl.java
+++ b/processing/src/main/java/org/apache/carbondata/processing/loading/converter/impl/DirectDictionaryFieldConverterImpl.java
@@ -17,8 +17,6 @@
 
 package org.apache.carbondata.processing.loading.converter.impl;
 
-import java.util.List;
-
 import org.apache.carbondata.core.constants.CarbonCommonConstants;
 import org.apache.carbondata.core.datastore.row.CarbonRow;
 import org.apache.carbondata.core.keygenerator.directdictionary.DirectDictionaryGenerator;
@@ -27,9 +25,10 @@ import org.apache.carbondata.core.metadata.datatype.DataTypes;
 import org.apache.carbondata.core.metadata.schema.table.column.CarbonColumn;
 import org.apache.carbondata.processing.loading.DataField;
 import org.apache.carbondata.processing.loading.converter.BadRecordLogHolder;
+import org.apache.carbondata.processing.loading.converter.FieldConverter;
 import org.apache.carbondata.processing.util.CarbonDataProcessorUtil;
 
-public class DirectDictionaryFieldConverterImpl extends AbstractDictionaryFieldConverterImpl {
+public class DirectDictionaryFieldConverterImpl implements FieldConverter {
 
   private DirectDictionaryGenerator directDictionaryGenerator;
 
@@ -110,8 +109,4 @@ public class DirectDictionaryFieldConverterImpl extends AbstractDictionaryFieldC
   public void clear() {
   }
 
-  @Override
-  public void fillColumnCardinality(List<Integer> cardinality) {
-    cardinality.add(Integer.MAX_VALUE);
-  }
 }
diff --git a/processing/src/main/java/org/apache/carbondata/processing/loading/converter/impl/RowConverterImpl.java b/processing/src/main/java/org/apache/carbondata/processing/loading/converter/impl/RowConverterImpl.java
index 50bc4e0..e036adf 100644
--- a/processing/src/main/java/org/apache/carbondata/processing/loading/converter/impl/RowConverterImpl.java
+++ b/processing/src/main/java/org/apache/carbondata/processing/loading/converter/impl/RowConverterImpl.java
@@ -175,24 +175,6 @@ public class RowConverterImpl implements RowConverter {
   }
 
   @Override
-  public int[] getCardinality() {
-    List<Integer> dimCardinality = new ArrayList<>();
-    if (fieldConverters != null) {
-      for (int i = 0; i < fieldConverters.length; i++) {
-        if (fieldConverters[i] instanceof AbstractDictionaryFieldConverterImpl) {
-          ((AbstractDictionaryFieldConverterImpl) fieldConverters[i])
-              .fillColumnCardinality(dimCardinality);
-        }
-      }
-    }
-    int[] cardinality = new int[dimCardinality.size()];
-    for (int i = 0; i < dimCardinality.size(); i++) {
-      cardinality[i] = dimCardinality.get(i);
-    }
-    return cardinality;
-  }
-
-  @Override
   public FieldConverter[] getFieldConverters() {
     return fieldConverters;
   }
diff --git a/processing/src/main/java/org/apache/carbondata/processing/loading/model/CarbonLoadModel.java b/processing/src/main/java/org/apache/carbondata/processing/loading/model/CarbonLoadModel.java
index 759aa83..5645ca0 100644
--- a/processing/src/main/java/org/apache/carbondata/processing/loading/model/CarbonLoadModel.java
+++ b/processing/src/main/java/org/apache/carbondata/processing/loading/model/CarbonLoadModel.java
@@ -45,8 +45,6 @@ public class CarbonLoadModel implements Serializable {
 
   private String tablePath;
 
-  private String parentTablePath;
-
   /*
      This points if the carbonTable is a Non Transactional Table or not.
      The path will be pointed by the tablePath. And there will be
@@ -226,14 +224,6 @@ public class CarbonLoadModel implements Serializable {
     isAggLoadRequest = aggLoadRequest;
   }
 
-  public String getParentTablePath() {
-    return parentTablePath;
-  }
-
-  public void setParentTablePath(String parentTablePath) {
-    this.parentTablePath = parentTablePath;
-  }
-
   /**
    * get escape char
    *
@@ -401,7 +391,6 @@ public class CarbonLoadModel implements Serializable {
     copy.isLoadWithoutConverterStep = isLoadWithoutConverterStep;
     copy.sortColumnsBoundsStr = sortColumnsBoundsStr;
     copy.loadMinSize = loadMinSize;
-    copy.parentTablePath = parentTablePath;
     copy.sdkWriterCores = sdkWriterCores;
     copy.columnCompressor = columnCompressor;
     copy.binaryDecoder = binaryDecoder;
@@ -454,7 +443,6 @@ public class CarbonLoadModel implements Serializable {
     copyObj.isAggLoadRequest = isAggLoadRequest;
     copyObj.sortColumnsBoundsStr = sortColumnsBoundsStr;
     copyObj.loadMinSize = loadMinSize;
-    copyObj.parentTablePath = parentTablePath;
     copyObj.sdkWriterCores = sdkWriterCores;
     copyObj.columnCompressor = columnCompressor;
     copyObj.binaryDecoder = binaryDecoder;
@@ -556,13 +544,6 @@ public class CarbonLoadModel implements Serializable {
     this.factTimeStamp = factTimeStamp;
   }
 
-  public String[] getDelimiters() {
-    checkAndInitializeComplexDelimiterList();
-    String[] delimiters = new String[complexDelimiters.size()];
-    delimiters = complexDelimiters.toArray(delimiters);
-    return delimiters;
-  }
-
   private void checkAndInitializeComplexDelimiterList() {
     if (null == complexDelimiters) {
       complexDelimiters = new ArrayList<>();
diff --git a/processing/src/main/java/org/apache/carbondata/processing/loading/steps/DataConverterProcessorStepImpl.java b/processing/src/main/java/org/apache/carbondata/processing/loading/steps/DataConverterProcessorStepImpl.java
index 1b694c1..28178f0 100644
--- a/processing/src/main/java/org/apache/carbondata/processing/loading/steps/DataConverterProcessorStepImpl.java
+++ b/processing/src/main/java/org/apache/carbondata/processing/loading/steps/DataConverterProcessorStepImpl.java
@@ -73,7 +73,6 @@ public class DataConverterProcessorStepImpl extends AbstractDataLoadProcessorSte
     badRecordLogger = BadRecordsLoggerProvider.createBadRecordLogger(configuration);
     RowConverter converter =
         new RowConverterImpl(child.getOutput(), configuration, badRecordLogger);
-    configuration.setCardinalityFinder(converter);
     converters.add(converter);
     converter.initialize();
 
diff --git a/processing/src/main/java/org/apache/carbondata/processing/loading/steps/InputProcessorStepWithNoConverterImpl.java b/processing/src/main/java/org/apache/carbondata/processing/loading/steps/InputProcessorStepWithNoConverterImpl.java
index fb1a62a..811d8e4 100644
--- a/processing/src/main/java/org/apache/carbondata/processing/loading/steps/InputProcessorStepWithNoConverterImpl.java
+++ b/processing/src/main/java/org/apache/carbondata/processing/loading/steps/InputProcessorStepWithNoConverterImpl.java
@@ -87,7 +87,6 @@ public class InputProcessorStepWithNoConverterImpl extends AbstractDataLoadProce
     rowConverter =
         new RowConverterImpl(configuration.getDataFields(), configuration, null);
     rowConverter.initialize();
-    configuration.setCardinalityFinder(rowConverter);
     noDictionaryMapping =
         CarbonDataProcessorUtil.getNoDictionaryMapping(configuration.getDataFields());
 
diff --git a/processing/src/main/java/org/apache/carbondata/processing/merger/CarbonCompactionExecutor.java b/processing/src/main/java/org/apache/carbondata/processing/merger/CarbonCompactionExecutor.java
index abfe11b..94ac46b 100644
--- a/processing/src/main/java/org/apache/carbondata/processing/merger/CarbonCompactionExecutor.java
+++ b/processing/src/main/java/org/apache/carbondata/processing/merger/CarbonCompactionExecutor.java
@@ -179,8 +179,7 @@ public class CarbonCompactionExecutor {
       String task, List<TableBlockInfo> tableBlockInfoList)
       throws IOException {
     SegmentProperties sourceSegmentProperties =
-        new SegmentProperties(tableBlockInfoList.get(0).getDataFileFooter().getColumnInTable(),
-            tableBlockInfoList.get(0).getDataFileFooter().getSegmentInfo().getColumnCardinality());
+        new SegmentProperties(tableBlockInfoList.get(0).getDataFileFooter().getColumnInTable());
     boolean hasColumnDrift = carbonTable.hasColumnDrift() &&
         RestructureUtil.hasColumnDriftOnSegment(carbonTable, sourceSegmentProperties);
     if (hasColumnDrift) {
@@ -211,7 +210,7 @@ public class CarbonCompactionExecutor {
       // get the columnValueSize for the dataFileFooter
       IntArrayWrapper columnValueSize = new IntArrayWrapper(
           getSourceSegmentProperties(Collections.singletonList(tableBlock.getDataFileFooter()))
-              .getColumnsValueSize());
+              .createColumnValueLength());
       List<TableBlockInfo> tempBlockInfoList =
           columnvalueSizeToTableBlockInfoMap.get(columnValueSize);
       if (tempBlockInfoList == null) {
@@ -236,22 +235,11 @@ public class CarbonCompactionExecutor {
   private SegmentProperties getSourceSegmentProperties(List<DataFileFooter> listMetadata) {
     SegmentProperties sourceSegProperties = null;
     if (restructuredBlockExists) {
-      // update cardinality of source segment according to new schema
-      Map<String, Integer> columnToCardinalityMap =
-          new HashMap<>(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE);
-      CarbonCompactionUtil
-          .addColumnCardinalityToMap(columnToCardinalityMap, listMetadata.get(0).getColumnInTable(),
-              listMetadata.get(0).getSegmentInfo().getColumnCardinality());
       List<ColumnSchema> updatedColumnSchemaList =
           new ArrayList<>(listMetadata.get(0).getColumnInTable().size());
-      int[] updatedColumnCardinalities = CarbonCompactionUtil
-          .updateColumnSchemaAndGetCardinality(columnToCardinalityMap, carbonTable,
-              updatedColumnSchemaList);
-      sourceSegProperties =
-          new SegmentProperties(updatedColumnSchemaList, updatedColumnCardinalities);
+      sourceSegProperties = new SegmentProperties(updatedColumnSchemaList);
     } else {
-      sourceSegProperties = new SegmentProperties(listMetadata.get(0).getColumnInTable(),
-          listMetadata.get(0).getSegmentInfo().getColumnCardinality());
+      sourceSegProperties = new SegmentProperties(listMetadata.get(0).getColumnInTable());
     }
     return sourceSegProperties;
   }
diff --git a/processing/src/main/java/org/apache/carbondata/processing/merger/CarbonCompactionUtil.java b/processing/src/main/java/org/apache/carbondata/processing/merger/CarbonCompactionUtil.java
index 3bdc9dc..5503924 100644
--- a/processing/src/main/java/org/apache/carbondata/processing/merger/CarbonCompactionUtil.java
... 883 lines suppressed ...