You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@carbondata.apache.org by ja...@apache.org on 2018/08/30 08:24:47 UTC

carbondata git commit: [CARBONDATA-2884] Rename the methods of ByteUtil class to avoid the misuse

Repository: carbondata
Updated Branches:
  refs/heads/master 612552e61 -> f012f5b13


[CARBONDATA-2884] Rename the methods of ByteUtil class to avoid the misuse

The method toBytes will execute XOR operation on data.
So the result is not the byte array of the real value.
Better to rename the methods of ByteUtil class to avoid the misuse

This closes #2657


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

Branch: refs/heads/master
Commit: f012f5b13ec97576f5f6935dd13747ac4322fd14
Parents: 612552e
Author: QiangCai <qi...@qq.com>
Authored: Fri Aug 24 09:56:43 2018 +0800
Committer: Jacky Li <ja...@qq.com>
Committed: Thu Aug 30 16:24:25 2018 +0800

----------------------------------------------------------------------
 .../chunk/store/ColumnPageWrapper.java          |  16 +-
 ...feVariableLengthDimensionDataChunkStore.java |   6 +-
 ...feVariableLengthDimensionDataChunkStore.java |   6 +-
 .../scan/complextypes/PrimitiveQueryType.java   |   2 +-
 .../scan/executor/util/RestructureUtil.java     |   2 +-
 .../executer/RangeValueFilterExecuterImpl.java  |   2 +-
 ...velRangeLessThanEqualFilterExecuterImpl.java |   2 +-
 ...RowLevelRangeLessThanFilterExecuterImpl.java |   2 +-
 .../apache/carbondata/core/util/ByteUtil.java   |  59 +++-
 .../carbondata/core/util/DataTypeUtil.java      |  36 +-
 .../carbondata/core/util/ByteUtilTest.java      | 329 ++++++++++++-------
 .../processing/datatypes/PrimitiveDataType.java |   8 +-
 12 files changed, 307 insertions(+), 163 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/carbondata/blob/f012f5b1/core/src/main/java/org/apache/carbondata/core/datastore/chunk/store/ColumnPageWrapper.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/datastore/chunk/store/ColumnPageWrapper.java b/core/src/main/java/org/apache/carbondata/core/datastore/chunk/store/ColumnPageWrapper.java
index a5d5917..65991a5 100644
--- a/core/src/main/java/org/apache/carbondata/core/datastore/chunk/store/ColumnPageWrapper.java
+++ b/core/src/main/java/org/apache/carbondata/core/datastore/chunk/store/ColumnPageWrapper.java
@@ -82,9 +82,9 @@ public class ColumnPageWrapper implements DimensionColumnPage {
         double doubleData = columnPage.getDouble(rowId);
         if (srcDataType == DataTypes.FLOAT) {
           float out = (float) doubleData;
-          return ByteUtil.toBytes(out);
+          return ByteUtil.toXorBytes(out);
         } else {
-          return ByteUtil.toBytes(doubleData);
+          return ByteUtil.toXorBytes(doubleData);
         }
       } else if (DataTypes.isDecimal(srcDataType)) {
         throw new RuntimeException("unsupported type: " + srcDataType);
@@ -95,22 +95,22 @@ public class ColumnPageWrapper implements DimensionColumnPage {
         long longData = columnPage.getLong(rowId);
         if ((srcDataType == DataTypes.BYTE)) {
           byte out = (byte) longData;
-          return ByteUtil.toBytes(out);
+          return ByteUtil.toXorBytes(out);
         } else if (srcDataType == DataTypes.BOOLEAN) {
           byte out = (byte) longData;
           return ByteUtil.toBytes(ByteUtil.toBoolean(out));
         } else if (srcDataType == DataTypes.SHORT) {
           short out = (short) longData;
-          return ByteUtil.toBytes(out);
+          return ByteUtil.toXorBytes(out);
         } else if (srcDataType == DataTypes.SHORT_INT) {
           int out = (int) longData;
-          return ByteUtil.toBytes(out);
+          return ByteUtil.toXorBytes(out);
         } else if (srcDataType == DataTypes.INT) {
           int out = (int) longData;
-          return ByteUtil.toBytes(out);
+          return ByteUtil.toXorBytes(out);
         } else {
           // timestamp and long
-          return ByteUtil.toBytes(longData);
+          return ByteUtil.toXorBytes(longData);
         }
       } else if ((targetDataType == DataTypes.STRING) || (targetDataType == DataTypes.VARCHAR) || (
           targetDataType == DataTypes.BYTE_ARRAY)) {
@@ -126,7 +126,7 @@ public class ColumnPageWrapper implements DimensionColumnPage {
       } else if (srcDataType == DataTypes.BYTE_ARRAY) {
         return columnPage.getBytes(rowId);
       }  else if (srcDataType == DataTypes.DOUBLE) {
-        return ByteUtil.toBytes(columnPage.getDouble(rowId));
+        return ByteUtil.toXorBytes(columnPage.getDouble(rowId));
       } else {
         throw new RuntimeException("unsupported type: " + targetDataType);
       }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/f012f5b1/core/src/main/java/org/apache/carbondata/core/datastore/chunk/store/impl/safe/SafeVariableLengthDimensionDataChunkStore.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/datastore/chunk/store/impl/safe/SafeVariableLengthDimensionDataChunkStore.java b/core/src/main/java/org/apache/carbondata/core/datastore/chunk/store/impl/safe/SafeVariableLengthDimensionDataChunkStore.java
index 52e7317..196dc4c 100644
--- a/core/src/main/java/org/apache/carbondata/core/datastore/chunk/store/impl/safe/SafeVariableLengthDimensionDataChunkStore.java
+++ b/core/src/main/java/org/apache/carbondata/core/datastore/chunk/store/impl/safe/SafeVariableLengthDimensionDataChunkStore.java
@@ -154,15 +154,15 @@ public abstract class SafeVariableLengthDimensionDataChunkStore
       } else if (dt == DataTypes.BOOLEAN) {
         vector.putBoolean(vectorRow, ByteUtil.toBoolean(data[currentDataOffset]));
       } else if (dt == DataTypes.SHORT) {
-        vector.putShort(vectorRow, ByteUtil.toShort(data, currentDataOffset, length));
+        vector.putShort(vectorRow, ByteUtil.toXorShort(data, currentDataOffset, length));
       } else if (dt == DataTypes.INT) {
-        vector.putInt(vectorRow, ByteUtil.toInt(data, currentDataOffset, length));
+        vector.putInt(vectorRow, ByteUtil.toXorInt(data, currentDataOffset, length));
       } else if (dt == DataTypes.LONG) {
         vector.putLong(vectorRow,
             DataTypeUtil.getDataBasedOnRestructuredDataType(data, vector.getBlockDataType(),
                 currentDataOffset, length));
       } else if (dt  == DataTypes.TIMESTAMP) {
-        vector.putLong(vectorRow, ByteUtil.toLong(data, currentDataOffset, length) * 1000L);
+        vector.putLong(vectorRow, ByteUtil.toXorLong(data, currentDataOffset, length) * 1000L);
       }
     }
   }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/f012f5b1/core/src/main/java/org/apache/carbondata/core/datastore/chunk/store/impl/unsafe/UnsafeVariableLengthDimensionDataChunkStore.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/datastore/chunk/store/impl/unsafe/UnsafeVariableLengthDimensionDataChunkStore.java b/core/src/main/java/org/apache/carbondata/core/datastore/chunk/store/impl/unsafe/UnsafeVariableLengthDimensionDataChunkStore.java
index 801a282..954cab2 100644
--- a/core/src/main/java/org/apache/carbondata/core/datastore/chunk/store/impl/unsafe/UnsafeVariableLengthDimensionDataChunkStore.java
+++ b/core/src/main/java/org/apache/carbondata/core/datastore/chunk/store/impl/unsafe/UnsafeVariableLengthDimensionDataChunkStore.java
@@ -247,15 +247,15 @@ public abstract class UnsafeVariableLengthDimensionDataChunkStore
       } else if (dt == DataTypes.BOOLEAN) {
         vector.putBoolean(vectorRow, ByteUtil.toBoolean(value[0]));
       } else if (dt == DataTypes.SHORT) {
-        vector.putShort(vectorRow, ByteUtil.toShort(value, 0, length));
+        vector.putShort(vectorRow, ByteUtil.toXorShort(value, 0, length));
       } else if (dt == DataTypes.INT) {
-        vector.putInt(vectorRow, ByteUtil.toInt(value, 0, length));
+        vector.putInt(vectorRow, ByteUtil.toXorInt(value, 0, length));
       } else if (dt == DataTypes.LONG) {
         vector.putLong(vectorRow,
             DataTypeUtil.getDataBasedOnRestructuredDataType(value, vector.getBlockDataType(), 0,
                 length));
       } else if (dt == DataTypes.TIMESTAMP) {
-        vector.putLong(vectorRow, ByteUtil.toLong(value, 0, length) * 1000L);
+        vector.putLong(vectorRow, ByteUtil.toXorLong(value, 0, length) * 1000L);
       }
     }
   }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/f012f5b1/core/src/main/java/org/apache/carbondata/core/scan/complextypes/PrimitiveQueryType.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/scan/complextypes/PrimitiveQueryType.java b/core/src/main/java/org/apache/carbondata/core/scan/complextypes/PrimitiveQueryType.java
index b3f13d7..abe33c4 100644
--- a/core/src/main/java/org/apache/carbondata/core/scan/complextypes/PrimitiveQueryType.java
+++ b/core/src/main/java/org/apache/carbondata/core/scan/complextypes/PrimitiveQueryType.java
@@ -155,7 +155,7 @@ public class PrimitiveQueryType extends ComplexQueryType implements GenericQuery
           actualData = null;
         } else {
           actualData = this.directDictGenForDate.getValueFromSurrogate(
-              ByteUtil.toInt(value, 0, CarbonCommonConstants.INT_SIZE_IN_BYTE));
+              ByteUtil.toXorInt(value, 0, CarbonCommonConstants.INT_SIZE_IN_BYTE));
         }
       } else {
         actualData = DataTypeUtil.getDataBasedOnDataTypeForNoDictionaryColumn(value, this.dataType);

http://git-wip-us.apache.org/repos/asf/carbondata/blob/f012f5b1/core/src/main/java/org/apache/carbondata/core/scan/executor/util/RestructureUtil.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/scan/executor/util/RestructureUtil.java b/core/src/main/java/org/apache/carbondata/core/scan/executor/util/RestructureUtil.java
index a74b73c..9fe2a38 100644
--- a/core/src/main/java/org/apache/carbondata/core/scan/executor/util/RestructureUtil.java
+++ b/core/src/main/java/org/apache/carbondata/core/scan/executor/util/RestructureUtil.java
@@ -296,7 +296,7 @@ public class RestructureUtil {
         value = new String(defaultValue, Charset.forName(CarbonCommonConstants.DEFAULT_CHARSET));
         noDictionaryDefaultValue = Long.parseLong(value);
       } else if (datatype == DataTypes.TIMESTAMP) {
-        long timestampValue = ByteUtil.toLong(defaultValue, 0, defaultValue.length);
+        long timestampValue = ByteUtil.toXorLong(defaultValue, 0, defaultValue.length);
         noDictionaryDefaultValue = timestampValue * 1000L;
       } else {
         noDictionaryDefaultValue =

http://git-wip-us.apache.org/repos/asf/carbondata/blob/f012f5b1/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/RangeValueFilterExecuterImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/RangeValueFilterExecuterImpl.java b/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/RangeValueFilterExecuterImpl.java
index 78055fe..9718dcf 100644
--- a/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/RangeValueFilterExecuterImpl.java
+++ b/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/RangeValueFilterExecuterImpl.java
@@ -615,7 +615,7 @@ public class RangeValueFilterExecuterImpl implements FilterExecuter {
           defaultValue = FilterUtil.getMaskKey(key, currentBlockDimension,
               this.segmentProperties.getSortColumnsGenerator());
         } else {
-          defaultValue = ByteUtil.toBytes(key);
+          defaultValue = ByteUtil.toXorBytes(key);
         }
       } else {
         if (dimColEvaluatorInfo.getDimension().getDataType() == DataTypes.STRING) {

http://git-wip-us.apache.org/repos/asf/carbondata/blob/f012f5b1/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/RowLevelRangeLessThanEqualFilterExecuterImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/RowLevelRangeLessThanEqualFilterExecuterImpl.java b/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/RowLevelRangeLessThanEqualFilterExecuterImpl.java
index b8f3cb1..f410322 100644
--- a/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/RowLevelRangeLessThanEqualFilterExecuterImpl.java
+++ b/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/RowLevelRangeLessThanEqualFilterExecuterImpl.java
@@ -323,7 +323,7 @@ public class RowLevelRangeLessThanEqualFilterExecuterImpl extends RowLevelFilter
         defaultValue = FilterUtil.getMaskKey(key, currentBlockDimension,
             this.segmentProperties.getSortColumnsGenerator());
       } else {
-        defaultValue = ByteUtil.toBytes(key);
+        defaultValue = ByteUtil.toXorBytes(key);
       }
     } else if (dimColEvaluatorInfoList.get(0).getDimension().getDataType() != DataTypes.STRING) {
       defaultValue = CarbonCommonConstants.EMPTY_BYTE_ARRAY;

http://git-wip-us.apache.org/repos/asf/carbondata/blob/f012f5b1/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/RowLevelRangeLessThanFilterExecuterImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/RowLevelRangeLessThanFilterExecuterImpl.java b/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/RowLevelRangeLessThanFilterExecuterImpl.java
index 15f3e21..1f3b9a4 100644
--- a/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/RowLevelRangeLessThanFilterExecuterImpl.java
+++ b/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/RowLevelRangeLessThanFilterExecuterImpl.java
@@ -322,7 +322,7 @@ public class RowLevelRangeLessThanFilterExecuterImpl extends RowLevelFilterExecu
         defaultValue = FilterUtil.getMaskKey(key, currentBlockDimension,
             this.segmentProperties.getSortColumnsGenerator());
       } else {
-        defaultValue = ByteUtil.toBytes(key);
+        defaultValue = ByteUtil.toXorBytes(key);
       }
     } else if (dimColEvaluatorInfoList.get(0).getDimension().getDataType() != DataTypes.STRING) {
       defaultValue = CarbonCommonConstants.EMPTY_BYTE_ARRAY;

http://git-wip-us.apache.org/repos/asf/carbondata/blob/f012f5b1/core/src/main/java/org/apache/carbondata/core/util/ByteUtil.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/util/ByteUtil.java b/core/src/main/java/org/apache/carbondata/core/util/ByteUtil.java
index 322c80a..3473aca 100644
--- a/core/src/main/java/org/apache/carbondata/core/util/ByteUtil.java
+++ b/core/src/main/java/org/apache/carbondata/core/util/ByteUtil.java
@@ -414,7 +414,6 @@ public final class ByteUtil {
    * @return
    */
   public static byte[] toBytes(short val) {
-    val = (short)(val ^ Short.MIN_VALUE);
     byte[] b = new byte[SIZEOF_SHORT];
     b[1] = (byte) val;
     val >>= 8;
@@ -448,7 +447,7 @@ public final class ByteUtil {
       n <<= 8;
       n ^= bytes[offset + 1] & 0xFF;
     }
-    return (short)(n ^ Short.MIN_VALUE);
+    return n;
   }
 
   /**
@@ -458,7 +457,6 @@ public final class ByteUtil {
    * @return
    */
   public static byte[] toBytes(int val) {
-    val = val ^ Integer.MIN_VALUE;
     byte[] b = new byte[4];
     for (int i = 3; i > 0; i--) {
       b[i] = (byte) val;
@@ -519,7 +517,7 @@ public final class ByteUtil {
         n ^= bytes[i] & 0xFF;
       }
     }
-    return n ^ Integer.MIN_VALUE;
+    return n;
   }
 
   public static int toInt(byte[] bytes, int offset) {
@@ -550,7 +548,6 @@ public final class ByteUtil {
    * @return
    */
   public static byte[] toBytes(long val) {
-    val = val ^ Long.MIN_VALUE;
     byte[] b = new byte[8];
     for (int i = 7; i > 0; i--) {
       b[i] = (byte) val;
@@ -589,7 +586,7 @@ public final class ByteUtil {
         l ^= bytes[i] & 0xFF;
       }
     }
-    return l ^ Long.MIN_VALUE;
+    return l;
   }
 
   private static IllegalArgumentException explainWrongLengthOrOffset(final byte[] bytes,
@@ -669,4 +666,54 @@ public final class ByteUtil {
     return flattenedData;
   }
 
+  /**
+   * If number type column is in sort_columns, the column will be no-dictionary column.
+   * It will compare byte arrays to sort the data.
+   * For example the binary string of int value as follows.
+   * 1  : 00000000 00000000 00000000 00000001
+   * -1 : 11111111 11111111 11111111 11111111
+   * In this case, the compare method of byte arrays will return a wrong result.(1 < -1)
+   * The root cause is that the sign bit of negative number is 1.
+   * These XOR methods will change the sign bit as follows.
+   * 1  ^ MIN_VALUE : 10000000 00000000 00000000 00000001
+   * -1 ^ MIN_VALUE : 01111111 11111111 11111111 11111111
+   * After the transform, the compare method of byte arrays will return a right result.(1 > -1)
+   */
+  public static byte[] toXorBytes(short val) {
+    val = (short) (val ^ Short.MIN_VALUE);
+    return toBytes(val);
+  }
+
+  public static byte[] toXorBytes(int val) {
+    val = val ^ Integer.MIN_VALUE;
+    return toBytes(val);
+  }
+
+  public static byte[] toXorBytes(long val) {
+    val = val ^ Long.MIN_VALUE;
+    return toBytes(val);
+  }
+
+  public static byte[] toXorBytes(double val) {
+    return toXorBytes(Double.doubleToLongBits(val));
+  }
+
+  /**
+   * The following methods convert byte array back to the real value.
+   */
+  public static short toXorShort(byte[] bytes, int offset, final int length) {
+    return (short) (toShort(bytes, offset, length) ^ Short.MIN_VALUE);
+  }
+
+  public static int toXorInt(byte[] bytes, int offset, final int length) {
+    return toInt(bytes, offset, length) ^ Integer.MIN_VALUE;
+  }
+
+  public static long toXorLong(byte[] bytes, int offset, final int length) {
+    return toLong(bytes, offset, length) ^ Long.MIN_VALUE;
+  }
+
+  public static double toXorDouble(byte[] value, int offset, int length) {
+    return Double.longBitsToDouble(toXorLong(value, offset, length));
+  }
 }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/f012f5b1/core/src/main/java/org/apache/carbondata/core/util/DataTypeUtil.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/util/DataTypeUtil.java b/core/src/main/java/org/apache/carbondata/core/util/DataTypeUtil.java
index 54b7441..fa08df9 100644
--- a/core/src/main/java/org/apache/carbondata/core/util/DataTypeUtil.java
+++ b/core/src/main/java/org/apache/carbondata/core/util/DataTypeUtil.java
@@ -324,13 +324,13 @@ public final class DataTypeUtil {
     if (actualDataType == DataTypes.BOOLEAN) {
       return ByteUtil.toBytes(BooleanConvert.parseBoolean(dimensionValue));
     } else if (actualDataType == DataTypes.SHORT) {
-      return ByteUtil.toBytes(Short.parseShort(dimensionValue));
+      return ByteUtil.toXorBytes(Short.parseShort(dimensionValue));
     } else if (actualDataType == DataTypes.INT) {
-      return ByteUtil.toBytes(Integer.parseInt(dimensionValue));
+      return ByteUtil.toXorBytes(Integer.parseInt(dimensionValue));
     } else if (actualDataType == DataTypes.LONG) {
-      return ByteUtil.toBytes(Long.parseLong(dimensionValue));
+      return ByteUtil.toXorBytes(Long.parseLong(dimensionValue));
     } else if (actualDataType == DataTypes.DOUBLE) {
-      return ByteUtil.toBytes(Double.parseDouble(dimensionValue));
+      return ByteUtil.toXorBytes(Double.parseDouble(dimensionValue));
     } else if (DataTypes.isDecimal(actualDataType)) {
       return bigDecimalToByte(new BigDecimal(dimensionValue));
     } else if (actualDataType == DataTypes.TIMESTAMP) {
@@ -344,7 +344,7 @@ public final class DataTypeUtil {
           dateFormatter = timeStampformatter.get();
         }
         dateToStr = dateFormatter.parse(dimensionValue);
-        return ByteUtil.toBytes(dateToStr.getTime());
+        return ByteUtil.toXorBytes(dateToStr.getTime());
       } catch (ParseException e) {
         throw new NumberFormatException(e.getMessage());
       }
@@ -401,13 +401,13 @@ public final class DataTypeUtil {
     if (actualDataType == DataTypes.BOOLEAN) {
       return ByteUtil.toBytes((Boolean) dimensionValue);
     } else if (actualDataType == DataTypes.SHORT) {
-      return ByteUtil.toBytes((Short) dimensionValue);
+      return ByteUtil.toXorBytes((Short) dimensionValue);
     } else if (actualDataType == DataTypes.INT) {
-      return ByteUtil.toBytes((Integer) dimensionValue);
+      return ByteUtil.toXorBytes((Integer) dimensionValue);
     } else if (actualDataType == DataTypes.LONG) {
-      return ByteUtil.toBytes((Long) dimensionValue);
+      return ByteUtil.toXorBytes((Long) dimensionValue);
     } else if (actualDataType == DataTypes.TIMESTAMP) {
-      return ByteUtil.toBytes((Long)dimensionValue);
+      return ByteUtil.toXorBytes((Long)dimensionValue);
     } else {
       // Default action for String/Varchar
       return ByteUtil.toBytes(dimensionValue.toString());
@@ -465,31 +465,31 @@ public final class DataTypeUtil {
         if (isEmptyByteArray(dataInBytes)) {
           return null;
         }
-        return ByteUtil.toShort(dataInBytes, 0, dataInBytes.length);
+        return ByteUtil.toXorShort(dataInBytes, 0, dataInBytes.length);
       } else if (actualDataType == DataTypes.INT) {
         if (isEmptyByteArray(dataInBytes)) {
           return null;
         }
-        return ByteUtil.toInt(dataInBytes, 0, dataInBytes.length);
+        return ByteUtil.toXorInt(dataInBytes, 0, dataInBytes.length);
       } else if (actualDataType == DataTypes.LONG) {
         if (isEmptyByteArray(dataInBytes)) {
           return null;
         }
-        return ByteUtil.toLong(dataInBytes, 0, dataInBytes.length);
+        return ByteUtil.toXorLong(dataInBytes, 0, dataInBytes.length);
       } else if (actualDataType == DataTypes.TIMESTAMP) {
         if (isEmptyByteArray(dataInBytes)) {
           return null;
         }
         if (isTimeStampConversion) {
-          return ByteUtil.toLong(dataInBytes, 0, dataInBytes.length) * 1000L;
+          return ByteUtil.toXorLong(dataInBytes, 0, dataInBytes.length) * 1000L;
         } else {
-          return ByteUtil.toLong(dataInBytes, 0, dataInBytes.length);
+          return ByteUtil.toXorLong(dataInBytes, 0, dataInBytes.length);
         }
       } else if (actualDataType == DataTypes.DOUBLE) {
         if (isEmptyByteArray(dataInBytes)) {
           return null;
         }
-        return ByteUtil.toDouble(dataInBytes, 0, dataInBytes.length);
+        return ByteUtil.toXorDouble(dataInBytes, 0, dataInBytes.length);
       } else if (DataTypes.isDecimal(actualDataType)) {
         if (isEmptyByteArray(dataInBytes)) {
           return null;
@@ -774,7 +774,7 @@ public final class DataTypeUtil {
           try {
             timeStampformatter.remove();
             Date dateToStr = timeStampformatter.get().parse(data);
-            return ByteUtil.toBytes(dateToStr.getTime());
+            return ByteUtil.toXorBytes(dateToStr.getTime());
           } catch (ParseException e) {
             LOGGER.error(
                 "Cannot convert value to Time/Long type value. Value is considered as null" + e
@@ -969,9 +969,9 @@ public final class DataTypeUtil {
       int currentDataOffset, int length) {
     long value = 0L;
     if (restructuredDataType == DataTypes.INT) {
-      value = ByteUtil.toInt(data, currentDataOffset, length);
+      value = ByteUtil.toXorInt(data, currentDataOffset, length);
     } else if (restructuredDataType == DataTypes.LONG) {
-      value = ByteUtil.toLong(data, currentDataOffset, length);
+      value = ByteUtil.toXorLong(data, currentDataOffset, length);
     }
     return value;
   }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/f012f5b1/core/src/test/java/org/apache/carbondata/core/util/ByteUtilTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/carbondata/core/util/ByteUtilTest.java b/core/src/test/java/org/apache/carbondata/core/util/ByteUtilTest.java
index d93aa49..c07de2a 100644
--- a/core/src/test/java/org/apache/carbondata/core/util/ByteUtilTest.java
+++ b/core/src/test/java/org/apache/carbondata/core/util/ByteUtilTest.java
@@ -17,156 +17,253 @@
 package org.apache.carbondata.core.util;
 
 import junit.framework.TestCase;
+
 import org.apache.carbondata.core.constants.CarbonCommonConstants;
 import org.apache.carbondata.core.util.ByteUtil.UnsafeComparer;
+
 import org.junit.Before;
 import org.junit.Test;
 
 import java.nio.ByteBuffer;
 import java.nio.charset.Charset;
 
-
 /**
  * This test will test the functionality of the Byte Util
  * for the comparision of 2 byte buffers
  */
 public class ByteUtilTest extends TestCase {
 
-    String dimensionValue1 = "aaaaaaaa1235";
-    String dimensionValue2 = "aaaaaaaa1234";
-    private ByteBuffer buff1;
-    private ByteBuffer buff2;
-
-    /**
-     * This method will form one single byte [] for all the high card dims.
-     *
-     * @param byteBufferArr
-     * @return
-     */
-    public static byte[] packByteBufferIntoSingleByteArray(
-            ByteBuffer[] byteBufferArr) {
-        // for empty array means there is no data to remove dictionary.
-        if (null == byteBufferArr || byteBufferArr.length == 0) {
-            return null;
-        }
-        int noOfCol = byteBufferArr.length;
-        short toDetermineLengthOfByteArr = 2;
-        short offsetLen = (short) (noOfCol * 2 + toDetermineLengthOfByteArr);
-        int totalBytes = calculateTotalBytes(byteBufferArr) + offsetLen;
-
-        ByteBuffer buffer = ByteBuffer.allocate(totalBytes);
-
-        // write the length of the byte [] as first short
-        buffer.putShort((short) (totalBytes - toDetermineLengthOfByteArr));
-        // writing the offset of the first element.
-        buffer.putShort(offsetLen);
-
-        // prepare index for byte []
-        for (int index = 0; index < byteBufferArr.length - 1; index++) {
-            ByteBuffer individualCol = byteBufferArr[index];
-            // short lengthOfbytes = individualCol.getShort();
-            int noOfBytes = individualCol.capacity();
-
-            buffer.putShort((short) (offsetLen + noOfBytes));
-            offsetLen += noOfBytes;
-            individualCol.rewind();
-        }
-
-        // put actual data.
-        for (int index = 0; index < byteBufferArr.length; index++) {
-            ByteBuffer individualCol = byteBufferArr[index];
-            buffer.put(individualCol.array());
-        }
-
-        buffer.rewind();
-        return buffer.array();
+  String dimensionValue1 = "aaaaaaaa1235";
+  String dimensionValue2 = "aaaaaaaa1234";
+  private ByteBuffer buff1;
+  private ByteBuffer buff2;
 
+  /**
+   * This method will form one single byte [] for all the high card dims.
+   *
+   * @param byteBufferArr ByteBuffer array
+   * @return byte array
+   */
+  public static byte[] packByteBufferIntoSingleByteArray(ByteBuffer[] byteBufferArr) {
+    // for empty array means there is no data to remove dictionary.
+    if (null == byteBufferArr || byteBufferArr.length == 0) {
+      return null;
     }
+    int noOfCol = byteBufferArr.length;
+    short toDetermineLengthOfByteArr = 2;
+    short offsetLen = (short) (noOfCol * 2 + toDetermineLengthOfByteArr);
+    int totalBytes = calculateTotalBytes(byteBufferArr) + offsetLen;
 
-    /**
-     * To calculate the total bytes in byte Buffer[].
-     *
-     * @param byteBufferArr
-     * @return
-     */
-    private static int calculateTotalBytes(ByteBuffer[] byteBufferArr) {
-        int total = 0;
-        for (int index = 0; index < byteBufferArr.length; index++) {
-            total += byteBufferArr[index].capacity();
-        }
-        return total;
-    }
+    ByteBuffer buffer = ByteBuffer.allocate(totalBytes);
 
-    /**
-     * @throws Exception
-     */
-    @Before
-    public void setUp() throws Exception {
+    // write the length of the byte [] as first short
+    buffer.putShort((short) (totalBytes - toDetermineLengthOfByteArr));
+    // writing the offset of the first element.
+    buffer.putShort(offsetLen);
 
-    }
-
-    @Test
-    public void testLessThan() {
-        dimensionValue1 = "aaaaa6aa1235";
-        dimensionValue2 = "aaaaa5aa1234";
+    // prepare index for byte []
+    for (int index = 0; index < byteBufferArr.length - 1; index++) {
+      ByteBuffer individualCol = byteBufferArr[index];
+      // short lengthOfbytes = individualCol.getShort();
+      int noOfBytes = individualCol.capacity();
 
-        prepareBuffers();
-        assertFalse(UnsafeComparer.INSTANCE.compareTo(buff1, buff2) < 0);
+      buffer.putShort((short) (offsetLen + noOfBytes));
+      offsetLen += noOfBytes;
+      individualCol.rewind();
     }
-    @Test
-    public void testIntConversion() {
-        byte[] data = new byte[4];
-        ByteUtil.setInt(data, 0, 968);
-        assertEquals(ByteUtil.toInt(data, 0), 968);
+
+    // put actual data.
+    for (int index = 0; index < byteBufferArr.length; index++) {
+      ByteBuffer individualCol = byteBufferArr[index];
+      buffer.put(individualCol.array());
     }
 
-    @Test
-    public void testEqualToCase() {
-        dimensionValue1 = "aaaaaaaa1234";
-        dimensionValue2 = "aaaaaaaa1234";
+    buffer.rewind();
+    return buffer.array();
 
-        prepareBuffers();
-        assertTrue(UnsafeComparer.INSTANCE.compareTo(buff1, buff2) == 0);
+  }
+
+  /**
+   * To calculate the total bytes in byte Buffer[].
+   *
+   * @param byteBufferArr
+   * @return
+   */
+  private static int calculateTotalBytes(ByteBuffer[] byteBufferArr) {
+    int total = 0;
+    for (int index = 0; index < byteBufferArr.length; index++) {
+      total += byteBufferArr[index].capacity();
     }
+    return total;
+  }
 
-    @Test
-    public void testLessThanInBoundaryCondition() {
-        dimensionValue1 = "aaaaaaaa12341";
-        dimensionValue2 = "aaaaaaaa12344";
+  /**
+   * @throws Exception
+   */
+  @Before
+  public void setUp() throws Exception {
 
-        prepareBuffers();
-        assertTrue(UnsafeComparer.INSTANCE.compareTo(buff1, buff2) < 0);
-    }
+  }
 
-    /**
-     * This will prepare the byte buffers in the required format for comparision.
-     */
-    private void prepareBuffers() {
-        ByteBuffer[] out1 = new ByteBuffer[1];
-        ByteBuffer buffer = ByteBuffer.allocate(dimensionValue1.length());
-        buffer.put(dimensionValue1.getBytes(Charset.forName(CarbonCommonConstants.DEFAULT_CHARSET)));
-        buffer.rewind();
-        out1[0] = buffer;
+  @Test
+  public void testLessThan() {
+    dimensionValue1 = "aaaaa6aa1235";
+    dimensionValue2 = "aaaaa5aa1234";
 
+    prepareBuffers();
+    assertFalse(UnsafeComparer.INSTANCE.compareTo(buff1, buff2) < 0);
+  }
 
-        ByteBuffer[] out2 = new ByteBuffer[1];
+  @Test
+  public void testIntConversion() {
+    byte[] data = new byte[4];
+    ByteUtil.setInt(data, 0, 968);
+    assertEquals(ByteUtil.toInt(data, 0), 968);
+  }
 
-        ByteBuffer buffer2 = ByteBuffer.allocate(dimensionValue2.length());
-        buffer2.put(dimensionValue2.getBytes(Charset.forName(CarbonCommonConstants.DEFAULT_CHARSET)));
-        buffer2.rewind();
-        out2[0] = buffer2;
+  @Test
+  public void testEqualToCase() {
+    dimensionValue1 = "aaaaaaaa1234";
+    dimensionValue2 = "aaaaaaaa1234";
 
-        byte[] arr1 = packByteBufferIntoSingleByteArray(out1);
-        byte[] arr2 = packByteBufferIntoSingleByteArray(out2);
+    prepareBuffers();
+    assertTrue(UnsafeComparer.INSTANCE.compareTo(buff1, buff2) == 0);
+  }
 
-        buff1 = ByteBuffer.wrap(arr1);
+  @Test
+  public void testLessThanInBoundaryCondition() {
+    dimensionValue1 = "aaaaaaaa12341";
+    dimensionValue2 = "aaaaaaaa12344";
 
-        buff1.position(4);
-        buff1.limit(buff1.position() + dimensionValue1.length());
+    prepareBuffers();
+    assertTrue(UnsafeComparer.INSTANCE.compareTo(buff1, buff2) < 0);
+  }
 
-        buff2 = ByteBuffer.wrap(arr2);
-        buff2.position(4);
-        buff2.limit(buff2.position() + dimensionValue2.length());
-    }
+  /**
+   * This will prepare the byte buffers in the required format for comparision.
+   */
+  private void prepareBuffers() {
+    ByteBuffer[] out1 = new ByteBuffer[1];
+    ByteBuffer buffer = ByteBuffer.allocate(dimensionValue1.length());
+    buffer.put(dimensionValue1.getBytes(Charset.forName(CarbonCommonConstants.DEFAULT_CHARSET)));
+    buffer.rewind();
+    out1[0] = buffer;
+
+    ByteBuffer[] out2 = new ByteBuffer[1];
+
+    ByteBuffer buffer2 = ByteBuffer.allocate(dimensionValue2.length());
+    buffer2.put(dimensionValue2.getBytes(Charset.forName(CarbonCommonConstants.DEFAULT_CHARSET)));
+    buffer2.rewind();
+    out2[0] = buffer2;
+
+    byte[] arr1 = packByteBufferIntoSingleByteArray(out1);
+    byte[] arr2 = packByteBufferIntoSingleByteArray(out2);
+
+    buff1 = ByteBuffer.wrap(arr1);
+
+    buff1.position(4);
+    buff1.limit(buff1.position() + dimensionValue1.length());
+
+    buff2 = ByteBuffer.wrap(arr2);
+    buff2.position(4);
+    buff2.limit(buff2.position() + dimensionValue2.length());
+  }
+
+  @Test
+  public void testToBytes() {
+    assertTrue(ByteUtil.toBoolean(ByteUtil.toBytes(true)));
+    assertFalse(ByteUtil.toBoolean(ByteUtil.toBytes(false)));
+
+    assertEquals(Short.MAX_VALUE,
+        ByteUtil.toShort(ByteUtil.toBytes(Short.MAX_VALUE), 0, 2));
+    assertEquals((short) (Short.MAX_VALUE / 2),
+        ByteUtil.toShort(ByteUtil.toBytes((short) (Short.MAX_VALUE / 2)), 0, 2));
+    assertEquals((short) 0,
+        ByteUtil.toShort(ByteUtil.toBytes((short) 0), 0, 2));
+    assertEquals((short) (Short.MIN_VALUE / 2),
+        ByteUtil.toShort(ByteUtil.toBytes((short) (Short.MIN_VALUE / 2)), 0, 2));
+    assertEquals(Short.MIN_VALUE,
+        ByteUtil.toShort(ByteUtil.toBytes(Short.MIN_VALUE), 0, 2));
+
+    assertEquals(Integer.MAX_VALUE,
+        ByteUtil.toInt(ByteUtil.toBytes(Integer.MAX_VALUE), 0, 4));
+    assertEquals(Integer.MAX_VALUE / 2,
+        ByteUtil.toInt(ByteUtil.toBytes(Integer.MAX_VALUE / 2), 0, 4));
+    assertEquals(0,
+        ByteUtil.toInt(ByteUtil.toBytes(0), 0, 4));
+    assertEquals(Integer.MIN_VALUE / 2,
+        ByteUtil.toInt(ByteUtil.toBytes(Integer.MIN_VALUE / 2), 0, 4));
+    assertEquals(Integer.MIN_VALUE,
+        ByteUtil.toInt(ByteUtil.toBytes(Integer.MIN_VALUE), 0, 4));
+
+    assertEquals(Long.MAX_VALUE,
+        ByteUtil.toLong(ByteUtil.toBytes(Long.MAX_VALUE), 0, 8));
+    assertEquals(Long.MAX_VALUE / 2,
+        ByteUtil.toLong(ByteUtil.toBytes(Long.MAX_VALUE / 2), 0, 8));
+    assertEquals(0L,
+        ByteUtil.toLong(ByteUtil.toBytes(0L), 0, 8));
+    assertEquals(Long.MIN_VALUE / 2,
+        ByteUtil.toLong(ByteUtil.toBytes(Long.MIN_VALUE / 2), 0, 8));
+    assertEquals(Long.MIN_VALUE,
+        ByteUtil.toLong(ByteUtil.toBytes(Long.MIN_VALUE), 0, 8));
+
+    assertEquals(Double.MAX_VALUE,
+        ByteUtil.toDouble(ByteUtil.toBytes(Double.MAX_VALUE), 0, 8));
+    assertEquals(Double.MAX_VALUE / 2,
+        ByteUtil.toDouble(ByteUtil.toBytes(Double.MAX_VALUE / 2), 0, 8));
+    assertEquals((double) 0,
+        ByteUtil.toDouble(ByteUtil.toBytes((double) 0), 0, 8));
+    assertEquals(Double.MIN_VALUE / 2,
+        ByteUtil.toDouble(ByteUtil.toBytes(Double.MIN_VALUE / 2), 0, 8));
+    assertEquals(Double.MIN_VALUE,
+        ByteUtil.toDouble(ByteUtil.toBytes(Double.MIN_VALUE), 0, 8));
+  }
+
+  @Test
+  public void testToXorBytes() {
+    assertEquals(Short.MAX_VALUE,
+        ByteUtil.toXorShort(ByteUtil.toXorBytes(Short.MAX_VALUE), 0, 2));
+    assertEquals((short) (Short.MAX_VALUE / 2),
+        ByteUtil.toXorShort(ByteUtil.toXorBytes((short) (Short.MAX_VALUE / 2)), 0, 2));
+    assertEquals((short) 0,
+        ByteUtil.toXorShort(ByteUtil.toXorBytes((short) 0), 0, 2));
+    assertEquals((short) (Short.MIN_VALUE / 2),
+        ByteUtil.toXorShort(ByteUtil.toXorBytes((short) (Short.MIN_VALUE / 2)), 0, 2));
+    assertEquals(Short.MIN_VALUE,
+        ByteUtil.toXorShort(ByteUtil.toXorBytes(Short.MIN_VALUE), 0, 2));
+
+    assertEquals(Integer.MAX_VALUE,
+        ByteUtil.toXorInt(ByteUtil.toXorBytes(Integer.MAX_VALUE), 0, 4));
+    assertEquals(Integer.MAX_VALUE / 2,
+        ByteUtil.toXorInt(ByteUtil.toXorBytes(Integer.MAX_VALUE / 2), 0, 4));
+    assertEquals(0,
+        ByteUtil.toXorInt(ByteUtil.toXorBytes(0), 0, 4));
+    assertEquals(Integer.MIN_VALUE / 2,
+        ByteUtil.toXorInt(ByteUtil.toXorBytes(Integer.MIN_VALUE / 2), 0, 4));
+    assertEquals(Integer.MIN_VALUE,
+        ByteUtil.toXorInt(ByteUtil.toXorBytes(Integer.MIN_VALUE), 0, 4));
+
+    assertEquals(Long.MAX_VALUE,
+        ByteUtil.toXorLong(ByteUtil.toXorBytes(Long.MAX_VALUE), 0, 8));
+    assertEquals(Long.MAX_VALUE / 2,
+        ByteUtil.toXorLong(ByteUtil.toXorBytes(Long.MAX_VALUE / 2), 0, 8));
+    assertEquals(0L,
+        ByteUtil.toXorLong(ByteUtil.toXorBytes(0L), 0, 8));
+    assertEquals(Long.MIN_VALUE / 2,
+        ByteUtil.toXorLong(ByteUtil.toXorBytes(Long.MIN_VALUE / 2), 0, 8));
+    assertEquals(Long.MIN_VALUE,
+        ByteUtil.toXorLong(ByteUtil.toXorBytes(Long.MIN_VALUE), 0, 8));
+
+    assertEquals(Double.MAX_VALUE,
+        ByteUtil.toXorDouble(ByteUtil.toXorBytes(Double.MAX_VALUE), 0, 8));
+    assertEquals(Double.MAX_VALUE / 2,
+        ByteUtil.toXorDouble(ByteUtil.toXorBytes(Double.MAX_VALUE / 2), 0, 8));
+    assertEquals((double) 0,
+        ByteUtil.toXorDouble(ByteUtil.toXorBytes((double) 0), 0, 8));
+    assertEquals(Double.MIN_VALUE / 2,
+        ByteUtil.toXorDouble(ByteUtil.toXorBytes(Double.MIN_VALUE / 2), 0, 8));
+    assertEquals(Double.MIN_VALUE,
+        ByteUtil.toXorDouble(ByteUtil.toXorBytes(Double.MIN_VALUE), 0, 8));
+  }
 
 }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/f012f5b1/processing/src/main/java/org/apache/carbondata/processing/datatypes/PrimitiveDataType.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/datatypes/PrimitiveDataType.java b/processing/src/main/java/org/apache/carbondata/processing/datatypes/PrimitiveDataType.java
index a6d8e9c..4e2b871 100644
--- a/processing/src/main/java/org/apache/carbondata/processing/datatypes/PrimitiveDataType.java
+++ b/processing/src/main/java/org/apache/carbondata/processing/datatypes/PrimitiveDataType.java
@@ -363,7 +363,7 @@ public class PrimitiveDataType implements GenericDataType<Object> {
               if (surrogateKey == CarbonCommonConstants.INVALID_SURROGATE_KEY) {
                 value = new byte[0];
               } else {
-                value = ByteUtil.toBytes(surrogateKey);
+                value = ByteUtil.toXorBytes(surrogateKey);
               }
             } else {
               // If the input is a long value then this means that logical type was provided by
@@ -372,10 +372,10 @@ public class PrimitiveDataType implements GenericDataType<Object> {
                   || this.carbonDimension.getDataType().equals(DataTypes.TIMESTAMP)
                   && input instanceof Long) {
                 if (dictionaryGenerator != null) {
-                  value = ByteUtil.toBytes(((DirectDictionary) dictionaryGenerator)
+                  value = ByteUtil.toXorBytes(((DirectDictionary) dictionaryGenerator)
                       .generateKey((long) input));
                 } else {
-                  value = ByteUtil.toBytes(Long.parseLong(parsedValue));
+                  value = ByteUtil.toXorBytes(Long.parseLong(parsedValue));
                 }
               } else {
                 value = DataTypeUtil.getBytesBasedOnDataTypeForNoDictionaryColumn(parsedValue,
@@ -392,7 +392,7 @@ public class PrimitiveDataType implements GenericDataType<Object> {
             Object value;
             if (dictionaryGenerator instanceof DirectDictionary
                 && input instanceof Long) {
-              value = ByteUtil.toBytes(
+              value = ByteUtil.toXorBytes(
                   ((DirectDictionary) dictionaryGenerator).generateKey((long) input));
             } else {
               value = DataTypeUtil.getDataDataTypeForNoDictionaryColumn(parsedValue,