You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@kylin.apache.org by sh...@apache.org on 2016/06/24 07:04:54 UTC

[16/50] kylin git commit: KYLIN-1766 Add date/time dimension encoding to replace date/time dictionary

KYLIN-1766 Add date/time dimension encoding to replace date/time dictionary


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

Branch: refs/heads/stream_m1
Commit: 35f3e639cd4d50932a5dc548a5199928657b6b1d
Parents: 1b1e397
Author: Li Yang <li...@apache.org>
Authored: Thu Jun 16 17:07:50 2016 +0800
Committer: Yang Li <li...@apache.org>
Committed: Sat Jun 18 09:06:43 2016 +0800

----------------------------------------------------------------------
 .../apache/kylin/common/util/DateFormat.java    |  22 ++-
 .../apache/kylin/cube/model/RowKeyColDesc.java  |  40 +++--
 .../org/apache/kylin/cube/model/RowKeyDesc.java |  16 +-
 .../apache/kylin/dict/DateStrDictionary.java    |  22 +--
 .../kylin/dict/lookup/LookupStringTable.java    |  17 ++
 .../kylin/dict/DateStrDictionaryTest.java       |  16 +-
 .../kylin/dict/TimeStrDictionaryTest.java       |  76 +++++++++
 .../kylin/dict/TimeStrDictionaryTests.java      |  76 ---------
 .../kylin/dimension/AbstractDateDimEnc.java     | 160 +++++++++++++++++++
 .../org/apache/kylin/dimension/DateDimEnc.java  |  70 ++++++++
 .../dimension/DimensionEncodingFactory.java     |   2 +
 .../org/apache/kylin/dimension/TimeDimEnc.java  |  57 +++++++
 .../org/apache/kylin/metadata/tuple/Tuple.java  |  28 ++--
 .../apache/kylin/dimension/DateDimEncTest.java  | 101 ++++++++++++
 .../apache/kylin/dimension/TimeDimEncTest.java  |  99 ++++++++++++
 pom.xml                                         |   2 +-
 16 files changed, 652 insertions(+), 152 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kylin/blob/35f3e639/core-common/src/main/java/org/apache/kylin/common/util/DateFormat.java
----------------------------------------------------------------------
diff --git a/core-common/src/main/java/org/apache/kylin/common/util/DateFormat.java b/core-common/src/main/java/org/apache/kylin/common/util/DateFormat.java
index 71c0344..2472992 100644
--- a/core-common/src/main/java/org/apache/kylin/common/util/DateFormat.java
+++ b/core-common/src/main/java/org/apache/kylin/common/util/DateFormat.java
@@ -18,12 +18,13 @@
 package org.apache.kylin.common.util;
 
 import java.text.ParseException;
-import java.text.SimpleDateFormat;
 import java.util.Date;
 import java.util.Map;
 import java.util.TimeZone;
 import java.util.concurrent.ConcurrentHashMap;
 
+import org.apache.commons.lang3.time.FastDateFormat;
+
 public class DateFormat {
 
     public static final String COMPACT_DATE_PATTERN = "yyyyMMdd";
@@ -37,20 +38,15 @@ public class DateFormat {
             DEFAULT_DATETIME_PATTERN_WITH_MILLISECONDS, //
             COMPACT_DATE_PATTERN };
 
-    static final private Map<String, ThreadLocal<SimpleDateFormat>> threadLocalMap = new ConcurrentHashMap<String, ThreadLocal<SimpleDateFormat>>();
+    static final private Map<String, FastDateFormat> formatMap = new ConcurrentHashMap<String, FastDateFormat>();
 
-    public static SimpleDateFormat getDateFormat(String datePattern) {
-        ThreadLocal<SimpleDateFormat> formatThreadLocal = threadLocalMap.get(datePattern);
-        if (formatThreadLocal == null) {
-            threadLocalMap.put(datePattern, formatThreadLocal = new ThreadLocal<SimpleDateFormat>());
-        }
-        SimpleDateFormat format = formatThreadLocal.get();
-        if (format == null) {
-            format = new SimpleDateFormat(datePattern);
-            format.setTimeZone(TimeZone.getTimeZone("GMT")); // NOTE: this must be GMT to calculate epoch date correctly
-            formatThreadLocal.set(format);
+    public static FastDateFormat getDateFormat(String datePattern) {
+        FastDateFormat r = formatMap.get(datePattern);
+        if (r == null) {
+            r = FastDateFormat.getInstance(datePattern, TimeZone.getTimeZone("GMT")); // NOTE: this must be GMT to calculate epoch date correctly
+            formatMap.put(datePattern, r);
         }
-        return format;
+        return r;
     }
 
     public static String formatToDateStr(long millis) {

http://git-wip-us.apache.org/repos/asf/kylin/blob/35f3e639/core-cube/src/main/java/org/apache/kylin/cube/model/RowKeyColDesc.java
----------------------------------------------------------------------
diff --git a/core-cube/src/main/java/org/apache/kylin/cube/model/RowKeyColDesc.java b/core-cube/src/main/java/org/apache/kylin/cube/model/RowKeyColDesc.java
index e1be041..c4174a6 100644
--- a/core-cube/src/main/java/org/apache/kylin/cube/model/RowKeyColDesc.java
+++ b/core-cube/src/main/java/org/apache/kylin/cube/model/RowKeyColDesc.java
@@ -18,11 +18,15 @@
 
 package org.apache.kylin.cube.model;
 
+import java.util.Map;
+
 import org.apache.commons.lang.StringUtils;
-import org.apache.kylin.common.util.StringUtil;
+import org.apache.kylin.dimension.DateDimEnc;
 import org.apache.kylin.dimension.DictionaryDimEnc;
 import org.apache.kylin.dimension.DimensionEncoding;
 import org.apache.kylin.dimension.DimensionEncodingFactory;
+import org.apache.kylin.dimension.TimeDimEnc;
+import org.apache.kylin.metadata.datatype.DataType;
 import org.apache.kylin.metadata.model.TblColRef;
 
 import com.fasterxml.jackson.annotation.JsonAutoDetect;
@@ -55,15 +59,32 @@ public class RowKeyColDesc {
     private int bitIndex;
     private TblColRef colRef;
 
-    public void init() {
-        Preconditions.checkState(StringUtils.isNotEmpty(this.encoding));
+    public void init(int index, Map<String, TblColRef> colNameAbbr, CubeDesc cubeDesc) {
+        column = column.toUpperCase();
+        bitIndex = index;
+        colRef = colNameAbbr.get(column);
+        if (colRef == null) {
+            throw new IllegalArgumentException("Cannot find rowkey column " + column + " in cube " + cubeDesc);
+        }
 
+        Preconditions.checkState(StringUtils.isNotEmpty(this.encoding));
         Object[] encodingConf = DimensionEncoding.parseEncodingConf(this.encoding);
-        this.encodingName = (String) encodingConf[0];
-        this.encodingArgs = (String[])encodingConf[1];
+        encodingName = (String) encodingConf[0];
+        encodingArgs = (String[]) encodingConf[1];
 
         if (!DimensionEncodingFactory.isVaildEncoding(this.encodingName))
             throw new IllegalArgumentException("Not supported row key col encoding: '" + this.encoding + "'");
+        
+        // convert date/time dictionary to DimensionEncoding implicitly, date/time dictionary is deprecated
+        if (DictionaryDimEnc.ENCODING_NAME.equals(encodingName)) {
+            DataType type = colRef.getType();
+            if (type.isDate()) {
+                encodingName = DateDimEnc.ENCODING_NAME;
+            }
+            if (type.isTime() || type.isTimestamp() || type.isDatetime()) {
+                encodingName = TimeDimEnc.ENCODING_NAME;
+            }
+        }
     }
 
     public String getEncoding() {
@@ -106,18 +127,10 @@ public class RowKeyColDesc {
         return bitIndex;
     }
 
-    void setBitIndex(int index) {
-        this.bitIndex = index;
-    }
-
     public TblColRef getColRef() {
         return colRef;
     }
 
-    void setColRef(TblColRef colRef) {
-        this.colRef = colRef;
-    }
-
     public String getIndex() {
         return index;
     }
@@ -125,6 +138,7 @@ public class RowKeyColDesc {
     public void setIndex(String index) {
         this.index = index;
     }
+
     @Override
     public String toString() {
         return Objects.toStringHelper(this).add("column", column).add("encoding", encoding).toString();

http://git-wip-us.apache.org/repos/asf/kylin/blob/35f3e639/core-cube/src/main/java/org/apache/kylin/cube/model/RowKeyDesc.java
----------------------------------------------------------------------
diff --git a/core-cube/src/main/java/org/apache/kylin/cube/model/RowKeyDesc.java b/core-cube/src/main/java/org/apache/kylin/cube/model/RowKeyDesc.java
index cf7d731..2dac2bf 100644
--- a/core-cube/src/main/java/org/apache/kylin/cube/model/RowKeyDesc.java
+++ b/core-cube/src/main/java/org/apache/kylin/cube/model/RowKeyDesc.java
@@ -18,7 +18,11 @@
 
 package org.apache.kylin.cube.model;
 
-import java.util.*;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
 
 import org.apache.commons.lang.ArrayUtils;
 import org.apache.kylin.dimension.DictionaryDimEnc;
@@ -106,15 +110,7 @@ public class RowKeyDesc {
 
         for (int i = 0; i < rowkeyColumns.length; i++) {
             RowKeyColDesc rowKeyColDesc = rowkeyColumns[i];
-            rowKeyColDesc.init();
-            String column = rowKeyColDesc.getColumn();
-            rowKeyColDesc.setColumn(column.toUpperCase());
-            rowKeyColDesc.setBitIndex(rowkeyColumns.length - i - 1);
-            rowKeyColDesc.setColRef(colNameAbbr.get(column));
-            if (rowKeyColDesc.getColRef() == null) {
-                throw new IllegalArgumentException("Cannot find rowkey column " + column + " in cube " + cubeDesc);
-            }
-
+            rowKeyColDesc.init(rowkeyColumns.length - i - 1, colNameAbbr, cubeDesc);
             columnMap.put(rowKeyColDesc.getColRef(), rowKeyColDesc);
 
             if (rowKeyColDesc.isShardBy()) {

http://git-wip-us.apache.org/repos/asf/kylin/blob/35f3e639/core-dictionary/src/main/java/org/apache/kylin/dict/DateStrDictionary.java
----------------------------------------------------------------------
diff --git a/core-dictionary/src/main/java/org/apache/kylin/dict/DateStrDictionary.java b/core-dictionary/src/main/java/org/apache/kylin/dict/DateStrDictionary.java
index 62b06aa..44a1fa4 100644
--- a/core-dictionary/src/main/java/org/apache/kylin/dict/DateStrDictionary.java
+++ b/core-dictionary/src/main/java/org/apache/kylin/dict/DateStrDictionary.java
@@ -31,6 +31,7 @@ import java.util.Date;
 
 import org.apache.commons.lang.StringUtils;
 import org.apache.kylin.common.util.Dictionary;
+import org.apache.kylin.dimension.DateDimEnc;
 
 /**
  * A dictionary for date string (date only, no time).
@@ -45,8 +46,6 @@ import org.apache.kylin.common.util.Dictionary;
 @SuppressWarnings("serial")
 public class DateStrDictionary extends Dictionary<String> {
 
-    static final int ID_9999_12_31 = 3652426; // assume 0 based
-
     private String pattern;
     private int baseId;
     private int maxId;
@@ -62,7 +61,7 @@ public class DateStrDictionary extends Dictionary<String> {
     private void init(String datePattern, int baseId) {
         this.pattern = datePattern;
         this.baseId = baseId;
-        this.maxId = baseId + ID_9999_12_31;
+        this.maxId = baseId + DateDimEnc.ID_9999_12_31;
     }
 
     @Override
@@ -93,7 +92,7 @@ public class DateStrDictionary extends Dictionary<String> {
     @Override
     final protected int getIdFromValueImpl(String value, int roundFlag) {
         Date date = stringToDate(value, pattern);
-        int id = calcIdFromSeqNo(getNumOfDaysSince0000(date));
+        int id = calcIdFromSeqNo((int) DateDimEnc.getNumOfDaysSince0000FromMillis(date.getTime()));
         if (id < baseId || id > maxId)
             throw new IllegalArgumentException("'" + value + "' encodes to '" + id + "' which is out of range [" + baseId + "," + maxId + "]");
 
@@ -104,19 +103,8 @@ public class DateStrDictionary extends Dictionary<String> {
     final protected String getValueFromIdImpl(int id) {
         if (id < baseId || id > maxId)
             throw new IllegalArgumentException("ID '" + id + "' is out of range [" + baseId + "," + maxId + "]");
-        Date d = getDateFromNumOfDaysSince0000(calcSeqNoFromId(id));
-        return dateToString(d, pattern);
-    }
-
-    private int getNumOfDaysSince0000(Date d) {
-        // 86400000 = 1000 * 60 * 60 * 24
-        // -719530 is offset of 0000-01-01
-        return (int) (d.getTime() / 86400000 + 719530);
-    }
-
-    private Date getDateFromNumOfDaysSince0000(int n) {
-        long millis = ((long) n - 719530) * 86400000;
-        return new Date(millis);
+        long millis = DateDimEnc.getMillisFromNumOfDaysSince0000(calcSeqNoFromId(id));
+        return dateToString(new Date(millis), pattern);
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/kylin/blob/35f3e639/core-dictionary/src/main/java/org/apache/kylin/dict/lookup/LookupStringTable.java
----------------------------------------------------------------------
diff --git a/core-dictionary/src/main/java/org/apache/kylin/dict/lookup/LookupStringTable.java b/core-dictionary/src/main/java/org/apache/kylin/dict/lookup/LookupStringTable.java
index ce73feb..9f6346a 100644
--- a/core-dictionary/src/main/java/org/apache/kylin/dict/lookup/LookupStringTable.java
+++ b/core-dictionary/src/main/java/org/apache/kylin/dict/lookup/LookupStringTable.java
@@ -20,6 +20,8 @@ package org.apache.kylin.dict.lookup;
 
 import java.io.IOException;
 
+import org.apache.kylin.common.util.DateFormat;
+import org.apache.kylin.metadata.model.ColumnDesc;
 import org.apache.kylin.metadata.model.TableDesc;
 import org.apache.kylin.source.ReadableTable;
 
@@ -29,12 +31,27 @@ import org.apache.kylin.source.ReadableTable;
  */
 public class LookupStringTable extends LookupTable<String> {
 
+    int[] keyIndexOfDates;
+    
     public LookupStringTable(TableDesc tableDesc, String[] keyColumns, ReadableTable table) throws IOException {
         super(tableDesc, keyColumns, table);
     }
 
     @Override
     protected String[] convertRow(String[] cols) {
+        if (keyIndexOfDates == null) {
+            keyIndexOfDates = new int[keyColumns.length];
+            for (int i = 0; i < keyColumns.length; i++) {
+                ColumnDesc col = tableDesc.findColumnByName(keyColumns[i]);
+                keyIndexOfDates[i] = col.getType().isDateTimeFamily() ? col.getZeroBasedIndex() : -1;
+            }
+        }
+        
+        for (int i = 0; i < keyIndexOfDates.length; i++) {
+            int c = keyIndexOfDates[i];
+            if (c >= 0)
+                cols[c] = String.valueOf(DateFormat.stringToMillis(cols[c]));
+        }
         return cols;
     }
 

http://git-wip-us.apache.org/repos/asf/kylin/blob/35f3e639/core-dictionary/src/test/java/org/apache/kylin/dict/DateStrDictionaryTest.java
----------------------------------------------------------------------
diff --git a/core-dictionary/src/test/java/org/apache/kylin/dict/DateStrDictionaryTest.java b/core-dictionary/src/test/java/org/apache/kylin/dict/DateStrDictionaryTest.java
index b7b9338..ffc8ceb 100644
--- a/core-dictionary/src/test/java/org/apache/kylin/dict/DateStrDictionaryTest.java
+++ b/core-dictionary/src/test/java/org/apache/kylin/dict/DateStrDictionaryTest.java
@@ -22,6 +22,7 @@ import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertNull;
 import static org.junit.Assert.fail;
 
+import org.apache.kylin.dimension.DateDimEnc;
 import org.junit.Before;
 import org.junit.Test;
 
@@ -37,7 +38,7 @@ public class DateStrDictionaryTest {
     @Test
     public void testMinMaxId() {
         assertEquals(0, dict.getIdFromValue("0000-01-01"));
-        assertEquals(DateStrDictionary.ID_9999_12_31, dict.getIdFromValue("9999-12-31"));
+        assertEquals(DateDimEnc.ID_9999_12_31, dict.getIdFromValue("9999-12-31"));
 
         try {
             dict.getValueFromId(-2); // -1 is id for NULL
@@ -47,7 +48,7 @@ public class DateStrDictionaryTest {
         }
 
         try {
-            dict.getValueFromId(DateStrDictionary.ID_9999_12_31 + 1);
+            dict.getValueFromId(DateDimEnc.ID_9999_12_31 + 1);
             fail("IllegalArgumentException expected");
         } catch (IllegalArgumentException e) {
             // good
@@ -79,6 +80,12 @@ public class DateStrDictionaryTest {
         checkPair("9999-12-31");
     }
 
+    private void checkPair(String dateStr) {
+        int id = dict.getIdFromValue(dateStr);
+        String dateStrBack = dict.getValueFromId(id);
+        assertEquals(dateStr, dateStrBack);
+    }
+
     @Test
     public void testIllegalArgument() {
         try {
@@ -96,9 +103,4 @@ public class DateStrDictionaryTest {
         }
     }
 
-    private void checkPair(String dateStr) {
-        int id = dict.getIdFromValue(dateStr);
-        String dateStrBack = dict.getValueFromId(id);
-        assertEquals(dateStr, dateStrBack);
-    }
 }

http://git-wip-us.apache.org/repos/asf/kylin/blob/35f3e639/core-dictionary/src/test/java/org/apache/kylin/dict/TimeStrDictionaryTest.java
----------------------------------------------------------------------
diff --git a/core-dictionary/src/test/java/org/apache/kylin/dict/TimeStrDictionaryTest.java b/core-dictionary/src/test/java/org/apache/kylin/dict/TimeStrDictionaryTest.java
new file mode 100644
index 0000000..ce07a86
--- /dev/null
+++ b/core-dictionary/src/test/java/org/apache/kylin/dict/TimeStrDictionaryTest.java
@@ -0,0 +1,76 @@
+/*
+ * 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.DateFormat;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+
+/**
+ */
+public class TimeStrDictionaryTest {
+    TimeStrDictionary dict;
+
+    @Before
+    public void setup() {
+        dict = new TimeStrDictionary();
+    }
+
+    @Test
+    public void basicTest() {
+        int a = dict.getIdFromValue("1999-01-01");
+        int b = dict.getIdFromValue("1999-01-01 00:00:00");
+        int c = dict.getIdFromValue("1999-01-01 00:00:00.000");
+        int d = dict.getIdFromValue("1999-01-01 00:00:00.022");
+
+        Assert.assertEquals(a, b);
+        Assert.assertEquals(a, c);
+        Assert.assertEquals(a, d);
+    }
+
+    @Test
+    public void testEncodeDecode() {
+        encodeDecode("1999-01-12");
+        encodeDecode("2038-01-09");
+        encodeDecode("2038-01-08");
+        encodeDecode("1970-01-01");
+        encodeDecode("1970-01-02");
+
+        encodeDecode("1999-01-12 11:00:01");
+        encodeDecode("2038-01-09 01:01:02");
+        encodeDecode("2038-01-19 03:14:06");
+        encodeDecode("1970-01-01 23:22:11");
+        encodeDecode("1970-01-02 23:22:11");
+    }
+
+    @Test
+    public void testIllegal() {
+        Assert.assertEquals(-1, dict.getIdFromValue("2038-01-19 03:14:07"));
+    }
+
+    public void encodeDecode(String origin) {
+        int a = dict.getIdFromValue(origin);
+        String back = dict.getValueFromId(a);
+
+        String originChoppingMilis = DateFormat.formatToTimeWithoutMilliStr(DateFormat.stringToMillis(origin));
+        Assert.assertEquals(originChoppingMilis, back);
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/kylin/blob/35f3e639/core-dictionary/src/test/java/org/apache/kylin/dict/TimeStrDictionaryTests.java
----------------------------------------------------------------------
diff --git a/core-dictionary/src/test/java/org/apache/kylin/dict/TimeStrDictionaryTests.java b/core-dictionary/src/test/java/org/apache/kylin/dict/TimeStrDictionaryTests.java
deleted file mode 100644
index 2b3cc74..0000000
--- a/core-dictionary/src/test/java/org/apache/kylin/dict/TimeStrDictionaryTests.java
+++ /dev/null
@@ -1,76 +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.dict;
-
-import org.apache.kylin.common.util.DateFormat;
-import org.junit.Assert;
-import org.junit.Before;
-import org.junit.Test;
-
-/**
- */
-public class TimeStrDictionaryTests {
-    TimeStrDictionary dict;
-
-    @Before
-    public void setup() {
-        dict = new TimeStrDictionary();
-    }
-
-    @Test
-    public void basicTest() {
-        int a = dict.getIdFromValue("1999-01-01");
-        int b = dict.getIdFromValue("1999-01-01 00:00:00");
-        int c = dict.getIdFromValue("1999-01-01 00:00:00.000");
-        int d = dict.getIdFromValue("1999-01-01 00:00:00.022");
-
-        Assert.assertEquals(a, b);
-        Assert.assertEquals(a, c);
-        Assert.assertEquals(a, d);
-    }
-
-    @Test
-    public void testEncodeDecode() {
-        encodeDecode("1999-01-12");
-        encodeDecode("2038-01-09");
-        encodeDecode("2038-01-08");
-        encodeDecode("1970-01-01");
-        encodeDecode("1970-01-02");
-
-        encodeDecode("1999-01-12 11:00:01");
-        encodeDecode("2038-01-09 01:01:02");
-        encodeDecode("2038-01-19 03:14:06");
-        encodeDecode("1970-01-01 23:22:11");
-        encodeDecode("1970-01-02 23:22:11");
-    }
-
-    @Test
-    public void testIllegal() {
-        Assert.assertEquals(-1, dict.getIdFromValue("2038-01-19 03:14:07"));
-    }
-
-    public void encodeDecode(String origin) {
-        int a = dict.getIdFromValue(origin);
-        String back = dict.getValueFromId(a);
-
-        String originChoppingMilis = DateFormat.formatToTimeWithoutMilliStr(DateFormat.stringToMillis(origin));
-        Assert.assertEquals(originChoppingMilis, back);
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/kylin/blob/35f3e639/core-metadata/src/main/java/org/apache/kylin/dimension/AbstractDateDimEnc.java
----------------------------------------------------------------------
diff --git a/core-metadata/src/main/java/org/apache/kylin/dimension/AbstractDateDimEnc.java b/core-metadata/src/main/java/org/apache/kylin/dimension/AbstractDateDimEnc.java
new file mode 100644
index 0000000..41460ed
--- /dev/null
+++ b/core-metadata/src/main/java/org/apache/kylin/dimension/AbstractDateDimEnc.java
@@ -0,0 +1,160 @@
+/*
+ * 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.io.Serializable;
+import java.io.UnsupportedEncodingException;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+
+import org.apache.kylin.common.util.BytesUtil;
+import org.apache.kylin.common.util.DateFormat;
+import org.apache.kylin.metadata.datatype.DataTypeSerializer;
+
+public class AbstractDateDimEnc extends DimensionEncoding {
+    private static final long serialVersionUID = 1L;
+
+    interface IMillisCodec extends Serializable {
+        long millisToCode(long millis);
+
+        long codeToMillis(long code);
+    }
+
+    // ============================================================================
+    private int fixedLen;
+    private IMillisCodec codec;
+
+    protected AbstractDateDimEnc(int fixedLen, IMillisCodec codec) {
+        this.fixedLen = fixedLen;
+        this.codec = codec;
+    }
+
+    @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;
+        }
+
+        try {
+            String str = new String(value, 0, valueLen, "ISO-8859-1");
+            encode(str, output, outputOffset);
+        } catch (UnsupportedEncodingException e) {
+            // never happen
+        }
+    }
+
+    void encode(String value, byte[] output, int outputOffset) {
+        if (value == null) {
+            Arrays.fill(output, outputOffset, outputOffset + fixedLen, NULL);
+            return;
+        }
+
+        long millis = DateFormat.stringToMillis(value);
+        long code = codec.millisToCode(millis);
+        BytesUtil.writeLong(code, output, outputOffset, fixedLen);
+    }
+
+    @Override
+    public String decode(byte[] bytes, int offset, int len) {
+        if (isNull(bytes, offset, len)) {
+            return null;
+        }
+
+        long code = BytesUtil.readLong(bytes, offset, fixedLen);
+        if (code < 0)
+            throw new IllegalArgumentException();
+        
+        long millis = codec.codeToMillis(code);
+        return String.valueOf(millis);
+    }
+
+    @Override
+    public DataTypeSerializer<Object> asDataTypeSerializer() {
+        return new 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.writeInt(fixedLen);
+        out.writeObject(codec);
+    }
+
+    @Override
+    public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException {
+        this.fixedLen = in.readInt();
+        this.codec = (IMillisCodec) in.readObject();
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/kylin/blob/35f3e639/core-metadata/src/main/java/org/apache/kylin/dimension/DateDimEnc.java
----------------------------------------------------------------------
diff --git a/core-metadata/src/main/java/org/apache/kylin/dimension/DateDimEnc.java b/core-metadata/src/main/java/org/apache/kylin/dimension/DateDimEnc.java
new file mode 100644
index 0000000..79e00ce
--- /dev/null
+++ b/core-metadata/src/main/java/org/apache/kylin/dimension/DateDimEnc.java
@@ -0,0 +1,70 @@
+/*
+ * 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;
+
+/**
+ * This encoding is meant to be IDENTICAL to DateStrDictionary for 100% backward compatibility.
+ */
+public class DateDimEnc extends AbstractDateDimEnc {
+    private static final long serialVersionUID = 1L;
+
+    public static final int ID_9999_12_31 = 3652426;
+
+    public static final String ENCODING_NAME = "date";
+
+    public static class Factory extends DimensionEncodingFactory {
+        @Override
+        public String getSupportedEncodingName() {
+            return ENCODING_NAME;
+        }
+
+        @Override
+        public DimensionEncoding createDimensionEncoding(String encodingName, String[] args) {
+            return new DateDimEnc();
+        }
+    };
+
+    public DateDimEnc() {
+        super(3, new IMillisCodec() {
+            private static final long serialVersionUID = 1L;
+
+            @Override
+            public long millisToCode(long millis) {
+                return getNumOfDaysSince0000FromMillis(millis);
+            }
+
+            @Override
+            public long codeToMillis(long code) {
+                return getMillisFromNumOfDaysSince0000(code);
+            }
+        });
+    }
+
+    public static long getNumOfDaysSince0000FromMillis(long millis) {
+        // 86400000 = 1000 * 60 * 60 * 24
+        // -719530 is offset of 0000-01-01
+        return (int) (millis / 86400000 + 719530);
+    }
+
+    public static long getMillisFromNumOfDaysSince0000(long n) {
+        long millis = ((long) n - 719530) * 86400000;
+        return millis;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/kylin/blob/35f3e639/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 812a5b4..9b0aa7c 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(DateDimEnc.ENCODING_NAME, new DateDimEnc.Factory());
+            map.put(TimeDimEnc.ENCODING_NAME, new TimeDimEnc.Factory());
 
             // custom encodings
             String[] clsNames = KylinConfig.getInstanceFromEnv().getCubeDimensionCustomEncodingFactories();

http://git-wip-us.apache.org/repos/asf/kylin/blob/35f3e639/core-metadata/src/main/java/org/apache/kylin/dimension/TimeDimEnc.java
----------------------------------------------------------------------
diff --git a/core-metadata/src/main/java/org/apache/kylin/dimension/TimeDimEnc.java b/core-metadata/src/main/java/org/apache/kylin/dimension/TimeDimEnc.java
new file mode 100644
index 0000000..8c66d5d
--- /dev/null
+++ b/core-metadata/src/main/java/org/apache/kylin/dimension/TimeDimEnc.java
@@ -0,0 +1,57 @@
+/*
+ * 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;
+
+/**
+ * This encoding is meant to be IDENTICAL to TimeStrDictionary for 100% backward compatibility.
+ */
+public class TimeDimEnc extends AbstractDateDimEnc {
+    private static final long serialVersionUID = 1L;
+
+    public static final String ENCODING_NAME = "time";
+
+    public static class Factory extends DimensionEncodingFactory {
+        @Override
+        public String getSupportedEncodingName() {
+            return ENCODING_NAME;
+        }
+
+        @Override
+        public DimensionEncoding createDimensionEncoding(String encodingName, String[] args) {
+            return new TimeDimEnc();
+        }
+    };
+
+    public TimeDimEnc() {
+        super(4, new IMillisCodec() {
+            private static final long serialVersionUID = 1L;
+
+            @Override
+            public long millisToCode(long millis) {
+                return millis / 1000;
+            }
+
+            @Override
+            public long codeToMillis(long code) {
+                return code * 1000;
+            }
+        });
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/kylin/blob/35f3e639/core-metadata/src/main/java/org/apache/kylin/metadata/tuple/Tuple.java
----------------------------------------------------------------------
diff --git a/core-metadata/src/main/java/org/apache/kylin/metadata/tuple/Tuple.java b/core-metadata/src/main/java/org/apache/kylin/metadata/tuple/Tuple.java
index 0324d29..a0cbbb6 100644
--- a/core-metadata/src/main/java/org/apache/kylin/metadata/tuple/Tuple.java
+++ b/core-metadata/src/main/java/org/apache/kylin/metadata/tuple/Tuple.java
@@ -19,16 +19,15 @@
 package org.apache.kylin.metadata.tuple;
 
 import java.math.BigDecimal;
-import java.util.Date;
 import java.util.List;
 
-import net.sf.ehcache.pool.sizeof.annotations.IgnoreSizeOf;
-
 import org.apache.kylin.common.util.DateFormat;
 import org.apache.kylin.metadata.datatype.DoubleMutable;
 import org.apache.kylin.metadata.datatype.LongMutable;
 import org.apache.kylin.metadata.model.TblColRef;
 
+import net.sf.ehcache.pool.sizeof.annotations.IgnoreSizeOf;
+
 /**
  * @author xjiang
  */
@@ -153,25 +152,19 @@ public class Tuple implements ITuple {
         return sb.toString();
     }
 
-    public static long epicDaysToMillis(int days) {
-        return 1L * days * (1000 * 3600 * 24);
-    }
-
-    public static int dateToEpicDays(String strValue) {
-        Date dateValue = DateFormat.stringToDate(strValue); // NOTE: forces GMT timezone
-        long millis = dateValue.getTime();
-        return (int) (millis / (1000 * 3600 * 24));
-    }
-
     public static long getTs(ITuple row, TblColRef partitionCol) {
         //ts column type differentiate
         if (partitionCol.getDatatype().equals("date")) {
-            return Tuple.epicDaysToMillis(Integer.valueOf(row.getValue(partitionCol).toString()));
+            return epicDaysToMillis(Integer.valueOf(row.getValue(partitionCol).toString()));
         } else {
             return Long.valueOf(row.getValue(partitionCol).toString());
         }
     }
 
+    private static long epicDaysToMillis(int days) {
+        return 1L * days * (1000 * 3600 * 24);
+    }
+
     public static Object convertOptiqCellValue(String strValue, String dataTypeName) {
         if (strValue == null)
             return null;
@@ -182,7 +175,7 @@ public class Tuple implements ITuple {
         // TODO use data type enum instead of string comparison
         if ("date".equals(dataTypeName)) {
             // convert epoch time
-            return dateToEpicDays(strValue);// Optiq expects Integer instead of Long. by honma
+            return Integer.valueOf(dateToEpicDays(strValue));// Optiq expects Integer instead of Long. by honma
         } else if ("timestamp".equals(dataTypeName) || "datetime".equals(dataTypeName)) {
             return Long.valueOf(DateFormat.stringToMillis(strValue));
         } else if ("tinyint".equals(dataTypeName)) {
@@ -206,4 +199,9 @@ public class Tuple implements ITuple {
         }
     }
 
+    private static int dateToEpicDays(String strValue) {
+        long millis = DateFormat.stringToMillis(strValue);
+        return (int) (millis / (1000 * 3600 * 24));
+    }
+
 }

http://git-wip-us.apache.org/repos/asf/kylin/blob/35f3e639/core-metadata/src/test/java/org/apache/kylin/dimension/DateDimEncTest.java
----------------------------------------------------------------------
diff --git a/core-metadata/src/test/java/org/apache/kylin/dimension/DateDimEncTest.java b/core-metadata/src/test/java/org/apache/kylin/dimension/DateDimEncTest.java
new file mode 100644
index 0000000..6a583ae
--- /dev/null
+++ b/core-metadata/src/test/java/org/apache/kylin/dimension/DateDimEncTest.java
@@ -0,0 +1,101 @@
+/*
+ * 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 static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.fail;
+
+import org.apache.kylin.common.util.BytesUtil;
+import org.apache.kylin.common.util.DateFormat;
+import org.junit.Before;
+import org.junit.Test;
+
+/**
+ * Note the test must be consistent with DateStrDictionaryTest,
+ * to ensure DateDimEnc is backward compatible with DateStrDictionary.
+ */
+public class DateDimEncTest {
+
+    DateDimEnc enc;
+    byte[] buf;
+
+    @Before
+    public void setup() {
+        enc = new DateDimEnc();
+        buf = new byte[enc.getLengthOfEncoding()];
+    }
+
+    private long encode(String value) {
+        enc.encode(value, buf, 0);
+        return BytesUtil.readLong(buf, 0, buf.length);
+    }
+    
+    private String decode(long code) {
+        BytesUtil.writeLong(code, buf, 0, buf.length);
+        return enc.decode(buf, 0, buf.length);
+    }
+
+    @Test
+    public void testMinMaxId() {
+        assertEquals(0, encode("0000-01-01"));
+        assertEquals(DateDimEnc.ID_9999_12_31, encode("9999-12-31"));
+
+        try {
+            encode("10000-1-1");
+            fail("IllegalArgumentException expected");
+        } catch (IllegalArgumentException e) {
+            // good
+        }
+    }
+
+    @Test
+    public void testNull() {
+        long nullId = encode(null);
+        assertNull(decode(nullId));
+        assertEquals(0xffffff, nullId & 0xffffff);
+    }
+
+    @Test
+    public void test() {
+        checkPair("0001-01-01");
+        checkPair("1970-01-02");
+        checkPair("1975-06-24");
+        checkPair("2024-10-04");
+        checkPair("9999-12-31");
+    }
+
+    private void checkPair(String dateStr) {
+        long id = encode(dateStr);
+        String millisStr = decode(id);
+        String dateStrBack = DateFormat.formatToDateStr(Long.parseLong(millisStr));
+        assertEquals(dateStr, dateStrBack);
+    }
+
+    @Test
+    public void testIllegalArgument() {
+        try {
+            encode("abcd");
+            fail("IllegalArgumentException expected");
+        } catch (IllegalArgumentException e) {
+            // good
+        }
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/kylin/blob/35f3e639/core-metadata/src/test/java/org/apache/kylin/dimension/TimeDimEncTest.java
----------------------------------------------------------------------
diff --git a/core-metadata/src/test/java/org/apache/kylin/dimension/TimeDimEncTest.java b/core-metadata/src/test/java/org/apache/kylin/dimension/TimeDimEncTest.java
new file mode 100644
index 0000000..d5a69c3
--- /dev/null
+++ b/core-metadata/src/test/java/org/apache/kylin/dimension/TimeDimEncTest.java
@@ -0,0 +1,99 @@
+/*
+ * 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 static org.junit.Assert.*;
+
+import org.apache.kylin.common.util.BytesUtil;
+import org.apache.kylin.common.util.DateFormat;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+
+/**
+ * Note the test must be consistent with TimeStrDictionaryTest,
+ * to ensure TimeDimEnc is backward compatible with TimeStrDictionary.
+ */
+public class TimeDimEncTest {
+    TimeDimEnc enc;
+    byte[] buf;
+
+    @Before
+    public void setup() {
+        enc = new TimeDimEnc();
+        buf = new byte[enc.getLengthOfEncoding()];
+    }
+
+    private long encode(String value) {
+        enc.encode(value, buf, 0);
+        return BytesUtil.readLong(buf, 0, buf.length);
+    }
+    
+    private String decode(long code) {
+        BytesUtil.writeLong(code, buf, 0, buf.length);
+        return enc.decode(buf, 0, buf.length);
+    }
+
+    @Test
+    public void basicTest() {
+        long a = encode("1999-01-01");
+        long b = encode("1999-01-01 00:00:00");
+        long c = encode("1999-01-01 00:00:00.000");
+        long d = encode("1999-01-01 00:00:00.022");
+
+        Assert.assertEquals(a, b);
+        Assert.assertEquals(a, c);
+        Assert.assertEquals(a, d);
+    }
+
+    @Test
+    public void testEncodeDecode() {
+        encodeDecode("1999-01-12");
+        encodeDecode("2038-01-09");
+        encodeDecode("2038-01-08");
+        encodeDecode("1970-01-01");
+        encodeDecode("1970-01-02");
+
+        encodeDecode("1999-01-12 11:00:01");
+        encodeDecode("2038-01-09 01:01:02");
+        encodeDecode("2038-01-19 03:14:06");
+        encodeDecode("1970-01-01 23:22:11");
+        encodeDecode("1970-01-02 23:22:11");
+    }
+
+    @Test
+    public void testIllegal() {
+        try {
+            encode("10000-1-1");
+            fail("IllegalArgumentException expected");
+        } catch (IllegalArgumentException e) {
+            // good
+        }
+    }
+
+    public void encodeDecode(String origin) {
+        long a = encode(origin);
+        String back = decode(a);
+
+        String originChoppingMilis = DateFormat.formatToTimeWithoutMilliStr(DateFormat.stringToMillis(origin));
+        String backMillis = DateFormat.formatToTimeWithoutMilliStr(Long.parseLong(back));
+        Assert.assertEquals(originChoppingMilis, backMillis);
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/kylin/blob/35f3e639/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 908fcc5..11be511 100644
--- a/pom.xml
+++ b/pom.xml
@@ -65,7 +65,7 @@
         <!-- Commons -->
         <commons-cli.version>1.2</commons-cli.version>
         <commons-lang.version>2.6</commons-lang.version>
-        <commons-lang3.version>3.1</commons-lang3.version>
+        <commons-lang3.version>3.4</commons-lang3.version>
         <commons-collections.version>3.2.1</commons-collections.version>
         <commons-io.version>2.4</commons-io.version>
         <commons-daemon.version>1.0.15</commons-daemon.version>