You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@kylin.apache.org by li...@apache.org on 2017/02/21 02:10:36 UTC

[1/2] kylin git commit: KYLIN-2417 Compare the performance between HDFSMetaStore and HBaseMetaStore

Repository: kylin
Updated Branches:
  refs/heads/master 5d2e6c10c -> 4dce0cf08


KYLIN-2417 Compare the performance between HDFSMetaStore and HBaseMetaStore

Signed-off-by: Li Yang <li...@apache.org>


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

Branch: refs/heads/master
Commit: 65acd70ed4ff1a6f16a1ede95c1c9b85dc2f5578
Parents: 5d2e6c1
Author: xiefan46 <95...@qq.com>
Authored: Wed Jan 25 17:07:43 2017 +0800
Committer: Li Yang <li...@apache.org>
Committed: Tue Feb 21 10:04:02 2017 +0800

----------------------------------------------------------------------
 .../common/persistence/ResourceStoreTest.java   | 42 +++++++++++++++++++-
 .../storage/hdfs/ITHDFSResourceStoreTest.java   | 19 +++++++++
 2 files changed, 60 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kylin/blob/65acd70e/core-common/src/test/java/org/apache/kylin/common/persistence/ResourceStoreTest.java
----------------------------------------------------------------------
diff --git a/core-common/src/test/java/org/apache/kylin/common/persistence/ResourceStoreTest.java b/core-common/src/test/java/org/apache/kylin/common/persistence/ResourceStoreTest.java
index ddaf481..79a0c30 100644
--- a/core-common/src/test/java/org/apache/kylin/common/persistence/ResourceStoreTest.java
+++ b/core-common/src/test/java/org/apache/kylin/common/persistence/ResourceStoreTest.java
@@ -30,17 +30,34 @@ import java.util.List;
 import java.util.NavigableSet;
 
 import org.apache.commons.lang.StringUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 /**
- * Be called by LocalFileResourceStoreTest and ITHBaseResourceStoreTest.
+ * Be called by LocalFileResourceStoreTest, ITHBaseResourceStoreTest and ITHDFSResourceStoreTest.
  */
 public class ResourceStoreTest {
 
+    private static final Logger logger = LoggerFactory.getLogger(ResourceStoreTest.class);
+
+    private static final String PERFORMANCE_TEST_ROOT_PATH = "/performance";
+
+    private static final int TEST_RESOURCE_COUNT = 1000;
+
     public static void testAStore(ResourceStore store) throws IOException {
         testBasics(store);
         testGetAllResources(store);
     }
 
+    public static void testPerformance(ResourceStore store) throws IOException {
+        logger.info("Test basic functions");
+        testAStore(store);
+        logger.info("Basic function ok. Start to test performance for class : " + store.getClass());
+        logger.info("Write metadata time : " + testWritePerformance(store));
+        logger.info("Read metadata time  " + testReadPerformance(store));
+        logger.info("Performance test end. Class : " + store.getClass());
+    }
+
     private static void testGetAllResources(ResourceStore store) throws IOException {
         final String folder = "/testFolder";
         List<StringEntity> result;
@@ -144,6 +161,29 @@ public class ResourceStoreTest {
         assertTrue(list == null || list.contains(path2) == false);
     }
 
+    private static long testWritePerformance(ResourceStore store) throws IOException {
+        store.deleteResource(PERFORMANCE_TEST_ROOT_PATH);
+        StringEntity content = new StringEntity("something");
+        long startTime = System.currentTimeMillis();
+        for (int i = 0; i < TEST_RESOURCE_COUNT; i++) {
+            String resourcePath = PERFORMANCE_TEST_ROOT_PATH + "/res_" + i;
+            store.putResource(resourcePath, content, 0, StringEntity.serializer);
+        }
+        return System.currentTimeMillis() - startTime;
+    }
+
+    private static long testReadPerformance(ResourceStore store) throws IOException {
+        long startTime = System.currentTimeMillis();
+        int step = 0; //avoid compiler optimization
+        for (int i = 0; i < TEST_RESOURCE_COUNT; i++) {
+            String resourcePath = PERFORMANCE_TEST_ROOT_PATH + "/res_" + i;
+            StringEntity t = store.getResource(resourcePath, StringEntity.class, StringEntity.serializer);
+            step |= t.toString().length();
+        }
+        logger.info("step : " + step);
+        return System.currentTimeMillis() - startTime;
+    }
+
     @SuppressWarnings("serial")
     public static class StringEntity extends RootPersistentEntity {
 

http://git-wip-us.apache.org/repos/asf/kylin/blob/65acd70e/kylin-it/src/test/java/org/apache/kylin/storage/hdfs/ITHDFSResourceStoreTest.java
----------------------------------------------------------------------
diff --git a/kylin-it/src/test/java/org/apache/kylin/storage/hdfs/ITHDFSResourceStoreTest.java b/kylin-it/src/test/java/org/apache/kylin/storage/hdfs/ITHDFSResourceStoreTest.java
index aa5a104..534839f 100644
--- a/kylin-it/src/test/java/org/apache/kylin/storage/hdfs/ITHDFSResourceStoreTest.java
+++ b/kylin-it/src/test/java/org/apache/kylin/storage/hdfs/ITHDFSResourceStoreTest.java
@@ -27,8 +27,10 @@ import org.apache.kylin.common.KylinConfig;
 import org.apache.kylin.common.persistence.ResourceStoreTest;
 import org.apache.kylin.common.util.HBaseMetadataTestCase;
 import org.apache.kylin.common.util.HadoopUtil;
+import org.apache.kylin.storage.hbase.HBaseResourceStore;
 import org.junit.After;
 import org.junit.Before;
+import org.junit.Ignore;
 import org.junit.Test;
 
 public class ITHDFSResourceStoreTest extends HBaseMetadataTestCase {
@@ -84,4 +86,21 @@ public class ITHDFSResourceStoreTest extends HBaseMetadataTestCase {
         assertTrue(fs.exists(new Path(path)));
     }
 
+    @Ignore
+    @Test
+    public void performanceTest() throws Exception{
+        //test hdfs performance
+        String oldUrl = kylinConfig.getMetadataUrl();
+        kylinConfig.setProperty("kylin.metadata.url", "kylin_metadata@hdfs");
+        HDFSResourceStore store = new HDFSResourceStore(kylinConfig);
+        ResourceStoreTest.testPerformance(store);
+        kylinConfig.setProperty("kylin.metadata.url", oldUrl);
+
+        //test hbase
+        oldUrl = kylinConfig.getMetadataUrl();
+        kylinConfig.setProperty("kylin.metadata.url", "kylin_metadata@hbase");
+        HBaseResourceStore store2 = new HBaseResourceStore(kylinConfig);
+        ResourceStoreTest.testPerformance(store2);
+        kylinConfig.setProperty("kylin.metadata.url", oldUrl);
+    }
 }


[2/2] kylin git commit: KYLIN-2384 Refactor and refine NumberDictionary.

Posted by li...@apache.org.
KYLIN-2384 Refactor and refine NumberDictionary.

Signed-off-by: Li Yang <li...@apache.org>


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

Branch: refs/heads/master
Commit: 4dce0cf0806c29522aa2380296fa38e373028008
Parents: 65acd70
Author: xiefan46 <95...@qq.com>
Authored: Thu Feb 16 15:31:21 2017 +0800
Committer: Li Yang <li...@apache.org>
Committed: Tue Feb 21 10:10:21 2017 +0800

----------------------------------------------------------------------
 .../kylin/dict/Number2BytesConverter.java       | 224 +++++++++++++++++++
 .../org/apache/kylin/dict/NumberDictionary.java | 163 +-------------
 .../apache/kylin/dict/NumberDictionary2.java    |  11 +-
 .../kylin/dict/NumberDictionaryBuilder.java     |   2 +-
 .../dict/NumberDictionaryForestBuilder.java     |  48 +---
 .../apache/kylin/dict/NumberDictionaryTest.java |   3 +-
 6 files changed, 232 insertions(+), 219 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kylin/blob/4dce0cf0/core-dictionary/src/main/java/org/apache/kylin/dict/Number2BytesConverter.java
----------------------------------------------------------------------
diff --git a/core-dictionary/src/main/java/org/apache/kylin/dict/Number2BytesConverter.java b/core-dictionary/src/main/java/org/apache/kylin/dict/Number2BytesConverter.java
new file mode 100644
index 0000000..4e40527
--- /dev/null
+++ b/core-dictionary/src/main/java/org/apache/kylin/dict/Number2BytesConverter.java
@@ -0,0 +1,224 @@
+/*
+ * 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.dict;
+
+import org.apache.kylin.common.util.Bytes;
+
+import java.io.Serializable;
+
+/**
+ * Created by xiefan on 17-1-20.
+ */
+public class Number2BytesConverter implements BytesConverter<String>, Serializable {
+
+    private static final long serialVersionUID = 1L;
+
+    public static final int MAX_DIGITS_BEFORE_DECIMAL_POINT_LEGACY = 16;
+
+    public static final int MAX_DIGITS_BEFORE_DECIMAL_POINT = 19;
+
+    int maxDigitsBeforeDecimalPoint;
+
+    static final transient ThreadLocal<NumberBytesCodec> LOCAL = new ThreadLocal<NumberBytesCodec>();
+
+    static NumberBytesCodec getCodec(int maxDigitsBeforeDecimalPoint) {
+        NumberBytesCodec codec = LOCAL.get();
+        if (codec == null) {
+            codec = new NumberBytesCodec(maxDigitsBeforeDecimalPoint);
+            LOCAL.set(codec);
+        }
+        return codec;
+    }
+
+    public Number2BytesConverter(){
+        this.maxDigitsBeforeDecimalPoint = MAX_DIGITS_BEFORE_DECIMAL_POINT;
+    }
+
+    public Number2BytesConverter(int maxDigitsBeforeDecimalPoint) {
+        this.maxDigitsBeforeDecimalPoint = maxDigitsBeforeDecimalPoint;
+    }
+
+    @Override
+    public byte[] convertToBytes(String v) {
+        NumberBytesCodec codec = getCodec(this.maxDigitsBeforeDecimalPoint);
+        byte[] num = Bytes.toBytes(v);
+        codec.encodeNumber(num, 0, num.length);
+        return Bytes.copy(codec.buf, codec.bufOffset, codec.bufLen);
+    }
+
+    @Override
+    public String convertFromBytes(byte[] b, int offset, int length) {
+        NumberBytesCodec codec = getCodec(this.maxDigitsBeforeDecimalPoint);
+        byte[] backup = codec.buf;
+        codec.buf = b;
+        codec.bufOffset = offset;
+        codec.bufLen = length;
+        int len = codec.decodeNumber(backup, 0);
+        codec.buf = backup;
+        return Bytes.toString(backup, 0, len);
+    }
+
+    // encode a number into an order preserving byte sequence
+    // for positives -- padding '0'
+    // for negatives -- '-' sign, padding '9', invert digits, and terminate by ';'
+    static class NumberBytesCodec {
+        int maxDigitsBeforeDecimalPoint;
+        byte[] buf;
+        int bufOffset;
+        int bufLen;
+
+        NumberBytesCodec(int maxDigitsBeforeDecimalPoint) {
+            this.maxDigitsBeforeDecimalPoint = maxDigitsBeforeDecimalPoint;
+            this.buf = new byte[maxDigitsBeforeDecimalPoint * 3];
+            this.bufOffset = 0;
+            this.bufLen = 0;
+        }
+
+        void encodeNumber(byte[] value, int offset, int len) {
+            if (len == 0) {
+                bufOffset = 0;
+                bufLen = 0;
+                return;
+            }
+
+
+            if (len > buf.length) {
+                throw new IllegalArgumentException("Too many digits for NumberDictionary: " + Bytes.toString(value, offset, len) + ". Internal buffer is only " + buf.length + " bytes");
+            }
+
+            boolean negative = value[offset] == '-';
+
+            // terminate negative ';'
+            int start = buf.length - len;
+            int end = buf.length;
+            if (negative) {
+                start--;
+                end--;
+                buf[end] = ';';
+            }
+
+            // copy & find decimal point
+            int decimalPoint = end;
+            for (int i = start, j = offset; i < end; i++, j++) {
+                buf[i] = value[j];
+                if (buf[i] == '.' && i < decimalPoint) {
+                    decimalPoint = i;
+                }
+            }
+            // remove '-' sign
+            if (negative) {
+                start++;
+            }
+
+            // prepend '0'
+            int nZeroPadding = maxDigitsBeforeDecimalPoint - (decimalPoint - start);
+            if (nZeroPadding < 0 || nZeroPadding + 1 > start)
+                throw new IllegalArgumentException("Too many digits for NumberDictionary: " + Bytes.toString(value, offset, len) + ". Expect " + maxDigitsBeforeDecimalPoint + " digits before decimal point at max.");
+            for (int i = 0; i < nZeroPadding; i++) {
+                buf[--start] = '0';
+            }
+
+            // consider negative
+            if (negative) {
+                buf[--start] = '-';
+                for (int i = start + 1; i < buf.length; i++) {
+                    int c = buf[i];
+                    if (c >= '0' && c <= '9') {
+                        buf[i] = (byte) ('9' - (c - '0'));
+                    }
+                }
+            } else {
+                buf[--start] = '0';
+            }
+
+            bufOffset = start;
+            bufLen = buf.length - start;
+
+            // remove 0 in tail after the decimal point
+            if (decimalPoint != end) {
+                if (negative == true) {
+                    while (buf[bufOffset + bufLen - 2] == '9' && (bufOffset + bufLen - 2 > decimalPoint)) {
+                        bufLen--;
+                    }
+
+                    if (bufOffset + bufLen - 2 == decimalPoint) {
+                        bufLen--;
+                    }
+
+                    buf[bufOffset + bufLen - 1] = ';';
+                } else {
+                    while (buf[bufOffset + bufLen - 1] == '0' && (bufOffset + bufLen - 1 > decimalPoint)) {
+                        bufLen--;
+                    }
+
+                    if (bufOffset + bufLen - 1 == decimalPoint) {
+                        bufLen--;
+                    }
+
+                }
+            }
+        }
+
+        int decodeNumber(byte[] returnValue, int offset) {
+            if (bufLen == 0) {
+                return 0;
+            }
+
+            int in = bufOffset;
+            int end = bufOffset + bufLen;
+            int out = offset;
+
+            // sign
+            boolean negative = buf[in] == '-';
+            if (negative) {
+                returnValue[out++] = '-';
+                in++;
+                end--;
+            }
+
+            // remove padding
+            byte padding = (byte) (negative ? '9' : '0');
+            for (; in < end; in++) {
+                if (buf[in] != padding)
+                    break;
+            }
+
+            // all paddings before '.', special case for '0'
+            if (in == end || !(buf[in] >= '0' && buf[in] <= '9')) {
+                returnValue[out++] = '0';
+            }
+
+            // copy the rest
+            if (negative) {
+                for (; in < end; in++, out++) {
+                    int c = buf[in];
+                    if (c >= '0' && c <= '9') {
+                        c = '9' - (c - '0');
+                    }
+                    returnValue[out] = (byte) c;
+                }
+            } else {
+                System.arraycopy(buf, in, returnValue, out, end - in);
+                out += end - in;
+            }
+
+            return out - offset;
+        }
+    }
+
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/kylin/blob/4dce0cf0/core-dictionary/src/main/java/org/apache/kylin/dict/NumberDictionary.java
----------------------------------------------------------------------
diff --git a/core-dictionary/src/main/java/org/apache/kylin/dict/NumberDictionary.java b/core-dictionary/src/main/java/org/apache/kylin/dict/NumberDictionary.java
index de28440..b987eda 100644
--- a/core-dictionary/src/main/java/org/apache/kylin/dict/NumberDictionary.java
+++ b/core-dictionary/src/main/java/org/apache/kylin/dict/NumberDictionary.java
@@ -18,167 +18,15 @@
 
 package org.apache.kylin.dict;
 
-import org.apache.kylin.common.util.Bytes;
 
 /**
  * @author yangli9
  * 
  */
 @SuppressWarnings("serial")
+@Deprecated
 public class NumberDictionary<T> extends TrieDictionary<T> {
 
-    public static final int MAX_DIGITS_BEFORE_DECIMAL_POINT_LEGACY = 16;
-    public static final int MAX_DIGITS_BEFORE_DECIMAL_POINT = 19;
-
-    // encode a number into an order preserving byte sequence
-    // for positives -- padding '0'
-    // for negatives -- '-' sign, padding '9', invert digits, and terminate by ';'
-    static class NumberBytesCodec {
-        int maxDigitsBeforeDecimalPoint;
-        byte[] buf;
-        int bufOffset;
-        int bufLen;
-
-        NumberBytesCodec(int maxDigitsBeforeDecimalPoint) {
-            this.maxDigitsBeforeDecimalPoint = maxDigitsBeforeDecimalPoint;
-            this.buf = new byte[maxDigitsBeforeDecimalPoint * 3];
-            this.bufOffset = 0;
-            this.bufLen = 0;
-        }
-
-        void encodeNumber(byte[] value, int offset, int len) {
-            if (len == 0) {
-                bufOffset = 0;
-                bufLen = 0;
-                return;
-            }
-
-
-            if (len > buf.length) {
-                throw new IllegalArgumentException("Too many digits for NumberDictionary: " + Bytes.toString(value, offset, len) + ". Internal buffer is only " + buf.length + " bytes");
-            }
-
-            boolean negative = value[offset] == '-';
-
-            // terminate negative ';'
-            int start = buf.length - len;
-            int end = buf.length;
-            if (negative) {
-                start--;
-                end--;
-                buf[end] = ';';
-            }
-
-            // copy & find decimal point
-            int decimalPoint = end;
-            for (int i = start, j = offset; i < end; i++, j++) {
-                buf[i] = value[j];
-                if (buf[i] == '.' && i < decimalPoint) {
-                    decimalPoint = i;
-                }
-            }
-            // remove '-' sign
-            if (negative) {
-                start++;
-            }
-
-            // prepend '0'
-            int nZeroPadding = maxDigitsBeforeDecimalPoint - (decimalPoint - start);
-            if (nZeroPadding < 0 || nZeroPadding + 1 > start)
-                throw new IllegalArgumentException("Too many digits for NumberDictionary: " + Bytes.toString(value, offset, len) + ". Expect " + maxDigitsBeforeDecimalPoint + " digits before decimal point at max.");
-            for (int i = 0; i < nZeroPadding; i++) {
-                buf[--start] = '0';
-            }
-
-            // consider negative
-            if (negative) {
-                buf[--start] = '-';
-                for (int i = start + 1; i < buf.length; i++) {
-                    int c = buf[i];
-                    if (c >= '0' && c <= '9') {
-                        buf[i] = (byte) ('9' - (c - '0'));
-                    }
-                }
-            } else {
-                buf[--start] = '0';
-            }
-
-            bufOffset = start;
-            bufLen = buf.length - start;
-
-            // remove 0 in tail after the decimal point
-            if (decimalPoint != end) {
-                if (negative == true) {
-                    while (buf[bufOffset + bufLen - 2] == '9' && (bufOffset + bufLen - 2 > decimalPoint)) {
-                        bufLen--;
-                    }
-
-                    if (bufOffset + bufLen - 2 == decimalPoint) {
-                        bufLen--;
-                    }
-
-                    buf[bufOffset + bufLen - 1] = ';';
-                } else {
-                    while (buf[bufOffset + bufLen - 1] == '0' && (bufOffset + bufLen - 1 > decimalPoint)) {
-                        bufLen--;
-                    }
-
-                    if (bufOffset + bufLen - 1 == decimalPoint) {
-                        bufLen--;
-                    }
-
-                }
-            }
-        }
-
-        int decodeNumber(byte[] returnValue, int offset) {
-            if (bufLen == 0) {
-                return 0;
-            }
-
-            int in = bufOffset;
-            int end = bufOffset + bufLen;
-            int out = offset;
-
-            // sign
-            boolean negative = buf[in] == '-';
-            if (negative) {
-                returnValue[out++] = '-';
-                in++;
-                end--;
-            }
-
-            // remove padding
-            byte padding = (byte) (negative ? '9' : '0');
-            for (; in < end; in++) {
-                if (buf[in] != padding)
-                    break;
-            }
-
-            // all paddings before '.', special case for '0'
-            if (in == end || !(buf[in] >= '0' && buf[in] <= '9')) {
-                returnValue[out++] = '0';
-            }
-
-            // copy the rest
-            if (negative) {
-                for (; in < end; in++, out++) {
-                    int c = buf[in];
-                    if (c >= '0' && c <= '9') {
-                        c = '9' - (c - '0');
-                    }
-                    returnValue[out] = (byte) c;
-                }
-            } else {
-                System.arraycopy(buf, in, returnValue, out, end - in);
-                out += end - in;
-            }
-
-            return out - offset;
-        }
-    }
-
-    static transient ThreadLocal<NumberBytesCodec> localCodec = new ThreadLocal<NumberBytesCodec>();
 
     // ============================================================================
 
@@ -190,15 +38,6 @@ public class NumberDictionary<T> extends TrieDictionary<T> {
         super(trieBytes);
     }
 
-    protected NumberBytesCodec getCodec() {
-        NumberBytesCodec codec = localCodec.get();
-        if (codec == null) {
-            codec = new NumberBytesCodec(MAX_DIGITS_BEFORE_DECIMAL_POINT_LEGACY);
-            localCodec.set(codec);
-        }
-        return codec;
-    }
-
     @Override
     protected boolean isNullObjectForm(T value) {
         return value == null || value.equals("");

http://git-wip-us.apache.org/repos/asf/kylin/blob/4dce0cf0/core-dictionary/src/main/java/org/apache/kylin/dict/NumberDictionary2.java
----------------------------------------------------------------------
diff --git a/core-dictionary/src/main/java/org/apache/kylin/dict/NumberDictionary2.java b/core-dictionary/src/main/java/org/apache/kylin/dict/NumberDictionary2.java
index 80e9940..3879d33 100644
--- a/core-dictionary/src/main/java/org/apache/kylin/dict/NumberDictionary2.java
+++ b/core-dictionary/src/main/java/org/apache/kylin/dict/NumberDictionary2.java
@@ -22,10 +22,9 @@ package org.apache.kylin.dict;
  * This class uses MAX_DIGITS_BEFORE_DECIMAL_POINT (=19) instead of legacy (=16).
  */
 @SuppressWarnings("serial")
+@Deprecated
 public class NumberDictionary2<T> extends NumberDictionary<T> {
 
-    static transient ThreadLocal<NumberBytesCodec> localCodec = new ThreadLocal<NumberBytesCodec>();
-
     // ============================================================================
 
     public NumberDictionary2() { // default constructor for Writable interface
@@ -36,13 +35,5 @@ public class NumberDictionary2<T> extends NumberDictionary<T> {
         super(trieBytes);
     }
 
-    protected NumberBytesCodec getCodec() {
-        NumberBytesCodec codec = localCodec.get();
-        if (codec == null) {
-            codec = new NumberBytesCodec(MAX_DIGITS_BEFORE_DECIMAL_POINT);
-            localCodec.set(codec);
-        }
-        return codec;
-    }
 
 }

http://git-wip-us.apache.org/repos/asf/kylin/blob/4dce0cf0/core-dictionary/src/main/java/org/apache/kylin/dict/NumberDictionaryBuilder.java
----------------------------------------------------------------------
diff --git a/core-dictionary/src/main/java/org/apache/kylin/dict/NumberDictionaryBuilder.java b/core-dictionary/src/main/java/org/apache/kylin/dict/NumberDictionaryBuilder.java
index 288e38f..26e4f89 100644
--- a/core-dictionary/src/main/java/org/apache/kylin/dict/NumberDictionaryBuilder.java
+++ b/core-dictionary/src/main/java/org/apache/kylin/dict/NumberDictionaryBuilder.java
@@ -29,7 +29,7 @@ public class NumberDictionaryBuilder extends TrieDictionaryBuilder<String> {
 
 
     public NumberDictionaryBuilder() {
-        super(new NumberDictionaryForestBuilder.Number2BytesConverter());
+        super(new Number2BytesConverter(Number2BytesConverter.MAX_DIGITS_BEFORE_DECIMAL_POINT));
     }
 
 

http://git-wip-us.apache.org/repos/asf/kylin/blob/4dce0cf0/core-dictionary/src/main/java/org/apache/kylin/dict/NumberDictionaryForestBuilder.java
----------------------------------------------------------------------
diff --git a/core-dictionary/src/main/java/org/apache/kylin/dict/NumberDictionaryForestBuilder.java b/core-dictionary/src/main/java/org/apache/kylin/dict/NumberDictionaryForestBuilder.java
index b072599..8b7026d 100644
--- a/core-dictionary/src/main/java/org/apache/kylin/dict/NumberDictionaryForestBuilder.java
+++ b/core-dictionary/src/main/java/org/apache/kylin/dict/NumberDictionaryForestBuilder.java
@@ -18,62 +18,20 @@
 
 package org.apache.kylin.dict;
 
-import java.io.Serializable;
-
-import org.apache.kylin.common.util.Bytes;
-import org.apache.kylin.dict.NumberDictionary.NumberBytesCodec;
-
 /**
  * Created by xiefan on 16-11-2.
  */
 public class NumberDictionaryForestBuilder extends TrieDictionaryForestBuilder<String> {
 
-    public static class Number2BytesConverter implements BytesConverter<String>, Serializable {
-        private static final long serialVersionUID = 1L;
-        
-        static final int MAX_DIGITS_BEFORE_DECIMAL_POINT = NumberDictionary.MAX_DIGITS_BEFORE_DECIMAL_POINT;
-        static final transient ThreadLocal<NumberBytesCodec> LOCAL = new ThreadLocal<NumberBytesCodec>();
-
-        static NumberBytesCodec getCodec() {
-            NumberBytesCodec codec = LOCAL.get();
-            if (codec == null) {
-                codec = new NumberBytesCodec(MAX_DIGITS_BEFORE_DECIMAL_POINT);
-                LOCAL.set(codec);
-            }
-            return codec;
-        }
-        
-        @Override
-        public byte[] convertToBytes(String v) {
-            NumberBytesCodec codec = getCodec();
-            byte[] num = Bytes.toBytes(v);
-            codec.encodeNumber(num, 0, num.length);
-            return Bytes.copy(codec.buf, codec.bufOffset, codec.bufLen);
-        }
-
-        @Override
-        public String convertFromBytes(byte[] b, int offset, int length) {
-            NumberBytesCodec codec = getCodec();
-            byte[] backup = codec.buf;
-            codec.buf = b;
-            codec.bufOffset = offset;
-            codec.bufLen = length;
-            int len = codec.decodeNumber(backup, 0);
-            codec.buf = backup;
-            return Bytes.toString(backup, 0, len);
-        }
-
-    }
-
     public NumberDictionaryForestBuilder() {
-        super(new Number2BytesConverter());
+        super(new Number2BytesConverter(Number2BytesConverter.MAX_DIGITS_BEFORE_DECIMAL_POINT));
     }
 
     public NumberDictionaryForestBuilder(int baseId) {
-        super(new Number2BytesConverter(), 0);
+        super(new Number2BytesConverter(Number2BytesConverter.MAX_DIGITS_BEFORE_DECIMAL_POINT), 0);
     }
 
     public NumberDictionaryForestBuilder(int baseId, int maxTrieSizeMB) {
-        super(new Number2BytesConverter(), 0, maxTrieSizeMB);
+        super(new Number2BytesConverter(Number2BytesConverter.MAX_DIGITS_BEFORE_DECIMAL_POINT), 0, maxTrieSizeMB);
     }
 }

http://git-wip-us.apache.org/repos/asf/kylin/blob/4dce0cf0/core-dictionary/src/test/java/org/apache/kylin/dict/NumberDictionaryTest.java
----------------------------------------------------------------------
diff --git a/core-dictionary/src/test/java/org/apache/kylin/dict/NumberDictionaryTest.java b/core-dictionary/src/test/java/org/apache/kylin/dict/NumberDictionaryTest.java
index 36eedf5..8da7208 100644
--- a/core-dictionary/src/test/java/org/apache/kylin/dict/NumberDictionaryTest.java
+++ b/core-dictionary/src/test/java/org/apache/kylin/dict/NumberDictionaryTest.java
@@ -18,6 +18,7 @@
 
 package org.apache.kylin.dict;
 
+import static org.apache.kylin.dict.Number2BytesConverter.MAX_DIGITS_BEFORE_DECIMAL_POINT;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.fail;
 
@@ -44,7 +45,7 @@ import com.google.common.collect.Sets;
  */
 public class NumberDictionaryTest extends LocalFileMetadataTestCase {
 
-    NumberDictionary.NumberBytesCodec codec = new NumberDictionary.NumberBytesCodec(NumberDictionary.MAX_DIGITS_BEFORE_DECIMAL_POINT);
+    Number2BytesConverter.NumberBytesCodec codec = new Number2BytesConverter.NumberBytesCodec(MAX_DIGITS_BEFORE_DECIMAL_POINT);
     Random rand = new Random();
 
     @Before