You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@kylin.apache.org by lu...@apache.org on 2015/01/07 15:46:54 UTC

[33/51] [partial] incubator-kylin git commit: migrate repo from github.com to apache git

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/dictionary/src/main/java/com/kylinolap/dict/lookup/SnapshotManager.java
----------------------------------------------------------------------
diff --git a/dictionary/src/main/java/com/kylinolap/dict/lookup/SnapshotManager.java b/dictionary/src/main/java/com/kylinolap/dict/lookup/SnapshotManager.java
new file mode 100644
index 0000000..a47e614
--- /dev/null
+++ b/dictionary/src/main/java/com/kylinolap/dict/lookup/SnapshotManager.java
@@ -0,0 +1,166 @@
+/*
+ * Copyright 2013-2014 eBay Software Foundation
+ *
+ * Licensed 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 com.kylinolap.dict.lookup;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.concurrent.ConcurrentHashMap;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.kylinolap.common.KylinConfig;
+import com.kylinolap.common.persistence.ResourceStore;
+import com.kylinolap.metadata.MetadataManager;
+import com.kylinolap.metadata.model.schema.TableDesc;
+
+/**
+ * @author yangli9
+ */
+public class SnapshotManager {
+
+    private static final Logger logger = LoggerFactory.getLogger(SnapshotManager.class);
+
+    // static cached instances
+    private static final ConcurrentHashMap<KylinConfig, SnapshotManager> SERVICE_CACHE = new ConcurrentHashMap<KylinConfig, SnapshotManager>();
+
+    public static SnapshotManager getInstance(KylinConfig config) {
+        SnapshotManager r = SERVICE_CACHE.get(config);
+        if (r == null) {
+            r = new SnapshotManager(config);
+            SERVICE_CACHE.put(config, r);
+            if (SERVICE_CACHE.size() > 1) {
+                logger.warn("More than one singleton exist");
+            }
+        }
+        return r;
+    }
+
+    // ============================================================================
+
+    private KylinConfig config;
+    private ConcurrentHashMap<String, SnapshotTable> snapshotCache; // resource
+    // path ==>
+    // SnapshotTable
+
+    private SnapshotManager(KylinConfig config) {
+        this.config = config;
+        snapshotCache = new ConcurrentHashMap<String, SnapshotTable>();
+    }
+
+    public void wipeoutCache() {
+        snapshotCache.clear();
+    }
+
+    public SnapshotTable getSnapshotTable(String resourcePath) throws IOException {
+        SnapshotTable r = snapshotCache.get(resourcePath);
+        if (r == null) {
+            r = load(resourcePath, true);
+            snapshotCache.put(resourcePath, r);
+        }
+        return r;
+    }
+
+    public void removeSnapshot(String resourcePath) throws IOException {
+        ResourceStore store = MetadataManager.getInstance(this.config).getStore();
+        store.deleteResource(resourcePath);
+        snapshotCache.remove(resourcePath);
+    }
+
+    public SnapshotTable buildSnapshot(ReadableTable table, TableDesc tableDesc) throws IOException {
+        SnapshotTable snapshot = new SnapshotTable(table);
+        snapshot.updateRandomUuid();
+
+        String dup = checkDupByInfo(snapshot);
+        if (dup != null) {
+            logger.info("Identical input " + table.getSignature() + ", reuse existing snapshot at " + dup);
+            return getSnapshotTable(dup);
+        }
+
+        snapshot.takeSnapshot(table, tableDesc);
+
+        return trySaveNewSnapshot(snapshot);
+    }
+
+    public SnapshotTable trySaveNewSnapshot(SnapshotTable snapshotTable) throws IOException {
+
+        String dupTable = checkDupByContent(snapshotTable);
+        if (dupTable != null) {
+            logger.info("Identical snapshot content " + snapshotTable + ", reuse existing snapshot at " + dupTable);
+            return getSnapshotTable(dupTable);
+        }
+
+        save(snapshotTable);
+        snapshotCache.put(snapshotTable.getResourcePath(), snapshotTable);
+
+        return snapshotTable;
+    }
+
+    private String checkDupByInfo(SnapshotTable snapshot) throws IOException {
+        ResourceStore store = MetadataManager.getInstance(this.config).getStore();
+        String resourceDir = snapshot.getResourceDir();
+        ArrayList<String> existings = store.listResources(resourceDir);
+        if (existings == null)
+            return null;
+
+        TableSignature sig = snapshot.getSignature();
+        for (String existing : existings) {
+            SnapshotTable existingTable = load(existing, false); // skip cache,
+            // direct
+            // load from
+            // store
+            if (sig.equals(existingTable.getSignature()))
+                return existing;
+        }
+
+        return null;
+    }
+
+    private String checkDupByContent(SnapshotTable snapshot) throws IOException {
+        ResourceStore store = MetadataManager.getInstance(this.config).getStore();
+        String resourceDir = snapshot.getResourceDir();
+        ArrayList<String> existings = store.listResources(resourceDir);
+        if (existings == null)
+            return null;
+
+        for (String existing : existings) {
+            SnapshotTable existingTable = load(existing, true); // skip cache, direct load from store
+            if (existingTable != null && existingTable.equals(snapshot))
+                return existing;
+        }
+
+        return null;
+    }
+
+    private void save(SnapshotTable snapshot) throws IOException {
+        ResourceStore store = MetadataManager.getInstance(this.config).getStore();
+        String path = snapshot.getResourcePath();
+        store.putResource(path, snapshot, SnapshotTableSerializer.FULL_SERIALIZER);
+    }
+
+    private SnapshotTable load(String resourcePath, boolean loadData) throws IOException {
+        ResourceStore store = MetadataManager.getInstance(this.config).getStore();
+
+        SnapshotTable table = store.getResource(resourcePath, SnapshotTable.class, loadData ? SnapshotTableSerializer.FULL_SERIALIZER : SnapshotTableSerializer.INFO_SERIALIZER);
+
+        if (loadData)
+            logger.debug("Loaded snapshot at " + resourcePath);
+
+        return table;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/dictionary/src/main/java/com/kylinolap/dict/lookup/SnapshotTable.java
----------------------------------------------------------------------
diff --git a/dictionary/src/main/java/com/kylinolap/dict/lookup/SnapshotTable.java b/dictionary/src/main/java/com/kylinolap/dict/lookup/SnapshotTable.java
new file mode 100644
index 0000000..6415045
--- /dev/null
+++ b/dictionary/src/main/java/com/kylinolap/dict/lookup/SnapshotTable.java
@@ -0,0 +1,180 @@
+/*
+ * Copyright 2013-2014 eBay Software Foundation
+ *
+ * Licensed 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 com.kylinolap.dict.lookup;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+
+import org.apache.commons.lang.ArrayUtils;
+import org.apache.hadoop.fs.Path;
+
+import com.fasterxml.jackson.annotation.JsonAutoDetect;
+import com.fasterxml.jackson.annotation.JsonAutoDetect.Visibility;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.kylinolap.common.persistence.ResourceStore;
+import com.kylinolap.common.persistence.RootPersistentEntity;
+import com.kylinolap.metadata.model.schema.TableDesc;
+
+/**
+ * @author yangli9
+ */
+@JsonAutoDetect(fieldVisibility = Visibility.NONE, getterVisibility = Visibility.NONE, isGetterVisibility = Visibility.NONE, setterVisibility = Visibility.NONE)
+public class SnapshotTable extends RootPersistentEntity implements ReadableTable {
+
+    @JsonProperty("signature")
+    private TableSignature signature;
+    @JsonProperty("column_delimeter")
+    private String columnDelimeter;
+
+    private ArrayList<String[]> rows;
+
+    // default constructor for JSON serialization
+    public SnapshotTable() {
+    }
+
+    SnapshotTable(ReadableTable table) throws IOException {
+        this.signature = table.getSignature();
+        this.columnDelimeter = table.getColumnDelimeter();
+    }
+
+    public void takeSnapshot(ReadableTable table, TableDesc tableDesc) throws IOException {
+        this.signature = table.getSignature();
+        this.columnDelimeter = table.getColumnDelimeter();
+
+        int maxIndex = tableDesc.getMaxColumnIndex();
+
+        TableReader reader = table.getReader();
+        ArrayList<String[]> allRows = new ArrayList<String[]>();
+        while (reader.next()) {
+            String[] row = reader.getRow();
+            if (row.length <= maxIndex) {
+                throw new IllegalStateException("Bad hive table row, " + tableDesc + " expect " + (maxIndex + 1) + " columns, but got " + Arrays.toString(row));
+            }
+            allRows.add(row);
+        }
+        this.rows = allRows;
+    }
+
+    public String getResourcePath() {
+        return ResourceStore.SNAPSHOT_RESOURCE_ROOT + "/" + new Path(signature.getPath()).getName() + "/" + uuid + ".snapshot";
+    }
+
+    public String getResourceDir() {
+        return ResourceStore.SNAPSHOT_RESOURCE_ROOT + "/" + new Path(signature.getPath()).getName();
+    }
+
+    @Override
+    public TableReader getReader() throws IOException {
+        return new TableReader() {
+
+            int i = -1;
+
+            @Override
+            public boolean next() throws IOException {
+                i++;
+                return i < rows.size();
+            }
+
+            @Override
+            public String[] getRow() {
+                return rows.get(i);
+            }
+
+            @Override
+            public void close() throws IOException {
+            }
+
+            @Override
+            public void setExpectedColumnNumber(int expectedColumnNumber) {
+                // noop
+            }
+        };
+    }
+
+    @Override
+    public TableSignature getSignature() throws IOException {
+        return signature;
+    }
+
+    @Override
+    public String getColumnDelimeter() throws IOException {
+        return columnDelimeter;
+    }
+
+    /**
+     * a naive implementation
+     *
+     * @return
+     */
+    @Override
+    public int hashCode() {
+        int[] parts = new int[this.rows.size()];
+        for (int i = 0; i < parts.length; ++i)
+            parts[i] = Arrays.hashCode(this.rows.get(i));
+        return Arrays.hashCode(parts);
+    }
+
+
+    @Override
+    public boolean equals(Object o) {
+        if ((o instanceof SnapshotTable) == false)
+            return false;
+        SnapshotTable that = (SnapshotTable) o;
+
+        //compare row by row
+        if (this.rows.size() != that.rows.size())
+            return false;
+        for (int i = 0; i < this.rows.size(); ++i) {
+            if (!ArrayUtils.isEquals(this.rows.get(i), that.rows.get(i)))
+                return false;
+        }
+        return true;
+    }
+
+    void writeData(DataOutput out) throws IOException {
+        out.writeInt(rows.size());
+        if (rows.size() > 0) {
+            int n = rows.get(0).length;
+            out.writeInt(n);
+            for (int i = 0; i < rows.size(); i++) {
+                String[] row = rows.get(i);
+                for (int j = 0; j < n; j++) {
+                    out.writeUTF(row[j]);
+                }
+            }
+        }
+    }
+
+    void readData(DataInput in) throws IOException {
+        int rowNum = in.readInt();
+        rows = new ArrayList<String[]>(rowNum);
+        if (rowNum > 0) {
+            int n = in.readInt();
+            for (int i = 0; i < rowNum; i++) {
+                String[] row = new String[n];
+                rows.add(row);
+                for (int j = 0; j < n; j++) {
+                    row[j] = in.readUTF();
+                }
+            }
+        }
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/dictionary/src/main/java/com/kylinolap/dict/lookup/SnapshotTableSerializer.java
----------------------------------------------------------------------
diff --git a/dictionary/src/main/java/com/kylinolap/dict/lookup/SnapshotTableSerializer.java b/dictionary/src/main/java/com/kylinolap/dict/lookup/SnapshotTableSerializer.java
new file mode 100644
index 0000000..a980cd4
--- /dev/null
+++ b/dictionary/src/main/java/com/kylinolap/dict/lookup/SnapshotTableSerializer.java
@@ -0,0 +1,61 @@
+package com.kylinolap.dict.lookup;
+
+import java.io.DataInputStream;
+import java.io.DataOutputStream;
+import java.io.IOException;
+
+import com.kylinolap.common.persistence.Serializer;
+import com.kylinolap.common.util.JsonUtil;
+
+/*
+ * Copyright 2013-2014 eBay Software Foundation
+ *
+ * Licensed 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.
+ */
+
+/**
+ * @author yangli9
+ * 
+ */
+public class SnapshotTableSerializer implements Serializer<SnapshotTable> {
+
+    public static final SnapshotTableSerializer FULL_SERIALIZER = new SnapshotTableSerializer(false);
+    public static final SnapshotTableSerializer INFO_SERIALIZER = new SnapshotTableSerializer(true);
+
+    private boolean infoOnly;
+
+    SnapshotTableSerializer(boolean infoOnly) {
+        this.infoOnly = infoOnly;
+    }
+
+    @Override
+    public void serialize(SnapshotTable obj, DataOutputStream out) throws IOException {
+        String json = JsonUtil.writeValueAsIndentString(obj);
+        out.writeUTF(json);
+
+        if (infoOnly == false)
+            obj.writeData(out);
+    }
+
+    @Override
+    public SnapshotTable deserialize(DataInputStream in) throws IOException {
+        String json = in.readUTF();
+        SnapshotTable obj = JsonUtil.readValue(json, SnapshotTable.class);
+
+        if (infoOnly == false)
+            obj.readData(in);
+
+        return obj;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/dictionary/src/main/java/com/kylinolap/dict/lookup/TableReader.java
----------------------------------------------------------------------
diff --git a/dictionary/src/main/java/com/kylinolap/dict/lookup/TableReader.java b/dictionary/src/main/java/com/kylinolap/dict/lookup/TableReader.java
new file mode 100644
index 0000000..241132e
--- /dev/null
+++ b/dictionary/src/main/java/com/kylinolap/dict/lookup/TableReader.java
@@ -0,0 +1,34 @@
+/*
+ * Copyright 2013-2014 eBay Software Foundation
+ *
+ * Licensed 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 com.kylinolap.dict.lookup;
+
+import java.io.Closeable;
+import java.io.IOException;
+
+/**
+ * Tables are typically CSV or SEQ file.
+ * 
+ * @author yangli9
+ */
+public interface TableReader extends Closeable {
+
+    public boolean next() throws IOException;
+
+    public String[] getRow();
+
+    public void setExpectedColumnNumber(int expectedColumnNumber);
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/dictionary/src/main/java/com/kylinolap/dict/lookup/TableSignature.java
----------------------------------------------------------------------
diff --git a/dictionary/src/main/java/com/kylinolap/dict/lookup/TableSignature.java b/dictionary/src/main/java/com/kylinolap/dict/lookup/TableSignature.java
new file mode 100644
index 0000000..7425cd8
--- /dev/null
+++ b/dictionary/src/main/java/com/kylinolap/dict/lookup/TableSignature.java
@@ -0,0 +1,109 @@
+/*
+ * Copyright 2013-2014 eBay Software Foundation
+ *
+ * Licensed 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 com.kylinolap.dict.lookup;
+
+import com.fasterxml.jackson.annotation.JsonAutoDetect;
+import com.fasterxml.jackson.annotation.JsonAutoDetect.Visibility;
+import com.fasterxml.jackson.annotation.JsonProperty;
+
+/**
+ * @author yangli9
+ */
+@JsonAutoDetect(fieldVisibility = Visibility.NONE, getterVisibility = Visibility.NONE, isGetterVisibility = Visibility.NONE, setterVisibility = Visibility.NONE)
+public class TableSignature {
+
+    @JsonProperty("path")
+    private String path;
+    @JsonProperty("size")
+    private long size;
+    @JsonProperty("last_modified_time")
+    private long lastModifiedTime;
+
+    // for JSON serialization
+    public TableSignature() {
+    }
+
+    public TableSignature(String path, long size, long lastModifiedTime) {
+        super();
+        this.path = path;
+        this.size = size;
+        this.lastModifiedTime = lastModifiedTime;
+    }
+
+    public void setPath(String path) {
+        this.path = path;
+    }
+
+    public void setSize(long size) {
+        this.size = size;
+    }
+
+    public void setLastModifiedTime(long lastModifiedTime) {
+        this.lastModifiedTime = lastModifiedTime;
+    }
+
+    public String getPath() {
+        return path;
+    }
+
+    public long getSize() {
+        return size;
+    }
+
+    public long getLastModifiedTime() {
+        return lastModifiedTime;
+    }
+
+    // ============================================================================
+
+    @Override
+    public int hashCode() {
+        final int prime = 31;
+        int result = 1;
+        result = prime * result + (int) (lastModifiedTime ^ (lastModifiedTime >>> 32));
+        result = prime * result + ((path == null) ? 0 : path.hashCode());
+        result = prime * result + (int) (size ^ (size >>> 32));
+        return result;
+    }
+
+    @Override
+    public boolean equals(Object obj) {
+        if (this == obj)
+            return true;
+        if (obj == null)
+            return false;
+        if (getClass() != obj.getClass())
+            return false;
+        TableSignature other = (TableSignature) obj;
+        if (lastModifiedTime != other.lastModifiedTime)
+            return false;
+        if (path == null) {
+            if (other.path != null)
+                return false;
+        } else if (!path.equals(other.path))
+            return false;
+        if (size != other.size)
+            return false;
+        return true;
+    }
+
+    @Override
+    public String toString() {
+        return "FileSignature [path=" + path + ", size=" + size + ", lastModifiedTime=" + lastModifiedTime + "]";
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/dictionary/src/main/resources/com/kylinolap/dict/common_indicators.txt
----------------------------------------------------------------------
diff --git a/dictionary/src/main/resources/com/kylinolap/dict/common_indicators.txt b/dictionary/src/main/resources/com/kylinolap/dict/common_indicators.txt
new file mode 100644
index 0000000..53cefae
--- /dev/null
+++ b/dictionary/src/main/resources/com/kylinolap/dict/common_indicators.txt
@@ -0,0 +1,13 @@
+Y
+y
+N
+n
+T
+t
+F
+f
+0
+1
+NULL
+Null
+null

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/dictionary/src/main/resources/com/kylinolap/dict/date(yyyy-mm-dd).txt
----------------------------------------------------------------------
diff --git a/dictionary/src/main/resources/com/kylinolap/dict/date(yyyy-mm-dd).txt b/dictionary/src/main/resources/com/kylinolap/dict/date(yyyy-mm-dd).txt
new file mode 100644
index 0000000..fca7eb3
--- /dev/null
+++ b/dictionary/src/main/resources/com/kylinolap/dict/date(yyyy-mm-dd).txt
@@ -0,0 +1,2 @@
+1970-01-01
+9999-12-31
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/dictionary/src/test/java/com/kylinolap/dict/DateStrDictionaryTest.java
----------------------------------------------------------------------
diff --git a/dictionary/src/test/java/com/kylinolap/dict/DateStrDictionaryTest.java b/dictionary/src/test/java/com/kylinolap/dict/DateStrDictionaryTest.java
new file mode 100644
index 0000000..c577138
--- /dev/null
+++ b/dictionary/src/test/java/com/kylinolap/dict/DateStrDictionaryTest.java
@@ -0,0 +1,72 @@
+/*
+ * Copyright 2013-2014 eBay Software Foundation
+ *
+ * Licensed 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 com.kylinolap.dict;
+
+import static org.junit.Assert.*;
+
+import org.junit.Before;
+import org.junit.Test;
+
+public class DateStrDictionaryTest {
+
+    DateStrDictionary dict;
+
+    @Before
+    public void setup() {
+        dict = new DateStrDictionary();
+    }
+
+    @Test
+    public void testNull() {
+        int nullId = dict.getIdFromValue(null);
+        assertNull(dict.getValueFromId(nullId));
+        int nullId2 = dict.getIdFromValueBytes(null, 0, 0);
+        assertEquals(dict.getValueBytesFromId(nullId2, null, 0), 0);
+        assertEquals(nullId, nullId2);
+    }
+
+    @Test
+    public void test() {
+        checkPair("0001-01-01");
+        checkPair("1970-01-02");
+        checkPair("1975-06-24");
+        checkPair("2024-10-04");
+        checkPair("9999-12-31");
+    }
+
+    @Test
+    public void testIllegalArgument() {
+        try {
+            dict.getIdFromValue("abcd");
+            fail("IllegalArgumentException expected");
+        } catch (IllegalArgumentException e) {
+            // good
+        }
+
+        try {
+            dict.getValueFromId(-2);
+            fail("IllegalArgumentException expected");
+        } catch (IllegalArgumentException e) {
+            // good
+        }
+    }
+
+    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/incubator-kylin/blob/4b631f92/dictionary/src/test/java/com/kylinolap/dict/DictionaryManagerTest.java
----------------------------------------------------------------------
diff --git a/dictionary/src/test/java/com/kylinolap/dict/DictionaryManagerTest.java b/dictionary/src/test/java/com/kylinolap/dict/DictionaryManagerTest.java
new file mode 100644
index 0000000..8452811
--- /dev/null
+++ b/dictionary/src/test/java/com/kylinolap/dict/DictionaryManagerTest.java
@@ -0,0 +1,80 @@
+/*
+ * Copyright 2013-2014 eBay Software Foundation
+ *
+ * Licensed 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 com.kylinolap.dict;
+
+import static org.junit.Assert.*;
+
+import java.util.HashSet;
+
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Ignore;
+import org.junit.Test;
+
+import com.kylinolap.common.util.JsonUtil;
+import com.kylinolap.common.util.LocalFileMetadataTestCase;
+import com.kylinolap.metadata.MetadataManager;
+import com.kylinolap.metadata.model.cube.CubeDesc;
+import com.kylinolap.metadata.model.cube.TblColRef;
+
+public class DictionaryManagerTest extends LocalFileMetadataTestCase {
+
+    DictionaryManager dictMgr;
+
+    @Before
+    public void setup() throws Exception {
+        createTestMetadata();
+        dictMgr = DictionaryManager.getInstance(this.getTestConfig());
+    }
+
+    @After
+    public void after() throws Exception {
+        cleanupTestMetadata();
+    }
+
+    @Test
+    @Ignore
+    public void basic() throws Exception {
+        CubeDesc cubeDesc = MetadataManager.getInstance(this.getTestConfig()).getCubeDesc("test_kylin_cube_without_slr_desc");
+        TblColRef col = cubeDesc.findColumnRef("TEST_SITES", "SITE_NAME");
+
+        DictionaryInfo info1 = dictMgr.buildDictionary(cubeDesc, col, null);
+        System.out.println(JsonUtil.writeValueAsIndentString(info1));
+
+        DictionaryInfo info2 = dictMgr.buildDictionary(cubeDesc, col, null);
+        System.out.println(JsonUtil.writeValueAsIndentString(info2));
+
+        assertTrue(info1.getUuid() == info2.getUuid());
+
+        assertTrue(info1 == dictMgr.getDictionaryInfo(info1.getResourcePath()));
+        assertTrue(info2 == dictMgr.getDictionaryInfo(info2.getResourcePath()));
+
+        assertTrue(info1.getDictionaryObject() == info2.getDictionaryObject());
+
+        touchDictValues(info1);
+    }
+
+    @SuppressWarnings("unchecked")
+    private void touchDictValues(DictionaryInfo info1) {
+        Dictionary<String> dict = (Dictionary<String>) info1.getDictionaryObject();
+
+        HashSet<String> set = new HashSet<String>();
+        for (int i = 0, n = info1.getCardinality(); i < n; i++) {
+            set.add(dict.getValueFromId(i));
+        }
+        assertEquals(info1.getCardinality(), set.size());
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/dictionary/src/test/java/com/kylinolap/dict/LookupTableTest.java
----------------------------------------------------------------------
diff --git a/dictionary/src/test/java/com/kylinolap/dict/LookupTableTest.java b/dictionary/src/test/java/com/kylinolap/dict/LookupTableTest.java
new file mode 100644
index 0000000..39cbbf0
--- /dev/null
+++ b/dictionary/src/test/java/com/kylinolap/dict/LookupTableTest.java
@@ -0,0 +1,70 @@
+/*
+ * Copyright 2013-2014 eBay Software Foundation
+ *
+ * Licensed 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 com.kylinolap.dict;
+
+import org.apache.hadoop.hbase.util.Bytes;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+import com.kylinolap.common.util.Array;
+import com.kylinolap.common.util.ByteArray;
+import com.kylinolap.common.util.LocalFileMetadataTestCase;
+import com.kylinolap.dict.lookup.FileTable;
+import com.kylinolap.dict.lookup.LookupBytesTable;
+import com.kylinolap.metadata.MetadataManager;
+import com.kylinolap.metadata.model.schema.TableDesc;
+
+/**
+ * @author yangli9
+ */
+public class LookupTableTest extends LocalFileMetadataTestCase {
+
+    @Before
+    public void setup() throws Exception {
+        createTestMetadata();
+    }
+
+    @After
+    public void after() throws Exception {
+        cleanupTestMetadata();
+    }
+
+    @Test
+    public void testBasic() throws Exception {
+        TableDesc siteTable = MetadataManager.getInstance(this.getTestConfig()).getTableDesc("TEST_SITES");
+        TableDesc categoryTable = MetadataManager.getInstance(this.getTestConfig()).getTableDesc("test_category_groupings");
+        LookupBytesTable lookup;
+
+        System.out.println("============================================================================");
+
+        lookup = new LookupBytesTable(siteTable, new String[] { "SITE_ID" }, new FileTable(LOCALMETA_TEST_DATA + "/data/TEST_SITES.csv", 10));
+        lookup.dump();
+
+        System.out.println("============================================================================");
+
+        lookup = new LookupBytesTable(categoryTable, new String[] { "leaf_categ_id", "site_id" }, new FileTable(LOCALMETA_TEST_DATA + "/data/TEST_CATEGORY_GROUPINGS.csv", 36));
+        lookup.dump();
+
+        System.out.println("============================================================================");
+
+        ByteArray k1 = new ByteArray(Bytes.toBytes("533"));
+        ByteArray k2 = new ByteArray(Bytes.toBytes("0"));
+        Array<ByteArray> key = new Array<ByteArray>(new ByteArray[] { k1, k2 });
+        System.out.println(lookup.getRow(key));
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/dictionary/src/test/java/com/kylinolap/dict/NumberDictionaryTest.java
----------------------------------------------------------------------
diff --git a/dictionary/src/test/java/com/kylinolap/dict/NumberDictionaryTest.java b/dictionary/src/test/java/com/kylinolap/dict/NumberDictionaryTest.java
new file mode 100644
index 0000000..8343877
--- /dev/null
+++ b/dictionary/src/test/java/com/kylinolap/dict/NumberDictionaryTest.java
@@ -0,0 +1,154 @@
+/*
+ * Copyright 2013-2014 eBay Software Foundation
+ *
+ * Licensed 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 com.kylinolap.dict;
+
+import static org.junit.Assert.*;
+
+import java.math.BigDecimal;
+import java.util.Collections;
+import java.util.List;
+import java.util.Random;
+import java.util.Set;
+
+import org.apache.hadoop.hbase.util.Bytes;
+import org.junit.Test;
+
+import com.google.common.collect.Lists;
+import com.google.common.collect.Sets;
+
+/**
+ * @author yangli9
+ * 
+ */
+public class NumberDictionaryTest {
+
+    NumberDictionary.NumberBytesCodec codec = new NumberDictionary.NumberBytesCodec();
+    Random rand = new Random();
+
+    @Test
+    public void testNumberEncode() {
+        checkCodec("12345", "00000000000012345");
+        checkCodec("12345.123", "00000000000012345.123");
+        checkCodec("-12345", "-9999999999987654;");
+        checkCodec("-12345.123", "-9999999999987654.876;");
+        checkCodec("0", "00000000000000000");
+        checkCodec("0.0", "00000000000000000.0");
+    }
+
+    private void checkCodec(String number, String code) {
+        assertEquals(code, encodeNumber(number));
+        assertEquals(number, decodeNumber(code));
+    }
+
+    private String decodeNumber(String code) {
+        byte[] buf = Bytes.toBytes(code);
+        System.arraycopy(buf, 0, codec.buf, 0, buf.length);
+        codec.bufOffset = 0;
+        codec.bufLen = buf.length;
+        int len = codec.decodeNumber(buf, 0);
+        return Bytes.toString(buf, 0, len);
+    }
+
+    private String encodeNumber(String number) {
+        byte[] num1 = Bytes.toBytes(number);
+        codec.encodeNumber(num1, 0, num1.length);
+        return Bytes.toString(codec.buf, codec.bufOffset, codec.bufLen);
+    }
+
+    @Test
+    public void testDictionary() {
+        int n = 100;
+
+        Set<BigDecimal> set = Sets.newHashSet();
+        NumberDictionaryBuilder<String> builder = new NumberDictionaryBuilder<String>(new StringBytesConverter());
+        for (int i = 0; i < n; i++) {
+            String num = randNumber();
+            builder.addValue(num);
+            set.add(new BigDecimal(num));
+        }
+
+        List<BigDecimal> sorted = Lists.newArrayList();
+        sorted.addAll(set);
+        Collections.sort(sorted);
+
+        // test exact match
+        NumberDictionary<String> dict = builder.build(0);
+        for (int i = 0; i < sorted.size(); i++) {
+            String dictNum = dict.getValueFromId(i);
+            System.out.println(sorted.get(i) + "\t" + dictNum);
+            assertEquals(sorted.get(i), new BigDecimal(dictNum));
+        }
+
+        // test rounding
+        for (int i = 0; i < n; i++) {
+            String randStr = randNumber();
+            BigDecimal rand = new BigDecimal(randStr);
+            int binarySearch = Collections.binarySearch(sorted, rand);
+            if (binarySearch >= 0)
+                continue;
+            int insertion = -(binarySearch + 1);
+            int expectedLowerId = insertion - 1;
+            int expectedHigherId = insertion;
+            // System.out.println("-- " + randStr + ", " + expectedLowerId +
+            // ", " + expectedHigherId);
+
+            if (expectedLowerId < 0) {
+                try {
+                    dict.getIdFromValue(randStr, -1);
+                    fail();
+                } catch (IllegalArgumentException ex) {
+                    // expect
+                }
+            } else {
+                assertEquals(expectedLowerId, dict.getIdFromValue(randStr, -1));
+            }
+
+            if (expectedHigherId >= sorted.size()) {
+                try {
+                    dict.getIdFromValue(randStr, 1);
+                    fail();
+                } catch (IllegalArgumentException ex) {
+                    // expect
+                }
+            } else {
+                assertEquals(expectedHigherId, dict.getIdFromValue(randStr, 1));
+            }
+        }
+    }
+
+    private String randNumber() {
+        int digits1 = rand.nextInt(10);
+        int digits2 = rand.nextInt(3);
+        int sign = rand.nextInt(2);
+        if (digits1 == 0 && digits2 == 0) {
+            return randNumber();
+        }
+        StringBuilder buf = new StringBuilder();
+        if (sign == 1)
+            buf.append("-");
+        for (int i = 0; i < digits1; i++)
+            buf.append("" + rand.nextInt(10));
+        if (digits2 > 0) {
+            buf.append(".");
+            for (int i = 0; i < digits2; i++)
+                buf.append("" + rand.nextInt(9) + 1); // BigDecimal thinks 4.5
+                                                      // != 4.50, my god!
+        }
+        return buf.toString();
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/dictionary/src/test/java/com/kylinolap/dict/SnapshotManagerTest.java
----------------------------------------------------------------------
diff --git a/dictionary/src/test/java/com/kylinolap/dict/SnapshotManagerTest.java b/dictionary/src/test/java/com/kylinolap/dict/SnapshotManagerTest.java
new file mode 100644
index 0000000..a9663c1
--- /dev/null
+++ b/dictionary/src/test/java/com/kylinolap/dict/SnapshotManagerTest.java
@@ -0,0 +1,81 @@
+/*
+ * Copyright 2013-2014 eBay Software Foundation
+ *
+ * Licensed 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 com.kylinolap.dict;
+
+import static org.junit.Assert.*;
+
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+import com.kylinolap.common.util.HBaseMetadataTestCase;
+import com.kylinolap.dict.lookup.HiveTable;
+import com.kylinolap.dict.lookup.SnapshotManager;
+import com.kylinolap.dict.lookup.SnapshotTable;
+import com.kylinolap.dict.lookup.TableReader;
+import com.kylinolap.metadata.MetadataManager;
+import com.kylinolap.metadata.model.schema.TableDesc;
+
+/**
+ * @author yangli9
+ * 
+ */
+public class SnapshotManagerTest extends HBaseMetadataTestCase {
+
+    SnapshotManager snapshotMgr;
+
+    @Before
+    public void setup() throws Exception {
+        createTestMetadata();
+
+        snapshotMgr = SnapshotManager.getInstance(this.getTestConfig());
+    }
+
+    @After
+    public void after() throws Exception {
+        cleanupTestMetadata();
+    }
+
+    @Test
+    public void basicTest() throws Exception {
+        String tableName = "TEST_SITES";
+        HiveTable hiveTable = new HiveTable(MetadataManager.getInstance(this.getTestConfig()), tableName);
+        TableDesc tableDesc = MetadataManager.getInstance(this.getTestConfig()).getTableDesc(tableName);
+        String snapshotPath = snapshotMgr.buildSnapshot(hiveTable, tableDesc).getResourcePath();
+
+        snapshotMgr.wipeoutCache();
+
+        SnapshotTable snapshot = snapshotMgr.getSnapshotTable(snapshotPath);
+
+        // compare hive & snapshot
+        TableReader hiveReader = hiveTable.getReader();
+        TableReader snapshotReader = snapshot.getReader();
+
+        while (true) {
+            boolean hiveNext = hiveReader.next();
+            boolean snapshotNext = snapshotReader.next();
+            assertEquals(hiveNext, snapshotNext);
+
+            if (hiveNext == false)
+                break;
+
+            String[] hiveRow = hiveReader.getRow();
+            String[] snapshotRow = snapshotReader.getRow();
+            assertArrayEquals(hiveRow, snapshotRow);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/dictionary/src/test/java/com/kylinolap/dict/TableReaderTest.java
----------------------------------------------------------------------
diff --git a/dictionary/src/test/java/com/kylinolap/dict/TableReaderTest.java b/dictionary/src/test/java/com/kylinolap/dict/TableReaderTest.java
new file mode 100644
index 0000000..f1f9bb5
--- /dev/null
+++ b/dictionary/src/test/java/com/kylinolap/dict/TableReaderTest.java
@@ -0,0 +1,45 @@
+/*
+ * Copyright 2013-2014 eBay Software Foundation
+ *
+ * Licensed 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 com.kylinolap.dict;
+
+import static org.junit.Assert.*;
+
+import java.io.IOException;
+import java.util.Arrays;
+
+import org.junit.Test;
+
+import com.kylinolap.dict.lookup.FileTableReader;
+import com.kylinolap.dict.lookup.ReadableTable;
+
+/**
+ * @author yangli9
+ * 
+ */
+public class TableReaderTest {
+
+    @Test
+    public void testBasicReader() throws IOException {
+        FileTableReader reader = new FileTableReader("src/test/resources/dict/DW_SITES", ReadableTable.DELIM_AUTO, 10);
+        while (reader.next()) {
+            assertEquals("[-1, Korea Auction.co.kr, S, 48, 0, 111, 2009-02-11, , DW_OFFPLAT, ]", Arrays.toString(reader.getRow()));
+            break;
+        }
+        reader.close();
+
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/dictionary/src/test/java/com/kylinolap/dict/TrieDictionaryTest.java
----------------------------------------------------------------------
diff --git a/dictionary/src/test/java/com/kylinolap/dict/TrieDictionaryTest.java b/dictionary/src/test/java/com/kylinolap/dict/TrieDictionaryTest.java
new file mode 100644
index 0000000..81287ad
--- /dev/null
+++ b/dictionary/src/test/java/com/kylinolap/dict/TrieDictionaryTest.java
@@ -0,0 +1,337 @@
+/*
+ * Copyright 2013-2014 eBay Software Foundation
+ *
+ * Licensed 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 com.kylinolap.dict;
+
+import static org.junit.Assert.*;
+
+import java.io.BufferedReader;
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.DataInputStream;
+import java.io.DataOutputStream;
+import java.io.FileInputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.InputStreamReader;
+import java.io.UnsupportedEncodingException;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.Random;
+import java.util.TreeSet;
+
+import org.junit.Test;
+
+public class TrieDictionaryTest {
+
+    public static void main(String[] args) throws Exception {
+        InputStream is = new FileInputStream("src/test/resources/dict/dw_category_grouping_names.dat");
+        // InputStream is =
+        // Util.getPackageResourceAsStream(TrieDictionaryTest.class,
+        // "eng_com.dic");
+        ArrayList<String> str = loadStrings(is);
+        benchmarkStringDictionary(str);
+    }
+
+    @Test
+    public void partOverflowTest() {
+        ArrayList<String> str = new ArrayList<String>();
+        // str.add("");
+        str.add("part");
+        str.add("par");
+        str.add("partition");
+        str.add("party");
+        str.add("parties");
+        str.add("paint");
+        String longStr = "paintjkjdfklajkdljfkdsajklfjklsadjkjekjrklewjrklewjklrjklewjkljkljkljkljweklrjewkljrklewjrlkjewkljrkljkljkjlkjjkljkljkljkljlkjlkjlkjljdfadfads" + "dddddddddddddddddddddddddddddddddddddddddddddddddkfjadslkfjdsakljflksadjklfjklsjfkljwelkrjewkljrklewjklrjelkwjrklewjrlkjwkljerklkljlkjrlkwejrk" + "dddddddddddddddddddddddddddddddddddddddddddddddddkfjadslkfjdsakljflksadjklfjklsjfkljwelkrjewkljrklewjklrjelkwjrklewjrlkjwkljerklkljlkjrlkwejrk" + "dddddddddddddddddddddddddddddddddddddddddddddddddkfjadslkfjdsakljflksadjklfjklsjfkljwelkrjewkljrklewjklrjelkwjrklewjrlkjwkljerklkljlkjrlkwejrk" + "dddddddddddddddddddddddddddddddddddddddddddddddddkfjadslkfjdsakljflksadjklfjklsjfkljwelkrjewkljrklewjklrjelkwjrklewjrlkjwkljerklkljlkjrlkwejrk" + "dddddddddddddddddddddddddddddddddddddddddddddddddkfjadslkfjdsakljflksadjklfjklsjfkljwelkrjewkljrklewjklrjelkwjrklewjrlkjwkljerklkljlkjrlkwejrk"
+                + "dddddddddddddddddddddddddddddddddddddddddddddddddkfjadslkfjdsakljflksadjklfjklsjfkljwelkrjewkljrklewjklrjelkwjrklewjrlkjwkljerklkljlkjrlkwejrk" + "dddddddddddddddddddddddddddddddddddddddddddddddddkfjadslkfjdsakljflksadjklfjklsjfkljwelkrjewkljrklewjklrjelkwjrklewjrlkjwkljerklkljlkjrlkwejrk";
+        System.out.println("The length of the long string is " + longStr.length());
+        str.add(longStr);
+
+        str.add("zzzzzz" + longStr);// another long string
+
+        TrieDictionaryBuilder<String> b = newDictBuilder(str);
+        TrieDictionary<String> dict = b.build(0);
+
+        TreeSet<String> set = new TreeSet<String>();
+        for (String s : str) {
+            set.add(s);
+        }
+
+        // test serialize
+        dict = testSerialize(dict);
+
+        // test basic id<==>value
+        Iterator<String> it = set.iterator();
+        int id = 0;
+        int previousId = -1;
+        for (; it.hasNext(); id++) {
+            String value = it.next();
+
+            // in case of overflow parts, there exist interpolation nodes
+            // they exist to make sure that any node's part is shorter than 255
+            int actualId = dict.getIdFromValue(value);
+            assertTrue(actualId >= id);
+            assertTrue(actualId > previousId);
+            previousId = actualId;
+
+            assertEquals(value, dict.getValueFromId(actualId));
+        }
+    }
+
+    @Test
+    public void emptyValueTest() {
+        ArrayList<String> str = new ArrayList<String>();
+        str.add("");
+        str.add("part");
+        str.add("par");
+        str.add("partition");
+        str.add("party");
+        str.add("parties");
+        str.add("paint");
+        testStringDictionary(str, null);
+    }
+
+    @Test
+    public void simpleTrieTest() {
+        ArrayList<String> str = new ArrayList<String>();
+        str.add("part");
+        str.add("part"); // meant to be dup
+        str.add("par");
+        str.add("partition");
+        str.add("party");
+        str.add("parties");
+        str.add("paint");
+
+        ArrayList<String> notFound = new ArrayList<String>();
+        notFound.add("");
+        notFound.add("p");
+        notFound.add("pa");
+        notFound.add("pb");
+        notFound.add("parti");
+        notFound.add("partz");
+        notFound.add("partyz");
+
+        testStringDictionary(str, notFound);
+    }
+
+    @Test
+    public void englishWordsTest() throws Exception {
+        InputStream is = new FileInputStream("src/test/resources/dict/eng_com.dic");
+        ArrayList<String> str = loadStrings(is);
+        testStringDictionary(str, null);
+    }
+
+    @Test
+    public void categoryNamesTest() throws Exception {
+        InputStream is = new FileInputStream("src/test/resources/dict/dw_category_grouping_names.dat");
+        ArrayList<String> str = loadStrings(is);
+        testStringDictionary(str, null);
+    }
+
+    private static void benchmarkStringDictionary(ArrayList<String> str) throws UnsupportedEncodingException {
+        TrieDictionaryBuilder<String> b = newDictBuilder(str);
+        b.stats().print();
+        TrieDictionary<String> dict = b.build(0);
+
+        TreeSet<String> set = new TreeSet<String>();
+        for (String s : str) {
+            set.add(s);
+        }
+
+        // prepare id==>value array and value==>id map
+        HashMap<String, Integer> map = new HashMap<String, Integer>();
+        String[] strArray = new String[set.size()];
+        byte[][] array = new byte[set.size()][];
+        Iterator<String> it = set.iterator();
+        for (int id = 0; it.hasNext(); id++) {
+            String value = it.next();
+            map.put(value, id);
+            strArray[id] = value;
+            array[id] = value.getBytes("UTF-8");
+        }
+
+        // System.out.println("Dict size in bytes:  " +
+        // MemoryUtil.deepMemoryUsageOf(dict));
+        // System.out.println("Map size in bytes:   " +
+        // MemoryUtil.deepMemoryUsageOf(map));
+        // System.out.println("Array size in bytes: " +
+        // MemoryUtil.deepMemoryUsageOf(strArray));
+
+        // warm-up, said that code only got JIT after run 1k-10k times,
+        // following jvm options may help
+        // -XX:CompileThreshold=1500
+        // -XX:+PrintCompilation
+        benchmark("Warm up", dict, set, map, strArray, array);
+        benchmark("Benchmark", dict, set, map, strArray, array);
+    }
+
+    private static int benchmark(String msg, TrieDictionary<String> dict, TreeSet<String> set, HashMap<String, Integer> map, String[] strArray, byte[][] array) {
+        int n = set.size();
+        int times = 10 * 1000 * 1000 / n; // run 10 million lookups
+        int keep = 0; // make sure JIT don't OPT OUT function calls under test
+        byte[] valueBytes = new byte[dict.getSizeOfValue()];
+        long start;
+
+        // benchmark value==>id, via HashMap
+        System.out.println(msg + " HashMap lookup value==>id");
+        start = System.currentTimeMillis();
+        for (int i = 0; i < times; i++) {
+            for (int j = 0; j < n; j++) {
+                keep |= map.get(strArray[j]);
+            }
+        }
+        long timeValueToIdByMap = System.currentTimeMillis() - start;
+        System.out.println(timeValueToIdByMap);
+
+        // benchmark value==>id, via Dict
+        System.out.println(msg + " Dictionary lookup value==>id");
+        start = System.currentTimeMillis();
+        for (int i = 0; i < times; i++) {
+            for (int j = 0; j < n; j++) {
+                keep |= dict.getIdFromValueBytes(array[j], 0, array[j].length);
+            }
+        }
+        long timeValueToIdByDict = System.currentTimeMillis() - start;
+        System.out.println(timeValueToIdByDict);
+
+        // benchmark id==>value, via Array
+        System.out.println(msg + " Array lookup id==>value");
+        start = System.currentTimeMillis();
+        for (int i = 0; i < times; i++) {
+            for (int j = 0; j < n; j++) {
+                keep |= strArray[j].length();
+            }
+        }
+        long timeIdToValueByArray = System.currentTimeMillis() - start;
+        System.out.println(timeIdToValueByArray);
+
+        // benchmark id==>value, via Dict
+        System.out.println(msg + " Dictionary lookup id==>value");
+        start = System.currentTimeMillis();
+        for (int i = 0; i < times; i++) {
+            for (int j = 0; j < n; j++) {
+                keep |= dict.getValueBytesFromId(j, valueBytes, 0);
+            }
+        }
+        long timeIdToValueByDict = System.currentTimeMillis() - start;
+        System.out.println(timeIdToValueByDict);
+
+        return keep;
+    }
+
+    private static void testStringDictionary(ArrayList<String> str, ArrayList<String> notFound) {
+        TrieDictionaryBuilder<String> b = newDictBuilder(str);
+        int baseId = new Random().nextInt(100);
+        TrieDictionary<String> dict = b.build(baseId);
+
+        TreeSet<String> set = new TreeSet<String>();
+        for (String s : str) {
+            set.add(s);
+        }
+
+        // test serialize
+        dict = testSerialize(dict);
+
+        // test basic id<==>value
+        Iterator<String> it = set.iterator();
+        int id = baseId;
+        for (; it.hasNext(); id++) {
+            String value = it.next();
+            // System.out.println("checking " + id + " <==> " + value);
+
+            assertEquals(id, dict.getIdFromValue(value));
+            assertEquals(value, dict.getValueFromId(id));
+        }
+        if (notFound != null) {
+            for (String s : notFound) {
+                try {
+                    dict.getIdFromValue(s);
+                    fail("For not found value '" + s + "', IllegalArgumentException is expected");
+                } catch (IllegalArgumentException e) {
+                    // good
+                }
+            }
+        }
+
+        // test null value
+        int nullId = dict.getIdFromValue(null);
+        assertNull(dict.getValueFromId(nullId));
+        int nullId2 = dict.getIdFromValueBytes(null, 0, 0);
+        assertEquals(dict.getValueBytesFromId(nullId2, null, 0), 0);
+        assertEquals(nullId, nullId2);
+    }
+
+    private static TrieDictionary<String> testSerialize(TrieDictionary<String> dict) {
+        try {
+            ByteArrayOutputStream bout = new ByteArrayOutputStream();
+            DataOutputStream dataout = new DataOutputStream(bout);
+            dict.write(dataout);
+            dataout.close();
+            ByteArrayInputStream bin = new ByteArrayInputStream(bout.toByteArray());
+            DataInputStream datain = new DataInputStream(bin);
+            TrieDictionary<String> r = new TrieDictionary<String>();
+            r.readFields(datain);
+            datain.close();
+            return r;
+        } catch (IOException e) {
+            throw new RuntimeException(e);
+        }
+    }
+
+    private static TrieDictionaryBuilder<String> newDictBuilder(ArrayList<String> str) {
+        TrieDictionaryBuilder<String> b = new TrieDictionaryBuilder<String>(new StringBytesConverter());
+        for (String s : str)
+            b.addValue(s);
+        return b;
+    }
+
+    private static ArrayList<String> loadStrings(InputStream is) throws Exception {
+        ArrayList<String> r = new ArrayList<String>();
+        BufferedReader reader = new BufferedReader(new InputStreamReader(is, "UTF-8"));
+        try {
+            String word;
+            while ((word = reader.readLine()) != null) {
+                word = word.trim();
+                if (word.isEmpty() == false)
+                    r.add(word);
+            }
+        } finally {
+            reader.close();
+            is.close();
+        }
+        return r;
+    }
+
+    @Test
+    public void testSuperLongStringValue() {
+        String longPrefix = "0123456789012345678901234567890123456789012345678901234567890123456789012345678901234567890123456789" + "0123456789012345678901234567890123456789012345678901234567890123456789012345678901234567890123456789";
+
+        TrieDictionaryBuilder<String> b = new TrieDictionaryBuilder<String>(new StringBytesConverter());
+        String v1 = longPrefix + "abcdefghijklmnopqrstuvwxyzabcdefghijklmnopqrstuvwxyzabcdefghijklmnopqrstuvwxyzabcdefghijklmnopqrstuvwxyzabcdefghijklmnopqrstuvwxyzabcdefghijklmnopqrstuvwxyz";
+        String v2 = longPrefix + "xyz";
+
+        b.addValue(v1);
+        b.addValue(v2);
+        TrieDictionary<String> dict = b.build(0);
+        dict.dump(System.out);
+    }
+
+    @Test
+    public void testRounding() {
+        // see NumberDictionaryTest.testRounding();
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/dictionary/src/test/resources/dict/DW_SITES
----------------------------------------------------------------------
diff --git a/dictionary/src/test/resources/dict/DW_SITES b/dictionary/src/test/resources/dict/DW_SITES
new file mode 100644
index 0000000..6f6af46
Binary files /dev/null and b/dictionary/src/test/resources/dict/DW_SITES differ