You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@kylin.apache.org by ma...@apache.org on 2016/09/20 03:43:52 UTC

[05/13] kylin git commit: KYLIN-2031 some more DimensionEncoding

KYLIN-2031 some more DimensionEncoding


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

Branch: refs/heads/master
Commit: a6cd40940248a99803d1596a32b62767646e39ac
Parents: 20ac92a
Author: Hongbin Ma <ma...@apache.org>
Authored: Mon Sep 19 19:18:06 2016 +0800
Committer: Hongbin Ma <ma...@apache.org>
Committed: Tue Sep 20 11:43:08 2016 +0800

----------------------------------------------------------------------
 .../org/apache/kylin/common/util/BytesUtil.java | 110 +++++-----
 .../gridtable/DimEncodingPreserveOrderTest.java | 113 ++++++++++
 .../dimension/DimensionEncodingFactory.java     |   2 +
 .../kylin/dimension/FixedLenHexDimEnc.java      | 149 ++++++++++---
 .../apache/kylin/dimension/IntegerDimEnc.java   |   3 +
 .../kylin/dimension/OneMoreByteVLongDimEnc.java | 206 +++++++++++++++++
 .../apache/kylin/dimension/SlimLongDimEnc.java  | 219 +++++++++++++++++++
 .../org/apache/kylin/dimension/VLongDimEnc.java | 196 -----------------
 .../kylin/dimension/FixedLenHexDimEncTest.java  | 182 +++++++++++++++
 .../kylin/dimension/IntegerDimEncTest.java      |   3 +
 .../dimension/OneMoreByteVLongDimEncTest.java   | 135 ++++++++++++
 .../kylin/dimension/SlimLongDimEncTest.java     | 151 +++++++++++++
 12 files changed, 1191 insertions(+), 278 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kylin/blob/a6cd4094/core-common/src/main/java/org/apache/kylin/common/util/BytesUtil.java
----------------------------------------------------------------------
diff --git a/core-common/src/main/java/org/apache/kylin/common/util/BytesUtil.java b/core-common/src/main/java/org/apache/kylin/common/util/BytesUtil.java
index 759ddbd..4ca6e64 100644
--- a/core-common/src/main/java/org/apache/kylin/common/util/BytesUtil.java
+++ b/core-common/src/main/java/org/apache/kylin/common/util/BytesUtil.java
@@ -25,6 +25,13 @@ public class BytesUtil {
 
     public static final byte[] EMPTY_BYTE_ARRAY = new byte[0];
 
+    public static void writeByte(byte num, byte[] bytes, int offset, int size) {
+        for (int i = offset + size - 1; i >= offset; i--) {
+            bytes[i] = num;
+            num >>>= 8;
+        }
+    }
+
     public static void writeShort(short num, byte[] bytes, int offset, int size) {
         for (int i = offset + size - 1; i >= offset; i--) {
             bytes[i] = (byte) num;
@@ -41,56 +48,85 @@ public class BytesUtil {
         return num;
     }
 
-    public static void writeLong(long num, byte[] bytes, int offset, int size) {
+    public static void writeUnsigned(int num, byte[] bytes, int offset, int size) {
         for (int i = offset + size - 1; i >= offset; i--) {
             bytes[i] = (byte) num;
             num >>>= 8;
         }
     }
 
-    public static long readLong(byte[] bytes, int offset, int size) {
-        long integer = 0;
+    public static int readUnsigned(byte[] bytes, int offset, int size) {
+        int integer = 0;
         for (int i = offset, n = offset + size; i < n; i++) {
             integer <<= 8;
-            integer |= (long) bytes[i] & 0xFF;
+            integer |= (int) bytes[i] & 0xFF;
         }
         return integer;
     }
 
-    public static long readLong(ByteBuffer buffer, int size) {
-        long integer = 0;
+    public static void writeUnsigned(int num, int size, ByteBuffer out) {
+        int mask = 0xff << ((size - 1) * 8);
+        for (int i = size; i > 0; i--) {
+            int v = (num & mask) >> (i - 1) * 8;
+            out.put((byte) v);
+            mask = mask >> 8;
+        }
+    }
+
+    public static int readUnsigned(ByteBuffer in, int size) {
+        int integer = 0;
         for (int i = 0; i < size; i++) {
+            integer = integer << 8;
+            integer |= (in.get() & 0xff);
+        }
+
+        return integer;
+    }
+
+    public static void writeSignedLong(long num, byte[] bytes, int offset, int size) {
+        writeLong(num, bytes, offset, size);
+    }
+
+    public static long readSignedLong(byte[] bytes, int offset, int size) {
+        long integer = (bytes[offset] & 0x80) == 0 ? 0 : -1;
+        for (int i = offset, n = offset + size; i < n; i++) {
             integer <<= 8;
-            integer |= (long) buffer.get() & 0xFF;
+            integer |= (int) bytes[i] & 0xFF;
         }
         return integer;
     }
 
-    public static void writeUnsigned(int num, byte[] bytes, int offset, int size) {
+    public static void writeLong(long num, byte[] bytes, int offset, int size) {
         for (int i = offset + size - 1; i >= offset; i--) {
             bytes[i] = (byte) num;
             num >>>= 8;
         }
     }
 
-    public static int readUnsigned(byte[] bytes, int offset, int size) {
-        int integer = 0;
+    public static long readLong(byte[] bytes, int offset, int size) {
+        long integer = 0;
         for (int i = offset, n = offset + size; i < n; i++) {
             integer <<= 8;
-            integer |= (int) bytes[i] & 0xFF;
+            integer |= (long) bytes[i] & 0xFF;
         }
         return integer;
     }
 
-    public static void writeSigned(int num, byte[] bytes, int offset, int size) {
-        writeUnsigned(num, bytes, offset, size);
+    public static void writeLong(long num, ByteBuffer out) {
+        for (int i = 0; i < 8; i++) {
+            out.put((byte) num);
+            num >>>= 8;
+        }
     }
 
-    public static int readSigned(byte[] bytes, int offset, int size) {
-        int integer = (bytes[offset] & 0x80) == 0 ? 0 : -1;
-        for (int i = offset, n = offset + size; i < n; i++) {
-            integer <<= 8;
-            integer |= (int) bytes[i] & 0xFF;
+    public static long readLong(ByteBuffer in) {
+        long integer = 0;
+        int mask = 0xff;
+        int shift = 0;
+        for (int i = 0; i < 8; i++) {
+            integer |= (in.get() << shift) & mask;
+            mask = mask << 8;
+            shift += 8;
         }
         return integer;
     }
@@ -204,44 +240,6 @@ public class BytesUtil {
         return -111 - value;
     }
 
-    public static void writeUnsigned(int num, int size, ByteBuffer out) {
-        int mask = 0xff << ((size - 1) * 8);
-        for (int i = size; i > 0; i--) {
-            int v = (num & mask) >> (i - 1) * 8;
-            out.put((byte) v);
-            mask = mask >> 8;
-        }
-    }
-
-    public static int readUnsigned(ByteBuffer in, int size) {
-        int integer = 0;
-        for (int i = 0; i < size; i++) {
-            integer = integer << 8;
-            integer |= (in.get() & 0xff);
-        }
-
-        return integer;
-    }
-
-    public static void writeLong(long num, ByteBuffer out) {
-        for (int i = 0; i < 8; i++) {
-            out.put((byte) num);
-            num >>>= 8;
-        }
-    }
-
-    public static long readLong(ByteBuffer in) {
-        long integer = 0;
-        int mask = 0xff;
-        int shift = 0;
-        for (int i = 0; i < 8; i++) {
-            integer |= (in.get() << shift) & mask;
-            mask = mask << 8;
-            shift += 8;
-        }
-        return integer;
-    }
-
     public static void writeUTFString(String str, ByteBuffer out) {
         byte[] bytes = str == null ? null : Bytes.toBytes(str);
         writeByteArray(bytes, out);

http://git-wip-us.apache.org/repos/asf/kylin/blob/a6cd4094/core-cube/src/test/java/org/apache/kylin/gridtable/DimEncodingPreserveOrderTest.java
----------------------------------------------------------------------
diff --git a/core-cube/src/test/java/org/apache/kylin/gridtable/DimEncodingPreserveOrderTest.java b/core-cube/src/test/java/org/apache/kylin/gridtable/DimEncodingPreserveOrderTest.java
new file mode 100644
index 0000000..0c84ed0
--- /dev/null
+++ b/core-cube/src/test/java/org/apache/kylin/gridtable/DimEncodingPreserveOrderTest.java
@@ -0,0 +1,113 @@
+/*
+ * 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.kylin.gridtable;
+
+import static org.junit.Assert.assertTrue;
+
+import java.util.List;
+
+import org.apache.kylin.common.util.ByteArray;
+import org.apache.kylin.common.util.Bytes;
+import org.apache.kylin.dimension.DimensionEncoding;
+import org.apache.kylin.dimension.FixedLenHexDimEnc;
+import org.apache.kylin.dimension.OneMoreByteVLongDimEnc;
+import org.apache.kylin.dimension.SlimLongDimEnc;
+import org.junit.Test;
+
+import com.google.common.collect.Lists;
+import com.google.common.collect.Ordering;
+
+public class DimEncodingPreserveOrderTest {
+    @Test
+    public void testOneMoreByteVLongDimEncPreserveOrder() {
+        OneMoreByteVLongDimEnc enc = new OneMoreByteVLongDimEnc(2);
+        List<ByteArray> encodedValues = Lists.newArrayList();
+        encodedValues.add(encode(enc, -32768L));
+        encodedValues.add(encode(enc, -10000L));
+        encodedValues.add(encode(enc, -100L));
+        encodedValues.add(encode(enc, 0L));
+        encodedValues.add(encode(enc, 100L));
+        encodedValues.add(encode(enc, 10000L));
+        encodedValues.add(encode(enc, 32767L));
+        encodedValues.add(encode(enc, null));
+
+        assertTrue(Ordering.from(new DefaultGTComparator()).isOrdered(encodedValues));
+    }
+
+    @Test
+    public void testVLongDimEncPreserveOrder() {
+        SlimLongDimEnc enc = new SlimLongDimEnc(2);
+        List<ByteArray> encodedValues = Lists.newArrayList();
+        encodedValues.add(encode(enc, -32767L));
+        encodedValues.add(encode(enc, -10000L));
+        encodedValues.add(encode(enc, -100L));
+        encodedValues.add(encode(enc, 0L));
+        encodedValues.add(encode(enc, 100L));
+        encodedValues.add(encode(enc, 10000L));
+        encodedValues.add(encode(enc, 32767L));
+        encodedValues.add(encode(enc, null));
+
+        assertTrue(Ordering.from(new DefaultGTComparator()).isOrdered(encodedValues));
+    }
+
+    @Test
+    public void testVLongDimEncPreserveOrder2() {
+        SlimLongDimEnc enc = new SlimLongDimEnc(8);
+        List<ByteArray> encodedValues = Lists.newArrayList();
+        encodedValues.add(encode(enc, -Long.MAX_VALUE));
+        encodedValues.add(encode(enc, -10000L));
+        encodedValues.add(encode(enc, -100L));
+        encodedValues.add(encode(enc, 0L));
+        encodedValues.add(encode(enc, 100L));
+        encodedValues.add(encode(enc, 10000L));
+        encodedValues.add(encode(enc, Long.MAX_VALUE));
+        encodedValues.add(encode(enc, null));
+
+        assertTrue(Ordering.from(new DefaultGTComparator()).isOrdered(encodedValues));
+    }
+
+    private ByteArray encode(DimensionEncoding enc, Object value) {
+        if (value != null) {
+            byte[] buf = new byte[enc.getLengthOfEncoding()];
+
+            String valueStr = "" + value;
+            byte[] bytes = Bytes.toBytes(valueStr);
+
+            enc.encode(bytes, bytes.length, buf, 0);
+            return new ByteArray(buf);
+        } else {
+            byte[] buf = new byte[enc.getLengthOfEncoding()];
+            enc.encode(null, 0, buf, 0);
+            return new ByteArray(buf);
+        }
+    }
+
+    @Test
+    public void testFixedLengthHexDimEncPreserveOrder() {
+        FixedLenHexDimEnc enc = new FixedLenHexDimEnc(4);
+        List<ByteArray> encodedValues = Lists.newArrayList();
+        encodedValues.add(encode(enc, "0000"));
+        encodedValues.add(encode(enc, "0001"));
+        encodedValues.add(encode(enc, "FFF0"));
+        encodedValues.add(encode(enc, null));
+
+        assertTrue(Ordering.from(new DefaultGTComparator()).isOrdered(encodedValues));
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/kylin/blob/a6cd4094/core-metadata/src/main/java/org/apache/kylin/dimension/DimensionEncodingFactory.java
----------------------------------------------------------------------
diff --git a/core-metadata/src/main/java/org/apache/kylin/dimension/DimensionEncodingFactory.java b/core-metadata/src/main/java/org/apache/kylin/dimension/DimensionEncodingFactory.java
index 4f70be8..739fa25 100644
--- a/core-metadata/src/main/java/org/apache/kylin/dimension/DimensionEncodingFactory.java
+++ b/core-metadata/src/main/java/org/apache/kylin/dimension/DimensionEncodingFactory.java
@@ -75,6 +75,8 @@ public abstract class DimensionEncodingFactory {
             // built-in encodings, note dictionary is a special case
             map.put(FixedLenDimEnc.ENCODING_NAME, new FixedLenDimEnc.Factory());
             map.put(IntegerDimEnc.ENCODING_NAME, new IntegerDimEnc.Factory());
+            map.put(SlimLongDimEnc.ENCODING_NAME, new SlimLongDimEnc.Factory());
+            map.put(FixedLenHexDimEnc.ENCODING_NAME, new FixedLenHexDimEnc.Factory());
             map.put(DateDimEnc.ENCODING_NAME, new DateDimEnc.Factory());
             map.put(TimeDimEnc.ENCODING_NAME, new TimeDimEnc.Factory());
 

http://git-wip-us.apache.org/repos/asf/kylin/blob/a6cd4094/core-metadata/src/main/java/org/apache/kylin/dimension/FixedLenHexDimEnc.java
----------------------------------------------------------------------
diff --git a/core-metadata/src/main/java/org/apache/kylin/dimension/FixedLenHexDimEnc.java b/core-metadata/src/main/java/org/apache/kylin/dimension/FixedLenHexDimEnc.java
index 71fe376..fbf1ccf 100644
--- a/core-metadata/src/main/java/org/apache/kylin/dimension/FixedLenHexDimEnc.java
+++ b/core-metadata/src/main/java/org/apache/kylin/dimension/FixedLenHexDimEnc.java
@@ -29,18 +29,75 @@ import org.apache.kylin.metadata.datatype.DataTypeSerializer;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import com.google.common.base.Preconditions;
+
 /**
  * used to store hex values like "1A2BFF"
+ * <p>
+ * <p>
+ * limitations: (take FixedLenHexDimEnc(2) as example: )
+ * <p>
+ * 1. "FFFF" will become null encode and decode
+ * 2. "AB" will become "AB00"
  * 
- * FixedLenHexDimEnc does not support expressing NULL values, nulls will become "FFFFFF" after encode and decode
+ * <p>
+ * Due to these limitations hex representation of hash values(with no padding, better with even characters) is more suitable
  */
 public class FixedLenHexDimEnc extends DimensionEncoding {
     private static final long serialVersionUID = 1L;
 
     private static Logger logger = LoggerFactory.getLogger(FixedLenHexDimEnc.class);
 
+    public static byte[] dict = new byte[256];
+    public static byte[] revdict = new byte[16];
+
+    static {
+        for (int i = 0; i < dict.length; i++) {
+            dict[i] = -1;
+        }
+        dict['0'] = 0;
+        dict['1'] = 1;
+        dict['2'] = 2;
+        dict['3'] = 3;
+        dict['4'] = 4;
+        dict['5'] = 5;
+        dict['6'] = 6;
+        dict['7'] = 7;
+        dict['8'] = 8;
+        dict['9'] = 9;
+        dict['A'] = 10;
+        dict['B'] = 11;
+        dict['C'] = 12;
+        dict['D'] = 13;
+        dict['E'] = 14;
+        dict['F'] = 15;
+        dict['a'] = 10;
+        dict['b'] = 11;
+        dict['c'] = 12;
+        dict['d'] = 13;
+        dict['e'] = 14;
+        dict['f'] = 15;
+
+        revdict[0] = '0';
+        revdict[1] = '1';
+        revdict[2] = '2';
+        revdict[3] = '3';
+        revdict[4] = '4';
+        revdict[5] = '5';
+        revdict[6] = '6';
+        revdict[7] = '7';
+        revdict[8] = '8';
+        revdict[9] = '9';
+        revdict[10] = 'A';
+        revdict[11] = 'B';
+        revdict[12] = 'C';
+        revdict[13] = 'D';
+        revdict[14] = 'E';
+        revdict[15] = 'F';
+    }
+
     // row key fixed length place holder
-    public static final byte ROWKEY_PLACE_HOLDER_BYTE = 9;
+    public static final byte ROWKEY_PLACE_HOLDER_BYTE = 0;
 
     public static final String ENCODING_NAME = "fixed_length_hex";
 
@@ -54,22 +111,26 @@ public class FixedLenHexDimEnc extends DimensionEncoding {
         public DimensionEncoding createDimensionEncoding(String encodingName, String[] args) {
             return new FixedLenHexDimEnc(Integer.parseInt(args[0]));
         }
-    };
+    }
 
     // ============================================================================
 
-    private int fixedLen;
+    private int hexLength;
     private int bytelen;
 
     transient private int avoidVerbose = 0;
+    transient private int avoidVerbose2 = 0;
 
     //no-arg constructor is required for Externalizable
     public FixedLenHexDimEnc() {
     }
 
     public FixedLenHexDimEnc(int len) {
-        this.fixedLen = len;
-        this.bytelen = (fixedLen + 1) / 2;
+        if (len < 1) {
+            throw new IllegalArgumentException("len has to be positive: " + len);
+        }
+        this.hexLength = len;
+        this.bytelen = (hexLength + 1) / 2;
     }
 
     @Override
@@ -81,51 +142,78 @@ public class FixedLenHexDimEnc extends DimensionEncoding {
 
         FixedLenHexDimEnc that = (FixedLenHexDimEnc) o;
 
-        return fixedLen == that.fixedLen;
-
+        return hexLength == that.hexLength;
     }
 
     @Override
     public int hashCode() {
-        return fixedLen;
+        return hexLength;
     }
 
     @Override
     public int getLengthOfEncoding() {
-        return fixedLen;
+        return bytelen;
     }
 
     @Override
     public void encode(byte[] value, int valueLen, byte[] output, int outputOffset) {
         if (value == null) {
-            Arrays.fill(output, outputOffset, outputOffset + fixedLen, NULL);
+            Arrays.fill(output, outputOffset, outputOffset + bytelen, NULL);
             return;
         }
 
-        if (valueLen > fixedLen) {
+        int endOffset = outputOffset + bytelen;
+
+        if (valueLen > hexLength) {
             if (avoidVerbose++ % 10000 == 0) {
-                logger.warn("Expect at most " + fixedLen + " bytes, but got " + valueLen + ", will truncate, value string: " + Bytes.toString(value, 0, valueLen) + " times:" + avoidVerbose);
+                logger.warn("Expect at most " + hexLength + " bytes, but got " + valueLen + ", will truncate, value string: " + Bytes.toString(value, 0, valueLen) + " times:" + avoidVerbose);
+            }
+        }
+
+        if (valueLen >= hexLength && isF(value, 0, hexLength)) {
+            if (avoidVerbose2++ % 10000 == 0) {
+                logger.warn("All 'F' value: " + Bytes.toString(value, 0, valueLen) + "will become null after encode/decode. times:" + avoidVerbose);
             }
         }
 
-        int n = Math.min(valueLen, fixedLen);
-        System.arraycopy(value, 0, output, outputOffset, n);
+        int n = Math.min(valueLen, hexLength);
+        for (int i = 0; i < n; i += 2) {
+            byte temp = 0;
+            byte iCode = dict[value[i]];
+            temp |= (iCode << 4);
 
-        if (n < fixedLen) {
-            Arrays.fill(output, outputOffset + n, outputOffset + fixedLen, ROWKEY_PLACE_HOLDER_BYTE);
+            int j = i + 1;
+            if (j < n) {
+                byte jCode = dict[value[j]];
+                temp |= jCode;
+            }
+
+            output[outputOffset++] = temp;
         }
+
+        Arrays.fill(output, outputOffset, endOffset, ROWKEY_PLACE_HOLDER_BYTE);
     }
 
     @Override
     public String decode(byte[] bytes, int offset, int len) {
+        Preconditions.checkArgument(len == bytelen, "len " + len + " not equals " + bytelen);
+
         if (isNull(bytes, offset, len)) {
             return null;
         }
 
-        while (len > 0 && bytes[offset + len - 1] == ROWKEY_PLACE_HOLDER_BYTE)
-            len--;
+        byte[] ret = new byte[hexLength];
+        for (int i = 0; i < ret.length; i += 2) {
+            byte temp = bytes[i / 2];
+            ret[i] = revdict[(temp & 0xF0) >>> 4];
 
-        return Bytes.toString(bytes, offset, len);
+            int j = i + 1;
+            if (j < hexLength) {
+                ret[j] = revdict[temp & 0x0F];
+            }
+        }
+
+        return Bytes.toString(ret, 0, ret.length);
     }
 
     @Override
@@ -140,7 +228,7 @@ public class FixedLenHexDimEnc extends DimensionEncoding {
         private byte[] currentBuf() {
             byte[] buf = current.get();
             if (buf == null) {
-                buf = new byte[fixedLen];
+                buf = new byte[bytelen];
                 current.set(buf);
             }
             return buf;
@@ -163,17 +251,17 @@ public class FixedLenHexDimEnc extends DimensionEncoding {
 
         @Override
         public int peekLength(ByteBuffer in) {
-            return fixedLen;
+            return bytelen;
         }
 
         @Override
         public int maxLength() {
-            return fixedLen;
+            return bytelen;
         }
 
         @Override
         public int getStorageBytesEstimate() {
-            return fixedLen;
+            return bytelen;
         }
 
         @Override
@@ -184,12 +272,21 @@ public class FixedLenHexDimEnc extends DimensionEncoding {
 
     @Override
     public void writeExternal(ObjectOutput out) throws IOException {
-        out.writeShort(fixedLen);
+        out.writeShort(hexLength);
     }
 
     @Override
     public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException {
-        fixedLen = in.readShort();
+        hexLength = in.readShort();
+    }
+
+    private boolean isF(byte[] value, int offset, int length) {
+        for (int i = offset; i < length + offset; ++i) {
+            if (value[i] != 'F') {
+                return false;
+            }
+        }
+        return true;
     }
 
 }

http://git-wip-us.apache.org/repos/asf/kylin/blob/a6cd4094/core-metadata/src/main/java/org/apache/kylin/dimension/IntegerDimEnc.java
----------------------------------------------------------------------
diff --git a/core-metadata/src/main/java/org/apache/kylin/dimension/IntegerDimEnc.java b/core-metadata/src/main/java/org/apache/kylin/dimension/IntegerDimEnc.java
index 89b14fc..b87e46a 100644
--- a/core-metadata/src/main/java/org/apache/kylin/dimension/IntegerDimEnc.java
+++ b/core-metadata/src/main/java/org/apache/kylin/dimension/IntegerDimEnc.java
@@ -30,6 +30,9 @@ import org.apache.kylin.metadata.datatype.DataTypeSerializer;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+/**
+ * deprecated use SlimLongDimEnc instead
+ */
 public class IntegerDimEnc extends DimensionEncoding {
     private static final long serialVersionUID = 1L;
 

http://git-wip-us.apache.org/repos/asf/kylin/blob/a6cd4094/core-metadata/src/main/java/org/apache/kylin/dimension/OneMoreByteVLongDimEnc.java
----------------------------------------------------------------------
diff --git a/core-metadata/src/main/java/org/apache/kylin/dimension/OneMoreByteVLongDimEnc.java b/core-metadata/src/main/java/org/apache/kylin/dimension/OneMoreByteVLongDimEnc.java
new file mode 100644
index 0000000..3ba9d2f
--- /dev/null
+++ b/core-metadata/src/main/java/org/apache/kylin/dimension/OneMoreByteVLongDimEnc.java
@@ -0,0 +1,206 @@
+/*
+ * 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.kylin.dimension;
+
+import java.io.IOException;
+import java.io.ObjectInput;
+import java.io.ObjectOutput;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+
+import org.apache.kylin.common.util.Bytes;
+import org.apache.kylin.common.util.BytesUtil;
+import org.apache.kylin.metadata.datatype.DataTypeSerializer;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * not being used yet, prepared for future
+ */
+public class OneMoreByteVLongDimEnc extends DimensionEncoding {
+    private static final long serialVersionUID = 1L;
+
+    private static Logger logger = LoggerFactory.getLogger(OneMoreByteVLongDimEnc.class);
+
+    private static final long[] CAP = { 0, 0x7fL, 0x7fffL, 0x7fffffL, 0x7fffffffL, 0x7fffffffffL, 0x7fffffffffffL, 0x7fffffffffffffL, 0x7fffffffffffffffL };
+    private static final long[] MASK = { 0, 0xffL, 0xffffL, 0xffffffL, 0xffffffffL, 0xffffffffffL, 0xffffffffffffL, 0xffffffffffffffL, 0xffffffffffffffffL };
+    private static final long[] TAIL = { 0, 0x80L, 0x8000L, 0x800000L, 0x80000000L, 0x8000000000L, 0x800000000000L, 0x80000000000000L, 0x8000000000000000L };
+    static {
+        for (int i = 1; i < TAIL.length; ++i) {
+            long head = ~MASK[i];
+            TAIL[i] = head | TAIL[i];
+        }
+    }
+    public static final String ENCODING_NAME = "one_more_byte_vlong";
+
+    public static class Factory extends DimensionEncodingFactory {
+        @Override
+        public String getSupportedEncodingName() {
+            return ENCODING_NAME;
+        }
+
+        @Override
+        public DimensionEncoding createDimensionEncoding(String encodingName, String[] args) {
+            return new OneMoreByteVLongDimEnc(Integer.parseInt(args[0]));
+        }
+    };
+
+    // ============================================================================
+
+    private int fixedLen;
+    private int byteLen;
+
+    transient private int avoidVerbose = 0;
+
+    //no-arg constructor is required for Externalizable
+    public OneMoreByteVLongDimEnc() {
+    }
+
+    public OneMoreByteVLongDimEnc(int len) {
+        if (len <= 0 || len >= CAP.length)
+            throw new IllegalArgumentException();
+
+        this.fixedLen = len;
+        this.byteLen = fixedLen + 1;//one additional byte to indicate null
+    }
+
+    @Override
+    public int getLengthOfEncoding() {
+        return byteLen;
+    }
+
+    @Override
+    public void encode(byte[] value, int valueLen, byte[] output, int outputOffset) {
+        if (value == null) {
+            Arrays.fill(output, outputOffset, outputOffset + byteLen, NULL);
+            return;
+        }
+
+        encode(Bytes.toString(value, 0, valueLen), output, outputOffset);
+    }
+
+    void encode(String valueStr, byte[] output, int outputOffset) {
+        if (valueStr == null) {
+            Arrays.fill(output, outputOffset, outputOffset + byteLen, NULL);
+            return;
+        }
+
+        long integer = Long.parseLong(valueStr);
+        if (integer > CAP[fixedLen] || integer < TAIL[fixedLen]) {
+            if (avoidVerbose++ % 10000 == 0) {
+                logger.warn("Expect at most " + fixedLen + " bytes, but got " + valueStr + ", will truncate, hit times:" + avoidVerbose);
+            }
+        }
+
+        BytesUtil.writeByte(integer >= 0 ? (byte) 1 : (byte) 0, output, outputOffset, 1);
+        BytesUtil.writeSignedLong(integer, output, outputOffset + 1, fixedLen);
+    }
+
+    @Override
+    public String decode(byte[] bytes, int offset, int len) {
+        if (isNull(bytes, offset, len)) {
+            return null;
+        }
+
+        long integer = BytesUtil.readSignedLong(bytes, offset + 1, len - 1);
+        return String.valueOf(integer);
+    }
+
+    @Override
+    public DataTypeSerializer<Object> asDataTypeSerializer() {
+        return new VLongSerializer();
+    }
+
+    public class VLongSerializer extends DataTypeSerializer<Object> {
+        // be thread-safe and avoid repeated obj creation
+        private ThreadLocal<byte[]> current = new ThreadLocal<byte[]>();
+
+        private byte[] currentBuf() {
+            byte[] buf = current.get();
+            if (buf == null) {
+                buf = new byte[byteLen];
+                current.set(buf);
+            }
+            return buf;
+        }
+
+        @Override
+        public void serialize(Object value, ByteBuffer out) {
+            byte[] buf = currentBuf();
+            String valueStr = value == null ? null : value.toString();
+            encode(valueStr, buf, 0);
+            out.put(buf);
+        }
+
+        @Override
+        public Object deserialize(ByteBuffer in) {
+            byte[] buf = currentBuf();
+            in.get(buf);
+            return decode(buf, 0, buf.length);
+        }
+
+        @Override
+        public int peekLength(ByteBuffer in) {
+            return byteLen;
+        }
+
+        @Override
+        public int maxLength() {
+            return byteLen;
+        }
+
+        @Override
+        public int getStorageBytesEstimate() {
+            return byteLen;
+        }
+
+        @Override
+        public Object valueOf(String str) {
+            return str;
+        }
+    }
+
+    @Override
+    public void writeExternal(ObjectOutput out) throws IOException {
+        out.writeShort(fixedLen);
+    }
+
+    @Override
+    public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException {
+        fixedLen = in.readShort();
+    }
+
+    @Override
+    public boolean equals(Object o) {
+        if (this == o)
+            return true;
+        if (o == null || getClass() != o.getClass())
+            return false;
+
+        OneMoreByteVLongDimEnc that = (OneMoreByteVLongDimEnc) o;
+
+        return fixedLen == that.fixedLen;
+
+    }
+
+    @Override
+    public int hashCode() {
+        return fixedLen;
+    }
+}

http://git-wip-us.apache.org/repos/asf/kylin/blob/a6cd4094/core-metadata/src/main/java/org/apache/kylin/dimension/SlimLongDimEnc.java
----------------------------------------------------------------------
diff --git a/core-metadata/src/main/java/org/apache/kylin/dimension/SlimLongDimEnc.java b/core-metadata/src/main/java/org/apache/kylin/dimension/SlimLongDimEnc.java
new file mode 100644
index 0000000..4ac871f
--- /dev/null
+++ b/core-metadata/src/main/java/org/apache/kylin/dimension/SlimLongDimEnc.java
@@ -0,0 +1,219 @@
+/*
+ * 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.kylin.dimension;
+
+import java.io.IOException;
+import java.io.ObjectInput;
+import java.io.ObjectOutput;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+
+import org.apache.kylin.common.util.Bytes;
+import org.apache.kylin.common.util.BytesUtil;
+import org.apache.kylin.metadata.datatype.DataTypeSerializer;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * replacement for IntegerDimEnc, the diff is VLongDimEnc supports negative values
+ */
+public class SlimLongDimEnc extends DimensionEncoding {
+    private static final long serialVersionUID = 1L;
+
+    private static Logger logger = LoggerFactory.getLogger(SlimLongDimEnc.class);
+
+    private static final long[] CAP = { 0, 0x7fL, 0x7fffL, 0x7fffffL, 0x7fffffffL, 0x7fffffffffL, 0x7fffffffffffL, 0x7fffffffffffffL, 0x7fffffffffffffffL };
+    private static final long[] MASK = { 0, 0xffL, 0xffffL, 0xffffffL, 0xffffffffL, 0xffffffffffL, 0xffffffffffffL, 0xffffffffffffffL, 0xffffffffffffffffL };
+    private static final long[] TAIL = { 0, 0x80L, 0x8000L, 0x800000L, 0x80000000L, 0x8000000000L, 0x800000000000L, 0x80000000000000L, 0x8000000000000000L };
+    static {
+        for (int i = 1; i < TAIL.length; ++i) {
+            long head = ~MASK[i];
+            TAIL[i] = head | TAIL[i];
+        }
+    }
+
+    public static final String ENCODING_NAME = "slimlong";
+
+    public static class Factory extends DimensionEncodingFactory {
+        @Override
+        public String getSupportedEncodingName() {
+            return ENCODING_NAME;
+        }
+
+        @Override
+        public DimensionEncoding createDimensionEncoding(String encodingName, String[] args) {
+            return new SlimLongDimEnc(Integer.parseInt(args[0]));
+        }
+    };
+
+    // ============================================================================
+
+    private int fixedLen;
+
+    transient private int avoidVerbose = 0;
+    transient private int avoidVerbose2 = 0;
+
+    //no-arg constructor is required for Externalizable
+    public SlimLongDimEnc() {
+    }
+
+    public SlimLongDimEnc(int len) {
+        if (len <= 0 || len >= CAP.length)
+            throw new IllegalArgumentException();
+
+        this.fixedLen = len;
+    }
+
+    @Override
+    public int getLengthOfEncoding() {
+        return fixedLen;
+    }
+
+    @Override
+    public void encode(byte[] value, int valueLen, byte[] output, int outputOffset) {
+        if (value == null) {
+            Arrays.fill(output, outputOffset, outputOffset + fixedLen, NULL);
+            return;
+        }
+
+        encode(Bytes.toString(value, 0, valueLen), output, outputOffset);
+    }
+
+    void encode(String valueStr, byte[] output, int outputOffset) {
+        if (valueStr == null) {
+            Arrays.fill(output, outputOffset, outputOffset + fixedLen, NULL);
+            return;
+        }
+
+        long integer = Long.parseLong(valueStr);
+        if (integer > CAP[fixedLen] || integer < TAIL[fixedLen]) {
+            if (avoidVerbose++ % 10000 == 0) {
+                logger.warn("Expect at most " + fixedLen + " bytes, but got " + valueStr + ", will truncate, hit times:" + avoidVerbose);
+            }
+        }
+
+        if (integer == TAIL[fixedLen]) {
+            if (avoidVerbose2++ % 10000 == 0) {
+                logger.warn("Value " + valueStr + " does not fit into " + fixedLen + " bytes ");
+            }
+        }
+
+        BytesUtil.writeLong(integer + CAP[fixedLen], output, outputOffset, fixedLen);//apply an offset to preserve binary order, overflow is okay
+    }
+
+    @Override
+    public String decode(byte[] bytes, int offset, int len) {
+        if (isNull(bytes, offset, len)) {
+            return null;
+        }
+
+        long integer = BytesUtil.readLong(bytes, offset, len) - CAP[fixedLen];
+
+        //only take useful bytes
+        integer = integer & MASK[fixedLen];
+        boolean positive = (integer & ((0x80) << ((fixedLen - 1) << 3))) == 0;
+        if (!positive) {
+            integer |= (~MASK[fixedLen]);
+        }
+
+        return String.valueOf(integer);
+    }
+
+    @Override
+    public DataTypeSerializer<Object> asDataTypeSerializer() {
+        return new IntegerSerializer();
+    }
+
+    public class IntegerSerializer extends DataTypeSerializer<Object> {
+        // be thread-safe and avoid repeated obj creation
+        private ThreadLocal<byte[]> current = new ThreadLocal<byte[]>();
+
+        private byte[] currentBuf() {
+            byte[] buf = current.get();
+            if (buf == null) {
+                buf = new byte[fixedLen];
+                current.set(buf);
+            }
+            return buf;
+        }
+
+        @Override
+        public void serialize(Object value, ByteBuffer out) {
+            byte[] buf = currentBuf();
+            String valueStr = value == null ? null : value.toString();
+            encode(valueStr, buf, 0);
+            out.put(buf);
+        }
+
+        @Override
+        public Object deserialize(ByteBuffer in) {
+            byte[] buf = currentBuf();
+            in.get(buf);
+            return decode(buf, 0, buf.length);
+        }
+
+        @Override
+        public int peekLength(ByteBuffer in) {
+            return fixedLen;
+        }
+
+        @Override
+        public int maxLength() {
+            return fixedLen;
+        }
+
+        @Override
+        public int getStorageBytesEstimate() {
+            return fixedLen;
+        }
+
+        @Override
+        public Object valueOf(String str) {
+            return str;
+        }
+    }
+
+    @Override
+    public void writeExternal(ObjectOutput out) throws IOException {
+        out.writeShort(fixedLen);
+    }
+
+    @Override
+    public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException {
+        fixedLen = in.readShort();
+    }
+
+    @Override
+    public boolean equals(Object o) {
+        if (this == o)
+            return true;
+        if (o == null || getClass() != o.getClass())
+            return false;
+
+        SlimLongDimEnc that = (SlimLongDimEnc) o;
+
+        return fixedLen == that.fixedLen;
+
+    }
+
+    @Override
+    public int hashCode() {
+        return fixedLen;
+    }
+}

http://git-wip-us.apache.org/repos/asf/kylin/blob/a6cd4094/core-metadata/src/main/java/org/apache/kylin/dimension/VLongDimEnc.java
----------------------------------------------------------------------
diff --git a/core-metadata/src/main/java/org/apache/kylin/dimension/VLongDimEnc.java b/core-metadata/src/main/java/org/apache/kylin/dimension/VLongDimEnc.java
deleted file mode 100644
index 000cd92..0000000
--- a/core-metadata/src/main/java/org/apache/kylin/dimension/VLongDimEnc.java
+++ /dev/null
@@ -1,196 +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.kylin.dimension;
-
-import java.io.IOException;
-import java.io.ObjectInput;
-import java.io.ObjectOutput;
-import java.nio.ByteBuffer;
-import java.util.Arrays;
-
-import org.apache.kylin.common.util.Bytes;
-import org.apache.kylin.common.util.BytesUtil;
-import org.apache.kylin.metadata.datatype.DataTypeSerializer;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-public class VLongDimEnc extends DimensionEncoding {
-    private static final long serialVersionUID = 1L;
-
-    private static Logger logger = LoggerFactory.getLogger(VLongDimEnc.class);
-
-    private static final long[] CAP = { 0, 0xffL, 0xffffL, 0xffffffL, 0xffffffffL, 0xffffffffffL, 0xffffffffffffL, 0xffffffffffffffL, Long.MAX_VALUE };
-    private static final long[] TAIL = { 0, 0x80L, 0xffffL, 0xffffffL, 0xffffffffL, 0xffffffffffL, 0xffffffffffffL, 0xffffffffffffffL, Long.MAX_VALUE };
-
-    public static final String ENCODING_NAME = "vlong";
-
-    public static class Factory extends DimensionEncodingFactory {
-        @Override
-        public String getSupportedEncodingName() {
-            return ENCODING_NAME;
-        }
-
-        @Override
-        public DimensionEncoding createDimensionEncoding(String encodingName, String[] args) {
-            return new VLongDimEnc(Integer.parseInt(args[0]));
-        }
-    };
-
-    // ============================================================================
-
-    private int fixedLen;
-    private int byteLen;
-
-    transient private int avoidVerbose = 0;
-
-    //no-arg constructor is required for Externalizable
-    public VLongDimEnc() {
-    }
-
-    public VLongDimEnc(int len) {
-        if (len <= 0 || len >= CAP.length)
-            throw new IllegalArgumentException();
-
-        this.fixedLen = len;
-        this.byteLen = (fixedLen + 1) / 2 + 1;//one additional byte to indicate null
-    }
-
-    @Override
-    public int getLengthOfEncoding() {
-        return byteLen;
-    }
-
-    @Override
-    public void encode(byte[] value, int valueLen, byte[] output, int outputOffset) {
-        if (value == null) {
-            Arrays.fill(output, outputOffset, outputOffset + byteLen, NULL);
-            return;
-        }
-
-        encode(Bytes.toString(value, 0, valueLen), output, outputOffset);
-    }
-
-    void encode(String valueStr, byte[] output, int outputOffset) {
-        if (valueStr == null) {
-            Arrays.fill(output, outputOffset, outputOffset + byteLen, NULL);
-            return;
-        }
-
-        long integer = Long.parseLong(valueStr);
-        if (integer > CAP[fixedLen]) {
-            if (avoidVerbose++ % 10000 == 0) {
-                logger.warn("Expect at most " + fixedLen + " bytes, but got " + valueStr + ", will truncate, hit times:" + avoidVerbose);
-            }
-        }
-
-        BytesUtil.writeLong(integer, output, outputOffset, fixedLen);
-    }
-
-    @Override
-    public String decode(byte[] bytes, int offset, int len) {
-        if (isNull(bytes, offset, len)) {
-            return null;
-        }
-
-        long integer = BytesUtil.readLong(bytes, offset, len);
-        return String.valueOf(integer);
-    }
-
-    @Override
-    public DataTypeSerializer<Object> asDataTypeSerializer() {
-        return new IntegerSerializer();
-    }
-
-    public class IntegerSerializer extends DataTypeSerializer<Object> {
-        // be thread-safe and avoid repeated obj creation
-        private ThreadLocal<byte[]> current = new ThreadLocal<byte[]>();
-
-        private byte[] currentBuf() {
-            byte[] buf = current.get();
-            if (buf == null) {
-                buf = new byte[fixedLen];
-                current.set(buf);
-            }
-            return buf;
-        }
-
-        @Override
-        public void serialize(Object value, ByteBuffer out) {
-            byte[] buf = currentBuf();
-            String valueStr = value == null ? null : value.toString();
-            encode(valueStr, buf, 0);
-            out.put(buf);
-        }
-
-        @Override
-        public Object deserialize(ByteBuffer in) {
-            byte[] buf = currentBuf();
-            in.get(buf);
-            return decode(buf, 0, buf.length);
-        }
-
-        @Override
-        public int peekLength(ByteBuffer in) {
-            return fixedLen;
-        }
-
-        @Override
-        public int maxLength() {
-            return fixedLen;
-        }
-
-        @Override
-        public int getStorageBytesEstimate() {
-            return fixedLen;
-        }
-
-        @Override
-        public Object valueOf(String str) {
-            return str;
-        }
-    }
-
-    @Override
-    public void writeExternal(ObjectOutput out) throws IOException {
-        out.writeShort(fixedLen);
-    }
-
-    @Override
-    public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException {
-        fixedLen = in.readShort();
-    }
-
-    @Override
-    public boolean equals(Object o) {
-        if (this == o)
-            return true;
-        if (o == null || getClass() != o.getClass())
-            return false;
-
-        VLongDimEnc that = (VLongDimEnc) o;
-
-        return fixedLen == that.fixedLen;
-
-    }
-
-    @Override
-    public int hashCode() {
-        return fixedLen;
-    }
-}

http://git-wip-us.apache.org/repos/asf/kylin/blob/a6cd4094/core-metadata/src/test/java/org/apache/kylin/dimension/FixedLenHexDimEncTest.java
----------------------------------------------------------------------
diff --git a/core-metadata/src/test/java/org/apache/kylin/dimension/FixedLenHexDimEncTest.java b/core-metadata/src/test/java/org/apache/kylin/dimension/FixedLenHexDimEncTest.java
new file mode 100644
index 0000000..6f0540b
--- /dev/null
+++ b/core-metadata/src/test/java/org/apache/kylin/dimension/FixedLenHexDimEncTest.java
@@ -0,0 +1,182 @@
+/*
+ * 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.kylin.dimension;
+
+import java.nio.ByteBuffer;
+
+import org.apache.kylin.common.util.Bytes;
+import org.apache.kylin.metadata.datatype.DataTypeSerializer;
+import org.junit.Assert;
+import org.junit.Test;
+
+public class FixedLenHexDimEncTest {
+
+    @Test
+    public void testConstructor() {
+        try {
+            new FixedLenHexDimEnc(0);
+            Assert.fail();
+        } catch (IllegalArgumentException e) {
+            // expect
+        }
+
+        new FixedLenHexDimEnc(8);
+    }
+
+    @Test
+    public void testNull() {
+        for (int i = 1; i < 9; i++) {
+            FixedLenHexDimEnc enc = new FixedLenHexDimEnc(i);
+
+            byte[] buf = new byte[enc.getLengthOfEncoding()];
+            enc.encode(null, 0, buf, 0);
+            Assert.assertTrue(DimensionEncoding.isNull(buf, 0, buf.length));
+            String decode = enc.decode(buf, 0, buf.length);
+            Assert.assertEquals(null, decode);
+
+            buf = new byte[enc.getLengthOfEncoding()];
+            DataTypeSerializer<Object> ser = enc.asDataTypeSerializer();
+            ser.serialize(null, ByteBuffer.wrap(buf));
+            Assert.assertTrue(DimensionEncoding.isNull(buf, 0, buf.length));
+            decode = (String) ser.deserialize(ByteBuffer.wrap(buf));
+            Assert.assertEquals(null, decode);
+        }
+    }
+
+    @Test
+    public void testEncodeDecode() {
+        FixedLenHexDimEnc enc = new FixedLenHexDimEnc(4);
+        testEncodeDecode(enc, "AF12");
+        testEncodeDecode(enc, "0000");
+        testEncodeDecode(enc, "FFF0");
+        try {
+            testEncodeDecode(enc, "abcd");
+            Assert.fail();
+        } catch (Throwable e) {
+            Assert.assertEquals("expected:<[abcd]> but was:<[ABCD]>", e.getMessage());
+        }
+        try {
+            testEncodeDecode(enc, "FFFF");
+            Assert.fail();
+        } catch (Throwable e) {
+            Assert.assertEquals("expected:<FFFF> but was:<null>", e.getMessage());
+        }
+        try {
+            testEncodeDecode(enc, "FFF");
+            Assert.fail();
+        } catch (Throwable e) {
+            Assert.assertEquals("expected:<FFF[]> but was:<FFF[0]>", e.getMessage());
+        }
+        try {
+            testEncodeDecode(enc, "FFFF0");
+            Assert.fail();
+        } catch (Throwable e) {
+            Assert.assertEquals("expected:<FFFF0> but was:<null>", e.getMessage());
+        }
+        try {
+            testEncodeDecode(enc, "FFF10");
+            Assert.fail();
+        } catch (Throwable e) {
+            Assert.assertEquals("expected:<FFF1[0]> but was:<FFF1[]>", e.getMessage());
+        }
+    }
+
+
+    @Test
+    public void testEncodeDecode2() {
+        FixedLenHexDimEnc enc = new FixedLenHexDimEnc(5);
+        testEncodeDecode(enc, "AF121");
+        testEncodeDecode(enc, "00000");
+        
+        //with a little extra room all F is supported
+        testEncodeDecode(enc, "FFFFF");
+        
+        try {
+            testEncodeDecode(enc, "FFF");
+            Assert.fail();
+        } catch (Throwable e) {
+            Assert.assertEquals("expected:<FFF[]> but was:<FFF[00]>", e.getMessage());
+        }
+        try {
+            testEncodeDecode(enc, "FFFFF0");
+            Assert.fail();
+        } catch (Throwable e) {
+            Assert.assertEquals("expected:<FFFFF[0]> but was:<FFFFF[]>", e.getMessage());
+        }
+        try {
+            testEncodeDecode(enc, "FFFF10");
+            Assert.fail();
+        } catch (Throwable e) {
+            Assert.assertEquals("expected:<FFFF1[0]> but was:<FFFF1[]>", e.getMessage());
+        }
+
+    }
+
+    private void testEncodeDecode(FixedLenHexDimEnc enc, String value) {
+        byte[] buf = new byte[enc.getLengthOfEncoding()];
+        String valueStr = value;
+        byte[] bytes = Bytes.toBytes(valueStr);
+        enc.encode(bytes, bytes.length, buf, 0);
+        String decode = enc.decode(buf, 0, buf.length);
+        Assert.assertEquals(valueStr, decode);
+    }
+
+    @Test
+    public void testSerDes() {
+
+        FixedLenHexDimEnc enc = new FixedLenHexDimEnc(4);
+        testSerDes(enc, "AF12");
+        testSerDes(enc, "0000");
+        testSerDes(enc, "FFF0");
+        try {
+            testSerDes(enc, "FFFF");
+            Assert.fail();
+        } catch (Throwable e) {
+            Assert.assertEquals("expected:<FFFF> but was:<null>", e.getMessage());
+        }
+        try {
+            testSerDes(enc, "FFF");
+            Assert.fail();
+        } catch (Throwable e) {
+            Assert.assertEquals("expected:<FFF[]> but was:<FFF[0]>", e.getMessage());
+        }
+        try {
+            testSerDes(enc, "FFFF0");
+            Assert.fail();
+        } catch (Throwable e) {
+            Assert.assertEquals("expected:<FFFF0> but was:<null>", e.getMessage());
+        }
+        try {
+            testSerDes(enc, "FFF10");
+            Assert.fail();
+        } catch (Throwable e) {
+            Assert.assertEquals("expected:<FFF1[0]> but was:<FFF1[]>", e.getMessage());
+        }
+    }
+
+    private void testSerDes(FixedLenHexDimEnc enc, String value) {
+        DataTypeSerializer<Object> ser = enc.asDataTypeSerializer();
+        byte[] buf = new byte[enc.getLengthOfEncoding()];
+        String valueStr = value;
+        ser.serialize(valueStr, ByteBuffer.wrap(buf));
+        String decode = (String) ser.deserialize(ByteBuffer.wrap(buf));
+        Assert.assertEquals(valueStr, decode);
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/kylin/blob/a6cd4094/core-metadata/src/test/java/org/apache/kylin/dimension/IntegerDimEncTest.java
----------------------------------------------------------------------
diff --git a/core-metadata/src/test/java/org/apache/kylin/dimension/IntegerDimEncTest.java b/core-metadata/src/test/java/org/apache/kylin/dimension/IntegerDimEncTest.java
index 34f0427..fccd8b9 100644
--- a/core-metadata/src/test/java/org/apache/kylin/dimension/IntegerDimEncTest.java
+++ b/core-metadata/src/test/java/org/apache/kylin/dimension/IntegerDimEncTest.java
@@ -25,6 +25,9 @@ import org.apache.kylin.metadata.datatype.DataTypeSerializer;
 import org.junit.Assert;
 import org.junit.Test;
 
+/**
+ * Deprecated. use VLongDimEnc instead
+ */
 public class IntegerDimEncTest {
 
     @Test

http://git-wip-us.apache.org/repos/asf/kylin/blob/a6cd4094/core-metadata/src/test/java/org/apache/kylin/dimension/OneMoreByteVLongDimEncTest.java
----------------------------------------------------------------------
diff --git a/core-metadata/src/test/java/org/apache/kylin/dimension/OneMoreByteVLongDimEncTest.java b/core-metadata/src/test/java/org/apache/kylin/dimension/OneMoreByteVLongDimEncTest.java
new file mode 100644
index 0000000..ba6ab55
--- /dev/null
+++ b/core-metadata/src/test/java/org/apache/kylin/dimension/OneMoreByteVLongDimEncTest.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.kylin.dimension;
+
+import java.nio.ByteBuffer;
+
+import org.apache.kylin.common.util.Bytes;
+import org.apache.kylin.metadata.datatype.DataTypeSerializer;
+import org.junit.Assert;
+import org.junit.Test;
+
+public class OneMoreByteVLongDimEncTest {
+
+    @Test
+    public void testConstructor() {
+        try {
+            new OneMoreByteVLongDimEnc(0);
+            Assert.fail();
+        } catch (IllegalArgumentException e) {
+            // expect
+        }
+        try {
+            new OneMoreByteVLongDimEnc(9);
+            Assert.fail();
+        } catch (IllegalArgumentException e) {
+            // expect
+        }
+        new OneMoreByteVLongDimEnc(8);
+    }
+
+    @Test
+    public void testNull() {
+        for (int i = 1; i < 9; i++) {
+            OneMoreByteVLongDimEnc enc = new OneMoreByteVLongDimEnc(i);
+
+            byte[] buf = new byte[enc.getLengthOfEncoding()];
+            enc.encode(null, 0, buf, 0);
+            Assert.assertTrue(DimensionEncoding.isNull(buf, 0, buf.length));
+            String decode = enc.decode(buf, 0, buf.length);
+            Assert.assertEquals(null, decode);
+
+            buf = new byte[enc.getLengthOfEncoding()];
+            DataTypeSerializer<Object> ser = enc.asDataTypeSerializer();
+            ser.serialize(null, ByteBuffer.wrap(buf));
+            Assert.assertTrue(DimensionEncoding.isNull(buf, 0, buf.length));
+            decode = (String) ser.deserialize(ByteBuffer.wrap(buf));
+            Assert.assertEquals(null, decode);
+        }
+    }
+
+    @Test
+    public void testEncodeDecode() {
+        OneMoreByteVLongDimEnc enc = new OneMoreByteVLongDimEnc(2);
+        testEncodeDecode(enc, 0);
+        testEncodeDecode(enc, 100);
+        testEncodeDecode(enc, 10000);
+        testEncodeDecode(enc, 32767);
+        testEncodeDecode(enc, -100);
+        testEncodeDecode(enc, -10000);
+        testEncodeDecode(enc, -32768);
+        try {
+            testEncodeDecode(enc, 32768);
+            Assert.fail();
+        } catch (Throwable e) {
+            Assert.assertEquals("expected:<[]32768> but was:<[-]32768>", e.getMessage());
+        }
+        try {
+            testEncodeDecode(enc, -32769);
+            Assert.fail();
+        } catch (Throwable e) {
+            Assert.assertEquals("expected:<[-32769]> but was:<[32767]>", e.getMessage());
+        }
+    }
+
+ 
+    private void testEncodeDecode(OneMoreByteVLongDimEnc enc, long value) {
+        String valueStr = "" + value;
+        byte[] buf = new byte[enc.getLengthOfEncoding()];
+        byte[] bytes = Bytes.toBytes(valueStr);
+        enc.encode(bytes, bytes.length, buf, 0);
+        String decode = enc.decode(buf, 0, buf.length);
+        Assert.assertEquals(valueStr, decode);
+    }
+
+
+    @Test
+    public void testSerDes() {
+        OneMoreByteVLongDimEnc enc = new OneMoreByteVLongDimEnc(2);
+        testSerDes(enc, 0);
+        testSerDes(enc, 100);
+        testSerDes(enc, 10000);
+        testSerDes(enc, 32767);
+        testSerDes(enc, -100);
+        testSerDes(enc, -10000);
+        testSerDes(enc, -32768);
+        try {
+            testSerDes(enc, 32768);
+            Assert.fail();
+        } catch (Throwable e) {
+            Assert.assertEquals("expected:<[]32768> but was:<[-]32768>", e.getMessage());
+        }
+        try {
+            testSerDes(enc, -32769);
+            Assert.fail();
+        } catch (Throwable e) {
+            Assert.assertEquals("expected:<[-32769]> but was:<[32767]>", e.getMessage());
+        }
+    }
+
+    private void testSerDes(OneMoreByteVLongDimEnc enc, long value) {
+        DataTypeSerializer<Object> ser = enc.asDataTypeSerializer();
+        byte[] buf = new byte[enc.getLengthOfEncoding()];
+        String valueStr = "" + value;
+        ser.serialize(valueStr, ByteBuffer.wrap(buf));
+        String decode = (String) ser.deserialize(ByteBuffer.wrap(buf));
+        Assert.assertEquals(valueStr, decode);
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/kylin/blob/a6cd4094/core-metadata/src/test/java/org/apache/kylin/dimension/SlimLongDimEncTest.java
----------------------------------------------------------------------
diff --git a/core-metadata/src/test/java/org/apache/kylin/dimension/SlimLongDimEncTest.java b/core-metadata/src/test/java/org/apache/kylin/dimension/SlimLongDimEncTest.java
new file mode 100644
index 0000000..a1e9516
--- /dev/null
+++ b/core-metadata/src/test/java/org/apache/kylin/dimension/SlimLongDimEncTest.java
@@ -0,0 +1,151 @@
+/*
+ * 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.kylin.dimension;
+
+import java.nio.ByteBuffer;
+
+import org.apache.kylin.common.util.Bytes;
+import org.apache.kylin.metadata.datatype.DataTypeSerializer;
+import org.junit.Assert;
+import org.junit.Test;
+
+public class SlimLongDimEncTest {
+
+    @Test
+    public void testConstructor() {
+        try {
+            new SlimLongDimEnc(0);
+            Assert.fail();
+        } catch (IllegalArgumentException e) {
+            // expect
+        }
+        try {
+            new SlimLongDimEnc(9);
+            Assert.fail();
+        } catch (IllegalArgumentException e) {
+            // expect
+        }
+        new SlimLongDimEnc(8);
+    }
+
+    @Test
+    public void testNull() {
+        for (int i = 1; i < 9; i++) {
+            SlimLongDimEnc enc = new SlimLongDimEnc(i);
+
+            byte[] buf = new byte[enc.getLengthOfEncoding()];
+            enc.encode(null, 0, buf, 0);
+            Assert.assertTrue(DimensionEncoding.isNull(buf, 0, buf.length));
+            String decode = enc.decode(buf, 0, buf.length);
+            Assert.assertEquals(null, decode);
+
+            buf = new byte[enc.getLengthOfEncoding()];
+            DataTypeSerializer<Object> ser = enc.asDataTypeSerializer();
+            ser.serialize(null, ByteBuffer.wrap(buf));
+            Assert.assertTrue(DimensionEncoding.isNull(buf, 0, buf.length));
+            decode = (String) ser.deserialize(ByteBuffer.wrap(buf));
+            Assert.assertEquals(null, decode);
+        }
+    }
+
+    @Test
+    public void testEncodeDecode() {
+        SlimLongDimEnc enc = new SlimLongDimEnc(2);
+        testEncodeDecode(enc, 0);
+        testEncodeDecode(enc, 100);
+        testEncodeDecode(enc, 10000);
+        testEncodeDecode(enc, 32767);
+        testEncodeDecode(enc, -100);
+        testEncodeDecode(enc, -10000);
+        testEncodeDecode(enc, -32767);
+        try {
+            testEncodeDecode(enc, 32768);
+            Assert.fail();
+        } catch (Throwable e) {
+            Assert.assertEquals("expected:<32768> but was:<null>", e.getMessage());
+        }
+        try {
+            testEncodeDecode(enc, -32768);
+            Assert.fail();
+        } catch (Throwable e) {
+            Assert.assertEquals("expected:<-32768> but was:<null>", e.getMessage());
+        }
+    }
+
+    @Test
+    public void testEncodeDecode2() {
+        SlimLongDimEnc enc = new SlimLongDimEnc(8);
+        testEncodeDecode(enc, 0);
+        testEncodeDecode(enc, 100);
+        testEncodeDecode(enc, 10000);
+        testEncodeDecode(enc, Long.MAX_VALUE);
+        testEncodeDecode(enc, -100);
+        testEncodeDecode(enc, -10000);
+        testEncodeDecode(enc, -Long.MAX_VALUE);
+        try {
+            testEncodeDecode(enc, Long.MIN_VALUE);
+            Assert.fail();
+        } catch (Throwable e) {
+            Assert.assertEquals("expected:<-9223372036854775808> but was:<null>", e.getMessage());
+        }
+    }
+
+    private void testEncodeDecode(SlimLongDimEnc enc, long value) {
+        String valueStr = "" + value;
+        byte[] buf = new byte[enc.getLengthOfEncoding()];
+        byte[] bytes = Bytes.toBytes(valueStr);
+        enc.encode(bytes, bytes.length, buf, 0);
+        String decode = enc.decode(buf, 0, buf.length);
+        Assert.assertEquals(valueStr, decode);
+    }
+
+    @Test
+    public void testSerDes() {
+        SlimLongDimEnc enc = new SlimLongDimEnc(2);
+        testSerDes(enc, 0);
+        testSerDes(enc, 100);
+        testSerDes(enc, 10000);
+        testSerDes(enc, 32767);
+        testSerDes(enc, -100);
+        testSerDes(enc, -10000);
+        testSerDes(enc, -32767);
+        try {
+            testSerDes(enc, 32768);
+            Assert.fail();
+        } catch (Throwable e) {
+            Assert.assertEquals("expected:<32768> but was:<null>", e.getMessage());
+        }
+        try {
+            testSerDes(enc, -32768);
+            Assert.fail();
+        } catch (Throwable e) {
+            Assert.assertEquals("expected:<-32768> but was:<null>", e.getMessage());
+        }
+    }
+
+    private void testSerDes(SlimLongDimEnc enc, long value) {
+        DataTypeSerializer<Object> ser = enc.asDataTypeSerializer();
+        byte[] buf = new byte[enc.getLengthOfEncoding()];
+        String valueStr = "" + value;
+        ser.serialize(valueStr, ByteBuffer.wrap(buf));
+        String decode = (String) ser.deserialize(ByteBuffer.wrap(buf));
+        Assert.assertEquals(valueStr, decode);
+    }
+
+}