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

[5/6] incubator-carbondata git commit: change package name

change package name

add ReadCompressModel

change to mantissa

change to mantissa

remove compType

change compression type

fix style

fix testcase

fix testcase

fix


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

Branch: refs/heads/master
Commit: 360edc8ddea8d7d79c7d9ac973aa7f6869a00359
Parents: b7f3be7
Author: jackylk <ja...@huawei.com>
Authored: Tue Dec 6 13:13:46 2016 +0800
Committer: chenliang613 <ch...@apache.org>
Committed: Tue Dec 6 23:47:38 2016 +0800

----------------------------------------------------------------------
 ...CompressedMeasureChunkFileBasedReaderV1.java |  27 +-
 ...CompressedMeasureChunkFileBasedReaderV2.java |  10 +-
 .../core/compression/ValueCompressor.java       |  25 +-
 .../store/compression/MeasureMetaDataModel.java | 105 +---
 .../store/compression/ReaderCompressModel.java  |  61 +++
 .../compression/ValueCompressionModel.java      | 236 ---------
 .../compression/ValueCompressonHolder.java      |  17 +-
 .../store/compression/WriterCompressModel.java  | 221 ++++++++
 .../decimal/UnCompressByteArray.java            | 137 +++++
 .../decimal/UnCompressMaxMinByte.java           | 135 +++++
 .../decimal/UnCompressMaxMinDefault.java        | 111 ++++
 .../decimal/UnCompressMaxMinFloat.java          | 102 ++++
 .../decimal/UnCompressMaxMinInt.java            | 127 +++++
 .../decimal/UnCompressMaxMinLong.java           | 126 +++++
 .../decimal/UnCompressMaxMinShort.java          | 126 +++++
 .../nondecimal/UnCompressNonDecimalByte.java    |  96 ++++
 .../nondecimal/UnCompressNonDecimalDefault.java |  96 ++++
 .../nondecimal/UnCompressNonDecimalFloat.java   | 100 ++++
 .../nondecimal/UnCompressNonDecimalInt.java     |  94 ++++
 .../nondecimal/UnCompressNonDecimalLong.java    |  96 ++++
 .../UnCompressNonDecimalMaxMinByte.java         | 111 ++++
 .../UnCompressNonDecimalMaxMinDefault.java      | 105 ++++
 .../UnCompressNonDecimalMaxMinFloat.java        | 110 ++++
 .../UnCompressNonDecimalMaxMinInt.java          | 108 ++++
 .../UnCompressNonDecimalMaxMinLong.java         | 112 ++++
 .../UnCompressNonDecimalMaxMinShort.java        | 110 ++++
 .../nondecimal/UnCompressNonDecimalShort.java   |  98 ++++
 .../compression/none/UnCompressNoneByte.java    | 128 +++++
 .../compression/none/UnCompressNoneDefault.java | 100 ++++
 .../compression/none/UnCompressNoneFloat.java   | 105 ++++
 .../compression/none/UnCompressNoneInt.java     | 143 ++++++
 .../compression/none/UnCompressNoneLong.java    | 129 +++++
 .../compression/none/UnCompressNoneShort.java   | 127 +++++
 .../compression/type/UnCompressByteArray.java   | 137 -----
 .../compression/type/UnCompressDefaultLong.java |  52 --
 .../compression/type/UnCompressMaxMinByte.java  | 138 -----
 .../type/UnCompressMaxMinByteForLong.java       |  80 ---
 .../type/UnCompressMaxMinDefault.java           | 116 -----
 .../type/UnCompressMaxMinDefaultLong.java       |  79 ---
 .../compression/type/UnCompressMaxMinFloat.java | 108 ----
 .../compression/type/UnCompressMaxMinInt.java   | 137 -----
 .../compression/type/UnCompressMaxMinLong.java  | 137 -----
 .../compression/type/UnCompressMaxMinShort.java | 136 -----
 .../type/UnCompressNonDecimalByte.java          |  96 ----
 .../type/UnCompressNonDecimalDefault.java       |  96 ----
 .../type/UnCompressNonDecimalFloat.java         | 100 ----
 .../type/UnCompressNonDecimalInt.java           |  95 ----
 .../type/UnCompressNonDecimalLong.java          |  96 ----
 .../type/UnCompressNonDecimalMaxMinByte.java    | 111 ----
 .../type/UnCompressNonDecimalMaxMinDefault.java | 105 ----
 .../type/UnCompressNonDecimalMaxMinFloat.java   | 110 ----
 .../type/UnCompressNonDecimalMaxMinInt.java     | 108 ----
 .../type/UnCompressNonDecimalMaxMinLong.java    | 112 ----
 .../type/UnCompressNonDecimalMaxMinShort.java   | 110 ----
 .../type/UnCompressNonDecimalShort.java         |  98 ----
 .../compression/type/UnCompressNoneByte.java    | 132 -----
 .../compression/type/UnCompressNoneDefault.java | 100 ----
 .../compression/type/UnCompressNoneFloat.java   | 105 ----
 .../compression/type/UnCompressNoneInt.java     | 145 ------
 .../compression/type/UnCompressNoneLong.java    | 129 -----
 .../compression/type/UnCompressNoneShort.java   | 127 -----
 ...ractHeavyCompressedDoubleArrayDataStore.java |   8 +-
 ...yCompressedDoubleArrayDataInMemoryStore.java |  21 +-
 .../AbstractDoubleArrayDataStore.java           |  82 ---
 .../DoubleArrayDataInMemoryStore.java           | 158 ------
 .../core/datastorage/util/StoreFactory.java     |  33 +-
 .../core/metadata/BlockletInfoColumnar.java     |   8 +-
 .../core/metadata/ValueEncoderMeta.java         |  20 +-
 .../core/util/CarbonMetadataUtil.java           |  12 +-
 .../apache/carbondata/core/util/CarbonUtil.java |  10 +-
 .../core/util/ValueCompressionUtil.java         | 511 ++++++++-----------
 ...mpressedMeasureChunkFileBasedReaderTest.java |  16 +-
 .../core/util/CarbonMetadataUtilTest.java       |  31 +-
 .../carbondata/core/util/CarbonUtilTest.java    |   7 +-
 .../core/util/ValueCompressionUtilTest.java     | 206 ++++----
 .../core/writer/CarbonFooterWriterTest.java     |   6 +-
 format/src/main/thrift/schema.thrift            |   2 +-
 .../execution/command/carbonTableSchema.scala   |   2 +-
 .../csvreaderstep/step-attributes.xml           |   2 +-
 .../store/CarbonFactDataHandlerColumnar.java    |  14 +-
 .../store/writer/CarbonFactDataWriter.java      |   4 +-
 .../processing/store/writer/NodeHolder.java     |   8 +-
 .../writer/v1/CarbonFactDataWriterImplV1.java   |   4 +-
 83 files changed, 3553 insertions(+), 4241 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/360edc8d/core/src/main/java/org/apache/carbondata/core/carbon/datastore/chunk/reader/measure/v1/CompressedMeasureChunkFileBasedReaderV1.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/carbon/datastore/chunk/reader/measure/v1/CompressedMeasureChunkFileBasedReaderV1.java b/core/src/main/java/org/apache/carbondata/core/carbon/datastore/chunk/reader/measure/v1/CompressedMeasureChunkFileBasedReaderV1.java
index c1204f2..1455030 100644
--- a/core/src/main/java/org/apache/carbondata/core/carbon/datastore/chunk/reader/measure/v1/CompressedMeasureChunkFileBasedReaderV1.java
+++ b/core/src/main/java/org/apache/carbondata/core/carbon/datastore/chunk/reader/measure/v1/CompressedMeasureChunkFileBasedReaderV1.java
@@ -25,10 +25,13 @@ import org.apache.carbondata.core.carbon.datastore.chunk.reader.measure.Abstract
 import org.apache.carbondata.core.carbon.metadata.blocklet.BlockletInfo;
 import org.apache.carbondata.core.carbon.metadata.blocklet.datachunk.DataChunk;
 import org.apache.carbondata.core.datastorage.store.FileHolder;
-import org.apache.carbondata.core.datastorage.store.compression.ValueCompressionModel;
+import org.apache.carbondata.core.datastorage.store.compression.ReaderCompressModel;
 import org.apache.carbondata.core.datastorage.store.compression.ValueCompressonHolder.UnCompressValue;
+
+
 import org.apache.carbondata.core.datastorage.store.dataholder.CarbonReadDataHolder;
-import org.apache.carbondata.core.util.CarbonUtil;
+import org.apache.carbondata.core.metadata.ValueEncoderMeta;
+import org.apache.carbondata.core.util.ValueCompressionUtil;
 
 /**
  * Compressed measure chunk reader
@@ -79,23 +82,19 @@ public class CompressedMeasureChunkFileBasedReaderV1 extends AbstractMeasureChun
    */
   @Override public MeasureColumnDataChunk readMeasureChunk(final FileHolder fileReader,
       final int blockIndex) {
-    MeasureColumnDataChunk datChunk = new MeasureColumnDataChunk();
-    // create a new uncompressor
-    final ValueCompressionModel compressionModel = CarbonUtil
-        .getValueCompressionModel(measureColumnChunks.get(blockIndex).getValueEncoderMeta());
-    UnCompressValue values =
-        compressionModel.getUnCompressValues()[0].getNew().getCompressorObject();
-    // create a new uncompressor
-    // read data from file and set to uncompressor
-    // read data from file and set to uncompressor
+    ValueEncoderMeta meta = measureColumnChunks.get(blockIndex).getValueEncoderMeta().get(0);
+    ReaderCompressModel compressModel = ValueCompressionUtil.getReaderCompressModel(meta);
+    UnCompressValue values = compressModel.getUnCompressValues().getNew().getCompressorObject();
     values.setValue(
         fileReader.readByteArray(filePath, measureColumnChunks.get(blockIndex).getDataPageOffset(),
             measureColumnChunks.get(blockIndex).getDataPageLength()));
     // get the data holder after uncompressing
     CarbonReadDataHolder measureDataHolder =
-        values.uncompress(compressionModel.getChangedDataType()[0])
-            .getValues(compressionModel.getDecimal()[0], compressionModel.getMaxValue()[0]);
-    // set the data chunk
+        values.uncompress(compressModel.getChangedDataType())
+            .getValues(compressModel.getMantissa(), compressModel.getMaxValue());
+
+    // create and set the data chunk
+    MeasureColumnDataChunk datChunk = new MeasureColumnDataChunk();
     datChunk.setMeasureDataHolder(measureDataHolder);
     // set the enun value indexes
     datChunk

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/360edc8d/core/src/main/java/org/apache/carbondata/core/carbon/datastore/chunk/reader/measure/v2/CompressedMeasureChunkFileBasedReaderV2.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/carbon/datastore/chunk/reader/measure/v2/CompressedMeasureChunkFileBasedReaderV2.java b/core/src/main/java/org/apache/carbondata/core/carbon/datastore/chunk/reader/measure/v2/CompressedMeasureChunkFileBasedReaderV2.java
index d053005..9cba57f 100644
--- a/core/src/main/java/org/apache/carbondata/core/carbon/datastore/chunk/reader/measure/v2/CompressedMeasureChunkFileBasedReaderV2.java
+++ b/core/src/main/java/org/apache/carbondata/core/carbon/datastore/chunk/reader/measure/v2/CompressedMeasureChunkFileBasedReaderV2.java
@@ -28,8 +28,8 @@ import org.apache.carbondata.core.carbon.metadata.blocklet.BlockletInfo;
 import org.apache.carbondata.core.carbon.metadata.blocklet.datachunk.PresenceMeta;
 import org.apache.carbondata.core.datastorage.store.FileHolder;
 import org.apache.carbondata.core.datastorage.store.compression.CompressorFactory;
-import org.apache.carbondata.core.datastorage.store.compression.ValueCompressionModel;
 import org.apache.carbondata.core.datastorage.store.compression.ValueCompressonHolder.UnCompressValue;
+import org.apache.carbondata.core.datastorage.store.compression.WriterCompressModel;
 import org.apache.carbondata.core.datastorage.store.dataholder.CarbonReadDataHolder;
 import org.apache.carbondata.core.metadata.ValueEncoderMeta;
 import org.apache.carbondata.core.util.CarbonUtil;
@@ -160,7 +160,7 @@ public class CompressedMeasureChunkFileBasedReaderV2 extends AbstractMeasureChun
       valueEncodeMeta.add(
           CarbonUtil.deserializeEncoderMeta(measureColumnChunk.getEncoder_meta().get(i).array()));
     }
-    ValueCompressionModel compressionModel = CarbonUtil.getValueCompressionModel(valueEncodeMeta);
+    WriterCompressModel compressionModel = CarbonUtil.getValueCompressionModel(valueEncodeMeta);
     UnCompressValue values =
         compressionModel.getUnCompressValues()[0].getNew().getCompressorObject();
     // create a new uncompressor
@@ -169,7 +169,7 @@ public class CompressedMeasureChunkFileBasedReaderV2 extends AbstractMeasureChun
     // get the data holder after uncompressing
     CarbonReadDataHolder measureDataHolder =
         values.uncompress(compressionModel.getChangedDataType()[0])
-            .getValues(compressionModel.getDecimal()[0], compressionModel.getMaxValue()[0]);
+            .getValues(compressionModel.getMantissa()[0], compressionModel.getMaxValue()[0]);
     // set the data chunk
     datChunk.setMeasureDataHolder(measureDataHolder);
     // set the enun value indexes
@@ -214,7 +214,7 @@ public class CompressedMeasureChunkFileBasedReaderV2 extends AbstractMeasureChun
         valueEncodeMeta.add(
             CarbonUtil.deserializeEncoderMeta(measureColumnChunk.getEncoder_meta().get(j).array()));
       }
-      ValueCompressionModel compressionModel = CarbonUtil.getValueCompressionModel(valueEncodeMeta);
+      WriterCompressModel compressionModel = CarbonUtil.getValueCompressionModel(valueEncodeMeta);
       UnCompressValue values =
           compressionModel.getUnCompressValues()[0].getNew().getCompressorObject();
       // create a new uncompressor
@@ -223,7 +223,7 @@ public class CompressedMeasureChunkFileBasedReaderV2 extends AbstractMeasureChun
       // get the data holder after uncompressing
       CarbonReadDataHolder measureDataHolder =
           values.uncompress(compressionModel.getChangedDataType()[0])
-              .getValues(compressionModel.getDecimal()[0], compressionModel.getMaxValue()[0]);
+              .getValues(compressionModel.getMantissa()[0], compressionModel.getMaxValue()[0]);
       // set the data chunk
       dataChunk.setMeasureDataHolder(measureDataHolder);
       // set the enun value indexes

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/360edc8d/core/src/main/java/org/apache/carbondata/core/compression/ValueCompressor.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/compression/ValueCompressor.java b/core/src/main/java/org/apache/carbondata/core/compression/ValueCompressor.java
index 5ecb90e..370c8d5 100644
--- a/core/src/main/java/org/apache/carbondata/core/compression/ValueCompressor.java
+++ b/core/src/main/java/org/apache/carbondata/core/compression/ValueCompressor.java
@@ -38,26 +38,15 @@ public abstract class ValueCompressor {
    */
   public Object getCompressedValues(COMPRESSION_TYPE compType, CarbonWriteDataHolder dataHolder,
       DataType changedDataType, Object maxValue, int decimal) {
-    Object o;
     switch (compType) {
-      case NONE:
-
-        o = compressNone(changedDataType, dataHolder);
-        return o;
-
-      case MAX_MIN:
-
-        o = compressMaxMin(changedDataType, dataHolder, maxValue);
-        return o;
-
-      case NON_DECIMAL_CONVERT:
-
-        o = compressNonDecimal(changedDataType, dataHolder, decimal);
-        return o;
-
+      case ADAPTIVE:
+        return compressNone(changedDataType, dataHolder);
+      case DELTA_DOUBLE:
+        return compressMaxMin(changedDataType, dataHolder, maxValue);
+      case BIGINT:
+        return compressNonDecimal(changedDataType, dataHolder, decimal);
       default:
-        o = compressNonDecimalMaxMin(changedDataType, dataHolder, decimal, maxValue);
-        return o;
+        return compressNonDecimalMaxMin(changedDataType, dataHolder, decimal, maxValue);
     }
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/360edc8d/core/src/main/java/org/apache/carbondata/core/datastorage/store/compression/MeasureMetaDataModel.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/datastorage/store/compression/MeasureMetaDataModel.java b/core/src/main/java/org/apache/carbondata/core/datastorage/store/compression/MeasureMetaDataModel.java
index 7d212bb..f207478 100644
--- a/core/src/main/java/org/apache/carbondata/core/datastorage/store/compression/MeasureMetaDataModel.java
+++ b/core/src/main/java/org/apache/carbondata/core/datastorage/store/compression/MeasureMetaDataModel.java
@@ -31,9 +31,9 @@ public class MeasureMetaDataModel {
   private Object[] minValue;
 
   /**
-   * decimal
+   * mantissa
    */
-  private int[] decimal;
+  private int[] mantissa;
 
   /**
    * measureCount
@@ -55,27 +55,11 @@ public class MeasureMetaDataModel {
    */
   private byte[] dataTypeSelected;
 
-  private Object[] minValueFactForAgg;
-
-  public MeasureMetaDataModel() {
-
-  }
-
-  /**
-   * MeasureMetaDataModel Constructor
-   *
-   * @param minValue
-   * @param maxValue
-   * @param decimal
-   * @param measureCount
-   * @param uniqueValue
-   * @param type
-   */
-  public MeasureMetaDataModel(Object[] minValue, Object[] maxValue, int[] decimal, int measureCount,
-      Object[] uniqueValue, char[] type, byte[] dataTypeSelected) {
+  public MeasureMetaDataModel(Object[] minValue, Object[] maxValue, int[] mantissa,
+      int measureCount, Object[] uniqueValue, char[] type, byte[] dataTypeSelected) {
     this.minValue = minValue;
     this.maxValue = maxValue;
-    this.decimal = decimal;
+    this.mantissa = mantissa;
     this.measureCount = measureCount;
     this.uniqueValue = uniqueValue;
     this.type = type;
@@ -92,15 +76,6 @@ public class MeasureMetaDataModel {
   }
 
   /**
-   * set max value
-   *
-   * @param maxValue
-   */
-  public void setMaxValue(Object[] maxValue) {
-    this.maxValue = maxValue;
-  }
-
-  /**
    * getMinValue
    *
    * @return
@@ -110,30 +85,12 @@ public class MeasureMetaDataModel {
   }
 
   /**
-   * setMinValue
-   *
-   * @param minValue
-   */
-  public void setMinValue(Object[] minValue) {
-    this.minValue = minValue;
-  }
-
-  /**
-   * getDecimal
+   * getMantissa
    *
    * @return
    */
-  public int[] getDecimal() {
-    return decimal;
-  }
-
-  /**
-   * setDecimal
-   *
-   * @param decimal
-   */
-  public void setDecimal(int[] decimal) {
-    this.decimal = decimal;
+  public int[] getMantissa() {
+    return mantissa;
   }
 
   /**
@@ -146,15 +103,6 @@ public class MeasureMetaDataModel {
   }
 
   /**
-   * setMeasureCount
-   *
-   * @param measureCount
-   */
-  public void setMeasureCount(int measureCount) {
-    this.measureCount = measureCount;
-  }
-
-  /**
    * getUniqueValue
    *
    * @return
@@ -164,15 +112,6 @@ public class MeasureMetaDataModel {
   }
 
   /**
-   * setUniqueValue
-   *
-   * @param uniqueValue
-   */
-  public void setUniqueValue(Object[] uniqueValue) {
-    this.uniqueValue = uniqueValue;
-  }
-
-  /**
    * @return the type
    */
   public char[] getType() {
@@ -180,38 +119,10 @@ public class MeasureMetaDataModel {
   }
 
   /**
-   * @param type the type to set
-   */
-  public void setType(char[] type) {
-    this.type = type;
-  }
-
-  /**
    * @return the dataTypeSelected
    */
   public byte[] getDataTypeSelected() {
     return dataTypeSelected;
   }
 
-  /**
-   * @param dataTypeSelected the dataTypeSelected to set
-   */
-  public void setDataTypeSelected(byte[] dataTypeSelected) {
-    this.dataTypeSelected = dataTypeSelected;
-  }
-
-  /**
-   * @return the minValueFactForAgg
-   */
-  public Object[] getMinValueFactForAgg() {
-    return minValueFactForAgg;
-  }
-
-  /**
-   * @param minValueFactForAgg the minValueFactForAgg to set
-   */
-  public void setMinValueFactForAgg(Object[] minValueFactForAgg) {
-    this.minValueFactForAgg = minValueFactForAgg;
-  }
-
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/360edc8d/core/src/main/java/org/apache/carbondata/core/datastorage/store/compression/ReaderCompressModel.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/datastorage/store/compression/ReaderCompressModel.java b/core/src/main/java/org/apache/carbondata/core/datastorage/store/compression/ReaderCompressModel.java
new file mode 100644
index 0000000..ad242f0
--- /dev/null
+++ b/core/src/main/java/org/apache/carbondata/core/datastorage/store/compression/ReaderCompressModel.java
@@ -0,0 +1,61 @@
+/*
+ * 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.datastorage.store.compression;
+
+import org.apache.carbondata.core.metadata.ValueEncoderMeta;
+import org.apache.carbondata.core.util.ValueCompressionUtil;
+
+// Used in read path for decompression preparation
+public class ReaderCompressModel {
+  private ValueEncoderMeta valueEncoderMeta;
+
+  private ValueCompressionUtil.DataType changedDataType;
+
+  private ValueCompressonHolder.UnCompressValue unCompressValues;
+
+  public void setValueEncoderMeta(ValueEncoderMeta valueEncoderMeta) {
+    this.valueEncoderMeta = valueEncoderMeta;
+  }
+
+  public ValueCompressionUtil.DataType getChangedDataType() {
+    return changedDataType;
+  }
+
+  public void setChangedDataType(ValueCompressionUtil.DataType changedDataType) {
+    this.changedDataType = changedDataType;
+  }
+
+  public Object getMaxValue() {
+    return valueEncoderMeta.getMaxValue();
+  }
+
+  public int getMantissa() {
+    return valueEncoderMeta.getMantissa();
+  }
+
+  public ValueCompressonHolder.UnCompressValue getUnCompressValues() {
+    return unCompressValues;
+  }
+
+  public void setUnCompressValues(ValueCompressonHolder.UnCompressValue unCompressValues) {
+    this.unCompressValues = unCompressValues;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/360edc8d/core/src/main/java/org/apache/carbondata/core/datastorage/store/compression/ValueCompressionModel.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/datastorage/store/compression/ValueCompressionModel.java b/core/src/main/java/org/apache/carbondata/core/datastorage/store/compression/ValueCompressionModel.java
deleted file mode 100644
index 94cbf19..0000000
--- a/core/src/main/java/org/apache/carbondata/core/datastorage/store/compression/ValueCompressionModel.java
+++ /dev/null
@@ -1,236 +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.datastorage.store.compression;
-
-import org.apache.carbondata.core.util.ValueCompressionUtil;
-
-public class ValueCompressionModel {
-  /**
-   * COMPRESSION_TYPE[] variable.
-   */
-  private ValueCompressionUtil.COMPRESSION_TYPE[] compType;
-
-  /**
-   * DataType[]  variable.
-   */
-  private ValueCompressionUtil.DataType[] changedDataType;
-  /**
-   * DataType[]  variable.
-   */
-  private ValueCompressionUtil.DataType[] actualDataType;
-
-  /**
-   * maxValue
-   */
-  private Object[] maxValue;
-  /**
-   * minValue.
-   */
-  private Object[] minValue;
-
-  private Object[] minValueFactForAgg;
-
-  /**
-   * uniqueValue
-   */
-  private Object[] uniqueValue;
-  /**
-   * decimal.
-   */
-  private int[] decimal;
-
-  /**
-   * aggType
-   */
-  private char[] type;
-
-  /**
-   * dataTypeSelected
-   */
-  private byte[] dataTypeSelected;
-  /**
-   * unCompressValues.
-   */
-  private ValueCompressonHolder.UnCompressValue[] unCompressValues;
-
-  /**
-   * @return the compType
-   */
-  public ValueCompressionUtil.COMPRESSION_TYPE[] getCompType() {
-    return compType;
-  }
-
-  /**
-   * @param compType the compType to set
-   */
-  public void setCompType(ValueCompressionUtil.COMPRESSION_TYPE[] compType) {
-    this.compType = compType;
-  }
-
-  /**
-   * @return the changedDataType
-   */
-  public ValueCompressionUtil.DataType[] getChangedDataType() {
-    return changedDataType;
-  }
-
-  /**
-   * @param changedDataType the changedDataType to set
-   */
-  public void setChangedDataType(ValueCompressionUtil.DataType[] changedDataType) {
-    this.changedDataType = changedDataType;
-  }
-
-  /**
-   * @return the actualDataType
-   */
-  public ValueCompressionUtil.DataType[] getActualDataType() {
-    return actualDataType;
-  }
-
-  /**
-   * @param actualDataType
-   */
-  public void setActualDataType(ValueCompressionUtil.DataType[] actualDataType) {
-    this.actualDataType = actualDataType;
-  }
-
-  /**
-   * @return the maxValue
-   */
-  public Object[] getMaxValue() {
-    return maxValue;
-  }
-
-  /**
-   * @param maxValue the maxValue to set
-   */
-  public void setMaxValue(Object[] maxValue) {
-    this.maxValue = maxValue;
-  }
-
-  /**
-   * @return the decimal
-   */
-  public int[] getDecimal() {
-    return decimal;
-  }
-
-  /**
-   * @param decimal the decimal to set
-   */
-  public void setDecimal(int[] decimal) {
-    this.decimal = decimal;
-  }
-
-  /**
-   * getUnCompressValues().
-   *
-   * @return the unCompressValues
-   */
-  public ValueCompressonHolder.UnCompressValue[] getUnCompressValues() {
-    return unCompressValues;
-  }
-
-  /**
-   * @param unCompressValues the unCompressValues to set
-   */
-  public void setUnCompressValues(ValueCompressonHolder.UnCompressValue[] unCompressValues) {
-    this.unCompressValues = unCompressValues;
-  }
-
-  /**
-   * getMinValue
-   *
-   * @return
-   */
-  public Object[] getMinValue() {
-    return minValue;
-  }
-
-  /**
-   * setMinValue.
-   *
-   * @param minValue
-   */
-  public void setMinValue(Object[] minValue) {
-    this.minValue = minValue;
-  }
-
-  /**
-   * @return the aggType
-   */
-  public char[] getType() {
-    return type;
-  }
-
-  /**
-   * @param type the type to set
-   */
-  public void setType(char[] type) {
-    this.type = type;
-  }
-
-  /**
-   * @return the dataTypeSelected
-   */
-  public byte[] getDataTypeSelected() {
-    return dataTypeSelected;
-  }
-
-  /**
-   * @param dataTypeSelected the dataTypeSelected to set
-   */
-  public void setDataTypeSelected(byte[] dataTypeSelected) {
-    this.dataTypeSelected = dataTypeSelected;
-  }
-
-  /**
-   * getUniqueValue
-   *
-   * @return
-   */
-  public Object[] getUniqueValue() {
-    return uniqueValue;
-  }
-
-  /**
-   * setUniqueValue
-   *
-   * @param uniqueValue
-   */
-  public void setUniqueValue(Object[] uniqueValue) {
-    this.uniqueValue = uniqueValue;
-  }
-
-  /**
-   * @return the minValueFactForAgg
-   */
-  public Object[] getMinValueFactForAgg() {
-    return minValueFactForAgg;
-  }
-
-  /**
-   * @param minValueFactForAgg the minValueFactForAgg to set
-   */
-  public void setMinValueFactForAgg(Object[] minValueFactForAgg) {
-    this.minValueFactForAgg = minValueFactForAgg;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/360edc8d/core/src/main/java/org/apache/carbondata/core/datastorage/store/compression/ValueCompressonHolder.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/datastorage/store/compression/ValueCompressonHolder.java b/core/src/main/java/org/apache/carbondata/core/datastorage/store/compression/ValueCompressonHolder.java
index 18e15f0..f161b9a 100644
--- a/core/src/main/java/org/apache/carbondata/core/datastorage/store/compression/ValueCompressonHolder.java
+++ b/core/src/main/java/org/apache/carbondata/core/datastorage/store/compression/ValueCompressonHolder.java
@@ -44,46 +44,31 @@ public final class ValueCompressonHolder {
   public static void unCompress(DataType dataType, UnCompressValue value, byte[] data) {
     switch (dataType) {
       case DATA_BYTE:
-
         value.setValue(compressor.unCompressByte(data));
         break;
-
       case DATA_SHORT:
-
         value.setValue(compressor.unCompressShort(data));
         break;
-
       case DATA_INT:
-
         value.setValue(compressor.unCompressInt(data));
         break;
-
       case DATA_LONG:
       case DATA_BIGINT:
-
         value.setValue(compressor.unCompressLong(data));
         break;
-
       case DATA_FLOAT:
-
         value.setValue(compressor.unCompressFloat(data));
         break;
       default:
-
         value.setValue(compressor.unCompressDouble(data));
         break;
-
     }
   }
 
   /**
-   * interface for  UnCompressValue<T>.
-   *
-   * @param <T>
+   * interface for UnCompressValue<T>.
    */
-
   public interface UnCompressValue<T> extends Cloneable {
-    //        Object getValue(int index, int decimal, double maxValue);
 
     void setValue(T value);
 

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/360edc8d/core/src/main/java/org/apache/carbondata/core/datastorage/store/compression/WriterCompressModel.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/datastorage/store/compression/WriterCompressModel.java b/core/src/main/java/org/apache/carbondata/core/datastorage/store/compression/WriterCompressModel.java
new file mode 100644
index 0000000..ac21c8e
--- /dev/null
+++ b/core/src/main/java/org/apache/carbondata/core/datastorage/store/compression/WriterCompressModel.java
@@ -0,0 +1,221 @@
+/*
+ * 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.datastorage.store.compression;
+
+import org.apache.carbondata.core.util.ValueCompressionUtil;
+
+public class WriterCompressModel {
+  /**
+   * COMPRESSION_TYPE[] variable.
+   */
+  private ValueCompressionUtil.COMPRESSION_TYPE[] compType;
+
+  /**
+   * DataType[]  variable.
+   */
+  private ValueCompressionUtil.DataType[] changedDataType;
+  /**
+   * DataType[]  variable.
+   */
+  private ValueCompressionUtil.DataType[] actualDataType;
+
+  /**
+   * maxValue
+   */
+  private Object[] maxValue;
+  /**
+   * minValue.
+   */
+  private Object[] minValue;
+
+  /**
+   * uniqueValue
+   */
+  private Object[] uniqueValue;
+  /**
+   * mantissa.
+   */
+  private int[] mantissa;
+
+  /**
+   * aggType
+   */
+  private char[] type;
+
+  /**
+   * dataTypeSelected
+   */
+  private byte[] dataTypeSelected;
+  /**
+   * unCompressValues.
+   */
+  private ValueCompressonHolder.UnCompressValue[] unCompressValues;
+
+  /**
+   * @return the compType
+   */
+  public ValueCompressionUtil.COMPRESSION_TYPE[] getCompType() {
+    return compType;
+  }
+
+  /**
+   * @param compType the compType to set
+   */
+  public void setCompType(ValueCompressionUtil.COMPRESSION_TYPE[] compType) {
+    this.compType = compType;
+  }
+
+  /**
+   * @return the changedDataType
+   */
+  public ValueCompressionUtil.DataType[] getChangedDataType() {
+    return changedDataType;
+  }
+
+  /**
+   * @param changedDataType the changedDataType to set
+   */
+  public void setChangedDataType(ValueCompressionUtil.DataType[] changedDataType) {
+    this.changedDataType = changedDataType;
+  }
+
+  /**
+   * @return the actualDataType
+   */
+  public ValueCompressionUtil.DataType[] getActualDataType() {
+    return actualDataType;
+  }
+
+  /**
+   * @param actualDataType
+   */
+  public void setActualDataType(ValueCompressionUtil.DataType[] actualDataType) {
+    this.actualDataType = actualDataType;
+  }
+
+  /**
+   * @return the maxValue
+   */
+  public Object[] getMaxValue() {
+    return maxValue;
+  }
+
+  /**
+   * @param maxValue the maxValue to set
+   */
+  public void setMaxValue(Object[] maxValue) {
+    this.maxValue = maxValue;
+  }
+
+  /**
+   * @return the mantissa
+   */
+  public int[] getMantissa() {
+    return mantissa;
+  }
+
+  /**
+   * @param mantissa the mantissa to set
+   */
+  public void setMantissa(int[] mantissa) {
+    this.mantissa = mantissa;
+  }
+
+  /**
+   * getUnCompressValues().
+   *
+   * @return the unCompressValues
+   */
+  public ValueCompressonHolder.UnCompressValue[] getUnCompressValues() {
+    return unCompressValues;
+  }
+
+  /**
+   * @param unCompressValues the unCompressValues to set
+   */
+  public void setUnCompressValues(ValueCompressonHolder.UnCompressValue[] unCompressValues) {
+    this.unCompressValues = unCompressValues;
+  }
+
+  /**
+   * getMinValue
+   *
+   * @return
+   */
+  public Object[] getMinValue() {
+    return minValue;
+  }
+
+  /**
+   * setMinValue.
+   *
+   * @param minValue
+   */
+  public void setMinValue(Object[] minValue) {
+    this.minValue = minValue;
+  }
+
+  /**
+   * @return the aggType
+   */
+  public char[] getType() {
+    return type;
+  }
+
+  /**
+   * @param type the type to set
+   */
+  public void setType(char[] type) {
+    this.type = type;
+  }
+
+  /**
+   * @return the dataTypeSelected
+   */
+  public byte[] getDataTypeSelected() {
+    return dataTypeSelected;
+  }
+
+  /**
+   * @param dataTypeSelected the dataTypeSelected to set
+   */
+  public void setDataTypeSelected(byte[] dataTypeSelected) {
+    this.dataTypeSelected = dataTypeSelected;
+  }
+
+  /**
+   * getUniqueValue
+   *
+   * @return
+   */
+  public Object[] getUniqueValue() {
+    return uniqueValue;
+  }
+
+  /**
+   * setUniqueValue
+   *
+   * @param uniqueValue
+   */
+  public void setUniqueValue(Object[] uniqueValue) {
+    this.uniqueValue = uniqueValue;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/360edc8d/core/src/main/java/org/apache/carbondata/core/datastorage/store/compression/decimal/UnCompressByteArray.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/datastorage/store/compression/decimal/UnCompressByteArray.java b/core/src/main/java/org/apache/carbondata/core/datastorage/store/compression/decimal/UnCompressByteArray.java
new file mode 100644
index 0000000..b8c8248
--- /dev/null
+++ b/core/src/main/java/org/apache/carbondata/core/datastorage/store/compression/decimal/UnCompressByteArray.java
@@ -0,0 +1,137 @@
+/*
+ * 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.datastorage.store.compression.decimal;
+
+import java.math.BigDecimal;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.carbondata.common.logging.LogService;
+import org.apache.carbondata.common.logging.LogServiceFactory;
+import org.apache.carbondata.core.constants.CarbonCommonConstants;
+import org.apache.carbondata.core.datastorage.store.compression.Compressor;
+import org.apache.carbondata.core.datastorage.store.compression.CompressorFactory;
+import org.apache.carbondata.core.datastorage.store.compression.ValueCompressonHolder;
+import org.apache.carbondata.core.datastorage.store.dataholder.CarbonReadDataHolder;
+import org.apache.carbondata.core.util.DataTypeUtil;
+import org.apache.carbondata.core.util.ValueCompressionUtil;
+
+public class UnCompressByteArray implements ValueCompressonHolder.UnCompressValue<byte[]> {
+  /**
+   * Attribute for Carbon LOGGER
+   */
+  private static final LogService LOGGER =
+      LogServiceFactory.getLogService(UnCompressMaxMinByte.class.getName());
+  /**
+   * compressor.
+   */
+  private static Compressor compressor = CompressorFactory.getInstance();
+
+  private ByteArrayType arrayType;
+  /**
+   * value.
+   */
+  private byte[] value;
+
+  public UnCompressByteArray(ByteArrayType type) {
+    if (type == ByteArrayType.BYTE_ARRAY) {
+      arrayType = ByteArrayType.BYTE_ARRAY;
+    } else {
+      arrayType = ByteArrayType.BIG_DECIMAL;
+    }
+
+  }
+
+  @Override public void setValue(byte[] value) {
+    this.value = value;
+
+  }
+
+  @Override public void setValueInBytes(byte[] value) {
+    this.value = value;
+
+  }
+
+  @Override public ValueCompressonHolder.UnCompressValue<byte[]> getNew() {
+    try {
+      return (ValueCompressonHolder.UnCompressValue) clone();
+    } catch (CloneNotSupportedException e) {
+      LOGGER.error(e, e.getMessage());
+    }
+    return null;
+  }
+
+  @Override public ValueCompressonHolder.UnCompressValue compress() {
+    UnCompressByteArray byte1 = new UnCompressByteArray(arrayType);
+    byte1.setValue(compressor.compressByte(value));
+    return byte1;
+  }
+
+  @Override
+  public ValueCompressonHolder.UnCompressValue uncompress(ValueCompressionUtil.DataType dataType) {
+    ValueCompressonHolder.UnCompressValue byte1 = new UnCompressByteArray(arrayType);
+    byte1.setValue(compressor.unCompressByte(value));
+    return byte1;
+  }
+
+  @Override public byte[] getBackArrayData() {
+    return this.value;
+  }
+
+  @Override public ValueCompressonHolder.UnCompressValue getCompressorObject() {
+    return new UnCompressByteArray(arrayType);
+  }
+
+  @Override public CarbonReadDataHolder getValues(int decimal, Object maxValueObject) {
+    List<byte[]> valsList = new ArrayList<byte[]>(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE);
+    ByteBuffer buffer = ByteBuffer.wrap(value);
+    buffer.rewind();
+    int length = 0;
+    byte[] actualValue = null;
+    //CHECKSTYLE:OFF    Approval No:Approval-367
+    while (buffer.hasRemaining()) {//CHECKSTYLE:ON
+      length = buffer.getInt();
+      actualValue = new byte[length];
+      buffer.get(actualValue);
+      valsList.add(actualValue);
+
+    }
+    CarbonReadDataHolder holder = new CarbonReadDataHolder();
+    byte[][] value = new byte[valsList.size()][];
+    valsList.toArray(value);
+    if (arrayType == ByteArrayType.BIG_DECIMAL) {
+      BigDecimal[] bigDecimalValues = new BigDecimal[value.length];
+      for (int i = 0; i < value.length; i++) {
+        bigDecimalValues[i] = DataTypeUtil.byteToBigDecimal(value[i]);
+      }
+      holder.setReadableBigDecimalValues(bigDecimalValues);
+      return holder;
+    }
+    holder.setReadableByteValues(value);
+    return holder;
+  }
+
+  public static enum ByteArrayType {
+    BYTE_ARRAY,
+    BIG_DECIMAL
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/360edc8d/core/src/main/java/org/apache/carbondata/core/datastorage/store/compression/decimal/UnCompressMaxMinByte.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/datastorage/store/compression/decimal/UnCompressMaxMinByte.java b/core/src/main/java/org/apache/carbondata/core/datastorage/store/compression/decimal/UnCompressMaxMinByte.java
new file mode 100644
index 0000000..50caa62
--- /dev/null
+++ b/core/src/main/java/org/apache/carbondata/core/datastorage/store/compression/decimal/UnCompressMaxMinByte.java
@@ -0,0 +1,135 @@
+/*
+ * 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.datastorage.store.compression.decimal;
+
+import org.apache.carbondata.common.logging.LogService;
+import org.apache.carbondata.common.logging.LogServiceFactory;
+import org.apache.carbondata.core.datastorage.store.compression.Compressor;
+import org.apache.carbondata.core.datastorage.store.compression.CompressorFactory;
+import org.apache.carbondata.core.datastorage.store.compression.ValueCompressonHolder;
+import org.apache.carbondata.core.datastorage.store.compression.ValueCompressonHolder.UnCompressValue;
+import org.apache.carbondata.core.datastorage.store.dataholder.CarbonReadDataHolder;
+import org.apache.carbondata.core.util.ValueCompressionUtil;
+import org.apache.carbondata.core.util.ValueCompressionUtil.DataType;
+
+public class UnCompressMaxMinByte implements UnCompressValue<byte[]> {
+
+  /**
+   * Attribute for Carbon LOGGER
+   */
+  private static final LogService LOGGER =
+      LogServiceFactory.getLogService(UnCompressMaxMinByte.class.getName());
+  /**
+   * compressor.
+   */
+  private static Compressor compressor = CompressorFactory.getInstance();
+  /**
+   * value.
+   */
+  protected byte[] value;
+
+  /**
+   * actual data type
+   */
+  protected DataType actualDataType;
+
+  //TODO SIMIAN
+
+  public UnCompressMaxMinByte(DataType actualDataType) {
+    this.actualDataType = actualDataType;
+  }
+
+  @Override public void setValue(byte[] value) {
+    this.value = value;
+
+  }
+
+  @Override public UnCompressValue getNew() {
+    try {
+      return (UnCompressValue) clone();
+    } catch (CloneNotSupportedException e) {
+      LOGGER.error(e, e.getMessage());
+    }
+    return null;
+  }
+
+  @Override public UnCompressValue compress() {
+    UnCompressMaxMinByte byte1 = new UnCompressMaxMinByte(actualDataType);
+    byte1.setValue(compressor.compressByte(value));
+    return byte1;
+  }
+
+  @Override public UnCompressValue uncompress(DataType dataType) {
+    UnCompressValue byte1 = ValueCompressionUtil.getUnCompressDecimalMaxMin(dataType,
+        actualDataType);
+    ValueCompressonHolder.unCompress(dataType, byte1, value);
+    return byte1;
+  }
+
+  @Override public byte[] getBackArrayData() {
+    return value;
+  }
+
+  @Override public void setValueInBytes(byte[] value) {
+    this.value = value;
+  }
+
+  /**
+   * @see ValueCompressonHolder.UnCompressValue#getCompressorObject()
+   */
+  @Override public UnCompressValue getCompressorObject() {
+    return new UnCompressMaxMinByte(actualDataType);
+  }
+
+  @Override public CarbonReadDataHolder getValues(int decimal, Object maxValueObject) {
+    switch (actualDataType) {
+      case DATA_BIGINT:
+        return unCompressLong(maxValueObject);
+      default:
+        return unCompressDouble(maxValueObject);
+    }
+  }
+
+  private CarbonReadDataHolder unCompressLong(Object maxValueObject) {
+    long maxValue = (long) maxValueObject;
+    long[] vals = new long[value.length];
+    CarbonReadDataHolder dataHolder = new CarbonReadDataHolder();
+    for (int i = 0; i < vals.length; i++) {
+      if (value[i] == 0) {
+        vals[i] = maxValue;
+      } else {
+        vals[i] = maxValue - value[i];
+      }
+    }
+    dataHolder.setReadableLongValues(vals);
+    return dataHolder;
+  }
+
+  private CarbonReadDataHolder unCompressDouble(Object maxValueObject) {
+    double maxValue = (double) maxValueObject;
+    double[] vals = new double[value.length];
+    CarbonReadDataHolder dataHolder = new CarbonReadDataHolder();
+    for (int i = 0; i < vals.length; i++) {
+      vals[i] = maxValue - value[i];
+    }
+    dataHolder.setReadableDoubleValues(vals);
+    return dataHolder;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/360edc8d/core/src/main/java/org/apache/carbondata/core/datastorage/store/compression/decimal/UnCompressMaxMinDefault.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/datastorage/store/compression/decimal/UnCompressMaxMinDefault.java b/core/src/main/java/org/apache/carbondata/core/datastorage/store/compression/decimal/UnCompressMaxMinDefault.java
new file mode 100644
index 0000000..dc03694
--- /dev/null
+++ b/core/src/main/java/org/apache/carbondata/core/datastorage/store/compression/decimal/UnCompressMaxMinDefault.java
@@ -0,0 +1,111 @@
+/*
+ * 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.datastorage.store.compression.decimal;
+
+import java.nio.ByteBuffer;
+
+import org.apache.carbondata.common.logging.LogService;
+import org.apache.carbondata.common.logging.LogServiceFactory;
+import org.apache.carbondata.core.datastorage.store.compression.Compressor;
+import org.apache.carbondata.core.datastorage.store.compression.CompressorFactory;
+import org.apache.carbondata.core.datastorage.store.compression.ValueCompressonHolder;
+import org.apache.carbondata.core.datastorage.store.dataholder.CarbonReadDataHolder;
+import org.apache.carbondata.core.util.ValueCompressionUtil;
+import org.apache.carbondata.core.util.ValueCompressionUtil.DataType;
+
+public class UnCompressMaxMinDefault implements ValueCompressonHolder.UnCompressValue<double[]> {
+
+  /**
+   * Attribute for Carbon LOGGER
+   */
+  private static final LogService LOGGER =
+      LogServiceFactory.getLogService(UnCompressMaxMinDefault.class.getName());
+
+  /**
+   * compressor.
+   */
+  private static Compressor compressor = CompressorFactory.getInstance();
+  /**
+   * value.
+   */
+  private double[] value;
+
+  /**
+   * actual data type
+   */
+  private DataType actualDataType;
+
+  public UnCompressMaxMinDefault(DataType actualDataType) {
+    this.actualDataType = actualDataType;
+  }
+
+  @Override public void setValue(double[] value) {
+    this.value = value;
+
+  }
+
+  @Override public ValueCompressonHolder.UnCompressValue getNew() {
+    try {
+      return (ValueCompressonHolder.UnCompressValue) clone();
+    } catch (CloneNotSupportedException ex5) {
+      LOGGER.error(ex5, ex5.getMessage());
+    }
+    return null;
+  }
+
+  @Override public ValueCompressonHolder.UnCompressValue compress() {
+    UnCompressMaxMinByte byte1 = new UnCompressMaxMinByte(actualDataType);
+    byte1.setValue(compressor.compressDouble(value));
+    return byte1;
+  }
+
+  @Override public ValueCompressonHolder.UnCompressValue uncompress(DataType dataType) {
+    return null;
+  }
+
+  @Override public byte[] getBackArrayData() {
+    return ValueCompressionUtil.convertToBytes(value);
+  }
+
+  @Override public void setValueInBytes(byte[] value) {
+    ByteBuffer buffer = ByteBuffer.wrap(value);
+    this.value = ValueCompressionUtil.convertToDoubleArray(buffer, value.length);
+  }
+
+  /**
+   * @see ValueCompressonHolder.UnCompressValue#getCompressorObject()
+   */
+  @Override public ValueCompressonHolder.UnCompressValue getCompressorObject() {
+    return new UnCompressMaxMinByte(actualDataType);
+  }
+
+  //TODO SIMIAN
+  @Override public CarbonReadDataHolder getValues(int decimal, Object maxValueObject) {
+    double maxValue = (double) maxValueObject;
+    double[] vals = new double[value.length];
+    CarbonReadDataHolder dataHolderInfoObj = new CarbonReadDataHolder();
+    for (int i = 0; i < vals.length; i++) {
+      vals[i] = maxValue - value[i];
+    }
+    dataHolderInfoObj.setReadableDoubleValues(vals);
+    return dataHolderInfoObj;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/360edc8d/core/src/main/java/org/apache/carbondata/core/datastorage/store/compression/decimal/UnCompressMaxMinFloat.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/datastorage/store/compression/decimal/UnCompressMaxMinFloat.java b/core/src/main/java/org/apache/carbondata/core/datastorage/store/compression/decimal/UnCompressMaxMinFloat.java
new file mode 100644
index 0000000..e5c3293
--- /dev/null
+++ b/core/src/main/java/org/apache/carbondata/core/datastorage/store/compression/decimal/UnCompressMaxMinFloat.java
@@ -0,0 +1,102 @@
+/*
+ * 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.datastorage.store.compression.decimal;
+
+import java.nio.ByteBuffer;
+
+import org.apache.carbondata.common.logging.LogService;
+import org.apache.carbondata.common.logging.LogServiceFactory;
+import org.apache.carbondata.core.datastorage.store.compression.Compressor;
+import org.apache.carbondata.core.datastorage.store.compression.CompressorFactory;
+import org.apache.carbondata.core.datastorage.store.compression.ValueCompressonHolder.UnCompressValue;
+import org.apache.carbondata.core.datastorage.store.dataholder.CarbonReadDataHolder;
+import org.apache.carbondata.core.util.ValueCompressionUtil;
+import org.apache.carbondata.core.util.ValueCompressionUtil.DataType;
+
+public class UnCompressMaxMinFloat implements UnCompressValue<float[]> {
+
+  /**
+   * Attribute for Carbon LOGGER
+   */
+  private static final LogService LOGGER =
+      LogServiceFactory.getLogService(UnCompressMaxMinFloat.class.getName());
+  /**
+   * floatCompressor
+   */
+  private static Compressor compressor = CompressorFactory.getInstance();
+  /**
+   * value.
+   */
+  private float[] value;
+
+  private DataType actualDataType;
+
+  public UnCompressMaxMinFloat(DataType actualDataType) {
+    this.actualDataType = actualDataType;
+  }
+
+  @Override public void setValue(float[] value) {
+    this.value = (float[]) value;
+
+  }
+
+  @Override public UnCompressValue getNew() {
+    try {
+      return (UnCompressValue) clone();
+    } catch (CloneNotSupportedException ex4) {
+      LOGGER.error(ex4, ex4.getMessage());
+    }
+    return null;
+  }
+
+  @Override public UnCompressValue compress() {
+    UnCompressMaxMinByte byte1 = new UnCompressMaxMinByte(actualDataType);
+    byte1.setValue(compressor.compressFloat(value));
+    return byte1;
+  }
+
+  @Override public UnCompressValue uncompress(DataType dTypeVal) {
+    return null;
+  }
+
+  @Override public byte[] getBackArrayData() {
+    return ValueCompressionUtil.convertToBytes(value);
+  }
+
+  @Override public void setValueInBytes(byte[] value) {
+    ByteBuffer buffer = ByteBuffer.wrap(value);
+    this.value = ValueCompressionUtil.convertToFloatArray(buffer, value.length);
+  }
+
+  @Override public UnCompressValue getCompressorObject() {
+    return new UnCompressMaxMinByte(actualDataType);
+  }
+
+  @Override public CarbonReadDataHolder getValues(int decimal, Object maxValueObject) {
+    double maxValue = (double) maxValueObject;
+    double[] vals = new double[value.length];
+    CarbonReadDataHolder dataHolderVal = new CarbonReadDataHolder();
+    for (int i = 0; i < vals.length; i++) {
+      vals[i] = maxValue - value[i];
+    }
+    dataHolderVal.setReadableDoubleValues(vals);
+    return dataHolderVal;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/360edc8d/core/src/main/java/org/apache/carbondata/core/datastorage/store/compression/decimal/UnCompressMaxMinInt.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/datastorage/store/compression/decimal/UnCompressMaxMinInt.java b/core/src/main/java/org/apache/carbondata/core/datastorage/store/compression/decimal/UnCompressMaxMinInt.java
new file mode 100644
index 0000000..8e42c64
--- /dev/null
+++ b/core/src/main/java/org/apache/carbondata/core/datastorage/store/compression/decimal/UnCompressMaxMinInt.java
@@ -0,0 +1,127 @@
+/*
+ * 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.datastorage.store.compression.decimal;
+
+import java.nio.ByteBuffer;
+
+import org.apache.carbondata.common.logging.LogService;
+import org.apache.carbondata.common.logging.LogServiceFactory;
+import org.apache.carbondata.core.datastorage.store.compression.Compressor;
+import org.apache.carbondata.core.datastorage.store.compression.CompressorFactory;
+import org.apache.carbondata.core.datastorage.store.compression.ValueCompressonHolder;
+import org.apache.carbondata.core.datastorage.store.dataholder.CarbonReadDataHolder;
+import org.apache.carbondata.core.util.ValueCompressionUtil;
+import org.apache.carbondata.core.util.ValueCompressionUtil.DataType;
+
+public class UnCompressMaxMinInt implements ValueCompressonHolder.UnCompressValue<int[]> {
+  /**
+   * Attribute for Carbon LOGGER
+   */
+  private static final LogService LOGGER =
+      LogServiceFactory.getLogService(UnCompressMaxMinInt.class.getName());
+
+  /**
+   * compressor.
+   */
+  private static Compressor compressor = CompressorFactory.getInstance();
+  /**
+   * value.
+   */
+  private int[] value;
+
+  private DataType actualDataType;
+
+  public UnCompressMaxMinInt(DataType actualDataType) {
+    this.actualDataType = actualDataType;
+  }
+
+  @Override public void setValue(int[] value) {
+    this.value = value;
+
+  }
+
+  @Override public ValueCompressonHolder.UnCompressValue getNew() {
+    try {
+      return (ValueCompressonHolder.UnCompressValue) clone();
+    } catch (CloneNotSupportedException e) {
+      LOGGER.error(e, e.getMessage());
+    }
+    return null;
+  }
+
+  @Override public ValueCompressonHolder.UnCompressValue compress() {
+    UnCompressMaxMinByte byte1 = new UnCompressMaxMinByte(actualDataType);
+    byte1.setValue(compressor.compressInt(value));
+    return byte1;
+  }
+
+  @Override public ValueCompressonHolder.UnCompressValue uncompress(
+      ValueCompressionUtil.DataType dataTypeValue) {
+    return null;
+  }
+
+  @Override public byte[] getBackArrayData() {
+    return ValueCompressionUtil.convertToBytes(value);
+  }
+
+  @Override public void setValueInBytes(byte[] value) {
+    ByteBuffer buffer = ByteBuffer.wrap(value);
+    this.value = ValueCompressionUtil.convertToIntArray(buffer, value.length);
+  }
+
+  /**
+   * @see ValueCompressonHolder.UnCompressValue#getCompressorObject()
+   */
+  @Override public ValueCompressonHolder.UnCompressValue getCompressorObject() {
+    return new UnCompressMaxMinByte(actualDataType);
+  }
+
+  @Override public CarbonReadDataHolder getValues(int decVal, Object maxValueObject) {
+    switch (actualDataType) {
+      case DATA_BIGINT:
+        return unCompressLong(maxValueObject);
+      default:
+        return unCompressDouble(maxValueObject);
+    }
+  }
+
+  private CarbonReadDataHolder unCompressDouble(Object maxValueObject) {
+    double maxValue = (double) maxValueObject;
+    double[] vals = new double[value.length];
+    CarbonReadDataHolder dataHolder = new CarbonReadDataHolder();
+    for (int i = 0; i < vals.length; i++) {
+      vals[i] = maxValue - value[i];
+    }
+    dataHolder.setReadableDoubleValues(vals);
+    return dataHolder;
+  }
+
+  private CarbonReadDataHolder unCompressLong(Object maxValueObject) {
+    long maxValue = (long) maxValueObject;
+    long[] vals = new long[value.length];
+    CarbonReadDataHolder dataHolder = new CarbonReadDataHolder();
+    for (int i = 0; i < vals.length; i++) {
+      vals[i] = maxValue - value[i];
+    }
+    dataHolder.setReadableLongValues(vals);
+    return dataHolder;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/360edc8d/core/src/main/java/org/apache/carbondata/core/datastorage/store/compression/decimal/UnCompressMaxMinLong.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/datastorage/store/compression/decimal/UnCompressMaxMinLong.java b/core/src/main/java/org/apache/carbondata/core/datastorage/store/compression/decimal/UnCompressMaxMinLong.java
new file mode 100644
index 0000000..5a266fa
--- /dev/null
+++ b/core/src/main/java/org/apache/carbondata/core/datastorage/store/compression/decimal/UnCompressMaxMinLong.java
@@ -0,0 +1,126 @@
+/*
+ * 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.datastorage.store.compression.decimal;
+
+import java.nio.ByteBuffer;
+
+import org.apache.carbondata.common.logging.LogService;
+import org.apache.carbondata.common.logging.LogServiceFactory;
+import org.apache.carbondata.core.datastorage.store.compression.Compressor;
+import org.apache.carbondata.core.datastorage.store.compression.CompressorFactory;
+import org.apache.carbondata.core.datastorage.store.compression.ValueCompressonHolder;
+import org.apache.carbondata.core.datastorage.store.dataholder.CarbonReadDataHolder;
+import org.apache.carbondata.core.util.ValueCompressionUtil;
+import org.apache.carbondata.core.util.ValueCompressionUtil.DataType;
+
+public class UnCompressMaxMinLong implements ValueCompressonHolder.UnCompressValue<long[]> {
+  /**
+   * Attribute for Carbon LOGGER
+   */
+  private static final LogService LOGGER =
+      LogServiceFactory.getLogService(UnCompressMaxMinLong.class.getName());
+  /**
+   * compressor.
+   */
+  private static Compressor compressor = CompressorFactory.getInstance();
+  /**
+   * value.
+   */
+  protected long[] value;
+
+  protected DataType actualDataType;
+
+  public UnCompressMaxMinLong(DataType actualDataType) {
+    this.actualDataType = actualDataType;
+  }
+
+  @Override public ValueCompressonHolder.UnCompressValue getNew() {
+    try {
+      return (ValueCompressonHolder.UnCompressValue) clone();
+    } catch (CloneNotSupportedException e) {
+      LOGGER.error(e, e.getMessage());
+    }
+    return null;
+  }
+
+  @Override public ValueCompressonHolder.UnCompressValue compress() {
+    UnCompressMaxMinByte unCompressByte = new UnCompressMaxMinByte(actualDataType);
+    unCompressByte.setValue(compressor.compressLong(value));
+    return unCompressByte;
+  }
+
+  @Override public void setValue(long[] value) {
+    this.value = value;
+
+  }
+
+  @Override
+  public ValueCompressonHolder.UnCompressValue uncompress(ValueCompressionUtil.DataType dataType) {
+    return null;
+  }
+
+  @Override public byte[] getBackArrayData() {
+    return ValueCompressionUtil.convertToBytes(value);
+  }
+
+  @Override public void setValueInBytes(byte[] value) {
+    ByteBuffer buffer = ByteBuffer.wrap(value);
+    this.value = ValueCompressionUtil.convertToLongArray(buffer, value.length);
+  }
+
+  /**
+   * @see ValueCompressonHolder.UnCompressValue#getCompressorObject()
+   */
+  @Override public ValueCompressonHolder.UnCompressValue getCompressorObject() {
+    return new UnCompressMaxMinByte(actualDataType);
+  }
+
+  @Override public CarbonReadDataHolder getValues(int decimal, Object maxValueObject) {
+    switch (actualDataType) {
+      case DATA_BIGINT:
+        return unCompressLong(maxValueObject);
+      default:
+        return unCompressDouble(maxValueObject);
+    }
+  }
+
+  private CarbonReadDataHolder unCompressDouble(Object maxValueObject) {
+    double maxValue = (double) maxValueObject;
+    double[] vals = new double[value.length];
+    CarbonReadDataHolder data = new CarbonReadDataHolder();
+    for (int i = 0; i < vals.length; i++) {
+      vals[i] = maxValue - value[i];
+    }
+    data.setReadableDoubleValues(vals);
+    return data;
+  }
+
+  private CarbonReadDataHolder unCompressLong(Object maxValueObject) {
+    long maxValue = (long) maxValueObject;
+    long[] vals = new long[value.length];
+    CarbonReadDataHolder data = new CarbonReadDataHolder();
+    for (int i = 0; i < vals.length; i++) {
+      vals[i] = maxValue - value[i];
+    }
+    data.setReadableLongValues(vals);
+    return data;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/360edc8d/core/src/main/java/org/apache/carbondata/core/datastorage/store/compression/decimal/UnCompressMaxMinShort.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/datastorage/store/compression/decimal/UnCompressMaxMinShort.java b/core/src/main/java/org/apache/carbondata/core/datastorage/store/compression/decimal/UnCompressMaxMinShort.java
new file mode 100644
index 0000000..edade84
--- /dev/null
+++ b/core/src/main/java/org/apache/carbondata/core/datastorage/store/compression/decimal/UnCompressMaxMinShort.java
@@ -0,0 +1,126 @@
+/*
+ * 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.datastorage.store.compression.decimal;
+
+import java.nio.ByteBuffer;
+
+import org.apache.carbondata.common.logging.LogService;
+import org.apache.carbondata.common.logging.LogServiceFactory;
+import org.apache.carbondata.core.datastorage.store.compression.Compressor;
+import org.apache.carbondata.core.datastorage.store.compression.CompressorFactory;
+import org.apache.carbondata.core.datastorage.store.compression.ValueCompressonHolder;
+import org.apache.carbondata.core.datastorage.store.dataholder.CarbonReadDataHolder;
+import org.apache.carbondata.core.util.ValueCompressionUtil;
+import org.apache.carbondata.core.util.ValueCompressionUtil.DataType;
+
+public class UnCompressMaxMinShort implements ValueCompressonHolder.UnCompressValue<short[]> {
+  /**
+   * Attribute for Carbon LOGGER
+   */
+  private static final LogService LOGGER =
+      LogServiceFactory.getLogService(UnCompressMaxMinShort.class.getName());
+  /**
+   * shortCompressor.
+   */
+  private static Compressor compressor = CompressorFactory.getInstance();
+  /**
+   * value.
+   */
+  private short[] value;
+
+  private DataType actualDataType;
+
+  public UnCompressMaxMinShort(DataType actualDataType) {
+    this.actualDataType = actualDataType;
+  }
+
+  @Override public void setValue(short[] value) {
+    this.value = value;
+
+  }
+
+  @Override public ValueCompressonHolder.UnCompressValue uncompress(DataType dataType) {
+    return null;
+  }
+
+  @Override public byte[] getBackArrayData() {
+    return ValueCompressionUtil.convertToBytes(value);
+  }
+
+  @Override public ValueCompressonHolder.UnCompressValue getNew() {
+    try {
+      return (ValueCompressonHolder.UnCompressValue) clone();
+    } catch (CloneNotSupportedException ex3) {
+      LOGGER.error(ex3, ex3.getMessage());
+    }
+    return null;
+  }
+
+  @Override public ValueCompressonHolder.UnCompressValue compress() {
+    UnCompressMaxMinByte byte1 = new UnCompressMaxMinByte(actualDataType);
+    byte1.setValue(compressor.compressShort(value));
+    return byte1;
+  }
+
+  @Override public void setValueInBytes(byte[] value) {
+    ByteBuffer buffer = ByteBuffer.wrap(value);
+    this.value = ValueCompressionUtil.convertToShortArray(buffer, value.length);
+  }
+
+  /**
+   * @see ValueCompressonHolder.UnCompressValue#getCompressorObject()
+   */
+  @Override public ValueCompressonHolder.UnCompressValue getCompressorObject() {
+    return new UnCompressMaxMinByte(actualDataType);
+  }
+
+  @Override public CarbonReadDataHolder getValues(int decimal, Object maxValueObject) {
+    switch (actualDataType) {
+      case DATA_BIGINT:
+        return unCompressLong(maxValueObject);
+      default:
+        return unCompressDouble(maxValueObject);
+    }
+
+  }
+
+  private CarbonReadDataHolder unCompressDouble(Object maxValueObject) {
+    double maxValue = (double) maxValueObject;
+    double[] vals = new double[value.length];
+    CarbonReadDataHolder carbonDataHolderObj = new CarbonReadDataHolder();
+    for (int i = 0; i < vals.length; i++) {
+      vals[i] = maxValue - value[i];
+    }
+    carbonDataHolderObj.setReadableDoubleValues(vals);
+    return carbonDataHolderObj;
+  }
+
+  private CarbonReadDataHolder unCompressLong(Object maxValueObject) {
+    long maxValue = (long) maxValueObject;
+    long[] vals = new long[value.length];
+    CarbonReadDataHolder carbonDataHolderObj = new CarbonReadDataHolder();
+    for (int i = 0; i < vals.length; i++) {
+      vals[i] = maxValue - value[i];
+    }
+    carbonDataHolderObj.setReadableLongValues(vals);
+    return carbonDataHolderObj;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/360edc8d/core/src/main/java/org/apache/carbondata/core/datastorage/store/compression/nondecimal/UnCompressNonDecimalByte.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/datastorage/store/compression/nondecimal/UnCompressNonDecimalByte.java b/core/src/main/java/org/apache/carbondata/core/datastorage/store/compression/nondecimal/UnCompressNonDecimalByte.java
new file mode 100644
index 0000000..9718609
--- /dev/null
+++ b/core/src/main/java/org/apache/carbondata/core/datastorage/store/compression/nondecimal/UnCompressNonDecimalByte.java
@@ -0,0 +1,96 @@
+/*
+ * 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.datastorage.store.compression.nondecimal;
+
+import org.apache.carbondata.common.logging.LogService;
+import org.apache.carbondata.common.logging.LogServiceFactory;
+import org.apache.carbondata.core.datastorage.store.compression.Compressor;
+import org.apache.carbondata.core.datastorage.store.compression.CompressorFactory;
+import org.apache.carbondata.core.datastorage.store.compression.ValueCompressonHolder;
+import org.apache.carbondata.core.datastorage.store.dataholder.CarbonReadDataHolder;
+import org.apache.carbondata.core.util.ValueCompressionUtil;
+import org.apache.carbondata.core.util.ValueCompressionUtil.DataType;
+
+public class UnCompressNonDecimalByte implements ValueCompressonHolder.UnCompressValue<byte[]> {
+  /**
+   * Attribute for Carbon LOGGER
+   */
+  private static final LogService LOGGER =
+      LogServiceFactory.getLogService(UnCompressNonDecimalByte.class.getName());
+  /**
+   * compressor.
+   */
+  private static Compressor compressor = CompressorFactory.getInstance();
+  /**
+   * value.
+   */
+  private byte[] value;
+
+  @Override public void setValue(byte[] value) {
+    this.value = value;
+  }
+
+  @Override public ValueCompressonHolder.UnCompressValue getNew() {
+    try {
+      return (ValueCompressonHolder.UnCompressValue) clone();
+    } catch (CloneNotSupportedException e) {
+      LOGGER.error(e, e.getMessage());
+    }
+    return null;
+  }
+
+  @Override public ValueCompressonHolder.UnCompressValue compress() {
+    UnCompressNonDecimalByte byte1 = new UnCompressNonDecimalByte();
+    byte1.setValue(compressor.compressByte(value));
+    return byte1;
+  }
+
+  @Override public ValueCompressonHolder.UnCompressValue uncompress(DataType dataType) {
+    ValueCompressonHolder.UnCompressValue byte1 =
+        ValueCompressionUtil.getUnCompressNonDecimal(dataType);
+    ValueCompressonHolder.unCompress(dataType, byte1, value);
+    return byte1;
+  }
+
+  @Override public void setValueInBytes(byte[] value) {
+    this.value = value;
+  }
+
+  @Override public byte[] getBackArrayData() {
+    return value;
+  }
+
+  /**
+   * @see ValueCompressonHolder.UnCompressValue#getCompressorObject()
+   */
+  @Override public ValueCompressonHolder.UnCompressValue getCompressorObject() {
+    return new UnCompressNonDecimalByte();
+  }
+
+  @Override public CarbonReadDataHolder getValues(int decimal, Object maxValueObject) {
+    double[] vals = new double[value.length];
+    CarbonReadDataHolder dataHolder = new CarbonReadDataHolder();
+    for (int i = 0; i < vals.length; i++) {
+      vals[i] = value[i] / Math.pow(10, decimal);
+    }
+    dataHolder.setReadableDoubleValues(vals);
+    return dataHolder;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/360edc8d/core/src/main/java/org/apache/carbondata/core/datastorage/store/compression/nondecimal/UnCompressNonDecimalDefault.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/datastorage/store/compression/nondecimal/UnCompressNonDecimalDefault.java b/core/src/main/java/org/apache/carbondata/core/datastorage/store/compression/nondecimal/UnCompressNonDecimalDefault.java
new file mode 100644
index 0000000..b345700
--- /dev/null
+++ b/core/src/main/java/org/apache/carbondata/core/datastorage/store/compression/nondecimal/UnCompressNonDecimalDefault.java
@@ -0,0 +1,96 @@
+/*
+ * 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.datastorage.store.compression.nondecimal;
+
+import java.nio.ByteBuffer;
+
+import org.apache.carbondata.common.logging.LogService;
+import org.apache.carbondata.common.logging.LogServiceFactory;
+import org.apache.carbondata.core.datastorage.store.compression.Compressor;
+import org.apache.carbondata.core.datastorage.store.compression.CompressorFactory;
+import org.apache.carbondata.core.datastorage.store.compression.ValueCompressonHolder;
+import org.apache.carbondata.core.datastorage.store.dataholder.CarbonReadDataHolder;
+import org.apache.carbondata.core.util.ValueCompressionUtil;
+import org.apache.carbondata.core.util.ValueCompressionUtil.DataType;
+
+public class UnCompressNonDecimalDefault
+    implements ValueCompressonHolder.UnCompressValue<double[]> {
+  /**
+   * Attribute for Carbon LOGGER
+   */
+  private static final LogService LOGGER =
+      LogServiceFactory.getLogService(UnCompressNonDecimalDefault.class.getName());
+  /**
+   * doubleCompressor.
+   */
+  private static Compressor compressor = CompressorFactory.getInstance();
+  /**
+   * value.
+   */
+  private double[] value;
+
+  @Override public ValueCompressonHolder.UnCompressValue getNew() {
+    try {
+      return (ValueCompressonHolder.UnCompressValue) clone();
+    } catch (CloneNotSupportedException cnse1) {
+      LOGGER.error(cnse1, cnse1.getMessage());
+    }
+    return null;
+  }
+
+  @Override public ValueCompressonHolder.UnCompressValue compress() {
+    UnCompressNonDecimalByte byte1 = new UnCompressNonDecimalByte();
+    byte1.setValue(compressor.compressDouble(value));
+    return byte1;
+  }
+
+  @Override public ValueCompressonHolder.UnCompressValue uncompress(DataType dataType) {
+    return null;
+  }
+
+  @Override public void setValue(double[] value) {
+    this.value = value;
+
+  }
+
+  @Override public void setValueInBytes(byte[] value) {
+    ByteBuffer buffer = ByteBuffer.wrap(value);
+    this.value = ValueCompressionUtil.convertToDoubleArray(buffer, value.length);
+  }
+
+  @Override public byte[] getBackArrayData() {
+    return ValueCompressionUtil.convertToBytes(value);
+  }
+
+  @Override public ValueCompressonHolder.UnCompressValue getCompressorObject() {
+    return new UnCompressNonDecimalByte();
+  }
+
+  @Override public CarbonReadDataHolder getValues(int decimal, Object maxValueObject) {
+    double[] dblVals = new double[value.length];
+    for (int i = 0; i < dblVals.length; i++) {
+      dblVals[i] = value[i] / Math.pow(10, decimal);
+    }
+    CarbonReadDataHolder dataHolder = new CarbonReadDataHolder();
+    dataHolder.setReadableDoubleValues(dblVals);
+    return dataHolder;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/360edc8d/core/src/main/java/org/apache/carbondata/core/datastorage/store/compression/nondecimal/UnCompressNonDecimalFloat.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/datastorage/store/compression/nondecimal/UnCompressNonDecimalFloat.java b/core/src/main/java/org/apache/carbondata/core/datastorage/store/compression/nondecimal/UnCompressNonDecimalFloat.java
new file mode 100644
index 0000000..ef126ff
--- /dev/null
+++ b/core/src/main/java/org/apache/carbondata/core/datastorage/store/compression/nondecimal/UnCompressNonDecimalFloat.java
@@ -0,0 +1,100 @@
+/*
+ * 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.datastorage.store.compression.nondecimal;
+
+import java.nio.ByteBuffer;
+
+import org.apache.carbondata.common.logging.LogService;
+import org.apache.carbondata.common.logging.LogServiceFactory;
+import org.apache.carbondata.core.datastorage.store.compression.Compressor;
+import org.apache.carbondata.core.datastorage.store.compression.CompressorFactory;
+import org.apache.carbondata.core.datastorage.store.compression.ValueCompressonHolder;
+import org.apache.carbondata.core.datastorage.store.dataholder.CarbonReadDataHolder;
+import org.apache.carbondata.core.util.ValueCompressionUtil;
+
+public class UnCompressNonDecimalFloat implements ValueCompressonHolder.UnCompressValue<float[]> {
+  /**
+   * Attribute for Carbon LOGGER
+   */
+  private static final LogService LOGGER =
+      LogServiceFactory.getLogService(UnCompressNonDecimalFloat.class.getName());
+  /**
+   * floatCompressor
+   */
+  private static Compressor compressor = CompressorFactory.getInstance();
+  /**
+   * value.
+   */
+
+  private float[] value;
+
+  @Override public void setValue(float[] value) {
+    this.value = value;
+
+  }
+
+  @Override public ValueCompressonHolder.UnCompressValue getNew() {
+    try {
+      return (ValueCompressonHolder.UnCompressValue) clone();
+    } catch (CloneNotSupportedException cnsexception) {
+      LOGGER
+          .error(cnsexception, cnsexception.getMessage());
+    }
+    return null;
+  }
+
+  public byte[] getBackArrayData() {
+    return ValueCompressionUtil.convertToBytes(value);
+  }
+
+  @Override public ValueCompressonHolder.UnCompressValue compress() {
+    UnCompressNonDecimalByte byte1 = new UnCompressNonDecimalByte();
+    byte1.setValue(compressor.compressFloat(value));
+    return byte1;
+  }
+
+  @Override
+  public ValueCompressonHolder.UnCompressValue uncompress(ValueCompressionUtil.DataType dataType) {
+    return null;
+  }
+
+  @Override public void setValueInBytes(byte[] value) {
+    ByteBuffer buffer = ByteBuffer.wrap(value);
+    this.value = ValueCompressionUtil.convertToFloatArray(buffer, value.length);
+  }
+
+  /**
+   * @see ValueCompressonHolder.UnCompressValue#getCompressorObject()
+   */
+  @Override public ValueCompressonHolder.UnCompressValue getCompressorObject() {
+    return new UnCompressNonDecimalByte();
+  }
+
+  @Override public CarbonReadDataHolder getValues(int decimal, Object maxValueObject) {
+    double[] vals = new double[value.length];
+    for (int m = 0; m < vals.length; m++) {
+      vals[m] = value[m] / Math.pow(10, decimal);
+    }
+    CarbonReadDataHolder dataHolder = new CarbonReadDataHolder();
+    dataHolder.setReadableDoubleValues(vals);
+    return dataHolder;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/360edc8d/core/src/main/java/org/apache/carbondata/core/datastorage/store/compression/nondecimal/UnCompressNonDecimalInt.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/datastorage/store/compression/nondecimal/UnCompressNonDecimalInt.java b/core/src/main/java/org/apache/carbondata/core/datastorage/store/compression/nondecimal/UnCompressNonDecimalInt.java
new file mode 100644
index 0000000..cd47bff
--- /dev/null
+++ b/core/src/main/java/org/apache/carbondata/core/datastorage/store/compression/nondecimal/UnCompressNonDecimalInt.java
@@ -0,0 +1,94 @@
+/*
+ * 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.datastorage.store.compression.nondecimal;
+
+import java.nio.ByteBuffer;
+
+import org.apache.carbondata.common.logging.LogService;
+import org.apache.carbondata.common.logging.LogServiceFactory;
+import org.apache.carbondata.core.datastorage.store.compression.Compressor;
+import org.apache.carbondata.core.datastorage.store.compression.CompressorFactory;
+import org.apache.carbondata.core.datastorage.store.compression.ValueCompressonHolder.UnCompressValue;
+import org.apache.carbondata.core.datastorage.store.dataholder.CarbonReadDataHolder;
+import org.apache.carbondata.core.util.ValueCompressionUtil;
+import org.apache.carbondata.core.util.ValueCompressionUtil.DataType;
+
+public class UnCompressNonDecimalInt implements UnCompressValue<int[]> {
+  /**
+   * Attribute for Carbon LOGGER
+   */
+  private static final LogService LOGGER =
+      LogServiceFactory.getLogService(UnCompressNonDecimalInt.class.getName());
+  /**
+   * intCompressor.
+   */
+  private static Compressor compressor = CompressorFactory.getInstance();
+  /**
+   * value.
+   */
+  private int[] value;
+
+  @Override public void setValue(int[] value) {
+    this.value = value;
+  }
+
+  @Override public UnCompressValue getNew() {
+    try {
+      return (UnCompressValue) clone();
+    } catch (CloneNotSupportedException csne1) {
+      LOGGER.error(csne1, csne1.getMessage());
+    }
+    return null;
+  }
+
+  @Override public UnCompressValue compress() {
+    UnCompressNonDecimalByte byte1 = new UnCompressNonDecimalByte();
+    byte1.setValue(compressor.compressInt(value));
+    return byte1;
+  }
+
+  @Override public byte[] getBackArrayData() {
+    return ValueCompressionUtil.convertToBytes(value);
+  }
+
+  @Override public void setValueInBytes(byte[] bytesArr) {
+    ByteBuffer buffer = ByteBuffer.wrap(bytesArr);
+    this.value = ValueCompressionUtil.convertToIntArray(buffer, bytesArr.length);
+  }
+
+  @Override public UnCompressValue getCompressorObject() {
+    return new UnCompressNonDecimalByte();
+  }
+
+  @Override public CarbonReadDataHolder getValues(int mantissa, Object maxValueObject) {
+    double[] vals = new double[value.length];
+    for (int k = 0; k < vals.length; k++) {
+      vals[k] = value[k] / Math.pow(10, mantissa);
+    }
+    CarbonReadDataHolder dataHolder = new CarbonReadDataHolder();
+    dataHolder.setReadableDoubleValues(vals);
+    return dataHolder;
+  }
+
+  @Override public UnCompressValue uncompress(DataType dataType) {
+    return null;
+  }
+
+}