You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@carbondata.apache.org by ku...@apache.org on 2018/09/10 08:55:27 UTC

carbondata git commit: [CARBONDATA-2889]Add decoder based fallback mechanism in local dictionary to reduce memory footprint

Repository: carbondata
Updated Branches:
  refs/heads/master 9ebab5748 -> 2ccdbb78c


[CARBONDATA-2889]Add decoder based fallback mechanism in local dictionary to reduce memory footprint

Problem
Currently, when the fallback is initiated for a column page in case of local dictionary,
we are keeping both encoded data and actual data in memory and then we form the new column page without
dictionary encoding and then at last we free the Encoded Column Page.Because of this offheap memory footprint increases.

Solution
We can reduce the offheap memory footprint. This can be done using decoder based fallback mechanism.
This means, no need to keep the actual data along with encoded data in encoded column page.
We can keep only encoded data and to form a new column page,
get the dictionary data from encoded column page by uncompressing and using dictionary data
get the actual data using local dictionary generator and put it in new column page created and
compress it again and give to consumer for writing blocklet.

This closes #2662


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

Branch: refs/heads/master
Commit: 2ccdbb78c461be8c68770f7732c233c319a65ad1
Parents: 9ebab57
Author: akashrn5 <ak...@gmail.com>
Authored: Mon Aug 20 10:29:26 2018 +0530
Committer: kumarvishal09 <ku...@gmail.com>
Committed: Mon Sep 10 14:24:55 2018 +0530

----------------------------------------------------------------------
 .../core/constants/CarbonCommonConstants.java   |  10 ++
 .../blocklet/BlockletEncodedColumnPage.java     |  42 +++++-
 .../datastore/blocklet/EncodedBlocklet.java     |  34 +++--
 .../reader/dimension/AbstractChunkReader.java   |  15 ---
 .../AbstractChunkReaderV2V3Format.java          |  12 --
 ...mpressedDimensionChunkFileBasedReaderV1.java |   2 +-
 ...mpressedDimensionChunkFileBasedReaderV2.java |   8 +-
 ...essedDimChunkFileBasedPageLevelReaderV3.java |   4 +-
 ...mpressedDimensionChunkFileBasedReaderV3.java |  10 +-
 .../page/ActualDataBasedFallbackEncoder.java    |  67 ++++++++++
 .../core/datastore/page/ColumnPage.java         |   9 +-
 .../page/DecoderBasedFallbackEncoder.java       | 132 +++++++++++++++++++
 .../page/FallbackColumnPageEncoder.java         |  86 ------------
 .../datastore/page/LocalDictColumnPage.java     |  19 ++-
 .../apache/carbondata/core/util/CarbonUtil.java |  73 ++++++++++
 .../VectorizedCarbonRecordReader.java           |   3 +-
 .../store/writer/v3/BlockletDataHolder.java     |  11 +-
 .../writer/v3/CarbonFactDataWriterImplV3.java   |   2 +-
 18 files changed, 389 insertions(+), 150 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/carbondata/blob/2ccdbb78/core/src/main/java/org/apache/carbondata/core/constants/CarbonCommonConstants.java
----------------------------------------------------------------------
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 3bdb2f7..7a34c98 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
@@ -881,6 +881,16 @@ public final class CarbonCommonConstants {
   public static final String LOCAL_DICTIONARY_SYSTEM_ENABLE = "carbon.local.dictionary.enable";
 
   /**
+   * System property to enable or disable decoder based local dictionary fallback
+   */
+  public static final String LOCAL_DICTIONARY_DECODER_BASED_FALLBACK =
+      "carbon.local.dictionary.decoder.fallback";
+
+  /**
+   * System property to enable or disable decoder based local dictionary fallback default value
+   */
+  public static final String LOCAL_DICTIONARY_DECODER_BASED_FALLBACK_DEFAULT = "true";
+  /**
    * Threshold value for local dictionary
    */
   public static final String LOCAL_DICTIONARY_THRESHOLD = "local_dictionary_threshold";

http://git-wip-us.apache.org/repos/asf/carbondata/blob/2ccdbb78/core/src/main/java/org/apache/carbondata/core/datastore/blocklet/BlockletEncodedColumnPage.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/datastore/blocklet/BlockletEncodedColumnPage.java b/core/src/main/java/org/apache/carbondata/core/datastore/blocklet/BlockletEncodedColumnPage.java
index 8abc0e4..135b1e2 100644
--- a/core/src/main/java/org/apache/carbondata/core/datastore/blocklet/BlockletEncodedColumnPage.java
+++ b/core/src/main/java/org/apache/carbondata/core/datastore/blocklet/BlockletEncodedColumnPage.java
@@ -26,10 +26,12 @@ import java.util.concurrent.Future;
 
 import org.apache.carbondata.common.logging.LogService;
 import org.apache.carbondata.common.logging.LogServiceFactory;
-import org.apache.carbondata.core.datastore.page.FallbackColumnPageEncoder;
+import org.apache.carbondata.core.datastore.page.ActualDataBasedFallbackEncoder;
+import org.apache.carbondata.core.datastore.page.DecoderBasedFallbackEncoder;
 import org.apache.carbondata.core.datastore.page.FallbackEncodedColumnPage;
 import org.apache.carbondata.core.datastore.page.encoding.EncodedColumnPage;
 import org.apache.carbondata.core.localdictionary.PageLevelDictionary;
+import org.apache.carbondata.core.localdictionary.generator.LocalDictionaryGenerator;
 import org.apache.carbondata.core.memory.MemoryException;
 import org.apache.carbondata.format.LocalDictionaryChunk;
 
@@ -75,9 +77,22 @@ public class BlockletEncodedColumnPage {
 
   private String columnName;
 
-  BlockletEncodedColumnPage(ExecutorService fallbackExecutorService) {
+  /**
+   * is decoder based fallback enabled
+   */
+  private boolean isDecoderBasedFallBackEnabled;
+
+  /**
+   * Local dictionary generator for column
+   */
+  private LocalDictionaryGenerator localDictionaryGenerator;
+
+  BlockletEncodedColumnPage(ExecutorService fallbackExecutorService,
+      boolean isDecoderBasedFallBackEnabled, LocalDictionaryGenerator localDictionaryGenerator) {
     this.fallbackExecutorService = fallbackExecutorService;
     this.fallbackFutureQueue = new ArrayDeque<>();
+    this.isDecoderBasedFallBackEnabled = isDecoderBasedFallBackEnabled;
+    this.localDictionaryGenerator = localDictionaryGenerator;
   }
 
   /**
@@ -86,7 +101,7 @@ public class BlockletEncodedColumnPage {
    * @param encodedColumnPage
    * encoded column page
    */
-  void addEncodedColumnColumnPage(EncodedColumnPage encodedColumnPage) {
+  void addEncodedColumnPage(EncodedColumnPage encodedColumnPage) {
     if (null == encodedColumnPageList) {
       this.encodedColumnPageList = new ArrayList<>();
       // if dimension page is local dictionary enabled and encoded with local dictionary
@@ -105,8 +120,7 @@ public class BlockletEncodedColumnPage {
       LOGGER.info(
           "Local dictionary Fallback is initiated for column: " + this.columnName + " for page:"
               + encodedColumnPageList.size());
-      fallbackFutureQueue.add(fallbackExecutorService
-          .submit(new FallbackColumnPageEncoder(encodedColumnPage, encodedColumnPageList.size())));
+      initiateFallBack(encodedColumnPage, encodedColumnPageList.size());
       // fill null so once page is decoded again fill the re-encoded page again
       this.encodedColumnPageList.add(null);
     }
@@ -128,8 +142,7 @@ public class BlockletEncodedColumnPage {
       // submit all the older pages encoded with dictionary for fallback
       for (int pageIndex = 0; pageIndex < encodedColumnPageList.size(); pageIndex++) {
         if (encodedColumnPageList.get(pageIndex).getActualPage().isLocalDictGeneratedPage()) {
-          fallbackFutureQueue.add(fallbackExecutorService.submit(
-              new FallbackColumnPageEncoder(encodedColumnPageList.get(pageIndex), pageIndex)));
+          initiateFallBack(encodedColumnPageList.get(pageIndex), pageIndex);
         }
       }
       //add to page list
@@ -188,4 +201,19 @@ public class BlockletEncodedColumnPage {
     }
     return null;
   }
+
+  /**
+   * This method initiates the fallback for local dictionary encoded column page
+   * @param encodedColumnPage
+   * @param pageIndex
+   */
+  private void initiateFallBack(EncodedColumnPage encodedColumnPage, int pageIndex) {
+    if (isDecoderBasedFallBackEnabled) {
+      fallbackFutureQueue.add(fallbackExecutorService.submit(
+          new DecoderBasedFallbackEncoder(encodedColumnPage, pageIndex, localDictionaryGenerator)));
+    } else {
+      fallbackFutureQueue.add(fallbackExecutorService.submit(
+          new ActualDataBasedFallbackEncoder(encodedColumnPage, encodedColumnPageList.size())));
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/2ccdbb78/core/src/main/java/org/apache/carbondata/core/datastore/blocklet/EncodedBlocklet.java
----------------------------------------------------------------------
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 794c439..d017145 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
@@ -18,10 +18,12 @@ package org.apache.carbondata.core.datastore.blocklet;
 
 import java.util.ArrayList;
 import java.util.List;
+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;
 
 /**
  * Holds the blocklet level data and metadata to be written in carbondata file
@@ -60,8 +62,21 @@ public class EncodedBlocklet {
    */
   private int numberOfPages;
 
-  public EncodedBlocklet(ExecutorService executorService) {
+  /**
+   * is decoder based fallback is enabled or not
+   */
+  private boolean isDecoderBasedFallBackEnabled;
+
+  /**
+   * local dictionary generator map of all local dictionary columns
+   */
+  private Map<String, LocalDictionaryGenerator> localDictionaryGeneratorMap;
+
+  public EncodedBlocklet(ExecutorService executorService, boolean isDecoderBasedFallBackEnabled,
+      Map<String, LocalDictionaryGenerator> localDictionaryGeneratorMap) {
     this.executorService = executorService;
+    this.isDecoderBasedFallBackEnabled = isDecoderBasedFallBackEnabled;
+    this.localDictionaryGeneratorMap = localDictionaryGeneratorMap;
   }
 
   /**
@@ -93,13 +108,14 @@ public class EncodedBlocklet {
       encodedMeasureColumnPages = new ArrayList<>();
       // adding measure pages
       for (int i = 0; i < encodedTablePage.getNumMeasures(); i++) {
-        BlockletEncodedColumnPage blockletEncodedColumnPage = new BlockletEncodedColumnPage(null);
-        blockletEncodedColumnPage.addEncodedColumnColumnPage(encodedTablePage.getMeasure(i));
+        BlockletEncodedColumnPage blockletEncodedColumnPage =
+            new BlockletEncodedColumnPage(null, false, null);
+        blockletEncodedColumnPage.addEncodedColumnPage(encodedTablePage.getMeasure(i));
         encodedMeasureColumnPages.add(blockletEncodedColumnPage);
       }
     } else {
       for (int i = 0; i < encodedTablePage.getNumMeasures(); i++) {
-        encodedMeasureColumnPages.get(i).addEncodedColumnColumnPage(encodedTablePage.getMeasure(i));
+        encodedMeasureColumnPages.get(i).addEncodedColumnPage(encodedTablePage.getMeasure(i));
       }
     }
   }
@@ -117,14 +133,16 @@ public class EncodedBlocklet {
       // adding measure pages
       for (int i = 0; i < encodedTablePage.getNumDimensions(); i++) {
         BlockletEncodedColumnPage blockletEncodedColumnPage =
-            new BlockletEncodedColumnPage(executorService);
-        blockletEncodedColumnPage.addEncodedColumnColumnPage(encodedTablePage.getDimension(i));
+            new BlockletEncodedColumnPage(executorService, isDecoderBasedFallBackEnabled,
+                localDictionaryGeneratorMap.get(
+                    encodedTablePage.getDimension(i).getActualPage().getColumnSpec()
+                        .getFieldName()));
+        blockletEncodedColumnPage.addEncodedColumnPage(encodedTablePage.getDimension(i));
         encodedDimensionColumnPages.add(blockletEncodedColumnPage);
       }
     } else {
       for (int i = 0; i < encodedTablePage.getNumDimensions(); i++) {
-        encodedDimensionColumnPages.get(i)
-            .addEncodedColumnColumnPage(encodedTablePage.getDimension(i));
+        encodedDimensionColumnPages.get(i).addEncodedColumnPage(encodedTablePage.getDimension(i));
       }
     }
   }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/2ccdbb78/core/src/main/java/org/apache/carbondata/core/datastore/chunk/reader/dimension/AbstractChunkReader.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/datastore/chunk/reader/dimension/AbstractChunkReader.java b/core/src/main/java/org/apache/carbondata/core/datastore/chunk/reader/dimension/AbstractChunkReader.java
index b08fad2..28e8741 100644
--- a/core/src/main/java/org/apache/carbondata/core/datastore/chunk/reader/dimension/AbstractChunkReader.java
+++ b/core/src/main/java/org/apache/carbondata/core/datastore/chunk/reader/dimension/AbstractChunkReader.java
@@ -80,19 +80,4 @@ public abstract class AbstractChunkReader implements DimensionColumnChunkReader
     this.numberOfRows = numberOfRows;
   }
 
-  /**
-   * Below method will be used to create the inverted index reverse
-   * this will be used to point to actual data in the chunk
-   *
-   * @param invertedIndex inverted index
-   * @return reverse inverted index
-   */
-  protected int[] getInvertedReverseIndex(int[] invertedIndex) {
-    int[] columnIndexTemp = new int[invertedIndex.length];
-
-    for (int i = 0; i < invertedIndex.length; i++) {
-      columnIndexTemp[invertedIndex[i]] = i;
-    }
-    return columnIndexTemp;
-  }
 }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/2ccdbb78/core/src/main/java/org/apache/carbondata/core/datastore/chunk/reader/dimension/AbstractChunkReaderV2V3Format.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/datastore/chunk/reader/dimension/AbstractChunkReaderV2V3Format.java b/core/src/main/java/org/apache/carbondata/core/datastore/chunk/reader/dimension/AbstractChunkReaderV2V3Format.java
index 60950c9..1c85020 100644
--- a/core/src/main/java/org/apache/carbondata/core/datastore/chunk/reader/dimension/AbstractChunkReaderV2V3Format.java
+++ b/core/src/main/java/org/apache/carbondata/core/datastore/chunk/reader/dimension/AbstractChunkReaderV2V3Format.java
@@ -22,7 +22,6 @@ import java.util.List;
 import org.apache.carbondata.core.datastore.FileReader;
 import org.apache.carbondata.core.datastore.chunk.impl.DimensionRawColumnChunk;
 import org.apache.carbondata.core.metadata.blocklet.BlockletInfo;
-import org.apache.carbondata.format.Encoding;
 
 /**
  * Abstract class for V2, V3 format dimension column reader
@@ -112,15 +111,4 @@ public abstract class AbstractChunkReaderV2V3Format extends AbstractChunkReader
   protected abstract DimensionRawColumnChunk[] readRawDimensionChunksInGroup(FileReader fileReader,
       int startColumnBlockletIndex, int endColumnBlockletIndex) throws IOException;
 
-  /**
-   * Below method will be used to check whether particular encoding is present
-   * in the dimension or not
-   *
-   * @param encoding encoding to search
-   * @return if encoding is present in dimension
-   */
-  protected boolean hasEncoding(List<Encoding> encodings, Encoding encoding) {
-    return encodings.contains(encoding);
-  }
-
 }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/2ccdbb78/core/src/main/java/org/apache/carbondata/core/datastore/chunk/reader/dimension/v1/CompressedDimensionChunkFileBasedReaderV1.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/datastore/chunk/reader/dimension/v1/CompressedDimensionChunkFileBasedReaderV1.java b/core/src/main/java/org/apache/carbondata/core/datastore/chunk/reader/dimension/v1/CompressedDimensionChunkFileBasedReaderV1.java
index 593d992..8256b7e 100644
--- a/core/src/main/java/org/apache/carbondata/core/datastore/chunk/reader/dimension/v1/CompressedDimensionChunkFileBasedReaderV1.java
+++ b/core/src/main/java/org/apache/carbondata/core/datastore/chunk/reader/dimension/v1/CompressedDimensionChunkFileBasedReaderV1.java
@@ -125,7 +125,7 @@ public class CompressedDimensionChunkFileBasedReaderV1 extends AbstractChunkRead
           .getUnCompressColumnIndex(dataChunk.getRowIdPageLength(),
               columnIndexData, numberComressor, 0);
       // get the reverse index
-      invertedIndexesReverse = getInvertedReverseIndex(invertedIndexes);
+      invertedIndexesReverse = CarbonUtil.getInvertedReverseIndex(invertedIndexes);
     }
     // if rle is applied then read the rle block chunk and then uncompress
     //then actual data based on rle block

http://git-wip-us.apache.org/repos/asf/carbondata/blob/2ccdbb78/core/src/main/java/org/apache/carbondata/core/datastore/chunk/reader/dimension/v2/CompressedDimensionChunkFileBasedReaderV2.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/datastore/chunk/reader/dimension/v2/CompressedDimensionChunkFileBasedReaderV2.java b/core/src/main/java/org/apache/carbondata/core/datastore/chunk/reader/dimension/v2/CompressedDimensionChunkFileBasedReaderV2.java
index 252a675..a44d710 100644
--- a/core/src/main/java/org/apache/carbondata/core/datastore/chunk/reader/dimension/v2/CompressedDimensionChunkFileBasedReaderV2.java
+++ b/core/src/main/java/org/apache/carbondata/core/datastore/chunk/reader/dimension/v2/CompressedDimensionChunkFileBasedReaderV2.java
@@ -147,7 +147,7 @@ public class CompressedDimensionChunkFileBasedReaderV2 extends AbstractChunkRead
         .unCompressByte(rawData.array(), copySourcePoint, dimensionColumnChunk.data_page_length);
     copySourcePoint += dimensionColumnChunk.data_page_length;
     // if row id block is present then read the row id chunk and uncompress it
-    if (hasEncoding(dimensionColumnChunk.encoders, Encoding.INVERTED_INDEX)) {
+    if (CarbonUtil.hasEncoding(dimensionColumnChunk.encoders, Encoding.INVERTED_INDEX)) {
       byte[] dataInv = new byte[dimensionColumnChunk.rowid_page_length];
       rawData.position(copySourcePoint);
       rawData.get(dataInv);
@@ -156,11 +156,11 @@ public class CompressedDimensionChunkFileBasedReaderV2 extends AbstractChunkRead
               numberComressor, 0);
       copySourcePoint += dimensionColumnChunk.rowid_page_length;
       // get the reverse index
-      invertedIndexesReverse = getInvertedReverseIndex(invertedIndexes);
+      invertedIndexesReverse = CarbonUtil.getInvertedReverseIndex(invertedIndexes);
     }
     // if rle is applied then read the rle block chunk and then uncompress
     //then actual data based on rle block
-    if (hasEncoding(dimensionColumnChunk.encoders, Encoding.RLE)) {
+    if (CarbonUtil.hasEncoding(dimensionColumnChunk.encoders, Encoding.RLE)) {
       byte[] dataRle = new byte[dimensionColumnChunk.rle_page_length];
       rawData.position(copySourcePoint);
       rawData.get(dataRle);
@@ -173,7 +173,7 @@ public class CompressedDimensionChunkFileBasedReaderV2 extends AbstractChunkRead
 
     // if no dictionary column then first create a no dictionary column chunk
     // and set to data chunk instance
-    if (!hasEncoding(dimensionColumnChunk.encoders, Encoding.DICTIONARY)) {
+    if (!CarbonUtil.hasEncoding(dimensionColumnChunk.encoders, Encoding.DICTIONARY)) {
       columnDataChunk =
           new VariableLengthDimensionColumnPage(dataPage, invertedIndexes, invertedIndexesReverse,
               numberOfRows, DimensionChunkStoreFactory.DimensionStoreType.VARIABLE_SHORT_LENGTH,

http://git-wip-us.apache.org/repos/asf/carbondata/blob/2ccdbb78/core/src/main/java/org/apache/carbondata/core/datastore/chunk/reader/dimension/v3/CompressedDimChunkFileBasedPageLevelReaderV3.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/datastore/chunk/reader/dimension/v3/CompressedDimChunkFileBasedPageLevelReaderV3.java b/core/src/main/java/org/apache/carbondata/core/datastore/chunk/reader/dimension/v3/CompressedDimChunkFileBasedPageLevelReaderV3.java
index 3f41c2b..e69984b 100644
--- a/core/src/main/java/org/apache/carbondata/core/datastore/chunk/reader/dimension/v3/CompressedDimChunkFileBasedPageLevelReaderV3.java
+++ b/core/src/main/java/org/apache/carbondata/core/datastore/chunk/reader/dimension/v3/CompressedDimChunkFileBasedPageLevelReaderV3.java
@@ -153,11 +153,11 @@ public class CompressedDimChunkFileBasedPageLevelReaderV3
         .get(dimensionRawColumnChunk.getColumnIndex()) + dataChunk3.getPage_offset()
         .get(pageNumber);
     int length = pageMetadata.data_page_length;
-    if (hasEncoding(pageMetadata.encoders, Encoding.INVERTED_INDEX)) {
+    if (CarbonUtil.hasEncoding(pageMetadata.encoders, Encoding.INVERTED_INDEX)) {
       length += pageMetadata.rowid_page_length;
     }
 
-    if (hasEncoding(pageMetadata.encoders, Encoding.RLE)) {
+    if (CarbonUtil.hasEncoding(pageMetadata.encoders, Encoding.RLE)) {
       length += pageMetadata.rle_page_length;
     }
     // get the data buffer

http://git-wip-us.apache.org/repos/asf/carbondata/blob/2ccdbb78/core/src/main/java/org/apache/carbondata/core/datastore/chunk/reader/dimension/v3/CompressedDimensionChunkFileBasedReaderV3.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/datastore/chunk/reader/dimension/v3/CompressedDimensionChunkFileBasedReaderV3.java b/core/src/main/java/org/apache/carbondata/core/datastore/chunk/reader/dimension/v3/CompressedDimensionChunkFileBasedReaderV3.java
index ef2c189..8a2b74e 100644
--- a/core/src/main/java/org/apache/carbondata/core/datastore/chunk/reader/dimension/v3/CompressedDimensionChunkFileBasedReaderV3.java
+++ b/core/src/main/java/org/apache/carbondata/core/datastore/chunk/reader/dimension/v3/CompressedDimensionChunkFileBasedReaderV3.java
@@ -276,16 +276,16 @@ public class CompressedDimensionChunkFileBasedReaderV3 extends AbstractChunkRead
     dataPage = COMPRESSOR.unCompressByte(pageData.array(), offset, pageMetadata.data_page_length);
     offset += pageMetadata.data_page_length;
     // if row id block is present then read the row id chunk and uncompress it
-    if (hasEncoding(pageMetadata.encoders, Encoding.INVERTED_INDEX)) {
+    if (CarbonUtil.hasEncoding(pageMetadata.encoders, Encoding.INVERTED_INDEX)) {
       invertedIndexes = CarbonUtil
           .getUnCompressColumnIndex(pageMetadata.rowid_page_length, pageData, offset);
       offset += pageMetadata.rowid_page_length;
       // get the reverse index
-      invertedIndexesReverse = getInvertedReverseIndex(invertedIndexes);
+      invertedIndexesReverse = CarbonUtil.getInvertedReverseIndex(invertedIndexes);
     }
     // if rle is applied then read the rle block chunk and then uncompress
     //then actual data based on rle block
-    if (hasEncoding(pageMetadata.encoders, Encoding.RLE)) {
+    if (CarbonUtil.hasEncoding(pageMetadata.encoders, Encoding.RLE)) {
       rlePage =
           CarbonUtil.getIntArray(pageData, offset, pageMetadata.rle_page_length);
       // uncompress the data with rle indexes
@@ -298,11 +298,11 @@ public class CompressedDimensionChunkFileBasedReaderV3 extends AbstractChunkRead
     DimensionColumnPage columnDataChunk = null;
     // if no dictionary column then first create a no dictionary column chunk
     // and set to data chunk instance
-    if (!hasEncoding(pageMetadata.encoders, Encoding.DICTIONARY)) {
+    if (!CarbonUtil.hasEncoding(pageMetadata.encoders, Encoding.DICTIONARY)) {
       DimensionChunkStoreFactory.DimensionStoreType dimStoreType =
           null != rawColumnPage.getLocalDictionary() ?
               DimensionChunkStoreFactory.DimensionStoreType.LOCAL_DICT :
-              (hasEncoding(pageMetadata.encoders, Encoding.DIRECT_COMPRESS_VARCHAR) ?
+              (CarbonUtil.hasEncoding(pageMetadata.encoders, Encoding.DIRECT_COMPRESS_VARCHAR) ?
                   DimensionChunkStoreFactory.DimensionStoreType.VARIABLE_INT_LENGTH :
                   DimensionChunkStoreFactory.DimensionStoreType.VARIABLE_SHORT_LENGTH);
       columnDataChunk =

http://git-wip-us.apache.org/repos/asf/carbondata/blob/2ccdbb78/core/src/main/java/org/apache/carbondata/core/datastore/page/ActualDataBasedFallbackEncoder.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/datastore/page/ActualDataBasedFallbackEncoder.java b/core/src/main/java/org/apache/carbondata/core/datastore/page/ActualDataBasedFallbackEncoder.java
new file mode 100644
index 0000000..6efb1b8
--- /dev/null
+++ b/core/src/main/java/org/apache/carbondata/core/datastore/page/ActualDataBasedFallbackEncoder.java
@@ -0,0 +1,67 @@
+/*
+ * 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;
+
+import java.util.concurrent.Callable;
+
+import org.apache.carbondata.core.datastore.TableSpec;
+import org.apache.carbondata.core.datastore.page.encoding.EncodedColumnPage;
+import org.apache.carbondata.core.util.CarbonUtil;
+
+/**
+ * Below class will be used to encode column pages for which local dictionary was generated
+ * but all the pages in blocklet was not encoded with local dictionary.
+ * This is required as all the pages of a column in blocklet either it will be local dictionary
+ * encoded or without local dictionary encoded.
+ */
+public class ActualDataBasedFallbackEncoder
+    implements Callable<FallbackEncodedColumnPage> {
+
+  /**
+   * actual local dictionary generated column page
+   */
+  private EncodedColumnPage encodedColumnPage;
+
+  /**
+   * actual index in the page
+   * this is required as in a blocklet few pages will be local dictionary
+   * encoded and few pages will be plain text encoding
+   * in this case local dictionary encoded page
+   */
+  private int pageIndex;
+
+  public ActualDataBasedFallbackEncoder(EncodedColumnPage encodedColumnPage,
+      int pageIndex) {
+    this.encodedColumnPage = encodedColumnPage;
+    this.pageIndex = pageIndex;
+  }
+
+  @Override public FallbackEncodedColumnPage call() throws Exception {
+    // disable encoding using local dictionary
+    encodedColumnPage.getActualPage().disableLocalDictEncoding();
+
+    // get column spec for existing column page
+    TableSpec.ColumnSpec columnSpec = encodedColumnPage.getActualPage().getColumnSpec();
+    FallbackEncodedColumnPage fallbackEncodedColumnPage = CarbonUtil
+        .getFallBackEncodedColumnPage(encodedColumnPage.getActualPage(), pageIndex, columnSpec);
+    // here freeing the memory of raw column page as fallback is done and column page will not
+    // be used.
+    // This is required to free the memory once it is of no use
+    encodedColumnPage.freeMemory();
+    return fallbackEncodedColumnPage;
+  }
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/2ccdbb78/core/src/main/java/org/apache/carbondata/core/datastore/page/ColumnPage.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/datastore/page/ColumnPage.java b/core/src/main/java/org/apache/carbondata/core/datastore/page/ColumnPage.java
index 1330aa3..5b42735 100644
--- a/core/src/main/java/org/apache/carbondata/core/datastore/page/ColumnPage.java
+++ b/core/src/main/java/org/apache/carbondata/core/datastore/page/ColumnPage.java
@@ -165,15 +165,18 @@ public abstract class ColumnPage {
   public static ColumnPage newLocalDictPage(TableSpec.ColumnSpec columnSpec, DataType dataType,
       int pageSize, LocalDictionaryGenerator localDictionaryGenerator,
       boolean isComplexTypePrimitive) throws MemoryException {
+    boolean isDecoderBasedFallBackEnabled = Boolean.parseBoolean(CarbonProperties.getInstance()
+        .getProperty(CarbonCommonConstants.LOCAL_DICTIONARY_DECODER_BASED_FALLBACK,
+            CarbonCommonConstants.LOCAL_DICTIONARY_DECODER_BASED_FALLBACK_DEFAULT));
     if (unsafe) {
       return new LocalDictColumnPage(new UnsafeVarLengthColumnPage(columnSpec, dataType, pageSize),
           new UnsafeFixLengthColumnPage(columnSpec, DataTypes.BYTE_ARRAY, pageSize,
-              CarbonCommonConstants.LOCAL_DICT_ENCODED_BYTEARRAY_SIZE),
-          localDictionaryGenerator, isComplexTypePrimitive);
+              CarbonCommonConstants.LOCAL_DICT_ENCODED_BYTEARRAY_SIZE), localDictionaryGenerator,
+          isComplexTypePrimitive, isDecoderBasedFallBackEnabled);
     } else {
       return new LocalDictColumnPage(new SafeVarLengthColumnPage(columnSpec, dataType, pageSize),
           new SafeFixLengthColumnPage(columnSpec, DataTypes.BYTE_ARRAY, pageSize),
-          localDictionaryGenerator, isComplexTypePrimitive);
+          localDictionaryGenerator, isComplexTypePrimitive, isDecoderBasedFallBackEnabled);
     }
   }
 

http://git-wip-us.apache.org/repos/asf/carbondata/blob/2ccdbb78/core/src/main/java/org/apache/carbondata/core/datastore/page/DecoderBasedFallbackEncoder.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/datastore/page/DecoderBasedFallbackEncoder.java b/core/src/main/java/org/apache/carbondata/core/datastore/page/DecoderBasedFallbackEncoder.java
new file mode 100644
index 0000000..d85d6cd
--- /dev/null
+++ b/core/src/main/java/org/apache/carbondata/core/datastore/page/DecoderBasedFallbackEncoder.java
@@ -0,0 +1,132 @@
+/*
+ * 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;
+
+import java.nio.ByteBuffer;
+import java.util.concurrent.Callable;
+
+import org.apache.carbondata.core.constants.CarbonCommonConstants;
+import org.apache.carbondata.core.datastore.TableSpec;
+import org.apache.carbondata.core.datastore.columnar.UnBlockIndexer;
+import org.apache.carbondata.core.datastore.compression.CompressorFactory;
+import org.apache.carbondata.core.datastore.page.encoding.EncodedColumnPage;
+import org.apache.carbondata.core.keygenerator.KeyGenerator;
+import org.apache.carbondata.core.keygenerator.factory.KeyGeneratorFactory;
+import org.apache.carbondata.core.localdictionary.generator.LocalDictionaryGenerator;
+import org.apache.carbondata.core.metadata.datatype.DataType;
+import org.apache.carbondata.core.util.CarbonUtil;
+import org.apache.carbondata.format.Encoding;
+
+public class DecoderBasedFallbackEncoder implements Callable<FallbackEncodedColumnPage> {
+  /**
+   * actual local dictionary generated column page
+   */
+  private EncodedColumnPage encodedColumnPage;
+
+  /**
+   * actual index in the page
+   * this is required as in a blocklet few pages will be local dictionary
+   * encoded and few pages will be plain text encoding
+   * in this case local dictionary encoded page
+   */
+  private int pageIndex;
+
+  private LocalDictionaryGenerator localDictionaryGenerator;
+
+  public DecoderBasedFallbackEncoder(EncodedColumnPage encodedColumnPage, int pageIndex,
+      LocalDictionaryGenerator localDictionaryGenerator) {
+    this.encodedColumnPage = encodedColumnPage;
+    this.pageIndex = pageIndex;
+    this.localDictionaryGenerator = localDictionaryGenerator;
+  }
+
+  @Override public FallbackEncodedColumnPage call() throws Exception {
+    int pageSize =
+        encodedColumnPage.getActualPage().getPageSize();
+    int offset = 0;
+    int[] reverseInvertedIndex = new int[pageSize];
+    for (int i = 0; i < pageSize; i++) {
+      reverseInvertedIndex[i] = i;
+    }
+    int[] rlePage;
+
+    // uncompress the encoded column page
+    byte[] bytes = CompressorFactory.getInstance().getCompressor()
+        .unCompressByte(encodedColumnPage.getEncodedData().array(), offset,
+            encodedColumnPage.getPageMetadata().data_page_length);
+
+    offset += encodedColumnPage.getPageMetadata().data_page_length;
+    ByteBuffer data = ByteBuffer.wrap(encodedColumnPage.getEncodedData().array());
+
+    // if encoded with inverted index, get all the inverted indexes
+    if (CarbonUtil
+        .hasEncoding(encodedColumnPage.getPageMetadata().encoders, Encoding.INVERTED_INDEX)) {
+      int[] invertedIndexes = CarbonUtil
+          .getUnCompressColumnIndex(encodedColumnPage.getPageMetadata().rowid_page_length, data,
+              offset);
+      offset += encodedColumnPage.getPageMetadata().rowid_page_length;
+      // get all the reverse inverted index
+      reverseInvertedIndex = CarbonUtil.getInvertedReverseIndex(invertedIndexes);
+    }
+    // if rle is applied then uncompress then actual data based on rle
+    if (CarbonUtil.hasEncoding(encodedColumnPage.getPageMetadata().encoders, Encoding.RLE)) {
+      rlePage =
+          CarbonUtil.getIntArray(data, offset, encodedColumnPage.getPageMetadata().rle_page_length);
+      // uncompress the data with rle indexes
+      bytes = UnBlockIndexer
+          .uncompressData(bytes, rlePage, CarbonCommonConstants.LOCAL_DICT_ENCODED_BYTEARRAY_SIZE);
+    }
+
+    // disable encoding using local dictionary
+    encodedColumnPage.getActualPage().disableLocalDictEncoding();
+
+    // get column spec for existing column page
+    TableSpec.ColumnSpec columnSpec = encodedColumnPage.getActualPage().getColumnSpec();
+
+    // get the dataType of column
+    DataType dataType = encodedColumnPage.getActualPage().getDataType();
+
+    // create a new column page which will have actual data instead of encoded data
+    ColumnPage actualDataColumnPage =
+        ColumnPage.newPage(columnSpec, dataType, encodedColumnPage.getActualPage().getPageSize());
+
+    // uncompressed data from encoded column page is dictionary data, get the dictionary data using
+    // keygenerator
+    KeyGenerator keyGenerator = KeyGeneratorFactory
+        .getKeyGenerator(new int[] { CarbonCommonConstants.LOCAL_DICTIONARY_MAX + 1 });
+
+    actualDataColumnPage.setStatsCollector(encodedColumnPage.getActualPage().statsCollector);
+
+    // get the actual data for each dictionary data and put the actual data in new page
+    int rowId = 0;
+    for (int i = 0; i < pageSize; i++) {
+      int index = reverseInvertedIndex[i] * 3;
+      int keyArray = (int) keyGenerator.getKeyArray(bytes, index)[0];
+      actualDataColumnPage
+          .putBytes(rowId++, localDictionaryGenerator.getDictionaryKeyBasedOnValue(keyArray));
+    }
+
+    FallbackEncodedColumnPage fallBackEncodedColumnPage =
+        CarbonUtil.getFallBackEncodedColumnPage(actualDataColumnPage, pageIndex, columnSpec);
+    // here freeing the memory of new column page created as fallback is done and
+    // fallBackEncodedColumnPage is created using new page of actual data
+    // This is required to free the memory once it is of no use
+    actualDataColumnPage.freeMemory();
+    return fallBackEncodedColumnPage;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/2ccdbb78/core/src/main/java/org/apache/carbondata/core/datastore/page/FallbackColumnPageEncoder.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/datastore/page/FallbackColumnPageEncoder.java b/core/src/main/java/org/apache/carbondata/core/datastore/page/FallbackColumnPageEncoder.java
deleted file mode 100644
index e16eb93..0000000
--- a/core/src/main/java/org/apache/carbondata/core/datastore/page/FallbackColumnPageEncoder.java
+++ /dev/null
@@ -1,86 +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;
-
-import java.util.concurrent.Callable;
-
-import org.apache.carbondata.core.datastore.TableSpec;
-import org.apache.carbondata.core.datastore.page.encoding.ColumnPageEncoder;
-import org.apache.carbondata.core.datastore.page.encoding.DefaultEncodingFactory;
-import org.apache.carbondata.core.datastore.page.encoding.EncodedColumnPage;
-
-/**
- * Below class will be used to encode column pages for which local dictionary was generated
- * but all the pages in blocklet was not encoded with local dictionary.
- * This is required as all the pages of a column in blocklet either it will be local dictionary
- * encoded or without local dictionary encoded.
- */
-public class FallbackColumnPageEncoder implements Callable<FallbackEncodedColumnPage> {
-
-  /**
-   * actual local dictionary generated column page
-   */
-  private EncodedColumnPage encodedColumnPage;
-
-  /**
-   * actual index in the page
-   * this is required as in a blocklet few pages will be local dictionary
-   * encoded and few pages will be plain text encoding
-   * in this case local dictionary encoded page
-   */
-  private int pageIndex;
-
-  public FallbackColumnPageEncoder(EncodedColumnPage encodedColumnPage, int pageIndex) {
-    this.encodedColumnPage = encodedColumnPage;
-    this.pageIndex = pageIndex;
-  }
-
-  @Override public FallbackEncodedColumnPage call() throws Exception {
-    // disable encoding using local dictionary
-    encodedColumnPage.getActualPage().disableLocalDictEncoding();
-    // new encoded column page
-    EncodedColumnPage newEncodedColumnPage;
-
-    // get column spec for existing column page
-    TableSpec.ColumnSpec columnSpec = encodedColumnPage.getActualPage().getColumnSpec();
-    switch (columnSpec.getColumnType()) {
-      case COMPLEX_ARRAY:
-      case COMPLEX_STRUCT:
-      case COMPLEX:
-        throw new RuntimeException("Unsupported DataType. Only COMPLEX_PRIMITIVE should come");
-
-      case COMPLEX_PRIMITIVE:
-        // for complex type column
-        newEncodedColumnPage = ColumnPageEncoder.encodedColumn(
-            encodedColumnPage.getActualPage());
-        break;
-      default:
-        // for primitive column
-        ColumnPageEncoder columnPageEncoder = DefaultEncodingFactory.getInstance()
-            .createEncoder(encodedColumnPage.getActualPage().getColumnSpec(),
-                encodedColumnPage.getActualPage());
-        newEncodedColumnPage = columnPageEncoder.encode(encodedColumnPage.getActualPage());
-    }
-    FallbackEncodedColumnPage fallbackEncodedColumnPage =
-        new FallbackEncodedColumnPage(newEncodedColumnPage, pageIndex);
-    // here freeing the memory of raw column page as fallback is done and column page will not
-    // be used.
-    // This is required to free the memory once it is of no use
-    encodedColumnPage.freeMemory();
-    return fallbackEncodedColumnPage;
-  }
-}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/2ccdbb78/core/src/main/java/org/apache/carbondata/core/datastore/page/LocalDictColumnPage.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/datastore/page/LocalDictColumnPage.java b/core/src/main/java/org/apache/carbondata/core/datastore/page/LocalDictColumnPage.java
index 94e56b8..904d7ef 100644
--- a/core/src/main/java/org/apache/carbondata/core/datastore/page/LocalDictColumnPage.java
+++ b/core/src/main/java/org/apache/carbondata/core/datastore/page/LocalDictColumnPage.java
@@ -67,11 +67,15 @@ public class LocalDictColumnPage extends ColumnPage {
   private KeyGenerator keyGenerator;
 
   private int[] dummyKey;
+
+  private boolean isDecoderBasedFallBackEnabled;
+
   /**
    * Create a new column page with input data type and page size.
    */
   protected LocalDictColumnPage(ColumnPage actualDataColumnPage, ColumnPage encodedColumnpage,
-      LocalDictionaryGenerator localDictionaryGenerator, boolean isComplexTypePrimitive) {
+      LocalDictionaryGenerator localDictionaryGenerator, boolean isComplexTypePrimitive,
+      boolean isDecoderBasedFallBackEnabled) {
     super(actualDataColumnPage.getColumnSpec(), actualDataColumnPage.getDataType(),
         actualDataColumnPage.getPageSize());
     // if threshold is not reached then create page level dictionary
@@ -88,6 +92,7 @@ public class LocalDictColumnPage extends ColumnPage {
       // else free the encoded column page memory as its of no use
       encodedColumnpage.freeMemory();
     }
+    this.isDecoderBasedFallBackEnabled = isDecoderBasedFallBackEnabled;
     this.actualDataColumnPage = actualDataColumnPage;
   }
 
@@ -179,7 +184,16 @@ public class LocalDictColumnPage extends ColumnPage {
   }
 
   @Override public void freeMemory() {
-    if (null == pageLevelDictionary) {
+    // free the encoded column page as data is already encoded and it is of no use, during fallback
+    // if goes to actual databased fallback, we need actual data and decoder based fallback we need
+    // just the encoded data to form a new page
+    if (null != encodedDataColumnPage) {
+      encodedDataColumnPage.freeMemory();
+    }
+    if (isDecoderBasedFallBackEnabled) {
+      actualDataColumnPage.freeMemory();
+      isActualPageMemoryFreed = true;
+    } else if (null == pageLevelDictionary) {
       actualDataColumnPage.freeMemory();
       isActualPageMemoryFreed = true;
     }
@@ -190,7 +204,6 @@ public class LocalDictColumnPage extends ColumnPage {
       actualDataColumnPage.freeMemory();
       isActualPageMemoryFreed = true;
     }
-    freeEncodedColumnPage();
   }
 
   private void freeEncodedColumnPage() {

http://git-wip-us.apache.org/repos/asf/carbondata/blob/2ccdbb78/core/src/main/java/org/apache/carbondata/core/util/CarbonUtil.java
----------------------------------------------------------------------
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 c247690..c9601c0 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
@@ -31,6 +31,7 @@ import org.apache.carbondata.core.cache.dictionary.DictionaryColumnUniqueIdentif
 import org.apache.carbondata.core.constants.CarbonCommonConstants;
 import org.apache.carbondata.core.datamap.Segment;
 import org.apache.carbondata.core.datastore.FileReader;
+import org.apache.carbondata.core.datastore.TableSpec;
 import org.apache.carbondata.core.datastore.block.AbstractIndex;
 import org.apache.carbondata.core.datastore.block.TableBlockInfo;
 import org.apache.carbondata.core.datastore.chunk.DimensionColumnPage;
@@ -40,6 +41,11 @@ import org.apache.carbondata.core.datastore.columnar.UnBlockIndexer;
 import org.apache.carbondata.core.datastore.exception.CarbonDataWriterException;
 import org.apache.carbondata.core.datastore.filesystem.CarbonFile;
 import org.apache.carbondata.core.datastore.impl.FileFactory;
+import org.apache.carbondata.core.datastore.page.ColumnPage;
+import org.apache.carbondata.core.datastore.page.FallbackEncodedColumnPage;
+import org.apache.carbondata.core.datastore.page.encoding.ColumnPageEncoder;
+import org.apache.carbondata.core.datastore.page.encoding.DefaultEncodingFactory;
+import org.apache.carbondata.core.datastore.page.encoding.EncodedColumnPage;
 import org.apache.carbondata.core.exception.InvalidConfigurationException;
 import org.apache.carbondata.core.indexstore.BlockletDetailInfo;
 import org.apache.carbondata.core.indexstore.blockletindex.SegmentIndexFileStore;
@@ -47,6 +53,7 @@ import org.apache.carbondata.core.keygenerator.mdkey.NumberCompressor;
 import org.apache.carbondata.core.localdictionary.generator.ColumnLocalDictionaryGenerator;
 import org.apache.carbondata.core.localdictionary.generator.LocalDictionaryGenerator;
 import org.apache.carbondata.core.locks.ICarbonLock;
+import org.apache.carbondata.core.memory.MemoryException;
 import org.apache.carbondata.core.metadata.AbsoluteTableIdentifier;
 import org.apache.carbondata.core.metadata.ColumnarFormatVersion;
 import org.apache.carbondata.core.metadata.SegmentFileStore;
@@ -3273,4 +3280,70 @@ public final class CarbonUtil {
   public static boolean isStandardCarbonTable(CarbonTable table) {
     return !(table.isSupportFlatFolder() || table.isHivePartitionTable());
   }
+
+
+  /**
+   * This method will form the FallbackEncodedColumnPage from input column page
+   * @param columnPage actual data column page got from encoded columnpage if decoder based fallback
+   * is disabled or newly created columnpage by extracting actual data from dictionary data, if
+   * decoder based fallback is enabled
+   * @param pageIndex pageIndex
+   * @param columnSpec ColumSpec
+   * @return FallbackEncodedColumnPage
+   * @throws IOException
+   * @throws MemoryException
+   */
+  public static FallbackEncodedColumnPage getFallBackEncodedColumnPage(ColumnPage columnPage,
+      int pageIndex, TableSpec.ColumnSpec columnSpec) throws IOException, MemoryException {
+    // new encoded column page
+    EncodedColumnPage newEncodedColumnPage;
+
+    switch (columnSpec.getColumnType()) {
+      case COMPLEX_ARRAY:
+      case COMPLEX_STRUCT:
+      case COMPLEX:
+        throw new RuntimeException("Unsupported DataType. Only COMPLEX_PRIMITIVE should come");
+
+      case COMPLEX_PRIMITIVE:
+        // for complex type column
+        newEncodedColumnPage = ColumnPageEncoder.encodedColumn(columnPage);
+        break;
+      default:
+        // for primitive column
+        ColumnPageEncoder columnPageEncoder =
+            DefaultEncodingFactory.getInstance().createEncoder(columnSpec, columnPage);
+        newEncodedColumnPage = columnPageEncoder.encode(columnPage);
+    }
+    FallbackEncodedColumnPage fallbackEncodedColumnPage =
+        new FallbackEncodedColumnPage(newEncodedColumnPage, pageIndex);
+    return fallbackEncodedColumnPage;
+  }
+
+  /**
+   * Below method will be used to check whether particular encoding is present
+   * in the dimension or not
+   *
+   * @param encoding encoding to search
+   * @return if encoding is present in dimension
+   */
+  public static boolean hasEncoding(List<org.apache.carbondata.format.Encoding> encodings,
+      org.apache.carbondata.format.Encoding encoding) {
+    return encodings.contains(encoding);
+  }
+
+  /**
+   * Below method will be used to create the inverted index reverse
+   * this will be used to point to actual data in the chunk
+   *
+   * @param invertedIndex inverted index
+   * @return reverse inverted index
+   */
+  public static int[] getInvertedReverseIndex(int[] invertedIndex) {
+    int[] columnIndexTemp = new int[invertedIndex.length];
+
+    for (int i = 0; i < invertedIndex.length; i++) {
+      columnIndexTemp[invertedIndex[i]] = i;
+    }
+    return columnIndexTemp;
+  }
 }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/2ccdbb78/integration/spark-datasource/src/main/scala/org/apache/carbondata/spark/vectorreader/VectorizedCarbonRecordReader.java
----------------------------------------------------------------------
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 d6b6b7e..fa37205 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
@@ -240,7 +240,8 @@ public class VectorizedCarbonRecordReader extends AbstractRecordReader<Object> {
             CarbonSparkDataSourceUtil.convertCarbonToSparkDataType(generator.getReturnType()), true, null);
       } else if (!dim.getDimension().hasEncoding(Encoding.DICTIONARY)) {
         if (dim.getDimension().getDataType() == DataTypes.STRING
-            || dim.getDimension().getDataType() == DataTypes.VARCHAR) {
+            || dim.getDimension().getDataType() == DataTypes.VARCHAR || dim.getDimension()
+            .getColumnSchema().isLocalDictColumn()) {
           this.isNoDictStringField[dim.getOrdinal()] = true;
         }
         fields[dim.getOrdinal()] = new StructField(dim.getColumnName(),

http://git-wip-us.apache.org/repos/asf/carbondata/blob/2ccdbb78/processing/src/main/java/org/apache/carbondata/processing/store/writer/v3/BlockletDataHolder.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/store/writer/v3/BlockletDataHolder.java b/processing/src/main/java/org/apache/carbondata/processing/store/writer/v3/BlockletDataHolder.java
index 7607cf0..6cbdd9c 100644
--- a/processing/src/main/java/org/apache/carbondata/processing/store/writer/v3/BlockletDataHolder.java
+++ b/processing/src/main/java/org/apache/carbondata/processing/store/writer/v3/BlockletDataHolder.java
@@ -18,8 +18,11 @@ package org.apache.carbondata.processing.store.writer.v3;
 
 import java.util.concurrent.ExecutorService;
 
+import org.apache.carbondata.core.constants.CarbonCommonConstants;
 import org.apache.carbondata.core.datastore.blocklet.EncodedBlocklet;
 import org.apache.carbondata.core.datastore.page.EncodedTablePage;
+import org.apache.carbondata.core.util.CarbonProperties;
+import org.apache.carbondata.processing.store.CarbonFactDataHandlerModel;
 import org.apache.carbondata.processing.store.TablePage;
 
 public class BlockletDataHolder {
@@ -31,8 +34,12 @@ public class BlockletDataHolder {
 
   private EncodedBlocklet encodedBlocklet;
 
-  public BlockletDataHolder(ExecutorService fallbackpool) {
-    encodedBlocklet = new EncodedBlocklet(fallbackpool);
+  public BlockletDataHolder(ExecutorService fallbackpool, CarbonFactDataHandlerModel model) {
+    encodedBlocklet = new EncodedBlocklet(fallbackpool, Boolean.parseBoolean(
+        CarbonProperties.getInstance()
+            .getProperty(CarbonCommonConstants.LOCAL_DICTIONARY_DECODER_BASED_FALLBACK,
+                CarbonCommonConstants.LOCAL_DICTIONARY_DECODER_BASED_FALLBACK_DEFAULT)),
+        model.getColumnLocalDictGenMap());
   }
 
   public void clear() {

http://git-wip-us.apache.org/repos/asf/carbondata/blob/2ccdbb78/processing/src/main/java/org/apache/carbondata/processing/store/writer/v3/CarbonFactDataWriterImplV3.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/store/writer/v3/CarbonFactDataWriterImplV3.java b/processing/src/main/java/org/apache/carbondata/processing/store/writer/v3/CarbonFactDataWriterImplV3.java
index 4fabbe4..8622fcd 100644
--- a/processing/src/main/java/org/apache/carbondata/processing/store/writer/v3/CarbonFactDataWriterImplV3.java
+++ b/processing/src/main/java/org/apache/carbondata/processing/store/writer/v3/CarbonFactDataWriterImplV3.java
@@ -82,7 +82,7 @@ public class CarbonFactDataWriterImplV3 extends AbstractFactDataWriter {
       blockletSizeThreshold = fileSizeInBytes;
       LOGGER.info("Blocklet size configure for table is: " + blockletSizeThreshold);
     }
-    blockletDataHolder = new BlockletDataHolder(fallbackExecutorService);
+    blockletDataHolder = new BlockletDataHolder(fallbackExecutorService, model);
   }
 
   @Override protected void writeBlockletInfoToFile()