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

[1/2] incubator-carbondata git commit: Double min max difference compression issue. 1.While compressing maxdifference typecast with proper DataType 2.While writting we follow like a. diff with max value , say d = max - x b. multiply with 10^decimal i.e w

Repository: incubator-carbondata
Updated Branches:
  refs/heads/master 4275277b9 -> 6537316de


Double min max difference compression issue.
1.While compressing maxdifference typecast with proper DataType
2.While writting we follow like
a. diff with max value , say d = max - x
b. multiply with 10^decimal i.e w = d*10^decimal
3.Similarly while reading we should follow like
a. divide written value 10^decimal i.e d = w/10^decimal
b. diff with max i.e x = max -d
4. There is problem in java with double subtraction
i.e double value = 3.141818 - 0.000610
java will result 3.1412080000000002
To avoid this we are converting to BigDecimal and then doing subtraction
5.Changed logic to calculate no of integer after decimal.


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

Branch: refs/heads/master
Commit: 276a3ff2706b6d00a25d5bee2390d1ba3c6ea475
Parents: 4275277
Author: Ashok Kumar <as...@gmail.com>
Authored: Thu Aug 25 16:10:29 2016 +0530
Committer: ravipesala <ra...@gmail.com>
Committed: Tue Aug 30 21:04:28 2016 +0530

----------------------------------------------------------------------
 .../type/UnCompressNonDecimalMaxMinByte.java    |  6 +-
 .../type/UnCompressNonDecimalMaxMinDefault.java |  5 +-
 .../type/UnCompressNonDecimalMaxMinFloat.java   |  5 +-
 .../type/UnCompressNonDecimalMaxMinInt.java     |  5 +-
 .../type/UnCompressNonDecimalMaxMinLong.java    |  5 +-
 .../type/UnCompressNonDecimalMaxMinShort.java   |  5 +-
 .../core/util/ValueCompressionUtil.java         | 26 ++++--
 .../ValueCompressionDataTypeTestCase.scala      | 97 ++++++++++++++++++++
 .../store/CarbonFactDataHandlerColumnar.java    | 17 +++-
 9 files changed, 157 insertions(+), 14 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/276a3ff2/core/src/main/java/org/apache/carbondata/core/datastorage/store/compression/type/UnCompressNonDecimalMaxMinByte.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/datastorage/store/compression/type/UnCompressNonDecimalMaxMinByte.java b/core/src/main/java/org/apache/carbondata/core/datastorage/store/compression/type/UnCompressNonDecimalMaxMinByte.java
index 1533679..1e6192c 100644
--- a/core/src/main/java/org/apache/carbondata/core/datastorage/store/compression/type/UnCompressNonDecimalMaxMinByte.java
+++ b/core/src/main/java/org/apache/carbondata/core/datastorage/store/compression/type/UnCompressNonDecimalMaxMinByte.java
@@ -19,6 +19,8 @@
 
 package org.apache.carbondata.core.datastorage.store.compression.type;
 
+import java.math.BigDecimal;
+
 import org.apache.carbondata.common.logging.LogService;
 import org.apache.carbondata.common.logging.LogServiceFactory;
 import org.apache.carbondata.core.datastorage.store.compression.Compressor;
@@ -93,7 +95,9 @@ public class UnCompressNonDecimalMaxMinByte
       if (value[i] == 0) {
         vals[i] = maxValue;
       } else {
-        vals[i] = (maxValue - value[i]) / Math.pow(10, decimalVal);
+        BigDecimal diff = BigDecimal.valueOf(value[i] / Math.pow(10, decimalVal));
+        BigDecimal max = BigDecimal.valueOf(maxValue);
+        vals[i] = max.subtract(diff).doubleValue();
       }
 
     }

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/276a3ff2/core/src/main/java/org/apache/carbondata/core/datastorage/store/compression/type/UnCompressNonDecimalMaxMinDefault.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/datastorage/store/compression/type/UnCompressNonDecimalMaxMinDefault.java b/core/src/main/java/org/apache/carbondata/core/datastorage/store/compression/type/UnCompressNonDecimalMaxMinDefault.java
index ccb765a..f596a64 100644
--- a/core/src/main/java/org/apache/carbondata/core/datastorage/store/compression/type/UnCompressNonDecimalMaxMinDefault.java
+++ b/core/src/main/java/org/apache/carbondata/core/datastorage/store/compression/type/UnCompressNonDecimalMaxMinDefault.java
@@ -19,6 +19,7 @@
 
 package org.apache.carbondata.core.datastorage.store.compression.type;
 
+import java.math.BigDecimal;
 import java.nio.ByteBuffer;
 
 import org.apache.carbondata.common.logging.LogService;
@@ -95,7 +96,9 @@ public class UnCompressNonDecimalMaxMinDefault implements UnCompressValue<double
       if (value[i] == 0) {
         vals[i] = maxVal;
       } else {
-        vals[i] = (maxVal - value[i]) / Math.pow(10, decimal);
+        BigDecimal diff = BigDecimal.valueOf(value[i] / Math.pow(10, decimal));
+        BigDecimal max = BigDecimal.valueOf(maxVal);
+        vals[i] = max.subtract(diff).doubleValue();
       }
 
     }

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/276a3ff2/core/src/main/java/org/apache/carbondata/core/datastorage/store/compression/type/UnCompressNonDecimalMaxMinFloat.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/datastorage/store/compression/type/UnCompressNonDecimalMaxMinFloat.java b/core/src/main/java/org/apache/carbondata/core/datastorage/store/compression/type/UnCompressNonDecimalMaxMinFloat.java
index 16bf101..aff9985 100644
--- a/core/src/main/java/org/apache/carbondata/core/datastorage/store/compression/type/UnCompressNonDecimalMaxMinFloat.java
+++ b/core/src/main/java/org/apache/carbondata/core/datastorage/store/compression/type/UnCompressNonDecimalMaxMinFloat.java
@@ -19,6 +19,7 @@
 
 package org.apache.carbondata.core.datastorage.store.compression.type;
 
+import java.math.BigDecimal;
 import java.nio.ByteBuffer;
 
 import org.apache.carbondata.common.logging.LogService;
@@ -98,7 +99,9 @@ public class UnCompressNonDecimalMaxMinFloat
       if (value[i] == 0) {
         vals[i] = maxValue;
       } else {
-        vals[i] = (maxValue - value[i]) / Math.pow(10, decimal);
+        BigDecimal diff = BigDecimal.valueOf(value[i] / Math.pow(10, decimal));
+        BigDecimal max = BigDecimal.valueOf(maxValue);
+        vals[i] = max.subtract(diff).doubleValue();
       }
     }
     holder.setReadableDoubleValues(vals);

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/276a3ff2/core/src/main/java/org/apache/carbondata/core/datastorage/store/compression/type/UnCompressNonDecimalMaxMinInt.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/datastorage/store/compression/type/UnCompressNonDecimalMaxMinInt.java b/core/src/main/java/org/apache/carbondata/core/datastorage/store/compression/type/UnCompressNonDecimalMaxMinInt.java
index c0ab9c0..245f782 100644
--- a/core/src/main/java/org/apache/carbondata/core/datastorage/store/compression/type/UnCompressNonDecimalMaxMinInt.java
+++ b/core/src/main/java/org/apache/carbondata/core/datastorage/store/compression/type/UnCompressNonDecimalMaxMinInt.java
@@ -19,6 +19,7 @@
 
 package org.apache.carbondata.core.datastorage.store.compression.type;
 
+import java.math.BigDecimal;
 import java.nio.ByteBuffer;
 
 import org.apache.carbondata.common.logging.LogService;
@@ -93,7 +94,9 @@ public class UnCompressNonDecimalMaxMinInt implements UnCompressValue<int[]> {
       if (value[i] == 0) {
         vals[i] = maxValue;
       } else {
-        vals[i] = (maxValue - value[i]) / Math.pow(10, decimal);
+        BigDecimal diff = BigDecimal.valueOf(value[i] / Math.pow(10, decimal));
+        BigDecimal max = BigDecimal.valueOf(maxValue);
+        vals[i] = max.subtract(diff).doubleValue();
       }
 
     }

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/276a3ff2/core/src/main/java/org/apache/carbondata/core/datastorage/store/compression/type/UnCompressNonDecimalMaxMinLong.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/datastorage/store/compression/type/UnCompressNonDecimalMaxMinLong.java b/core/src/main/java/org/apache/carbondata/core/datastorage/store/compression/type/UnCompressNonDecimalMaxMinLong.java
index d8ed560..122db75 100644
--- a/core/src/main/java/org/apache/carbondata/core/datastorage/store/compression/type/UnCompressNonDecimalMaxMinLong.java
+++ b/core/src/main/java/org/apache/carbondata/core/datastorage/store/compression/type/UnCompressNonDecimalMaxMinLong.java
@@ -19,6 +19,7 @@
 
 package org.apache.carbondata.core.datastorage.store.compression.type;
 
+import java.math.BigDecimal;
 import java.nio.ByteBuffer;
 
 import org.apache.carbondata.common.logging.LogService;
@@ -99,7 +100,9 @@ public class UnCompressNonDecimalMaxMinLong
       if (value[i] == 0) {
         vals[i] = maxValue;
       } else {
-        vals[i] = (maxValue - value[i]) / Math.pow(10, decimal);
+        BigDecimal diff = BigDecimal.valueOf(value[i] / Math.pow(10, decimal));
+        BigDecimal max = BigDecimal.valueOf(maxValue);
+        vals[i] = max.subtract(diff).doubleValue();
       }
 
     }

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/276a3ff2/core/src/main/java/org/apache/carbondata/core/datastorage/store/compression/type/UnCompressNonDecimalMaxMinShort.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/datastorage/store/compression/type/UnCompressNonDecimalMaxMinShort.java b/core/src/main/java/org/apache/carbondata/core/datastorage/store/compression/type/UnCompressNonDecimalMaxMinShort.java
index f110560..e82ac80 100644
--- a/core/src/main/java/org/apache/carbondata/core/datastorage/store/compression/type/UnCompressNonDecimalMaxMinShort.java
+++ b/core/src/main/java/org/apache/carbondata/core/datastorage/store/compression/type/UnCompressNonDecimalMaxMinShort.java
@@ -19,6 +19,7 @@
 
 package org.apache.carbondata.core.datastorage.store.compression.type;
 
+import java.math.BigDecimal;
 import java.nio.ByteBuffer;
 
 import org.apache.carbondata.common.logging.LogService;
@@ -97,7 +98,9 @@ public class UnCompressNonDecimalMaxMinShort
       if (value[i] == 0) {
         vals[i] = maxValue;
       } else {
-        vals[i] = (maxValue - value[i]) / Math.pow(10, decimal);
+        BigDecimal diff = BigDecimal.valueOf(value[i] / Math.pow(10, decimal));
+        BigDecimal max = BigDecimal.valueOf(maxValue);
+        vals[i] = max.subtract(diff).doubleValue();
       }
 
     }

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/276a3ff2/core/src/main/java/org/apache/carbondata/core/util/ValueCompressionUtil.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/util/ValueCompressionUtil.java b/core/src/main/java/org/apache/carbondata/core/util/ValueCompressionUtil.java
index 4a229d6..33c7a72 100644
--- a/core/src/main/java/org/apache/carbondata/core/util/ValueCompressionUtil.java
+++ b/core/src/main/java/org/apache/carbondata/core/util/ValueCompressionUtil.java
@@ -19,6 +19,7 @@
 
 package org.apache.carbondata.core.util;
 
+import java.math.BigDecimal;
 import java.nio.ByteBuffer;
 import java.util.Arrays;
 
@@ -482,13 +483,16 @@ public final class ValueCompressionUtil {
   private static Object compressNonDecimalMaxMin(DataType changedDataType, double[] value,
       int decimal, double maxValue) {
     int i = 0;
+    BigDecimal max = BigDecimal.valueOf(maxValue);
     switch (changedDataType) {
       case DATA_BYTE:
 
         byte[] result = new byte[value.length];
 
         for (double a : value) {
-          result[i] = (byte) (Math.round((maxValue - a) * Math.pow(10, decimal)));
+          BigDecimal val = BigDecimal.valueOf(a);
+          double diff = max.subtract(val).doubleValue();
+          result[i] = (byte) (Math.round(diff * Math.pow(10, decimal)));
           i++;
         }
         return result;
@@ -498,7 +502,9 @@ public final class ValueCompressionUtil {
         short[] shortResult = new short[value.length];
 
         for (double a : value) {
-          shortResult[i] = (byte) (Math.round((maxValue - a) * Math.pow(10, decimal)));
+          BigDecimal val = BigDecimal.valueOf(a);
+          double diff = max.subtract(val).doubleValue();
+          shortResult[i] = (short) (Math.round(diff * Math.pow(10, decimal)));
           i++;
         }
         return shortResult;
@@ -508,7 +514,9 @@ public final class ValueCompressionUtil {
         int[] intResult = new int[value.length];
 
         for (double a : value) {
-          intResult[i] = (byte) (Math.round((maxValue - a) * Math.pow(10, decimal)));
+          BigDecimal val = BigDecimal.valueOf(a);
+          double diff = max.subtract(val).doubleValue();
+          intResult[i] = (int) (Math.round(diff * Math.pow(10, decimal)));
           i++;
         }
         return intResult;
@@ -518,7 +526,9 @@ public final class ValueCompressionUtil {
         long[] longResult = new long[value.length];
 
         for (double a : value) {
-          longResult[i] = (byte) (Math.round((maxValue - a) * Math.pow(10, decimal)));
+          BigDecimal val = BigDecimal.valueOf(a);
+          double diff = max.subtract(val).doubleValue();
+          longResult[i] = (long) (Math.round(diff * Math.pow(10, decimal)));
           i++;
         }
         return longResult;
@@ -528,7 +538,9 @@ public final class ValueCompressionUtil {
         float[] floatResult = new float[value.length];
 
         for (double a : value) {
-          floatResult[i] = (byte) (Math.round((maxValue - a) * Math.pow(10, decimal)));
+          BigDecimal val = BigDecimal.valueOf(a);
+          double diff = max.subtract(val).doubleValue();
+          floatResult[i] = (float) (Math.round(diff * Math.pow(10, decimal)));
           i++;
         }
         return floatResult;
@@ -538,7 +550,9 @@ public final class ValueCompressionUtil {
         double[] defaultResult = new double[value.length];
 
         for (double a : value) {
-          defaultResult[i] = (byte) (Math.round((maxValue - a) * Math.pow(10, decimal)));
+          BigDecimal val = BigDecimal.valueOf(a);
+          double diff = max.subtract(val).doubleValue();
+          defaultResult[i] =  (Math.round(diff * Math.pow(10, decimal)));
           i++;
         }
         return defaultResult;

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/276a3ff2/integration/spark/src/test/scala/org/apache/carbondata/spark/testsuite/detailquery/ValueCompressionDataTypeTestCase.scala
----------------------------------------------------------------------
diff --git a/integration/spark/src/test/scala/org/apache/carbondata/spark/testsuite/detailquery/ValueCompressionDataTypeTestCase.scala b/integration/spark/src/test/scala/org/apache/carbondata/spark/testsuite/detailquery/ValueCompressionDataTypeTestCase.scala
new file mode 100644
index 0000000..5375167
--- /dev/null
+++ b/integration/spark/src/test/scala/org/apache/carbondata/spark/testsuite/detailquery/ValueCompressionDataTypeTestCase.scala
@@ -0,0 +1,97 @@
+/*
+ * 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.spark.testsuite.detailquery
+
+import org.apache.spark.sql.common.util.CarbonHiveContext._
+import org.apache.spark.sql.common.util.QueryTest
+import org.scalatest.BeforeAndAfterAll
+
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastorage.store.impl.FileFactory
+import org.apache.carbondata.core.datastorage.store.impl.FileFactory.FileType
+import org.apache.carbondata.core.util.CarbonProperties
+
+/**
+ * Created by  on 8/8/2016.
+ */
+class ValueCompressionDataTypeTestCase extends QueryTest with BeforeAndAfterAll {
+  val tempDirPath = "./src/test/resources/temp"
+
+  override def beforeAll {
+    FileFactory.mkdirs(tempDirPath,FileType.LOCAL)
+  }
+
+  test("ActualDataType:double,ChangedDatatype:Short,CompressionType:NonDecimalMaxMin") {
+    val tempFilePath = "./src/test/resources/temp/double2short.csv"
+    try {
+      sql("CREATE TABLE double2short (name String, value double) STORED BY 'org.apache.carbondata.format'")
+      sql("CREATE TABLE double2short_hive (name String, value double)row format delimited fields terminated by ','")
+      val data ="a,3.141111\nb,3.141212\nc,3.141313\nd,3.141515\ne,3.141616\nf,3.141616\ng,3.141717\nh,3.141818";
+      writedata(tempFilePath, data)
+      sql(s"LOAD data local inpath '${tempFilePath}' into table double2short options('fileheader'='name,value')")
+      sql(s"LOAD data local inpath '${tempFilePath}' into table double2short_hive")
+      checkAnswer(sql("select * from double2short"),
+        sql("select * from double2short_hive"))
+
+    } catch{
+      case ex:Exception => ex.printStackTrace()
+                           assert(false)
+    } finally {
+      sql("drop table if exists double2short")
+      sql("drop table if exists double2short_hive")
+      deleteFile(tempFilePath)
+    }
+  }
+  
+  test("ActualDataType:double,ChangedDatatype:byte,CompressionType:NonDecimalMaxMin") {
+    val tempFilePath = "./src/test/resources/temp/double2byte.csv"
+    try {
+      sql("CREATE TABLE double2byte (name String, value double) STORED BY 'org.apache.carbondata.format'")
+      sql("CREATE TABLE double2byte_hive (name String, value double)row format delimited fields terminated by ','")
+      val data ="a,4.200001\nb,4.200009";
+      writedata(tempFilePath, data)
+      sql(s"LOAD data local inpath '${tempFilePath}' into table double2byte options('fileheader'='name,value')")
+      sql(s"LOAD data local inpath '${tempFilePath}' into table double2byte_hive")
+      checkAnswer(sql("select * from double2byte"),
+        sql("select * from double2byte_hive"))
+
+    } catch{
+      case ex:Exception => ex.printStackTrace()
+                           assert(false)
+    } finally {
+      sql("drop table if exists double2byte")
+      sql("drop table if exists double2byte_hive")
+      deleteFile(tempFilePath)
+    }
+  }
+
+  def writedata(filePath: String, data: String) = {
+    val dis = FileFactory.getDataOutputStream(filePath, FileFactory.getFileType(filePath))
+    dis.writeBytes(data.toString())
+    dis.close()
+  }
+  def deleteFile(filePath: String) {
+    val file = FileFactory.getCarbonFile(filePath, FileFactory.getFileType(filePath))
+    file.delete()
+  }
+
+  override def afterAll {
+    deleteFile(tempDirPath)
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/276a3ff2/processing/src/main/java/org/apache/carbondata/processing/store/CarbonFactDataHandlerColumnar.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/store/CarbonFactDataHandlerColumnar.java b/processing/src/main/java/org/apache/carbondata/processing/store/CarbonFactDataHandlerColumnar.java
index 0d63a8e..4f6a2ba 100644
--- a/processing/src/main/java/org/apache/carbondata/processing/store/CarbonFactDataHandlerColumnar.java
+++ b/processing/src/main/java/org/apache/carbondata/processing/store/CarbonFactDataHandlerColumnar.java
@@ -878,6 +878,19 @@ public class CarbonFactDataHandlerColumnar implements CarbonFactHandler {
   }
 
   /**
+   * @param double value
+   * @return it return no of value after decimal
+   */
+  private int getDecimalCount(double value) {
+    String strValue = BigDecimal.valueOf(Math.abs(value)).toPlainString();
+    int integerPlaces = strValue.indexOf('.');
+    int decimalPlaces = 0;
+    if (-1 != integerPlaces) {
+      decimalPlaces = strValue.length() - integerPlaces - 1;
+    }
+    return decimalPlaces;
+  }
+  /**
    * This method will be used to update the max value for each measure
    */
   private void calculateMaxMin(Object[] max, Object[] min, int[] decimal, int[] msrIndex,
@@ -892,7 +905,7 @@ public class CarbonFactDataHandlerColumnar implements CarbonFactHandler {
           double minVal = (double) min[count];
           max[count] = (maxVal > value ? max[count] : value);
           min[count] = (minVal < value ? min[count] : value);
-          int num = (value % 1 == 0) ? 0 : CarbonCommonConstants.CARBON_DECIMAL_POINTERS_DEFAULT;
+          int num = getDecimalCount(value);
           decimal[count] = (decimal[count] > num ? decimal[count] : num);
         } else if (type[count] == CarbonCommonConstants.BIG_INT_MEASURE) {
           long value = (long) row[count];
@@ -900,7 +913,7 @@ public class CarbonFactDataHandlerColumnar implements CarbonFactHandler {
           long minVal = (long) min[count];
           max[count] = (maxVal > value ? max[count] : value);
           min[count] = (minVal < value ? min[count] : value);
-          int num = (value % 1 == 0) ? 0 : CarbonCommonConstants.CARBON_DECIMAL_POINTERS_DEFAULT;
+          int num = getDecimalCount(value);
           decimal[count] = (decimal[count] > num ? decimal[count] : num);
         } else if (type[count] == CarbonCommonConstants.BIG_DECIMAL_MEASURE) {
           byte[] buff = null;


[2/2] incubator-carbondata git commit: [CARBONDATA-152]Double min max difference compression issue. This closes #95

Posted by ra...@apache.org.
[CARBONDATA-152]Double min max difference compression issue. This closes #95


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

Branch: refs/heads/master
Commit: 6537316de0eed035cc4c062560831d500e3d40ed
Parents: 4275277 276a3ff
Author: ravipesala <ra...@gmail.com>
Authored: Tue Aug 30 21:05:31 2016 +0530
Committer: ravipesala <ra...@gmail.com>
Committed: Tue Aug 30 21:05:31 2016 +0530

----------------------------------------------------------------------
 .../type/UnCompressNonDecimalMaxMinByte.java    |  6 +-
 .../type/UnCompressNonDecimalMaxMinDefault.java |  5 +-
 .../type/UnCompressNonDecimalMaxMinFloat.java   |  5 +-
 .../type/UnCompressNonDecimalMaxMinInt.java     |  5 +-
 .../type/UnCompressNonDecimalMaxMinLong.java    |  5 +-
 .../type/UnCompressNonDecimalMaxMinShort.java   |  5 +-
 .../core/util/ValueCompressionUtil.java         | 26 ++++--
 .../ValueCompressionDataTypeTestCase.scala      | 97 ++++++++++++++++++++
 .../store/CarbonFactDataHandlerColumnar.java    | 17 +++-
 9 files changed, 157 insertions(+), 14 deletions(-)
----------------------------------------------------------------------