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 2015/03/28 01:05:06 UTC

[29/50] incubator-kylin git commit: add serializer for Date/Time/Timestamp

add serializer for Date/Time/Timestamp


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

Branch: refs/heads/streaming-localdict
Commit: 24accccc59009dd305cd70fb96cfe3160ad8ffa1
Parents: bbbcae8
Author: Li, Yang <ya...@ebay.com>
Authored: Fri Mar 27 14:37:49 2015 +0800
Committer: Li, Yang <ya...@ebay.com>
Committed: Fri Mar 27 14:37:49 2015 +0800

----------------------------------------------------------------------
 .../apache/kylin/dict/DateStrDictionary.java    |  73 +--------
 .../kylin/invertedindex/index/TableRecord.java  |   7 +-
 .../metadata/serializer/DataTypeSerializer.java |   4 +-
 .../metadata/serializer/DateTimeSerializer.java |  39 +++++
 .../metadata/tool/HiveSourceTableLoader.java    | 155 -------------------
 .../apache/kylin/metadata/util/DateFormat.java  |  76 +++++++++
 .../metadata/util/HiveSourceTableLoader.java    | 155 +++++++++++++++++++
 .../tool/HiveSourceTableLoaderTest.java         |   2 +-
 .../apache/kylin/rest/service/CubeService.java  |   2 +-
 .../kylin/storage/hbase/HBaseKeyRange.java      |  12 +-
 .../org/apache/kylin/storage/tuple/Tuple.java   |   6 +-
 .../kylin/storage/gridtable/GridTableTest.java  |   8 +-
 12 files changed, 292 insertions(+), 247 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/24accccc/dictionary/src/main/java/org/apache/kylin/dict/DateStrDictionary.java
----------------------------------------------------------------------
diff --git a/dictionary/src/main/java/org/apache/kylin/dict/DateStrDictionary.java b/dictionary/src/main/java/org/apache/kylin/dict/DateStrDictionary.java
index 95f67ff..4523e67 100644
--- a/dictionary/src/main/java/org/apache/kylin/dict/DateStrDictionary.java
+++ b/dictionary/src/main/java/org/apache/kylin/dict/DateStrDictionary.java
@@ -18,17 +18,14 @@
 
 package org.apache.kylin.dict;
 
+import static org.apache.kylin.metadata.util.DateFormat.*;
+
 import java.io.DataInput;
 import java.io.DataOutput;
 import java.io.IOException;
 import java.io.PrintStream;
 import java.io.UnsupportedEncodingException;
-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.lang.StringUtils;
 
@@ -44,74 +41,8 @@ import org.apache.commons.lang.StringUtils;
  */
 public class DateStrDictionary extends Dictionary<String> {
 
-    static final String DEFAULT_DATE_PATTERN = "yyyy-MM-dd";
-    static final String DEFAULT_DATETIME_PATTERN_WITHOUT_MILLISECONDS = "yyyy-MM-dd HH:mm:ss";
-    static final String DEFAULT_DATETIME_PATTERN_WITH_MILLISECONDS = "yyyy-MM-dd HH:mm:ss.SSS";
-
     static final int ID_9999_12_31 = 3652426; // assume 0 based
 
-    static final private Map<String, ThreadLocal<SimpleDateFormat>> threadLocalMap = new ConcurrentHashMap<String, ThreadLocal<SimpleDateFormat>>();
-
-    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);
-        }
-        return format;
-    }
-
-    public static String dateToString(Date date) {
-        return dateToString(date, DEFAULT_DATETIME_PATTERN_WITHOUT_MILLISECONDS);
-    }
-
-    public static String dateToString(Date date, String pattern) {
-        return getDateFormat(pattern).format(date);
-    }
-
-    public static Date stringToDate(String str) {
-        return stringToDate(str, DEFAULT_DATE_PATTERN);
-    }
-
-    public static Date stringToDate(String str, String pattern) {
-        Date date = null;
-        try {
-            date = getDateFormat(pattern).parse(str);
-        } catch (ParseException e) {
-            throw new IllegalArgumentException("'" + str + "' is not a valid date of pattern '" + pattern + "'", e);
-        }
-        return date;
-    }
-
-    public static long stringToMillis(String str) {
-        if (isAllDigits(str)) {
-            return Long.parseLong(str);
-        } else if (str.length() == 10) {
-            return stringToDate(str, DEFAULT_DATE_PATTERN).getTime();
-        } else if (str.length() == 19) {
-            return stringToDate(str, DEFAULT_DATETIME_PATTERN_WITHOUT_MILLISECONDS).getTime();
-        } else if (str.length() == 23) {
-            return stringToDate(str, DEFAULT_DATETIME_PATTERN_WITH_MILLISECONDS).getTime();
-        } else {
-            throw new IllegalArgumentException("there is no valid date pattern for:" + str);
-        }
-    }
-    
-    private static boolean isAllDigits(String str) {
-        for (int i = 0, n = str.length(); i < n; i++) {
-            if (Character.isDigit(str.charAt(i)) == false)
-                return false;
-        }
-        return true;
-    }
-
-    // ============================================================================
-
     private String pattern;
     private int baseId;
     private int maxId;

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/24accccc/invertedindex/src/main/java/org/apache/kylin/invertedindex/index/TableRecord.java
----------------------------------------------------------------------
diff --git a/invertedindex/src/main/java/org/apache/kylin/invertedindex/index/TableRecord.java b/invertedindex/src/main/java/org/apache/kylin/invertedindex/index/TableRecord.java
index ce1b7e0..15869f9 100644
--- a/invertedindex/src/main/java/org/apache/kylin/invertedindex/index/TableRecord.java
+++ b/invertedindex/src/main/java/org/apache/kylin/invertedindex/index/TableRecord.java
@@ -18,14 +18,11 @@
 
 package org.apache.kylin.invertedindex.index;
 
-import com.google.common.collect.Lists;
-import org.apache.kylin.dict.DateStrDictionary;
 import org.apache.commons.lang.ObjectUtils;
 import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
 import org.apache.hadoop.io.LongWritable;
 import org.apache.kylin.dict.Dictionary;
-
-import java.util.List;
+import org.apache.kylin.metadata.util.DateFormat;
 
 /**
  * @author yangli9, honma
@@ -67,7 +64,7 @@ public class TableRecord implements Cloneable {
 
     public long getTimestamp() {
         String str = getValueString(info.getTimestampColumn());
-        return DateStrDictionary.stringToMillis(str);
+        return DateFormat.stringToMillis(str);
     }
 
     public int length(int col) {

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/24accccc/metadata/src/main/java/org/apache/kylin/metadata/serializer/DataTypeSerializer.java
----------------------------------------------------------------------
diff --git a/metadata/src/main/java/org/apache/kylin/metadata/serializer/DataTypeSerializer.java b/metadata/src/main/java/org/apache/kylin/metadata/serializer/DataTypeSerializer.java
index 094c2f1..63d4ddd 100644
--- a/metadata/src/main/java/org/apache/kylin/metadata/serializer/DataTypeSerializer.java
+++ b/metadata/src/main/java/org/apache/kylin/metadata/serializer/DataTypeSerializer.java
@@ -42,7 +42,9 @@ abstract public class DataTypeSerializer<T> implements BytesSerializer<T> {
         implementations.put("integer", LongSerializer.class);
         implementations.put("int", LongSerializer.class);
         implementations.put("smallint", LongSerializer.class);
-        implementations.put("date", StringSerializer.class);
+        implementations.put("date", DateTimeSerializer.class);
+        implementations.put("datetime", DateTimeSerializer.class);
+        implementations.put("timestamp", DateTimeSerializer.class);
     }
 
     public static DataTypeSerializer<?> create(String dataType) {

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/24accccc/metadata/src/main/java/org/apache/kylin/metadata/serializer/DateTimeSerializer.java
----------------------------------------------------------------------
diff --git a/metadata/src/main/java/org/apache/kylin/metadata/serializer/DateTimeSerializer.java b/metadata/src/main/java/org/apache/kylin/metadata/serializer/DateTimeSerializer.java
new file mode 100644
index 0000000..465c158
--- /dev/null
+++ b/metadata/src/main/java/org/apache/kylin/metadata/serializer/DateTimeSerializer.java
@@ -0,0 +1,39 @@
+package org.apache.kylin.metadata.serializer;
+
+import java.nio.ByteBuffer;
+
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.io.LongWritable;
+import org.apache.kylin.metadata.util.DateFormat;
+
+public class DateTimeSerializer extends DataTypeSerializer<LongWritable> {
+
+    // avoid mass object creation
+    LongWritable current = new LongWritable();
+
+    @Override
+    public void serialize(LongWritable value, ByteBuffer out) {
+        out.putLong(value.get());
+    }
+
+    @Override
+    public LongWritable deserialize(ByteBuffer in) {
+        current.set(in.getLong());
+        return current;
+    }
+
+    @Override
+    public int peekLength(ByteBuffer in) {
+        return 8;
+    }
+
+    @Override
+    public LongWritable valueOf(byte[] value) {
+        if (value == null)
+            current.set(0L);
+        else
+            current.set(DateFormat.stringToMillis(Bytes.toString(value)));
+        return current;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/24accccc/metadata/src/main/java/org/apache/kylin/metadata/tool/HiveSourceTableLoader.java
----------------------------------------------------------------------
diff --git a/metadata/src/main/java/org/apache/kylin/metadata/tool/HiveSourceTableLoader.java b/metadata/src/main/java/org/apache/kylin/metadata/tool/HiveSourceTableLoader.java
deleted file mode 100644
index 5297188..0000000
--- a/metadata/src/main/java/org/apache/kylin/metadata/tool/HiveSourceTableLoader.java
+++ /dev/null
@@ -1,155 +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.metadata.tool;
-
-import com.google.common.collect.Lists;
-import com.google.common.collect.Maps;
-import com.google.common.collect.Sets;
-import org.apache.hadoop.hive.metastore.api.FieldSchema;
-import org.apache.hadoop.hive.metastore.api.Table;
-import org.apache.kylin.common.KylinConfig;
-import org.apache.kylin.common.util.HadoopUtil;
-import org.apache.kylin.common.util.HiveClient;
-import org.apache.kylin.metadata.MetadataConstants;
-import org.apache.kylin.metadata.MetadataManager;
-import org.apache.kylin.metadata.model.ColumnDesc;
-import org.apache.kylin.metadata.model.TableDesc;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.io.IOException;
-import java.util.*;
-
-/**
- * Management class to sync hive table metadata with command See main method for
- * how to use the class
- *
- * @author jianliu
- */
-public class HiveSourceTableLoader {
-
-    @SuppressWarnings("unused")
-    private static final Logger logger = LoggerFactory.getLogger(HiveSourceTableLoader.class);
-
-    public static final String OUTPUT_SURFIX = "json";
-    public static final String TABLE_FOLDER_NAME = "table";
-    public static final String TABLE_EXD_FOLDER_NAME = "table_exd";
-
-    public static Set<String> reloadHiveTables(String[] hiveTables, KylinConfig config) throws IOException {
-
-        Map<String, Set<String>> db2tables = Maps.newHashMap();
-        for (String table : hiveTables) {
-            String[] parts = HadoopUtil.parseHiveTableName(table);
-            Set<String> set = db2tables.get(parts[0]);
-            if (set == null) {
-                set = Sets.newHashSet();
-                db2tables.put(parts[0], set);
-            }
-            set.add(parts[1]);
-        }
-
-        // extract from hive
-        Set<String> loadedTables = Sets.newHashSet();
-        for (String database : db2tables.keySet()) {
-            List<String> loaded = extractHiveTables(database, db2tables.get(database), config);
-            loadedTables.addAll(loaded);
-        }
-
-        return loadedTables;
-    }
-
-    private static List<String> extractHiveTables(String database, Set<String> tables, KylinConfig config) throws IOException {
-
-        List<String> loadedTables = Lists.newArrayList();
-        MetadataManager metaMgr = MetadataManager.getInstance(KylinConfig.getInstanceFromEnv());
-        for (String tableName : tables) {
-            Table table = null;
-            HiveClient hiveClient = new HiveClient();
-            List<FieldSchema> partitionFields = null;
-            List<FieldSchema> fields = null;
-            try {
-                table = hiveClient.getHiveTable(database, tableName);
-                partitionFields = table.getPartitionKeys();
-                fields = hiveClient.getHiveTableFields(database, tableName);
-            } catch (Exception e) {
-                e.printStackTrace();
-                throw new IOException(e);
-            }
-
-            if (fields != null && partitionFields != null && partitionFields.size() > 0) {
-                fields.addAll(partitionFields);
-            }
-
-            long tableSize = hiveClient.getFileSizeForTable(table);
-            long tableFileNum = hiveClient.getFileNumberForTable(table);
-            TableDesc tableDesc = metaMgr.getTableDesc(database + "." + tableName);
-            if (tableDesc == null) {
-                tableDesc = new TableDesc();
-                tableDesc.setDatabase(database.toUpperCase());
-                tableDesc.setName(tableName.toUpperCase());
-                tableDesc.setUuid(UUID.randomUUID().toString());
-                tableDesc.setLastModified(0);
-            }
-
-            int columnNumber = fields.size();
-            List<ColumnDesc> columns = new ArrayList<ColumnDesc>(columnNumber);
-            for (int i = 0; i < columnNumber; i++) {
-                FieldSchema field = fields.get(i);
-                ColumnDesc cdesc = new ColumnDesc();
-                cdesc.setName(field.getName().toUpperCase());
-                cdesc.setDatatype(field.getType());
-                cdesc.setId(String.valueOf(i + 1));
-                columns.add(cdesc);
-            }
-            tableDesc.setColumns(columns.toArray(new ColumnDesc[columnNumber]));
-
-            StringBuffer partitionColumnString = new StringBuffer();
-            for (int i = 0, n = partitionFields.size(); i < n; i++) {
-                if (i > 0)
-                    partitionColumnString.append(", ");
-                partitionColumnString.append(partitionFields.get(i).getName().toUpperCase());
-            }
-
-            Map<String, String> map = metaMgr.getTableDescExd(tableDesc.getIdentity());
-
-            if (map == null) {
-                map = Maps.newHashMap();
-            }
-            map.put(MetadataConstants.TABLE_EXD_TABLENAME, table.getTableName());
-            map.put(MetadataConstants.TABLE_EXD_LOCATION, table.getSd().getLocation());
-            map.put(MetadataConstants.TABLE_EXD_IF, table.getSd().getInputFormat());
-            map.put(MetadataConstants.TABLE_EXD_OF, table.getSd().getOutputFormat());
-            map.put(MetadataConstants.TABLE_EXD_OWNER, table.getOwner());
-            map.put(MetadataConstants.TABLE_EXD_LAT, String.valueOf(table.getLastAccessTime()));
-            map.put(MetadataConstants.TABLE_EXD_PC, partitionColumnString.toString());
-            map.put(MetadataConstants.TABLE_EXD_TFS, String.valueOf(tableSize));
-            map.put(MetadataConstants.TABLE_EXD_TNF, String.valueOf(tableFileNum));
-            map.put(MetadataConstants.TABLE_EXD_PARTITIONED, Boolean.valueOf(partitionFields != null && partitionFields.size() > 0).toString());
-
-            metaMgr.saveSourceTable(tableDesc);
-            metaMgr.saveTableExd(tableDesc.getIdentity(), map);
-            loadedTables.add(tableDesc.getIdentity());
-        }
-
-
-        return loadedTables;
-    }
-
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/24accccc/metadata/src/main/java/org/apache/kylin/metadata/util/DateFormat.java
----------------------------------------------------------------------
diff --git a/metadata/src/main/java/org/apache/kylin/metadata/util/DateFormat.java b/metadata/src/main/java/org/apache/kylin/metadata/util/DateFormat.java
new file mode 100644
index 0000000..c0967e3
--- /dev/null
+++ b/metadata/src/main/java/org/apache/kylin/metadata/util/DateFormat.java
@@ -0,0 +1,76 @@
+package org.apache.kylin.metadata.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;
+
+public class DateFormat {
+
+    public static final String DEFAULT_DATE_PATTERN = "yyyy-MM-dd";
+    public static final String DEFAULT_DATETIME_PATTERN_WITHOUT_MILLISECONDS = "yyyy-MM-dd HH:mm:ss";
+    public static final String DEFAULT_DATETIME_PATTERN_WITH_MILLISECONDS = "yyyy-MM-dd HH:mm:ss.SSS";
+
+    static final private Map<String, ThreadLocal<SimpleDateFormat>> threadLocalMap = new ConcurrentHashMap<String, ThreadLocal<SimpleDateFormat>>();
+
+    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);
+        }
+        return format;
+    }
+
+    public static String dateToString(Date date) {
+        return dateToString(date, DEFAULT_DATETIME_PATTERN_WITHOUT_MILLISECONDS);
+    }
+
+    public static String dateToString(Date date, String pattern) {
+        return getDateFormat(pattern).format(date);
+    }
+
+    public static Date stringToDate(String str) {
+        return stringToDate(str, DEFAULT_DATE_PATTERN);
+    }
+
+    public static Date stringToDate(String str, String pattern) {
+        Date date = null;
+        try {
+            date = getDateFormat(pattern).parse(str);
+        } catch (ParseException e) {
+            throw new IllegalArgumentException("'" + str + "' is not a valid date of pattern '" + pattern + "'", e);
+        }
+        return date;
+    }
+
+    public static long stringToMillis(String str) {
+        if (isAllDigits(str)) {
+            return Long.parseLong(str);
+        } else if (str.length() == 10) {
+            return stringToDate(str, DEFAULT_DATE_PATTERN).getTime();
+        } else if (str.length() == 19) {
+            return stringToDate(str, DEFAULT_DATETIME_PATTERN_WITHOUT_MILLISECONDS).getTime();
+        } else if (str.length() == 23) {
+            return stringToDate(str, DEFAULT_DATETIME_PATTERN_WITH_MILLISECONDS).getTime();
+        } else {
+            throw new IllegalArgumentException("there is no valid date pattern for:" + str);
+        }
+    }
+    
+    private static boolean isAllDigits(String str) {
+        for (int i = 0, n = str.length(); i < n; i++) {
+            if (Character.isDigit(str.charAt(i)) == false)
+                return false;
+        }
+        return true;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/24accccc/metadata/src/main/java/org/apache/kylin/metadata/util/HiveSourceTableLoader.java
----------------------------------------------------------------------
diff --git a/metadata/src/main/java/org/apache/kylin/metadata/util/HiveSourceTableLoader.java b/metadata/src/main/java/org/apache/kylin/metadata/util/HiveSourceTableLoader.java
new file mode 100644
index 0000000..fe5c2b3
--- /dev/null
+++ b/metadata/src/main/java/org/apache/kylin/metadata/util/HiveSourceTableLoader.java
@@ -0,0 +1,155 @@
+/*
+ * 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.metadata.util;
+
+import com.google.common.collect.Lists;
+import com.google.common.collect.Maps;
+import com.google.common.collect.Sets;
+import org.apache.hadoop.hive.metastore.api.FieldSchema;
+import org.apache.hadoop.hive.metastore.api.Table;
+import org.apache.kylin.common.KylinConfig;
+import org.apache.kylin.common.util.HadoopUtil;
+import org.apache.kylin.common.util.HiveClient;
+import org.apache.kylin.metadata.MetadataConstants;
+import org.apache.kylin.metadata.MetadataManager;
+import org.apache.kylin.metadata.model.ColumnDesc;
+import org.apache.kylin.metadata.model.TableDesc;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.*;
+
+/**
+ * Management class to sync hive table metadata with command See main method for
+ * how to use the class
+ *
+ * @author jianliu
+ */
+public class HiveSourceTableLoader {
+
+    @SuppressWarnings("unused")
+    private static final Logger logger = LoggerFactory.getLogger(HiveSourceTableLoader.class);
+
+    public static final String OUTPUT_SURFIX = "json";
+    public static final String TABLE_FOLDER_NAME = "table";
+    public static final String TABLE_EXD_FOLDER_NAME = "table_exd";
+
+    public static Set<String> reloadHiveTables(String[] hiveTables, KylinConfig config) throws IOException {
+
+        Map<String, Set<String>> db2tables = Maps.newHashMap();
+        for (String table : hiveTables) {
+            String[] parts = HadoopUtil.parseHiveTableName(table);
+            Set<String> set = db2tables.get(parts[0]);
+            if (set == null) {
+                set = Sets.newHashSet();
+                db2tables.put(parts[0], set);
+            }
+            set.add(parts[1]);
+        }
+
+        // extract from hive
+        Set<String> loadedTables = Sets.newHashSet();
+        for (String database : db2tables.keySet()) {
+            List<String> loaded = extractHiveTables(database, db2tables.get(database), config);
+            loadedTables.addAll(loaded);
+        }
+
+        return loadedTables;
+    }
+
+    private static List<String> extractHiveTables(String database, Set<String> tables, KylinConfig config) throws IOException {
+
+        List<String> loadedTables = Lists.newArrayList();
+        MetadataManager metaMgr = MetadataManager.getInstance(KylinConfig.getInstanceFromEnv());
+        for (String tableName : tables) {
+            Table table = null;
+            HiveClient hiveClient = new HiveClient();
+            List<FieldSchema> partitionFields = null;
+            List<FieldSchema> fields = null;
+            try {
+                table = hiveClient.getHiveTable(database, tableName);
+                partitionFields = table.getPartitionKeys();
+                fields = hiveClient.getHiveTableFields(database, tableName);
+            } catch (Exception e) {
+                e.printStackTrace();
+                throw new IOException(e);
+            }
+
+            if (fields != null && partitionFields != null && partitionFields.size() > 0) {
+                fields.addAll(partitionFields);
+            }
+
+            long tableSize = hiveClient.getFileSizeForTable(table);
+            long tableFileNum = hiveClient.getFileNumberForTable(table);
+            TableDesc tableDesc = metaMgr.getTableDesc(database + "." + tableName);
+            if (tableDesc == null) {
+                tableDesc = new TableDesc();
+                tableDesc.setDatabase(database.toUpperCase());
+                tableDesc.setName(tableName.toUpperCase());
+                tableDesc.setUuid(UUID.randomUUID().toString());
+                tableDesc.setLastModified(0);
+            }
+
+            int columnNumber = fields.size();
+            List<ColumnDesc> columns = new ArrayList<ColumnDesc>(columnNumber);
+            for (int i = 0; i < columnNumber; i++) {
+                FieldSchema field = fields.get(i);
+                ColumnDesc cdesc = new ColumnDesc();
+                cdesc.setName(field.getName().toUpperCase());
+                cdesc.setDatatype(field.getType());
+                cdesc.setId(String.valueOf(i + 1));
+                columns.add(cdesc);
+            }
+            tableDesc.setColumns(columns.toArray(new ColumnDesc[columnNumber]));
+
+            StringBuffer partitionColumnString = new StringBuffer();
+            for (int i = 0, n = partitionFields.size(); i < n; i++) {
+                if (i > 0)
+                    partitionColumnString.append(", ");
+                partitionColumnString.append(partitionFields.get(i).getName().toUpperCase());
+            }
+
+            Map<String, String> map = metaMgr.getTableDescExd(tableDesc.getIdentity());
+
+            if (map == null) {
+                map = Maps.newHashMap();
+            }
+            map.put(MetadataConstants.TABLE_EXD_TABLENAME, table.getTableName());
+            map.put(MetadataConstants.TABLE_EXD_LOCATION, table.getSd().getLocation());
+            map.put(MetadataConstants.TABLE_EXD_IF, table.getSd().getInputFormat());
+            map.put(MetadataConstants.TABLE_EXD_OF, table.getSd().getOutputFormat());
+            map.put(MetadataConstants.TABLE_EXD_OWNER, table.getOwner());
+            map.put(MetadataConstants.TABLE_EXD_LAT, String.valueOf(table.getLastAccessTime()));
+            map.put(MetadataConstants.TABLE_EXD_PC, partitionColumnString.toString());
+            map.put(MetadataConstants.TABLE_EXD_TFS, String.valueOf(tableSize));
+            map.put(MetadataConstants.TABLE_EXD_TNF, String.valueOf(tableFileNum));
+            map.put(MetadataConstants.TABLE_EXD_PARTITIONED, Boolean.valueOf(partitionFields != null && partitionFields.size() > 0).toString());
+
+            metaMgr.saveSourceTable(tableDesc);
+            metaMgr.saveTableExd(tableDesc.getIdentity(), map);
+            loadedTables.add(tableDesc.getIdentity());
+        }
+
+
+        return loadedTables;
+    }
+
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/24accccc/metadata/src/test/java/org/apache/kylin/metadata/tool/HiveSourceTableLoaderTest.java
----------------------------------------------------------------------
diff --git a/metadata/src/test/java/org/apache/kylin/metadata/tool/HiveSourceTableLoaderTest.java b/metadata/src/test/java/org/apache/kylin/metadata/tool/HiveSourceTableLoaderTest.java
index cd773ba..1f48b77 100644
--- a/metadata/src/test/java/org/apache/kylin/metadata/tool/HiveSourceTableLoaderTest.java
+++ b/metadata/src/test/java/org/apache/kylin/metadata/tool/HiveSourceTableLoaderTest.java
@@ -26,9 +26,9 @@ import java.util.Set;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
 import org.junit.Test;
-
 import org.apache.kylin.common.KylinConfig;
 import org.apache.kylin.common.util.HBaseMetadataTestCase;
+import org.apache.kylin.metadata.util.HiveSourceTableLoader;
 
 public class HiveSourceTableLoaderTest extends HBaseMetadataTestCase {
 

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/24accccc/server/src/main/java/org/apache/kylin/rest/service/CubeService.java
----------------------------------------------------------------------
diff --git a/server/src/main/java/org/apache/kylin/rest/service/CubeService.java b/server/src/main/java/org/apache/kylin/rest/service/CubeService.java
index de97a7b..d786b1e 100644
--- a/server/src/main/java/org/apache/kylin/rest/service/CubeService.java
+++ b/server/src/main/java/org/apache/kylin/rest/service/CubeService.java
@@ -44,7 +44,7 @@ import org.apache.kylin.metadata.project.ProjectManager;
 import org.apache.kylin.metadata.project.RealizationEntry;
 import org.apache.kylin.metadata.realization.RealizationStatusEnum;
 import org.apache.kylin.metadata.realization.RealizationType;
-import org.apache.kylin.metadata.tool.HiveSourceTableLoader;
+import org.apache.kylin.metadata.util.HiveSourceTableLoader;
 import org.apache.kylin.rest.constant.Constant;
 import org.apache.kylin.rest.controller.QueryController;
 import org.apache.kylin.rest.exception.InternalErrorException;

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/24accccc/storage/src/main/java/org/apache/kylin/storage/hbase/HBaseKeyRange.java
----------------------------------------------------------------------
diff --git a/storage/src/main/java/org/apache/kylin/storage/hbase/HBaseKeyRange.java b/storage/src/main/java/org/apache/kylin/storage/hbase/HBaseKeyRange.java
index 1c81eac..e766317 100644
--- a/storage/src/main/java/org/apache/kylin/storage/hbase/HBaseKeyRange.java
+++ b/storage/src/main/java/org/apache/kylin/storage/hbase/HBaseKeyRange.java
@@ -26,9 +26,6 @@ import java.util.Set;
 
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.Pair;
-
-import com.google.common.collect.Lists;
-import com.google.common.collect.Maps;
 import org.apache.kylin.common.util.BytesUtil;
 import org.apache.kylin.cube.CubeSegment;
 import org.apache.kylin.cube.cuboid.Cuboid;
@@ -37,8 +34,11 @@ import org.apache.kylin.cube.kv.FuzzyKeyEncoder;
 import org.apache.kylin.cube.kv.FuzzyMaskEncoder;
 import org.apache.kylin.cube.kv.RowConstants;
 import org.apache.kylin.cube.model.CubeDesc;
-import org.apache.kylin.dict.DateStrDictionary;
 import org.apache.kylin.metadata.model.TblColRef;
+import org.apache.kylin.metadata.util.DateFormat;
+
+import com.google.common.collect.Lists;
+import com.google.common.collect.Maps;
 
 /**
  * 
@@ -135,10 +135,10 @@ public class HBaseKeyRange implements Comparable<HBaseKeyRange> {
 
     private void initPartitionRange(ColumnValueRange dimRange) {
         if (null != dimRange.getBeginValue()) {
-            this.partitionColumnStartDate = DateStrDictionary.stringToDate(dimRange.getBeginValue()).getTime();
+            this.partitionColumnStartDate = DateFormat.stringToDate(dimRange.getBeginValue()).getTime();
         }
         if (null != dimRange.getEndValue()) {
-            this.partitionColumnEndDate = DateStrDictionary.stringToDate(dimRange.getEndValue()).getTime();
+            this.partitionColumnEndDate = DateFormat.stringToDate(dimRange.getEndValue()).getTime();
         }
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/24accccc/storage/src/main/java/org/apache/kylin/storage/tuple/Tuple.java
----------------------------------------------------------------------
diff --git a/storage/src/main/java/org/apache/kylin/storage/tuple/Tuple.java b/storage/src/main/java/org/apache/kylin/storage/tuple/Tuple.java
index dd904d4..2d18597 100644
--- a/storage/src/main/java/org/apache/kylin/storage/tuple/Tuple.java
+++ b/storage/src/main/java/org/apache/kylin/storage/tuple/Tuple.java
@@ -26,10 +26,10 @@ import org.apache.kylin.common.util.Array;
 import org.apache.kylin.cube.CubeManager;
 import org.apache.kylin.cube.CubeSegment;
 import org.apache.kylin.cube.model.CubeDesc.DeriveInfo;
-import org.apache.kylin.dict.DateStrDictionary;
 import org.apache.kylin.dict.lookup.LookupStringTable;
 import org.apache.kylin.metadata.model.TblColRef;
 import org.apache.kylin.metadata.tuple.ITuple;
+import org.apache.kylin.metadata.util.DateFormat;
 
 /**
  * @author xjiang
@@ -133,7 +133,7 @@ public class Tuple implements ITuple {
         // TODO use data type enum instead of string comparison
         if ("date".equals(dataType)) {
             // convert epoch time
-            Date dateValue = DateStrDictionary.stringToDate(strValue); // NOTE: forces GMT timezone
+            Date dateValue = DateFormat.stringToDate(strValue); // NOTE: forces GMT timezone
             long millis = dateValue.getTime();
             long days = millis / (1000 * 3600 * 24);
             return Integer.valueOf((int) days); // Optiq expects Integer instead of Long. by honma
@@ -150,7 +150,7 @@ public class Tuple implements ITuple {
         } else if ("decimal".equals(dataType)) {
             return new BigDecimal(strValue);
         } else if ("timestamp".equals(dataType)) {
-            return Long.valueOf(DateStrDictionary.stringToMillis(strValue));
+            return Long.valueOf(DateFormat.stringToMillis(strValue));
         } else {
             return strValue;
         }

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/24accccc/storage/src/test/java/org/apache/kylin/storage/gridtable/GridTableTest.java
----------------------------------------------------------------------
diff --git a/storage/src/test/java/org/apache/kylin/storage/gridtable/GridTableTest.java b/storage/src/test/java/org/apache/kylin/storage/gridtable/GridTableTest.java
index 1a69138..6561c6e 100644
--- a/storage/src/test/java/org/apache/kylin/storage/gridtable/GridTableTest.java
+++ b/storage/src/test/java/org/apache/kylin/storage/gridtable/GridTableTest.java
@@ -114,7 +114,7 @@ public class GridTableTest {
         return scanner;
     }
 
-    private GTBuilder rebuild(GridTable table) throws IOException {
+    static GTBuilder rebuild(GridTable table) throws IOException {
         GTRecord r = new GTRecord(table.getInfo());
         GTBuilder builder = table.rebuild();
 
@@ -135,7 +135,7 @@ public class GridTableTest {
         return builder;
     }
 
-    private void rebuildViaAppend(GridTable table) throws IOException {
+    static void rebuildViaAppend(GridTable table) throws IOException {
         GTRecord r = new GTRecord(table.getInfo());
         GTBuilder builder;
 
@@ -170,13 +170,13 @@ public class GridTableTest {
         System.out.println("Written Row Count: " + builder.getWrittenRowCount());
     }
 
-    public static GTInfo basicInfo() {
+    static GTInfo basicInfo() {
         Builder builder = infoBuilder();
         GTInfo info = builder.build();
         return info;
     }
 
-    public static GTInfo advancedInfo() {
+    static GTInfo advancedInfo() {
         Builder builder = infoBuilder();
         builder.enableColumnBlock(new BitSet[] { setOf(0, 1, 2), setOf(3, 4) });
         builder.enableRowBlock(4);