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:22 UTC

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

Repository: incubator-kylin
Updated Branches:
  refs/heads/master 3fa1ea97b -> 4b631f92c


http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/server/src/main/java/com/kylinolap/rest/service/QueryService.java
----------------------------------------------------------------------
diff --git a/server/src/main/java/com/kylinolap/rest/service/QueryService.java b/server/src/main/java/com/kylinolap/rest/service/QueryService.java
new file mode 100644
index 0000000..9277a08
--- /dev/null
+++ b/server/src/main/java/com/kylinolap/rest/service/QueryService.java
@@ -0,0 +1,482 @@
+/*
+ * 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.rest.service;
+
+import java.io.IOException;
+import java.sql.Connection;
+import java.sql.DatabaseMetaData;
+import java.sql.PreparedStatement;
+import java.sql.ResultSet;
+import java.sql.ResultSetMetaData;
+import java.sql.SQLException;
+import java.sql.Statement;
+import java.sql.Time;
+import java.sql.Timestamp;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Date;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import javax.sql.DataSource;
+
+import net.hydromatic.avatica.ColumnMetaData.Rep;
+
+import org.apache.commons.io.IOUtils;
+import org.apache.hadoop.hbase.client.Get;
+import org.apache.hadoop.hbase.client.HTableInterface;
+import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.client.Result;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.springframework.security.access.AccessDeniedException;
+import org.springframework.security.access.prepost.PreAuthorize;
+import org.springframework.security.core.context.SecurityContextHolder;
+import org.springframework.stereotype.Component;
+
+import com.kylinolap.common.KylinConfig;
+import com.kylinolap.common.persistence.HBaseConnection;
+import com.kylinolap.cube.CubeInstance;
+import com.kylinolap.cube.CubeManager;
+import com.kylinolap.cube.cuboid.Cuboid;
+import com.kylinolap.query.relnode.OLAPContext;
+import com.kylinolap.rest.constant.Constant;
+import com.kylinolap.rest.metrics.QueryMetrics;
+import com.kylinolap.rest.model.ColumnMeta;
+import com.kylinolap.rest.model.Query;
+import com.kylinolap.rest.model.SelectedColumnMeta;
+import com.kylinolap.rest.model.TableMeta;
+import com.kylinolap.rest.request.PrepareSqlRequest;
+import com.kylinolap.rest.request.PrepareSqlRequest.StateParam;
+import com.kylinolap.rest.request.SQLRequest;
+import com.kylinolap.rest.response.SQLResponse;
+import com.kylinolap.rest.util.QueryUtil;
+import com.kylinolap.rest.util.Serializer;
+
+/**
+ * @author xduo
+ */
+@Component("queryService")
+public class QueryService extends BasicService {
+
+    private static final Logger logger = LoggerFactory.getLogger(QueryService.class);
+
+    public static final String USER_QUERY_FAMILY = "q";
+    private Serializer<Query[]> querySerializer = new Serializer<Query[]>(Query[].class);
+    private static final String DEFAULT_TABLE_PREFIX = "kylin_metadata";
+    private static final String USER_TABLE_NAME = "_user";
+    private static final String USER_QUERY_COLUMN = "c";
+    private String hbaseUrl = null;
+    private String tableNameBase = null;
+    private String userTableName = null;
+
+    public QueryService() {
+        String metadataUrl = KylinConfig.getInstanceFromEnv().getMetadataUrl();
+        // split TABLE@HBASE_URL
+        int cut = metadataUrl.indexOf('@');
+        tableNameBase = cut < 0 ? DEFAULT_TABLE_PREFIX : metadataUrl.substring(0, cut);
+        hbaseUrl = cut < 0 ? metadataUrl : metadataUrl.substring(cut + 1);
+        userTableName = tableNameBase + USER_TABLE_NAME;
+    }
+
+    public List<TableMeta> getMetadata(String project) throws SQLException {
+        return getMetadata(getCubeManager(), project, true);
+    }
+
+    public SQLResponse query(SQLRequest sqlRequest) throws Exception {
+        SQLResponse fakeResponse = QueryUtil.tableauIntercept(sqlRequest.getSql());
+        if (null != fakeResponse) {
+            logger.debug("Return fake response, is exception? " + fakeResponse.getIsException());
+
+            return fakeResponse;
+        }
+
+        String correctedSql = QueryUtil.healSickSql(sqlRequest.getSql());
+        if (correctedSql.equals(sqlRequest.getSql()) == false)
+            logger.debug("The corrected query: " + correctedSql);
+
+        return executeQuery(correctedSql, sqlRequest);
+    }
+
+    public void saveQuery(final String creator, final Query query) throws IOException {
+        List<Query> queries = getQueries(creator);
+        queries.add(query);
+        Query[] queryArray = new Query[queries.size()];
+
+        byte[] bytes = querySerializer.serialize(queries.toArray(queryArray));
+        HTableInterface htable = null;
+        try {
+            htable = HBaseConnection.get(hbaseUrl).getTable(userTableName);
+            Put put = new Put(Bytes.toBytes(creator));
+            put.add(Bytes.toBytes(USER_QUERY_FAMILY), Bytes.toBytes(USER_QUERY_COLUMN), bytes);
+
+            htable.put(put);
+            htable.flushCommits();
+        } finally {
+            IOUtils.closeQuietly(htable);
+        }
+    }
+
+    public void removeQuery(final String creator, final String id) throws IOException {
+        List<Query> queries = getQueries(creator);
+        Iterator<Query> queryIter = queries.iterator();
+
+        boolean changed = false;
+        while (queryIter.hasNext()) {
+            Query temp = queryIter.next();
+            if (temp.getId().equals(id)) {
+                queryIter.remove();
+                changed = true;
+                break;
+            }
+        }
+
+        if (!changed) {
+            return;
+        }
+
+        Query[] queryArray = new Query[queries.size()];
+        byte[] bytes = querySerializer.serialize(queries.toArray(queryArray));
+        HTableInterface htable = null;
+        try {
+            htable = HBaseConnection.get(hbaseUrl).getTable(userTableName);
+            Put put = new Put(Bytes.toBytes(creator));
+            put.add(Bytes.toBytes(USER_QUERY_FAMILY), Bytes.toBytes(USER_QUERY_COLUMN), bytes);
+
+            htable.put(put);
+            htable.flushCommits();
+        } finally {
+            IOUtils.closeQuietly(htable);
+        }
+    }
+
+    public List<Query> getQueries(final String creator) throws IOException {
+        if (null == creator) {
+            return null;
+        }
+
+        List<Query> queries = new ArrayList<Query>();
+        HTableInterface htable = null;
+        try {
+            htable = HBaseConnection.get(hbaseUrl).getTable(userTableName);
+            Get get = new Get(Bytes.toBytes(creator));
+            get.addFamily(Bytes.toBytes(USER_QUERY_FAMILY));
+            Result result = htable.get(get);
+            Query[] query = querySerializer.deserialize(result.getValue(Bytes.toBytes(USER_QUERY_FAMILY), Bytes.toBytes(USER_QUERY_COLUMN)));
+
+            if (null != query) {
+                queries.addAll(Arrays.asList(query));
+            }
+        } finally {
+            IOUtils.closeQuietly(htable);
+        }
+
+        return queries;
+    }
+
+    public void logQuery(final SQLRequest request, final SQLResponse response, final Date startTime, final Date endTime) {
+        final String user = SecurityContextHolder.getContext().getAuthentication().getName();
+        final Set<String> cubeNames = new HashSet<String>();
+        final Set<Long> cuboidIds = new HashSet<Long>();
+        long totalScanCount = 0;
+        float duration = (endTime.getTime() - startTime.getTime()) / (float) 1000;
+
+        if (!response.isHitCache() && null != OLAPContext.getThreadLocalContexts()) {
+            for (OLAPContext ctx : OLAPContext.getThreadLocalContexts()) {
+                Cuboid cuboid = ctx.storageContext.getCuboid();
+                if (cuboid != null) {
+                    //Some queries do not involve cuboid, e.g. lookup table query
+                    cuboidIds.add(cuboid.getId());
+                }
+
+                if (ctx.cubeInstance != null) {
+                    String cubeName = ctx.cubeInstance.getName();
+                    cubeNames.add(cubeName);
+                }
+
+                totalScanCount += ctx.storageContext.getTotalScanCount();
+            }
+        }
+
+        int resultRowCount = 0;
+        if (!response.getIsException() && response.getResults() != null) {
+            resultRowCount = response.getResults().size();
+        }
+
+        QueryMetrics.getInstance().increase("duration", duration);
+        QueryMetrics.getInstance().increase("totalScanCount", (float) totalScanCount);
+        QueryMetrics.getInstance().increase("count", (float) 1);
+
+        String newLine = System.getProperty("line.separator");
+        StringBuilder stringBuilder = new StringBuilder();
+        stringBuilder.append(newLine);
+        stringBuilder.append("==========================[QUERY]===============================").append(newLine);
+        stringBuilder.append("SQL: ").append(request.getSql()).append(newLine);
+        stringBuilder.append("User: ").append(user).append(newLine);
+        stringBuilder.append("Success: ").append((null == response.getExceptionMessage())).append(newLine);
+        stringBuilder.append("Duration: ").append(duration).append(newLine);
+        stringBuilder.append("Project: ").append(request.getProject()).append(newLine);
+        stringBuilder.append("Cube Names: ").append(cubeNames).append(newLine);
+        stringBuilder.append("Cuboid Ids: ").append(cuboidIds).append(newLine);
+        stringBuilder.append("Total scan count: ").append(totalScanCount).append(newLine);
+        stringBuilder.append("Result row count: ").append(resultRowCount).append(newLine);
+        stringBuilder.append("Accept Partial: ").append(request.isAcceptPartial()).append(newLine);
+        stringBuilder.append("Hit Cache: ").append(response.isHitCache()).append(newLine);
+        stringBuilder.append("Message: ").append(response.getExceptionMessage()).append(newLine);
+        stringBuilder.append("==========================[QUERY]===============================").append(newLine);
+
+        logger.info(stringBuilder.toString());
+    }
+
+    /**
+     * @param sql
+     * @throws SQLException
+     */
+    @PreAuthorize(Constant.ACCESS_HAS_ROLE_ADMIN + " or hasPermission(#cube, 'ADMINISTRATION') or hasPermission(#cube, 'MANAGEMENT')" + " or hasPermission(#cube, 'OPERATION') or hasPermission(#cube, 'READ')")
+    public void checkAuthorization(CubeInstance cube) throws AccessDeniedException {
+    }
+
+    protected SQLResponse executeQuery(String sql, SQLRequest sqlRequest) throws Exception {
+        sql = sql.trim().replace(";", "");
+
+        int limit = sqlRequest.getLimit();
+        if (limit > 0 && !sql.toLowerCase().contains("limit")) {
+            sql += (" LIMIT " + limit);
+        }
+
+        int offset = sqlRequest.getOffset();
+        if (offset > 0 && !sql.toLowerCase().contains("offset")) {
+            sql += (" OFFSET " + offset);
+        }
+
+        // add extra parameters into olap context, like acceptPartial
+        Map<String, String> parameters = new HashMap<String, String>();
+        parameters.put(OLAPContext.PRM_ACCEPT_PARTIAL_RESULT, String.valueOf(sqlRequest.isAcceptPartial()));
+        OLAPContext.setParameters(parameters);
+
+        return execute(sql, sqlRequest);
+    }
+
+    protected List<TableMeta> getMetadata(CubeManager cubeMgr, String project, boolean cubedOnly) throws SQLException {
+
+        Connection conn = null;
+        ResultSet columnMeta = null;
+        List<TableMeta> tableMetas = null;
+
+        try {
+            DataSource dataSource = getOLAPDataSource(project);
+            conn = dataSource.getConnection();
+            DatabaseMetaData metaData = conn.getMetaData();
+
+            logger.debug("getting table metas");
+            ResultSet JDBCTableMeta = metaData.getTables(null, null, null, null);
+
+            tableMetas = new LinkedList<TableMeta>();
+            Map<String, TableMeta> tableMap = new HashMap<String, TableMeta>();
+            while (JDBCTableMeta.next()) {
+                String catalogName = JDBCTableMeta.getString(1);
+                String schemaName = JDBCTableMeta.getString(2);
+
+                // Not every JDBC data provider offers full 10 columns, for
+                // example,
+                // PostgreSQL has only 5
+                TableMeta tblMeta = new TableMeta(catalogName == null ? Constant.FakeCatalogName : catalogName, schemaName == null ? Constant.FakeSchemaName : schemaName, JDBCTableMeta.getString(3), JDBCTableMeta.getString(4), JDBCTableMeta.getString(5), null, null, null, null, null);
+
+                if (!cubedOnly || getProjectManager().isExposedTable(project, tblMeta.getTABLE_NAME())) {
+                    tableMetas.add(tblMeta);
+                    tableMap.put(tblMeta.getTABLE_SCHEM() + "#" + tblMeta.getTABLE_NAME(), tblMeta);
+                }
+            }
+
+            logger.debug("getting column metas");
+            columnMeta = metaData.getColumns(null, null, null, null);
+
+            while (columnMeta.next()) {
+                String catalogName = columnMeta.getString(1);
+                String schemaName = columnMeta.getString(2);
+
+                // kylin(optiq) is not strictly following JDBC specification
+                ColumnMeta colmnMeta = new ColumnMeta(catalogName == null ? Constant.FakeCatalogName : catalogName, schemaName == null ? Constant.FakeSchemaName : schemaName, columnMeta.getString(3), columnMeta.getString(4), columnMeta.getInt(5), columnMeta.getString(6), columnMeta.getInt(7), getInt(columnMeta.getString(8)), columnMeta.getInt(9), columnMeta.getInt(10), columnMeta.getInt(11), columnMeta.getString(12), columnMeta.getString(13), getInt(columnMeta.getString(14)), getInt(columnMeta.getString(15)), columnMeta.getInt(16), columnMeta.getInt(17), columnMeta.getString(18), columnMeta.getString(19), columnMeta.getString(20), columnMeta.getString(21), getShort(columnMeta.getString(22)), columnMeta.getString(23));
+
+                if (!cubedOnly || getProjectManager().isExposedColumn(project, colmnMeta.getTABLE_NAME(), colmnMeta.getCOLUMN_NAME())) {
+                    tableMap.get(colmnMeta.getTABLE_SCHEM() + "#" + colmnMeta.getTABLE_NAME()).addColumn(colmnMeta);
+                }
+            }
+            logger.debug("done column metas");
+        } finally {
+            close(columnMeta, null, conn);
+        }
+
+        return tableMetas;
+    }
+
+    /**
+     * @param sql
+     * @param project
+     * @return
+     * @throws Exception
+     */
+    private SQLResponse execute(String sql, SQLRequest sqlRequest) throws Exception {
+        Connection conn = null;
+        Statement stat = null;
+        ResultSet resultSet = null;
+        List<List<String>> results = new LinkedList<List<String>>();
+        List<SelectedColumnMeta> columnMetas = new LinkedList<SelectedColumnMeta>();
+
+        try {
+            conn = getOLAPDataSource(sqlRequest.getProject()).getConnection();
+
+            if (sqlRequest instanceof PrepareSqlRequest) {
+                PreparedStatement preparedState = conn.prepareStatement(sql);
+
+                for (int i = 0; i < ((PrepareSqlRequest) sqlRequest).getParams().length; i++) {
+                    setParam(preparedState, i + 1, ((PrepareSqlRequest) sqlRequest).getParams()[i]);
+                }
+
+                resultSet = preparedState.executeQuery();
+            } else {
+                stat = conn.createStatement();
+                resultSet = stat.executeQuery(sql);
+            }
+
+            ResultSetMetaData metaData = resultSet.getMetaData();
+            int columnCount = metaData.getColumnCount();
+
+            // Fill in selected column meta
+            for (int i = 1; i <= columnCount; ++i) {
+                columnMetas.add(new SelectedColumnMeta(metaData.isAutoIncrement(i), metaData.isCaseSensitive(i), metaData.isSearchable(i), metaData.isCurrency(i), metaData.isNullable(i), metaData.isSigned(i), metaData.getColumnDisplaySize(i), metaData.getColumnLabel(i), metaData.getColumnName(i), metaData.getSchemaName(i), metaData.getCatalogName(i), metaData.getTableName(i), metaData.getPrecision(i), metaData.getScale(i), metaData.getColumnType(i), metaData.getColumnTypeName(i), metaData.isReadOnly(i), metaData.isWritable(i), metaData.isDefinitelyWritable(i)));
+            }
+
+            List<String> oneRow = new LinkedList<String>();
+
+            // fill in results
+            while (resultSet.next()) {
+                for (int i = 0; i < columnCount; i++) {
+                    oneRow.add((resultSet.getString(i + 1)));
+                }
+
+                results.add(new LinkedList<String>(oneRow));
+                oneRow.clear();
+            }
+        } finally {
+            close(resultSet, stat, conn);
+        }
+
+        boolean isPartialResult = false;
+        String cube = "";
+        long totalScanCount = 0;
+        if (OLAPContext.getThreadLocalContexts() != null) { // contexts can be null in case of 'explain plan for'
+            for (OLAPContext ctx : OLAPContext.getThreadLocalContexts()) {
+                isPartialResult |= ctx.storageContext.isPartialResultReturned();
+                cube = ctx.cubeInstance.getName();
+                totalScanCount += ctx.storageContext.getTotalScanCount();
+            }
+        }
+
+        SQLResponse response = new SQLResponse(columnMetas, results, cube, 0, false, null, isPartialResult);
+        response.setTotalScanCount(totalScanCount);
+
+        return response;
+    }
+
+    /**
+     * @param preparedState
+     * @param param
+     * @throws SQLException
+     */
+    private void setParam(PreparedStatement preparedState, int index, StateParam param) throws SQLException {
+        boolean isNull = (null == param.getValue());
+
+        Class<?> clazz;
+        try {
+            clazz = Class.forName(param.getClassName());
+        } catch (ClassNotFoundException e) {
+            throw new RuntimeException(e.getMessage(), e);
+        }
+
+        Rep rep = Rep.of(clazz);
+
+        switch (rep) {
+        case PRIMITIVE_CHAR:
+        case CHARACTER:
+        case STRING:
+            preparedState.setString(index, isNull ? null : String.valueOf(param.getValue()));
+            break;
+        case PRIMITIVE_INT:
+        case INTEGER:
+            preparedState.setInt(index, isNull ? 0 : Integer.valueOf(param.getValue()));
+            break;
+        case PRIMITIVE_SHORT:
+        case SHORT:
+            preparedState.setShort(index, isNull ? 0 : Short.valueOf(param.getValue()));
+            break;
+        case PRIMITIVE_LONG:
+        case LONG:
+            preparedState.setLong(index, isNull ? 0 : Long.valueOf(param.getValue()));
+            break;
+        case PRIMITIVE_FLOAT:
+        case FLOAT:
+            preparedState.setFloat(index, isNull ? 0 : Float.valueOf(param.getValue()));
+            break;
+        case PRIMITIVE_DOUBLE:
+        case DOUBLE:
+            preparedState.setDouble(index, isNull ? 0 : Double.valueOf(param.getValue()));
+            break;
+        case PRIMITIVE_BOOLEAN:
+        case BOOLEAN:
+            preparedState.setBoolean(index, !isNull && Boolean.parseBoolean(param.getValue()));
+            break;
+        case PRIMITIVE_BYTE:
+        case BYTE:
+            preparedState.setByte(index, isNull ? 0 : Byte.valueOf(param.getValue()));
+            break;
+        case JAVA_UTIL_DATE:
+        case JAVA_SQL_DATE:
+            preparedState.setDate(index, isNull ? null : java.sql.Date.valueOf(param.getValue()));
+            break;
+        case JAVA_SQL_TIME:
+            preparedState.setTime(index, isNull ? null : Time.valueOf(param.getValue()));
+            break;
+        case JAVA_SQL_TIMESTAMP:
+            preparedState.setTimestamp(index, isNull ? null : Timestamp.valueOf(param.getValue()));
+            break;
+        default:
+            preparedState.setObject(index, isNull ? null : param.getValue());
+        }
+    }
+
+    private int getInt(String content) {
+        try {
+            return Integer.parseInt(content);
+        } catch (Exception e) {
+            return -1;
+        }
+    }
+
+    private short getShort(String content) {
+        try {
+            return Short.parseShort(content);
+        } catch (Exception e) {
+            return -1;
+        }
+    }
+}


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

Posted by lu...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/LICENSE
----------------------------------------------------------------------
diff --git a/LICENSE b/LICENSE
new file mode 100644
index 0000000..29dc295
--- /dev/null
+++ b/LICENSE
@@ -0,0 +1,54 @@
+Apache License
+
+Version 2.0, January 2004
+
+http://www.apache.org/licenses/
+
+TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION
+
+1. Definitions.
+
+"License" shall mean the terms and conditions for use, reproduction, and distribution as defined by Sections 1 through 9 of this document.
+
+"Licensor" shall mean the copyright owner or entity authorized by the copyright owner that is granting the License.
+
+"Legal Entity" shall mean the union of the acting entity and all other entities that control, are controlled by, or are under common control with that entity. For the purposes of this definition, "control" means (i) the power, direct or indirect, to cause the direction or management of such entity, whether by contract or otherwise, or (ii) ownership of fifty percent (50%) or more of the outstanding shares, or (iii) beneficial ownership of such entity.
+
+"You" (or "Your") shall mean an individual or Legal Entity exercising permissions granted by this License.
+
+"Source" form shall mean the preferred form for making modifications, including but not limited to software source code, documentation source, and configuration files.
+
+"Object" form shall mean any form resulting from mechanical transformation or translation of a Source form, including but not limited to compiled object code, generated documentation, and conversions to other media types.
+
+"Work" shall mean the work of authorship, whether in Source or Object form, made available under the License, as indicated by a copyright notice that is included in or attached to the work (an example is provided in the Appendix below).
+
+"Derivative Works" shall mean any work, whether in Source or Object form, that is based on (or derived from) the Work and for which the editorial revisions, annotations, elaborations, or other modifications represent, as a whole, an original work of authorship. For the purposes of this License, Derivative Works shall not include works that remain separable from, or merely link (or bind by name) to the interfaces of, the Work and Derivative Works thereof.
+
+"Contribution" shall mean any work of authorship, including the original version of the Work and any modifications or additions to that Work or Derivative Works thereof, that is intentionally submitted to Licensor for inclusion in the Work by the copyright owner or by an individual or Legal Entity authorized to submit on behalf of the copyright owner. For the purposes of this definition, "submitted" means any form of electronic, verbal, or written communication sent to the Licensor or its representatives, including but not limited to communication on electronic mailing lists, source code control systems, and issue tracking systems that are managed by, or on behalf of, the Licensor for the purpose of discussing and improving the Work, but excluding communication that is conspicuously marked or otherwise designated in writing by the copyright owner as "Not a Contribution."
+
+"Contributor" shall mean Licensor and any individual or Legal Entity on behalf of whom a Contribution has been received by Licensor and subsequently incorporated within the Work.
+
+2. Grant of Copyright License. Subject to the terms and conditions of this License, each Contributor hereby grants to You a perpetual, worldwide, non-exclusive, no-charge, royalty-free, irrevocable copyright license to reproduce, prepare Derivative Works of, publicly display, publicly perform, sublicense, and distribute the Work and such Derivative Works in Source or Object form.
+
+3. Grant of Patent License. Subject to the terms and conditions of this License, each Contributor hereby grants to You a perpetual, worldwide, non-exclusive, no-charge, royalty-free, irrevocable (except as stated in this section) patent license to make, have made, use, offer to sell, sell, import, and otherwise transfer the Work, where such license applies only to those patent claims licensable by such Contributor that are necessarily infringed by their Contribution(s) alone or by combination of their Contribution(s) with the Work to which such Contribution(s) was submitted. If You institute patent litigation against any entity (including a cross-claim or counterclaim in a lawsuit) alleging that the Work or a Contribution incorporated within the Work constitutes direct or contributory patent infringement, then any patent licenses granted to You under this License for that Work shall terminate as of the date such litigation is filed.
+
+4. Redistribution. You may reproduce and distribute copies of the Work or Derivative Works thereof in any medium, with or without modifications, and in Source or Object form, provided that You meet the following conditions:
+
+You must give any other recipients of the Work or Derivative Works a copy of this License; and
+You must cause any modified files to carry prominent notices stating that You changed the files; and
+You must retain, in the Source form of any Derivative Works that You distribute, all copyright, patent, trademark, and attribution notices from the Source form of the Work, excluding those notices that do not pertain to any part of the Derivative Works; and
+If the Work includes a "NOTICE" text file as part of its distribution, then any Derivative Works that You distribute must include a readable copy of the attribution notices contained within such NOTICE file, excluding those notices that do not pertain to any part of the Derivative Works, in at least one of the following places: within a NOTICE text file distributed as part of the Derivative Works; within the Source form or documentation, if provided along with the Derivative Works; or, within a display generated by the Derivative Works, if and wherever such third-party notices normally appear. The contents of the NOTICE file are for informational purposes only and do not modify the License. You may add Your own attribution notices within Derivative Works that You distribute, alongside or as an addendum to the NOTICE text from the Work, provided that such additional attribution notices cannot be construed as modifying the License. 
+
+You may add Your own copyright statement to Your modifications and may provide additional or different license terms and conditions for use, reproduction, or distribution of Your modifications, or for any such Derivative Works as a whole, provided Your use, reproduction, and distribution of the Work otherwise complies with the conditions stated in this License.
+5. Submission of Contributions. Unless You explicitly state otherwise, any Contribution intentionally submitted for inclusion in the Work by You to the Licensor shall be under the terms and conditions of this License, without any additional terms or conditions. Notwithstanding the above, nothing herein shall supersede or modify the terms of any separate license agreement you may have executed with Licensor regarding such Contributions.
+
+6. Trademarks. This License does not grant permission to use the trade names, trademarks, service marks, or product names of the Licensor, except as required for reasonable and customary use in describing the origin of the Work and reproducing the content of the NOTICE file.
+
+7. Disclaimer of Warranty. Unless required by applicable law or agreed to in writing, Licensor provides the Work (and each Contributor provides its Contributions) on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied, including, without limitation, any warranties or conditions of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A PARTICULAR PURPOSE. You are solely responsible for determining the appropriateness of using or redistributing the Work and assume any risks associated with Your exercise of permissions under this License.
+
+8. Limitation of Liability. In no event and under no legal theory, whether in tort (including negligence), contract, or otherwise, unless required by applicable law (such as deliberate and grossly negligent acts) or agreed to in writing, shall any Contributor be liable to You for damages, including any direct, indirect, special, incidental, or consequential damages of any character arising as a result of this License or out of the use or inability to use the Work (including but not limited to damages for loss of goodwill, work stoppage, computer failure or malfunction, or any and all other commercial damages or losses), even if such Contributor has been advised of the possibility of such damages.
+
+9. Accepting Warranty or Additional Liability. While redistributing the Work or Derivative Works thereof, You may choose to offer, and charge a fee for, acceptance of support, warranty, indemnity, or other liability obligations and/or rights consistent with this License. However, in accepting such obligations, You may act only on Your own behalf and on Your sole responsibility, not on behalf of any other Contributor, and only if You agree to indemnify, defend, and hold each Contributor harmless for any liability incurred by, or claims asserted against, such Contributor by reason of your accepting any such warranty or additional liability.
+
+END OF TERMS AND CONDITIONS
+

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/README.md
----------------------------------------------------------------------
diff --git a/README.md b/README.md
index ec3ce87..44f6859 100644
--- a/README.md
+++ b/README.md
@@ -1,7 +1,94 @@
-Apache Incubator Kylin project
+Kylin OLAP
+===========
 
-This README is a place-holder
 
-Source code will migrate from github.com to here soon.
+Overview
+------------
+> Extreme OLAP Engine for Big Data
 
+An open source distributed Analytical Engine from eBay to provide SQL interface and multi-dimensional analysis (OLAP) on Hadoop to support TB to PB size analysis.
+
+
+Getting Started 
+---------------
+
+### Prerequisites ###
+
+#### Hadoop ####
+- Hadoop: 2.2.0.2.0.6.0-61 or above
+- Hive: 0.12.0.2.0.6.0-61 or above
+- HBase: 0.96.0.2.0.6.0-61-hadoop2
+
+
+_Tested with Hortonworks HDP 2.1.3 and Cloudera Quickstart VM 5.1._
+
+#### Misc ####
+- Command hadoop, hive, hbase is workable on your hadoop cluster
+- JDK Runtime: JDK7 (OpenJDK or Oracle JDK)
+- Maven
+- Git
+- Tomcat (CATALINA_HOME being set)
+- Npm
+
+#### Before Install ####
+- For the quick tutorial, we assume that your hadoop has priviledges disabled which means any user could run hadoop and hive command
+
+
+Installation 
+------------
+It is very easy to install Kylin for exploration/development. There are 3 supported options:
+
+
+1. Sandbox         (HDP or CDH sandbox)
+2. Dev Environment (IDE + Sandbox)
+3. Docker container     (Docker)
+
+#### Sandbox (On-Hadoop-CLI installation) ####
+
+If you are free to install Kylin on your hadoop CLI machine or Hadoop sandbox, this is the most convenient scenario, for it puts everything in a single machine.
+
+
+
+![On-Hadoop-CLI-installation](https://github.com/KylinOLAP/kylinolap.github.io/blob/master/docs/installation/Picture1.png)
+
+For normal users, the application in the above picture means `Kylin Web`, which contains a web interface for cube building, querying and all sorts of management. Kylin Web launches a query engine for querying and a cube build engine for building cubes. These two engines interact with the components in Hadoop CLI, like hive and hbase.
+
+For a hands-on tutorial please visit [On-Hadoop-CLI installation](https://github.com/KylinOLAP/Kylin/wiki/On-Hadoop-CLI-installation).
+
+#### Dev Environment (Off-Hadoop-CLI Installation) ####
+
+This is typically for development environment setup.
+
+Applications does not necessarily mean Kylin Web, it can also be any test cases or java applications running at your local IDE(query engine and cube build engine can be launched programmatically in your code). In this case, it is no longer reasonable to assume that all the Kylin components reside in the same machine as your Hadoop CLI.  Fortunately, Kylin still works under such condition with proper settings on your CLI machine.
+
+![Off-CLI Installation](https://github.com/KylinOLAP/kylinolap.github.io/blob/master/docs/installation/Picture2.png)
+
+For a hands-on tutorial please visit [Off-Hadoop-CLI installation](https://github.com/KylinOLAP/Kylin/wiki/Off-Hadoop-CLI-Installation-(Dev-Env-Setup))
+
+#### Docker Container ####
+With help from [SequenceIQ](http://sequenceiq.com/), there's docker container for Kylin (along with Hadoop, HBase and Hive) available now:[sequenceiq/docker-kylin](https://github.com/sequenceiq/docker-kylin).  The only thing you will need to do is to pull the container from the official Docker repository to be up and running in few minutes. 
+
+Features:
+
+- Size            - Light weight compared to downloading and setting up HDP or CDH sandbox.
+- Cluster support - Supports multi node installation. 
+- Fully Automated - No manual steps. One command does it all 
+
+For a hands-on tutorial please visit [Kylin Docker installation](https://github.com/KylinOLAP/Kylin/wiki/On-Hadoop-Kylin-installation-using-Docker). 
+
+
+
+### Resources ###
+
+* Web Site: <http://kylin.io>
+
+* Google Group:  [Kylin OLAP Group](https://groups.google.com/forum/#!forum/kylin-olap)
+
+* Developer Mail: <ky...@googlegroups.com>
+
+* How To Contribute: See [wiki](https://github.com/KylinOLAP/Kylin/wiki/How-to-Contribute)
+
+* Presentation: [Kylin Hadoop OLAP Engine v1.0](https://github.com/KylinOLAP/Kylin/blob/master/docs/Kylin_Hadoop_OLAP_Engine_v1.0.pdf?raw=true)
+
+*  Apache Proposal: [Apache Kylin](https://wiki.apache.org/incubator/KylinProposal)
 

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/atopcalcite/.settings/org.eclipse.core.resources.prefs
----------------------------------------------------------------------
diff --git a/atopcalcite/.settings/org.eclipse.core.resources.prefs b/atopcalcite/.settings/org.eclipse.core.resources.prefs
new file mode 100644
index 0000000..29abf99
--- /dev/null
+++ b/atopcalcite/.settings/org.eclipse.core.resources.prefs
@@ -0,0 +1,6 @@
+eclipse.preferences.version=1
+encoding//src/main/java=UTF-8
+encoding//src/main/resources=UTF-8
+encoding//src/test/java=UTF-8
+encoding//src/test/resources=UTF-8
+encoding/<project>=UTF-8

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/atopcalcite/.settings/org.eclipse.jdt.core.prefs
----------------------------------------------------------------------
diff --git a/atopcalcite/.settings/org.eclipse.jdt.core.prefs b/atopcalcite/.settings/org.eclipse.jdt.core.prefs
new file mode 100644
index 0000000..7c91e8d
--- /dev/null
+++ b/atopcalcite/.settings/org.eclipse.jdt.core.prefs
@@ -0,0 +1,379 @@
+eclipse.preferences.version=1
+org.eclipse.jdt.core.compiler.annotation.inheritNullAnnotations=disabled
+org.eclipse.jdt.core.compiler.annotation.missingNonNullByDefaultAnnotation=ignore
+org.eclipse.jdt.core.compiler.annotation.nonnull=org.eclipse.jdt.annotation.NonNull
+org.eclipse.jdt.core.compiler.annotation.nonnullbydefault=org.eclipse.jdt.annotation.NonNullByDefault
+org.eclipse.jdt.core.compiler.annotation.nullable=org.eclipse.jdt.annotation.Nullable
+org.eclipse.jdt.core.compiler.annotation.nullanalysis=disabled
+org.eclipse.jdt.core.compiler.codegen.inlineJsrBytecode=enabled
+org.eclipse.jdt.core.compiler.codegen.targetPlatform=1.7
+org.eclipse.jdt.core.compiler.codegen.unusedLocal=preserve
+org.eclipse.jdt.core.compiler.compliance=1.7
+org.eclipse.jdt.core.compiler.debug.lineNumber=generate
+org.eclipse.jdt.core.compiler.debug.localVariable=generate
+org.eclipse.jdt.core.compiler.debug.sourceFile=generate
+org.eclipse.jdt.core.compiler.problem.annotationSuperInterface=warning
+org.eclipse.jdt.core.compiler.problem.assertIdentifier=error
+org.eclipse.jdt.core.compiler.problem.autoboxing=ignore
+org.eclipse.jdt.core.compiler.problem.comparingIdentical=warning
+org.eclipse.jdt.core.compiler.problem.deadCode=warning
+org.eclipse.jdt.core.compiler.problem.deprecation=warning
+org.eclipse.jdt.core.compiler.problem.deprecationInDeprecatedCode=disabled
+org.eclipse.jdt.core.compiler.problem.deprecationWhenOverridingDeprecatedMethod=disabled
+org.eclipse.jdt.core.compiler.problem.discouragedReference=warning
+org.eclipse.jdt.core.compiler.problem.emptyStatement=ignore
+org.eclipse.jdt.core.compiler.problem.enumIdentifier=error
+org.eclipse.jdt.core.compiler.problem.explicitlyClosedAutoCloseable=ignore
+org.eclipse.jdt.core.compiler.problem.fallthroughCase=ignore
+org.eclipse.jdt.core.compiler.problem.fatalOptionalError=disabled
+org.eclipse.jdt.core.compiler.problem.fieldHiding=ignore
+org.eclipse.jdt.core.compiler.problem.finalParameterBound=warning
+org.eclipse.jdt.core.compiler.problem.finallyBlockNotCompletingNormally=warning
+org.eclipse.jdt.core.compiler.problem.forbiddenReference=warning
+org.eclipse.jdt.core.compiler.problem.hiddenCatchBlock=warning
+org.eclipse.jdt.core.compiler.problem.includeNullInfoFromAsserts=disabled
+org.eclipse.jdt.core.compiler.problem.incompatibleNonInheritedInterfaceMethod=warning
+org.eclipse.jdt.core.compiler.problem.incompleteEnumSwitch=warning
+org.eclipse.jdt.core.compiler.problem.indirectStaticAccess=ignore
+org.eclipse.jdt.core.compiler.problem.localVariableHiding=ignore
+org.eclipse.jdt.core.compiler.problem.methodWithConstructorName=warning
+org.eclipse.jdt.core.compiler.problem.missingDefaultCase=ignore
+org.eclipse.jdt.core.compiler.problem.missingDeprecatedAnnotation=ignore
+org.eclipse.jdt.core.compiler.problem.missingEnumCaseDespiteDefault=disabled
+org.eclipse.jdt.core.compiler.problem.missingHashCodeMethod=ignore
+org.eclipse.jdt.core.compiler.problem.missingOverrideAnnotation=ignore
+org.eclipse.jdt.core.compiler.problem.missingOverrideAnnotationForInterfaceMethodImplementation=enabled
+org.eclipse.jdt.core.compiler.problem.missingSerialVersion=warning
+org.eclipse.jdt.core.compiler.problem.missingSynchronizedOnInheritedMethod=ignore
+org.eclipse.jdt.core.compiler.problem.noEffectAssignment=warning
+org.eclipse.jdt.core.compiler.problem.noImplicitStringConversion=warning
+org.eclipse.jdt.core.compiler.problem.nonExternalizedStringLiteral=ignore
+org.eclipse.jdt.core.compiler.problem.nonnullParameterAnnotationDropped=warning
+org.eclipse.jdt.core.compiler.problem.nullAnnotationInferenceConflict=error
+org.eclipse.jdt.core.compiler.problem.nullReference=warning
+org.eclipse.jdt.core.compiler.problem.nullSpecViolation=error
+org.eclipse.jdt.core.compiler.problem.nullUncheckedConversion=warning
+org.eclipse.jdt.core.compiler.problem.overridingPackageDefaultMethod=warning
+org.eclipse.jdt.core.compiler.problem.parameterAssignment=ignore
+org.eclipse.jdt.core.compiler.problem.possibleAccidentalBooleanAssignment=ignore
+org.eclipse.jdt.core.compiler.problem.potentialNullReference=ignore
+org.eclipse.jdt.core.compiler.problem.potentiallyUnclosedCloseable=ignore
+org.eclipse.jdt.core.compiler.problem.rawTypeReference=warning
+org.eclipse.jdt.core.compiler.problem.redundantNullAnnotation=warning
+org.eclipse.jdt.core.compiler.problem.redundantNullCheck=ignore
+org.eclipse.jdt.core.compiler.problem.redundantSpecificationOfTypeArguments=ignore
+org.eclipse.jdt.core.compiler.problem.redundantSuperinterface=ignore
+org.eclipse.jdt.core.compiler.problem.reportMethodCanBePotentiallyStatic=ignore
+org.eclipse.jdt.core.compiler.problem.reportMethodCanBeStatic=ignore
+org.eclipse.jdt.core.compiler.problem.specialParameterHidingField=disabled
+org.eclipse.jdt.core.compiler.problem.staticAccessReceiver=warning
+org.eclipse.jdt.core.compiler.problem.suppressOptionalErrors=disabled
+org.eclipse.jdt.core.compiler.problem.suppressWarnings=enabled
+org.eclipse.jdt.core.compiler.problem.syntacticNullAnalysisForFields=disabled
+org.eclipse.jdt.core.compiler.problem.syntheticAccessEmulation=ignore
+org.eclipse.jdt.core.compiler.problem.typeParameterHiding=warning
+org.eclipse.jdt.core.compiler.problem.unavoidableGenericTypeProblems=enabled
+org.eclipse.jdt.core.compiler.problem.uncheckedTypeOperation=warning
+org.eclipse.jdt.core.compiler.problem.unclosedCloseable=warning
+org.eclipse.jdt.core.compiler.problem.undocumentedEmptyBlock=ignore
+org.eclipse.jdt.core.compiler.problem.unhandledWarningToken=warning
+org.eclipse.jdt.core.compiler.problem.unnecessaryElse=ignore
+org.eclipse.jdt.core.compiler.problem.unnecessaryTypeCheck=ignore
+org.eclipse.jdt.core.compiler.problem.unqualifiedFieldAccess=ignore
+org.eclipse.jdt.core.compiler.problem.unusedDeclaredThrownException=ignore
+org.eclipse.jdt.core.compiler.problem.unusedDeclaredThrownExceptionExemptExceptionAndThrowable=enabled
+org.eclipse.jdt.core.compiler.problem.unusedDeclaredThrownExceptionIncludeDocCommentReference=enabled
+org.eclipse.jdt.core.compiler.problem.unusedDeclaredThrownExceptionWhenOverriding=disabled
+org.eclipse.jdt.core.compiler.problem.unusedImport=warning
+org.eclipse.jdt.core.compiler.problem.unusedLabel=warning
+org.eclipse.jdt.core.compiler.problem.unusedLocal=warning
+org.eclipse.jdt.core.compiler.problem.unusedObjectAllocation=ignore
+org.eclipse.jdt.core.compiler.problem.unusedParameter=ignore
+org.eclipse.jdt.core.compiler.problem.unusedParameterIncludeDocCommentReference=enabled
+org.eclipse.jdt.core.compiler.problem.unusedParameterWhenImplementingAbstract=disabled
+org.eclipse.jdt.core.compiler.problem.unusedParameterWhenOverridingConcrete=disabled
+org.eclipse.jdt.core.compiler.problem.unusedPrivateMember=warning
+org.eclipse.jdt.core.compiler.problem.unusedTypeParameter=ignore
+org.eclipse.jdt.core.compiler.problem.unusedWarningToken=warning
+org.eclipse.jdt.core.compiler.problem.varargsArgumentNeedCast=warning
+org.eclipse.jdt.core.compiler.source=1.7
+org.eclipse.jdt.core.formatter.align_type_members_on_columns=false
+org.eclipse.jdt.core.formatter.alignment_for_arguments_in_allocation_expression=16
+org.eclipse.jdt.core.formatter.alignment_for_arguments_in_annotation=0
+org.eclipse.jdt.core.formatter.alignment_for_arguments_in_enum_constant=16
+org.eclipse.jdt.core.formatter.alignment_for_arguments_in_explicit_constructor_call=16
+org.eclipse.jdt.core.formatter.alignment_for_arguments_in_method_invocation=16
+org.eclipse.jdt.core.formatter.alignment_for_arguments_in_qualified_allocation_expression=16
+org.eclipse.jdt.core.formatter.alignment_for_assignment=0
+org.eclipse.jdt.core.formatter.alignment_for_binary_expression=16
+org.eclipse.jdt.core.formatter.alignment_for_compact_if=16
+org.eclipse.jdt.core.formatter.alignment_for_conditional_expression=80
+org.eclipse.jdt.core.formatter.alignment_for_enum_constants=0
+org.eclipse.jdt.core.formatter.alignment_for_expressions_in_array_initializer=16
+org.eclipse.jdt.core.formatter.alignment_for_method_declaration=0
+org.eclipse.jdt.core.formatter.alignment_for_multiple_fields=16
+org.eclipse.jdt.core.formatter.alignment_for_parameters_in_constructor_declaration=16
+org.eclipse.jdt.core.formatter.alignment_for_parameters_in_method_declaration=16
+org.eclipse.jdt.core.formatter.alignment_for_resources_in_try=80
+org.eclipse.jdt.core.formatter.alignment_for_selector_in_method_invocation=16
+org.eclipse.jdt.core.formatter.alignment_for_superclass_in_type_declaration=16
+org.eclipse.jdt.core.formatter.alignment_for_superinterfaces_in_enum_declaration=16
+org.eclipse.jdt.core.formatter.alignment_for_superinterfaces_in_type_declaration=16
+org.eclipse.jdt.core.formatter.alignment_for_throws_clause_in_constructor_declaration=16
+org.eclipse.jdt.core.formatter.alignment_for_throws_clause_in_method_declaration=16
+org.eclipse.jdt.core.formatter.alignment_for_union_type_in_multicatch=16
+org.eclipse.jdt.core.formatter.blank_lines_after_imports=1
+org.eclipse.jdt.core.formatter.blank_lines_after_package=1
+org.eclipse.jdt.core.formatter.blank_lines_before_field=0
+org.eclipse.jdt.core.formatter.blank_lines_before_first_class_body_declaration=0
+org.eclipse.jdt.core.formatter.blank_lines_before_imports=1
+org.eclipse.jdt.core.formatter.blank_lines_before_member_type=1
+org.eclipse.jdt.core.formatter.blank_lines_before_method=1
+org.eclipse.jdt.core.formatter.blank_lines_before_new_chunk=1
+org.eclipse.jdt.core.formatter.blank_lines_before_package=0
+org.eclipse.jdt.core.formatter.blank_lines_between_import_groups=1
+org.eclipse.jdt.core.formatter.blank_lines_between_type_declarations=1
+org.eclipse.jdt.core.formatter.brace_position_for_annotation_type_declaration=end_of_line
+org.eclipse.jdt.core.formatter.brace_position_for_anonymous_type_declaration=end_of_line
+org.eclipse.jdt.core.formatter.brace_position_for_array_initializer=end_of_line
+org.eclipse.jdt.core.formatter.brace_position_for_block=end_of_line
+org.eclipse.jdt.core.formatter.brace_position_for_block_in_case=end_of_line
+org.eclipse.jdt.core.formatter.brace_position_for_constructor_declaration=end_of_line
+org.eclipse.jdt.core.formatter.brace_position_for_enum_constant=end_of_line
+org.eclipse.jdt.core.formatter.brace_position_for_enum_declaration=end_of_line
+org.eclipse.jdt.core.formatter.brace_position_for_method_declaration=end_of_line
+org.eclipse.jdt.core.formatter.brace_position_for_switch=end_of_line
+org.eclipse.jdt.core.formatter.brace_position_for_type_declaration=end_of_line
+org.eclipse.jdt.core.formatter.comment.clear_blank_lines_in_block_comment=false
+org.eclipse.jdt.core.formatter.comment.clear_blank_lines_in_javadoc_comment=false
+org.eclipse.jdt.core.formatter.comment.format_block_comments=false
+org.eclipse.jdt.core.formatter.comment.format_header=false
+org.eclipse.jdt.core.formatter.comment.format_html=true
+org.eclipse.jdt.core.formatter.comment.format_javadoc_comments=false
+org.eclipse.jdt.core.formatter.comment.format_line_comments=false
+org.eclipse.jdt.core.formatter.comment.format_source_code=true
+org.eclipse.jdt.core.formatter.comment.indent_parameter_description=true
+org.eclipse.jdt.core.formatter.comment.indent_root_tags=true
+org.eclipse.jdt.core.formatter.comment.insert_new_line_before_root_tags=insert
+org.eclipse.jdt.core.formatter.comment.insert_new_line_for_parameter=insert
+org.eclipse.jdt.core.formatter.comment.line_length=80
+org.eclipse.jdt.core.formatter.comment.new_lines_at_block_boundaries=true
+org.eclipse.jdt.core.formatter.comment.new_lines_at_javadoc_boundaries=true
+org.eclipse.jdt.core.formatter.comment.preserve_white_space_between_code_and_line_comments=false
+org.eclipse.jdt.core.formatter.compact_else_if=true
+org.eclipse.jdt.core.formatter.continuation_indentation=2
+org.eclipse.jdt.core.formatter.continuation_indentation_for_array_initializer=2
+org.eclipse.jdt.core.formatter.disabling_tag=@formatter\:off
+org.eclipse.jdt.core.formatter.enabling_tag=@formatter\:on
+org.eclipse.jdt.core.formatter.format_guardian_clause_on_one_line=false
+org.eclipse.jdt.core.formatter.format_line_comment_starting_on_first_column=true
+org.eclipse.jdt.core.formatter.indent_body_declarations_compare_to_annotation_declaration_header=true
+org.eclipse.jdt.core.formatter.indent_body_declarations_compare_to_enum_constant_header=true
+org.eclipse.jdt.core.formatter.indent_body_declarations_compare_to_enum_declaration_header=true
+org.eclipse.jdt.core.formatter.indent_body_declarations_compare_to_type_header=true
+org.eclipse.jdt.core.formatter.indent_breaks_compare_to_cases=true
+org.eclipse.jdt.core.formatter.indent_empty_lines=false
+org.eclipse.jdt.core.formatter.indent_statements_compare_to_block=true
+org.eclipse.jdt.core.formatter.indent_statements_compare_to_body=true
+org.eclipse.jdt.core.formatter.indent_switchstatements_compare_to_cases=true
+org.eclipse.jdt.core.formatter.indent_switchstatements_compare_to_switch=false
+org.eclipse.jdt.core.formatter.indentation.size=4
+org.eclipse.jdt.core.formatter.insert_new_line_after_annotation_on_field=insert
+org.eclipse.jdt.core.formatter.insert_new_line_after_annotation_on_local_variable=insert
+org.eclipse.jdt.core.formatter.insert_new_line_after_annotation_on_method=insert
+org.eclipse.jdt.core.formatter.insert_new_line_after_annotation_on_package=insert
+org.eclipse.jdt.core.formatter.insert_new_line_after_annotation_on_parameter=do not insert
+org.eclipse.jdt.core.formatter.insert_new_line_after_annotation_on_type=insert
+org.eclipse.jdt.core.formatter.insert_new_line_after_label=do not insert
+org.eclipse.jdt.core.formatter.insert_new_line_after_opening_brace_in_array_initializer=do not insert
+org.eclipse.jdt.core.formatter.insert_new_line_at_end_of_file_if_missing=do not insert
+org.eclipse.jdt.core.formatter.insert_new_line_before_catch_in_try_statement=do not insert
+org.eclipse.jdt.core.formatter.insert_new_line_before_closing_brace_in_array_initializer=do not insert
+org.eclipse.jdt.core.formatter.insert_new_line_before_else_in_if_statement=do not insert
+org.eclipse.jdt.core.formatter.insert_new_line_before_finally_in_try_statement=do not insert
+org.eclipse.jdt.core.formatter.insert_new_line_before_while_in_do_statement=do not insert
+org.eclipse.jdt.core.formatter.insert_new_line_in_empty_annotation_declaration=insert
+org.eclipse.jdt.core.formatter.insert_new_line_in_empty_anonymous_type_declaration=insert
+org.eclipse.jdt.core.formatter.insert_new_line_in_empty_block=insert
+org.eclipse.jdt.core.formatter.insert_new_line_in_empty_enum_constant=insert
+org.eclipse.jdt.core.formatter.insert_new_line_in_empty_enum_declaration=insert
+org.eclipse.jdt.core.formatter.insert_new_line_in_empty_method_body=insert
+org.eclipse.jdt.core.formatter.insert_new_line_in_empty_type_declaration=insert
+org.eclipse.jdt.core.formatter.insert_space_after_and_in_type_parameter=insert
+org.eclipse.jdt.core.formatter.insert_space_after_assignment_operator=insert
+org.eclipse.jdt.core.formatter.insert_space_after_at_in_annotation=do not insert
+org.eclipse.jdt.core.formatter.insert_space_after_at_in_annotation_type_declaration=do not insert
+org.eclipse.jdt.core.formatter.insert_space_after_binary_operator=insert
+org.eclipse.jdt.core.formatter.insert_space_after_closing_angle_bracket_in_type_arguments=insert
+org.eclipse.jdt.core.formatter.insert_space_after_closing_angle_bracket_in_type_parameters=insert
+org.eclipse.jdt.core.formatter.insert_space_after_closing_brace_in_block=insert
+org.eclipse.jdt.core.formatter.insert_space_after_closing_paren_in_cast=insert
+org.eclipse.jdt.core.formatter.insert_space_after_colon_in_assert=insert
+org.eclipse.jdt.core.formatter.insert_space_after_colon_in_case=insert
+org.eclipse.jdt.core.formatter.insert_space_after_colon_in_conditional=insert
+org.eclipse.jdt.core.formatter.insert_space_after_colon_in_for=insert
+org.eclipse.jdt.core.formatter.insert_space_after_colon_in_labeled_statement=insert
+org.eclipse.jdt.core.formatter.insert_space_after_comma_in_allocation_expression=insert
+org.eclipse.jdt.core.formatter.insert_space_after_comma_in_annotation=insert
+org.eclipse.jdt.core.formatter.insert_space_after_comma_in_array_initializer=insert
+org.eclipse.jdt.core.formatter.insert_space_after_comma_in_constructor_declaration_parameters=insert
+org.eclipse.jdt.core.formatter.insert_space_after_comma_in_constructor_declaration_throws=insert
+org.eclipse.jdt.core.formatter.insert_space_after_comma_in_enum_constant_arguments=insert
+org.eclipse.jdt.core.formatter.insert_space_after_comma_in_enum_declarations=insert
+org.eclipse.jdt.core.formatter.insert_space_after_comma_in_explicitconstructorcall_arguments=insert
+org.eclipse.jdt.core.formatter.insert_space_after_comma_in_for_increments=insert
+org.eclipse.jdt.core.formatter.insert_space_after_comma_in_for_inits=insert
+org.eclipse.jdt.core.formatter.insert_space_after_comma_in_method_declaration_parameters=insert
+org.eclipse.jdt.core.formatter.insert_space_after_comma_in_method_declaration_throws=insert
+org.eclipse.jdt.core.formatter.insert_space_after_comma_in_method_invocation_arguments=insert
+org.eclipse.jdt.core.formatter.insert_space_after_comma_in_multiple_field_declarations=insert
+org.eclipse.jdt.core.formatter.insert_space_after_comma_in_multiple_local_declarations=insert
+org.eclipse.jdt.core.formatter.insert_space_after_comma_in_parameterized_type_reference=insert
+org.eclipse.jdt.core.formatter.insert_space_after_comma_in_superinterfaces=insert
+org.eclipse.jdt.core.formatter.insert_space_after_comma_in_type_arguments=insert
+org.eclipse.jdt.core.formatter.insert_space_after_comma_in_type_parameters=insert
+org.eclipse.jdt.core.formatter.insert_space_after_ellipsis=insert
+org.eclipse.jdt.core.formatter.insert_space_after_opening_angle_bracket_in_parameterized_type_reference=do not insert
+org.eclipse.jdt.core.formatter.insert_space_after_opening_angle_bracket_in_type_arguments=do not insert
+org.eclipse.jdt.core.formatter.insert_space_after_opening_angle_bracket_in_type_parameters=do not insert
+org.eclipse.jdt.core.formatter.insert_space_after_opening_brace_in_array_initializer=insert
+org.eclipse.jdt.core.formatter.insert_space_after_opening_bracket_in_array_allocation_expression=do not insert
+org.eclipse.jdt.core.formatter.insert_space_after_opening_bracket_in_array_reference=do not insert
+org.eclipse.jdt.core.formatter.insert_space_after_opening_paren_in_annotation=do not insert
+org.eclipse.jdt.core.formatter.insert_space_after_opening_paren_in_cast=do not insert
+org.eclipse.jdt.core.formatter.insert_space_after_opening_paren_in_catch=do not insert
+org.eclipse.jdt.core.formatter.insert_space_after_opening_paren_in_constructor_declaration=do not insert
+org.eclipse.jdt.core.formatter.insert_space_after_opening_paren_in_enum_constant=do not insert
+org.eclipse.jdt.core.formatter.insert_space_after_opening_paren_in_for=do not insert
+org.eclipse.jdt.core.formatter.insert_space_after_opening_paren_in_if=do not insert
+org.eclipse.jdt.core.formatter.insert_space_after_opening_paren_in_method_declaration=do not insert
+org.eclipse.jdt.core.formatter.insert_space_after_opening_paren_in_method_invocation=do not insert
+org.eclipse.jdt.core.formatter.insert_space_after_opening_paren_in_parenthesized_expression=do not insert
+org.eclipse.jdt.core.formatter.insert_space_after_opening_paren_in_switch=do not insert
+org.eclipse.jdt.core.formatter.insert_space_after_opening_paren_in_synchronized=do not insert
+org.eclipse.jdt.core.formatter.insert_space_after_opening_paren_in_try=do not insert
+org.eclipse.jdt.core.formatter.insert_space_after_opening_paren_in_while=do not insert
+org.eclipse.jdt.core.formatter.insert_space_after_postfix_operator=do not insert
+org.eclipse.jdt.core.formatter.insert_space_after_prefix_operator=do not insert
+org.eclipse.jdt.core.formatter.insert_space_after_question_in_conditional=insert
+org.eclipse.jdt.core.formatter.insert_space_after_question_in_wildcard=do not insert
+org.eclipse.jdt.core.formatter.insert_space_after_semicolon_in_for=insert
+org.eclipse.jdt.core.formatter.insert_space_after_semicolon_in_try_resources=insert
+org.eclipse.jdt.core.formatter.insert_space_after_unary_operator=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_and_in_type_parameter=insert
+org.eclipse.jdt.core.formatter.insert_space_before_assignment_operator=insert
+org.eclipse.jdt.core.formatter.insert_space_before_at_in_annotation_type_declaration=insert
+org.eclipse.jdt.core.formatter.insert_space_before_binary_operator=insert
+org.eclipse.jdt.core.formatter.insert_space_before_closing_angle_bracket_in_parameterized_type_reference=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_closing_angle_bracket_in_type_arguments=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_closing_angle_bracket_in_type_parameters=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_closing_brace_in_array_initializer=insert
+org.eclipse.jdt.core.formatter.insert_space_before_closing_bracket_in_array_allocation_expression=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_closing_bracket_in_array_reference=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_closing_paren_in_annotation=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_closing_paren_in_cast=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_closing_paren_in_catch=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_closing_paren_in_constructor_declaration=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_closing_paren_in_enum_constant=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_closing_paren_in_for=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_closing_paren_in_if=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_closing_paren_in_method_declaration=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_closing_paren_in_method_invocation=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_closing_paren_in_parenthesized_expression=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_closing_paren_in_switch=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_closing_paren_in_synchronized=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_closing_paren_in_try=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_closing_paren_in_while=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_colon_in_assert=insert
+org.eclipse.jdt.core.formatter.insert_space_before_colon_in_case=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_colon_in_conditional=insert
+org.eclipse.jdt.core.formatter.insert_space_before_colon_in_default=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_colon_in_for=insert
+org.eclipse.jdt.core.formatter.insert_space_before_colon_in_labeled_statement=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_comma_in_allocation_expression=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_comma_in_annotation=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_comma_in_array_initializer=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_comma_in_constructor_declaration_parameters=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_comma_in_constructor_declaration_throws=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_comma_in_enum_constant_arguments=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_comma_in_enum_declarations=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_comma_in_explicitconstructorcall_arguments=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_comma_in_for_increments=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_comma_in_for_inits=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_comma_in_method_declaration_parameters=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_comma_in_method_declaration_throws=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_comma_in_method_invocation_arguments=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_comma_in_multiple_field_declarations=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_comma_in_multiple_local_declarations=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_comma_in_parameterized_type_reference=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_comma_in_superinterfaces=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_comma_in_type_arguments=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_comma_in_type_parameters=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_ellipsis=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_opening_angle_bracket_in_parameterized_type_reference=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_opening_angle_bracket_in_type_arguments=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_opening_angle_bracket_in_type_parameters=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_opening_brace_in_annotation_type_declaration=insert
+org.eclipse.jdt.core.formatter.insert_space_before_opening_brace_in_anonymous_type_declaration=insert
+org.eclipse.jdt.core.formatter.insert_space_before_opening_brace_in_array_initializer=insert
+org.eclipse.jdt.core.formatter.insert_space_before_opening_brace_in_block=insert
+org.eclipse.jdt.core.formatter.insert_space_before_opening_brace_in_constructor_declaration=insert
+org.eclipse.jdt.core.formatter.insert_space_before_opening_brace_in_enum_constant=insert
+org.eclipse.jdt.core.formatter.insert_space_before_opening_brace_in_enum_declaration=insert
+org.eclipse.jdt.core.formatter.insert_space_before_opening_brace_in_method_declaration=insert
+org.eclipse.jdt.core.formatter.insert_space_before_opening_brace_in_switch=insert
+org.eclipse.jdt.core.formatter.insert_space_before_opening_brace_in_type_declaration=insert
+org.eclipse.jdt.core.formatter.insert_space_before_opening_bracket_in_array_allocation_expression=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_opening_bracket_in_array_reference=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_opening_bracket_in_array_type_reference=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_opening_paren_in_annotation=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_opening_paren_in_annotation_type_member_declaration=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_opening_paren_in_catch=insert
+org.eclipse.jdt.core.formatter.insert_space_before_opening_paren_in_constructor_declaration=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_opening_paren_in_enum_constant=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_opening_paren_in_for=insert
+org.eclipse.jdt.core.formatter.insert_space_before_opening_paren_in_if=insert
+org.eclipse.jdt.core.formatter.insert_space_before_opening_paren_in_method_declaration=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_opening_paren_in_method_invocation=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_opening_paren_in_parenthesized_expression=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_opening_paren_in_switch=insert
+org.eclipse.jdt.core.formatter.insert_space_before_opening_paren_in_synchronized=insert
+org.eclipse.jdt.core.formatter.insert_space_before_opening_paren_in_try=insert
+org.eclipse.jdt.core.formatter.insert_space_before_opening_paren_in_while=insert
+org.eclipse.jdt.core.formatter.insert_space_before_parenthesized_expression_in_return=insert
+org.eclipse.jdt.core.formatter.insert_space_before_parenthesized_expression_in_throw=insert
+org.eclipse.jdt.core.formatter.insert_space_before_postfix_operator=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_prefix_operator=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_question_in_conditional=insert
+org.eclipse.jdt.core.formatter.insert_space_before_question_in_wildcard=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_semicolon=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_semicolon_in_for=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_semicolon_in_try_resources=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_unary_operator=do not insert
+org.eclipse.jdt.core.formatter.insert_space_between_brackets_in_array_type_reference=do not insert
+org.eclipse.jdt.core.formatter.insert_space_between_empty_braces_in_array_initializer=do not insert
+org.eclipse.jdt.core.formatter.insert_space_between_empty_brackets_in_array_allocation_expression=do not insert
+org.eclipse.jdt.core.formatter.insert_space_between_empty_parens_in_annotation_type_member_declaration=do not insert
+org.eclipse.jdt.core.formatter.insert_space_between_empty_parens_in_constructor_declaration=do not insert
+org.eclipse.jdt.core.formatter.insert_space_between_empty_parens_in_enum_constant=do not insert
+org.eclipse.jdt.core.formatter.insert_space_between_empty_parens_in_method_declaration=do not insert
+org.eclipse.jdt.core.formatter.insert_space_between_empty_parens_in_method_invocation=do not insert
+org.eclipse.jdt.core.formatter.join_lines_in_comments=true
+org.eclipse.jdt.core.formatter.join_wrapped_lines=true
+org.eclipse.jdt.core.formatter.keep_else_statement_on_same_line=false
+org.eclipse.jdt.core.formatter.keep_empty_array_initializer_on_one_line=false
+org.eclipse.jdt.core.formatter.keep_imple_if_on_one_line=false
+org.eclipse.jdt.core.formatter.keep_then_statement_on_same_line=false
+org.eclipse.jdt.core.formatter.lineSplit=999
+org.eclipse.jdt.core.formatter.never_indent_block_comments_on_first_column=false
+org.eclipse.jdt.core.formatter.never_indent_line_comments_on_first_column=false
+org.eclipse.jdt.core.formatter.number_of_blank_lines_at_beginning_of_method_body=0
+org.eclipse.jdt.core.formatter.number_of_empty_lines_to_preserve=1
+org.eclipse.jdt.core.formatter.put_empty_statement_on_new_line=true
+org.eclipse.jdt.core.formatter.tabulation.char=space
+org.eclipse.jdt.core.formatter.tabulation.size=4
+org.eclipse.jdt.core.formatter.use_on_off_tags=false
+org.eclipse.jdt.core.formatter.use_tabs_only_for_leading_indentations=false
+org.eclipse.jdt.core.formatter.wrap_before_binary_operator=true
+org.eclipse.jdt.core.formatter.wrap_before_or_operator_multicatch=true
+org.eclipse.jdt.core.formatter.wrap_outer_expressions_when_nested=true

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/atopcalcite/.settings/org.eclipse.jdt.ui.prefs
----------------------------------------------------------------------
diff --git a/atopcalcite/.settings/org.eclipse.jdt.ui.prefs b/atopcalcite/.settings/org.eclipse.jdt.ui.prefs
new file mode 100644
index 0000000..681b0b0
--- /dev/null
+++ b/atopcalcite/.settings/org.eclipse.jdt.ui.prefs
@@ -0,0 +1,7 @@
+eclipse.preferences.version=1
+formatter_profile=_Space Indent & Long Lines
+formatter_settings_version=12
+org.eclipse.jdt.ui.ignorelowercasenames=true
+org.eclipse.jdt.ui.importorder=java;javax;org;com;
+org.eclipse.jdt.ui.ondemandthreshold=99
+org.eclipse.jdt.ui.staticondemandthreshold=1

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/atopcalcite/pom.xml
----------------------------------------------------------------------
diff --git a/atopcalcite/pom.xml b/atopcalcite/pom.xml
new file mode 100644
index 0000000..e36f434
--- /dev/null
+++ b/atopcalcite/pom.xml
@@ -0,0 +1,28 @@
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+    <modelVersion>4.0.0</modelVersion>
+
+    <artifactId>atopcalcite</artifactId>
+    <packaging>jar</packaging>
+    <name>Kylin:AtopCalcite</name>
+
+    <parent>
+        <groupId>com.kylinolap</groupId>
+        <artifactId>kylin</artifactId>
+        <version>0.6.3-SNAPSHOT</version>
+    </parent>
+
+    <properties>
+    </properties>
+    
+    <dependencies>
+        <dependency>
+            <groupId>org.apache.calcite</groupId>
+            <artifactId>calcite-core</artifactId>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.calcite</groupId>
+            <artifactId>calcite-avatica</artifactId>
+        </dependency>
+    </dependencies>
+
+</project>


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

Posted by lu...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/dictionary/src/test/resources/dict/eng_com.dic
----------------------------------------------------------------------
diff --git a/dictionary/src/test/resources/dict/eng_com.dic b/dictionary/src/test/resources/dict/eng_com.dic
new file mode 100644
index 0000000..786c8a4
--- /dev/null
+++ b/dictionary/src/test/resources/dict/eng_com.dic
@@ -0,0 +1,150843 @@
+aa
+ac
+ad
+ag
+ah
+ai
+ak
+al
+am
+an
+ap
+ar
+as
+at
+au
+ax
+ay
+az
+ba
+bc
+bd
+be
+bi
+bk
+bo
+bp
+bs
+bx
+by
+ca
+cb
+cc
+cd
+ce
+ci
+cm
+co
+cs
+ct
+cu
+cv
+cy
+cz
+da
+db
+dc
+de
+dj
+dl
+do
+dr
+dt
+du
+dv
+dx
+dy
+dz
+ea
+ed
+eg
+eh
+el
+em
+en
+er
+es
+et
+eu
+ex
+fa
+fc
+fd
+fe
+ff
+fl
+fm
+fr
+fs
+ft
+fu
+ga
+gb
+ge
+gi
+gm
+go
+gp
+gu
+ha
+he
+hi
+ho
+hp
+hq
+hr
+hz
+ia
+ic
+id
+ie
+if
+ii
+il
+in
+ip
+iq
+ir
+is
+it
+iv
+ix
+jr
+kc
+kg
+kl
+km
+ko
+ks
+ku
+kw
+ky
+la
+lb
+li
+ll
+ln
+lo
+lp
+lt
+lu
+lw
+ma
+mc
+md
+me
+mi
+mo
+mr
+ms
+mu
+my
+na
+nb
+nc
+nd
+ne
+nf
+nh
+ni
+nj
+nm
+no
+np
+nr
+nt
+nu
+nv
+nw
+ny
+od
+of
+oh
+ok
+on
+op
+or
+os
+ow
+ox
+oz
+pa
+pc
+pd
+pe
+pg
+ph
+pi
+pj
+pl
+pm
+po
+pp
+pq
+pr
+ps
+pt
+pu
+px
+qt
+ra
+rd
+re
+rh
+rn
+ro
+rr
+rt
+rv
+rx
+sa
+sc
+sd
+se
+sf
+sh
+so
+sq
+st
+sw
+ta
+tb
+ti
+tn
+to
+tt
+tv
+tx
+ua
+uh
+uk
+un
+up
+us
+ut
+uv
+va
+vc
+vd
+vi
+vp
+vt
+wa
+we
+wi
+wp
+wt
+wv
+wy
+xi
+xs
+xt
+xv
+xx
+yd
+ye
+yo
+yr
+zn
+abc
+abe
+ace
+act
+add
+ado
+aft
+age
+ago
+aha
+aid
+ail
+aim
+air
+alb
+ale
+ali
+all
+alm
+alp
+alt
+ama
+ami
+amp
+amt
+amu
+amy
+ana
+and
+ani
+ann
+ant
+any
+ape
+apo
+apt
+arc
+are
+ark
+arm
+art
+ash
+ask
+asp
+ass
+ate
+aug
+auk
+aut
+ave
+awe
+awl
+awn
+axe
+aye
+baa
+bad
+bag
+bah
+ban
+bar
+bas
+bat
+bay
+bed
+bee
+beg
+ben
+bet
+bey
+bib
+bid
+big
+bin
+bio
+bit
+boa
+bob
+bog
+bon
+boo
+bop
+bow
+box
+boy
+bra
+bro
+bud
+bug
+bum
+bun
+bur
+bus
+but
+buy
+bye
+byu
+cab
+cad
+cam
+can
+cap
+car
+cat
+caw
+cay
+ceo
+chi
+cio
+cit
+cob
+cod
+cog
+col
+com
+con
+coo
+cop
+cot
+cow
+coy
+cpu
+crt
+cry
+cub
+cud
+cue
+cul
+cum
+cup
+cur
+cut
+dab
+dad
+dam
+dan
+day
+deb
+dec
+def
+dei
+del
+den
+des
+dew
+did
+die
+dig
+dim
+din
+dip
+dir
+dna
+doa
+doc
+doe
+dog
+don
+dos
+dot
+dow
+doz
+dry
+dub
+dud
+due
+dug
+dun
+duo
+dup
+dye
+ear
+eat
+ebb
+ecg
+eeg
+eel
+eft
+egg
+ego
+eke
+elf
+elk
+ell
+elm
+emu
+enc
+end
+ene
+eon
+epa
+era
+ere
+erg
+ese
+esp
+est
+eta
+etc
+eve
+ewe
+ext
+eye
+fad
+fag
+fan
+far
+fat
+fax
+fay
+fee
+fen
+few
+fey
+fez
+fib
+fie
+fig
+fin
+fir
+fit
+fix
+fly
+fob
+foe
+fog
+fop
+for
+fox
+fri
+fry
+fun
+fur
+gab
+gad
+gag
+gal
+gam
+gao
+gap
+gar
+gas
+gay
+gee
+gel
+gem
+gen
+get
+gig
+gin
+gnu
+gob
+god
+gop
+got
+gov
+gum
+gun
+gut
+guy
+gym
+gyp
+hac
+had
+hag
+hah
+ham
+hap
+has
+hat
+haw
+hay
+hem
+hen
+hep
+her
+hew
+hex
+hey
+hid
+hie
+him
+hip
+his
+hit
+hob
+hoc
+hod
+hoe
+hog
+hon
+hop
+hot
+how
+hub
+hue
+hug
+huh
+hum
+hun
+hut
+ice
+icy
+ida
+ifs
+ilk
+ill
+inc
+ind
+ink
+inn
+ins
+int
+ion
+iou
+ire
+irk
+irs
+ism
+its
+iud
+ivs
+ivy
+iwo
+jab
+jag
+jam
+jan
+jap
+jar
+jaw
+jay
+jet
+jew
+jib
+jig
+jim
+job
+joe
+jog
+jot
+joy
+jug
+jul
+jun
+jus
+jut
+kat
+keg
+ken
+key
+kid
+kin
+kip
+kit
+lab
+lad
+lag
+lam
+lap
+las
+lat
+law
+lax
+lay
+led
+lee
+leg
+lei
+lek
+leo
+let
+leu
+lex
+lib
+lid
+lie
+lip
+lit
+liz
+lob
+log
+lop
+los
+lot
+low
+lox
+lug
+lye
+mad
+mae
+mag
+maj
+mal
+man
+mao
+map
+mar
+mat
+maw
+max
+may
+med
+meg
+men
+met
+mew
+mia
+mid
+mig
+mil
+min
+mit
+mix
+mob
+mod
+moi
+mom
+mon
+moo
+mop
+mow
+mrs
+mud
+mug
+mum
+nab
+nag
+nap
+nay
+nea
+neb
+net
+nev
+new
+nib
+nil
+nip
+nit
+nix
+nod
+nog
+nol
+non
+nor
+nos
+not
+nov
+now
+nub
+nun
+nut
+nyc
+oaf
+oak
+oar
+oat
+obi
+odd
+ode
+oem
+off
+oft
+ohm
+ohs
+oil
+old
+ole
+one
+ont
+opt
+orb
+ore
+oui
+our
+out
+ova
+owe
+owl
+own
+pad
+pal
+pan
+pap
+par
+pas
+pat
+paw
+pay
+pea
+peg
+pen
+pep
+per
+pet
+pew
+phi
+pie
+pig
+pin
+pip
+pit
+pix
+plo
+ply
+pob
+pod
+poe
+poi
+pop
+pos
+pot
+pow
+pox
+pre
+pro
+pry
+psi
+pub
+pug
+pun
+pup
+pus
+put
+pvc
+qat
+qed
+que
+qui
+quo
+rad
+raf
+rag
+rah
+ram
+ran
+rap
+rat
+raw
+ray
+rec
+red
+ref
+reg
+rei
+rem
+reo
+rep
+res
+ret
+rev
+rex
+rho
+rib
+rid
+rig
+rim
+rio
+rip
+rna
+rob
+rod
+roe
+rom
+rot
+row
+rub
+rue
+rug
+rum
+run
+rut
+rya
+rye
+sac
+sad
+sag
+sam
+san
+sao
+sap
+sat
+saw
+sax
+say
+sci
+sea
+sec
+see
+sen
+sep
+seq
+set
+sew
+sex
+she
+shy
+sic
+sid
+sin
+sip
+sir
+sis
+sit
+six
+ski
+sky
+sly
+sob
+sod
+sol
+son
+sop
+sos
+sot
+sow
+sox
+soy
+spa
+spy
+sri
+sty
+sub
+sue
+sui
+sum
+sun
+sup
+tab
+tad
+tag
+taj
+tan
+tao
+tap
+tar
+tat
+tau
+taw
+tax
+tay
+tea
+ted
+tee
+tel
+ten
+tex
+the
+tho
+thy
+tic
+tie
+tim
+tin
+tip
+tit
+tnt
+toe
+tog
+tom
+ton
+too
+top
+tot
+tov
+tow
+toy
+try
+tub
+tue
+tug
+tux
+tvs
+two
+ubi
+ufo
+ugh
+uke
+ump
+una
+uno
+upi
+ups
+urb
+urn
+usa
+use
+uso
+uss
+ute
+vag
+val
+van
+vas
+vat
+vet
+vex
+via
+vie
+vim
+vip
+vis
+viz
+vol
+von
+vow
+vox
+wac
+wad
+wag
+wan
+war
+was
+wax
+way
+web
+wed
+wee
+wet
+who
+why
+wig
+win
+wit
+woe
+wok
+won
+woo
+wow
+wry
+yak
+yam
+yap
+yaw
+yds
+yea
+yen
+yes
+yet
+yew
+yin
+yip
+yod
+yom
+yon
+you
+yrs
+ytd
+zap
+zed
+zen
+zip
+zit
+zoo
+aaas
+abac
+abba
+abbe
+abbr
+abby
+abcs
+abed
+abeg
+abet
+able
+ably
+abri
+abut
+acct
+aced
+aces
+ache
+achy
+acid
+aclu
+acme
+acne
+acre
+acta
+acto
+acts
+acyl
+adam
+adar
+adds
+aden
+adit
+advt
+adze
+aeon
+aero
+aery
+afar
+afro
+afto
+agar
+aged
+ager
+ages
+agio
+agog
+ague
+ahem
+ahoy
+aide
+aids
+aiel
+ails
+aims
+ainu
+airs
+airy
+ajar
+akin
+alae
+alan
+alar
+alas
+alba
+alec
+alee
+ales
+alex
+alga
+alia
+alii
+alit
+alls
+ally
+alma
+alms
+aloe
+alon
+alow
+alps
+also
+alta
+alto
+alum
+amah
+ambo
+amen
+amex
+amid
+ammo
+amok
+amon
+amos
+amps
+amyl
+anal
+andy
+anew
+ankh
+anna
+anne
+anno
+anoa
+anon
+ansa
+anta
+ante
+anti
+ants
+anum
+anus
+apdu
+aped
+apes
+apex
+apse
+aqua
+arab
+arak
+aral
+aran
+arch
+arck
+arcs
+area
+areg
+ares
+argo
+aria
+arid
+aril
+ariz
+arks
+arms
+army
+arse
+arts
+arty
+arum
+aryl
+asap
+asce
+asci
+ases
+ashy
+asia
+asks
+asme
+assn
+asst
+aste
+asti
+astr
+atom
+atop
+attn
+atty
+auks
+aula
+auld
+aunt
+aura
+auth
+auto
+aver
+avid
+avis
+aviv
+avon
+avow
+away
+awed
+awes
+awls
+awns
+awol
+awry
+axal
+axed
+axel
+axer
+axes
+axil
+axis
+axle
+axon
+ayah
+ayes
+azan
+azar
+baal
+baas
+baba
+babe
+babs
+babu
+baby
+bach
+back
+bade
+bags
+baht
+bail
+bait
+bake
+bald
+bale
+bali
+balk
+ball
+balm
+banc
+band
+bane
+bang
+bank
+bans
+bant
+baps
+bapt
+barb
+bard
+bare
+barf
+bark
+barm
+barn
+baro
+barr
+bars
+bart
+base
+bash
+bask
+bass
+bast
+bate
+bath
+bats
+batt
+baud
+bawd
+bawl
+bays
+bdrm
+bead
+beak
+beam
+bean
+bear
+beat
+beau
+beck
+beds
+beeb
+beef
+been
+beep
+beer
+bees
+beet
+begs
+belg
+bell
+belt
+bely
+bema
+bend
+bene
+beng
+bent
+benz
+berg
+berk
+berm
+bern
+bert
+bess
+best
+beta
+bete
+beth
+bets
+betz
+bevy
+beys
+bhan
+bias
+bibs
+bice
+bide
+bids
+biel
+bier
+biff
+bike
+bile
+bilk
+bill
+bind
+bine
+bing
+bins
+biog
+bioi
+biol
+bios
+bird
+biro
+bise
+bisk
+bite
+bits
+bitt
+blab
+blah
+bldg
+bleb
+bled
+blet
+blew
+blip
+blob
+bloc
+blot
+blow
+blub
+blue
+blur
+blvd
+boac
+boar
+boas
+boat
+bobs
+bock
+bode
+bodo
+body
+boer
+bogs
+bogy
+bohr
+boil
+bois
+bola
+bold
+bole
+boll
+bolo
+bolt
+bomb
+bona
+bond
+bone
+bong
+boni
+bonn
+bono
+bons
+bony
+boob
+book
+boom
+boon
+boor
+boos
+boot
+bops
+bora
+bore
+borg
+born
+bort
+bosh
+bosk
+boss
+bote
+both
+bout
+bowl
+bows
+boys
+brab
+brad
+brae
+brag
+bran
+bras
+brat
+braw
+bray
+bred
+brei
+bren
+brew
+bric
+brie
+brig
+brim
+brio
+brit
+brix
+bros
+brow
+brut
+bubo
+buck
+buds
+buff
+bugs
+bulb
+bulg
+bulk
+bull
+buln
+bump
+bums
+buna
+bung
+bunk
+buns
+bunt
+buoy
+burg
+burk
+burl
+burn
+burp
+burr
+burs
+bury
+bush
+busk
+buss
+bust
+busy
+butt
+buys
+buzz
+byes
+byre
+byte
+cabs
+cade
+cads
+cafe
+caff
+caf�
+cage
+cagy
+cain
+cake
+caky
+calc
+calf
+calk
+call
+calm
+calx
+camb
+came
+camp
+cams
+cane
+cans
+cant
+cape
+capo
+caps
+capt
+card
+care
+carl
+carp
+cars
+cart
+case
+cash
+cask
+cast
+casu
+cats
+caul
+cava
+cave
+cavy
+caws
+cays
+ceca
+cede
+ceil
+cell
+celt
+cent
+cera
+cere
+cert
+cess
+cete
+chad
+chap
+char
+chat
+chef
+chem
+chew
+chez
+chic
+chid
+chin
+chip
+chit
+chiv
+chop
+chow
+chub
+chug
+chum
+ciao
+cigs
+cill
+cine
+cion
+circ
+cire
+cist
+cite
+city
+clad
+clam
+clan
+clap
+claw
+clay
+clef
+cleg
+clem
+clew
+clip
+clod
+clog
+clop
+clot
+cloy
+club
+clue
+clun
+coal
+coat
+coax
+cobb
+cobs
+coca
+cock
+coco
+coda
+code
+cods
+coed
+cogs
+coho
+coif
+coil
+coin
+coir
+coke
+cola
+cold
+cole
+coli
+coll
+colo
+colt
+coly
+coma
+comb
+come
+comm
+comp
+conc
+cone
+conf
+cong
+coni
+conj
+conk
+conn
+cons
+cont
+cony
+cook
+cool
+coon
+coop
+coos
+coot
+cope
+cops
+copt
+copy
+cord
+core
+corf
+cork
+corm
+corn
+corp
+corr
+cors
+cosa
+cosh
+cost
+cosy
+cote
+coth
+cots
+coup
+cove
+cowl
+cows
+coxa
+cozy
+cpus
+crab
+crag
+cram
+cran
+crap
+craw
+cree
+cres
+crew
+crib
+cris
+crit
+crop
+crow
+crud
+crus
+crux
+ctge
+ctrl
+cuba
+cube
+cubs
+cuds
+cued
+cues
+cuff
+cull
+culm
+culs
+cult
+cunt
+cups
+curb
+curd
+cure
+curl
+curs
+curt
+cush
+cusp
+cuss
+cute
+cuts
+cyan
+cyma
+cyme
+cyst
+czar
+dabs
+dace
+dada
+dado
+dads
+daft
+dago
+dags
+dail
+dais
+dale
+dals
+dame
+damn
+damp
+dams
+dane
+dank
+daps
+dard
+dare
+dark
+darn
+dart
+dash
+data
+date
+datu
+daub
+davy
+dawn
+days
+daze
+dces
+dead
+deaf
+deal
+dean
+dear
+debs
+debt
+debu
+deck
+decl
+deco
+dedi
+deed
+deem
+deep
+deer
+dees
+defo
+deft
+defy
+deit
+deja
+dele
+delf
+deli
+dell
+deme
+demo
+demy
+dene
+dens
+dent
+deny
+dept
+derv
+desc
+desk
+deus
+deut
+deux
+deva
+dewy
+dhow
+dhss
+diad
+diag
+dial
+diam
+dias
+dibs
+dice
+dick
+dict
+dido
+died
+diem
+dies
+diet
+diff
+digs
+dike
+diks
+dill
+dime
+dims
+dine
+ding
+dink
+dins
+dint
+diol
+dipl
+dips
+dire
+dirk
+dirt
+disc
+dish
+disk
+dist
+diva
+dive
+divi
+dlit
+doch
+dock
+docs
+dodo
+doer
+does
+doff
+doge
+dogs
+dole
+doli
+doll
+dolo
+dolt
+dome
+dona
+done
+dong
+doni
+dons
+doom
+door
+dopa
+dope
+dorm
+dory
+dose
+doss
+dost
+dote
+doth
+dots
+doug
+dour
+dove
+down
+doxy
+doze
+dozy
+drab
+drag
+dram
+drat
+draw
+dray
+dreg
+drew
+drey
+drip
+drop
+drub
+drug
+drum
+drys
+dsto
+dt's
+duad
+dual
+dubs
+duce
+duck
+duct
+dude
+duds
+duel
+dues
+duet
+duff
+dugs
+duke
+dull
+duly
+dumb
+dump
+dune
+dung
+dunk
+duns
+dunt
+duos
+dupe
+dura
+dusk
+dust
+duty
+dyad
+dyak
+dyed
+dyer
+dyes
+dyke
+dyne
+each
+earl
+earn
+ears
+ease
+east
+easy
+eats
+eave
+ebbs
+ebon
+ecad
+echo
+econ
+ecru
+edam
+eddy
+eden
+edge
+edgy
+edit
+eels
+eely
+eery
+efts
+egad
+eggs
+eggy
+egos
+eire
+eked
+ekes
+elan
+elhi
+elks
+ells
+elms
+else
+emir
+emit
+emmy
+empt
+emus
+ends
+enol
+envy
+eons
+epee
+epha
+epic
+epos
+eras
+erev
+ergo
+ergs
+eric
+erie
+erin
+erne
+erns
+eros
+errs
+erse
+erst
+espy
+esse
+etch
+eton
+etui
+euro
+even
+ever
+eves
+evil
+ewer
+ewes
+exam
+exec
+exes
+exit
+exon
+expo
+eyas
+eyed
+eyer
+eyes
+eyot
+eyra
+eyre
+ezra
+face
+fact
+fade
+fado
+fads
+faec
+faff
+fags
+fahr
+fail
+fain
+fair
+fait
+fake
+fall
+falx
+fame
+fane
+fang
+fans
+fare
+farl
+farm
+faro
+fart
+fash
+fast
+fata
+fate
+fatl
+fats
+faun
+faut
+faux
+fawn
+fays
+faze
+fddi
+fdic
+fear
+feat
+feci
+feds
+feed
+feel
+fees
+feet
+fehl
+feil
+fell
+felo
+felt
+feme
+fend
+fens
+fern
+fess
+feta
+fete
+feud
+fiat
+fibs
+fica
+fide
+fido
+fids
+fief
+fife
+fifi
+fifo
+figs
+fiji
+fila
+file
+fill
+film
+fils
+find
+fine
+fink
+finn
+fins
+fire
+firm
+firs
+fisc
+fish
+fisk
+fist
+fits
+five
+fixe
+fizz
+flab
+flag
+flak
+flam
+flan
+flap
+flat
+flaw
+flax
+flay
+flea
+fled
+flee
+flem
+flew
+flex
+flip
+flit
+floc
+floe
+flog
+flop
+flor
+flow
+flub
+flue
+flun
+flux
+foal
+foam
+fobs
+foci
+foes
+fogs
+fogy
+fohn
+foie
+foil
+fold
+folk
+foll
+fond
+fons
+font
+food
+fool
+foot
+fops
+fora
+forb
+ford
+fore
+fori
+fork
+form
+foro
+fort
+foss
+foul
+four
+fowl
+foxy
+fran
+frap
+frau
+fray
+frcm
+frco
+frcp
+frcs
+fred
+free
+freq
+fret
+frgs
+frit
+friz
+froe
+frog
+from
+frow
+frps
+frsc
+fuck
+fuel
+fuga
+fuji
+full
+fume
+fumy
+fund
+funk
+furl
+furs
+fury
+fuse
+fuss
+fuze
+fuzz
+gabs
+gaby
+gads
+gael
+gaff
+gage
+gags
+gain
+gait
+gala
+gale
+gall
+gals
+gama
+game
+gamp
+gams
+gamy
+gang
+gant
+gaol
+gape
+gaps
+gapy
+garb
+gars
+gary
+gash
+gasp
+gate
+gats
+gaud
+gaul
+gaup
+gaur
+gave
+gawk
+gawp
+gays
+gaza
+gaze
+gean
+gear
+geek
+geld
+gels
+gelt
+gems
+gena
+gene
+gens
+gent
+genu
+geog
+geos
+germ
+gets
+geum
+ghat
+ghee
+gibe
+gibs
+gift
+gigs
+gila
+gild
+gill
+gilt
+gimp
+gink
+gins
+gios
+gips
+gird
+girl
+girn
+giro
+girt
+gist
+gits
+give
+glad
+glam
+gled
+glee
+glen
+gley
+glia
+glib
+glim
+glob
+glop
+glow
+glue
+glum
+glut
+gnat
+gnaw
+gnow
+gnus
+goad
+goal
+goas
+goat
+gobi
+gobo
+gobs
+goby
+gods
+goer
+goes
+gogh
+gogo
+gold
+golf
+gone
+gong
+good
+goof
+gook
+goon
+gore
+gory
+gosh
+goth
+goto
+gout
+govt
+gowk
+gown
+goys
+grab
+grad
+gram
+gras
+grav
+gray
+greg
+grep
+grew
+grey
+grid
+grig
+grim
+grin
+grip
+grit
+grog
+grot
+grow
+grub
+gual
+guam
+guan
+guck
+guff
+gulf
+gull
+gulp
+gums
+gung
+gunk
+guns
+guru
+gush
+gust
+guts
+guys
+gybe
+gyms
+gyps
+gyre
+gyri
+gyro
+gyve
+haaf
+haar
+habu
+hack
+hade
+hadj
+haec
+haem
+haft
+hags
+haik
+hail
+hair
+hajj
+haka
+hake
+hale
+half
+hall
+halm
+halo
+halt
+hame
+hamm
+hams
+hand
+hang
+hank
+hard
+hare
+hark
+harl
+harm
+harp
+hart
+hash
+hasp
+hast
+hate
+hath
+hats
+haul
+have
+hawk
+haws
+hays
+haze
+hazy
+he'd
+he's
+head
+heal
+heap
+hear
+heat
+heck
+heed
+heel
+heft
+heir
+held
+hele
+hell
+helm
+help
+heme
+hemp
+hems
+hens
+herb
+herd
+here
+herl
+herm
+hern
+hero
+herr
+hers
+heth
+hewn
+hews
+hick
+hide
+hied
+hies
+hifi
+high
+hike
+hila
+hili
+hill
+hilt
+hind
+hint
+hips
+hire
+hiss
+hist
+hits
+hive
+hoar
+hoax
+hobo
+hobs
+hock
+hods
+hoed
+hoer
+hoes
+hogs
+hoik
+hoke
+hold
+hole
+holm
+holt
+holy
+home
+homo
+homy
+hone
+hong
+honk
+hood
+hoof
+hook
+hoop
+hoot
+hope
+hopi
+hops
+hord
+horm
+horn
+hors
+hose
+hosp
+host
+hour
+hove
+howl
+hubs
+hued
+hues
+huff
+huge
+hugh
+hugo
+hugs
+hula
+hulk
+hull
+hump
+hums
+hung
+hunk
+huns
+hunt
+hurl
+hurt
+hush
+husk
+huss
+huts
+hyde
+hymn
+hype
+hypo
+i'll
+i've
+iaea
+iamb
+iata
+ibex
+ibid
+ibis
+ibos
+icao
+icbm
+iced
+ices
+icky
+icon
+icsh
+idea
+idee
+idem
+ideo
+ides
+idle
+idly
+idol
+ieee
+iffy
+ikon
+ilea
+ilex
+ilia
+ilks
+ills
+illy
+imam
+impf
+imps
+inca
+inch
+incr
+indo
+info
+inks
+inky
+inns
+insp
+inst
+intl
+into
+intr
+ions
+iota
+ious
+iowa
+ipsa
+ipse
+ipso
+iran
+iraq
+ires
+iris
+irks
+iron
+isbn
+isis
+isle
+isms
+it'd
+it's
+ital
+itch
+item
+iter
+itsy
+itty
+iuds
+iura
+jabs
+jack
+jade
+jags
+jail
+jain
+jake
+jamb
+jams
+jane
+jape
+jars
+jato
+java
+jaws
+jays
+jazz
+jean
+jeep
+jeer
+jell
+jerk
+jess
+jest
+jete
+jets
+jews
+jibe
+jibs
+jiff
+jigs
+jill
+jilt
+jima
+jimp
+jink
+jinn
+jinx
+jive
+jizz
+joan
+jobs
+jock
+joel
+joey
+jogs
+john
+joie
+join
+joke
+joky
+jolt
+jose
+josh
+joss
+jota
+jots
+jour
+jove
+jowl
+joys
+juan
+jube
+jude
+judo
+judy
+juga
+jugs
+juju
+july
+jump
+june
+junk
+juno
+jupe
+jura
+jure
+juri
+jury
+just
+jute
+juts
+kadi
+kagu
+kahn
+kaka
+kaki
+kala
+kale
+kama
+kame
+kana
+kans
+kant
+kaon
+kara
+kart
+kata
+kate
+kats
+kava
+kayo
+kcal
+keas
+keel
+keen
+keep
+kegs
+kelp
+kelt
+kemp
+keno
+kens
+kent
+kepi
+kept
+kerb
+kerf
+kern
+kerr
+kesh
+keto
+keys
+khan
+khat
+kibe
+kick
+kids
+kier
+kiev
+kill
+kiln
+kilo
+kilt
+kind
+kine
+king
+kink
+kino
+kips
+kirk
+kish
+kiss
+kite
+kith
+kits
+kiva
+kiwi
+klan
+klei
+klie
+klux
+knag
+knap
+knar
+knee
+knew
+knit
+knob
+knop
+knot
+know
+knox
+koan
+kobs
+koch
+koel
+kohl
+kola
+kong
+kook
+kops
+koso
+koto
+kris
+kroo
+kudu
+kung
+kurd
+kuru
+kwhr
+kyat
+kype
+labs
+lace
+lack
+lacy
+lade
+lads
+lady
+lags
+laic
+laid
+lain
+lair
+lait
+lake
+lakh
+laky
+lama
+lamb
+lame
+lamp
+lams
+land
+lane
+lang
+lank
+lans
+laos
+lapp
+laps
+lard
+lark
+lase
+lash
+lass
+last
+lata
+late
+lath
+lats
+laud
+lava
+lave
+lawn
+laws
+lays
+laze
+lazy
+lead
+leaf
+leak
+lean
+leap
+leas
+leat
+leek
+leer
+lees
+left
+lege
+legs
+lehr
+leio
+leis
+leks
+lema
+lend
+leno
+lens
+lent
+lenz
+leon
+leos
+lese
+less
+lest
+lets
+lett
+levi
+levy
+lewd
+li'l
+liar
+lias
+libs
+lice
+lich
+lick
+lido
+lids
+lied
+lief
+lien
+lies
+lieu
+life
+lifo
+lift
+like
+lilt
+lily
+lima
+limb
+lime
+limn
+limo
+limp
+limy
+line
+ling
+link
+lino
+lint
+liny
+lion
+lips
+lira
+lire
+lisp
+list
+lite
+lith
+live
+load
+loaf
+loam
+loan
+lobe
+lobi
+lobs
+loch
+loci
+lock
+loco
+lode
+loft
+loge
+logo
+logs
+logy
+loin
+loll
+lone
+long
+look
+loom
+loon
+loop
+loos
+loot
+lope
+lops
+lord
+lore
+lorn
+lory
+lose
+loss
+lost
+loth
+lots
+loud
+lour
+lout
+love
+lows
+luau
+lube
+luck
+lucy
+ludo
+luer
+lues
+luff
+luge
+lugs
+luke
+lull
+lump
+lune
+lung
+lure
+lurk
+lush
+lust
+lute
+lyes
+lyme
+lynx
+lyon
+lyra
+lyre
+lyse
+maar
+mace
+mach
+macy
+made
+mado
+magi
+mags
+maid
+mail
+maim
+main
+make
+mako
+mala
+male
+mali
+mall
+malm
+malo
+malt
+mama
+mana
+mane
+mann
+mans
+manu
+manx
+many
+maps
+mara
+marc
+mare
+mark
+marl
+mars
+mart
+marx
+mary
+mash
+mask
+mass
+mast
+mate
+math
+mats
+matt
+maud
+maul
+maws
+maxi
+maya
+mayo
+mays
+maze
+mazy
+mead
+meal
+mean
+meas
+meat
+meed
+meek
+meet
+mega
+mein
+meld
+mell
+melt
+memo
+mend
+mens
+menu
+meow
+mere
+mesa
+mesh
+mess
+mete
+mets
+meum
+mewl
+mews
+mhos
+mica
+mice
+mich
+midi
+mien
+miff
+migs
+mike
+mild
+mile
+milk
+mill
+mils
+milt
+mime
+mind
+mine
+ming
+minh
+mini
+mink
+minn
+mins
+mint
+minx
+mire
+mirv
+miry
+misc
+mise
+miss
+mist
+mite
+mitt
+moan
+moat
+mobs
+mock
+mode
+modi
+modo
+mods
+mohs
+moil
+mold
+mole
+moll
+molt
+moms
+monk
+mono
+mons
+mont
+mood
+moon
+moor
+moos
+moot
+mope
+mops
+mopy
+mora
+more
+morn
+moro
+mort
+moss
+most
+mota
+mote
+moth
+mots
+motu
+move
+mown
+mows
+moxa
+much
+muck
+muff
+mugs
+mule
+mull
+mums
+muon
+murk
+muse
+mush
+musk
+muss
+must
+mute
+mutt
+myna
+myth
+nabs
+naff
+nags
+naif
+nail
+naja
+name
+nana
+nape
+naps
+narc
+nard
+nark
+nary
+nasa
+nasi
+nata
+natl
+nato
+nave
+navy
+nays
+naze
+nazi
+neap
+near
+neat
+nebr
+nebs
+neck
+need
+neel
+neon
+nerd
+nero
+ness
+nest
+nets
+nett
+neve
+nevi
+newf
+news
+newt
+next
+nibs
+nice
+nick
+nidi
+nife
+niff
+nigh
+nile
+nina
+nine
+nips
+nisi
+nits
+nixe
+nixy
+noah
+nobs
+nock
+node
+nodi
+nods
+noel
+noes
+nogs
+noil
+noli
+nolo
+noma
+nome
+noms
+none
+nons
+nook
+noon
+nope
+norm
+nose
+nosh
+nosy
+nota
+note
+noth
+noun
+nous
+nova
+novo
+noxa
+nubs
+nuda
+nude
+nuke
+null
+numb
+nunc
+nuns
+nurs
+nuts
+o'er
+oafs
+oaks
+oars
+oast
+oath
+oats
+obex
+obey
+obis
+obit
+oboe
+oche
+octo
+od'd
+oddi
+odds
+odes
+odin
+oems
+offs
+ogee
+ogle
+ogre
+ohio
+ohms
+oils
+oily
+oink
+ok'd
+ok's
+okay
+okey
+okie
+okla
+okra
+okta
+olea
+oleo
+oles
+olid
+olio
+oman
+omen
+omit
+once
+ones
+only
+onto
+onus
+onyx
+oohs
+oops
+ooze
+oozy
+opah
+opal
+opec
+open
+opts
+opus
+orad
+orae
+oral
+orbs
+orem
+ores
+orgy
+orig
+oris
+orle
+orly
+orth
+oryx
+oslo
+ossa
+otic
+ouch
+ours
+oust
+outs
+ouzo
+oval
+oven
+over
+ovid
+ovum
+owed
+owes
+owls
+owns
+oxen
+oyer
+oyez
+paca
+pace
+pack
+pact
+pads
+page
+paid
+pail
+pain
+pair
+pais
+pale
+pall
+palm
+palp
+pals
+palt
+paly
+pand
+pane
+pang
+pans
+pant
+papa
+para
+pard
+pare
+pari
+park
+parr
+pars
+part
+pase
+paso
+pass
+past
+pate
+path
+pats
+paua
+paul
+pave
+pawl
+pawn
+paws
+pays
+peak
+peal
+pean
+pear
+peas
+peat
+peck
+pede
+peed
+peek
+peel
+peen
+peep
+peer
+pees
+pegs
+peip
+peke
+pele
+pelf
+pell
+pelt
+pend
+penh
+penn
+pens
+pent
+peon
+pepo
+peps
+pere
+peri
+perk
+perm
+pert
+peru
+peso
+pest
+pete
+pets
+pews
+phew
+phil
+phon
+phot
+phut
+phys
+pial
+pian
+pica
+pick
+pics
+pict
+pied
+pier
+pies
+pigs
+pika
+pike
+pile
+pili
+pill
+pimp
+pina
+pine
+ping
+pink
+pins
+pint
+piny
+pion
+pipe
+pips
+pisa
+pise
+pish
+piss
+pith
+pits
+pity
+pius
+pixy
+pkwy
+plan
+plat
+play
+plea
+pleb
+pled
+plie
+plod
+plop
+plot
+plow
+ploy
+plug
+plum
+plus
+pock
+poco
+pods
+poem
+poet
+pogo
+poke
+poky
+pole
+polk
+poll
+polo
+pols
+poly
+pome
+pomp
+pond
+pone
+pong
+pons
+pony
+pood
+pooh
+pool
+poop
+poor
+pope
+pops
+pore
+pori
+pork
+porn
+port
+pose
+posh
+poss
+post
+posy
+pots
+pott
+pour
+pout
+pows
+poxy
+pram
+pray
+prep
+pres
+prey
+prig
+prim
+prix
+prob
+proc
+prod
+prof
+prog
+prom
+pron
+prop
+pros
+prot
+prow
+prox
+psst
+ptas
+pubs
+puce
+puck
+pudu
+puff
+pugs
+puis
+puke
+puku
+pule
+pull
+pulp
+puma
+pump
+puna
+punk
+puns
+punt
+puny
+pupa
+pups
+pure
+purl
+purr
+push
+puss
+puts
+putt
+pyla
+pyre
+quad
+quae
+quag
+qual
+quam
+quat
+quay
+quem
+quer
+ques
+quia
+quid
+quin
+quip
+quit
+quiz
+quod
+quos
+raaf
+race
+rack
+racy
+raft
+raga
+rage
+rags
+raid
+rail
+rain
+raja
+rake
+rale
+rami
+ramp
+rams
+rand
+rang
+rani
+rank
+rant
+rape
+raps
+rapt
+rare
+rase
+rash
+rasp
+rata
+rate
+rats
+rave
+rays
+raze
+razz
+rbis
+rcmp
+rcpt
+read
+real
+ream
+reap
+rear
+rebs
+recd
+reck
+rect
+redd
+redo
+reds
+reed
+reef
+reek
+reel
+refs
+reil
+rein
+reit
+rely
+rems
+rend
+rene
+reno
+rent
+repp
+reps
+reqd
+resp
+rest
+retd
+rete
+rety
+reus
+revs
+reye
+rhea
+rhus
+rial
+rias
+ribs
+rica
+rice
+rich
+rick
+rico
+ride
+rids
+riel
+rife
+riff
+rift
+rigs
+rile
+rill
+rima
+rime
+rims
+rimu
+rimy
+rind
+ring
+rink
+riot
+ripe
+rips
+rise
+risk
+rite
+rive
+road
+roam
+roan
+roar
+robe
+robs
+rock
+rode
+rods
+roed
+roes
+roil
+role
+roll
+rome
+romp
+roms
+rood
+roof
+rook
+room
+roos
+root
+rope
+ropm
+ropy
+rosa
+rose
+rosh
+ross
+rosy
+rota
+rotc
+rote
+rotl
+rots
+roue
+roup
+rous
+rout
+roux
+rove
+rows
+rpcs
+rspb
+rspv
+rsvp
+rtls
+rtpm
+rtse
+rtus
+rube
+rubs
+ruby
+ruck
+rudd
+rude
+rued
+ruer
+rues
+ruff
+ruga
+rugs
+ruhr
+ruin
+rule
+rump
+rums
+rune
+rung
+runs
+runt
+ruse
+rush
+rusk
+rust
+ruth
+ruts
+ryas
+ryes
+ryot
+sack
+sacs
+safe
+saga
+sage
+sago
+sags
+said
+sail
+sake
+saki
+sale
+salk
+salt
+same
+sand
+sane
+sang
+sank
+sans
+sanz
+sapo
+saps
+sard
+sari
+sark
+sash
+sass
+sate
+sats
+saul
+save
+sawn
+saws
+saxe
+says
+scab
+scad
+scag
+scam
+scan
+scar
+scat
+scil
+scop
+scot
+scow
+scry
+scsi
+scud
+scum
+scup
+scut
+seal
+seam
+sear
+seas
+seat
+sech
+seck
+sect
+seed
+seek
+seel
+seem
+seen
+seep
+seer
+sees
+seif
+self
+sell
+seme
+semi
+send
+sent
+sept
+sera
+serb
+sere
+serf
+serg
+serv
+sese
+seta
+sets
+sewn
+sews
+sexe
+sext
+sexy
+shad
+shag
+shah
+sham
+shan
+shaw
+shay
+shea
+shed
+sher
+shes
+shew
+shih
+shim
+shin
+ship
+shit
+shiv
+shod
+shoe
+shoo
+shop
+shot
+show
+shun
+shut
+sial
+siam
+sice
+sick
+side
+sift
+sigh
+sign
+sikh
+sild
+silk
+sill
+silo
+silt
+sima
+sine
+sing
+sinh
+sink
+sinn
+sins
+sips
+sire
+sirs
+site
+sits
+situ
+sitz
+siwe
+size
+sizy
+skag
+skat
+skeg
+skep
+skew
+skid
+skim
+skin
+skip
+skis
+skit
+skua
+skye
+slab
+slad
+slag
+slam
+slap
+slat
+slav
+slaw
+slay
+sled
+slew
+slid
+slim
+slip
+slit
+slob
+sloe
+slog
+slop
+slot
+slow
+slub
+slue
+slug
+slum
+slur
+slut
+smew
+smog
+smug
+smut
+snag
+snap
+snib
+snip
+snob
+snot
+snow
+snub
+snug
+soak
+soap
+soar
+sobs
+sock
+soda
+sods
+sofa
+soft
+soho
+soia
+soil
+soir
+soit
+soja
+soke
+sola
+sold
+sole
+soli
+solo
+soma
+some
+sone
+song
+sons
+sook
+sool
+soon
+soot
+soph
+sops
+sorb
+sore
+sori
+sort
+soso
+sots
+soul
+soup
+sour
+sown
+sows
+soya
+spam
+span
+spar
+spas
+spat
+spay
+spca
+spcc
+spec
+sped
+spes
+spew
+spin
+spit
+spiv
+spot
+spri
+spry
+spud
+spue
+spun
+spur
+sqls
+ssts
+stab
+stag
+stan
+star
+stat
+stay
+stem
+sten
+step
+stet
+stew
+stir
+stoa
+stop
+stow
+stub
+stud
+stum
+stun
+stye
+styx
+subs
+such
+suck
+sudd
+suds
+sued
+suer
+sues
+suet
+suey
+suez
+suit
+sulk
+sumo
+sump
+sums
+sung
+sunk
+suns
+sups
+supt
+sura
+surd
+sure
+surf
+suus
+swab
+swag
+swam
+swan
+swap
+swat
+sway
+swed
+swig
+swim
+swob
+swop
+swot
+swum
+sync
+syne
+tabs
+tabu
+tach
+tack
+taco
+tact
+tads
+taft
+tags
+tail
+tain
+taka
+take
+talc
+tale
+tali
+talk
+tall
+tame
+tamp
+tams
+tana
+tang
+tanh
+tank
+tans
+tapa
+tape
+taps
+tapu
+tare
+tarn
+taro
+tarp
+tars
+tart
+task
+tass
+tats
+taut
+tawa
+taws
+taxa
+taxi
+taxy
+tbsp
+teak
+teal
+team
+tear
+teas
+teat
+tech
+teds
+teed
+teem
+teen
+tees
+tela
+tell
+temp
+tend
+tenn
+tens
+tent
+tepa
+tera
+term
+tern
+terr
+test
+tete
+text
+tgif
+thai
+than
+that
+thaw
+thee
+them
+then
+thew
+they
+thin
+this
+tho'
+thon
+thor
+thou
+thud
+thug
+thur
+thus
+tick
+tics
+tide
+tidy
+tied
+tiel
+tier
+ties
+tiff
+tiki
+tile
+till
+tilt
+time
+tine
+ting
+tins
+tint
+tiny
+tipi
+tips
+tire
+tiro
+titi
+tits
+tium
+tnpk
+toad
+toby
+toch
+todo
+tods
+tody
+toea
+toed
+toes
+toff
+toft
+tofu
+toga
+togo
+togs
+toil
+toke
+tola
+told
+toll
+tolu
+tomb
+tome
+toms
+tone
+tong
+tonk
+tons
+tony
+took
+tool
+toot
+tope
+topi
+topo
+tops
+torc
+tore
+tori
+torn
+torr
+tors
+tort
+tory
+tosh
+toss
+tote
+toto
+tots
+tour
+tout
+town
+tows
+toys
+trad
+tram
+trap
+tray
+tree
+trek
+tret
+trew
+trey
+trig
+trim
+trio
+trip
+trod
+tron
+trop
+trot
+trow
+troy
+true
+trug
+tsar
+tuan
+tuba
+tube
+tubo
+tubs
+tuck
+tues
+tufa
+tuff
+tuft
+tugs
+tuis
+tuna
+tunc
+tune
+tung
+turd
+turf
+turk
+turn
+tush
+tusk
+tutu
+tuum
+twan
+twas
+twee
+twig
+twin
+twit
+twos
+tyke
+type
+typo
+tyre
+tyro
+tzar
+uart
+ucla
+udal
+ufos
+ugly
+ulna
+umbo
+umps
+unci
+unco
+undo
+undy
+unit
+univ
+unix
+unto
+unum
+upon
+ural
+urbs
+urdu
+urea
+urge
+uria
+uric
+urim
+urns
+ursa
+usaf
+uscg
+usda
+used
+usee
+user
+uses
+usmc
+ussr
+usus
+utah
+uvea
+uxor
+uzbe
+vade
+vagi
+vail
+vain
+vale
+vamp
+vane
+vang
+vans
+vary
+vasa
+vase
+vast
+vats
+veal
+veda
+veep
+veer
+veil
+vein
+vela
+veld
+vena
+vend
+venn
+vent
+vera
+verb
+veri
+vert
+very
+vest
+veta
+veto
+vets
+viae
+vial
+vias
+vice
+vide
+vidi
+vied
+vier
+vies
+view
+viii
+viis
+vile
+vine
+viny
+viol
+vips
+visa
+vise
+vita
+viva
+vive
+vivo
+vlsi
+voce
+voge
+void
+voir
+voix
+vola
+vole
+volt
+vote
+vows
+vult
+wacs
+wade
+wadi
+wads
+waft
+wage
+wags
+waif
+wail
+wain
+wait
+wake
+wale
+walk
+wall
+walt
+wand
+wane
+wang
+want
+ward
+ware
+warm
+warn
+warp
+wars
+wart
+wary
+wash
+wasp
+wats
+watt
+wave
+wavy
+waxy
+ways
+wctu
+we'd
+weak
+weal
+wean
+wear
+webs
+weds
+weed
+week
+weep
+weer
+weft
+weil
+weir
+weld
+well
+welt
+wend
+wens
+went
+wept
+were
+wert
+west
+wets
+wham
+what
+when
+whet
+whew
+whey
+whig
+whim
+whin
+whip
+whir
+whit
+whiz
+whoa
+whom
+whop
+whys
+wick
+wide
+wife
+wigs
+wild
+wile
+will
+wilt
+wily
+wimp
+wind
+wine
+wing
+wink
+wino
+wins
+winy
+wipe
+wire
+wiry
+wise
+wish
+wisp
+with
+wits
+wive
+wkly
+woad
+woes
+woke
+woks
+wold
+wolf
+womb
+wont
+wood
+woof
+wool
+woos
+word
+wore
+work
+worm
+worn
+wort
+wove
+wows
+wrap
+wren
+writ
+wrnt
+wron
+wych
+wynd
+wyom
+xfig
+xiii
+xing
+xlib
+xmas
+xray
+xvii
+xyst
+yacc
+yack
+yair
+yaks
+yale
+yams
+yang
+yank
+yaps
+yard
+yarn
+yawl
+yawn
+yawp
+yaws
+yeah
+yean
+year
+yeas
+yegg
+yell
+yelp
+yens
+yeti
+yews
+yilt
+yips
+ylem
+ymca
+yobs
+yoga
+yogi
+yoke
+yolk
+yore
+york
+youn
+your
+yowl
+yuan
+yuck
+yule
+yurt
+ywca
+ywha
+zany
+zaps
+zeal
+zebu
+zein
+zero
+zest
+zeta
+zeus
+zinc
+zing
+zinn
+zion
+zips
+ziti
+zoea
+zoic
+zoll
+zona
+zone
+zonk
+zoom
+zoos
+zulu
+zuni
+aaron
+abaca
+abaci
+aback
+abaft
+abase
+abash
+abate
+abbes
+abbey
+abbot
+abeam
+abegg
+abets
+abhor
+abide
+abies
+abler
+abner
+abode
+aborn
+abort
+about
+above
+abrus
+abuse
+abuts
+abuzz
+abwab
+abysm
+abyss
+acari
+accts
+acerb
+aceta
+ached
+aches
+acids
+acidy
+acing
+acini
+ackee
+acmes
+acnae
+acned
+acnes
+acoin
+acorn
+acral
+acred
+acres
+acrid
+acted
+actin
+actio
+actor
+actus
+acute
+adage
+adams
+adapt
+added
+adder
+addle
+adeem
+adeps
+adept
+adieu
+adios
+adlib
+adman
+admen
+admin
+admit
+admix
+adobe
+adolf
+adopt
+adore
+adorn
+aduki
+adult
+adzes
+aegis
+aeons
+aequo
+aerie
+aesop
+aetas
+affix
+afire
+afoot
+afore
+afoul
+afros
+after
+again
+agape
+agate
+agave
+agent
+agers
+agger
+aggie
+aggro
+agile
+aging
+agios
+agist
+agley
+aglow
+agnus
+agony
+agree
+agues
+ahead
+ahems
+aided
+aider
+aides
+ailed
+aimed
+aimer
+ain't
+ainus
+aioli
+aired
+airer
+aires
+aisle
+aitch
+akela
+akron
+alack
+alaes
+alamo
+alarm
+alary
+alate
+albae
+album
+alden
+alder
+aldol
+alees
+alert
+aleut
+algae
+algal
+algas
+algid
+algin
+algol
+algor
+alias
+alibi
+alice
+alien
+align
+alike
+aline
+alios
+alive
+alkyd
+alkyl
+allah
+allan
+allay
+allen
+alley
+allod
+allot
+allow
+alloy
+allyl
+aloes
+aloft
+aloha
+aloin
+alone
+along
+aloof
+aloud
+alpha
+altar
+alter
+altos
+altum
+alvei
+amaas
+amahs
+amain
+amass
+amaze
+amber
+ambit
+amble
+ambon
+ambos
+ambry
+ameba
+ameli
+amend
+amens
+amice
+amici
+amide
+amigo
+amine
+amino
+amish
+amiss
+amity
+amman
+amnia
+amnii
+among
+amove
+ample
+amply
+ampul
+amtex
+amuck
+amuse
+andes
+andre
+anent
+angel
+anger
+angle
+anglo
+angry
+angst
+angus
+anile
+anima
+animo
+anion
+anise
+anita
+ankhs
+ankle
+annal
+annas
+annat
+annex
+annie
+annoy
+annul
+annum
+anode
+anoia
+anomy
+ansae
+antae
+anted
+antes
+antic
+antis
+antra
+antsy
+anvil
+anzac
+aorta
+apace
+apart
+apdus
+apeak
+apery
+aphid
+aphis
+apian
+aping
+apish
+apium
+apnea
+aport
+appal
+appel
+apple
+apply
+appro
+april
+apron
+apses
+apsis
+apter
+aptly
+aquae
+arabs
+araby
+arced
+arcos
+arcus
+areal
+areas
+areca
+arena
+arete
+argal
+argas
+argil
+argon
+argos
+argot
+argue
+argus
+arian
+arias
+ariel
+aries
+arils
+arise
+armed
+armer
+armis
+aroma
+arose
+aroyl
+arran
+arras
+array
+arret
+arrha
+arris
+arrow
+arson
+arums
+aryan
+asana
+ascii
+ascot
+ascus
+asdic
+ashen
+asher
+ashes
+ashet
+asian
+aside
+asked
+asker
+askew
+aspca
+aspen
+aspic
+ass's
+assam
+assay
+asses
+asset
+assoc
+aster
+astir
+astro
+ataxy
+atilt
+atlas
+atoll
+atoms
+atone
+atony
+atopy
+atria
+atrip
+attar
+attic
+audio
+audit
+auger
+aught
+augur
+aulae
+aulas
+aunts
+aunty
+aurae
+aural
+auras
+auric
+aurin
+auris
+aurum
+autos
+autre
+auxin
+avail
+avant
+avast
+avena
+avens
+avers
+avert
+avian
+avion
+avium
+avoid
+avows
+awacs
+await
+awake
+award
+aware
+awash
+awful
+awing
+awned
+awoke
+axels
+axers
+axial
+axile
+axing
+axiom
+axite
+axles
+axone
+axons
+ayahs
+ayers
+azide
+azoic
+azole
+azote
+azoth
+aztec
+azure
+baaed
+babas
+babel
+babes
+baccy
+backs
+bacon
+baddy
+badge
+badly
+baffy
+bagel
+baggy
+bahai
+bahts
+bails
+bairn
+baits
+baize
+baked
+baker
+bakes
+balas
+baled
+baler
+bales
+balks
+balky
+balls
+bally
+balms
+balmy
+balsa
+balun
+banal
+banco
+bands
+bandy
+banes
+bangs
+banjo
+banks
+banns
+banti
+bantu
+barbi
+barbs
+bards
+bardy
+bared
+barer
+bares
+baret
+barfs
+barge
+baric
+barit
+barks
+barmy
+barns
+baron
+barre
+barry
+barye
+basad
+basal
+based
+basel
+baser
+bases
+basic
+basil
+basin
+basis
+basks
+bason
+bassi
+basso
+baste
+batch
+bated
+bates
+bathe
+baths
+batik
+baton
+batty
+batyl
+bauds
+baulk
+bawds
+bawdy
+bawls
+bayed
+bayou
+beach
+beads
+beady
+beaks
+beaky
+beams
+beamy
+beano
+beans
+beard
+bears
+beast
+beats
+beaus
+beaut
+beaux
+bedbu
+bedew
+bedim
+beech
+beefs
+beefy
+beeps
+beers
+beets
+befit
+befog
+began
+beget
+begin
+begot
+begum
+begun
+beige
+being
+bejel
+belah
+belay
+belch
+belie
+bella
+belle
+belli
+bello
+bells
+belly
+belon
+below
+belts
+bemas
+bemba
+bembo
+bence
+bench
+bends
+bendy
+benne
+benny
+bents
+beret
+bergh
+bergs
+berks
+berne
+berry
+berth
+beryl
+beset
+besom
+besot
+bests
+betas
+betel
+betes
+betsy
+betty
+bevel
+bezel
+bhang
+bialy
+bible
+bicep
+biddy
+bided
+bides
+bidet
+biens
+biers
+biffs
+bifid
+bight
+bigot
+bigwi
+bijou
+biked
+biker
+bikes
+bikie
+bilby
+bilge
+bilks
+bills
+billy
+bindi
+binds
+binet
+binge
+bingo
+bingy
+biome
+biota
+biped
+bipod
+birch
+birds
+birth
+bison
+bitch
+bited
+biter
+bites
+bitot
+bitsy
+bitts
+bitty
+bixin
+blabs
+black
+blade
+blahs
+blain
+blame
+blanc
+bland
+blank
+blare
+blase
+blast
+blaud
+blaze
+bleak
+blear
+bleat
+blebs
+bleed
+bleep
+blend
+blent
+bless
+blest
+blimp
+blind
+blini
+blink
+blips
+bliss
+blitz
+bloat
+blobs
+block
+blocs
+bloke
+blond
+blood
+bloom
+blots
+blown
+blows
+blowy
+blubs
+blued
+bluer
+blues
+bluey
+bluff
+blunt
+blurb
+blurs
+blurt
+blush
+board
+boars
+boast
+boats
+bobby
+bobsy
+boche
+boded
+bodes
+boeck
+boers
+boeuf
+bogey
+boggy
+bogie
+bogus
+boils
+boing
+boise
+bolas
+boles
+bolls
+bolly
+bolos
+bolts
+bolus
+bombe
+bombs
+bonae
+bonce
+bonds
+boned
+boner
+bones
+bongo
+bongs
+bonis
+bonne
+bonny
+bonos
+bonum
+bonus
+bonze
+booay
+boobs
+booby
+booed
+booer
+books
+booms
+boong
+boons
+boors
+boose
+boost
+booth
+boots
+booty
+booze
+boozy
+borak
+borax
+bored
+boree
+borer
+bores
+boric
+boris
+borne
+boron
+borts
+borty
+bosie
+bosky
+bosom
+boson
+bossa
+bossy
+bosun
+botch
+bothy
+bough
+bouin
+boule
+bound
+bourn
+bouts
+bovid
+bowed
+bowel
+bowen
+bower
+bowie
+bowls
+boxed
+boxer
+boxes
+boyle
+brace
+bract
+brads
+braes
+brags
+braid
+brail
+brain
+brake
+brand
+brans
+brash
+brass
+brats
+brave
+bravo
+brawl
+brawn
+braxy
+brays
+braze
+bread
+break
+bream
+breed
+breis
+brent
+breve
+brevi
+brews
+brian
+briar
+bribe
+brick
+bride
+brief
+brier
+brigs
+brill
+brims
+brine
+bring
+brink
+briny
+brisk
+brits
+broad
+broca
+broch
+brock
+broid
+broil
+broke
+bromo
+bronx
+brood
+brook
+broom
+broth
+brown
+brows
+bruce
+bruin
+bruit
+brunn
+brunt
+brush
+brusk
+brute
+bryan
+bucca
+buchu
+bucks
+budda
+buddy
+budge
+buffs
+buffy
+buggy
+bugle
+buick
+build
+built
+bulbi
+bulbs
+bulge
+bulgy
+bulks
+bulky
+bulla
+bulls
+bully
+bumbo
+bumph
+bumps
+bumpy
+bunch
+bunco
+bungs
+bunko
+bunks
+bunny
+bunts
+bunya
+buoys
+buren
+buret
+burgh
+burgs
+burin
+burke
+burks
+burls
+burly
+burma
+burns
+burnt
+burps
+burro
+burrs
+burry
+bursa
+burse
+burst
+busby
+bused
+buses
+bushy
+busks
+busts
+busty
+butch
+butte
+butts
+butty
+butyl
+buxom
+buyer
+buzzy
+bwana
+bylaw
+byres
+byron
+bytes
+byway
+c'est
+cabal
+cabby
+caber
+cabin
+cable
+cabot
+cacao
+cache
+cacti
+caddy
+cades
+cadet
+cadge
+cadit
+cadre
+caeca
+cafes
+caffs
+caged
+cagel
+cager
+cages
+cagey
+cairn
+cairo
+cajun
+caked
+cakes
+cakey
+calif
+calix
+calks
+calla
+calli
+calls
+calms
+calve
+calyx
+cambo
+camel
+cameo
+cames
+campi
+camps
+campy
+can't
+canal
+candy
+caned
+caner
+canes
+canis
+canna
+canny
+canoe
+canon
+canst
+canto
+cants
+capax
+caped
+caper
+capes
+capiz
+capon
+capos
+cappy
+caput
+carat
+carbo
+cards
+cared
+carer
+cares
+caret
+cargo
+carib
+carlo
+carne
+carny
+carob
+carol
+carom
+carpe
+carpi
+carps
+carry
+carte
+carts
+carum
+carve
+cased
+casei
+cases
+casks
+casta
+caste
+casts
+casus
+catch
+cater
+catha
+cathy
+catty
+cauda
+caulk
+causa
+cause
+cavae
+caval
+caved
+caver
+caves
+cavia
+cavil
+cavum
+cawed
+ccitt
+cease
+cecal
+cecil
+cecum
+cedar
+ceded
+ceder
+cedes
+ceils
+cella
+cello
+cells
+celom
+celts
+cense
+cento
+cents
+ceorl
+cepit
+cerci
+cerea
+cered
+ceres
+ceria
+ceryl
+cetes
+cetyl
+chace
+chafe
+chaff
+chain
+chair
+chalk
+champ
+chanc
+chant
+chaos
+chape
+chaps
+chard
+charm
+charr
+chars
+chart
+chary
+chase
+chasm
+chats
+cheap
+cheat
+check
+cheek
+cheep
+cheer
+chefs
+chela
+chert
+chess
+chest
+chews
+chewy
+chick
+chico
+chide
+chief
+chiff
+child
+chile
+chili
+chill
+chime
+chimp
+china
+chine
+chink
+chino
+chins
+chips
+chirm
+chirp
+chirr
+chits
+chive
+chivs
+chivy
+chloe
+chock
+choir
+choke
+choko
+choky
+chomp
+chops
+chord
+chore
+chose
+choux
+chows
+chris
+chubs
+chuck
+chuff
+chugs
+chump
+chums
+chunk
+churl
+churn
+chute
+chyle
+chyli
+chyme
+cider
+cigar
+ciggy
+cilia
+cills
+cimex
+cinch
+cines
+circa
+circs
+cirri
+cissy
+cists
+cited
+citer
+cites
+civet
+civic
+civil
+civvy
+clack
+clade
+clads
+claim
+clair
+clamp
+clams
+clang
+clank
+clans
+claps
+clare
+clark
+claro
+clary
+clash
+clasp
+class
+claud
+claus
+clava
+clavi
+claws
+clays
+clean
+clear
+cleat
+cleek
+clefs
+cleft
+clegs
+clerk
+clews
+click
+cliff
+climb
+clime
+cline
+cling
+clink
+clint
+clips
+clive
+clivi
+cloak
+clock
+clods
+clogs
+cloke
+clomp
+clone
+clops
+close
+cloth
+clots
+cloud
+clout
+clove
+clown
+cloys
+cloze
+clubs
+cluck
+clued
+clues
+clump
+clung
+clunk
+coach
+coact
+coals
+coapt
+coast
+coati
+coats
+coble
+cobol
+cobra
+cocao
+cocci
+cocks
+cocky
+cocoa
+cocos
+codal
+codas
+codec
+coded
+coder
+codes
+codex
+codon
+coeds
+cohen
+coifs
+coign
+coils
+coins
+coked
+cokes
+colas
+colds
+coley
+colic
+colin
+colla
+colog
+colon
+colte
+colts
+colza
+comae
+comal
+comas
+combe
+combo
+combs
+comdg
+comdr
+comdt
+comer
+comes
+comet
+comfy
+comic
+comma
+comme
+comps
+conch
+condo
+cones
+coney
+conga
+congo
+conic
+conks
+const
+contd
+conte
+contr
+conus
+cooed
+cooee
+cooks
+cooky
+cools
+cooly
+coomb
+coons
+coops
+coopt
+coots
+copal
+coped
+coper
+copes
+copra
+copse
+copts
+coral
+coram
+cords
+cored
+corer
+cores
+corgi
+coria
+corks
+corky
+corms
+corns
+cornu
+corny
+corol
+corps
+corti
+cosec
+coset
+cosmo
+costa
+costs
+cotan
+cotes
+cotta
+couch
+cough
+could
+count
+coupe
+coups
+court
+coved
+coven
+cover
+coves
+covet
+covey
+covin
+cowan
+cowed
+cower
+cowls
+coxae
+coxal
+coyer
+coyly
+coypu
+cozen
+crabs
+crack
+craft
+crags
+crake
+cramp
+crams
+crane
+crank
+crape
+craps
+crash
+crass
+crate
+crave
+crawl
+craws
+craze
+crazy
+creak
+cream
+credo
+creed
+creek
+creel
+creep
+crees
+creme
+crena
+crepe
+crept
+crepy
+cress
+crest
+creta
+crete
+crews
+cribs
+crick
+cried
+crier
+cries
+crime
+crimp
+crisp
+crith
+croak
+croat
+crock
+croft
+crohn
+crone
+cronk
+crony
+crook
+croon
+crops
+crore
+cross
+croup
+crowd
+crown
+crows
+croze
+cruck
+crude
+cruel
+cruet
+crumb
+crump
+crura
+cruse
+crush
+crust
+crypt
+csiro
+cuban
+cubby
+cubeb
+cubed
+cuber
+cubes
+cubic
+cubit
+cuddy
+cuffs
+cuing
+culch
+culet
+culex
+culls
+culpa
+cults
+cumin
+cunei
+cunni
+cupel
+cupid
+cuppa
+cupsa
+curbs
+curds
+curdy
+cured
+curer
+cures
+curet
+curia
+curie
+curio
+curls
+curly
+curry
+curse
+curst
+cursu
+curve
+curvy
+cusec
+cushy
+cusps
+cuter
+cutes
+cutey
+cutie
+cutin
+cutis
+cutty
+cutup
+cybor
+cycad
+cycle
+cyder
+cylex
+cymae
+cymas
+cymba
+cymes
+cymru
+cymry
+cynic
+cyril
+cysts
+cyton
+czars
+czech
+daces
+dacha
+dadda
+daddy
+dados
+daffy
+daggy
+dagos
+daily
+dairy
+daisy
+dakin
+dalai
+dalek
+dales
+dally
+dames
+damna
+damno
+damns
+damps
+dance
+dandy
+danes
+dante
+daraf
+darby
+dards
+dared
+darer
+dares
+darns
+darts
+datal
+dated
+dater
+dates
+datum
+daubs
+dauby
+daunt
+david
+davis
+davit
+dawns
+dazed
+dazes
+deaco
+deals
+dealt
+deans
+dears
+death
+debar
+debet
+debit
+debts
+debug
+debut
+decal
+decay
+decem
+decks
+decor
+decoy
+decpc
+decry
+decus
+dedit
+deeds
+deems
+deeps
+deers
+defat
+defer
+defoe
+defog
+degas
+deice
+deify
+deign
+deism
+deist
+deity
+dekka
+dekko
+delay
+deled
+deles
+delft
+delhi
+delis
+delle
+dells
+delta
+delve
+demes
+demit
+demob
+demon
+demos
+demur
+denes
+denim
+denom
+dense
+dents
+depot
+depth
+derby
+deriv
+derma
+derry
+desex
+desks
+deter
+deuce
+devas
+devil
+devon
+dewar
+dhall
+dhobi
+dhole
+dhoti
+dhows
+dials
+diana
+diane
+diary
+diazo
+diced
+dicer
+dices
+dicey
+dicit
+dicks
+dicky
+dicot
+dicta
+didos
+didst
+diego
+diels
+dietl
+diets
+diffs
+dight
+digit
+dikes
+dildo
+dills
+dilly
+dimer
+dimes
+dimly
+dinar
+dined
+diner
+dines
+dingo
+dings
+dingy
+dinky
+dints
+diode
+diols
+diose
+dippy
+dipus
+dirac
+direr
+dirge
+dirks
+dirts
+dirty
+disci
+disco
+discs
+dishy
+disks
+disto
+distr
+ditch
+ditto
+ditty
+divan
+divas
+dived
+diver
+dives
+divot
+divvy
+dixie
+dixit
+dizzy
+djinn
+dlitt
+dnase
+dobby
+docks
+dodge
+dodgy
+dodos
+doers
+doest
+doeth
+doffs
+doges
+doggo
+doggy
+dogie
+dogle
+dogma
+doily
+doing
+dolby
+dolce
+doled
+doles
+dolls
+dolly
+dolts
+dolus
+domed
+domes
+don't
+donah
+donee
+dongs
+donna
+donor
+donut
+dooms
+doomy
+doors
+doped
+doper
+dopes
+dopey
+doric
+doris
+dorms
+dormy
+dorsa
+dorsi
+dosed
+doser
+doses
+dotal
+doted
+doter
+dotes
+dotty
+douay
+doubt
+dough
+douse
+dover
+doves
+dowdy
+dowel
+dower
+downs
+downy
+dowry
+dowse
+doyen
+doyly
+dozed
+dozen
+dozer
+dozes
+dphil
+drabs
+draco
+draff
+draft
+drags
+drail
+drain
+drake
+drama
+drams
+drank
+drape
+drawl
+drawn
+draws
+drays
+dread
+dream
+drear
+dregs
+dress
+dribs
+dried
+drier
+dries
+drift
+drill
+drily
+drink
+drips
+dript
+drive
+droit
+droll
+drone
+drool
+droop
+drops
+dross
+drove
+drown
+drubs
+drugs
+druid
+drums
+drunk
+drupe
+druse
+dryad
+dryer
+dryly
+duads
+dubio
+ducal
+ducat
+duces
+duchy
+ducks
+ducky
+ducts
+duddy
+dudes
+duels
+duely
+duets
+duffy
+dukes
+dulia
+dulls
+dully
+dulon
+dulse
+dummy
+dumps
+dumpy
+dunce
+dunes
+dungs
+dungy
+dunks
+dunny
+duped
+duper
+dupes
+duple
+dural
+durra
+durst
+durum
+dusks
+dusky
+dusts
+dusty
+dutch
+duvet
+dwarf
+dwell
+dwelt
+dyads
+dyers
+dying
+dykes
+dynel
+dynes
+eadem
+eager
+eagle
+eagre
+eared
+earls
+early
+earns
+earth
+earwi
+eased
+easel
+easer
+eases
+eaten
+eater
+eaved
+eaves
+ebbed
+ebony
+echis
+eclat
+ecoid
+ectad
+ectal
+edema
+edgar
+edged
+edger
+edges
+edict
+edify
+edith
+edits
+educe
+educt
+eerie
+egest
+egged
+egger
+eggno
+egret
+egypt
+eider
+eight
+eigne
+eject
+eking
+eland
+elate
+elbow
+elder
+elect
+elegy
+elemi
+elfin
+elide
+elite
+ellen
+elmer
+elope
+elude
+elute
+elvan
+elver
+elves
+elvis
+email
+embay
+embed
+ember
+embow
+embus
+emcee
+emend
+emery
+emily
+emirs
+emits
+emmer
+emmet
+emmys
+emote
+empty
+enact
+enate
+ended
+ender
+endow
+endue
+enema
+enemy
+enjoy
+ennui
+enols
+enrol
+ensue
+entad
+ental
+enter
+entia
+entre
+entry
+enure
+envoi
+envoy
+enzed
+eosin
+epact
+epees
+ephah
+ephas
+ephod
+ephor
+epics
+epoch
+epode
+epoxy
+epsom
+equal
+equip
+erase
+erect
+ergon
+ergot
+erica
+eries
+ernes
+erode
+erose
+erred
+error
+eruct
+erupt
+escro
+esker
+essay
+essex
+ester
+estop
+ethel
+ether
+ethic
+ethos
+ethyl
+etude
+etuis
+etyma
+euler
+euros
+evade
+evans
+evens
+event
+evert
+every
+evict
+evils
+evoke
+ewers
+ewing
+exact
+exalt
+exams
+excel
+exeat
+exert
+exigi
+exile
+exine
+exist
+exits
+exons
+expel
+expos
+expwy
+extol
+extra
+exude
+exult
+exurb
+exxon
+eyers
+eying
+eyrie
+eyrir
+fable
+fabry
+faced
+facer
+faces
+facet
+facia
+facie
+facio
+facta
+facti
+facto
+facts
+faddy
+faded
+fader
+fades
+fados
+faery
+faffs
+fagot
+fails
+faint
+faire
+fairs
+fairy
+faith
+faits
+faked
+faker
+fakes
+fakir
+falls
+false
+falsi
+famed
+fames
+fancy
+fango
+fangs
+fanny
+fanon
+farad
+farce
+farci
+farcy
+fared
+farer
+fares
+farls
+farms
+farsi
+farth
+fasts
+fatal
+fated
+fates
+fatly
+fatso
+fatty
+fatui
+fault
+fauna
+fauns
+faust
+fauve
+favus
+fawns
+faxed
+faxes
+fazed
+fazes
+fears
+feast
+feats
+feaze
+fecal
+feces
+fecit
+feeds
+feels
+feign
+feing
+feint
+felix
+fells
+felly
+felon
+felos
+felts
+femes
+femme
+femur
+fence
+fends
+fenny
+feoff
+ferae
+feral
+feria
+fermi
+ferns
+ferny
+ferri
+ferry
+fesse
+fetal
+fetch
+feted
+fetes
+fetid
+fetor
+fetus
+feuar
+feuds
+fever
+fewer
+fezes
+fiats
+fiber
+fibre
+fiche
+fichu
+ficin
+ficus
+fidei
+fidel
+fidem
+fides
+fiefs
+field
+fiend
+fieri
+fiery
+fifer
+fifes
+fifth
+fifty
+figgy
+fight
+fijis
+filar
+filch
+filed
+filer
+files
+filet
+filii
+fills
+filly
+films
+filmy
+filth
+filum
+final
+finch
+finds
+fined
+finer
+fines
+finis
+finks
+finno
+finns
+finny
+fiord
+fired
+firer
+fires
+firma
+firms
+firry
+first
+firth
+fishy
+fists
+fitch
+fitly
+fiver
+fives
+fixed
+fixer
+fixes
+fixup
+fizzy
+fjeld
+fjord
+flack
+flags
+flail
+flair
+flake
+flaky
+flame
+flams
+flamy
+flank
+flans
+flaps
+flare
+flash
+flask
+flats
+flatu
+flava
+flaws
+flawy
+flaxy
+flays
+fleam
+fleas
+fleck
+fleer
+flees
+fleet
+flesh
+flews
+flexo
+flick
+flied
+flier
+flies
+fling
+flint
+flips
+flirt
+flits
+float
+flocc
+flock
+flocs
+floes
+flogs
+flong
+flood
+floor
+flops
+flora
+flory
+floss
+flote
+flour
+flout
+flown
+flows
+flubs
+flues
+fluff
+fluid
+fluke
+fluky
+flume
+flump
+flung
+flunk
+fluor
+flush
+flute
+fluty
+flyby
+flyer
+foals
+foams
+foamy
+focal
+focus
+foehn
+foeti
+fogey
+foggy
+foils
+foist
+folds
+folia
+folic
+folie
+folio
+folks
+folky
+folly
+fonda
+fonds
+fonts
+foods
+foody
+fools
+foots
+foray
+force
+fords
+forel
+fores
+forge
+forgo
+forks
+forma
+forme
+forms
+forte
+forth
+forti
+forts
+forty
+forum
+fossa
+fouls
+found
+fount
+fours
+fovea
+fowls
+foxed
+foxes
+foyer
+frags
+frail
+frame
+franc
+frank
+franz
+fraps
+frass
+fraud
+fraus
+frays
+frcvs
+freak
+freck
+freed
+freer
+frees
+frena
+freon
+fresh
+freta
+frets
+freud
+friar
+frics
+fried
+frier
+fries
+frill
+frisk
+frits
+fritt
+fritz
+frock
+froes
+frogs
+frond
+frons
+front
+frosh
+frost
+froth
+frown
+frows
+froze
+fruit
+frump
+fryer
+fucus
+fuddy
+fudge
+fuels
+fugae
+fugal
+fuggy
+fugit
+fugue
+fulls
+fully
+fumed
+fumer
+fumes
+fumet
+fundi
+funds
+fungi
+funic
+funis
+funks
+funky
+funny
+furan
+furls
+furor
+furry
+furta
+furze
+furzy
+fused
+fusee
+fusel
+fuses
+fusil
+fussy
+fusty
+futon
+fuzed
+fuzes
+fuzzy
+gabby
+gable
+gabon
+gadid
+gadus
+gaels
+gaffe
+gaffs
+gaged
+gager
+gages
+gaily
+gains
+gaits
+gaius
+galah
+galas
+galea
+galee
+gales
+galla
+galls
+galop
+galvo
+gamba
+gamed
+gamer
+games
+gamey
+gamic
+gamin
+gamma
+gammy
+gamut
+gange
+gangs
+ganja
+gaols
+gaped
+gaper
+gapes
+gappy
+garbo
+garbs
+garde
+garth
+gased
+gases
+gasps
+gassy
+gatch
+gated
+gates
+gator
+gauds
+gaudy
+gauge
+gauls
+gault
+gaunt
+gaups
+gaurs
+gauss
+gauze
+gauzy
+gavel
+gawks
+gawky
+gawps
+gayal
+gayer
+gayly
+gazed
+gazer
+gazes
+gdiff
+geans
+gears
+gecko
+geeks
+geese
+geest
+gehri
+gelds
+gelid
+gemma
+gemmy
+gemot
+genae
+genal
+genes
+genet
+genic
+genie
+genii
+genip
+genoa
+genom
+genre
+genro
+gents
+genua
+genus
+geode
+geoid
+gerah
+germs
+germy
+gesso
+geste
+getup
+ghana
+ghats
+ghaut
+ghazi
+ghees
+ghost
+ghoul
+ghyll
+giant
+gibed
+giber
+gibes
+gibli
+gibus
+giddy
+gifts
+gigot
+gigue
+gilds
+gilet
+gills
+gilts
+gimel
+gimps
+gimpy
+gipon
+gippo
+gippy
+gipsy
+girds
+girls
+girly
+girns
+giron
+giros
+girth
+girts
+gists
+given
+giver
+gives
+gizmo
+glace
+glade
+glads
+glair
+gland
+glans
+glare
+glary
+glass
+glaze
+glazy
+gleam
+glean
+glebe
+glede
+glees
+gleet
+glens
+glial
+glide
+glims
+glint
+glitz
+gloat
+globe
+globs
+glogg
+glome
+glomi
+gloom
+glory
+gloss
+glove
+glows
+gloze
+glued
+gluer
+glues
+gluey
+glume
+gluon
+gluts
+glyph
+gnarl
+gnash
+gnats
+gnawn
+gnaws
+gnome
+gnows
+goads
+goals
+goats
+goaty
+gobos
+godet
+godly
+goers
+gofer
+gogga
+going
+golds
+golem
+golfs
+golgi
+golly
+gonad
+goner
+gongs
+gonia
+gooch
+goodo
+goods
+goody
+gooey
+goofs
+goofy
+gooks
+gooly
+goons
+goose
+goosy
+gopak
+goral
+gored
+gores
+gorge
+gorki
+gorse
+gorsy
+goths
+gouda
+gouge
+gourd
+gouty
+gowan
+gower
+gowks
+gowns
+goyim
+grabs
+grace
+grade
+grads
+graft
+grail
+grain
+graip
+grama
+grams
+grana
+grand
+grant
+grape
+graph
+grapy
+grasp
+grass
+grata
+grate
+grato
+grave
+gravs
+gravy
+grays
+graze
+great
+grebe
+greco
+greed
+greek
+green
+greet
+greta
+greys
+gride
+grids
+grief
+griff
+grift
+grigs
+grike
+grill
+grime
+grimm
+grimy
+grind
+grins
+gripe
+grips
+gripy
+grist
+grith
+grits
+groan
+groat
+grogs
+groin
+groom
+grope
+gross
+grosz
+grots
+group
+grout
+grove
+growl
+grown
+grows
+grubb
+grubs
+gruel
+gruff
+grume
+grump
+grunt
+gryke
+guaco
+guano
+guard
+guava
+guess
+guest
+guide
+guild
+guile
+guilt
+guise
+gular
+gulch
+gules
+gulfs
+gulfy
+gulls
+gully
+gulps
+gulpy
+gumbo
+gumma
+gummy
+gunge
+gungy
+gunny
+gunya
+guppy
+gurus
+gushy
+gussy
+gusto
+gusts
+gusty
+gutsy
+gutta
+gutty
+guyed
+guyer
+guyot
+gwent
+gybed
+gybes
+gypsy
+gyral
+gyred
+gyres
+gyron
+gyros
+gyrus
+gyved
+gyver
+gyves
+habet
+habit
+habus
+hacek
+hacks
+hadal
+haded
+hades
+hadji
+hadst
+hafiz
+hafts
+hague
+haiks
+haiku
+hails
+hairs
+hairy
+haiti
+hajji
+hakea
+hakes
+hakim
+halal
+haled
+haler
+hales
+halid
+hallo
+halls
+halma
+halms
+halos
+halts
+halva
+halve
+hamal
+hames
+hammy
+hamza
+hance
+hands
+handy
+hangi
+hangs
+hanks
+hanky
+hanoi
+hapax
+haply
+happy
+hardy
+harem
+hares
+harks
+harms
+harpo
+harps
+harpy
+harry
+harsh
+harts
+harum
+hasid
+hasps
+haste
+hasty
+hatch
+hated
+hater
+hates
+haugh
+haulm
+hauls
+haunt
+haute
+haven
+haver
+haves
+havoc
+hawed
+hawks
+hawse
+haydn
+hayed
+hayes
+hazed
+hazel
+hazer
+hazes
+he'll
+heads
+heady
+heals
+heaps
+heard
+hears
+heart
+heath
+heats
+heave
+heavy
+hedge
+hedgy
+heeds
+heels
+hefts
+hefty
+heidi
+heirs
+heist
+heled
+helen
+helix
+hello
+hells
+helms
+helot
+helps
+helve
+hemad
+hemal
+hemic
+hemin
+hemps
+hence
+henge
+henle
+henna
+henry
+hepar
+herbs
+herby
+herds
+heres
+herls
+herma
+herms
+heron
+heros
+hertz
+hevea
+hewed
+hewer
+hexad
+hexed
+hexes
+hexyl
+hicks
+hider
+hides
+highs
+hight
+hiked
+hiker
+hikes
+hilar
+hills
+hilly
+hilts
+hilum
+hilus
+hinau
+hindi
+hinds
+hindu
+hinge
+hinny
+hints
+hippo
+hippy
+hired
+hirer
+hires
+hises
+hists
+hitch
+hived
+hives
+hoagy
+hoard
+hoary
+hobby
+hobos
+hocks
+hocus
+hoers
+hogan
+hogty
+hoing
+hoise
+hoist
+hoked
+hokes
+hokey
+hokum
+holds
+holed
+holer
+holes
+holey
+hollo
+holly
+holme
+holts
+homan
+homed
+homer
+homes
+homey
+homos
+honda
+honed
+honer
+hones
+honey
+hongi
+honks
+honky
+hooch
+hoods
+hooey
+hoofs
+hooke
+hooks
+hooky
+hoops
+hoots
+hoped
+hoper
+hopes
+hopis
+horae
+horde
+horns
+horny
+horol
+horse
+horst
+horsy
+hosea
+hosed
+hoses
+hosta
+hosts
+hotel
+hotly
+hough
+hound
+houri
+hours
+house
+hovel
+hover
+howdy
+howls
+hoyle
+hubby
+huffs
+huffy
+huger
+huges
+hulas
+hulks
+hulky
+hulls
+human
+humic
+humid
+humin
+humph
+humps
+humpy
+humus
+hunch
+hunks
+hunky
+hunts
+hurls
+hurly
+huron
+hurry
+hurst
+hurts
+husks
+husky
+hussy
+hutch
+hydra
+hydro
+hyena
+hying
+hymen
+hymns
+hyoid
+hyped
+hyper
+hypes
+hypha
+hypos
+hyrax
+hyson
+iambs
+ibiza
+ibsen
+iceni
+ichor
+icier
+icily
+icing
+icons
+ictal
+ictus
+idaho
+ideal
+ideas
+idees
+idems
+idiom
+idiot
+idled
+idler
+idles
+idols
+idose
+idyll
+igfet
+igloo
+ignes
+ignis
+ilang
+ileac
+ileal
+ileum
+ileus
+iliac
+iliad
+ilium
+iller
+illus
+image
+imago
+imams
+imbed
+imbue
+imide
+imido
+imine
+imino
+imped
+impel
+imper
+imply
+inane
+inapt
+incan
+incas
+incog
+incur
+incus
+indef
+index
+india
+indic
+indiv
+indri
+indus
+inept
+inert
+infer
+infin
+infix
+infra
+ingot
+iniac
+inial
+inion
+inked
+inker
+inkle
+inlaw
+inlay
+inlet
+inner
+inops
+inorg
+input
+inset
+instr
+inter
+intra
+intro
+inuit
+inure
+inver
+iodic
+iodin
+ionic
+iotas
+iowan
+iraqi
+irate
+irene
+irish
+irked
+irons
+irony
+irreg
+isaac
+islam
+isles
+islet
+isn't
+isolo
+issei
+issue
+it'll
+italo
+italy
+itchy
+items
+ivied
+ivies
+ivory
+izard
+jabot
+jacks
+jacob
+jaded
+jades
+jaffa
+jaggy
+jails
+jalap
+jambe
+jambs
+james
+jammy
+janet
+janus
+japan
+japed
+japer
+japes
+jared
+jason
+jaspe
+jatos
+jaunt
+javan
+javel
+jawed
+jazzy
+jeans
+jeeps
+jeers
+jells
+jelly
+jemmy
+jenny
+jerks
+jerky
+jerry
+jesse
+jests
+jesus
+jetes
+jetty
+jewel
+jewry
+jibed
+jiber
+jibes
+jiffs
+jiffy
+jihad
+jilts
+jimmy
+jingo
+jinks
+jinni
+jinns
+jived
+jiver
+jives
+jocks
+joeys
+johne
+johns
+joins
+joint
+joist
+joked
+joker
+jokes
+jokey
+jolie
+jolly
+jolts
+jolty
+jonah
+jones
+jorum
+joses
+joule
+joust
+jowls
+jowly
+joyce
+juans
+jubes
+judah
+judas
+judex
+judge
+jugal
+jugum
+juice
+juicy
+julep
+julia
+jumar
+jumbo
+jumps
+jumpy
+junco
+junks
+junky
+junta
+junto
+jural
+jurat
+jured
+jurel
+juris
+juror
+justa
+juste
+justo
+jutes
+juxta
+kabob
+kafka
+kagus
+kakas
+kakis
+kales
+kames
+kanga
+kanji
+kanzu
+kaons
+kapok
+kappa
+kaput
+karas
+karat
+karen
+karma
+karri
+karst
+karts
+kasha
+kathy
+kauri
+kayak
+kayos
+kazoo
+keats
+kebab
+kedge
+keels
+keens
+keeps
+kefir
+kelly
+kelps
+kelpy
+kelts
+kempt
+kempy
+kenny
+kente
+kenya
+kepis
+kerbs
+kerfs
+kerma
+kerns
+kerry
+ketal
+ketch
+kevel
+keyed
+khaki
+khans
+kheda
+kiaat
+kiang
+kibei
+kibes
+kicks
+kicky
+kiddy
+kiers
+kills
+kilns
+kilos
+kilts
+kinds
+kings
+kinin
+kinks
+kinky
+kiosk
+kiowa
+kippa
+kirby
+kirks
+kited
+kiter
+kites
+kithe
+kitty
+kivas
+kiwis
+klans
+klebs
+kleig
+klieg
+kline
+kloof
+klutz
+knack
+knags
+knave
+knead
+kneed
+kneel
+knees
+knell
+knelt
+knife
+knish
+knits
+knobs
+knock
+knoll
+knoop
+knots
+knout
+known
+knows
+knuri
+knurl
+knurr
+koala
+koans
+kodak
+kofta
+koine
+kojic
+kooks
+kooky
+koori
+kopek
+kopje
+koran
+korea
+korma
+kosin
+kosso
+kotos
+kraal
+kraft
+krait
+krans
+kraut
+krebs
+krill
+krona
+krone
+kudos
+kudzu
+kukri
+kulak
+kulan
+kumys
+kurta
+kurus
+kvass
+kwela
+kyats
+kylie
+kylin
+kylix
+kyoto
+kypes
+kyrie
+label
+labia
+labra
+laced
+lacer
+laces
+lacks
+laddy
+laded
+laden
+lader
+lades
+ladle
+laesa
+lagan
+lager
+lagon
+laird
+lairs
+lairy
+laity
+laked
+laker
+lakes
+lakhs
+lamas
+lambs
+lamed
+lamer
+lames
+lamia
+lamps
+lanae
+lanai
+lance
+lands
+lanes
+lange
+lanka
+lanky
+lapdo
+lapel
+lapin
+lapis
+lappa
+lapps
+lapse
+larch
+lards
+lardy
+lares
+large
+largo
+larks
+larky
+larry
+larva
+lased
+laser
+lases
+lassa
+lasso
+lasts
+latah
+latch
+laten
+later
+latex
+lathe
+laths
+lathy
+latin
+latus
+laude
+lauds
+laugh
+lavas
+laved
+laver
+laves
+lawns
+lawny
+laxer
+laxly
+layer
+lazar
+lazed
+lazes
+leach
+leads
+leady
+leafs
+leafy
+leaks
+leaky
+leans
+leant
+leaps
+leapt
+learn
+lease
+leash
+least
+leats
+leave
+ledge
+ledgy
+leech
+leeks
+leers
+leery
+lefty
+legal
+legem
+leger
+leges
+leggy
+legis
+legit
+legum
+lehrs
+lemma
+lemon
+lemur
+lends
+lenes
+lenin
+lenis
+lenos
+leone
+leper
+lepra
+leses
+lethe
+letup
+levee
+level
+lever
+levin
+levis
+lewis
+lexis
+leydi
+liana
+liars
+libel
+libra
+libre
+libya
+licit
+licks
+lidos
+liege
+liens
+lieut
+lifer
+lifts
+ligan
+ligen
+liger
+light
+ligna
+ligon
+liked
+liken
+likes
+lilac
+lilts
+limas
+limbo
+limbs
+limed
+limen
+limes
+limey
+limit
+limns
+limon
+limos
+limps
+linac
+linda
+linea
+lined
+linen
+liner
+lines
+liney
+lingo
+lings
+lingy
+linin
+links
+lints
+linty
+lions
+lipid
+lippy
+liras
+lisle
+lisps
+lists
+liszt
+litem
+lites
+lithe
+litis
+lived
+liven
+liver
+lives
+livid
+llama
+llano
+loach
+loads
+loafs
+loams
+loamy
+loans
+loath
+lobar
+lobby
+lobed
+lobes
+lobus
+local
+lochs
+locke
+locks
+locos
+locum
+locus
+loden
+lodes
+lodge
+loess
+lofts
+lofty
+logan
+loges
+logia
+logic
+logie
+logos
+loins
+lolls
+lolly
+loner
+longs
+looks
+looms
+loons
+loony
+loops
+loopy
+loose
+loots
+loped
+loper
+lopes
+loran
+lords
+lorry
+loser
+loses
+lossy
+lotic
+lotio
+lotto
+lotus
+lough
+louis
+loupe
+louse
+lousy
+louts
+loved
+lover
+loves
+lovey
+lowan
+lowed
+lower
+lowly
+loxia
+loyal
+luaus
+lubes
+lubra
+lucan
+lucia
+lucid
+lucks
+lucky
+lucre
+lucri
+ludwi
+luffs
+luged
+luges
+lugol
+lulls
+lumen
+lumps
+lumpy
+lunar
+lunch
+lunge
+lungs
+lupin
+lupus
+lurch
+lured
+lurer
+lures
+lurid
+lurks
+lusts
+lusty
+lusus
+lutea
+lutes
+luxus
+lyase
+lying
+lymph
+lynch
+lyons
+lyres
+lyric
+lysed
+lyses
+lysin
+lysis
+lysol
+lyssa
+lytic
+lytta
+ma'am
+maana
+maare
+maars
+macaw
+maced
+macer
+maces
+macho
+macle
+macro
+madam
+madly
+mafia
+magic
+magis
+magma
+magna
+magot
+magus
+mahoe
+maids
+mails
+maims
+maine
+mains
+maire
+maize
+major
+maker
+makes
+makos
+malar
+malay
+males
+malic
+malls
+malta
+malts
+malty
+malum
+malus
+mamas
+mamba
+mambo
+mamma
+mammy
+maned
+manes
+mange
+mango
+mangy
+mania
+manic
+manit
+manky
+manly
+manna
+manor
+manse
+manta
+manum
+manus
+maori
+mapau
+maple
+marae
+maras
+march
+mardi
+mardy
+marek
+mares
+marey
+marge
+maria
+marie
+marks
+marls
+marly
+marri
+marry
+marsh
+marts
+maser
+mashy
+masks
+mason
+masse
+massy
+masts
+matai
+match
+mated
+mater
+mates
+matey
+maths
+matin
+matte
+matzo
+mauds
+mauls
+maund
+mauve
+maven
+mavin
+mavis
+maxim
+maxis
+mayan
+mayas
+maybe
+mayor
+mazed
+mazel
+mazer
+mazes
+mazic
+mccoy
+meads
+meals
+mealy
+means
+meant
+meany
+meats
+meaty
+mecca
+medal
+media
+medic
+medii
+medoc
+meeds
+meets
+meith
+mekon
+melba
+melds
+melee
+melic
+melon
+melts
+memos
+mends
+mensa
+menta
+mente
+menus
+meows
+merci
+mercy
+meres
+merge
+merit
+merry
+mesad
+mesal
+mesas
+meshy
+mesne
+meson
+messy
+metal
+meted
+metes
+metra
+metro
+mewed
+mewls
+miami
+miaow
+miasm
+micah
+micra
+micro
+midas
+middy
+midge
+midis
+midst
+miens
+miffs
+might
+mikes
+milan
+milch
+miler
+miles
+milia
+milks
+milky
+mills
+mimed
+mimeo
+mimer
+mimes
+mimic
+mince
+minds
+mined
+miner
+mines
+minim
+minis
+minks
+minor
+mints
+minty
+minus
+mired
+mires
+mirth
+mirvs
+misdo
+miser
+mists
+misty
+miter
+mites
+mitis
+mitts
+mixed
+mixer
+mixes
+mixup
+moans
+moats
+mobil
+mocha
+mocks
+modal
+model
+modem
+modes
+modis
+modus
+mogul
+moils
+moire
+moist
+molal
+molar
+molds
+moldy
+moles
+molls
+molts
+mommy
+monad
+monas
+monde
+money
+monks
+monte
+month
+mooch
+moods
+moody
+mooed
+moons
+moors
+moose
+moots
+moped
+moper
+mopes
+mopey
+morae
+moral
+moras
+morax
+moray
+morbi
+morel
+mores
+moria
+morns
+moron
+morse
+moses
+mosey
+mossy
+motel
+motes
+moths
+motif
+motor
+motto
+mould
+moult
+mound
+mount
+mourn
+mouse
+mousy
+mouth
+moved
+mover
+moves
+movie
+mowed
+mower
+moxie
+mucic
+mucin
+mucks
+mucky
+mucor
+mucro
+mucus
+muddy
+muffs
+mufti
+muggy
+mulch
+mulct
+mules
+mulls
+multi
+multo
+mumbo
+mummy
+mumps
+munch
+muons
+mural
+murky
+musca
+mused
+muser
+muses
+mushy
+music
+musks
+musky
+mussy
+musts
+musty
+muted
+mutes
+mutts
+mutua
+mutus
+mylar
+mynas
+myoid
+myoma
+myope
+myrrh
+myths
+naacp
+nabob
+nacre
+nadir
+naevi
+naffs
+nahum
+naiad
+nails
+naive
+naked
+naker
+namby
+named
+namer
+names
+namma
+nancy
+nanna
+nanny
+napes
+napoo
+nappa
+nappe
+nappy
+narcs
+nares
+naris
+narks
+narky
+nasal
+nasty
+nasus
+natal
+nates
+natis
+natty
+naval
+navar
+navel
+naves
+navig
+navis
+navvy
+nawab
+nazis
+neaps
+nears
+neath
+necks
+needs
+needy
+negri
+negro
+negus
+nehru
+neigh
+nelly
+nepal
+neper
+nepos
+nerds
+nerve
+nervi
+nervy
+nests
+never
+nevus
+newel
+newer
+newly
+newsy
+newts
+nexus
+ngaio
+nicer
+niche
+nicks
+nicol
+nidal
+nidus
+niece
+niffy
+nifty
+niger
+night
+nigra
+nihil
+nikau
+nimbi
+nines
+ninny
+ninon
+ninth
+nippy
+nisei
+nisin
+nissl
+nisus
+niter
+nitre
+nitty
+nival
+nixed
+nixes
+nixie
+nixon
+nobby
+nobel
+nobis
+noble
+nobly
+nocks
+nodal
+noddy
+nodes
+nodus
+noels
+nohow
+noire
+noise
+noisy
+nolle
+nomad
+nomen
+nomes
+nomic
+nomoi
+nomos
+nonce
+nones
+nooks
+noons
+noose
+norma
+norms
+norse
+north
+nosed
+noser
+noses
+nosey
+notal
+notch
+noted
+notes
+notre
+notum
+nouns
+novae
+novas
+novel
+noway
+noxae
+noxal
+nubby
+nubus
+nucha
+nuder
+nudes
+nudge
+nudum
+nukes
+nulla
+nulli
+nullo
+nulls
+numbs
+numen
+numis
+nurse
+nutty
+nylon
+nymph
+nyxes
+nyxis
+oaken
+oakum
+oared
+oases
+oasis
+oaten
+oaths
+obeah
+obeli
+obese
+obeys
+obiit
+obits
+objet
+oboes
+oboli
+obsta
+occur
+ocean
+ocher
+ochre
+ochry
+ocrea
+octad
+octal
+octet
+oculi
+odder
+oddly
+odeon
+odium
+offal
+offer
+often
+ogham
+ogive
+ogled
+ogler
+ogles
+ogres
+ohmic
+oidia
+oiled
+oiler
+oinks
+okapi
+okays
+okras
+oktas
+olden
+older
+oldie
+oleic
+olein
+oleum
+oleyl
+olios
+olive
+ology
+omaha
+omasa
+ombre
+omega
+omens
+omits
+omnia
+oncer
+onely
+onere
+onion
+onium
+onset
+oomph
+ootid
+oozed
+oozes
+opaca
+opals
+opens
+opera
+opere
+opine
+opium
+opsin
+opted
+optic
+oracy
+orale
+orals
+orate
+orbed
+orbit
+orcin
+order
+organ
+oribi
+oriel
+orion
+orlon
+orlop
+ormer
+oroya
+orris
+osage
+osaka
+oscar
+osier
+osmic
+osmol
+ossea
+ostia
+other
+otter
+ought
+ouija
+ounce
+ousel
+ousts
+outdo
+outer
+outgo
+outre
+ovale
+ovals
+ovary
+ovate
+ovens
+overs
+overt
+ovine
+ovoid
+ovoli
+ovolo
+ovule
+owing
+owlet
+owned
+owner
+oxbow
+oxide
+oxime
+oxter
+ozena
+ozone
+paced
+pacem
+pacer
+paces
+pacey
+packs
+pacta
+pacts
+paddy
+padre
+paean
+paeon
+pagan
+paged
+pager
+pages
+paget
+pails
+paine
+pains
+paint
+pairs
+paisa
+paise
+paled
+paler
+pales
+palls
+pally
+palmi
+palms
+palmy
+palps
+palsy
+palts
+pampa
+panda
+panel
+panes
+panga
+pangs
+panic
+panne
+panni
+pansy
+panto
+pants
+panty
+papal
+papas
+papaw
+paper
+pappi
+pappy
+papua
+parae
+paras
+parch
+parco
+pards
+pared
+paren
+parer
+pares
+paris
+parka
+parks
+parky
+parma
+parol
+parry
+parse
+parte
+parti
+parts
+party
+parve
+pases
+pasha
+pashm
+passe
+passu
+pasta
+paste
+pasts
+pasty
+patch
+pated
+paten
+pater
+pates
+paths
+patio
+patsy
+patty
+pauli
+paulo
+pause
+paved
+paver
+paves
+pavis
+pavor
+pawed
+pawky
+pawls
+pawns
+payed
+payee
+payer
+payor
+peace
+peach
+peaks
+peaky
+peals
+pearl
+pears
+pease
+peaty
+pecan
+peche
+pecks
+pedal
+pedes
+pedis
+peeks
+peels
+peens
+peeps
+peers
+peeve
+peggy
+peing
+pekan
+pekes
+pekoe
+pelts
+penal
+pence
+pends
+penes
+penis
+penna
+penni
+penny
+pense
+peons
+peony
+pepos
+peppy
+pepsi
+perch
+peril
+perks
+perky
+perry
+perse
+perth
+pesky
+pesos
+pests
+petal
+peter
+petit
+petri
+petro
+petty
+peyer
+phage
+pharm
+phase
+phial
+phlox
+phone
+phono
+phons
+phony
+phose
+photo
+phren
+phyla
+phyma
+piano
+picas
+pichi
+picks
+picky
+picot
+picts
+picul
+pidog
+piece
+piers
+pieta
+piety
+piggy
+pigmy
+pikas
+piked
+piker
+pikes
+pilaf
+pilar
+pilau
+pilch
+pilea
+piled
+pilei
+piles
+pills
+pilot
+pilus
+pimps
+pinch
+pined
+pines
+pingo
+pings
+pinko
+pinks
+pinky
+pinna
+pinny
+pinon
+pinta
+pinto
+pints
+pinup
+pious
+piped
+piper
+pipes
+pipet
+pipit
+pippy
+pique
+piste
+pitch
+piths
+pithy
+piton
+pitot
+pitta
+pivot
+pixel
+pixie
+pizza
+place
+plage
+plaid
+plain
+plait
+plana
+plane
+plank
+plano
+plans
+plant
+plash
+plasm
+plate
+plato
+plats
+platy
+plays
+plaza
+plead
+pleas
+pleat
+plebe
+plebs
+plena
+plene
+plica
+plied
+plier
+plies
+plink
+plods
+plonk
+plops
+plots
+plows
+ploys
+pluck
+plugs
+plumb
+plume
+plump
+plums
+plumy
+plunk
+plush
+pluto
+poach
+poaka
+pocks
+pocus
+poddy
+podge
+podgy
+podia
+poems
+poena
+poesy
+poets
+pogge
+pogos
+poilu
+poind
+point
+poise
+poked
+poker
+pokes
+polar
+poled
+poler
+poles
+polio
+polje
+polka
+polks
+polls
+polyp
+polys
+pommy
+ponce
+ponds
+pones
+pongo
+pongy
+pooch
+poohs
+pools
+poops
+popes
+poppy
+porch
+pored
+pores
+porgy
+poria
+porky
+porno
+porta
+ports
+porus
+posed
+poser
+poses
+posey
+posit
+posix
+posse
+posts
+potto
+potty
+pouch
+poult
+pound
+pours
+pouts
+powan
+power
+poxes
+prams
+prang
+prank
+prase
+prate
+prawn
+prays
+preen
+preps
+presa
+press
+prexy
+preys
+price
+prick
+pride
+pried
+prier
+pries
+prigs
+prill
+prima
+prime
+primo
+primp
+prims
+prink
+print
+prion
+prior
+prise
+prism
+prist
+prius
+privy
+prize
+proal
+probe
+prods
+proem
+profs
+prole
+promo
+proms
+prone
+prong
+proof
+props
+prose
+pross
+prosy
+proud
+prove
+prowl
+prows
+proxy
+prude
+prune
+psalm
+pseud
+psfig
+pshaw
+psoae
+psoai
+psoas
+psora
+psych
+pubes
+pubic
+pubis
+pucks
+pudge
+pudgy
+pudic
+pudus
+puffs
+puffy
+puggy
+puked
+pukes
+pukka
+puled
+puler
+pules
+pulex
+pulls
+pulps
+pulpy
+pulse
+pumas
+pumps
+punch
+punka
+punks
+punts
+punty
+pupae
+pupal
+pupas
+pupil
+puppy
+puree
+purer
+purge
+purim
+purls
+purrs
+purse
+pursy
+pusan
+pushy
+pussy
+putts
+putty
+pygal
+pygmy
+pylae
+pylar
+pylas
+pylic
+pylon
+pyoid
+pyran
+pyres
+pyrex
+pyxed
+pyxes
+pyxie
+pyxis
+pzazz
+qatar
+qiana
+quack
+quads
+quaff
+quaho
+quail
+quake
+quaky
+quale
+qualm
+quant
+quare
+quark
+quart
+quash
+quasi
+quays
+queen
+queer
+quell
+quern
+quers
+query
+quest
+queue
+quick
+quids
+quiet
+quiff
+quill
+quilt
+quina
+quins
+quint
+quips
+quire
+quirk
+quirt
+quite
+quito
+quits
+quoad
+quoin
+quoit
+quota
+quote
+quoth
+rabat
+rabbi
+rabic
+rabid
+raced
+racer
+races
+racks
+radar
+radii
+radio
+radix
+radon
+rafts
+ragas
+raged
+rages
+ragta
+raids
+rails
+rains
+rainy
+raise
+rajah
+rajas
+raked
+raker
+rakes
+rally
+ralph
+ramal
+raman
+ramie
+ramps
+ramus
+rance
+ranch
+randy
+ranee
+range
+rangy
+ranis
+ranks
+rants
+raped
+raper
+rapes
+raphe
+rapid
+raree
+rarer
+rased
+rasps
+raspy
+ratal
+rated
+ratel
+rater
+rates
+ratio
+ratty
+raved
+ravel
+raven
+raver
+raves
+rawer
+rawly
+rayed
+rayon
+razed
+razer
+razes
+razor
+reach
+react
+readd
+reads
+ready
+realm
+reals
+reams
+reaps
+rearm
+rears
+rebel
+rebid
+rebus
+rebut
+recap
+recce
+recta
+recti
+recto
+recur
+redan
+redia
+redid
+redly
+redox
+redry
+redux
+redye
+reeds
+reedy
+reefs
+reefy
+reeks
+reeky
+reels
+reeve
+refer
+reffo
+refit
+refry
+regal
+regia
+regie
+regno
+rehab
+rehan
+rehem
+rehun
+reich
+reify
+reign
+reins
+rejig
+rekey
+relax
+relay
+relet
+relic
+relit
+remak
+reman
+remex
+remit
+remix
+renal
+rends
+renew
+renin
+rents
+reoil
+reord
+repay
+repel
+repin
+reply
+repot
+reran
+rerum
+rerun
+resat
+reset
+resin
+resit
+resow
+rests
+retch
+retia
+retie
+retro
+retry
+reuse
+revel
+revet
+revue
+rewed
+rheas
+rhein
+rheme
+rheum
+rhine
+rhino
+rhode
+rhone
+rhumb
+rhyme
+rials
+riata
+ribes
+rican
+ricco
+riced
+rices
+riche
+ricin
+ricks
+rider
+rides
+ridge
+ridgy
+riels
+riffs
+rifle
+rifts
+riggs
+right
+rigid
+riled
+riles
+rille
+rills
+rimae
+rimed
+rimer
+rimes
+rinds
+rings
+rinks
+rinse
+riots
+ripen
+riper
+ripes
+risen
+riser
+rises
+risks
+risky
+risus
+rites
+ritzy
+rival
+rived
+riven
+river
+rives
+rivet
+riyal
+rnase
+roach
+roads
+roams
+roars
+roast
+robed
+robes
+robin
+roble
+robot
+roche
+rocks
+rocky
+rodeo
+rodin
+roger
+roget
+rogue
+roils
+roist
+roles
+rolls
+roman
+romeo
+romes
+romps
+rondo
+roneo
+roods
+roofs
+rooks
+rooky
+rooms
+roomy
+roost
+roots
+rooty
+roped
+roper
+ropes
+rorty
+roses
+rosie
+rosin
+rotes
+rotls
+rotor
+rotty
+roues
+rouge
+rough
+round
+roupy
+rouse
+roust
+route
+routs
+roved
+rover
+roves
+rowan
+rowdy
+rowed
+rowel
+rower
+royal
+rspca
+rsvps
+rubes
+rubin
+ruble
+rubor
+rubus
+ruche
+rucks
+ruddy
+ruder
+ruffe
+ruffs
+rugae
+rugal
+rugby
+ruing
+ruins
+ruled
+ruler
+rules
+rumba
+rumen
+rumex
+rumly
+rummy
+rumps
+rumpy
+runes
+rungs
+runic
+runny
+runts
+runty
+rupee
+rupia
+rural
+ruses
+rushy
+rusks
+russe
+rusts
+rusty
+rutin
+rutty
+saber
+sabin
+sable
+sably
+sabot
+sabra
+sabre
+sacci
+sachs
+sacks
+sacra
+sadhu
+sadly
+safer
+safes
+sagas
+sager
+sages
+saggy
+sahib
+saiga
+sails
+saint
+saith
+saker
+sakes
+salad
+salem
+salep
+sales
+salic
+sally
+salmi
+salol
+salon
+salop
+salpa
+salsa
+salts
+salty
+salve
+salvo
+samba
+sambo
+samoa
+samos
+sanae
+sands
+sandy
+saner
+sanka
+santa
+sapid
+sapos
+sappy
+sarah
+saran
+saree
+sarge
+sarin
+saris
+sarky
+saron
+saros
+sarum
+sassy
+satan
+satay
+sated
+satem
+sates
+satin
+satre
+satyr
+sauce
+sauch
+saucy
+saudi
+sauna
+saury
+saute
+saved
+saver
+saves
+savin
+savoy
+savvy
+sawed
+sawer
+saxes
+saxon
+sayee
+sayer
+scabs
+scads
+scala
+scald
+scale
+scalp
+scaly
+scamp
+scams
+scans
+scant
+scape
+scapi
+scare
+scarf
+scarp
+scars
+scary
+scats
+scena
+scend
+scene
+scent
+schmo
+schwa
+scion
+scire
+scoff
+scold
+scone
+scoop
+scoot
+scopa
+scope
+score
+scorn
+scots
+scott
+scour
+scout
+scowl
+scows
+scrag
+scram
+scran
+scrap
+scraw
+scree
+screw
+scrim
+scrip
+scrod
+scrub
+scrum
+scuba
+scuds
+scuff
+scull
+scums
+scups
+scurf
+scuta
+scute
+seago
+seals
+sealy
+seams
+seamy
+sears
+seats
+sebum
+secco
+sects
+secus
+sedan
+seder
+sedge
+sedgy
+sedum
+seeds
+seedy
+seeks
+seels
+seems
+seeps
+seers
+segni
+segno
+segue
+seige
+seign
+seine
+seing
+seise
+seize
+sella
+sells
+selva
+semen
+sends
+senna
+senor
+sensa
+sense
+sensu
+senza
+seoul
+sepal
+sepia
+sepoy
+septa
+serac
+serbs
+serfs
+serge
+serif
+serin
+serow
+serra
+serum
+serve
+servo
+setae
+setal
+seton
+setts
+setup
+seven
+sever
+sevum
+sewed
+sewer
+sexed
+sexes
+seyor
+se�or
+shack
+shade
+shads
+shady
+shaft
+shags
+shahs
+shake
+shako
+shaky
+shale
+shall
+shalt
+shame
+shams
+shank
+shape
+shard
+share
+shark
+sharp
+shave
+shawl
+shawm
+shays
+she'd
+she's
+shead
+sheaf
+shear
+sheds
+sheen
+sheep
+sheer
+sheet
+sheik
+shelf
+shell
+shelt
+sheol
+sherd
+shers
+shiai
+shied
+shier
+shies
+shift
+shiga
+shill
+shily
+shims
+shine
+shins
+shiny
+ships
+shire
+shirk
+shirr
+shirt
+shish
+shiva
+shive
+shivs
+shoal
+shoat
+shock
+shoed
+shoer
+shoes
+shoji
+shone
+shook
+shoos
+shoot
+shope
+shops
+shore
+shorn
+short
+shots
+shott
+shout
+shove
+shown
+shows
+showy
+shred
+shrew
+shrub
+shrug
+shuck
+shuns
+shunt
+shush
+shuts
+shyer
+shyly
+sibyl
+sicks
+sided
+sides
+sidle
+siege
+sieve
+sifts
+sighs
+sight
+sigla
+sigma
+signa
+signs
+sikhs
+silex
+silks
+silky
+sills
+silly
+silos
+silts
+silty
+simon
+sinai
+sinal
+since
+sines
+sinew
+singe
+sings
+sinks
+sinus
+sioux
+sippy
+sired
+siren
+sires
+sirup
+sisal
+sissy
+sitae
+sitar
+sited
+sites
+sitka
+situp
+situs
+sixed
+sixer
+sixes
+sixmo
+sixth
+sixty
+sizar
+sized
+sizer
+sizes
+skate
+skeet
+skegs
+skein
+skelp
+skene
+skews
+skids
+skied
+skier
+skies
+skiff
+skiis
+skill
+skimp
+skims
+skink
+skins
+skint
+skips
+skirl
+skirr
+skirt
+skite
+skits
+skive
+skulk
+skull
+skunk
+skyed
+slabs
+slack
+slain
+slake
+slams
+slang
+slant
+slaps
+slash
+slate
+slats
+slaty
+slave
+slavs
+slays
+sleds
+sleek
+sleep
+sleet
+slept
+slews
+slice
+slick
+slide
+slier
+slily
+slime
+slims
+slimy
+sling
+slink
+slips
+slits
+slobs
+slogs
+sloop
+slope
+slops
+slosh
+sloth
+slots
+slows
+slubs
+slued
+slues
+sluff
+slugs
+slump
+slums
+slung
+slunk
+slurp
+slurs
+slush
+sluts
+slyer
+slyke
+slyly
+slype
+smack
+small
+smalt
+smarm
+smart
+smash
+smear
+smell
+smelt
+smile
+smirk
+smite
+smith
+smock
+smoke
+smoko
+smoky
+smolt
+smote
+snack
+snafu
+snags
+snail
+snake
+snaky
+snaps
+snare
+snarl
+snath
+sneak
+sneck
+sneer
+snell
+snick
+snide
+sniff
+snipe
+snips
+snobs
+snoek
+snood
+snook
+snoop
+snoot
+snore
+snort
+snout
+snows
+snowy
+snubs
+snuck
+snuff
+soaks
+soaps
+soapy
+soars
+sober
+socii
+socks
+socle
+sodas
+sodom
+sofar
+sofas
+sofia
+softy
+soggy
+soils
+solar
+soled
+solei
+soles
+solfa
+solid
+solip
+solon
+solos
+solum
+solus
+solve
+somas
+sonar
+sonde
+sones
+songs
+sonic
+sonny
+sonsy
+sooky
+sooth
+sooty
+sophs
+sopor
+soppy
+sorer
+sores
+sorra
+sorry
+sorts
+sorus
+sotto
+sough
+souls
+sound
+soups
+soupy
+sours
+souse
+south
+sowed
+sower
+space
+spade
+spain
+spake
+spall
+spang
+spank
+spans
+sparc
+spare
+spark
+spars
+spasm
+spate
+spats
+spawn
+spays
+speak
+spear
+speck
+specs
+speed
+speel
+spell
+spelt
+spend
+spent
+sperm
+spews
+spica
+spice
+spick
+spicy
+spied
+spiel
+spier
+spies
+spiff
+spike
+spiky
+spile
+spill
+spilt
+spina
+spine
+spino
+spins
+spiny
+spire
+spirt
+spiry
+spite
+spits
+spitz
+splat
+splay
+split
+spode
+spoil
+spoke
+spoof
+spook
+spool
+spoon
+spoor
+spore
+sport
+spots
+spout
+sprag
+sprat
+spray
+spree
+sprig
+sprit
+sprue
+spuds
+spued
+spume
+spumy
+spunk
+spurn
+spurs
+spurt
+sputa
+squab
+squad
+squat
+squaw
+squib
+squid
+squit
+squiz
+stabs
+stack
+staff
+stage
+stags
+stagy
+staid
+stain
+stair
+stake
+stale
+stalk
+stall
+stamp
+stand
+stank
+staph
+stare
+stark
+stars
+start
+stash
+state
+stats
+statu
+stave
+stays
+stead
+steak
+steal
+steam
+steed
+steel
+steep
+steer
+stein
+stele
+stems
+steno
+stens
+steps
+stere
+stern
+steve
+stews
+stich
+stick
+sties
+stiff
+stile
+stili
+still
+stilt
+sting
+stink
+stint
+stipe
+stirk
+stirp
+stirs
+stoae
+stoat
+stock
+stoep
+stoic
+stoke
+stole
+stoma
+stomp
+stone
+stonk
+stony
+stood
+stook
+stool
+stoop
+stope
+stops
+store
+stork
+storm
+story
+stoss
+stoup
+stout
+stove
+stows
+strad
+strap
+straw
+stray
+strep
+strew
+stria
+strip
+strop
+strum
+strut
+stubs
+stuck
+studs
+study
+stuff
+stull
+stump
+stums
+stung
+stunk
+stuns
+stunt
+stupe
+style
+styli
+suave
+succi
+sucks
+sucre
+sudan
+sudsy
+suede
+suers
+suety
+sugar
+suing
+suint
+suite
+suits
+sulci
+sulfa
+sulfo
+sulks
+sulky
+sully
+sumac
+summa
+sumps
+sunny
+sunup
+super
+supra
+surah
+sural
+surds
+surer
+surfs
+surfy
+surge
+surly
+surra
+susan
+sushi
+sutra
+swabs
+swage
+swags
+swain
+swale
+swami
+swamp
+swang
+swank
+swans
+swaps
+sward
+swarf
+swarm
+swart
+swash
+swath
+swats
+sways
+swear
+sweat
+swede
+sweep
+sweet
+swell
+swelt
+swept
+swift
+swigs
+swill
+swims
+swine
+swing
+swipe
+swirl
+swish
+swiss
+swobs
+swoon
+swoop
+swops
+sword
+swore
+sworn
+swots
+swung
+sylph
+sylva
+synch
+syncs
+synod
+syria
+syrup
+tabby
+tabes
+tabid
+table
+taboo
+tabor
+tabun
+tabus
+tacet
+tache
+tacho
+tachs
+tacit
+tacks
+tacky
+tacos
+taffy
+taiga
+tails
+taint
+taipo
+takas
+taken
+taker
+takes
+takin
+tales
+talks
+talky
+tally
+talon
+talus
+tamed
+tamer
+tames
+tamil
+tammy
+tampa
+tamps
+tanga
+tangi
+tango
+tangs
+tangy
+tanka
+tanks
+tansy
+tanto
+taped
+taper
+tapes
+tapir
+tapis
+tarde
+tardy
+tared
+tares
+tarns
+taros
+tarot
+tarry
+tarsi
+tarts
+tasks
+taste
+tasty
+tatar
+tater
+tatoo
+tatty
+taunt
+taupe
+tawai
+tawed
+tawer
+tawny
+tawse
+taxed
+taxer
+taxes
+taxin
+taxis
+taxon
+teach
+teaks
+teals
+teams
+tears
+teart
+teary
+tease
+teats
+tecta
+tecum
+teddy
+teems
+teens
+teeny
+teeth
+teind
+teing
+telae
+telex
+telia
+telic
+tells
+telly
+tempi
+tempo
+temps
+tempt
+tench
+tendo
+tends
+tenet
+tenia
+tenon
+tenor
+tense
+tenth
+tents
+tenus
+tepal
+tepee
+tepid
+terai
+teras
+terce
+teres
+terga
+tergo
+terms
+terne
+terns
+terra
+terre
+terry
+terse
+terza
+tesla
+testa
+teste
+tests
+testy
+tetra
+texan
+texas
+texis
+texts
+thais
+thana
+thane
+thank
+thats
+thaws
+theca
+theft
+thegn
+their
+theme
+there
+therm
+these
+theta
+thews
+thewy
+thick
+thief
+thigh
+thill
+thine
+thing
+think
+thins
+thiol
+third
+thole
+thong
+thorn
+thoro
+thorp
+those
+three
+threw
+thro'
+throb
+throe
+thron
+throw
+thrum
+thuds
+thugs
+thuja
+thumb
+thump
+thurs
+thyme
+thymi
+thymy
+tiara
+tiber
+tibet
+tibia
+ticks
+tidal
+tided
+tides
+tiers
+tiffs
+tiger
+tight
+tigon
+tilde
+tiled
+tiler
+tiles
+tills
+tilth
+tilts
+timed
+timer
+times
+timid
+tinea
+tined
+tines
+tinge
+tings
+tinny
+tints
+tinty
+tipsy
+tired
+tires
+tiros
+titan
+titer
+tithe
+titis
+title
+titre
+titus
+tizzy
+toads
+toady
+toast
+today
+toddy
+toeas
+togae
+togas
+toile
+toils
+toing
+tokay
+toked
+token
+tokes
+tokyo
+tolan
+tolas
+tolls
+tolyl
+tombs
+tomes
+tommy
+tonal
+tondi
+tondo
+toned
+toner
+tones
+tonga
+tongs
+tonic
+tonka
+tonks
+tonne
+tonus
+tools
+tooth
+toots
+topaz
+toped
+topee
+toper
+topes
+tophi
+topic
+topis
+topoi
+topos
+topsy
+toque
+torah
+torch
+toric
+torii
+torsi
+torsk
+torso
+torte
+torts
+torus
+total
+toted
+totem
+toter
+totes
+touch
+tough
+tours
+touts
+towed
+towel
+tower
+towns
+towny
+toxic
+toxin
+toyed
+toyer
+trace
+track
+tract
+tracy
+trade
+tragi
+trail
+train
+trait
+tramp
+trams
+trans
+traps
+trash
+trass
+trave
+trawl
+trays
+tread
+treat
+treed
+treen
+trees
+treks
+trend
+tress
+trews
+triad
+trial
+trias
+tribe
+trice
+trick
+tried
+trier
+tries
+trigs
+trike
+trill
+trims
+trine
+triol
+trior
+trios
+tripe
+trips
+trite
+trois
+troll
+tromp
+trona
+trons
+troop
+trope
+troth
+trots
+trout
+trove
+truce
+truck
+trued
+truer
+trues
+trugs
+truly
+trump
+trunk
+truss
+trust
+truth
+tryma
+tryst
+tsars
+tuans
+tuart
+tubae
+tubal
+tubas
+tubby
+tubed
+tuber
+tubes
+tucks
+tudor
+tufts
+tufty
+tulip
+tulle
+tulsa
+tumid
+tummy
+tunas
+tuned
+tuner
+tunes
+tunga
+tunic
+tunis
+tunny
+tupik
+tuple
+tuque
+turbo
+turds
+turfs
+turfy
+turin
+turks
+turns
+turps
+turvy
+tusks
+tusky
+tutee
+tutor
+tutti
+tutty
+tutus
+tuxes
+twain
+twang
+tweak
+tweed
+tween
+tweet
+twerp
+twice
+twigs
+twill
+twine
+twins
+twirl
+twirp
+twist
+twits
+tying
+tykes
+tyler
+typal
+typed
+types
+typog
+typos
+tyres
+tyrol
+tyros
+tzars
+uboat
+udder
+uhuru
+ukase
+ulcer
+ulcus
+ulnad
+ulnae
+ulnar
+ulnas
+uloid
+ultra
+umbed
+umbel
+umber
+umbos
+umbra
+umiak
+unage
+unaid
+unaim
+unair
+unapt
+unarm
+unary
+unawe
+unbar
+unbow
+uncal
+uncap
+uncia
+uncle
+unclo
+uncus
+uncut
+under
+undid
+undue
+unfed
+unfit
+unfix
+ungod
+unhar
+unhun
+unify
+union
+unite
+units
+unity
+unlaw
+unlay
+unlet
+unlit
+unman
+unmet
+unmix
+unown
+unpay
+unpeg
+unpin
+unplu
+unrig
+unrip
+unsay
+unsee
+unset
+unsex
+unsun
+untax
+untie
+until
+unvex
+unvow
+unwax
+unwed
+unwon
+unzip
+upcom
+upend
+upish
+upped
+upper
+upset
+upzon
+urate
+urban
+ureal
+urged
+urger
+urges
+urine
+usage
+usees
+users
+usher
+using
+usque
+usual
+usura
+usurp
+usury
+utahn
+uteri
+utero
+utile
+utili
+utrum
+utter
+uveal
+uvula
+uzbeg
+uzbek
+vacua
+vacuo
+vades
+vadis
+vagal
+vague
+vagus
+vakil
+vales
+valet
+valid
+valse
+value
+valva
+valve
+vamps
+vaned
+vanes
+vapid
+varia
+varix
+varum
+varus
+varve
+vasal
+vases
+vasto
+vater
+vatic
+vault
+vaunt
+vaxen
+vedic
+veeps
+veers
+vegan
+vegas
+veils
+veins
+veiny
+velar
+veldt
+velum
+venae
+venal
+vendi
+vends
+venin
+venit
+venom
+vente
+vents
+venue
+venus
+verae
+verba
+verbs
+verde
+verdi
+verge
+versa
+verse
+verso
+verst
+verve
+vesta
+vests
+vetch
+vexed
+vexer
+vexes
+viaes
+vials
+viand
+vibes
+vibex
+vicar
+vices
+vichy
+video
+views
+viewy
+vigil
+viler
+villa
+villi
+vimen
+vinca
+vinci
+vined
+vines
+vinic
+vinyl
+viola
+viols
+viper
+viral
+vireo
+vires
+virga
+virgo
+virtu
+virum
+virus
+visas
+vised
+vises
+visit
+visne
+visor
+vista
+visum
+vitae
+vital
+vitam
+vitro
+vitta
+vitus
+vivax
+vivid
+vivos
+vivre
+vivum
+vixen
+vizor
+vobis
+vocal
+vocat
+voces
+vodka
+voges
+vogue
+voice
+voids
+voila
+voile
+volar
+voles
+volet
+volga
+volta
+volte
+volts
+volva
+vomer
+vomit
+voted
+voter
+votes
+vouch
+vowed
+vowel
+vower
+vults
+vulva
+vying
+wacky
+waddy
+waded
+wader
+wades
+wadis
+wafer
+wafts
+waged
+wager
+wages
+wagga
+wagon
+wahoo
+waifs
+wails
+waist
+waits
+waive
+waked
+waken
+wakes
+waled
+wales
+walks
+walls
+waltz
+wands
+waned
+wanes
+waney
+wanly
+wants
+wards
+wares
+warms
+warns
+warps
+warts
+warty
+washy
+wasps
+waspy
+waste
+wasty
+watch
+water
+watts
+waugh
+waved
+waver
+waves
+waxed
+waxen
+waxer
+waxes
+wayne
+we'll
+we're
+we've
+weald
+weals
+weans
+wears
+weary
+weave
+webby
+weber
+wedge
+wedgy
+weeds
+weedy
+weeks
+weeny
+weeps
+weepy
+weest
+wefts
+weigh
+weird
+weirs


<TRUNCATED>
http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/docs/Apache Kylin 2014 Dec.pdf
----------------------------------------------------------------------
diff --git a/docs/Apache Kylin 2014 Dec.pdf b/docs/Apache Kylin 2014 Dec.pdf
new file mode 100644
index 0000000..bf398ba
Binary files /dev/null and b/docs/Apache Kylin 2014 Dec.pdf differ


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

Posted by lu...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/common/src/main/java/com/kylinolap/common/hll/HyperLogLogPlusTable.java
----------------------------------------------------------------------
diff --git a/common/src/main/java/com/kylinolap/common/hll/HyperLogLogPlusTable.java b/common/src/main/java/com/kylinolap/common/hll/HyperLogLogPlusTable.java
new file mode 100644
index 0000000..82811c3
--- /dev/null
+++ b/common/src/main/java/com/kylinolap/common/hll/HyperLogLogPlusTable.java
@@ -0,0 +1,197 @@
+/*
+ * 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.common.hll;
+
+import java.util.SortedMap;
+import java.util.TreeMap;
+
+/**
+ * @author xjiang
+ * 
+ */
+public class HyperLogLogPlusTable {
+
+    // threshold and bias data taken from google's bias correction data set:
+    // https://docs.google.com/document/d/1gyjfMHy43U9OWBXxfaeG-3MjGzejW1dlpyMwEYAAWEI/view?fullscreen#
+    public static final double[] thresholdData = { 10, 20, 40, 80, 220, 400, 900, 1800, 3100, 6500, 15500, 20000, 50000, 120000, 350000 };
+
+    public static final double[][] rawEstimateData = {
+            // precision 4
+            { 11, 11.717, 12.207, 12.7896, 13.2882, 13.8204, 14.3772, 14.9342, 15.5202, 16.161, 16.7722, 17.4636, 18.0396, 18.6766, 19.3566, 20.0454, 20.7936, 21.4856, 22.2666, 22.9946, 23.766, 24.4692, 25.3638, 26.0764, 26.7864, 27.7602, 28.4814, 29.433, 30.2926, 31.0664, 31.9996, 32.7956, 33.5366, 34.5894, 35.5738, 36.2698, 37.3682, 38.0544, 39.2342, 40.0108, 40.7966, 41.9298, 42.8704, 43.6358, 44.5194, 45.773, 46.6772, 47.6174, 48.4888, 49.3304, 50.2506, 51.4996, 52.3824, 53.3078, 54.3984, 55.5838, 56.6618, 57.2174, 58.3514, 59.0802, 60.1482, 61.0376, 62.3598, 62.8078, 63.9744, 64.914, 65.781, 67.1806, 68.0594, 68.8446, 69.7928, 70.8248, 71.8324, 72.8598, 73.6246, 74.7014, 75.393, 76.6708, 77.2394, },
+            // precision 5
+            { 23, 23.1194, 23.8208, 24.2318, 24.77, 25.2436, 25.7774, 26.2848, 26.8224, 27.3742, 27.9336, 28.503, 29.0494, 29.6292, 30.2124, 30.798, 31.367, 31.9728, 32.5944, 33.217, 33.8438, 34.3696, 35.0956, 35.7044, 36.324, 37.0668, 37.6698, 38.3644, 39.049, 39.6918, 40.4146, 41.082, 41.687, 42.5398, 43.2462, 43.857, 44.6606, 45.4168, 46.1248, 46.9222, 47.6804, 48.447, 49.3454, 49.9594, 50.7636, 51.5776, 52.331, 53.19, 53.9676, 54.7564, 55.5314, 56.4442, 57.3708, 57.9774, 58.9624, 59.8796, 60.755, 61.472, 62.2076, 63.1024, 63.8908, 64.7338, 65.7728, 66.629, 67.413, 68.3266, 69.1524, 70.2642, 71.1806, 72.0566, 72.9192, 73.7598, 74.3516, 75.5802, 76.4386, 77.4916, 78.1524, 79.1892, 79.8414, 80.8798, 81.8376, 82.4698, 83.7656, 84.331, 85.5914, 86.6012, 87.7016, 88.5582, 89.3394, 90.3544, 91.4912, 92.308, 93.3552, 93.9746, 95.2052, 95.727, 97.1322, 98.3944, 98.7588, 100.242, 101.1914, 102.2538, 102.8776, 103.6292, 105.1932, 105.9152, 107.0868, 107.6728, 108.7144, 110.3114, 110.8716,
+                    111.245, 112.7908, 113.7064, 114.636, 115.7464, 116.1788, 117.7464, 118.4896, 119.6166, 120.5082, 121.7798, 122.9028, 123.4426, 124.8854, 125.705, 126.4652, 128.3464, 128.3462, 130.0398, 131.0342, 131.0042, 132.4766, 133.511, 134.7252, 135.425, 136.5172, 138.0572, 138.6694, 139.3712, 140.8598, 141.4594, 142.554, 143.4006, 144.7374, 146.1634, 146.8994, 147.605, 147.9304, 149.1636, 150.2468, 151.5876, 152.2096, 153.7032, 154.7146, 155.807, 156.9228, 157.0372, 158.5852, },
+            // precision 6
+            { 46, 46.1902, 47.271, 47.8358, 48.8142, 49.2854, 50.317, 51.354, 51.8924, 52.9436, 53.4596, 54.5262, 55.6248, 56.1574, 57.2822, 57.837, 58.9636, 60.074, 60.7042, 61.7976, 62.4772, 63.6564, 64.7942, 65.5004, 66.686, 67.291, 68.5672, 69.8556, 70.4982, 71.8204, 72.4252, 73.7744, 75.0786, 75.8344, 77.0294, 77.8098, 79.0794, 80.5732, 81.1878, 82.5648, 83.2902, 84.6784, 85.3352, 86.8946, 88.3712, 89.0852, 90.499, 91.2686, 92.6844, 94.2234, 94.9732, 96.3356, 97.2286, 98.7262, 100.3284, 101.1048, 102.5962, 103.3562, 105.1272, 106.4184, 107.4974, 109.0822, 109.856, 111.48, 113.2834, 114.0208, 115.637, 116.5174, 118.0576, 119.7476, 120.427, 122.1326, 123.2372, 125.2788, 126.6776, 127.7926, 129.1952, 129.9564, 131.6454, 133.87, 134.5428, 136.2, 137.0294, 138.6278, 139.6782, 141.792, 143.3516, 144.2832, 146.0394, 147.0748, 148.4912, 150.849, 151.696, 153.5404, 154.073, 156.3714, 157.7216, 158.7328, 160.4208, 161.4184, 163.9424, 165.2772, 166.411, 168.1308, 168.769, 170.9258,
+                    172.6828, 173.7502, 175.706, 176.3886, 179.0186, 180.4518, 181.927, 183.4172, 184.4114, 186.033, 188.5124, 189.5564, 191.6008, 192.4172, 193.8044, 194.997, 197.4548, 198.8948, 200.2346, 202.3086, 203.1548, 204.8842, 206.6508, 206.6772, 209.7254, 210.4752, 212.7228, 214.6614, 215.1676, 217.793, 218.0006, 219.9052, 221.66, 223.5588, 225.1636, 225.6882, 227.7126, 229.4502, 231.1978, 232.9756, 233.1654, 236.727, 238.1974, 237.7474, 241.1346, 242.3048, 244.1948, 245.3134, 246.879, 249.1204, 249.853, 252.6792, 253.857, 254.4486, 257.2362, 257.9534, 260.0286, 260.5632, 262.663, 264.723, 265.7566, 267.2566, 267.1624, 270.62, 272.8216, 273.2166, 275.2056, 276.2202, 278.3726, 280.3344, 281.9284, 283.9728, 284.1924, 286.4872, 287.587, 289.807, 291.1206, 292.769, 294.8708, 296.665, 297.1182, 299.4012, 300.6352, 302.1354, 304.1756, 306.1606, 307.3462, 308.5214, 309.4134, 310.8352, 313.9684, 315.837, 316.7796, 318.9858, },
+            // precision 7
+            { 92, 93.4934, 94.9758, 96.4574, 97.9718, 99.4954, 101.5302, 103.0756, 104.6374, 106.1782, 107.7888, 109.9522, 111.592, 113.2532, 114.9086, 116.5938, 118.9474, 120.6796, 122.4394, 124.2176, 125.9768, 128.4214, 130.2528, 132.0102, 133.8658, 135.7278, 138.3044, 140.1316, 142.093, 144.0032, 145.9092, 148.6306, 150.5294, 152.5756, 154.6508, 156.662, 159.552, 161.3724, 163.617, 165.5754, 167.7872, 169.8444, 172.7988, 174.8606, 177.2118, 179.3566, 181.4476, 184.5882, 186.6816, 189.0824, 191.0258, 193.6048, 196.4436, 198.7274, 200.957, 203.147, 205.4364, 208.7592, 211.3386, 213.781, 215.8028, 218.656, 221.6544, 223.996, 226.4718, 229.1544, 231.6098, 234.5956, 237.0616, 239.5758, 242.4878, 244.5244, 248.2146, 250.724, 252.8722, 255.5198, 258.0414, 261.941, 264.9048, 266.87, 269.4304, 272.028, 274.4708, 278.37, 281.0624, 283.4668, 286.5532, 289.4352, 293.2564, 295.2744, 298.2118, 300.7472, 304.1456, 307.2928, 309.7504, 312.5528, 315.979, 318.2102, 322.1834, 324.3494, 327.325,
+                    330.6614, 332.903, 337.2544, 339.9042, 343.215, 345.2864, 348.0814, 352.6764, 355.301, 357.139, 360.658, 363.1732, 366.5902, 369.9538, 373.0828, 375.922, 378.9902, 382.7328, 386.4538, 388.1136, 391.2234, 394.0878, 396.708, 401.1556, 404.1852, 406.6372, 409.6822, 412.7796, 416.6078, 418.4916, 422.131, 424.5376, 428.1988, 432.211, 434.4502, 438.5282, 440.912, 444.0448, 447.7432, 450.8524, 453.7988, 456.7858, 458.8868, 463.9886, 466.5064, 468.9124, 472.6616, 475.4682, 478.582, 481.304, 485.2738, 488.6894, 490.329, 496.106, 497.6908, 501.1374, 504.5322, 506.8848, 510.3324, 513.4512, 516.179, 520.4412, 522.6066, 526.167, 528.7794, 533.379, 536.067, 538.46, 542.9116, 545.692, 547.9546, 552.493, 555.2722, 557.335, 562.449, 564.2014, 569.0738, 571.0974, 574.8564, 578.2996, 581.409, 583.9704, 585.8098, 589.6528, 594.5998, 595.958, 600.068, 603.3278, 608.2016, 609.9632, 612.864, 615.43, 620.7794, 621.272, 625.8644, 629.206, 633.219, 634.5154, 638.6102, },
+            // precision 8
+            { 184.2152, 187.2454, 190.2096, 193.6652, 196.6312, 199.6822, 203.249, 206.3296, 210.0038, 213.2074, 216.4612, 220.27, 223.5178, 227.4412, 230.8032, 234.1634, 238.1688, 241.6074, 245.6946, 249.2664, 252.8228, 257.0432, 260.6824, 264.9464, 268.6268, 272.2626, 276.8376, 280.4034, 284.8956, 288.8522, 292.7638, 297.3552, 301.3556, 305.7526, 309.9292, 313.8954, 318.8198, 322.7668, 327.298, 331.6688, 335.9466, 340.9746, 345.1672, 349.3474, 354.3028, 358.8912, 364.114, 368.4646, 372.9744, 378.4092, 382.6022, 387.843, 392.5684, 397.1652, 402.5426, 407.4152, 412.5388, 417.3592, 422.1366, 427.486, 432.3918, 437.5076, 442.509, 447.3834, 453.3498, 458.0668, 463.7346, 469.1228, 473.4528, 479.7, 484.644, 491.0518, 495.5774, 500.9068, 506.432, 512.1666, 517.434, 522.6644, 527.4894, 533.6312, 538.3804, 544.292, 550.5496, 556.0234, 562.8206, 566.6146, 572.4188, 579.117, 583.6762, 590.6576, 595.7864, 601.509, 607.5334, 612.9204, 619.772, 624.2924, 630.8654, 636.1836, 642.745, 649.1316,
+                    655.0386, 660.0136, 666.6342, 671.6196, 678.1866, 684.4282, 689.3324, 695.4794, 702.5038, 708.129, 713.528, 720.3204, 726.463, 732.7928, 739.123, 744.7418, 751.2192, 756.5102, 762.6066, 769.0184, 775.2224, 781.4014, 787.7618, 794.1436, 798.6506, 805.6378, 811.766, 819.7514, 824.5776, 828.7322, 837.8048, 843.6302, 849.9336, 854.4798, 861.3388, 867.9894, 873.8196, 880.3136, 886.2308, 892.4588, 899.0816, 905.4076, 912.0064, 917.3878, 923.619, 929.998, 937.3482, 943.9506, 947.991, 955.1144, 962.203, 968.8222, 975.7324, 981.7826, 988.7666, 994.2648, 1000.3128, 1007.4082, 1013.7536, 1020.3376, 1026.7156, 1031.7478, 1037.4292, 1045.393, 1051.2278, 1058.3434, 1062.8726, 1071.884, 1076.806, 1082.9176, 1089.1678, 1095.5032, 1102.525, 1107.2264, 1115.315, 1120.93, 1127.252, 1134.1496, 1139.0408, 1147.5448, 1153.3296, 1158.1974, 1166.5262, 1174.3328, 1175.657, 1184.4222, 1190.9172, 1197.1292, 1204.4606, 1210.4578, 1218.8728, 1225.3336, 1226.6592, 1236.5768, 1241.363,
+                    1249.4074, 1254.6566, 1260.8014, 1266.5454, 1274.5192, },
+            // precision 9
+            { 369, 374.8294, 381.2452, 387.6698, 394.1464, 400.2024, 406.8782, 413.6598, 420.462, 427.2826, 433.7102, 440.7416, 447.9366, 455.1046, 462.285, 469.0668, 476.306, 483.8448, 491.301, 498.9886, 506.2422, 513.8138, 521.7074, 529.7428, 537.8402, 545.1664, 553.3534, 561.594, 569.6886, 577.7876, 585.65, 594.228, 602.8036, 611.1666, 620.0818, 628.0824, 637.2574, 646.302, 655.1644, 664.0056, 672.3802, 681.7192, 690.5234, 700.2084, 708.831, 718.485, 728.1112, 737.4764, 746.76, 756.3368, 766.5538, 775.5058, 785.2646, 795.5902, 804.3818, 814.8998, 824.9532, 835.2062, 845.2798, 854.4728, 864.9582, 875.3292, 886.171, 896.781, 906.5716, 916.7048, 927.5322, 937.875, 949.3972, 958.3464, 969.7274, 980.2834, 992.1444, 1003.4264, 1013.0166, 1024.018, 1035.0438, 1046.34, 1057.6856, 1068.9836, 1079.0312, 1091.677, 1102.3188, 1113.4846, 1124.4424, 1135.739, 1147.1488, 1158.9202, 1169.406, 1181.5342, 1193.2834, 1203.8954, 1216.3286, 1226.2146, 1239.6684, 1251.9946, 1262.123, 1275.4338,
+                    1285.7378, 1296.076, 1308.9692, 1320.4964, 1333.0998, 1343.9864, 1357.7754, 1368.3208, 1380.4838, 1392.7388, 1406.0758, 1416.9098, 1428.9728, 1440.9228, 1453.9292, 1462.617, 1476.05, 1490.2996, 1500.6128, 1513.7392, 1524.5174, 1536.6322, 1548.2584, 1562.3766, 1572.423, 1587.1232, 1596.5164, 1610.5938, 1622.5972, 1633.1222, 1647.7674, 1658.5044, 1671.57, 1683.7044, 1695.4142, 1708.7102, 1720.6094, 1732.6522, 1747.841, 1756.4072, 1769.9786, 1782.3276, 1797.5216, 1808.3186, 1819.0694, 1834.354, 1844.575, 1856.2808, 1871.1288, 1880.7852, 1893.9622, 1906.3418, 1920.6548, 1932.9302, 1945.8584, 1955.473, 1968.8248, 1980.6446, 1995.9598, 2008.349, 2019.8556, 2033.0334, 2044.0206, 2059.3956, 2069.9174, 2082.6084, 2093.7036, 2106.6108, 2118.9124, 2132.301, 2144.7628, 2159.8422, 2171.0212, 2183.101, 2193.5112, 2208.052, 2221.3194, 2233.3282, 2247.295, 2257.7222, 2273.342, 2286.5638, 2299.6786, 2310.8114, 2322.3312, 2335.516, 2349.874, 2363.5968, 2373.865, 2387.1918,
+                    2401.8328, 2414.8496, 2424.544, 2436.7592, 2447.1682, 2464.1958, 2474.3438, 2489.0006, 2497.4526, 2513.6586, 2527.19, 2540.7028, 2553.768, },
+            // precision 10
+            { 738.1256, 750.4234, 763.1064, 775.4732, 788.4636, 801.0644, 814.488, 827.9654, 841.0832, 854.7864, 868.1992, 882.2176, 896.5228, 910.1716, 924.7752, 938.899, 953.6126, 968.6492, 982.9474, 998.5214, 1013.1064, 1028.6364, 1044.2468, 1059.4588, 1075.3832, 1091.0584, 1106.8606, 1123.3868, 1139.5062, 1156.1862, 1172.463, 1189.339, 1206.1936, 1223.1292, 1240.1854, 1257.2908, 1275.3324, 1292.8518, 1310.5204, 1328.4854, 1345.9318, 1364.552, 1381.4658, 1400.4256, 1419.849, 1438.152, 1456.8956, 1474.8792, 1494.118, 1513.62, 1532.5132, 1551.9322, 1570.7726, 1590.6086, 1610.5332, 1630.5918, 1650.4294, 1669.7662, 1690.4106, 1710.7338, 1730.9012, 1750.4486, 1770.1556, 1791.6338, 1812.7312, 1833.6264, 1853.9526, 1874.8742, 1896.8326, 1918.1966, 1939.5594, 1961.07, 1983.037, 2003.1804, 2026.071, 2047.4884, 2070.0848, 2091.2944, 2114.333, 2135.9626, 2158.2902, 2181.0814, 2202.0334, 2224.4832, 2246.39, 2269.7202, 2292.1714, 2314.2358, 2338.9346, 2360.891, 2384.0264, 2408.3834, 2430.1544
 ,
+                    2454.8684, 2476.9896, 2501.4368, 2522.8702, 2548.0408, 2570.6738, 2593.5208, 2617.0158, 2640.2302, 2664.0962, 2687.4986, 2714.2588, 2735.3914, 2759.6244, 2781.8378, 2808.0072, 2830.6516, 2856.2454, 2877.2136, 2903.4546, 2926.785, 2951.2294, 2976.468, 3000.867, 3023.6508, 3049.91, 3073.5984, 3098.162, 3121.5564, 3146.2328, 3170.9484, 3195.5902, 3221.3346, 3242.7032, 3271.6112, 3296.5546, 3317.7376, 3345.072, 3369.9518, 3394.326, 3418.1818, 3444.6926, 3469.086, 3494.2754, 3517.8698, 3544.248, 3565.3768, 3588.7234, 3616.979, 3643.7504, 3668.6812, 3695.72, 3719.7392, 3742.6224, 3770.4456, 3795.6602, 3819.9058, 3844.002, 3869.517, 3895.6824, 3920.8622, 3947.1364, 3973.985, 3995.4772, 4021.62, 4046.628, 4074.65, 4096.2256, 4121.831, 4146.6406, 4173.276, 4195.0744, 4223.9696, 4251.3708, 4272.9966, 4300.8046, 4326.302, 4353.1248, 4374.312, 4403.0322, 4426.819, 4450.0598, 4478.5206, 4504.8116, 4528.8928, 4553.9584, 4578.8712, 4603.8384, 4632.3872, 4655.5128, 4675.821,
+                    4704.6222, 4731.9862, 4755.4174, 4781.2628, 4804.332, 4832.3048, 4862.8752, 4883.4148, 4906.9544, 4935.3516, 4954.3532, 4984.0248, 5011.217, 5035.3258, 5057.3672, 5084.1828, },
+            // precision 11
+            { 1477, 1501.6014, 1526.5802, 1551.7942, 1577.3042, 1603.2062, 1629.8402, 1656.2292, 1682.9462, 1709.9926, 1737.3026, 1765.4252, 1793.0578, 1821.6092, 1849.626, 1878.5568, 1908.527, 1937.5154, 1967.1874, 1997.3878, 2027.37, 2058.1972, 2089.5728, 2120.1012, 2151.9668, 2183.292, 2216.0772, 2247.8578, 2280.6562, 2313.041, 2345.714, 2380.3112, 2414.1806, 2447.9854, 2481.656, 2516.346, 2551.5154, 2586.8378, 2621.7448, 2656.6722, 2693.5722, 2729.1462, 2765.4124, 2802.8728, 2838.898, 2876.408, 2913.4926, 2951.4938, 2989.6776, 3026.282, 3065.7704, 3104.1012, 3143.7388, 3181.6876, 3221.1872, 3261.5048, 3300.0214, 3339.806, 3381.409, 3421.4144, 3461.4294, 3502.2286, 3544.651, 3586.6156, 3627.337, 3670.083, 3711.1538, 3753.5094, 3797.01, 3838.6686, 3882.1678, 3922.8116, 3967.9978, 4009.9204, 4054.3286, 4097.5706, 4140.6014, 4185.544, 4229.5976, 4274.583, 4316.9438, 4361.672, 4406.2786, 4451.8628, 4496.1834, 4543.505, 4589.1816, 4632.5188, 4678.2294, 4724.8908, 4769.0194, 4817.052,
+                    4861.4588, 4910.1596, 4956.4344, 5002.5238, 5048.13, 5093.6374, 5142.8162, 5187.7894, 5237.3984, 5285.6078, 5331.0858, 5379.1036, 5428.6258, 5474.6018, 5522.7618, 5571.5822, 5618.59, 5667.9992, 5714.88, 5763.454, 5808.6982, 5860.3644, 5910.2914, 5953.571, 6005.9232, 6055.1914, 6104.5882, 6154.5702, 6199.7036, 6251.1764, 6298.7596, 6350.0302, 6398.061, 6448.4694, 6495.933, 6548.0474, 6597.7166, 6646.9416, 6695.9208, 6742.6328, 6793.5276, 6842.1934, 6894.2372, 6945.3864, 6996.9228, 7044.2372, 7094.1374, 7142.2272, 7192.2942, 7238.8338, 7288.9006, 7344.0908, 7394.8544, 7443.5176, 7490.4148, 7542.9314, 7595.6738, 7641.9878, 7694.3688, 7743.0448, 7797.522, 7845.53, 7899.594, 7950.3132, 7996.455, 8050.9442, 8092.9114, 8153.1374, 8197.4472, 8252.8278, 8301.8728, 8348.6776, 8401.4698, 8453.551, 8504.6598, 8553.8944, 8604.1276, 8657.6514, 8710.3062, 8758.908, 8807.8706, 8862.1702, 8910.4668, 8960.77, 9007.2766, 9063.164, 9121.0534, 9164.1354, 9218.1594, 9267.767,
+                    9319.0594, 9372.155, 9419.7126, 9474.3722, 9520.1338, 9572.368, 9622.7702, 9675.8448, 9726.5396, 9778.7378, 9827.6554, 9878.1922, 9928.7782, 9978.3984, 10026.578, 10076.5626, 10137.1618, 10177.5244, 10229.9176, },
+            // precision 12
+            { 2954, 3003.4782, 3053.3568, 3104.3666, 3155.324, 3206.9598, 3259.648, 3312.539, 3366.1474, 3420.2576, 3474.8376, 3530.6076, 3586.451, 3643.38, 3700.4104, 3757.5638, 3815.9676, 3875.193, 3934.838, 3994.8548, 4055.018, 4117.1742, 4178.4482, 4241.1294, 4304.4776, 4367.4044, 4431.8724, 4496.3732, 4561.4304, 4627.5326, 4693.949, 4761.5532, 4828.7256, 4897.6182, 4965.5186, 5034.4528, 5104.865, 5174.7164, 5244.6828, 5316.6708, 5387.8312, 5459.9036, 5532.476, 5604.8652, 5679.6718, 5753.757, 5830.2072, 5905.2828, 5980.0434, 6056.6264, 6134.3192, 6211.5746, 6290.0816, 6367.1176, 6447.9796, 6526.5576, 6606.1858, 6686.9144, 6766.1142, 6847.0818, 6927.9664, 7010.9096, 7091.0816, 7175.3962, 7260.3454, 7344.018, 7426.4214, 7511.3106, 7596.0686, 7679.8094, 7765.818, 7852.4248, 7936.834, 8022.363, 8109.5066, 8200.4554, 8288.5832, 8373.366, 8463.4808, 8549.7682, 8642.0522, 8728.3288, 8820.9528, 8907.727, 9001.0794, 9091.2522, 9179.988, 9269.852, 9362.6394, 9453.642, 9546.9024, 9640.6616
 ,
+                    9732.6622, 9824.3254, 9917.7484, 10007.9392, 10106.7508, 10196.2152, 10289.8114, 10383.5494, 10482.3064, 10576.8734, 10668.7872, 10764.7156, 10862.0196, 10952.793, 11049.9748, 11146.0702, 11241.4492, 11339.2772, 11434.2336, 11530.741, 11627.6136, 11726.311, 11821.5964, 11918.837, 12015.3724, 12113.0162, 12213.0424, 12306.9804, 12408.4518, 12504.8968, 12604.586, 12700.9332, 12798.705, 12898.5142, 12997.0488, 13094.788, 13198.475, 13292.7764, 13392.9698, 13486.8574, 13590.1616, 13686.5838, 13783.6264, 13887.2638, 13992.0978, 14081.0844, 14189.9956, 14280.0912, 14382.4956, 14486.4384, 14588.1082, 14686.2392, 14782.276, 14888.0284, 14985.1864, 15088.8596, 15187.0998, 15285.027, 15383.6694, 15495.8266, 15591.3736, 15694.2008, 15790.3246, 15898.4116, 15997.4522, 16095.5014, 16198.8514, 16291.7492, 16402.6424, 16499.1266, 16606.2436, 16697.7186, 16796.3946, 16902.3376, 17005.7672, 17100.814, 17206.8282, 17305.8262, 17416.0744, 17508.4092, 17617.0178, 17715.4554,
+                    17816.758, 17920.1748, 18012.9236, 18119.7984, 18223.2248, 18324.2482, 18426.6276, 18525.0932, 18629.8976, 18733.2588, 18831.0466, 18940.1366, 19032.2696, 19131.729, 19243.4864, 19349.6932, 19442.866, 19547.9448, 19653.2798, 19754.4034, 19854.0692, 19965.1224, 20065.1774, 20158.2212, 20253.353, 20366.3264, 20463.22, },
+            // precision 13
+            { 5908.5052, 6007.2672, 6107.347, 6208.5794, 6311.2622, 6414.5514, 6519.3376, 6625.6952, 6732.5988, 6841.3552, 6950.5972, 7061.3082, 7173.5646, 7287.109, 7401.8216, 7516.4344, 7633.3802, 7751.2962, 7870.3784, 7990.292, 8110.79, 8233.4574, 8356.6036, 8482.2712, 8607.7708, 8735.099, 8863.1858, 8993.4746, 9123.8496, 9255.6794, 9388.5448, 9522.7516, 9657.3106, 9792.6094, 9930.5642, 10068.794, 10206.7256, 10347.81, 10490.3196, 10632.0778, 10775.9916, 10920.4662, 11066.124, 11213.073, 11358.0362, 11508.1006, 11659.1716, 11808.7514, 11959.4884, 12112.1314, 12265.037, 12420.3756, 12578.933, 12734.311, 12890.0006, 13047.2144, 13207.3096, 13368.5144, 13528.024, 13689.847, 13852.7528, 14018.3168, 14180.5372, 14346.9668, 14513.5074, 14677.867, 14846.2186, 15017.4186, 15184.9716, 15356.339, 15529.2972, 15697.3578, 15871.8686, 16042.187, 16216.4094, 16389.4188, 16565.9126, 16742.3272, 16919.0042, 17094.7592, 17273.965, 17451.8342, 17634.4254, 17810.5984, 17988.9242, 18171.051,
+                    18354.7938, 18539.466, 18721.0408, 18904.9972, 19081.867, 19271.9118, 19451.8694, 19637.9816, 19821.2922, 20013.1292, 20199.3858, 20387.8726, 20572.9514, 20770.7764, 20955.1714, 21144.751, 21329.9952, 21520.709, 21712.7016, 21906.3868, 22096.2626, 22286.0524, 22475.051, 22665.5098, 22862.8492, 23055.5294, 23249.6138, 23437.848, 23636.273, 23826.093, 24020.3296, 24213.3896, 24411.7392, 24602.9614, 24805.7952, 24998.1552, 25193.9588, 25389.0166, 25585.8392, 25780.6976, 25981.2728, 26175.977, 26376.5252, 26570.1964, 26773.387, 26962.9812, 27163.0586, 27368.164, 27565.0534, 27758.7428, 27961.1276, 28163.2324, 28362.3816, 28565.7668, 28758.644, 28956.9768, 29163.4722, 29354.7026, 29561.1186, 29767.9948, 29959.9986, 30164.0492, 30366.9818, 30562.5338, 30762.9928, 30976.1592, 31166.274, 31376.722, 31570.3734, 31770.809, 31974.8934, 32179.5286, 32387.5442, 32582.3504, 32794.076, 32989.9528, 33191.842, 33392.4684, 33595.659, 33801.8672, 34000.3414, 34200.0922, 34402.6792,
+                    34610.0638, 34804.0084, 35011.13, 35218.669, 35418.6634, 35619.0792, 35830.6534, 36028.4966, 36229.7902, 36438.6422, 36630.7764, 36833.3102, 37048.6728, 37247.3916, 37453.5904, 37669.3614, 37854.5526, 38059.305, 38268.0936, 38470.2516, 38674.7064, 38876.167, 39068.3794, 39281.9144, 39492.8566, 39684.8628, 39898.4108, 40093.1836, 40297.6858, 40489.7086, 40717.2424, },
+            // precision 14
+            { 11817.475, 12015.0046, 12215.3792, 12417.7504, 12623.1814, 12830.0086, 13040.0072, 13252.503, 13466.178, 13683.2738, 13902.0344, 14123.9798, 14347.394, 14573.7784, 14802.6894, 15033.6824, 15266.9134, 15502.8624, 15741.4944, 15980.7956, 16223.8916, 16468.6316, 16715.733, 16965.5726, 17217.204, 17470.666, 17727.8516, 17986.7886, 18247.6902, 18510.9632, 18775.304, 19044.7486, 19314.4408, 19587.202, 19862.2576, 20135.924, 20417.0324, 20697.9788, 20979.6112, 21265.0274, 21550.723, 21841.6906, 22132.162, 22428.1406, 22722.127, 23020.5606, 23319.7394, 23620.4014, 23925.2728, 24226.9224, 24535.581, 24845.505, 25155.9618, 25470.3828, 25785.9702, 26103.7764, 26420.4132, 26742.0186, 27062.8852, 27388.415, 27714.6024, 28042.296, 28365.4494, 28701.1526, 29031.8008, 29364.2156, 29704.497, 30037.1458, 30380.111, 30723.8168, 31059.5114, 31404.9498, 31751.6752, 32095.2686, 32444.7792, 32794.767, 33145.204, 33498.4226, 33847.6502, 34209.006, 34560.849, 34919.4838, 35274.9778, 35635.1322
 ,
+                    35996.3266, 36359.1394, 36722.8266, 37082.8516, 37447.7354, 37815.9606, 38191.0692, 38559.4106, 38924.8112, 39294.6726, 39663.973, 40042.261, 40416.2036, 40779.2036, 41161.6436, 41540.9014, 41921.1998, 42294.7698, 42678.5264, 43061.3464, 43432.375, 43818.432, 44198.6598, 44583.0138, 44970.4794, 45353.924, 45729.858, 46118.2224, 46511.5724, 46900.7386, 47280.6964, 47668.1472, 48055.6796, 48446.9436, 48838.7146, 49217.7296, 49613.7796, 50010.7508, 50410.0208, 50793.7886, 51190.2456, 51583.1882, 51971.0796, 52376.5338, 52763.319, 53165.5534, 53556.5594, 53948.2702, 54346.352, 54748.7914, 55138.577, 55543.4824, 55941.1748, 56333.7746, 56745.1552, 57142.7944, 57545.2236, 57935.9956, 58348.5268, 58737.5474, 59158.5962, 59542.6896, 59958.8004, 60349.3788, 60755.0212, 61147.6144, 61548.194, 61946.0696, 62348.6042, 62763.603, 63162.781, 63560.635, 63974.3482, 64366.4908, 64771.5876, 65176.7346, 65597.3916, 65995.915, 66394.0384, 66822.9396, 67203.6336, 67612.2032,
+                    68019.0078, 68420.0388, 68821.22, 69235.8388, 69640.0724, 70055.155, 70466.357, 70863.4266, 71276.2482, 71677.0306, 72080.2006, 72493.0214, 72893.5952, 73314.5856, 73714.9852, 74125.3022, 74521.2122, 74933.6814, 75341.5904, 75743.0244, 76166.0278, 76572.1322, 76973.1028, 77381.6284, 77800.6092, 78189.328, 78607.0962, 79012.2508, 79407.8358, 79825.725, 80238.701, 80646.891, 81035.6436, 81460.0448, 81876.3884, },
+            // precision 15
+            { 23635.0036, 24030.8034, 24431.4744, 24837.1524, 25246.7928, 25661.326, 26081.3532, 26505.2806, 26933.9892, 27367.7098, 27805.318, 28248.799, 28696.4382, 29148.8244, 29605.5138, 30066.8668, 30534.2344, 31006.32, 31480.778, 31962.2418, 32447.3324, 32938.0232, 33432.731, 33930.728, 34433.9896, 34944.1402, 35457.5588, 35974.5958, 36497.3296, 37021.9096, 37554.326, 38088.0826, 38628.8816, 39171.3192, 39723.2326, 40274.5554, 40832.3142, 41390.613, 41959.5908, 42532.5466, 43102.0344, 43683.5072, 44266.694, 44851.2822, 45440.7862, 46038.0586, 46640.3164, 47241.064, 47846.155, 48454.7396, 49076.9168, 49692.542, 50317.4778, 50939.65, 51572.5596, 52210.2906, 52843.7396, 53481.3996, 54127.236, 54770.406, 55422.6598, 56078.7958, 56736.7174, 57397.6784, 58064.5784, 58730.308, 59404.9784, 60077.0864, 60751.9158, 61444.1386, 62115.817, 62808.7742, 63501.4774, 64187.5454, 64883.6622, 65582.7468, 66274.5318, 66976.9276, 67688.7764, 68402.138, 69109.6274, 69822.9706, 70543.6108,
+                    71265.5202, 71983.3848, 72708.4656, 73433.384, 74158.4664, 74896.4868, 75620.9564, 76362.1434, 77098.3204, 77835.7662, 78582.6114, 79323.9902, 80067.8658, 80814.9246, 81567.0136, 82310.8536, 83061.9952, 83821.4096, 84580.8608, 85335.547, 86092.5802, 86851.6506, 87612.311, 88381.2016, 89146.3296, 89907.8974, 90676.846, 91451.4152, 92224.5518, 92995.8686, 93763.5066, 94551.2796, 95315.1944, 96096.1806, 96881.0918, 97665.679, 98442.68, 99229.3002, 100011.0994, 100790.6386, 101580.1564, 102377.7484, 103152.1392, 103944.2712, 104730.216, 105528.6336, 106324.9398, 107117.6706, 107890.3988, 108695.2266, 109485.238, 110294.7876, 111075.0958, 111878.0496, 112695.2864, 113464.5486, 114270.0474, 115068.608, 115884.3626, 116673.2588, 117483.3716, 118275.097, 119085.4092, 119879.2808, 120687.5868, 121499.9944, 122284.916, 123095.9254, 123912.5038, 124709.0454, 125503.7182, 126323.259, 127138.9412, 127943.8294, 128755.646, 129556.5354, 130375.3298, 131161.4734, 131971.1962,
+                    132787.5458, 133588.1056, 134431.351, 135220.2906, 136023.398, 136846.6558, 137667.0004, 138463.663, 139283.7154, 140074.6146, 140901.3072, 141721.8548, 142543.2322, 143356.1096, 144173.7412, 144973.0948, 145794.3162, 146609.5714, 147420.003, 148237.9784, 149050.5696, 149854.761, 150663.1966, 151494.0754, 152313.1416, 153112.6902, 153935.7206, 154746.9262, 155559.547, 156401.9746, 157228.7036, 158008.7254, 158820.75, 159646.9184, 160470.4458, 161279.5348, 162093.3114, 162918.542, 163729.2842, },
+            // precision 16
+            { 47271, 48062.3584, 48862.7074, 49673.152, 50492.8416, 51322.9514, 52161.03, 53009.407, 53867.6348, 54734.206, 55610.5144, 56496.2096, 57390.795, 58297.268, 59210.6448, 60134.665, 61068.0248, 62010.4472, 62962.5204, 63923.5742, 64895.0194, 65876.4182, 66862.6136, 67862.6968, 68868.8908, 69882.8544, 70911.271, 71944.0924, 72990.0326, 74040.692, 75100.6336, 76174.7826, 77252.5998, 78340.2974, 79438.2572, 80545.4976, 81657.2796, 82784.6336, 83915.515, 85059.7362, 86205.9368, 87364.4424, 88530.3358, 89707.3744, 90885.9638, 92080.197, 93275.5738, 94479.391, 95695.918, 96919.2236, 98148.4602, 99382.3474, 100625.6974, 101878.0284, 103141.6278, 104409.4588, 105686.2882, 106967.5402, 108261.6032, 109548.1578, 110852.0728, 112162.231, 113479.0072, 114806.2626, 116137.9072, 117469.5048, 118813.5186, 120165.4876, 121516.2556, 122875.766, 124250.5444, 125621.2222, 127003.2352, 128387.848, 129775.2644, 131181.7776, 132577.3086, 133979.9458, 135394.1132, 136800.9078, 138233.217,
+                    139668.5308, 141085.212, 142535.2122, 143969.0684, 145420.2872, 146878.1542, 148332.7572, 149800.3202, 151269.66, 152743.6104, 154213.0948, 155690.288, 157169.4246, 158672.1756, 160160.059, 161650.6854, 163145.7772, 164645.6726, 166159.1952, 167682.1578, 169177.3328, 170700.0118, 172228.8964, 173732.6664, 175265.5556, 176787.799, 178317.111, 179856.6914, 181400.865, 182943.4612, 184486.742, 186033.4698, 187583.7886, 189148.1868, 190688.4526, 192250.1926, 193810.9042, 195354.2972, 196938.7682, 198493.5898, 200079.2824, 201618.912, 203205.5492, 204765.5798, 206356.1124, 207929.3064, 209498.7196, 211086.229, 212675.1324, 214256.7892, 215826.2392, 217412.8474, 218995.6724, 220618.6038, 222207.1166, 223781.0364, 225387.4332, 227005.7928, 228590.4336, 230217.8738, 231805.1054, 233408.9, 234995.3432, 236601.4956, 238190.7904, 239817.2548, 241411.2832, 243002.4066, 244640.1884, 246255.3128, 247849.3508, 249479.9734, 251106.8822, 252705.027, 254332.9242, 255935.129,
+                    257526.9014, 259154.772, 260777.625, 262390.253, 264004.4906, 265643.59, 267255.4076, 268873.426, 270470.7252, 272106.4804, 273722.4456, 275337.794, 276945.7038, 278592.9154, 280204.3726, 281841.1606, 283489.171, 285130.1716, 286735.3362, 288364.7164, 289961.1814, 291595.5524, 293285.683, 294899.6668, 296499.3434, 298128.0462, 299761.8946, 301394.2424, 302997.6748, 304615.1478, 306269.7724, 307886.114, 309543.1028, 311153.2862, 312782.8546, 314421.2008, 316033.2438, 317692.9636, 319305.2648, 320948.7406, 322566.3364, 324228.4224, 325847.1542, },
+            // precision 17
+            { 94542, 96125.811, 97728.019, 99348.558, 100987.9705, 102646.7565, 104324.5125, 106021.7435, 107736.7865, 109469.272, 111223.9465, 112995.219, 114787.432, 116593.152, 118422.71, 120267.2345, 122134.6765, 124020.937, 125927.2705, 127851.255, 129788.9485, 131751.016, 133726.8225, 135722.592, 137736.789, 139770.568, 141821.518, 143891.343, 145982.1415, 148095.387, 150207.526, 152355.649, 154515.6415, 156696.05, 158887.7575, 161098.159, 163329.852, 165569.053, 167837.4005, 170121.6165, 172420.4595, 174732.6265, 177062.77, 179412.502, 181774.035, 184151.939, 186551.6895, 188965.691, 191402.8095, 193857.949, 196305.0775, 198774.6715, 201271.2585, 203764.78, 206299.3695, 208818.1365, 211373.115, 213946.7465, 216532.076, 219105.541, 221714.5375, 224337.5135, 226977.5125, 229613.0655, 232270.2685, 234952.2065, 237645.3555, 240331.1925, 243034.517, 245756.0725, 248517.6865, 251232.737, 254011.3955, 256785.995, 259556.44, 262368.335, 265156.911, 267965.266, 270785.583, 273616.0495
 ,
+                    276487.4835, 279346.639, 282202.509, 285074.3885, 287942.2855, 290856.018, 293774.0345, 296678.5145, 299603.6355, 302552.6575, 305492.9785, 308466.8605, 311392.581, 314347.538, 317319.4295, 320285.9785, 323301.7325, 326298.3235, 329301.3105, 332301.987, 335309.791, 338370.762, 341382.923, 344431.1265, 347464.1545, 350507.28, 353619.2345, 356631.2005, 359685.203, 362776.7845, 365886.488, 368958.2255, 372060.6825, 375165.4335, 378237.935, 381328.311, 384430.5225, 387576.425, 390683.242, 393839.648, 396977.8425, 400101.9805, 403271.296, 406409.8425, 409529.5485, 412678.7, 415847.423, 419020.8035, 422157.081, 425337.749, 428479.6165, 431700.902, 434893.1915, 438049.582, 441210.5415, 444379.2545, 447577.356, 450741.931, 453959.548, 457137.0935, 460329.846, 463537.4815, 466732.3345, 469960.5615, 473164.681, 476347.6345, 479496.173, 482813.1645, 486025.6995, 489249.4885, 492460.1945, 495675.8805, 498908.0075, 502131.802, 505374.3855, 508550.9915, 511806.7305, 515026.776
 ,
+                    518217.0005, 521523.9855, 524705.9855, 527950.997, 531210.0265, 534472.497, 537750.7315, 540926.922, 544207.094, 547429.4345, 550666.3745, 553975.3475, 557150.7185, 560399.6165, 563662.697, 566916.7395, 570146.1215, 573447.425, 576689.6245, 579874.5745, 583202.337, 586503.0255, 589715.635, 592910.161, 596214.3885, 599488.035, 602740.92, 605983.0685, 609248.67, 612491.3605, 615787.912, 619107.5245, 622307.9555, 625577.333, 628840.4385, 632085.2155, 635317.6135, 638691.7195, 641887.467, 645139.9405, 648441.546, 651666.252, 654941.845, },
+            // precision 18
+            { 189084, 192250.913, 195456.774, 198696.946, 201977.762, 205294.444, 208651.754, 212042.099, 215472.269, 218941.91, 222443.912, 225996.845, 229568.199, 233193.568, 236844.457, 240543.233, 244279.475, 248044.27, 251854.588, 255693.2, 259583.619, 263494.621, 267445.385, 271454.061, 275468.769, 279549.456, 283646.446, 287788.198, 291966.099, 296181.164, 300431.469, 304718.618, 309024.004, 313393.508, 317760.803, 322209.731, 326675.061, 331160.627, 335654.47, 340241.442, 344841.833, 349467.132, 354130.629, 358819.432, 363574.626, 368296.587, 373118.482, 377914.93, 382782.301, 387680.669, 392601.981, 397544.323, 402529.115, 407546.018, 412593.658, 417638.657, 422762.865, 427886.169, 433017.167, 438213.273, 443441.254, 448692.421, 453937.533, 459239.049, 464529.569, 469910.083, 475274.03, 480684.473, 486070.26, 491515.237, 496995.651, 502476.617, 507973.609, 513497.19, 519083.233, 524726.509, 530305.505, 535945.728, 541584.404, 547274.055, 552967.236, 558667.862, 564360.216,
+                    570128.148, 575965.08, 581701.952, 587532.523, 593361.144, 599246.128, 605033.418, 610958.779, 616837.117, 622772.818, 628672.04, 634675.369, 640574.831, 646585.739, 652574.547, 658611.217, 664642.684, 670713.914, 676737.681, 682797.313, 688837.897, 694917.874, 701009.882, 707173.648, 713257.254, 719415.392, 725636.761, 731710.697, 737906.209, 744103.074, 750313.39, 756504.185, 762712.579, 768876.985, 775167.859, 781359, 787615.959, 793863.597, 800245.477, 806464.582, 812785.294, 819005.925, 825403.057, 831676.197, 837936.284, 844266.968, 850642.711, 856959.756, 863322.774, 869699.931, 876102.478, 882355.787, 888694.463, 895159.952, 901536.143, 907872.631, 914293.672, 920615.14, 927130.974, 933409.404, 939922.178, 946331.47, 952745.93, 959209.264, 965590.224, 972077.284, 978501.961, 984953.19, 991413.271, 997817.479, 1004222.658, 1010725.676, 1017177.138, 1023612.529, 1030098.236, 1036493.719, 1043112.207, 1049537.036, 1056008.096, 1062476.184, 1068942.337,
+                    1075524.95, 1081932.864, 1088426.025, 1094776.005, 1101327.448, 1107901.673, 1114423.639, 1120884.602, 1127324.923, 1133794.24, 1140328.886, 1146849.376, 1153346.682, 1159836.502, 1166478.703, 1172953.304, 1179391.502, 1185950.982, 1192544.052, 1198913.41, 1205430.994, 1212015.525, 1218674.042, 1225121.683, 1231551.101, 1238126.379, 1244673.795, 1251260.649, 1257697.86, 1264320.983, 1270736.319, 1277274.694, 1283804.95, 1290211.514, 1296858.568, 1303455.691, } };
+
+    public static final double[][] biasData = {
+            // precision 4
+            { 10, 9.717, 9.207, 8.7896, 8.2882, 7.8204, 7.3772, 6.9342, 6.5202, 6.161, 5.7722, 5.4636, 5.0396, 4.6766, 4.3566, 4.0454, 3.7936, 3.4856, 3.2666, 2.9946, 2.766, 2.4692, 2.3638, 2.0764, 1.7864, 1.7602, 1.4814, 1.433, 1.2926, 1.0664, 0.999600000000001, 0.7956, 0.5366, 0.589399999999998, 0.573799999999999, 0.269799999999996, 0.368200000000002, 0.0544000000000011, 0.234200000000001, 0.0108000000000033, -0.203400000000002, -0.0701999999999998, -0.129600000000003, -0.364199999999997, -0.480600000000003, -0.226999999999997, -0.322800000000001, -0.382599999999996, -0.511200000000002, -0.669600000000003, -0.749400000000001, -0.500399999999999, -0.617600000000003, -0.6922, -0.601599999999998, -0.416200000000003, -0.338200000000001, -0.782600000000002, -0.648600000000002, -0.919800000000002, -0.851799999999997, -0.962400000000002, -0.6402, -1.1922, -1.0256, -1.086, -1.21899999999999, -0.819400000000002, -0.940600000000003, -1.1554, -1.2072, -1.1752, -1.16759999999999,
+                    -1.14019999999999, -1.3754, -1.29859999999999, -1.607, -1.3292, -1.7606, },
+            // precision 5
+            { 22, 21.1194, 20.8208, 20.2318, 19.77, 19.2436, 18.7774, 18.2848, 17.8224, 17.3742, 16.9336, 16.503, 16.0494, 15.6292, 15.2124, 14.798, 14.367, 13.9728, 13.5944, 13.217, 12.8438, 12.3696, 12.0956, 11.7044, 11.324, 11.0668, 10.6698, 10.3644, 10.049, 9.6918, 9.4146, 9.082, 8.687, 8.5398, 8.2462, 7.857, 7.6606, 7.4168, 7.1248, 6.9222, 6.6804, 6.447, 6.3454, 5.9594, 5.7636, 5.5776, 5.331, 5.19, 4.9676, 4.7564, 4.5314, 4.4442, 4.3708, 3.9774, 3.9624, 3.8796, 3.755, 3.472, 3.2076, 3.1024, 2.8908, 2.7338, 2.7728, 2.629, 2.413, 2.3266, 2.1524, 2.2642, 2.1806, 2.0566, 1.9192, 1.7598, 1.3516, 1.5802, 1.43859999999999, 1.49160000000001, 1.1524, 1.1892, 0.841399999999993, 0.879800000000003, 0.837599999999995, 0.469800000000006, 0.765600000000006, 0.331000000000003, 0.591399999999993, 0.601200000000006, 0.701599999999999, 0.558199999999999, 0.339399999999998, 0.354399999999998, 0.491200000000006, 0.308000000000007, 0.355199999999996, -0.0254000000000048, 0.205200000000005,
+                    -0.272999999999996, 0.132199999999997, 0.394400000000005, -0.241200000000006, 0.242000000000004, 0.191400000000002, 0.253799999999998, -0.122399999999999, -0.370800000000003, 0.193200000000004, -0.0848000000000013, 0.0867999999999967, -0.327200000000005, -0.285600000000002, 0.311400000000006, -0.128399999999999, -0.754999999999995, -0.209199999999996, -0.293599999999998, -0.364000000000004, -0.253600000000006, -0.821200000000005, -0.253600000000006, -0.510400000000004, -0.383399999999995, -0.491799999999998, -0.220200000000006, -0.0972000000000008, -0.557400000000001, -0.114599999999996, -0.295000000000002, -0.534800000000004, 0.346399999999988, -0.65379999999999, 0.0398000000000138, 0.0341999999999985, -0.995800000000003, -0.523400000000009, -0.489000000000004, -0.274799999999999, -0.574999999999989, -0.482799999999997, 0.0571999999999946, -0.330600000000004, -0.628800000000012, -0.140199999999993, -0.540600000000012, -0.445999999999998, -0.599400000000003,
+                    -0.262599999999992, 0.163399999999996, -0.100599999999986, -0.39500000000001, -1.06960000000001, -0.836399999999998, -0.753199999999993, -0.412399999999991, -0.790400000000005, -0.29679999999999, -0.28540000000001, -0.193000000000012, -0.0772000000000048, -0.962799999999987, -0.414800000000014, },
+            // precision 6
+            { 45, 44.1902, 43.271, 42.8358, 41.8142, 41.2854, 40.317, 39.354, 38.8924, 37.9436, 37.4596, 36.5262, 35.6248, 35.1574, 34.2822, 33.837, 32.9636, 32.074, 31.7042, 30.7976, 30.4772, 29.6564, 28.7942, 28.5004, 27.686, 27.291, 26.5672, 25.8556, 25.4982, 24.8204, 24.4252, 23.7744, 23.0786, 22.8344, 22.0294, 21.8098, 21.0794, 20.5732, 20.1878, 19.5648, 19.2902, 18.6784, 18.3352, 17.8946, 17.3712, 17.0852, 16.499, 16.2686, 15.6844, 15.2234, 14.9732, 14.3356, 14.2286, 13.7262, 13.3284, 13.1048, 12.5962, 12.3562, 12.1272, 11.4184, 11.4974, 11.0822, 10.856, 10.48, 10.2834, 10.0208, 9.637, 9.51739999999999, 9.05759999999999, 8.74760000000001, 8.42700000000001, 8.1326, 8.2372, 8.2788, 7.6776, 7.79259999999999, 7.1952, 6.9564, 6.6454, 6.87, 6.5428, 6.19999999999999, 6.02940000000001, 5.62780000000001, 5.6782, 5.792, 5.35159999999999, 5.28319999999999, 5.0394, 5.07480000000001, 4.49119999999999, 4.84899999999999, 4.696, 4.54040000000001, 4.07300000000001, 4.37139999999999, 3.7216,
+                    3.7328, 3.42080000000001, 3.41839999999999, 3.94239999999999, 3.27719999999999, 3.411, 3.13079999999999, 2.76900000000001, 2.92580000000001, 2.68279999999999, 2.75020000000001, 2.70599999999999, 2.3886, 3.01859999999999, 2.45179999999999, 2.92699999999999, 2.41720000000001, 2.41139999999999, 2.03299999999999, 2.51240000000001, 2.5564, 2.60079999999999, 2.41720000000001, 1.80439999999999, 1.99700000000001, 2.45480000000001, 1.8948, 2.2346, 2.30860000000001, 2.15479999999999, 1.88419999999999, 1.6508, 0.677199999999999, 1.72540000000001, 1.4752, 1.72280000000001, 1.66139999999999, 1.16759999999999, 1.79300000000001, 1.00059999999999, 0.905200000000008, 0.659999999999997, 1.55879999999999, 1.1636, 0.688199999999995, 0.712600000000009, 0.450199999999995, 1.1978, 0.975599999999986, 0.165400000000005, 1.727, 1.19739999999999, -0.252600000000001, 1.13460000000001, 1.3048, 1.19479999999999, 0.313400000000001, 0.878999999999991, 1.12039999999999, 0.853000000000009,
+                    1.67920000000001, 0.856999999999999, 0.448599999999999, 1.2362, 0.953399999999988, 1.02859999999998, 0.563199999999995, 0.663000000000011, 0.723000000000013, 0.756599999999992, 0.256599999999992, -0.837600000000009, 0.620000000000005, 0.821599999999989, 0.216600000000028, 0.205600000000004, 0.220199999999977, 0.372599999999977, 0.334400000000016, 0.928400000000011, 0.972800000000007, 0.192400000000021, 0.487199999999973, -0.413000000000011, 0.807000000000016, 0.120600000000024, 0.769000000000005, 0.870799999999974, 0.66500000000002, 0.118200000000002, 0.401200000000017, 0.635199999999998, 0.135400000000004, 0.175599999999974, 1.16059999999999, 0.34620000000001, 0.521400000000028, -0.586599999999976, -1.16480000000001, 0.968399999999974, 0.836999999999989, 0.779600000000016, 0.985799999999983, },
+            // precision 7
+            { 91, 89.4934, 87.9758, 86.4574, 84.9718, 83.4954, 81.5302, 80.0756, 78.6374, 77.1782, 75.7888, 73.9522, 72.592, 71.2532, 69.9086, 68.5938, 66.9474, 65.6796, 64.4394, 63.2176, 61.9768, 60.4214, 59.2528, 58.0102, 56.8658, 55.7278, 54.3044, 53.1316, 52.093, 51.0032, 49.9092, 48.6306, 47.5294, 46.5756, 45.6508, 44.662, 43.552, 42.3724, 41.617, 40.5754, 39.7872, 38.8444, 37.7988, 36.8606, 36.2118, 35.3566, 34.4476, 33.5882, 32.6816, 32.0824, 31.0258, 30.6048, 29.4436, 28.7274, 27.957, 27.147, 26.4364, 25.7592, 25.3386, 24.781, 23.8028, 23.656, 22.6544, 21.996, 21.4718, 21.1544, 20.6098, 19.5956, 19.0616, 18.5758, 18.4878, 17.5244, 17.2146, 16.724, 15.8722, 15.5198, 15.0414, 14.941, 14.9048, 13.87, 13.4304, 13.028, 12.4708, 12.37, 12.0624, 11.4668, 11.5532, 11.4352, 11.2564, 10.2744, 10.2118, 9.74720000000002, 10.1456, 9.2928, 8.75040000000001, 8.55279999999999, 8.97899999999998, 8.21019999999999, 8.18340000000001, 7.3494, 7.32499999999999, 7.66140000000001, 6.90300000000002,
+                    7.25439999999998, 6.9042, 7.21499999999997, 6.28640000000001, 6.08139999999997, 6.6764, 6.30099999999999, 5.13900000000001, 5.65800000000002, 5.17320000000001, 4.59019999999998, 4.9538, 5.08280000000002, 4.92200000000003, 4.99020000000002, 4.7328, 5.4538, 4.11360000000002, 4.22340000000003, 4.08780000000002, 3.70800000000003, 4.15559999999999, 4.18520000000001, 3.63720000000001, 3.68220000000002, 3.77960000000002, 3.6078, 2.49160000000001, 3.13099999999997, 2.5376, 3.19880000000001, 3.21100000000001, 2.4502, 3.52820000000003, 2.91199999999998, 3.04480000000001, 2.7432, 2.85239999999999, 2.79880000000003, 2.78579999999999, 1.88679999999999, 2.98860000000002, 2.50639999999999, 1.91239999999999, 2.66160000000002, 2.46820000000002, 1.58199999999999, 1.30399999999997, 2.27379999999999, 2.68939999999998, 1.32900000000001, 3.10599999999999, 1.69080000000002, 2.13740000000001, 2.53219999999999, 1.88479999999998, 1.33240000000001, 1.45119999999997, 1.17899999999997,
+                    2.44119999999998, 1.60659999999996, 2.16700000000003, 0.77940000000001, 2.37900000000002, 2.06700000000001, 1.46000000000004, 2.91160000000002, 1.69200000000001, 0.954600000000028, 2.49300000000005, 2.2722, 1.33500000000004, 2.44899999999996, 1.20140000000004, 3.07380000000001, 2.09739999999999, 2.85640000000001, 2.29960000000005, 2.40899999999999, 1.97040000000004, 0.809799999999996, 1.65279999999996, 2.59979999999996, 0.95799999999997, 2.06799999999998, 2.32780000000002, 4.20159999999998, 1.96320000000003, 1.86400000000003, 1.42999999999995, 3.77940000000001, 1.27200000000005, 1.86440000000005, 2.20600000000002, 3.21900000000005, 1.5154, 2.61019999999996, },
+            // precision 8
+            { 183.2152, 180.2454, 177.2096, 173.6652, 170.6312, 167.6822, 164.249, 161.3296, 158.0038, 155.2074, 152.4612, 149.27, 146.5178, 143.4412, 140.8032, 138.1634, 135.1688, 132.6074, 129.6946, 127.2664, 124.8228, 122.0432, 119.6824, 116.9464, 114.6268, 112.2626, 109.8376, 107.4034, 104.8956, 102.8522, 100.7638, 98.3552, 96.3556, 93.7526, 91.9292, 89.8954, 87.8198, 85.7668, 83.298, 81.6688, 79.9466, 77.9746, 76.1672, 74.3474, 72.3028, 70.8912, 69.114, 67.4646, 65.9744, 64.4092, 62.6022, 60.843, 59.5684, 58.1652, 56.5426, 55.4152, 53.5388, 52.3592, 51.1366, 49.486, 48.3918, 46.5076, 45.509, 44.3834, 43.3498, 42.0668, 40.7346, 40.1228, 38.4528, 37.7, 36.644, 36.0518, 34.5774, 33.9068, 32.432, 32.1666, 30.434, 29.6644, 28.4894, 27.6312, 26.3804, 26.292, 25.5496000000001, 25.0234, 24.8206, 22.6146, 22.4188, 22.117, 20.6762, 20.6576, 19.7864, 19.509, 18.5334, 17.9204, 17.772, 16.2924, 16.8654, 15.1836, 15.745, 15.1316, 15.0386, 14.0136, 13.6342, 12.6196, 12.1866, 12.4281999999999,
+                    11.3324, 10.4794000000001, 11.5038, 10.129, 9.52800000000002, 10.3203999999999, 9.46299999999997, 9.79280000000006, 9.12300000000005, 8.74180000000001, 9.2192, 7.51020000000005, 7.60659999999996, 7.01840000000004, 7.22239999999999, 7.40139999999997, 6.76179999999999, 7.14359999999999, 5.65060000000005, 5.63779999999997, 5.76599999999996, 6.75139999999999, 5.57759999999996, 3.73220000000003, 5.8048, 5.63019999999995, 4.93359999999996, 3.47979999999995, 4.33879999999999, 3.98940000000005, 3.81960000000004, 3.31359999999995, 3.23080000000004, 3.4588, 3.08159999999998, 3.4076, 3.00639999999999, 2.38779999999997, 2.61900000000003, 1.99800000000005, 3.34820000000002, 2.95060000000001, 0.990999999999985, 2.11440000000005, 2.20299999999997, 2.82219999999995, 2.73239999999998, 2.7826, 3.76660000000004, 2.26480000000004, 2.31280000000004, 2.40819999999997, 2.75360000000001, 3.33759999999995, 2.71559999999999, 1.7478000000001, 1.42920000000004, 2.39300000000003,
+                    2.22779999999989, 2.34339999999997, 0.87259999999992, 3.88400000000001, 1.80600000000004, 1.91759999999999, 1.16779999999994, 1.50320000000011, 2.52500000000009, 0.226400000000012, 2.31500000000005, 0.930000000000064, 1.25199999999995, 2.14959999999996, 0.0407999999999902, 2.5447999999999, 1.32960000000003, 0.197400000000016, 2.52620000000002, 3.33279999999991, -1.34300000000007, 0.422199999999975, 0.917200000000093, 1.12920000000008, 1.46060000000011, 1.45779999999991, 2.8728000000001, 3.33359999999993, -1.34079999999994, 1.57680000000005, 0.363000000000056, 1.40740000000005, 0.656600000000026, 0.801400000000058, -0.454600000000028, 1.51919999999996, },
+            // precision 9
+            { 368, 361.8294, 355.2452, 348.6698, 342.1464, 336.2024, 329.8782, 323.6598, 317.462, 311.2826, 305.7102, 299.7416, 293.9366, 288.1046, 282.285, 277.0668, 271.306, 265.8448, 260.301, 254.9886, 250.2422, 244.8138, 239.7074, 234.7428, 229.8402, 225.1664, 220.3534, 215.594, 210.6886, 205.7876, 201.65, 197.228, 192.8036, 188.1666, 184.0818, 180.0824, 176.2574, 172.302, 168.1644, 164.0056, 160.3802, 156.7192, 152.5234, 149.2084, 145.831, 142.485, 139.1112, 135.4764, 131.76, 129.3368, 126.5538, 122.5058, 119.2646, 116.5902, 113.3818, 110.8998, 107.9532, 105.2062, 102.2798, 99.4728, 96.9582, 94.3292, 92.171, 89.7809999999999, 87.5716, 84.7048, 82.5322, 79.875, 78.3972, 75.3464, 73.7274, 71.2834, 70.1444, 68.4263999999999, 66.0166, 64.018, 62.0437999999999, 60.3399999999999, 58.6856, 57.9836, 55.0311999999999, 54.6769999999999, 52.3188, 51.4846, 49.4423999999999, 47.739, 46.1487999999999, 44.9202, 43.4059999999999, 42.5342000000001, 41.2834, 38.8954000000001, 38.3286000000001,
+                    36.2146, 36.6684, 35.9946, 33.123, 33.4338, 31.7378000000001, 29.076, 28.9692, 27.4964, 27.0998, 25.9864, 26.7754, 24.3208, 23.4838, 22.7388000000001, 24.0758000000001, 21.9097999999999, 20.9728, 19.9228000000001, 19.9292, 16.617, 17.05, 18.2996000000001, 15.6128000000001, 15.7392, 14.5174, 13.6322, 12.2583999999999, 13.3766000000001, 11.423, 13.1232, 9.51639999999998, 10.5938000000001, 9.59719999999993, 8.12220000000002, 9.76739999999995, 7.50440000000003, 7.56999999999994, 6.70440000000008, 6.41419999999994, 6.71019999999999, 5.60940000000005, 4.65219999999999, 6.84099999999989, 3.4072000000001, 3.97859999999991, 3.32760000000007, 5.52160000000003, 3.31860000000006, 2.06940000000009, 4.35400000000004, 1.57500000000005, 0.280799999999999, 2.12879999999996, -0.214799999999968, -0.0378000000000611, -0.658200000000079, 0.654800000000023, -0.0697999999999865, 0.858400000000074, -2.52700000000004, -2.1751999999999, -3.35539999999992, -1.04019999999991,
+                    -0.651000000000067, -2.14439999999991, -1.96659999999997, -3.97939999999994, -0.604400000000169, -3.08260000000018, -3.39159999999993, -5.29640000000018, -5.38920000000007, -5.08759999999984, -4.69900000000007, -5.23720000000003, -3.15779999999995, -4.97879999999986, -4.89899999999989, -7.48880000000008, -5.94799999999987, -5.68060000000014, -6.67180000000008, -4.70499999999993, -7.27779999999984, -4.6579999999999, -4.4362000000001, -4.32139999999981, -5.18859999999995, -6.66879999999992, -6.48399999999992, -5.1260000000002, -4.4032000000002, -6.13500000000022, -5.80819999999994, -4.16719999999987, -4.15039999999999, -7.45600000000013, -7.24080000000004, -9.83179999999993, -5.80420000000004, -8.6561999999999, -6.99940000000015, -10.5473999999999, -7.34139999999979, -6.80999999999995, -6.29719999999998, -6.23199999999997, },
+            // precision 10
+            { 737.1256, 724.4234, 711.1064, 698.4732, 685.4636, 673.0644, 660.488, 647.9654, 636.0832, 623.7864, 612.1992, 600.2176, 588.5228, 577.1716, 565.7752, 554.899, 543.6126, 532.6492, 521.9474, 511.5214, 501.1064, 490.6364, 480.2468, 470.4588, 460.3832, 451.0584, 440.8606, 431.3868, 422.5062, 413.1862, 404.463, 395.339, 386.1936, 378.1292, 369.1854, 361.2908, 353.3324, 344.8518, 337.5204, 329.4854, 321.9318, 314.552, 306.4658, 299.4256, 292.849, 286.152, 278.8956, 271.8792, 265.118, 258.62, 252.5132, 245.9322, 239.7726, 233.6086, 227.5332, 222.5918, 216.4294, 210.7662, 205.4106, 199.7338, 194.9012, 188.4486, 183.1556, 178.6338, 173.7312, 169.6264, 163.9526, 159.8742, 155.8326, 151.1966, 147.5594, 143.07, 140.037, 134.1804, 131.071, 127.4884, 124.0848, 120.2944, 117.333, 112.9626, 110.2902, 107.0814, 103.0334, 99.4832000000001, 96.3899999999999, 93.7202000000002, 90.1714000000002, 87.2357999999999, 85.9346, 82.8910000000001, 80.0264000000002, 78.3834000000002,
+                    75.1543999999999, 73.8683999999998, 70.9895999999999, 69.4367999999999, 64.8701999999998, 65.0408000000002, 61.6738, 59.5207999999998, 57.0158000000001, 54.2302, 53.0962, 50.4985999999999, 52.2588000000001, 47.3914, 45.6244000000002, 42.8377999999998, 43.0072, 40.6516000000001, 40.2453999999998, 35.2136, 36.4546, 33.7849999999999, 33.2294000000002, 32.4679999999998, 30.8670000000002, 28.6507999999999, 28.9099999999999, 27.5983999999999, 26.1619999999998, 24.5563999999999, 23.2328000000002, 21.9484000000002, 21.5902000000001, 21.3346000000001, 17.7031999999999, 20.6111999999998, 19.5545999999999, 15.7375999999999, 17.0720000000001, 16.9517999999998, 15.326, 13.1817999999998, 14.6925999999999, 13.0859999999998, 13.2754, 10.8697999999999, 11.248, 7.3768, 4.72339999999986, 7.97899999999981, 8.7503999999999, 7.68119999999999, 9.7199999999998, 7.73919999999998, 5.6224000000002, 7.44560000000001, 6.6601999999998, 5.9058, 4.00199999999995, 4.51699999999983,
+                    4.68240000000014, 3.86220000000003, 5.13639999999987, 5.98500000000013, 2.47719999999981, 2.61999999999989, 1.62800000000016, 4.65000000000009, 0.225599999999758, 0.831000000000131, -0.359400000000278, 1.27599999999984, -2.92559999999958, -0.0303999999996449, 2.37079999999969, -2.0033999999996, 0.804600000000391, 0.30199999999968, 1.1247999999996, -2.6880000000001, 0.0321999999996478, -1.18099999999959, -3.9402, -1.47940000000017, -0.188400000000001, -2.10720000000038, -2.04159999999956, -3.12880000000041, -4.16160000000036, -0.612799999999879, -3.48719999999958, -8.17900000000009, -5.37780000000021, -4.01379999999972, -5.58259999999973, -5.73719999999958, -7.66799999999967, -5.69520000000011, -1.1247999999996, -5.58520000000044, -8.04560000000038, -4.64840000000004, -11.6468000000004, -7.97519999999986, -5.78300000000036, -7.67420000000038, -10.6328000000003, -9.81720000000041, },
+            // precision 11
+            { 1476, 1449.6014, 1423.5802, 1397.7942, 1372.3042, 1347.2062, 1321.8402, 1297.2292, 1272.9462, 1248.9926, 1225.3026, 1201.4252, 1178.0578, 1155.6092, 1132.626, 1110.5568, 1088.527, 1066.5154, 1045.1874, 1024.3878, 1003.37, 982.1972, 962.5728, 942.1012, 922.9668, 903.292, 884.0772, 864.8578, 846.6562, 828.041, 809.714, 792.3112, 775.1806, 757.9854, 740.656, 724.346, 707.5154, 691.8378, 675.7448, 659.6722, 645.5722, 630.1462, 614.4124, 600.8728, 585.898, 572.408, 558.4926, 544.4938, 531.6776, 517.282, 505.7704, 493.1012, 480.7388, 467.6876, 456.1872, 445.5048, 433.0214, 420.806, 411.409, 400.4144, 389.4294, 379.2286, 369.651, 360.6156, 350.337, 342.083, 332.1538, 322.5094, 315.01, 305.6686, 298.1678, 287.8116, 280.9978, 271.9204, 265.3286, 257.5706, 249.6014, 242.544, 235.5976, 229.583, 220.9438, 214.672, 208.2786, 201.8628, 195.1834, 191.505, 186.1816, 178.5188, 172.2294, 167.8908, 161.0194, 158.052, 151.4588, 148.1596, 143.4344, 138.5238, 133.13, 127.6374, 124.8162,
+                    118.7894, 117.3984, 114.6078, 109.0858, 105.1036, 103.6258, 98.6018000000004, 95.7618000000002, 93.5821999999998, 88.5900000000001, 86.9992000000002, 82.8800000000001, 80.4539999999997, 74.6981999999998, 74.3644000000004, 73.2914000000001, 65.5709999999999, 66.9232000000002, 65.1913999999997, 62.5882000000001, 61.5702000000001, 55.7035999999998, 56.1764000000003, 52.7596000000003, 53.0302000000001, 49.0609999999997, 48.4694, 44.933, 46.0474000000004, 44.7165999999997, 41.9416000000001, 39.9207999999999, 35.6328000000003, 35.5276000000003, 33.1934000000001, 33.2371999999996, 33.3864000000003, 33.9228000000003, 30.2371999999996, 29.1373999999996, 25.2272000000003, 24.2942000000003, 19.8338000000003, 18.9005999999999, 23.0907999999999, 21.8544000000002, 19.5176000000001, 15.4147999999996, 16.9314000000004, 18.6737999999996, 12.9877999999999, 14.3688000000002, 12.0447999999997, 15.5219999999999, 12.5299999999997, 14.5940000000001, 14.3131999999996, 9.45499999999993,
+                    12.9441999999999, 3.91139999999996, 13.1373999999996, 5.44720000000052, 9.82779999999912, 7.87279999999919, 3.67760000000089, 5.46980000000076, 5.55099999999948, 5.65979999999945, 3.89439999999922, 3.1275999999998, 5.65140000000065, 6.3062000000009, 3.90799999999945, 1.87060000000019, 5.17020000000048, 2.46680000000015, 0.770000000000437, -3.72340000000077, 1.16400000000067, 8.05340000000069, 0.135399999999208, 2.15940000000046, 0.766999999999825, 1.0594000000001, 3.15500000000065, -0.287399999999252, 2.37219999999979, -2.86620000000039, -1.63199999999961, -2.22979999999916, -0.15519999999924, -1.46039999999994, -0.262199999999211, -2.34460000000036, -2.8078000000005, -3.22179999999935, -5.60159999999996, -8.42200000000048, -9.43740000000071, 0.161799999999857, -10.4755999999998, -10.0823999999993, },
+            // precision 12
+            { 2953, 2900.4782, 2848.3568, 2796.3666, 2745.324, 2694.9598, 2644.648, 2595.539, 2546.1474, 2498.2576, 2450.8376, 2403.6076, 2357.451, 2311.38, 2266.4104, 2221.5638, 2176.9676, 2134.193, 2090.838, 2048.8548, 2007.018, 1966.1742, 1925.4482, 1885.1294, 1846.4776, 1807.4044, 1768.8724, 1731.3732, 1693.4304, 1657.5326, 1621.949, 1586.5532, 1551.7256, 1517.6182, 1483.5186, 1450.4528, 1417.865, 1385.7164, 1352.6828, 1322.6708, 1291.8312, 1260.9036, 1231.476, 1201.8652, 1173.6718, 1145.757, 1119.2072, 1092.2828, 1065.0434, 1038.6264, 1014.3192, 988.5746, 965.0816, 940.1176, 917.9796, 894.5576, 871.1858, 849.9144, 827.1142, 805.0818, 783.9664, 763.9096, 742.0816, 724.3962, 706.3454, 688.018, 667.4214, 650.3106, 633.0686, 613.8094, 597.818, 581.4248, 563.834, 547.363, 531.5066, 520.455400000001, 505.583199999999, 488.366, 476.480799999999, 459.7682, 450.0522, 434.328799999999, 423.952799999999, 408.727000000001, 399.079400000001, 387.252200000001, 373.987999999999,
+                    360.852000000001, 351.6394, 339.642, 330.902400000001, 322.661599999999, 311.662200000001, 301.3254, 291.7484, 279.939200000001, 276.7508, 263.215200000001, 254.811400000001, 245.5494, 242.306399999999, 234.8734, 223.787200000001, 217.7156, 212.0196, 200.793, 195.9748, 189.0702, 182.449199999999, 177.2772, 170.2336, 164.741, 158.613600000001, 155.311, 147.5964, 142.837, 137.3724, 132.0162, 130.0424, 121.9804, 120.451800000001, 114.8968, 111.585999999999, 105.933199999999, 101.705, 98.5141999999996, 95.0488000000005, 89.7880000000005, 91.4750000000004, 83.7764000000006, 80.9698000000008, 72.8574000000008, 73.1615999999995, 67.5838000000003, 62.6263999999992, 63.2638000000006, 66.0977999999996, 52.0843999999997, 58.9956000000002, 47.0912000000008, 46.4956000000002, 48.4383999999991, 47.1082000000006, 43.2392, 37.2759999999998, 40.0283999999992, 35.1864000000005, 35.8595999999998, 32.0998, 28.027, 23.6694000000007, 33.8266000000003, 26.3736000000008,
+                    27.2008000000005, 21.3245999999999, 26.4115999999995, 23.4521999999997, 19.5013999999992, 19.8513999999996, 10.7492000000002, 18.6424000000006, 13.1265999999996, 18.2436000000016, 6.71860000000015, 3.39459999999963, 6.33759999999893, 7.76719999999841, 0.813999999998487, 3.82819999999992, 0.826199999999517, 8.07440000000133, -1.59080000000176, 5.01780000000144, 0.455399999998917, -0.24199999999837, 0.174800000000687, -9.07640000000174, -4.20160000000033, -3.77520000000004, -4.75179999999818, -5.3724000000002, -8.90680000000066, -6.10239999999976, -5.74120000000039, -9.95339999999851, -3.86339999999836, -13.7304000000004, -16.2710000000006, -7.51359999999841, -3.30679999999847, -13.1339999999982, -10.0551999999989, -6.72019999999975, -8.59660000000076, -10.9307999999983, -1.8775999999998, -4.82259999999951, -13.7788, -21.6470000000008, -10.6735999999983, -15.7799999999988, },
+            // precision 13
+            { 5907.5052, 5802.2672, 5697.347, 5593.5794, 5491.2622, 5390.5514, 5290.3376, 5191.6952, 5093.5988, 4997.3552, 4902.5972, 4808.3082, 4715.5646, 4624.109, 4533.8216, 4444.4344, 4356.3802, 4269.2962, 4183.3784, 4098.292, 4014.79, 3932.4574, 3850.6036, 3771.2712, 3691.7708, 3615.099, 3538.1858, 3463.4746, 3388.8496, 3315.6794, 3244.5448, 3173.7516, 3103.3106, 3033.6094, 2966.5642, 2900.794, 2833.7256, 2769.81, 2707.3196, 2644.0778, 2583.9916, 2523.4662, 2464.124, 2406.073, 2347.0362, 2292.1006, 2238.1716, 2182.7514, 2128.4884, 2077.1314, 2025.037, 1975.3756, 1928.933, 1879.311, 1831.0006, 1783.2144, 1738.3096, 1694.5144, 1649.024, 1606.847, 1564.7528, 1525.3168, 1482.5372, 1443.9668, 1406.5074, 1365.867, 1329.2186, 1295.4186, 1257.9716, 1225.339, 1193.2972, 1156.3578, 1125.8686, 1091.187, 1061.4094, 1029.4188, 1000.9126, 972.3272, 944.004199999999, 915.7592, 889.965, 862.834200000001, 840.4254, 812.598399999999, 785.924200000001, 763.050999999999, 741.793799999999, 721.466,
+                    699.040799999999, 677.997200000002, 649.866999999998, 634.911800000002, 609.8694, 591.981599999999, 570.2922, 557.129199999999, 538.3858, 521.872599999999, 502.951400000002, 495.776399999999, 475.171399999999, 459.751, 439.995200000001, 426.708999999999, 413.7016, 402.3868, 387.262599999998, 372.0524, 357.050999999999, 342.5098, 334.849200000001, 322.529399999999, 311.613799999999, 295.848000000002, 289.273000000001, 274.093000000001, 263.329600000001, 251.389599999999, 245.7392, 231.9614, 229.7952, 217.155200000001, 208.9588, 199.016599999999, 190.839199999999, 180.6976, 176.272799999999, 166.976999999999, 162.5252, 151.196400000001, 149.386999999999, 133.981199999998, 130.0586, 130.164000000001, 122.053400000001, 110.7428, 108.1276, 106.232400000001, 100.381600000001, 98.7668000000012, 86.6440000000002, 79.9768000000004, 82.4722000000002, 68.7026000000005, 70.1186000000016, 71.9948000000004, 58.998599999999, 59.0492000000013, 56.9818000000014, 47.5338000000011,
+                    42.9928, 51.1591999999982, 37.2740000000013, 42.7220000000016, 31.3734000000004, 26.8090000000011, 25.8934000000008, 26.5286000000015, 29.5442000000003, 19.3503999999994, 26.0760000000009, 17.9527999999991, 14.8419999999969, 10.4683999999979, 8.65899999999965, 9.86720000000059, 4.34139999999752, -0.907800000000861, -3.32080000000133, -0.936199999996461, -11.9916000000012, -8.87000000000262, -6.33099999999831, -11.3366000000024, -15.9207999999999, -9.34659999999712, -15.5034000000014, -19.2097999999969, -15.357799999998, -28.2235999999975, -30.6898000000001, -19.3271999999997, -25.6083999999973, -24.409599999999, -13.6385999999984, -33.4473999999973, -32.6949999999997, -28.9063999999998, -31.7483999999968, -32.2935999999972, -35.8329999999987, -47.620600000002, -39.0855999999985, -33.1434000000008, -46.1371999999974, -37.5892000000022, -46.8164000000033, -47.3142000000007, -60.2914000000019, -37.7575999999972, },
+            // precision 14
+            { 11816.475, 11605.0046, 11395.3792, 11188.7504, 10984.1814, 10782.0086, 10582.0072, 10384.503, 10189.178, 9996.2738, 9806.0344, 9617.9798, 9431.394, 9248.7784, 9067.6894, 8889.6824, 8712.9134, 8538.8624, 8368.4944, 8197.7956, 8031.8916, 7866.6316, 7703.733, 7544.5726, 7386.204, 7230.666, 7077.8516, 6926.7886, 6778.6902, 6631.9632, 6487.304, 6346.7486, 6206.4408, 6070.202, 5935.2576, 5799.924, 5671.0324, 5541.9788, 5414.6112, 5290.0274, 5166.723, 5047.6906, 4929.162, 4815.1406, 4699.127, 4588.5606, 4477.7394, 4369.4014, 4264.2728, 4155.9224, 4055.581, 3955.505, 3856.9618, 3761.3828, 3666.9702, 3575.7764, 3482.4132, 3395.0186, 3305.8852, 3221.415, 3138.6024, 3056.296, 2970.4494, 2896.1526, 2816.8008, 2740.2156, 2670.497, 2594.1458, 2527.111, 2460.8168, 2387.5114, 2322.9498, 2260.6752, 2194.2686, 2133.7792, 2074.767, 2015.204, 1959.4226, 1898.6502, 1850.006, 1792.849, 1741.4838, 1687.9778, 1638.1322, 1589.3266, 1543.1394, 1496.8266, 1447.8516, 1402.7354, 1361.9606,
+                    1327.0692, 1285.4106, 1241.8112, 1201.6726, 1161.973, 1130.261, 1094.2036, 1048.2036, 1020.6436, 990.901400000002, 961.199800000002, 924.769800000002, 899.526400000002, 872.346400000002, 834.375, 810.432000000001, 780.659800000001, 756.013800000001, 733.479399999997, 707.923999999999, 673.858, 652.222399999999, 636.572399999997, 615.738599999997, 586.696400000001, 564.147199999999, 541.679600000003, 523.943599999999, 505.714599999999, 475.729599999999, 461.779600000002, 449.750800000002, 439.020799999998, 412.7886, 400.245600000002, 383.188199999997, 362.079599999997, 357.533799999997, 334.319000000003, 327.553399999997, 308.559399999998, 291.270199999999, 279.351999999999, 271.791400000002, 252.576999999997, 247.482400000001, 236.174800000001, 218.774599999997, 220.155200000001, 208.794399999999, 201.223599999998, 182.995600000002, 185.5268, 164.547400000003, 176.5962, 150.689599999998, 157.8004, 138.378799999999, 134.021200000003, 117.614399999999,
+                    108.194000000003, 97.0696000000025, 89.6042000000016, 95.6030000000028, 84.7810000000027, 72.635000000002, 77.3482000000004, 59.4907999999996, 55.5875999999989, 50.7346000000034, 61.3916000000027, 50.9149999999936, 39.0384000000049, 58.9395999999979, 29.633600000001, 28.2032000000036, 26.0078000000067, 17.0387999999948, 9.22000000000116, 13.8387999999977, 8.07240000000456, 14.1549999999988, 15.3570000000036, 3.42660000000615, 6.24820000000182, -2.96940000000177, -8.79940000000352, -5.97860000000219, -14.4048000000039, -3.4143999999942, -13.0148000000045, -11.6977999999945, -25.7878000000055, -22.3185999999987, -24.409599999999, -31.9756000000052, -18.9722000000038, -22.8678000000073, -30.8972000000067, -32.3715999999986, -22.3907999999938, -43.6720000000059, -35.9038, -39.7492000000057, -54.1641999999993, -45.2749999999942, -42.2989999999991, -44.1089999999967, -64.3564000000042, -49.9551999999967, -42.6116000000038, },
+            // precision 15
+            { 23634.0036, 23210.8034, 22792.4744, 22379.1524, 21969.7928, 21565.326, 21165.3532, 20770.2806, 20379.9892, 19994.7098, 19613.318, 19236.799, 18865.4382, 18498.8244, 18136.5138, 17778.8668, 17426.2344, 17079.32, 16734.778, 16397.2418, 16063.3324, 15734.0232, 15409.731, 15088.728, 14772.9896, 14464.1402, 14157.5588, 13855.5958, 13559.3296, 13264.9096, 12978.326, 12692.0826, 12413.8816, 12137.3192, 11870.2326, 11602.5554, 11340.3142, 11079.613, 10829.5908, 10583.5466, 10334.0344, 10095.5072, 9859.694, 9625.2822, 9395.7862, 9174.0586, 8957.3164, 8738.064, 8524.155, 8313.7396, 8116.9168, 7913.542, 7718.4778, 7521.65, 7335.5596, 7154.2906, 6968.7396, 6786.3996, 6613.236, 6437.406, 6270.6598, 6107.7958, 5945.7174, 5787.6784, 5635.5784, 5482.308, 5337.9784, 5190.0864, 5045.9158, 4919.1386, 4771.817, 4645.7742, 4518.4774, 4385.5454, 4262.6622, 4142.74679999999, 4015.5318, 3897.9276, 3790.7764, 3685.13800000001, 3573.6274, 3467.9706, 3368.61079999999, 3271.5202, 3170.3848,
+                    3076.4656, 2982.38400000001, 2888.4664, 2806.4868, 2711.9564, 2634.1434, 2551.3204, 2469.7662, 2396.61139999999, 2318.9902, 2243.8658, 2171.9246, 2105.01360000001, 2028.8536, 1960.9952, 1901.4096, 1841.86079999999, 1777.54700000001, 1714.5802, 1654.65059999999, 1596.311, 1546.2016, 1492.3296, 1433.8974, 1383.84600000001, 1339.4152, 1293.5518, 1245.8686, 1193.50659999999, 1162.27959999999, 1107.19439999999, 1069.18060000001, 1035.09179999999, 999.679000000004, 957.679999999993, 925.300199999998, 888.099400000006, 848.638600000006, 818.156400000007, 796.748399999997, 752.139200000005, 725.271200000003, 692.216, 671.633600000001, 647.939799999993, 621.670599999998, 575.398799999995, 561.226599999995, 532.237999999998, 521.787599999996, 483.095799999996, 467.049599999998, 465.286399999997, 415.548599999995, 401.047399999996, 380.607999999993, 377.362599999993, 347.258799999996, 338.371599999999, 310.096999999994, 301.409199999995, 276.280799999993, 265.586800000005,
+                    258.994399999996, 223.915999999997, 215.925399999993, 213.503800000006, 191.045400000003, 166.718200000003, 166.259000000005, 162.941200000001, 148.829400000002, 141.645999999993, 123.535399999993, 122.329800000007, 89.473399999988, 80.1962000000058, 77.5457999999926, 59.1056000000099, 83.3509999999951, 52.2906000000075, 36.3979999999865, 40.6558000000077, 42.0003999999899, 19.6630000000005, 19.7153999999864, -8.38539999999921, -0.692799999989802, 0.854800000000978, 3.23219999999856, -3.89040000000386, -5.25880000001052, -24.9052000000083, -22.6837999999989, -26.4286000000138, -34.997000000003, -37.0216000000073, -43.430400000012, -58.2390000000014, -68.8034000000043, -56.9245999999985, -57.8583999999973, -77.3097999999882, -73.2793999999994, -81.0738000000129, -87.4530000000086, -65.0254000000132, -57.296399999992, -96.2746000000043, -103.25, -96.081600000005, -91.5542000000132, -102.465200000006, -107.688599999994, -101.458000000013, -109.715800000005, },
+            // precision 16
+            { 47270, 46423.3584, 45585.7074, 44757.152, 43938.8416, 43130.9514, 42330.03, 41540.407, 40759.6348, 39988.206, 39226.5144, 38473.2096, 37729.795, 36997.268, 36272.6448, 35558.665, 34853.0248, 34157.4472, 33470.5204, 32793.5742, 32127.0194, 31469.4182, 30817.6136, 30178.6968, 29546.8908, 28922.8544, 28312.271, 27707.0924, 27114.0326, 26526.692, 25948.6336, 25383.7826, 24823.5998, 24272.2974, 23732.2572, 23201.4976, 22674.2796, 22163.6336, 21656.515, 21161.7362, 20669.9368, 20189.4424, 19717.3358, 19256.3744, 18795.9638, 18352.197, 17908.5738, 17474.391, 17052.918, 16637.2236, 16228.4602, 15823.3474, 15428.6974, 15043.0284, 14667.6278, 14297.4588, 13935.2882, 13578.5402, 13234.6032, 12882.1578, 12548.0728, 12219.231, 11898.0072, 11587.2626, 11279.9072, 10973.5048, 10678.5186, 10392.4876, 10105.2556, 9825.766, 9562.5444, 9294.2222, 9038.2352, 8784.848, 8533.2644, 8301.7776, 8058.30859999999, 7822.94579999999, 7599.11319999999, 7366.90779999999, 7161.217, 6957.53080000001,
+                    6736.212, 6548.21220000001, 6343.06839999999, 6156.28719999999, 5975.15419999999, 5791.75719999999, 5621.32019999999, 5451.66, 5287.61040000001, 5118.09479999999, 4957.288, 4798.4246, 4662.17559999999, 4512.05900000001, 4364.68539999999, 4220.77720000001, 4082.67259999999, 3957.19519999999, 3842.15779999999, 3699.3328, 3583.01180000001, 3473.8964, 3338.66639999999, 3233.55559999999, 3117.799, 3008.111, 2909.69140000001, 2814.86499999999, 2719.46119999999, 2624.742, 2532.46979999999, 2444.7886, 2370.1868, 2272.45259999999, 2196.19260000001, 2117.90419999999, 2023.2972, 1969.76819999999, 1885.58979999999, 1833.2824, 1733.91200000001, 1682.54920000001, 1604.57980000001, 1556.11240000001, 1491.3064, 1421.71960000001, 1371.22899999999, 1322.1324, 1264.7892, 1196.23920000001, 1143.8474, 1088.67240000001, 1073.60380000001, 1023.11660000001, 959.036400000012, 927.433199999999, 906.792799999996, 853.433599999989, 841.873800000001, 791.1054, 756.899999999994,
+                    704.343200000003, 672.495599999995, 622.790399999998, 611.254799999995, 567.283200000005, 519.406599999988, 519.188400000014, 495.312800000014, 451.350799999986, 443.973399999988, 431.882199999993, 392.027000000002, 380.924200000009, 345.128999999986, 298.901400000002, 287.771999999997, 272.625, 247.253000000026, 222.490600000019, 223.590000000026, 196.407599999977, 176.425999999978, 134.725199999986, 132.4804, 110.445599999977, 86.7939999999944, 56.7038000000175, 64.915399999998, 38.3726000000024, 37.1606000000029, 46.170999999973, 49.1716000000015, 15.3362000000197, 6.71639999997569, -34.8185999999987, -39.4476000000141, 12.6830000000191, -12.3331999999937, -50.6565999999875, -59.9538000000175, -65.1054000000004, -70.7576000000117, -106.325200000021, -126.852200000023, -110.227599999984, -132.885999999999, -113.897200000007, -142.713800000027, -151.145399999979, -150.799200000009, -177.756200000003, -156.036399999983, -182.735199999996, -177.259399999981,
+                    -198.663600000029, -174.577600000019, -193.84580000001, },
+            // precision 17
+            { 94541, 92848.811, 91174.019, 89517.558, 87879.9705, 86262.7565, 84663.5125, 83083.7435, 81521.7865, 79977.272, 78455.9465, 76950.219, 75465.432, 73994.152, 72546.71, 71115.2345, 69705.6765, 68314.937, 66944.2705, 65591.255, 64252.9485, 62938.016, 61636.8225, 60355.592, 59092.789, 57850.568, 56624.518, 55417.343, 54231.1415, 53067.387, 51903.526, 50774.649, 49657.6415, 48561.05, 47475.7575, 46410.159, 45364.852, 44327.053, 43318.4005, 42325.6165, 41348.4595, 40383.6265, 39436.77, 38509.502, 37594.035, 36695.939, 35818.6895, 34955.691, 34115.8095, 33293.949, 32465.0775, 31657.6715, 30877.2585, 30093.78, 29351.3695, 28594.1365, 27872.115, 27168.7465, 26477.076, 25774.541, 25106.5375, 24452.5135, 23815.5125, 23174.0655, 22555.2685, 21960.2065, 21376.3555, 20785.1925, 20211.517, 19657.0725, 19141.6865, 18579.737, 18081.3955, 17578.995, 17073.44, 16608.335, 16119.911, 15651.266, 15194.583, 14749.0495, 14343.4835, 13925.639, 13504.509, 13099.3885, 12691.2855, 12328.018,
+                    11969.0345, 11596.5145, 11245.6355, 10917.6575, 10580.9785, 10277.8605, 9926.58100000001, 9605.538, 9300.42950000003, 8989.97850000003, 8728.73249999998, 8448.3235, 8175.31050000002, 7898.98700000002, 7629.79100000003, 7413.76199999999, 7149.92300000001, 6921.12650000001, 6677.1545, 6443.28000000003, 6278.23450000002, 6014.20049999998, 5791.20299999998, 5605.78450000001, 5438.48800000001, 5234.2255, 5059.6825, 4887.43349999998, 4682.935, 4496.31099999999, 4322.52250000002, 4191.42499999999, 4021.24200000003, 3900.64799999999, 3762.84250000003, 3609.98050000001, 3502.29599999997, 3363.84250000003, 3206.54849999998, 3079.70000000001, 2971.42300000001, 2867.80349999998, 2727.08100000001, 2630.74900000001, 2496.6165, 2440.902, 2356.19150000002, 2235.58199999999, 2120.54149999999, 2012.25449999998, 1933.35600000003, 1820.93099999998, 1761.54800000001, 1663.09350000002, 1578.84600000002, 1509.48149999999, 1427.3345, 1379.56150000001, 1306.68099999998, 1212.63449999999,
+                    1084.17300000001, 1124.16450000001, 1060.69949999999, 1007.48849999998, 941.194499999983, 879.880500000028, 836.007500000007, 782.802000000025, 748.385499999975, 647.991500000004, 626.730500000005, 570.776000000013, 484.000500000024, 513.98550000001, 418.985499999952, 386.996999999974, 370.026500000036, 355.496999999974, 356.731499999994, 255.92200000002, 259.094000000041, 205.434499999974, 165.374500000034, 197.347500000033, 95.718499999959, 67.6165000000037, 54.6970000000438, 31.7395000000251, -15.8784999999916, 8.42500000004657, -26.3754999999655, -118.425500000012, -66.6629999999423, -42.9745000000112, -107.364999999991, -189.839000000036, -162.611499999999, -164.964999999967, -189.079999999958, -223.931499999948, -235.329999999958, -269.639500000048, -249.087999999989, -206.475499999942, -283.04449999996, -290.667000000016, -304.561499999953, -336.784499999951, -380.386500000022, -283.280499999993, -364.533000000054, -389.059499999974, -364.454000000027,
+                    -415.748000000021, -417.155000000028, },
+            // precision 18
+            { 189083, 185696.913, 182348.774, 179035.946, 175762.762, 172526.444, 169329.754, 166166.099, 163043.269, 159958.91, 156907.912, 153906.845, 150924.199, 147996.568, 145093.457, 142239.233, 139421.475, 136632.27, 133889.588, 131174.2, 128511.619, 125868.621, 123265.385, 120721.061, 118181.769, 115709.456, 113252.446, 110840.198, 108465.099, 106126.164, 103823.469, 101556.618, 99308.004, 97124.508, 94937.803, 92833.731, 90745.061, 88677.627, 86617.47, 84650.442, 82697.833, 80769.132, 78879.629, 77014.432, 75215.626, 73384.587, 71652.482, 69895.93, 68209.301, 66553.669, 64921.981, 63310.323, 61742.115, 60205.018, 58698.658, 57190.657, 55760.865, 54331.169, 52908.167, 51550.273, 50225.254, 48922.421, 47614.533, 46362.049, 45098.569, 43926.083, 42736.03, 41593.473, 40425.26, 39316.237, 38243.651, 37170.617, 36114.609, 35084.19, 34117.233, 33206.509, 32231.505, 31318.728, 30403.404, 29540.0550000001, 28679.236, 27825.862, 26965.216, 26179.148, 25462.08, 24645.952, 23922.523,
+                    23198.144, 22529.128, 21762.4179999999, 21134.779, 20459.117, 19840.818, 19187.04, 18636.3689999999, 17982.831, 17439.7389999999, 16874.547, 16358.2169999999, 15835.684, 15352.914, 14823.681, 14329.313, 13816.897, 13342.874, 12880.882, 12491.648, 12021.254, 11625.392, 11293.7610000001, 10813.697, 10456.209, 10099.074, 9755.39000000001, 9393.18500000006, 9047.57900000003, 8657.98499999999, 8395.85900000005, 8033, 7736.95900000003, 7430.59699999995, 7258.47699999996, 6924.58200000005, 6691.29399999999, 6357.92500000005, 6202.05700000003, 5921.19700000004, 5628.28399999999, 5404.96799999999, 5226.71100000001, 4990.75600000005, 4799.77399999998, 4622.93099999998, 4472.478, 4171.78700000001, 3957.46299999999, 3868.95200000005, 3691.14300000004, 3474.63100000005, 3341.67200000002, 3109.14000000001, 3071.97400000005, 2796.40399999998, 2756.17799999996, 2611.46999999997, 2471.93000000005, 2382.26399999997, 2209.22400000005, 2142.28399999999, 2013.96100000001,
+                    1911.18999999994, 1818.27099999995, 1668.47900000005, 1519.65800000005, 1469.67599999998, 1367.13800000004, 1248.52899999998, 1181.23600000003, 1022.71900000004, 1088.20700000005, 959.03600000008, 876.095999999903, 791.183999999892, 703.337000000058, 731.949999999953, 586.86400000006, 526.024999999907, 323.004999999888, 320.448000000091, 340.672999999952, 309.638999999966, 216.601999999955, 102.922999999952, 19.2399999999907, -0.114000000059605, -32.6240000000689, -89.3179999999702, -153.497999999905, -64.2970000000205, -143.695999999996, -259.497999999905, -253.017999999924, -213.948000000091, -397.590000000084, -434.006000000052, -403.475000000093, -297.958000000101, -404.317000000039, -528.898999999976, -506.621000000043, -513.205000000075, -479.351000000024, -596.139999999898, -527.016999999993, -664.681000000099, -680.306000000099, -704.050000000047, -850.486000000034, -757.43200000003, -713.308999999892, } };
+
+    public static boolean isLinearCounting(int precision, double estimate) {
+        return estimate <= thresholdData[precision - 4];
+    }
+
+    public static boolean isBiasCorrection(int numberOfBuckets, double estimate) {
+        return estimate <= 5 * numberOfBuckets;
+    }
+
+    public static double biasCorrection(int precision, double estimate) {
+        double[] estimateVector = rawEstimateData[precision - 4];
+        SortedMap<Double, Integer> estimateDistances = calcDistances(estimate, estimateVector);
+        int[] nearestNeighbors = getNearestNeighbors(estimateDistances);
+        double bias = getBias(precision, nearestNeighbors);
+        return estimate - bias;
+    }
+
+    public static SortedMap<Double, Integer> calcDistances(double estimate, double[] estimateVector) {
+        SortedMap<Double, Integer> distances = new TreeMap<Double, Integer>();
+        int index = 0;
+        for (double anEstimateVector : estimateVector) {
+            distances.put(Math.pow(estimate - anEstimateVector, 2), index++);
+        }
+        return distances;
+    }
+
+    public static double getBias(int precision, int[] nearestNeighbors) {
+        double[] biasVector = biasData[precision - 4];
+        double biasTotal = 0.0d;
+        for (int nearestNeighbor : nearestNeighbors) {
+            biasTotal += biasVector[nearestNeighbor];
+        }
+        return biasTotal / (nearestNeighbors.length);
+    }
+
+    public static int[] getNearestNeighbors(SortedMap<Double, Integer> distanceMap) {
+        int[] nearest = new int[6];
+        int i = 0;
+        for (Integer index : distanceMap.values()) {
+            nearest[i++] = index;
+            if (i >= 6) {
+                break;
+            }
+        }
+        return nearest;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/common/src/main/java/com/kylinolap/common/persistence/AclEntity.java
----------------------------------------------------------------------
diff --git a/common/src/main/java/com/kylinolap/common/persistence/AclEntity.java b/common/src/main/java/com/kylinolap/common/persistence/AclEntity.java
new file mode 100644
index 0000000..519f701
--- /dev/null
+++ b/common/src/main/java/com/kylinolap/common/persistence/AclEntity.java
@@ -0,0 +1,27 @@
+/*
+ * 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.common.persistence;
+
+/**
+ * @author xduo
+ * 
+ */
+public interface AclEntity {
+
+    public String getId();
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/common/src/main/java/com/kylinolap/common/persistence/FileResourceStore.java
----------------------------------------------------------------------
diff --git a/common/src/main/java/com/kylinolap/common/persistence/FileResourceStore.java b/common/src/main/java/com/kylinolap/common/persistence/FileResourceStore.java
new file mode 100644
index 0000000..fcc0312
--- /dev/null
+++ b/common/src/main/java/com/kylinolap/common/persistence/FileResourceStore.java
@@ -0,0 +1,121 @@
+/*
+ * 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.common.persistence;
+
+import java.io.ByteArrayInputStream;
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.util.ArrayList;
+
+import org.apache.commons.io.IOUtils;
+
+import com.kylinolap.common.KylinConfig;
+
+public class FileResourceStore extends ResourceStore {
+
+    File root;
+
+    public FileResourceStore(KylinConfig kylinConfig) {
+        super(kylinConfig);
+        root = new File(kylinConfig.getMetadataUrl()).getAbsoluteFile();
+        if (root.exists() == false)
+            throw new IllegalArgumentException("File not exist by '" + kylinConfig.getMetadataUrl() + "': " + root.getAbsolutePath());
+    }
+
+    @Override
+    protected ArrayList<String> listResourcesImpl(String resPath) throws IOException {
+        String[] names = file(resPath).list();
+        if (names == null) // not a directory
+            return null;
+
+        ArrayList<String> r = new ArrayList<String>(names.length);
+        String prefix = resPath.endsWith("/") ? resPath : resPath + "/";
+        for (String n : names) {
+            r.add(prefix + n);
+        }
+        return r;
+    }
+
+    @Override
+    protected boolean existsImpl(String resPath) throws IOException {
+        File f = file(resPath);
+        return f.exists() && f.isFile(); // directory is not considered a
+                                         // resource
+    }
+
+    @Override
+    protected InputStream getResourceImpl(String resPath) throws IOException {
+        File f = file(resPath);
+        if (f.exists() && f.isFile())
+            return new FileInputStream(file(resPath));
+        else
+            return null;
+    }
+
+    @Override
+    protected long getResourceTimestampImpl(String resPath) throws IOException {
+        File f = file(resPath);
+        return f.lastModified();
+    }
+
+    @Override
+    protected void putResourceImpl(String resPath, InputStream content, long ts) throws IOException {
+        File f = file(resPath);
+        f.getParentFile().mkdirs();
+        FileOutputStream out = new FileOutputStream(f);
+        try {
+            IOUtils.copy(content, out);
+        } finally {
+            IOUtils.closeQuietly(out);
+        }
+
+        f.setLastModified(ts);
+    }
+
+    @Override
+    protected long checkAndPutResourceImpl(String resPath, byte[] content, long oldTS, long newTS) throws IOException, IllegalStateException {
+        File f = file(resPath);
+        if ((f.exists() && f.lastModified() != oldTS) || (f.exists() == false && oldTS != 0))
+            throw new IllegalStateException("Overwriting conflict " + resPath + ", expect old TS " + oldTS + ", but found " + f.lastModified());
+
+        putResourceImpl(resPath, new ByteArrayInputStream(content), newTS);
+
+        // some FS lose precision on given time stamp
+        return getResourceTimestamp(resPath);
+    }
+
+    @Override
+    protected void deleteResourceImpl(String resPath) throws IOException {
+        File f = file(resPath);
+        f.delete();
+    }
+
+    @Override
+    protected String getReadableResourcePathImpl(String resPath) {
+        return file(resPath).toString();
+    }
+
+    private File file(String resPath) {
+        if (resPath.equals("/"))
+            return root;
+        else
+            return new File(root, resPath);
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/common/src/main/java/com/kylinolap/common/persistence/HBaseConnection.java
----------------------------------------------------------------------
diff --git a/common/src/main/java/com/kylinolap/common/persistence/HBaseConnection.java b/common/src/main/java/com/kylinolap/common/persistence/HBaseConnection.java
new file mode 100644
index 0000000..fe90cb5
--- /dev/null
+++ b/common/src/main/java/com/kylinolap/common/persistence/HBaseConnection.java
@@ -0,0 +1,122 @@
+/*
+ * 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.common.persistence;
+
+import java.io.IOException;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.HColumnDescriptor;
+import org.apache.hadoop.hbase.HTableDescriptor;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.TableNotFoundException;
+import org.apache.hadoop.hbase.client.HBaseAdmin;
+import org.apache.hadoop.hbase.client.HConnection;
+import org.apache.hadoop.hbase.client.HConnectionManager;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.kylinolap.common.util.HadoopUtil;
+
+/**
+ * @author yangli9
+ * 
+ */
+public class HBaseConnection {
+
+    private static final Logger logger = LoggerFactory.getLogger(HBaseConnection.class);
+
+    private static final Map<String, Configuration> ConfigCache = new ConcurrentHashMap<String, Configuration>();
+    private static final Map<String, HConnection> ConnPool = new ConcurrentHashMap<String, HConnection>();
+
+    static {
+        Runtime.getRuntime().addShutdownHook(new Thread() {
+            @Override
+            public void run() {
+                for (HConnection conn : ConnPool.values()) {
+                    try {
+                        conn.close();
+                    } catch (IOException e) {
+                        e.printStackTrace();
+                    }
+                }
+            }
+        });
+    }
+
+    public static HConnection get(String url) {
+        // find configuration
+        Configuration conf = ConfigCache.get(url);
+        if (conf == null) {
+            conf = HadoopUtil.newHBaseConfiguration(url);
+            ConfigCache.put(url, conf);
+        }
+
+        HConnection connection = ConnPool.get(url);
+        try {
+            // I don't use DCL since recreate a connection is not a big issue.
+            if (connection == null) {
+                connection = HConnectionManager.createConnection(conf);
+                ConnPool.put(url, connection);
+            }
+        } catch (Throwable t) {
+            throw new StorageException("Error when open connection " + url, t);
+        }
+
+        return connection;
+    }
+
+    public static void createHTableIfNeeded(String hbaseUrl, String tableName, String... families) throws IOException {
+        createHTableIfNeeded(HBaseConnection.get(hbaseUrl), tableName, families);
+    }
+    
+    public static void createHTableIfNeeded(HConnection conn, String tableName, String... families) throws IOException {
+        HBaseAdmin hbase = new HBaseAdmin(conn);
+
+        try {
+            boolean tableExist = false;
+            try {
+                hbase.getTableDescriptor(TableName.valueOf(tableName));
+                tableExist = true;
+            } catch (TableNotFoundException e) {
+            }
+
+            if (tableExist) {
+                logger.debug("HTable '" + tableName + "' already exists");
+                return;
+            }
+
+            logger.debug("Creating HTable '" + tableName + "'");
+
+            HTableDescriptor desc = new HTableDescriptor(TableName.valueOf(tableName));
+
+            if (null != families && families.length > 0) {
+                for (String family : families) {
+                    HColumnDescriptor fd = new HColumnDescriptor(family);
+                    fd.setInMemory(true); // metadata tables are best in memory
+                    desc.addFamily(fd);
+                }
+            }
+            hbase.createTable(desc);
+
+            logger.debug("HTable '" + tableName + "' created");
+        } finally {
+            hbase.close();
+        }
+    }
+}


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

Posted by lu...@apache.org.
migrate repo from github.com to apache git


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

Branch: refs/heads/master
Commit: 4b631f92c0b12d0de1bf3dabe23978f9318e9aaf
Parents: 3fa1ea9
Author: lukehan <lu...@apache.org>
Authored: Wed Jan 7 22:42:00 2015 +0800
Committer: lukehan <lu...@apache.org>
Committed: Wed Jan 7 22:42:00 2015 +0800

----------------------------------------------------------------------
 LICENSE                                         |     54 +
 README.md                                       |     93 +-
 .../.settings/org.eclipse.core.resources.prefs  |      6 +
 .../.settings/org.eclipse.jdt.core.prefs        |    379 +
 atopcalcite/.settings/org.eclipse.jdt.ui.prefs  |      7 +
 atopcalcite/pom.xml                             |     28 +
 .../hydromatic/optiq/runtime/SqlFunctions.java  |   1705 +
 .../eigenbase/sql2rel/SqlToRelConverter.java    |   4782 +
 .../.settings/org.eclipse.core.resources.prefs  |      5 +
 common/.settings/org.eclipse.jdt.core.prefs     |    379 +
 common/.settings/org.eclipse.jdt.ui.prefs       |      7 +
 common/pom.xml                                  |     88 +
 .../java/com/kylinolap/common/KylinConfig.java  |    630 +
 .../common/hll/HyperLogLogPlusCounter.java      |    357 +
 .../common/hll/HyperLogLogPlusTable.java        |    197 +
 .../kylinolap/common/persistence/AclEntity.java |     27 +
 .../common/persistence/FileResourceStore.java   |    121 +
 .../common/persistence/HBaseConnection.java     |    122 +
 .../common/persistence/HBaseResourceStore.java  |    307 +
 .../common/persistence/JsonSerializer.java      |     45 +
 .../common/persistence/ResourceStore.java       |    256 +
 .../common/persistence/ResourceTool.java        |    141 +
 .../persistence/RootPersistentEntity.java       |    104 +
 .../common/persistence/Serializer.java          |     33 +
 .../common/persistence/StorageException.java    |     35 +
 .../common/restclient/AbstractRestCache.java    |     47 +
 .../common/restclient/Broadcaster.java          |    261 +
 .../common/restclient/MultiValueCache.java      |     86 +
 .../kylinolap/common/restclient/RestClient.java |    127 +
 .../common/restclient/SingleValueCache.java     |     86 +
 .../common/util/AbstractKylinTestCase.java      |     41 +
 .../java/com/kylinolap/common/util/Array.java   |     69 +
 .../com/kylinolap/common/util/ByteArray.java    |     57 +
 .../kylinolap/common/util/BytesSerializer.java  |     33 +
 .../com/kylinolap/common/util/BytesUtil.java    |    348 +
 .../kylinolap/common/util/ClasspathUtil.java    |     43 +
 .../common/util/CliCommandExecutor.java         |    133 +
 .../common/util/HBaseMetadataTestCase.java      |     50 +
 .../common/util/HBaseRegionSizeCalculator.java  |    125 +
 .../com/kylinolap/common/util/HadoopUtil.java   |    106 +
 .../com/kylinolap/common/util/JsonUtil.java     |     83 +
 .../common/util/LocalFileMetadataTestCase.java  |     67 +
 .../com/kylinolap/common/util/LongAsFloat.java  |      5 +
 .../com/kylinolap/common/util/MailService.java  |    102 +
 .../kylinolap/common/util/MyLogFormatter.java   |     75 +
 .../kylinolap/common/util/RandomSampler.java    |     53 +
 .../com/kylinolap/common/util/SSHClient.java    |    268 +
 .../kylinolap/common/util/SSHClientOutput.java  |     50 +
 .../com/kylinolap/common/util/SSHLogger.java    |     25 +
 .../kylinolap/common/util/StringSplitter.java   |     44 +
 .../com/kylinolap/common/util/StringUtil.java   |     99 +
 common/src/main/resources/kylinlog4j.properties |     10 +
 common/src/main/resources/log4j.properties      |      9 +
 .../persistence/HBaseResourceStoreTest.java     |    206 +
 .../persistence/LocalFileResourceStoreTest.java |    166 +
 .../common/persistence/ResourceToolTest.java    |     28 +
 .../common/restclient/RestClientTest.java       |     25 +
 .../kylinolap/common/util/BasicHadoopTest.java  |     65 +
 .../com/kylinolap/common/util/BasicTest.java    |     21 +
 .../common/util/HyperLogLogCounterTest.java     |    209 +
 .../common/util/InstallJarIntoMavenTest.java    |     45 +
 .../kylinolap/common/util/MailServiceTest.java  |     60 +
 .../common/util/RandomSamplerTest.java          |     46 +
 .../kylinolap/common/util/SSHClientTest.java    |     83 +
 cube/.settings/org.eclipse.core.resources.prefs |      6 +
 cube/.settings/org.eclipse.jdt.core.prefs       |    379 +
 cube/.settings/org.eclipse.jdt.ui.prefs         |      7 +
 cube/pom.xml                                    |    157 +
 cube/src/main/assembly/assemble.xml             |     19 +
 .../com/kylinolap/cube/CubeBuildTypeEnum.java   |     32 +
 .../java/com/kylinolap/cube/CubeInstance.java   |    435 +
 .../java/com/kylinolap/cube/CubeManager.java    |    660 +
 .../java/com/kylinolap/cube/CubeSegment.java    |    307 +
 .../kylinolap/cube/CubeSegmentStatusEnum.java   |     25 +
 .../com/kylinolap/cube/CubeSegmentTypeEnum.java |     25 +
 .../kylinolap/cube/CubeSegmentValidator.java    |    240 +
 .../java/com/kylinolap/cube/CubeStatusEnum.java |     22 +
 .../cube/cli/DictionaryGeneratorCLI.java        |     66 +
 .../kylinolap/cube/cli/DumpDictionaryCLI.java   |     55 +
 .../kylinolap/cube/common/BytesSplitter.java    |    141 +
 .../kylinolap/cube/common/RowKeySplitter.java   |     88 +
 .../kylinolap/cube/common/SplittedBytes.java    |     30 +
 .../java/com/kylinolap/cube/cuboid/Cuboid.java  |    323 +
 .../com/kylinolap/cube/cuboid/CuboidCLI.java    |    166 +
 .../kylinolap/cube/cuboid/CuboidScheduler.java  |    190 +
 .../kylinolap/cube/dataGen/ColumnConfig.java    |     54 +
 .../cube/dataGen/FactTableGenerator.java        |    645 +
 .../com/kylinolap/cube/dataGen/GenConfig.java   |     63 +
 .../cube/estimation/CubeSizeEstimationCLI.java  |    163 +
 .../cube/exception/CubeIntegrityException.java  |     33 +
 .../cube/invertedindex/BitMapContainer.java     |    190 +
 .../invertedindex/ColumnValueContainer.java     |     36 +
 .../invertedindex/CompressedValueContainer.java |    132 +
 .../cube/invertedindex/IIKeyValueCodec.java     |    243 +
 .../cube/invertedindex/InvertedIndexCLI.java    |    101 +
 .../cube/invertedindex/ShardingHash.java        |     15 +
 .../invertedindex/ShardingSliceBuilder.java     |     35 +
 .../com/kylinolap/cube/invertedindex/Slice.java |    175 +
 .../cube/invertedindex/SliceBuilder.java        |    122 +
 .../cube/invertedindex/TableRecord.java         |    122 +
 .../cube/invertedindex/TableRecordBytes.java    |     84 +
 .../cube/invertedindex/TableRecordFactory.java  |      8 +
 .../cube/invertedindex/TableRecordInfo.java     |    170 +
 .../invertedindex/TableRecordInfoDigest.java    |     89 +
 .../cube/kv/AbstractRowKeyEncoder.java          |     75 +
 .../com/kylinolap/cube/kv/FuzzyKeyEncoder.java  |     40 +
 .../com/kylinolap/cube/kv/FuzzyMaskEncoder.java |     52 +
 .../com/kylinolap/cube/kv/RowConstants.java     |     48 +
 .../com/kylinolap/cube/kv/RowKeyColumnIO.java   |    165 +
 .../kylinolap/cube/kv/RowKeyColumnOrder.java    |    106 +
 .../com/kylinolap/cube/kv/RowKeyDecoder.java    |    137 +
 .../com/kylinolap/cube/kv/RowKeyEncoder.java    |    140 +
 .../com/kylinolap/cube/kv/RowValueDecoder.java  |    136 +
 .../cube/measure/BigDecimalMaxAggregator.java   |     51 +
 .../cube/measure/BigDecimalMinAggregator.java   |     52 +
 .../cube/measure/BigDecimalSerializer.java      |     61 +
 .../cube/measure/BigDecimalSumAggregator.java   |     48 +
 .../cube/measure/DoubleMaxAggregator.java       |     52 +
 .../cube/measure/DoubleMinAggregator.java       |     52 +
 .../cube/measure/DoubleSerializer.java          |     53 +
 .../cube/measure/DoubleSumAggregator.java       |     49 +
 .../kylinolap/cube/measure/HLLCAggregator.java  |     56 +
 .../kylinolap/cube/measure/HLLCSerializer.java  |     65 +
 .../kylinolap/cube/measure/LDCAggregator.java   |     62 +
 .../cube/measure/LongMaxAggregator.java         |     52 +
 .../cube/measure/LongMinAggregator.java         |     52 +
 .../kylinolap/cube/measure/LongSerializer.java  |     55 +
 .../cube/measure/LongSumAggregator.java         |     49 +
 .../cube/measure/MeasureAggregator.java         |    100 +
 .../cube/measure/MeasureAggregators.java        |     80 +
 .../kylinolap/cube/measure/MeasureCodec.java    |     82 +
 .../cube/measure/MeasureSerializer.java         |     66 +
 .../measure/fixedlen/FixedLenMeasureCodec.java  |     21 +
 .../measure/fixedlen/FixedPointLongCodec.java   |     54 +
 .../kylinolap/cube/project/ProjectInstance.java |    234 +
 .../kylinolap/cube/project/ProjectManager.java  |    582 +
 .../cube/project/ProjectStatusEnum.java         |     22 +
 .../kylinolap/cube/project/ProjectTable.java    |     93 +
 .../kylinolap/cube/CubeManagerCacheTest.java    |     75 +
 .../com/kylinolap/cube/CubeManagerTest.java     |     88 +
 .../cube/CubeSizeEstimationCLITest.java         |     94 +
 .../kylinolap/cube/SegmentManagementTest.java   |    710 +
 .../cube/common/BytesSplitterTest.java          |     50 +
 .../cube/common/RowKeySplitterTest.java         |     69 +
 .../kylinolap/cube/cuboid/CombinationTest.java  |     92 +
 .../cube/cuboid/CuboidSchedulerTest.java        |    309 +
 .../com/kylinolap/cube/cuboid/CuboidTest.java   |    181 +
 .../com/kylinolap/cube/dataGen/DataGenTest.java |     32 +
 .../invertedindex/InvertedIndexLocalTest.java   |    198 +
 .../kylinolap/cube/kv/RowKeyDecoderTest.java    |    115 +
 .../kylinolap/cube/kv/RowKeyEncoderTest.java    |    143 +
 .../kylinolap/cube/kv/RowValueDecoderTest.java  |     88 +
 .../cube/measure/MeasureCodecTest.java          |     75 +
 .../cube/project/ProjectManagerTest.java        |    165 +
 cube/src/test/resources/placeholder.txt         |      0
 deploy.sh                                       |    206 +
 deploy/install.sh                               |     15 +
 deploy/server.xml                               |    156 +
 deploy/web.xml                                  |   4618 +
 .../.settings/org.eclipse.core.resources.prefs  |      6 +
 dictionary/.settings/org.eclipse.jdt.core.prefs |    379 +
 dictionary/.settings/org.eclipse.jdt.ui.prefs   |      7 +
 dictionary/pom.xml                              |     78 +
 .../java/com/kylinolap/dict/BytesConverter.java |     23 +
 .../com/kylinolap/dict/DateStrDictionary.java   |    240 +
 .../java/com/kylinolap/dict/Dictionary.java     |    189 +
 .../com/kylinolap/dict/DictionaryGenerator.java |    203 +
 .../java/com/kylinolap/dict/DictionaryInfo.java |    169 +
 .../dict/DictionaryInfoSerializer.java          |     76 +
 .../com/kylinolap/dict/DictionaryManager.java   |    345 +
 .../com/kylinolap/dict/NumberDictionary.java    |    181 +
 .../kylinolap/dict/NumberDictionaryBuilder.java |     46 +
 .../kylinolap/dict/StringBytesConverter.java    |     32 +
 .../java/com/kylinolap/dict/TrieDictionary.java |    478 +
 .../kylinolap/dict/TrieDictionaryBuilder.java   |    536 +
 .../dict/lookup/DictionaryLookupTable.java      |    109 +
 .../com/kylinolap/dict/lookup/FileTable.java    |     68 +
 .../kylinolap/dict/lookup/FileTableReader.java  |    218 +
 .../com/kylinolap/dict/lookup/HiveTable.java    |    129 +
 .../kylinolap/dict/lookup/LookupBytesTable.java |     50 +
 .../dict/lookup/LookupStringTable.java          |     43 +
 .../com/kylinolap/dict/lookup/LookupTable.java  |    167 +
 .../kylinolap/dict/lookup/ReadableTable.java    |     35 +
 .../kylinolap/dict/lookup/SnapshotManager.java  |    166 +
 .../kylinolap/dict/lookup/SnapshotTable.java    |    180 +
 .../dict/lookup/SnapshotTableSerializer.java    |     61 +
 .../com/kylinolap/dict/lookup/TableReader.java  |     34 +
 .../kylinolap/dict/lookup/TableSignature.java   |    109 +
 .../com/kylinolap/dict/common_indicators.txt    |     13 +
 .../com/kylinolap/dict/date(yyyy-mm-dd).txt     |      2 +
 .../kylinolap/dict/DateStrDictionaryTest.java   |     72 +
 .../kylinolap/dict/DictionaryManagerTest.java   |     80 +
 .../com/kylinolap/dict/LookupTableTest.java     |     70 +
 .../kylinolap/dict/NumberDictionaryTest.java    |    154 +
 .../com/kylinolap/dict/SnapshotManagerTest.java |     81 +
 .../com/kylinolap/dict/TableReaderTest.java     |     45 +
 .../com/kylinolap/dict/TrieDictionaryTest.java  |    337 +
 dictionary/src/test/resources/dict/DW_SITES     |    Bin 0 -> 5020 bytes
 .../dict/dw_category_grouping_names.dat         | 448529 ++++++++++++++++
 dictionary/src/test/resources/dict/eng_com.dic  | 150843 ++++++
 docs/Apache Kylin 2014 Dec.pdf                  |    Bin 0 -> 2043052 bytes
 docs/README.md                                  |      0
 examples/.gitignore                             |      2 +
 examples/ReleaseNotes.md                        |     10 +
 .../localmeta/cube/test_kylin_cube_ii.json      |     39 +
 .../test_kylin_cube_with_slr_1_new_segment.json |     43 +
 .../cube/test_kylin_cube_with_slr_empty.json    |     10 +
 ...est_kylin_cube_with_slr_left_join_empty.json |     11 +
 ...est_kylin_cube_with_slr_left_join_ready.json |     43 +
 .../cube/test_kylin_cube_with_slr_ready.json    |     45 +
 ...st_kylin_cube_with_slr_ready_2_segments.json |     73 +
 .../cube/test_kylin_cube_without_slr_empty.json |     10 +
 ..._kylin_cube_without_slr_left_join_empty.json |     11 +
 ..._kylin_cube_without_slr_left_join_ready.json |     43 +
 ..._without_slr_left_join_ready_2_segments.json |     73 +
 .../cube/test_kylin_cube_without_slr_ready.json |     43 +
 .../localmeta/cube_desc/test_kylin_cube_ii.json |    130 +
 .../test_kylin_cube_with_slr_desc.json          |    176 +
 ...test_kylin_cube_with_slr_left_join_desc.json |    186 +
 .../test_kylin_cube_without_slr_desc.json       |    285 +
 ...t_kylin_cube_without_slr_left_join_desc.json |    283 +
 .../test_case_data/localmeta/data/.gitignore    |      1 +
 .../localmeta/data/TEST_CAL_DT.csv              |    731 +
 .../localmeta/data/TEST_CATEGORY_GROUPINGS.csv  |    144 +
 .../localmeta/data/TEST_KYLIN_FACT.csv          |    402 +
 .../localmeta/data/TEST_SELLER_TYPE_DIM.csv     |      8 +
 .../localmeta/data/TEST_SITES.csv               |    262 +
 .../localmeta/data/data_gen_config.json         |     29 +
 .../64ac4f82-f2af-476e-85b9-f0805001014e.dict   |    Bin 0 -> 554 bytes
 .../f5e85644-db92-42b5-9ad5-240ab227d7b0.dict   |    Bin 0 -> 554 bytes
 .../aa634e3e-22d3-4cc2-9de5-085e9ac35c1f.dict   |    Bin 0 -> 518 bytes
 .../ed0c3451-593c-494c-9019-64f63fcb0b8e.dict   |    Bin 0 -> 518 bytes
 .../16d8185c-ee6b-4f8c-a919-756d9809f937.dict   |    Bin 0 -> 2515 bytes
 .../9bb0b83a-97b2-434f-905f-4d9e04b62018.dict   |    Bin 0 -> 2488 bytes
 .../c12ae49d-9dbe-4a58-b169-19afac317696.dict   |    Bin 0 -> 2515 bytes
 .../eaed91b0-4182-4ee5-a733-1047a622ee29.dict   |    Bin 0 -> 2515 bytes
 .../ec83ebce-7534-4e62-ac6d-7445ee141ab4.dict   |    Bin 0 -> 2515 bytes
 .../270fbfb0-281c-4602-8413-2970a7439c47.dict   |    Bin 0 -> 2913 bytes
 .../7228fad8-a764-4bd4-8934-50e0d7cbcb19.dict   |    Bin 0 -> 2940 bytes
 .../8f92faa4-7d2d-455c-8623-6e1d1b272afe.dict   |    Bin 0 -> 2940 bytes
 .../ad09f2d5-054a-4e1b-a776-7cc07399a6c1.dict   |    Bin 0 -> 2940 bytes
 .../eacccee7-d120-4f4d-97d0-c99a5b83ec32.dict   |    Bin 0 -> 2940 bytes
 .../2602386c-debb-4968-8d2f-b52b8215e385.dict   |    Bin 0 -> 1841 bytes
 .../4243889f-bc81-4807-a975-7041bbbf35e7.dict   |    Bin 0 -> 1841 bytes
 .../494683c1-52d1-4d9a-a3f7-8c53b30fd898.dict   |    Bin 0 -> 1841 bytes
 .../8b4b1c06-fb74-486b-a2ad-74420afebcda.dict   |    Bin 0 -> 1841 bytes
 .../ac520edd-f9d2-419f-a7de-587bfb97dc81.dict   |    Bin 0 -> 1841 bytes
 .../0410d2c4-4686-40bc-ba14-170042a2de94.dict   |    Bin 0 -> 1498 bytes
 .../51ff5e6c-22b3-444e-9915-d376a10f20cb.dict   |    Bin 0 -> 1498 bytes
 .../70c8d6ef-f55b-4cdd-8be4-9c2b236cc8e9.dict   |    Bin 0 -> 1471 bytes
 .../a4e57e55-48fc-4f25-a9c8-485deed25925.dict   |    Bin 0 -> 1498 bytes
 .../ac797050-e152-4923-a20a-02282fcf6086.dict   |    Bin 0 -> 1471 bytes
 .../aceae914-4246-4251-a0c2-692fe7a300df.dict   |    Bin 0 -> 1498 bytes
 .../b298089f-9656-4693-b9b2-8fea46f06dd5.dict   |    Bin 0 -> 1498 bytes
 .../c2af25cf-6c79-45e6-a6f6-6d2a8ecc6592.dict   |    Bin 0 -> 1498 bytes
 .../48433f91-0d68-495f-b7f2-295414591275.dict   |    Bin 0 -> 480 bytes
 .../73c9bfe1-6496-4ff8-9467-6cbee2924c16.dict   |    Bin 0 -> 552 bytes
 .../4059cab1-9b99-47ed-a494-751da52a7d58.dict   |    Bin 0 -> 1730 bytes
 .../96b7c577-b209-45b3-a848-4d2d7af5c0cc.dict   |    Bin 0 -> 1842 bytes
 .../9fc2360e-172a-43f3-acef-be16748a9bb7.dict   |    Bin 0 -> 1751 bytes
 .../bd9f6b22-36ba-4e6b-92aa-0d585faf0b39.dict   |    Bin 0 -> 593 bytes
 .../7df5789b-0280-453c-b406-b75cad6770d1.dict   |    Bin 0 -> 613 bytes
 .../927dde3f-6999-4434-b57c-adfa73160334.dict   |    Bin 0 -> 1223 bytes
 .../6e285e1e-ed16-4012-9f1e-f950dd6927ce.dict   |    Bin 0 -> 1117 bytes
 .../8300bf83-053e-48bb-8c87-88c8d483afd1.dict   |    Bin 0 -> 586 bytes
 .../c8d19f95-b6cd-4219-a114-54aaddcb2909.dict   |    Bin 0 -> 1707 bytes
 .../0c5d77ec-316b-47e0-ba9a-0616be890ad6.dict   |    Bin 0 -> 661 bytes
 .../14fe66b3-5956-498c-bd93-40182cac5510.dict   |    Bin 0 -> 661 bytes
 .../1d383a36-81b9-4177-a822-04eab3683e5b.dict   |    Bin 0 -> 661 bytes
 .../2a44ff38-f64b-42e7-9fcf-66afccac8047.dict   |    Bin 0 -> 634 bytes
 .../c6bf9b51-6e90-4337-8082-4e2fdf78307f.dict   |    Bin 0 -> 661 bytes
 .../0bec6bb3-1b0d-469c-8289-b8c4ca5d5001.dict   |    Bin 0 -> 2189 bytes
 .../31edf35b-ffca-4f24-8229-f87dc34e3087.dict   |    Bin 0 -> 2189 bytes
 .../652bd393-678a-4f16-a504-fd8ce1229355.dict   |    Bin 0 -> 2189 bytes
 .../792eb972-d046-48e6-9428-f6a3aed92fad.dict   |    Bin 0 -> 2189 bytes
 .../914536f8-1e14-43e9-b8ef-525bb5bdb3b2.dict   |    Bin 0 -> 2189 bytes
 .../d25e554e-deac-4e4a-9289-96f0d121d8fc.dict   |    Bin 0 -> 2189 bytes
 .../ff7e8943-ac0f-4e66-b9ed-510f6a0b875d.dict   |    Bin 0 -> 2162 bytes
 .../invertedindex_desc/test_kylin_cube_ii.json  |     24 +
 .../test_case_data/localmeta/kylin.properties   |     66 +
 .../test_case_data/localmeta/kylin_job_conf.xml |     60 +
 .../localmeta/project/default.json              |     10 +
 .../localmeta/project/onlyinner.json            |      8 +
 .../localmeta/project/onlyleft.json             |      8 +
 .../localmeta/table/TEST_CAL_DT.json            |    407 +
 .../table/TEST_CATEGORY_GROUPINGS.json          |    151 +
 .../localmeta/table/TEST_KYLIN_FACT.json        |     43 +
 .../localmeta/table/TEST_SELLER_TYPE_DIM.json   |     43 +
 .../localmeta/table/TEST_SITES.json             |     47 +
 ...6337839-a90f-46fa-a67b-37e330fe9685.snapshot |    Bin 0 -> 588604 bytes
 ...af48c94-86de-4e22-a4fd-c49b06cbaa4f.snapshot |    Bin 0 -> 588631 bytes
 ...6de9855-8815-4f3b-8ac6-c9386db0b196.snapshot |    Bin 0 -> 588631 bytes
 ...ff1339e-f804-47f3-b42c-1d4fa4ff0cf7.snapshot |    Bin 0 -> 588631 bytes
 ...a1fbc0d-8790-4f32-99e1-7c60a9d124d4.snapshot |    Bin 0 -> 588631 bytes
 ...5f004b5-0be1-4ccf-8950-20f8f86c1999.snapshot |    Bin 0 -> 588631 bytes
 ...6cbec7f-0326-443d-b45f-54549d5609c6.snapshot |    Bin 0 -> 588631 bytes
 ...91cd3a0-3ae5-4306-87ab-1624d45a6cde.snapshot |    Bin 0 -> 588604 bytes
 ...9a675fc-64d2-40d1-bbd7-492db2510cd1.snapshot |    Bin 0 -> 55219 bytes
 ...9715f33-15c9-4745-83f9-f2b9817d9100.snapshot |    Bin 0 -> 55219 bytes
 ...f8a90f1-bbbe-49ff-937a-2a2a6e5f8615.snapshot |    Bin 0 -> 55219 bytes
 ...172b442-ae10-447e-9071-c7dbb2bb38cc.snapshot |    Bin 0 -> 55219 bytes
 ...fe75ccd-9b24-4cdf-ac9d-b4038e947f89.snapshot |    Bin 0 -> 988 bytes
 ...7238b77-a45d-420f-a249-5f1572ea4d62.snapshot |    Bin 0 -> 988 bytes
 ...43dd3f1-9a34-4721-8abc-df90962e94d8.snapshot |    Bin 0 -> 988 bytes
 ...1b6a60a-14d4-4653-92f7-c96d01191c75.snapshot |    Bin 0 -> 961 bytes
 ...818054b-1a76-4cbd-bdd8-7871a240360c.snapshot |    Bin 0 -> 988 bytes
 ...271e1f2-dd65-4249-a1f2-aca02a651158.snapshot |    Bin 0 -> 21417 bytes
 ...888015f-f059-4ccc-8324-0cdd70b46673.snapshot |    Bin 0 -> 21417 bytes
 ...8130338-fcf4-429e-91b0-cd8dfd397280.snapshot |    Bin 0 -> 21417 bytes
 ...d70b82e-43f6-4999-b012-0e91a9bb8408.snapshot |    Bin 0 -> 21417 bytes
 ...95e0b5b-a399-4f08-8f40-b0fa53b565e0.snapshot |    Bin 0 -> 21398 bytes
 .../sandbox/capacity-scheduler.xml              |    111 +
 examples/test_case_data/sandbox/core-site.xml   |     93 +
 .../test_case_data/sandbox/hadoop-policy.xml    |    219 +
 examples/test_case_data/sandbox/hbase-site.xml  |    179 +
 examples/test_case_data/sandbox/hdfs-site.xml   |    175 +
 examples/test_case_data/sandbox/httpfs-site.xml |     17 +
 .../test_case_data/sandbox/kylin.properties     |     75 +
 .../sandbox/kylin_job_conf.lzo_disabled.xml     |     60 +
 .../sandbox/kylin_job_conf.lzo_enabled.xml      |     60 +
 .../test_case_data/sandbox/kylin_job_conf.xml   |     60 +
 examples/test_case_data/sandbox/mapred-site.xml |    155 +
 examples/test_case_data/sandbox/yarn-site.xml   |    147 +
 jdbc/.settings/org.eclipse.core.resources.prefs |      5 +
 jdbc/.settings/org.eclipse.jdt.core.prefs       |    379 +
 jdbc/.settings/org.eclipse.jdt.ui.prefs         |      7 +
 jdbc/pom.xml                                    |     80 +
 .../main/java/com/kylinolap/jdbc/Driver.java    |    137 +
 .../com/kylinolap/jdbc/KylinConnectionImpl.java |    157 +
 .../com/kylinolap/jdbc/KylinEnumerator.java     |     85 +
 .../com/kylinolap/jdbc/KylinJdbc40Factory.java  |     28 +
 .../com/kylinolap/jdbc/KylinJdbc41Factory.java  |    125 +
 .../java/com/kylinolap/jdbc/KylinMetaImpl.java  |    825 +
 .../java/com/kylinolap/jdbc/KylinPrepare.java   |     96 +
 .../com/kylinolap/jdbc/KylinPrepareImpl.java    |     45 +
 .../jdbc/KylinPrepareStatementImpl.java         |    168 +
 .../java/com/kylinolap/jdbc/KylinResultSet.java |     43 +
 .../com/kylinolap/jdbc/KylinStatementImpl.java  |     52 +
 .../jdbc/stub/ConnectionException.java          |     53 +
 .../java/com/kylinolap/jdbc/stub/DataSet.java   |     53 +
 .../com/kylinolap/jdbc/stub/KylinClient.java    |    376 +
 .../jdbc/stub/KylinColumnMetaData.java          |     36 +
 .../com/kylinolap/jdbc/stub/RemoteClient.java   |     57 +
 .../kylinolap/jdbc/stub/SQLResponseStub.java    |    320 +
 .../com/kylinolap/jdbc/stub/TableMetaStub.java  |    329 +
 .../util/DefaultSslProtocolSocketFactory.java   |    148 +
 .../jdbc/util/DefaultX509TrustManager.java      |    112 +
 .../com/kylinolap/jdbc/util/SQLTypeMap.java     |    179 +
 .../com-kylinolap-kylin-jdbc.properties         |     10 +
 jdbc/src/main/resources/log4j.properties        |     14 +
 .../java/com/kylinolap/jdbc/DriverTest.java     |    158 +
 .../java/com/kylinolap/jdbc/DummyClient.java    |     69 +
 .../java/com/kylinolap/jdbc/DummyDriver.java    |     16 +
 .../com/kylinolap/jdbc/DummyJdbc41Factory.java  |     19 +
 job/.gitignore                                  |      1 +
 job/.settings/org.eclipse.core.resources.prefs  |      6 +
 job/.settings/org.eclipse.jdt.core.prefs        |    379 +
 job/.settings/org.eclipse.jdt.ui.prefs          |      7 +
 job/pom.xml                                     |    196 +
 job/src/main/assembly/job.xml                   |     22 +
 job/src/main/java/com/kylinolap/job/JobDAO.java |    240 +
 .../java/com/kylinolap/job/JobInstance.java     |    479 +
 .../com/kylinolap/job/JobInstanceBuilder.java   |    490 +
 .../main/java/com/kylinolap/job/JobManager.java |    314 +
 .../java/com/kylinolap/job/JobStepOutput.java   |     71 +
 .../java/com/kylinolap/job/JoinedFlatTable.java |    220 +
 .../com/kylinolap/job/cmd/ICommandOutput.java   |     41 +
 .../java/com/kylinolap/job/cmd/IJobCommand.java |     30 +
 .../com/kylinolap/job/cmd/JavaHadoopCmd.java    |     73 +
 .../kylinolap/job/cmd/JavaHadoopCmdOutput.java  |    180 +
 .../kylinolap/job/cmd/JobCommandFactory.java    |     97 +
 .../java/com/kylinolap/job/cmd/ShellCmd.java    |    236 +
 .../com/kylinolap/job/cmd/ShellCmdOutput.java   |     82 +
 .../com/kylinolap/job/cmd/ShellHadoopCmd.java   |     60 +
 .../kylinolap/job/cmd/ShellHadoopCmdOutput.java |    161 +
 .../kylinolap/job/constant/BatchConstants.java  |     46 +
 .../kylinolap/job/constant/JobConstants.java    |     57 +
 .../kylinolap/job/constant/JobStatusEnum.java   |     46 +
 .../job/constant/JobStepCmdTypeEnum.java        |     25 +
 .../job/constant/JobStepStatusEnum.java         |     48 +
 .../job/constant/SchedulerTypeEnum.java         |     25 +
 .../deployment/DeploymentUtilityChecker.java    |     21 +
 .../job/deployment/HbaseConfigPrinter.java      |    131 +
 .../com/kylinolap/job/engine/JobEngine.java     |    227 +
 .../kylinolap/job/engine/JobEngineConfig.java   |    254 +
 .../com/kylinolap/job/engine/JobFetcher.java    |    117 +
 .../kylinolap/job/engine/QuatzScheduler.java    |    193 +
 .../exception/InvalidJobInstanceException.java  |     30 +
 .../exception/InvalidJobStatusException.java    |     33 +
 .../kylinolap/job/exception/JobException.java   |     56 +
 .../kylinolap/job/flow/AsyncJobFlowNode.java    |    119 +
 .../java/com/kylinolap/job/flow/JobFlow.java    |    159 +
 .../com/kylinolap/job/flow/JobFlowListener.java |    419 +
 .../com/kylinolap/job/flow/JobFlowNode.java     |    165 +
 .../kylinolap/job/hadoop/AbstractHadoopJob.java |    291 +
 .../cardinality/ColumnCardinalityMapper.java    |     81 +
 .../cardinality/ColumnCardinalityReducer.java   |     86 +
 .../cardinality/HiveColumnCardinalityJob.java   |    254 +
 .../job/hadoop/cube/BaseCuboidJob.java          |     35 +
 .../job/hadoop/cube/BaseCuboidMapper.java       |    246 +
 .../kylinolap/job/hadoop/cube/CubeHFileJob.java |    113 +
 .../job/hadoop/cube/CubeHFileMapper.java        |    178 +
 .../kylinolap/job/hadoop/cube/CuboidJob.java    |    194 +
 .../job/hadoop/cube/CuboidReducer.java          |     98 +
 .../cube/FactDistinctColumnsCombiner.java       |     54 +
 .../job/hadoop/cube/FactDistinctColumnsJob.java |    129 +
 .../hadoop/cube/FactDistinctColumnsMapper.java  |    111 +
 .../hadoop/cube/FactDistinctColumnsReducer.java |     89 +
 .../job/hadoop/cube/KeyDistributionJob.java     |    139 +
 .../job/hadoop/cube/KeyDistributionMapper.java  |    104 +
 .../job/hadoop/cube/KeyDistributionReducer.java |     93 +
 .../job/hadoop/cube/MergeCuboidJob.java         |    112 +
 .../job/hadoop/cube/MergeCuboidMapper.java      |    174 +
 .../kylinolap/job/hadoop/cube/NDCuboidJob.java  |     36 +
 .../job/hadoop/cube/NDCuboidMapper.java         |    138 +
 .../job/hadoop/cube/NewBaseCuboidMapper.java    |    342 +
 .../hadoop/cube/RangeKeyDistributionJob.java    |    116 +
 .../hadoop/cube/RangeKeyDistributionMapper.java |     64 +
 .../cube/RangeKeyDistributionReducer.java       |     91 +
 .../cube/RowKeyDistributionCheckerJob.java      |     97 +
 .../cube/RowKeyDistributionCheckerMapper.java   |    107 +
 .../cube/RowKeyDistributionCheckerReducer.java  |     44 +
 .../job/hadoop/cube/StorageCleanupJob.java      |    217 +
 .../job/hadoop/dict/CreateDictionaryJob.java    |     67 +
 .../dict/CreateInvertedIndexDictionaryJob.java  |     66 +
 .../kylinolap/job/hadoop/hbase/BulkLoadJob.java |     95 +
 .../job/hadoop/hbase/CreateHTableJob.java       |    186 +
 .../job/hadoop/hive/JoinedFlatTableDesc.java    |    171 +
 .../job/hadoop/hive/SqlHiveDataTypeMapping.java |     43 +
 .../job/hadoop/invertedindex/IIBulkLoadJob.java |     83 +
 .../hadoop/invertedindex/IICreateHFileJob.java  |     96 +
 .../invertedindex/IICreateHFileMapper.java      |     53 +
 .../hadoop/invertedindex/IICreateHTableJob.java |    118 +
 .../IIDistinctColumnsCombiner.java              |     54 +
 .../invertedindex/IIDistinctColumnsJob.java     |    153 +
 .../invertedindex/IIDistinctColumnsMapper.java  |     70 +
 .../invertedindex/IIDistinctColumnsReducer.java |     73 +
 .../hadoop/invertedindex/InvertedIndexJob.java  |    163 +
 .../invertedindex/InvertedIndexMapper.java      |     95 +
 .../invertedindex/InvertedIndexPartitioner.java |     73 +
 .../invertedindex/InvertedIndexReducer.java     |     94 +
 .../invertedindex/RandomKeyDistributionJob.java |    114 +
 .../RandomKeyDistributionMapper.java            |     66 +
 .../RandomKeyDistributionReducer.java           |     63 +
 .../kylinolap/job/tools/CubeMigrationCLI.java   |    418 +
 .../tools/DefaultSslProtocolSocketFactory.java  |    148 +
 .../job/tools/DefaultX509TrustManager.java      |    112 +
 .../job/tools/DeployCoprocessorCLI.java         |    270 +
 .../job/tools/HadoopStatusChecker.java          |    194 +
 .../job/tools/HtableAlterMetadataCLI.java       |     72 +
 .../job/tools/LZOSupportnessChecker.java        |     21 +
 .../com/kylinolap/job/tools/OptionsHelper.java  |     78 +
 .../kylinolap/job/BuildCubeWithEngineTest.java  |    252 +
 .../test/java/com/kylinolap/job/DeployUtil.java |    216 +
 .../test/java/com/kylinolap/job/JobDAOTest.java |    107 +
 .../java/com/kylinolap/job/JobInstanceTest.java |    101 +
 .../com/kylinolap/job/SampleCubeSetupTest.java  |     84 +
 .../job/engine/GenericJobEngineTest.java        |    441 +
 .../job/engine/JobInstanceBuilderTest.java      |    183 +
 .../job/hadoop/cube/BaseCuboidJobTest.java      |     80 +
 .../cube/BaseCuboidMapperPerformanceTest.java   |     63 +
 .../job/hadoop/cube/BaseCuboidMapperTest.java   |    144 +
 .../com/kylinolap/job/hadoop/cube/CopySeq.java  |     79 +
 .../job/hadoop/cube/CubeHFileMapper2Test.java   |     92 +
 .../job/hadoop/cube/CubeHFileMapperTest.java    |     78 +
 .../job/hadoop/cube/CubeReducerTest.java        |    122 +
 .../job/hadoop/cube/KeyDistributionJobTest.java |     64 +
 .../hadoop/cube/KeyDistributionMapperTest.java  |    153 +
 .../job/hadoop/cube/MergeCuboidJobTest.java     |     86 +
 .../job/hadoop/cube/MergeCuboidMapperTest.java  |    189 +
 .../job/hadoop/cube/MockupMapContext.java       |    312 +
 .../job/hadoop/cube/NDCuboidJobTest.java        |     85 +
 .../job/hadoop/cube/NDCuboidMapperTest.java     |    101 +
 .../cube/RandomKeyDistributionMapperTest.java   |     70 +
 .../cube/RandomKeyDistributionReducerTest.java  |     69 +
 .../cube/RangeKeyDistributionJobTest.java       |     69 +
 .../cube/RangeKeyDistributionMapperTest.java    |    109 +
 .../cube/RangeKeyDistributionReducerTest.java   |    105 +
 .../job/hadoop/hbase/CreateHTableTest.java      |     63 +
 .../job/hadoop/hbase/TestHbaseClient.java       |     84 +
 .../kylinolap/job/hadoop/hdfs/HdfsOpsTest.java  |     50 +
 .../job/hadoop/hive/JoinedFlatTableTest.java    |     81 +
 .../job/tools/ColumnCardinalityJobTest.java     |     60 +
 .../job/tools/ColumnCardinalityMapperTest.java  |    125 +
 .../job/tools/ColumnCardinalityReducerTest.java |    111 +
 .../kylinolap/job/tools/CubeMigrationTests.java |     40 +
 .../test/resources/data/6d_cuboid/part-r-00000  |    Bin 0 -> 3279797 bytes
 .../resources/data/base_cuboid/part-r-00000     |    Bin 0 -> 870735 bytes
 job/src/test/resources/data/flat_table/000000_0 |    Bin 0 -> 249140 bytes
 .../resources/data/test_cal_dt/part-r-00000     |    366 +
 .../expected_result/flat_item/part-r-00000      |    Bin 0 -> 565 bytes
 .../jarfile/SampleBadJavaProgram.jarfile        |    Bin 0 -> 1006 bytes
 .../resources/jarfile/SampleJavaProgram.jarfile |    Bin 0 -> 1166 bytes
 .../test/resources/json/dummy_jobinstance.json  |    195 +
 .../resources/partition_list/_partition.lst     |    Bin 0 -> 326 bytes
 .../test/resources/partition_list/part-r-00000  |    Bin 0 -> 15004 bytes
 kylin_architecture.png                          |    Bin 0 -> 449125 bytes
 .../.settings/org.eclipse.core.resources.prefs  |      6 +
 metadata/.settings/org.eclipse.jdt.core.prefs   |    379 +
 metadata/.settings/org.eclipse.jdt.ui.prefs     |      7 +
 metadata/pom.xml                                |    108 +
 metadata/src/main/assembly/assemble.xml         |     21 +
 .../kylinolap/metadata/MetadataConstances.java  |     52 +
 .../com/kylinolap/metadata/MetadataManager.java |    482 +
 .../kylinolap/metadata/model/cube/CubeDesc.java |    827 +
 .../metadata/model/cube/CubePartitionDesc.java  |     86 +
 .../metadata/model/cube/DimensionDesc.java      |    209 +
 .../metadata/model/cube/FunctionDesc.java       |    193 +
 .../metadata/model/cube/HBaseColumnDesc.java    |    117 +
 .../model/cube/HBaseColumnFamilyDesc.java       |     57 +
 .../metadata/model/cube/HBaseMappingDesc.java   |     92 +
 .../metadata/model/cube/HierarchyDesc.java      |     65 +
 .../kylinolap/metadata/model/cube/JoinDesc.java |    134 +
 .../metadata/model/cube/MeasureDesc.java        |    100 +
 .../metadata/model/cube/ParameterDesc.java      |    122 +
 .../metadata/model/cube/RowKeyColDesc.java      |     84 +
 .../metadata/model/cube/RowKeyDesc.java         |    290 +
 .../metadata/model/cube/TblColRef.java          |    141 +
 .../model/invertedindex/InvertedIndexDesc.java  |    146 +
 .../metadata/model/schema/ColumnDesc.java       |    125 +
 .../metadata/model/schema/DataType.java         |    288 +
 .../metadata/model/schema/DatabaseDesc.java     |     75 +
 .../metadata/model/schema/TableDesc.java        |    129 +
 .../metadata/tool/HiveSourceTableLoader.java    |    251 +
 .../validation/CubeMetadataValidator.java       |     73 +
 .../metadata/validation/IValidatorRule.java     |     28 +
 .../metadata/validation/ResultLevel.java        |     36 +
 .../SourceTableMetadataValidator.java           |     32 +
 .../metadata/validation/ValidateContext.java    |    101 +
 .../rule/AggregationGroupSizeRule.java          |     64 +
 .../metadata/validation/rule/FunctionRule.java  |    181 +
 .../rule/IKylinValidationConstances.java        |     31 +
 .../validation/rule/MandatoryColumnRule.java    |     74 +
 .../validation/rule/RowKeyAttrRule.java         |     73 +
 .../com/kylinolap/metadata/CubeDescTest.java    |     50 +
 .../kylinolap/metadata/MetadataManagerTest.java |     73 +
 .../rule/AggregationGroupSizeRuleTest.java      |     98 +
 .../rule/MandatoryColumnRuleTest.java           |     55 +
 .../validation/rule/RowKeyAttrRuleTest.java     |     56 +
 .../com/kylinolap/metadata/kylin_env.properties |      5 +
 .../src/test/resources/data/TEST1_desc.json     |    192 +
 .../src/test/resources/data/TEST2_desc.json     |    192 +
 .../src/test/resources/data/TEST3_desc.json     |    192 +
 metadata/src/test/test.json                     |    181 +
 package.sh                                      |     19 +
 pom.xml                                         |    542 +
 .../.settings/org.eclipse.core.resources.prefs  |      6 +
 query/.settings/org.eclipse.jdt.core.prefs      |    379 +
 query/.settings/org.eclipse.jdt.ui.prefs        |      7 +
 query/pom.xml                                   |    183 +
 .../main/java/com/kylinolap/query/QueryCli.java |     86 +
 .../query/enumerator/CubeEnumerator.java        |    223 +
 .../query/enumerator/HiveEnumerator.java        |    130 +
 .../query/enumerator/LookupTableEnumerator.java |     96 +
 .../kylinolap/query/enumerator/OLAPQuery.java   |     65 +
 .../query/optrule/OLAPAggregateRule.java        |     52 +
 .../kylinolap/query/optrule/OLAPFilterRule.java |     49 +
 .../kylinolap/query/optrule/OLAPJoinRule.java   |     81 +
 .../kylinolap/query/optrule/OLAPLimitRule.java  |     56 +
 .../query/optrule/OLAPProjectRule.java          |     48 +
 .../kylinolap/query/optrule/OLAPSortRule.java   |     51 +
 .../optrule/OLAPToEnumerableConverterRule.java  |     46 +
 .../kylinolap/query/relnode/ColumnRowType.java  |     84 +
 .../query/relnode/OLAPAggregateRel.java         |    370 +
 .../kylinolap/query/relnode/OLAPContext.java    |    122 +
 .../kylinolap/query/relnode/OLAPFilterRel.java  |    326 +
 .../kylinolap/query/relnode/OLAPJoinRel.java    |    294 +
 .../kylinolap/query/relnode/OLAPLimitRel.java   |    134 +
 .../kylinolap/query/relnode/OLAPProjectRel.java |    299 +
 .../com/kylinolap/query/relnode/OLAPRel.java    |    156 +
 .../kylinolap/query/relnode/OLAPSortRel.java    |    153 +
 .../kylinolap/query/relnode/OLAPTableScan.java  |    246 +
 .../relnode/OLAPToEnumerableConverter.java      |    109 +
 .../query/routing/CubeNotFoundException.java    |     34 +
 .../kylinolap/query/routing/QueryRouter.java    |    311 +
 .../com/kylinolap/query/schema/OLAPSchema.java  |    123 +
 .../query/schema/OLAPSchemaFactory.java         |    127 +
 .../com/kylinolap/query/schema/OLAPTable.java   |    235 +
 .../query/sqlfunc/HLLDistinctCountAggFunc.java  |    151 +
 .../kylinolap/query/sqlfunc/QuarterBase.java    |     20 +
 .../kylinolap/query/sqlfunc/QuarterFunc.java    |     32 +
 .../kylinolap/query/test/CombinationTest.java   |     55 +
 .../com/kylinolap/query/test/H2Database.java    |    134 +
 .../kylinolap/query/test/KylinQueryTest.java    |    259 +
 .../com/kylinolap/query/test/KylinTestBase.java |    471 +
 query/src/test/resources/logging.properties     |      5 +
 query/src/test/resources/query/h2/query07.sql   |      3 +
 query/src/test/resources/query/h2/query09.sql   |      5 +
 query/src/test/resources/query/h2/query10.sql   |      7 +
 query/src/test/resources/query/sql/.gitignore   |      2 +
 query/src/test/resources/query/sql/query00.sql  |      4 +
 query/src/test/resources/query/sql/query01.sql  |      2 +
 query/src/test/resources/query/sql/query02.sql  |      1 +
 query/src/test/resources/query/sql/query03.sql  |      2 +
 query/src/test/resources/query/sql/query04.sql  |      3 +
 query/src/test/resources/query/sql/query05.sql  |      4 +
 query/src/test/resources/query/sql/query06.sql  |      5 +
 query/src/test/resources/query/sql/query07.sql  |      4 +
 query/src/test/resources/query/sql/query08.sql  |      4 +
 query/src/test/resources/query/sql/query09.sql  |      5 +
 query/src/test/resources/query/sql/query10.sql  |      8 +
 query/src/test/resources/query/sql/query11.sql  |      8 +
 query/src/test/resources/query/sql/query12.sql  |      7 +
 query/src/test/resources/query/sql/query13.sql  |      6 +
 query/src/test/resources/query/sql/query14.sql  |     17 +
 query/src/test/resources/query/sql/query15.sql  |     12 +
 query/src/test/resources/query/sql/query16.sql  |     16 +
 query/src/test/resources/query/sql/query17.sql  |     18 +
 query/src/test/resources/query/sql/query18.sql  |     14 +
 query/src/test/resources/query/sql/query19.sql  |     18 +
 query/src/test/resources/query/sql/query20.sql  |     14 +
 query/src/test/resources/query/sql/query21.sql  |     22 +
 query/src/test/resources/query/sql/query22.sql  |     24 +
 query/src/test/resources/query/sql/query23.sql  |     23 +
 query/src/test/resources/query/sql/query24.sql  |     27 +
 query/src/test/resources/query/sql/query25.sql  |     28 +
 query/src/test/resources/query/sql/query26.sql  |     30 +
 query/src/test/resources/query/sql/query27.sql  |     29 +
 query/src/test/resources/query/sql/query28.sql  |     12 +
 query/src/test/resources/query/sql/query29.sql  |     14 +
 query/src/test/resources/query/sql/query30.sql  |      1 +
 query/src/test/resources/query/sql/query31.sql  |     22 +
 query/src/test/resources/query/sql/query32.sql  |     21 +
 query/src/test/resources/query/sql/query33.sql  |     25 +
 query/src/test/resources/query/sql/query34.sql  |     23 +
 query/src/test/resources/query/sql/query35.sql  |     23 +
 query/src/test/resources/query/sql/query36.sql  |     16 +
 query/src/test/resources/query/sql/query37.sql  |     18 +
 query/src/test/resources/query/sql/query38.sql  |      4 +
 query/src/test/resources/query/sql/query39.sql  |      2 +
 query/src/test/resources/query/sql/query40.sql  |      4 +
 query/src/test/resources/query/sql/query41.sql  |     12 +
 query/src/test/resources/query/sql/query42.sql  |      5 +
 query/src/test/resources/query/sql/query43.sql  |      7 +
 query/src/test/resources/query/sql/query44.sql  |     10 +
 query/src/test/resources/query/sql/query45.sql  |      6 +
 query/src/test/resources/query/sql/query46.sql  |      1 +
 query/src/test/resources/query/sql/query47.sql  |      1 +
 query/src/test/resources/query/sql/query48.sql  |      1 +
 query/src/test/resources/query/sql/query49.sql  |     18 +
 query/src/test/resources/query/sql/query50.sql  |      7 +
 query/src/test/resources/query/sql/query51.sql  |      7 +
 query/src/test/resources/query/sql/query52.sql  |      4 +
 query/src/test/resources/query/sql/query53.sql  |      6 +
 query/src/test/resources/query/sql/query54.sql  |      7 +
 query/src/test/resources/query/sql/query55.sql  |      1 +
 query/src/test/resources/query/sql/query56.sql  |      8 +
 query/src/test/resources/query/sql/query57.sql  |      8 +
 query/src/test/resources/query/sql/query58.sql  |      8 +
 query/src/test/resources/query/sql/query59.sql  |      1 +
 query/src/test/resources/query/sql/query60.sql  |      6 +
 query/src/test/resources/query/sql/query61.sql  |      6 +
 query/src/test/resources/query/sql/query62.sql  |      3 +
 query/src/test/resources/query/sql/query63.sql  |      2 +
 query/src/test/resources/query/sql/query64.sql  |      3 +
 query/src/test/resources/query/sql/query65.sql  |      1 +
 query/src/test/resources/query/sql/query66.sql  |      2 +
 query/src/test/resources/query/sql/query67.sql  |      4 +
 query/src/test/resources/query/sql/query68.sql  |      8 +
 query/src/test/resources/query/sql/query69.sql  |      7 +
 query/src/test/resources/query/sql/query70.sql  |      6 +
 query/src/test/resources/query/sql/query71.sql  |      6 +
 query/src/test/resources/query/sql/query72.sql  |      6 +
 query/src/test/resources/query/sql/query73.sql  |      1 +
 query/src/test/resources/query/sql/query74.sql  |      1 +
 query/src/test/resources/query/sql/query75.sql  |      4 +
 query/src/test/resources/query/sql/query76.sql  |      4 +
 .../resources/query/sql_all_types/query00.sql   |      1 +
 .../resources/query/sql_all_types/query01.sql   |      1 +
 .../resources/query/sql_all_types/query02.sql   |      1 +
 .../resources/query/sql_all_types/query03.sql   |      1 +
 .../resources/query/sql_all_types/query04.sql   |      1 +
 .../resources/query/sql_all_types/query05.sql   |      1 +
 .../resources/query/sql_all_types/query06.sql   |      1 +
 .../resources/query/sql_all_types/query07.sql   |      1 +
 .../resources/query/sql_all_types/query08.sql   |      1 +
 .../resources/query/sql_all_types/query09.sql   |      1 +
 .../resources/query/sql_casewhen/query01.sql    |      8 +
 .../resources/query/sql_casewhen/query02.sql    |      4 +
 .../resources/query/sql_casewhen/query03.sql    |      7 +
 .../query/sql_casewhen/query55.sql.disable      |      4 +
 .../resources/query/sql_casewhen/query56.sql    |      4 +
 .../resources/query/sql_derived/query01.sql     |     29 +
 .../resources/query/sql_derived/query02.sql     |     29 +
 .../resources/query/sql_derived/query03.sql     |      3 +
 .../resources/query/sql_derived/query04.sql     |      2 +
 .../resources/query/sql_derived/query05.sql     |      5 +
 .../resources/query/sql_derived/query06.sql     |      2 +
 .../resources/query/sql_derived/query07.sql     |      4 +
 .../query/sql_derived/query08.sql.disabled      |      1 +
 .../resources/query/sql_derived/query09.sql     |      1 +
 .../resources/query/sql_derived/query10.sql     |     26 +
 .../resources/query/sql_distinct/query00.sql    |      6 +
 .../resources/query/sql_distinct/query01.sql    |      7 +
 .../resources/query/sql_distinct/query02.sql    |      8 +
 .../resources/query/sql_distinct/query03.sql    |      8 +
 .../resources/query/sql_distinct/query04.sql    |      9 +
 .../resources/query/sql_distinct/query05.sql    |      7 +
 .../resources/query/sql_distinct/query06.sql    |      8 +
 .../resources/query/sql_distinct/query07.sql    |      6 +
 .../resources/query/sql_dynamic/query01.dat     |      2 +
 .../resources/query/sql_dynamic/query01.sql     |     10 +
 .../resources/query/sql_dynamic/query02.dat     |      2 +
 .../resources/query/sql_dynamic/query02.sql     |     12 +
 .../test/resources/query/sql_hive/query01.sql   |      2 +
 .../test/resources/query/sql_hive/query10.sql   |      8 +
 .../query/sql_invalid/query29_invalid_SQL.sql   |     37 +
 .../query/sql_invalid/query30_invalid_SQL.sql   |     36 +
 .../query/sql_invalid/query31_invalid_SQL.sql   |     37 +
 .../query/sql_invalid/query32_invalid_SQL.sql   |     38 +
 .../query/sql_invalid/query33_invalid_SQL.sql   |     38 +
 .../query/sql_invalid/query34_invalid_SQL.sql   |      6 +
 .../query_count_distinct_on_dimension.sql       |      1 +
 .../test/resources/query/sql_lookup/query01.sql |      6 +
 .../test/resources/query/sql_lookup/query02.sql |      1 +
 .../test/resources/query/sql_lookup/query03.sql |      1 +
 .../test/resources/query/sql_lookup/query04.sql |      1 +
 .../query/sql_optimize/enable-limit01.sql       |      1 +
 .../resources/query/sql_orderby/query01.sql     |     10 +
 .../resources/query/sql_orderby/query02.sql     |      7 +
 .../resources/query/sql_orderby/query03.sql     |      6 +
 .../resources/query/sql_subquery/query00.sql    |      4 +
 .../resources/query/sql_subquery/query01.sql    |      8 +
 .../query/sql_subquery/query02.sql.disable      |      7 +
 .../resources/query/sql_subquery/query03.sql    |      9 +
 .../resources/query/sql_subquery/query04.sql    |      9 +
 .../resources/query/sql_subquery/query05.sql    |     10 +
 .../resources/query/sql_subquery/query06.sql    |      6 +
 .../resources/query/sql_subquery/query07.sql    |      6 +
 .../resources/query/sql_tableau/query00.sql     |      5 +
 .../resources/query/sql_tableau/query01.sql     |     15 +
 .../resources/query/sql_tableau/query02.sql     |     12 +
 .../resources/query/sql_tableau/query03.sql     |     17 +
 .../resources/query/sql_tableau/query04.sql     |     21 +
 .../resources/query/sql_tableau/query05.sql     |     22 +
 .../resources/query/sql_tableau/query06.sql     |     22 +
 .../resources/query/sql_tableau/query07.sql     |      8 +
 .../query/sql_tableau/query08.sql.disabled      |      6 +
 .../query/sql_tableau/query09.sql.disabled      |     39 +
 .../resources/query/sql_tableau/query10.sql     |      5 +
 .../resources/query/sql_tableau/query11.sql     |      6 +
 .../resources/query/sql_tableau/query12.sql     |      5 +
 .../resources/query/sql_tableau/query13.sql     |      4 +
 .../resources/query/sql_tableau/query14.sql     |      1 +
 .../resources/query/sql_tableau/query15.sql     |      4 +
 .../resources/query/sql_tableau/query16.sql     |      4 +
 .../resources/query/sql_tableau/query17.sql     |      1 +
 .../resources/query/sql_tableau/query18.sql     |      3 +
 .../resources/query/sql_tableau/query19.sql     |      8 +
 .../resources/query/sql_tableau/query20.sql     |      8 +
 .../resources/query/sql_tableau/query21.sql     |      9 +
 .../query/sql_tableau/query22.sql.disabled      |     10 +
 .../resources/query/sql_tableau/query23.sql     |      5 +
 .../resources/query/sql_tableau/query24.sql     |      6 +
 .../resources/query/sql_tableau/query25.sql     |     10 +
 .../resources/query/sql_tableau/query27.sql     |      6 +
 .../resources/query/sql_tableau/query67.sql     |      5 +
 .../resources/query/sql_verifyCount/query04.sql |      1 +
 .../query/sql_verifyCount/query04.sql.expected  |      1 +
 .../resources/query/sql_verifyCount/query05.sql |      1 +
 .../query/sql_verifyCount/query05.sql.expected  |      1 +
 .../resources/query/sql_verifyCount/query06.sql |      1 +
 .../query/sql_verifyCount/query06.sql.expected  |      1 +
 .../resources/query/sql_verifyCount/query07.sql |      1 +
 .../query/sql_verifyCount/query07.sql.expected  |      1 +
 .../resources/query/tableau_probing/query01.sql |      1 +
 .../resources/query/tableau_probing/query02.sql |      1 +
 .../resources/query/tableau_probing/query03.sql |      1 +
 .../resources/query/tableau_probing/query04.sql |      1 +
 .../resources/query/tableau_probing/query05.sql |      1 +
 .../resources/query/tableau_probing/query06.sql |      1 +
 .../resources/query/tableau_probing/query07.sql |      1 +
 .../.settings/org.eclipse.core.resources.prefs  |      5 +
 server/.settings/org.eclipse.jdt.core.prefs     |    379 +
 server/.settings/org.eclipse.jdt.ui.prefs       |      7 +
 server/ServiceConfig.md                         |     42 +
 server/pom.xml                                  |    390 +
 .../java/com/kylinolap/rest/DebugTomcat.java    |     65 +
 .../com/kylinolap/rest/constant/Constant.java   |     50 +
 .../rest/controller/AccessController.java       |    117 +
 .../rest/controller/AdminController.java        |     95 +
 .../rest/controller/BasicController.java        |     75 +
 .../rest/controller/CacheController.java        |    113 +
 .../rest/controller/CubeController.java         |    468 +
 .../rest/controller/CubeDescController.java     |     67 +
 .../rest/controller/JobController.java          |    226 +
 .../rest/controller/ProjectController.java      |    118 +
 .../rest/controller/QueryController.java        |    266 +
 .../rest/controller/TableController.java        |    214 +
 .../rest/controller/UserController.java         |     66 +
 .../rest/exception/BadRequestException.java     |     59 +
 .../rest/exception/ForbiddenException.java      |     39 +
 .../rest/exception/InternalErrorException.java  |     60 +
 .../rest/exception/NotFoundException.java       |     32 +
 .../rest/interceptor/CacheIntercaptor.java      |     42 +
 .../com/kylinolap/rest/metrics/JobMetrics.java  |    115 +
 .../KylinInstrumentedFilterContextListener.java |     39 +
 .../kylinolap/rest/metrics/QueryMetrics.java    |     80 +
 .../com/kylinolap/rest/model/ColumnMeta.java    |    262 +
 .../java/com/kylinolap/rest/model/Query.java    |    102 +
 .../rest/model/SelectedColumnMeta.java          |    140 +
 .../com/kylinolap/rest/model/TableMeta.java     |    148 +
 .../kylinolap/rest/request/AccessRequest.java   |     65 +
 .../rest/request/CardinalityRequest.java        |     75 +
 .../rest/request/CreateProjectRequest.java      |     29 +
 .../com/kylinolap/rest/request/CubeRequest.java |    120 +
 .../kylinolap/rest/request/JobBuildRequest.java |     55 +
 .../kylinolap/rest/request/JobListRequest.java  |     76 +
 .../com/kylinolap/rest/request/MetaRequest.java |     24 +
 .../kylinolap/rest/request/MetricsRequest.java  |     57 +
 .../rest/request/PrepareSqlRequest.java         |     62 +
 .../com/kylinolap/rest/request/SQLRequest.java  |    115 +
 .../kylinolap/rest/request/SaveSqlRequest.java  |     70 +
 .../rest/request/UpdateConfigRequest.java       |     44 +
 .../rest/request/UpdateProjectRequest.java      |     38 +
 .../rest/response/AccessEntryResponse.java      |     68 +
 .../kylinolap/rest/response/ErrorResponse.java  |     36 +
 .../rest/response/GeneralResponse.java          |     29 +
 .../kylinolap/rest/response/HBaseResponse.java  |     73 +
 .../rest/response/MetricsResponse.java          |     53 +
 .../kylinolap/rest/response/SQLResponse.java    |    141 +
 .../rest/response/TableDescResponse.java        |     77 +
 .../rest/security/AclEntityFactory.java         |     54 +
 .../kylinolap/rest/security/AclPermission.java  |     40 +
 .../rest/security/AclPermissionFactory.java     |     75 +
 .../rest/security/AuthoritiesPopulator.java     |     75 +
 .../rest/security/CrossDomainFilter.java        |     72 +
 .../kylinolap/rest/security/LdapProvider.java   |    103 +
 .../security/PasswordPlaceholderConfigurer.java |     82 +
 .../rest/security/UnauthorisedEntryPoint.java   |     26 +
 .../kylinolap/rest/security/UserManager.java    |     11 +
 .../kylinolap/rest/service/AccessService.java   |    289 +
 .../com/kylinolap/rest/service/AclService.java  |    465 +
 .../kylinolap/rest/service/AdminService.java    |    109 +
 .../kylinolap/rest/service/BasicService.java    |    193 +
 .../com/kylinolap/rest/service/CubeService.java |    598 +
 .../com/kylinolap/rest/service/JobService.java  |    184 +
 .../kylinolap/rest/service/MetricsService.java  |     60 +
 .../kylinolap/rest/service/ProjectService.java  |    126 +
 .../kylinolap/rest/service/QueryService.java    |    482 +
 .../com/kylinolap/rest/service/UserService.java |    240 +
 .../com/kylinolap/rest/util/ClasspathUtil.java  |     42 +
 .../java/com/kylinolap/rest/util/QueryUtil.java |    184 +
 .../com/kylinolap/rest/util/Serializer.java     |     32 +
 server/src/main/resources/.gitignore            |      2 +
 .../src/main/resources/applicationContext.xml   |    133 +
 server/src/main/resources/dbSqls/schema.sql     |    146 +
 server/src/main/resources/ehcache-test.xml      |     23 +
 server/src/main/resources/ehcache.xml           |     23 +
 server/src/main/resources/kylinSecurity.xml     |    147 +
 server/src/main/resources/log4j.properties      |     38 +
 .../src/main/webapp/WEB-INF/kylin-servlet.xml   |     19 +
 server/src/main/webapp/WEB-INF/web.xml          |    118 +
 server/src/main/webapp/index.html               |      9 +
 .../java/com/kylinolap/rest/bean/BeanTest.java  |     73 +
 .../com/kylinolap/rest/bean/BeanValidator.java  |    158 +
 .../rest/controller/AccessControllerTest.java   |     87 +
 .../rest/controller/AdminControllerTest.java    |     56 +
 .../rest/controller/BaseControllerTest.java     |     69 +
 .../rest/controller/ControllerTestSuite.java    |     29 +
 .../rest/controller/CubeControllerTest.java     |     87 +
 .../rest/controller/JobControllerTest.java      |    103 +
 .../rest/controller/ProjectControllerTest.java  |     79 +
 .../rest/controller/QueryControllerTest.java    |     72 +
 .../rest/controller/UserControllerTest.java     |     65 +
 .../rest/service/AccessServiceTest.java         |    138 +
 .../kylinolap/rest/service/CubeServiceTest.java |     60 +
 .../kylinolap/rest/service/JobServiceTest.java  |     47 +
 .../rest/service/QueryServiceTest.java          |     64 +
 .../kylinolap/rest/service/ServiceTestBase.java |     82 +
 .../rest/service/ServiceTestSuite.java          |     29 +
 .../kylinolap/rest/service/UserServiceTest.java |     36 +
 .../.settings/org.eclipse.core.resources.prefs  |      6 +
 storage/.settings/org.eclipse.jdt.core.prefs    |    379 +
 storage/.settings/org.eclipse.jdt.ui.prefs      |      7 +
 storage/pom.xml                                 |    117 +
 .../com/kylinolap/storage/IStorageEngine.java   |     34 +
 .../com/kylinolap/storage/StorageContext.java   |    212 +
 .../kylinolap/storage/StorageEngineFactory.java |     37 +
 .../storage/filter/BitMapFilterEvaluator.java   |    229 +
 .../storage/filter/CaseTupleFilter.java         |    109 +
 .../storage/filter/ColumnTupleFilter.java       |    113 +
 .../storage/filter/CompareTupleFilter.java      |    240 +
 .../storage/filter/ConstantTupleFilter.java     |    101 +
 .../storage/filter/DynamicTupleFilter.java      |     83 +
 .../storage/filter/ExtractTupleFilter.java      |    106 +
 .../storage/filter/LogicalTupleFilter.java      |    124 +
 .../kylinolap/storage/filter/TupleFilter.java   |    207 +
 .../storage/filter/TupleFilterSerializer.java   |    180 +
 .../storage/hbase/ColumnValueRange.java         |    170 +
 .../hbase/ConcurrentHBaseTupleIterator.java     |    472 +
 .../storage/hbase/CubeSegmentTupleIterator.java |    374 +
 .../storage/hbase/DerivedFilterTranslator.java  |    212 +
 .../storage/hbase/FuzzyValueCombination.java    |    138 +
 .../storage/hbase/HBaseClientKVIterator.java    |     92 +
 .../kylinolap/storage/hbase/HBaseKeyRange.java  |    275 +
 .../storage/hbase/HBaseStorageEngine.java       |    601 +
 .../hbase/InvertedIndexStorageEngine.java       |    146 +
 .../kylinolap/storage/hbase/PingHBaseCLI.java   |     88 +
 .../storage/hbase/RegionScannerAdapter.java     |     95 +
 .../storage/hbase/ResultScannerAdapter.java     |     98 +
 .../storage/hbase/ScanOutOfLimitException.java  |     29 +
 .../hbase/SerializedHBaseTupleIterator.java     |    127 +
 .../coprocessor/HbaseServerKVIterator.java      |     78 +
 .../storage/hbase/coprocessor/IIEndpoint.java   |     94 +
 .../hbase/coprocessor/IIResponseAdapter.java    |      7 +
 .../coprocessor/example/ExampleEndpoint.java    |    143 +
 .../example/generated/ExampleProtos.java        |   1148 +
 .../example/generated/NodeProtos.java           |    938 +
 .../hbase/coprocessor/generated/IIProtos.java   |   1237 +
 .../hbase/observer/AggregateRegionObserver.java |     96 +
 .../hbase/observer/AggregationCache.java        |    195 +
 .../hbase/observer/AggregationScanner.java      |    158 +
 .../hbase/observer/CoprocessorEnabler.java      |    173 +
 .../storage/hbase/observer/SRowAggregators.java |    261 +
 .../storage/hbase/observer/SRowFilter.java      |    198 +
 .../storage/hbase/observer/SRowProjector.java   |    176 +
 .../storage/hbase/observer/SRowTuple.java       |     95 +
 .../storage/hbase/observer/SRowType.java        |    138 +
 .../kylinolap/storage/hbase/protobuf/II.proto   |     20 +
 .../kylinolap/storage/hbase/protobuf/Node.proto |     28 +
 .../storage/hbase/protobuf/example.proto        |     37 +
 .../com/kylinolap/storage/tuple/ITuple.java     |     41 +
 .../kylinolap/storage/tuple/ITupleIterator.java |     31 +
 .../java/com/kylinolap/storage/tuple/Tuple.java |    234 +
 .../com/kylinolap/storage/tuple/TupleInfo.java  |    117 +
 .../com/kylinolap/storage/MiniClusterTest.java  |     17 +
 .../filter/BitMapFilterEvaluatorTest.java       |    185 +
 .../storage/filter/FilterBaseTest.java          |    207 +
 .../storage/filter/FilterEvaluateTest.java      |    179 +
 .../storage/filter/FilterSerializeTest.java     |    198 +
 .../hbase/FuzzyValueCombinationTest.java        |    116 +
 .../storage/hbase/InvertedIndexHBaseTest.java   |    144 +
 .../hbase/coprocessor/ExampleEndpointTest.java  |     99 +
 .../storage/hbase/coprocessor/ProtobufTest.java |     41 +
 .../observer/AggregateRegionObserverTest.java   |    343 +
 .../hbase/observer/RowAggregatorsTest.java      |     63 +
 .../hbase/observer/RowProjectorTest.java        |     88 +
 .../storage/hbase/observer/RowTypeTest.java     |     65 +
 .../com/kylinolap/storage/test/StorageTest.java |    235 +
 webapp/.bowerrc                                 |      3 +
 webapp/.editorconfig                            |     21 +
 webapp/.gitignore                               |      9 +
 webapp/.jshintrc                                |     22 +
 webapp/CHANGELOG.md                             |      1 +
 webapp/Gruntfile.js                             |    275 +
 webapp/README.md                                |      4 +
 webapp/app/.gitignore                           |      1 +
 webapp/app/components/ace-builds/.bower.json    |     14 +
 webapp/app/components/ace-builds/ChangeLog.txt  |    275 +
 webapp/app/components/ace-builds/LICENSE        |     24 +
 webapp/app/components/ace-builds/README.md      |     21 +
 .../ace-builds/demo/kitchen-sink/logo.png       |    Bin 0 -> 49263 bytes
 .../ace-builds/demo/kitchen-sink/styles.css     |     47 +
 webapp/app/components/ace-builds/editor.html    |     39 +
 .../components/ace-builds/kitchen-sink-req.html |    268 +
 .../app/components/ace-builds/kitchen-sink.html |    282 +
 .../components/ace-builds/kitchen-sink/demo.js  |   6643 +
 .../kitchen-sink/docs/AsciiDoc.asciidoc         |   6040 +
 .../ace-builds/kitchen-sink/docs/AsciiDoc.html  |   1544 +
 .../ace-builds/kitchen-sink/docs/Haxe.hx        |     17 +
 .../ace-builds/kitchen-sink/docs/Jack.jack      |    247 +
 .../ace-builds/kitchen-sink/docs/Makefile       |    122 +
 .../ace-builds/kitchen-sink/docs/Nix.nix        |     57 +
 .../ace-builds/kitchen-sink/docs/OpenSCAD.scad  |     21 +
 .../ace-builds/kitchen-sink/docs/abap.abap      |     36 +
 .../kitchen-sink/docs/actionscript.as           |     51 +
 .../ace-builds/kitchen-sink/docs/ada.ada        |      5 +
 .../kitchen-sink/docs/assembly_x86.asm          |     18 +
 .../ace-builds/kitchen-sink/docs/autohotkey.ahk |     35 +
 .../ace-builds/kitchen-sink/docs/batchfile.bat  |     15 +
 .../kitchen-sink/docs/c9search.c9search_results |     25 +
 .../ace-builds/kitchen-sink/docs/c_cpp.cpp      |     44 +
 .../ace-builds/kitchen-sink/docs/cirru.cirru    |     42 +
 .../ace-builds/kitchen-sink/docs/clojure.clj    |     19 +
 .../ace-builds/kitchen-sink/docs/cobol.CBL      |      1 +
 .../ace-builds/kitchen-sink/docs/coffee.coffee  |     22 +
 .../kitchen-sink/docs/coffeescript.coffee       |     22 +
 .../ace-builds/kitchen-sink/docs/coldfusion.cfm |      5 +
 .../ace-builds/kitchen-sink/docs/cpp.cpp        |     44 +
 .../ace-builds/kitchen-sink/docs/csharp.cs      |      4 +
 .../ace-builds/kitchen-sink/docs/css.css        |     28 +
 .../ace-builds/kitchen-sink/docs/curly.curly    |     16 +
 .../components/ace-builds/kitchen-sink/docs/d.d |     14 +
 .../ace-builds/kitchen-sink/docs/dart.dart      |     19 +
 .../ace-builds/kitchen-sink/docs/diff.diff      |     69 +
 .../ace-builds/kitchen-sink/docs/dot.dot        |    110 +
 .../ace-builds/kitchen-sink/docs/ejs.ejs        |     31 +
 .../ace-builds/kitchen-sink/docs/erlang.erl     |     20 +
 .../ace-builds/kitchen-sink/docs/forth.frt      |     41 +
 .../ace-builds/kitchen-sink/docs/freemarker.ftl |     46 +
 .../ace-builds/kitchen-sink/docs/ftl.ftl        |     46 +
 .../kitchen-sink/docs/gherkin.feature           |     28 +
 .../ace-builds/kitchen-sink/docs/glsl.glsl      |     20 +
 .../ace-builds/kitchen-sink/docs/golang.go      |     34 +
 .../ace-builds/kitchen-sink/docs/groovy.groovy  |     41 +
 .../ace-builds/kitchen-sink/docs/haml.haml      |     36 +
 .../ace-builds/kitchen-sink/docs/handlebars.hbs |      8 +
 .../ace-builds/kitchen-sink/docs/haskell.hs     |     20 +
 .../ace-builds/kitchen-sink/docs/htaccess       |     10 +
 .../ace-builds/kitchen-sink/docs/html.html      |     17 +
 .../ace-builds/kitchen-sink/docs/html_ruby.erb  |     26 +
 .../ace-builds/kitchen-sink/docs/ini.ini        |      4 +
 .../ace-builds/kitchen-sink/docs/jade.jade      |     45 +
 .../ace-builds/kitchen-sink/docs/java.java      |     15 +
 .../ace-builds/kitchen-sink/docs/javascript.js  |      5 +
 .../ace-builds/kitchen-sink/docs/json.json      |     66 +
 .../ace-builds/kitchen-sink/docs/jsoniq.jq      |      1 +
 .../ace-builds/kitchen-sink/docs/jsp.jsp        |     46 +
 .../ace-builds/kitchen-sink/docs/jsx.jsx        |      9 +
 .../ace-builds/kitchen-sink/docs/julia.jl       |     15 +
 .../ace-builds/kitchen-sink/docs/latex.tex      |     22 +
 .../ace-builds/kitchen-sink/docs/less.less      |     28 +
 .../ace-builds/kitchen-sink/docs/liquid.liquid  |     76 +
 .../ace-builds/kitchen-sink/docs/lisp.lisp      |     22 +
 .../ace-builds/kitchen-sink/docs/livescript.ls  |    245 +
 .../ace-builds/kitchen-sink/docs/logiql.logic   |     16 +
 .../ace-builds/kitchen-sink/docs/lsl.lsl        |     75 +
 .../ace-builds/kitchen-sink/docs/lua.lua        |     38 +
 .../ace-builds/kitchen-sink/docs/luapage.lp     |     71 +
 .../ace-builds/kitchen-sink/docs/lucene.lucene  |      1 +
 .../ace-builds/kitchen-sink/docs/markdown.md    |    186 +
 .../ace-builds/kitchen-sink/docs/matlab.matlab  |      1 +
 .../ace-builds/kitchen-sink/docs/mel.mel        |     33 +
 .../ace-builds/kitchen-sink/docs/mushcode.mc    |      8 +
 .../ace-builds/kitchen-sink/docs/mysql.mysql    |      1 +
 .../ace-builds/kitchen-sink/docs/objectivec.m   |    104 +
 .../ace-builds/kitchen-sink/docs/ocaml.ml       |     18 +
 .../ace-builds/kitchen-sink/docs/pascal.pas     |     48 +
 .../ace-builds/kitchen-sink/docs/perl.pl        |     37 +
 .../ace-builds/kitchen-sink/docs/pgsql.pgsql    |    118 +
 .../ace-builds/kitchen-sink/docs/php.php        |     19 +
 .../ace-builds/kitchen-sink/docs/plaintext.txt  |     11 +
 .../ace-builds/kitchen-sink/docs/powershell.ps1 |     24 +
 .../ace-builds/kitchen-sink/docs/prolog.plg     |     18 +
 .../kitchen-sink/docs/properties.properties     |     15 +
 .../ace-builds/kitchen-sink/docs/protobuf.proto |     16 +
 .../ace-builds/kitchen-sink/docs/python.py      |     19 +
 .../components/ace-builds/kitchen-sink/docs/r.r |     20 +
 .../ace-builds/kitchen-sink/docs/rdoc.Rd        |     64 +
 .../ace-builds/kitchen-sink/docs/rhtml.Rhtml    |     22 +
 .../ace-builds/kitchen-sink/docs/ruby.rb        |     35 +
 .../ace-builds/kitchen-sink/docs/rust.rs        |     20 +
 .../ace-builds/kitchen-sink/docs/sass.sass      |     39 +
 .../ace-builds/kitchen-sink/docs/scad.scad      |     21 +
 .../ace-builds/kitchen-sink/docs/scala.scala    |     69 +
 .../ace-builds/kitchen-sink/docs/scheme.scm     |     21 +
 .../ace-builds/kitchen-sink/docs/scss.scss      |     20 +
 .../ace-builds/kitchen-sink/docs/sh.sh          |     40 +
 .../ace-builds/kitchen-sink/docs/sjs.sjs        |     28 +
 .../ace-builds/kitchen-sink/docs/smarty.smarty  |      7 +
 .../kitchen-sink/docs/snippets.snippets         |     26 +
 .../kitchen-sink/docs/soy_template.soy          |     46 +
 .../ace-builds/kitchen-sink/docs/space.space    |     56 +
 .../ace-builds/kitchen-sink/docs/sql.sql        |      6 +
 .../ace-builds/kitchen-sink/docs/stylus.styl    |     55 +
 .../ace-builds/kitchen-sink/docs/svg.svg        |     83 +
 .../ace-builds/kitchen-sink/docs/tcl.tcl        |     40 +
 .../ace-builds/kitchen-sink/docs/tex.tex        |     20 +
 .../kitchen-sink/docs/textile.textile           |     29 +
 .../ace-builds/kitchen-sink/docs/tinymush.mc    |      8 +
 .../kitchen-sink/docs/tmSnippet.tmSnippet       |     26 +
 .../ace-builds/kitchen-sink/docs/toml.toml      |     29 +
 .../ace-builds/kitchen-sink/docs/twig.twig      |     30 +
 .../ace-builds/kitchen-sink/docs/typescript.ts  |     72 +
 .../ace-builds/kitchen-sink/docs/vbscript.vbs   |     23 +
 .../ace-builds/kitchen-sink/docs/velocity.vm    |     44 +
 .../ace-builds/kitchen-sink/docs/verilog.v      |     12 +
 .../ace-builds/kitchen-sink/docs/vhdl.vhd       |     34 +
 .../ace-builds/kitchen-sink/docs/xml.xml        |     55 +
 .../ace-builds/kitchen-sink/docs/xquery.xq      |      6 +
 .../ace-builds/kitchen-sink/docs/yaml.yaml      |     35 +
 .../components/ace-builds/kitchen-sink/logo.png |    Bin 0 -> 49263 bytes
 .../ace-builds/kitchen-sink/styles.css          |     49 +
 .../components/ace-builds/scrollable-page.html  |    152 +
 .../ace-builds/src-min-noconflict/ace.js        |      1 +
 .../src-min-noconflict/ext-chromevox.js         |      1 +
 .../ext-elastic_tabstops_lite.js                |      1 +
 .../ace-builds/src-min-noconflict/ext-emmet.js  |      1 +
 .../src-min-noconflict/ext-error_marker.js      |      1 +
 .../src-min-noconflict/ext-keybinding_menu.js   |      1 +
 .../src-min-noconflict/ext-language_tools.js    |      1 +
 .../src-min-noconflict/ext-modelist.js          |      1 +
 .../ace-builds/src-min-noconflict/ext-old_ie.js |      1 +
 .../src-min-noconflict/ext-searchbox.js         |      1 +
 .../src-min-noconflict/ext-settings_menu.js     |      1 +
 .../src-min-noconflict/ext-spellcheck.js        |      1 +
 .../ace-builds/src-min-noconflict/ext-split.js  |      1 +
 .../src-min-noconflict/ext-static_highlight.js  |      1 +
 .../src-min-noconflict/ext-statusbar.js         |      1 +
 .../src-min-noconflict/ext-textarea.js          |      1 +
 .../src-min-noconflict/ext-themelist.js         |      1 +
 .../src-min-noconflict/ext-whitespace.js        |      1 +
 .../src-min-noconflict/keybinding-emacs.js      |      1 +
 .../src-min-noconflict/keybinding-vim.js        |      1 +
 .../ace-builds/src-min-noconflict/mode-abap.js  |      1 +
 .../src-min-noconflict/mode-actionscript.js     |      1 +
 .../ace-builds/src-min-noconflict/mode-ada.js   |      1 +
 .../src-min-noconflict/mode-apache_conf.js      |      1 +
 .../src-min-noconflict/mode-asciidoc.js         |      1 +
 .../src-min-noconflict/mode-assembly_x86.js     |      1 +
 .../src-min-noconflict/mode-autohotkey.js       |      1 +
 .../src-min-noconflict/mode-batchfile.js        |      1 +
 .../src-min-noconflict/mode-c9search.js         |      1 +
 .../ace-builds/src-min-noconflict/mode-c_cpp.js |      1 +
 .../ace-builds/src-min-noconflict/mode-cirru.js |      1 +
 .../src-min-noconflict/mode-clojure.js          |      1 +
 .../ace-builds/src-min-noconflict/mode-cobol.js |      1 +
 .../src-min-noconflict/mode-coffee.js           |      1 +
 .../src-min-noconflict/mode-coldfusion.js       |      1 +
 .../src-min-noconflict/mode-csharp.js           |      1 +
 .../ace-builds/src-min-noconflict/mode-css.js   |      1 +
 .../ace-builds/src-min-noconflict/mode-curly.js |      1 +
 .../ace-builds/src-min-noconflict/mode-d.js     |      1 +
 .../ace-builds/src-min-noconflict/mode-dart.js  |      1 +
 .../ace-builds/src-min-noconflict/mode-diff.js  |      1 +
 .../src-min-noconflict/mode-django.js           |      1 +
 .../ace-builds/src-min-noconflict/mode-dot.js   |      1 +
 .../ace-builds/src-min-noconflict/mode-ejs.js   |      1 +
 .../src-min-noconflict/mode-erlang.js           |      1 +
 .../ace-builds/src-min-noconflict/mode-forth.js |      1 +
 .../ace-builds/src-min-noconflict/mode-ftl.js   |      1 +
 .../src-min-noconflict/mode-gherkin.js          |      1 +
 .../ace-builds/src-min-noconflict/mode-glsl.js  |      1 +
 .../src-min-noconflict/mode-golang.js           |      1 +
 .../src-min-noconflict/mode-groovy.js           |      1 +
 .../ace-builds/src-min-noconflict/mode-haml.js  |      1 +
 .../src-min-noconflict/mode-handlebars.js       |      1 +
 .../src-min-noconflict/mode-haskell.js          |      1 +
 .../ace-builds/src-min-noconflict/mode-haxe.js  |      1 +
 .../ace-builds/src-min-noconflict/mode-html.js  |      1 +
 .../src-min-noconflict/mode-html_completions.js |      1 +
 .../src-min-noconflict/mode-html_ruby.js        |      1 +
 .../ace-builds/src-min-noconflict/mode-ini.js   |      1 +
 .../ace-builds/src-min-noconflict/mode-jack.js  |      1 +
 .../ace-builds/src-min-noconflict/mode-jade.js  |      1 +
 .../ace-builds/src-min-noconflict/mode-java.js  |      1 +
 .../src-min-noconflict/mode-javascript.js       |      1 +
 .../ace-builds/src-min-noconflict/mode-json.js  |      1 +
 .../src-min-noconflict/mode-jsoniq.js           |      1 +
 .../ace-builds/src-min-noconflict/mode-jsp.js   |      1 +
 .../ace-builds/src-min-noconflict/mode-jsx.js   |      1 +
 .../ace-builds/src-min-noconflict/mode-julia.js |      1 +
 .../ace-builds/src-min-noconflict/mode-latex.js |      1 +
 .../ace-builds/src-min-noconflict/mode-less.js  |      1 +
 .../src-min-noconflict/mode-liquid.js           |      1 +
 .../ace-builds/src-min-noconflict/mode-lisp.js  |      1 +
 .../src-min-noconflict/mode-livescript.js       |      1 +
 .../src-min-noconflict/mode-logiql.js           |      1 +
 .../ace-builds/src-min-noconflict/mode-lsl.js   |      1 +
 .../ace-builds/src-min-noconflict/mode-lua.js   |      1 +
 .../src-min-noconflict/mode-luapage.js          |      1 +
 .../src-min-noconflict/mode-lucene.js           |      1 +
 .../src-min-noconflict/mode-makefile.js         |      1 +
 .../src-min-noconflict/mode-markdown.js         |      1 +
 .../src-min-noconflict/mode-matlab.js           |      1 +
 .../ace-builds/src-min-noconflict/mode-mel.js   |      1 +
 .../src-min-noconflict/mode-mushcode.js         |      1 +
 .../mode-mushcode_high_rules.js                 |      1 +
 .../ace-builds/src-min-noconflict/mode-mysql.js |      1 +
 .../ace-builds/src-min-noconflict/mode-nix.js   |      1 +
 .../src-min-noconflict/mode-objectivec.js       |      1 +
 .../ace-builds/src-min-noconflict/mode-ocaml.js |      1 +
 .../src-min-noconflict/mode-pascal.js           |      1 +
 .../ace-builds/src-min-noconflict/mode-perl.js  |      1 +
 .../ace-builds/src-min-noconflict/mode-pgsql.js |      1 +
 .../ace-builds/src-min-noconflict/mode-php.js   |      1 +
 .../src-min-noconflict/mode-plain_text.js       |      1 +
 .../src-min-noconflict/mode-powershell.js       |      1 +
 .../src-min-noconflict/mode-prolog.js           |      1 +
 .../src-min-noconflict/mode-properties.js       |      1 +
 .../src-min-noconflict/mode-protobuf.js         |      1 +
 .../src-min-noconflict/mode-python.js           |      1 +
 .../ace-builds/src-min-noconflict/mode-r.js     |      1 +
 .../ace-builds/src-min-noconflict/mode-rdoc.js  |      1 +
 .../ace-builds/src-min-noconflict/mode-rhtml.js |      1 +
 .../ace-builds/src-min-noconflict/mode-ruby.js  |      1 +
 .../ace-builds/src-min-noconflict/mode-rust.js  |      1 +
 .../ace-builds/src-min-noconflict/mode-sass.js  |      1 +
 .../ace-builds/src-min-noconflict/mode-scad.js  |      1 +
 .../ace-builds/src-min-noconflict/mode-scala.js |      1 +
 .../src-min-noconflict/mode-scheme.js           |      1 +
 .../ace-builds/src-min-noconflict/mode-scss.js  |      1 +
 .../ace-builds/src-min-noconflict/mode-sh.js    |      1 +
 .../ace-builds/src-min-noconflict/mode-sjs.js   |      1 +
 .../src-min-noconflict/mode-smarty.js           |      1 +
 .../src-min-noconflict/mode-snippets.js         |      1 +
 .../src-min-noconflict/mode-soy_template.js     |      1 +
 .../ace-builds/src-min-noconflict/mode-space.js |      1 +
 .../ace-builds/src-min-noconflict/mode-sql.js   |      1 +
 .../src-min-noconflict/mode-stylus.js           |      1 +
 .../ace-builds/src-min-noconflict/mode-svg.js   |      1 +
 .../ace-builds/src-min-noconflict/mode-tcl.js   |      1 +
 .../ace-builds/src-min-noconflict/mode-tex.js   |      1 +
 .../ace-builds/src-min-noconflict/mode-text.js  |      0
 .../src-min-noconflict/mode-textile.js          |      1 +
 .../ace-builds/src-min-noconflict/mode-toml.js  |      1 +
 .../ace-builds/src-min-noconflict/mode-twig.js  |      1 +
 .../src-min-noconflict/mode-typescript.js       |      1 +
 .../src-min-noconflict/mode-vbscript.js         |      1 +
 .../src-min-noconflict/mode-velocity.js         |      1 +
 .../src-min-noconflict/mode-verilog.js          |      1 +
 .../ace-builds/src-min-noconflict/mode-vhdl.js  |      1 +
 .../ace-builds/src-min-noconflict/mode-xml.js   |      1 +
 .../src-min-noconflict/mode-xquery.js           |      1 +
 .../ace-builds/src-min-noconflict/mode-yaml.js  |      1 +
 .../src-min-noconflict/snippets/abap.js         |      1 +
 .../src-min-noconflict/snippets/actionscript.js |      1 +
 .../src-min-noconflict/snippets/ada.js          |      1 +
 .../src-min-noconflict/snippets/apache_conf.js  |      1 +
 .../src-min-noconflict/snippets/asciidoc.js     |      1 +
 .../src-min-noconflict/snippets/assembly_x86.js |      1 +
 .../src-min-noconflict/snippets/autohotkey.js   |      1 +
 .../src-min-noconflict/snippets/batchfile.js    |      1 +
 .../src-min-noconflict/snippets/c9search.js     |      1 +
 .../src-min-noconflict/snippets/c_cpp.js        |      1 +
 .../src-min-noconflict/snippets/cirru.js        |      1 +
 .../src-min-noconflict/snippets/clojure.js      |      1 +
 .../src-min-noconflict/snippets/cobol.js        |      1 +
 .../src-min-noconflict/snippets/coffee.js       |      1 +
 .../src-min-noconflict/snippets/coldfusion.js   |      1 +
 .../src-min-noconflict/snippets/csharp.js       |      1 +
 .../src-min-noconflict/snippets/css.js          |      1 +
 .../src-min-noconflict/snippets/curly.js        |      1 +
 .../ace-builds/src-min-noconflict/snippets/d.js |      1 +
 .../src-min-noconflict/snippets/dart.js         |      1 +
 .../src-min-noconflict/snippets/diff.js         |      1 +
 .../src-min-noconflict/snippets/django.js       |      1 +
 .../src-min-noconflict/snippets/dot.js          |      1 +
 .../src-min-noconflict/snippets/ejs.js          |      1 +
 .../src-min-noconflict/snippets/erlang.js       |      1 +
 .../src-min-noconflict/snippets/forth.js        |      1 +
 .../src-min-noconflict/snippets/ftl.js          |      1 +
 .../src-min-noconflict/snippets/gherkin.js      |      1 +
 .../src-min-noconflict/snippets/glsl.js         |      1 +
 .../src-min-noconflict/snippets/golang.js       |      1 +
 .../src-min-noconflict/snippets/groovy.js       |      1 +
 .../src-min-noconflict/snippets/haml.js         |      1 +
 .../src-min-noconflict/snippets/handlebars.js   |      1 +
 .../src-min-noconflict/snippets/haskell.js      |      1 +
 .../src-min-noconflict/snippets/haxe.js         |      1 +
 .../src-min-noconflict/snippets/html.js         |      1 +
 .../snippets/html_completions.js                |      1 +
 .../src-min-noconflict/snippets/html_ruby.js    |      1 +
 .../src-min-noconflict/snippets/ini.js          |      1 +
 .../src-min-noconflict/snippets/jack.js         |      1 +
 .../src-min-noconflict/snippets/jade.js         |      1 +
 .../src-min-noconflict/snippets/java.js         |      1 +
 .../src-min-noconflict/snippets/javascript.js   |      1 +
 .../src-min-noconflict/snippets/json.js         |      1 +
 .../src-min-noconflict/snippets/jsoniq.js       |      1 +
 .../src-min-noconflict/snippets/jsp.js          |      1 +
 .../src-min-noconflict/snippets/jsx.js          |      1 +
 .../src-min-noconflict/snippets/julia.js        |      1 +
 .../src-min-noconflict/snippets/latex.js        |      1 +
 .../src-min-noconflict/snippets/less.js         |      1 +
 .../src-min-noconflict/snippets/liquid.js       |      1 +
 .../src-min-noconflict/snippets/lisp.js         |      1 +
 .../src-min-noconflict/snippets/livescript.js   |      1 +
 .../src-min-noconflict/snippets/logiql.js       |      1 +
 .../src-min-noconflict/snippets/lsl.js          |      1 +
 .../src-min-noconflict/snippets/lua.js          |      1 +
 .../src-min-noconflict/snippets/luapage.js      |      1 +
 .../src-min-noconflict/snippets/lucene.js       |      1 +
 .../src-min-noconflict/snippets/makefile.js     |      1 +
 .../src-min-noconflict/snippets/markdown.js     |      1 +
 .../src-min-noconflict/snippets/matlab.js       |      1 +
 .../src-min-noconflict/snippets/mel.js          |      1 +
 .../src-min-noconflict/snippets/mushcode.js     |      1 +
 .../snippets/mushcode_high_rules.js             |      1 +
 .../src-min-noconflict/snippets/mysql.js        |      1 +
 .../src-min-noconflict/snippets/nix.js          |      1 +
 .../src-min-noconflict/snippets/objectivec.js   |      1 +
 .../src-min-noconflict/snippets/ocaml.js        |      1 +
 .../src-min-noconflict/snippets/pascal.js       |      1 +
 .../src-min-noconflict/snippets/perl.js         |      1 +
 .../src-min-noconflict/snippets/pgsql.js        |      1 +
 .../src-min-noconflict/snippets/php.js          |      1 +
 .../src-min-noconflict/snippets/plain_text.js   |      1 +
 .../src-min-noconflict/snippets/powershell.js   |      1 +
 .../src-min-noconflict/snippets/prolog.js       |      1 +
 .../src-min-noconflict/snippets/properties.js   |      1 +
 .../src-min-noconflict/snippets/protobuf.js     |      1 +
 .../src-min-noconflict/snippets/python.js       |      1 +
 .../ace-builds/src-min-noconflict/snippets/r.js |      1 +
 .../src-min-noconflict/snippets/rdoc.js         |      1 +
 .../src-min-noconflict/snippets/rhtml.js        |      1 +
 .../src-min-noconflict/snippets/ruby.js         |      1 +
 .../src-min-noconflict/snippets/rust.js         |      1 +
 .../src-min-noconflict/snippets/sass.js         |      1 +
 .../src-min-noconflict/snippets/scad.js         |      1 +
 .../src-min-noconflict/snippets/scala.js        |      1 +
 .../src-min-noconflict/snippets/scheme.js       |      1 +
 .../src-min-noconflict/snippets/scss.js         |      1 +
 .../src-min-noconflict/snippets/sh.js           |      1 +
 .../src-min-noconflict/snippets/sjs.js          |      1 +
 .../src-min-noconflict/snippets/smarty.js       |      1 +
 .../src-min-noconflict/snippets/snippets.js     |      1 +
 .../src-min-noconflict/snippets/soy_template.js |      1 +
 .../src-min-noconflict/snippets/space.js        |      1 +
 .../src-min-noconflict/snippets/sql.js          |      1 +
 .../src-min-noconflict/snippets/stylus.js       |      1 +
 .../src-min-noconflict/snippets/svg.js          |      1 +
 .../src-min-noconflict/snippets/tcl.js          |      1 +
 .../src-min-noconflict/snippets/tex.js          |      1 +
 .../src-min-noconflict/snippets/text.js         |      1 +
 .../src-min-noconflict/snippets/textile.js      |      1 +
 .../src-min-noconflict/snippets/toml.js         |      1 +
 .../src-min-noconflict/snippets/twig.js         |      1 +
 .../src-min-noconflict/snippets/typescript.js   |      1 +
 .../src-min-noconflict/snippets/vbscript.js     |      1 +
 .../src-min-noconflict/snippets/velocity.js     |      1 +
 .../src-min-noconflict/snippets/verilog.js      |      1 +
 .../src-min-noconflict/snippets/vhdl.js         |      1 +
 .../src-min-noconflict/snippets/xml.js          |      1 +
 .../src-min-noconflict/snippets/xquery.js       |      1 +
 .../src-min-noconflict/snippets/yaml.js         |      1 +
 .../src-min-noconflict/theme-ambiance.js        |      1 +
 .../src-min-noconflict/theme-chaos.js           |      1 +
 .../src-min-noconflict/theme-chrome.js          |      1 +
 .../src-min-noconflict/theme-clouds.js          |      1 +
 .../src-min-noconflict/theme-clouds_midnight.js |      1 +
 .../src-min-noconflict/theme-cobalt.js          |      1 +
 .../src-min-noconflict/theme-crimson_editor.js  |      1 +
 .../ace-builds/src-min-noconflict/theme-dawn.js |      1 +
 .../src-min-noconflict/theme-dreamweaver.js     |      1 +
 .../src-min-noconflict/theme-eclipse.js         |      1 +
 .../src-min-noconflict/theme-github.js          |      1 +
 .../src-min-noconflict/theme-idle_fingers.js    |      1 +
 .../src-min-noconflict/theme-katzenmilch.js     |      1 +
 .../ace-builds/src-min-noconflict/theme-kr.js   |      1 +
 .../src-min-noconflict/theme-kuroir.js          |      1 +
 .../src-min-noconflict/theme-merbivore.js       |      1 +
 .../src-min-noconflict/theme-merbivore_soft.js  |      1 +
 .../src-min-noconflict/theme-mono_industrial.js |      1 +
 .../src-min-noconflict/theme-monokai.js         |      1 +
 .../src-min-noconflict/theme-pastel_on_dark.js  |      1 +
 .../src-min-noconflict/theme-solarized_dark.js  |      1 +
 .../src-min-noconflict/theme-solarized_light.js |      1 +
 .../src-min-noconflict/theme-terminal.js        |      1 +
 .../src-min-noconflict/theme-textmate.js        |      1 +
 .../src-min-noconflict/theme-tomorrow.js        |      1 +
 .../src-min-noconflict/theme-tomorrow_night.js  |      1 +
 .../theme-tomorrow_night_blue.js                |      1 +
 .../theme-tomorrow_night_bright.js              |      1 +
 .../theme-tomorrow_night_eighties.js            |      1 +
 .../src-min-noconflict/theme-twilight.js        |      1 +
 .../src-min-noconflict/theme-vibrant_ink.js     |      1 +
 .../src-min-noconflict/theme-xcode.js           |      1 +
 .../src-min-noconflict/worker-coffee.js         |      1 +
 .../ace-builds/src-min-noconflict/worker-css.js |      1 +
 .../src-min-noconflict/worker-html.js           |      1 +
 .../src-min-noconflict/worker-javascript.js     |      1 +
 .../src-min-noconflict/worker-json.js           |      1 +
 .../ace-builds/src-min-noconflict/worker-lua.js |      1 +
 .../ace-builds/src-min-noconflict/worker-php.js |      1 +
 .../src-min-noconflict/worker-xquery.js         |      1 +
 webapp/app/components/ace-builds/src-min/ace.js |      1 +
 .../ace-builds/src-min/ext-chromevox.js         |      1 +
 .../src-min/ext-elastic_tabstops_lite.js        |      1 +
 .../components/ace-builds/src-min/ext-emmet.js  |      1 +
 .../ace-builds/src-min/ext-error_marker.js      |      1 +
 .../ace-builds/src-min/ext-keybinding_menu.js   |      1 +
 .../ace-builds/src-min/ext-language_tools.js    |      1 +
 .../ace-builds/src-min/ext-modelist.js          |      1 +
 .../components/ace-builds/src-min/ext-old_ie.js |      1 +
 .../ace-builds/src-min/ext-options.js           |      1 +
 .../ace-builds/src-min/ext-searchbox.js         |      1 +
 .../ace-builds/src-min/ext-settings_menu.js     |      1 +
 .../ace-builds/src-min/ext-spellcheck.js        |      1 +
 .../components/ace-builds/src-min/ext-split.js  |      1 +
 .../ace-builds/src-min/ext-static_highlight.js  |      1 +
 .../ace-builds/src-min/ext-statusbar.js         |      1 +
 .../ace-builds/src-min/ext-textarea.js          |      1 +
 .../ace-builds/src-min/ext-themelist.js         |      1 +
 .../ace-builds/src-min/ext-whitespace.js        |      1 +
 .../ace-builds/src-min/keybinding-emacs.js      |      1 +
 .../ace-builds/src-min/keybinding-vim.js        |      1 +
 .../components/ace-builds/src-min/mode-abap.js  |      1 +
 .../ace-builds/src-min/mode-actionscript.js     |      1 +
 .../components/ace-builds/src-min/mode-ada.js   |      1 +
 .../ace-builds/src-min/mode-apache_conf.js      |      1 +
 .../ace-builds/src-min/mode-asciidoc.js         |      1 +
 .../ace-builds/src-min/mode-assembly_x86.js     |      1 +
 .../ace-builds/src-min/mode-autohotkey.js       |      1 +
 .../ace-builds/src-min/mode-batchfile.js        |      1 +
 .../ace-builds/src-min/mode-c9search.js         |      1 +
 .../components/ace-builds/src-min/mode-c_cpp.js |      1 +
 .../components/ace-builds/src-min/mode-cirru.js |      1 +
 .../ace-builds/src-min/mode-clojure.js          |      1 +
 .../components/ace-builds/src-min/mode-cobol.js |      1 +
 .../ace-builds/src-min/mode-coffee.js           |      1 +
 .../ace-builds/src-min/mode-coldfusion.js       |      1 +
 .../ace-builds/src-min/mode-csharp.js           |      1 +
 .../components/ace-builds/src-min/mode-css.js   |      1 +
 .../components/ace-builds/src-min/mode-curly.js |      1 +
 .../app/components/ace-builds/src-min/mode-d.js |      1 +
 .../components/ace-builds/src-min/mode-dart.js  |      1 +
 .../components/ace-builds/src-min/mode-diff.js  |      1 +
 .../ace-builds/src-min/mode-django.js           |      1 +
 .../components/ace-builds/src-min/mode-dot.js   |      1 +
 .../components/ace-builds/src-min/mode-ejs.js   |      1 +
 .../ace-builds/src-min/mode-erlang.js           |      1 +
 .../components/ace-builds/src-min/mode-forth.js |      1 +
 .../components/ace-builds/src-min/mode-ftl.js   |      1 +
 .../ace-builds/src-min/mode-gherkin.js          |      1 +
 .../components/ace-builds/src-min/mode-glsl.js  |      1 +
 .../ace-builds/src-min/mode-golang.js           |      1 +
 .../ace-builds/src-min/mode-groovy.js           |      1 +
 .../components/ace-builds/src-min/mode-haml.js  |      1 +
 .../ace-builds/src-min/mode-handlebars.js       |      1 +
 .../ace-builds/src-min/mode-haskell.js          |      1 +
 .../components/ace-builds/src-min/mode-haxe.js  |      1 +
 .../components/ace-builds/src-min/mode-html.js  |      1 +
 .../ace-builds/src-min/mode-html_completions.js |      1 +
 .../ace-builds/src-min/mode-html_ruby.js        |      1 +
 .../components/ace-builds/src-min/mode-ini.js   |      1 +
 .../components/ace-builds/src-min/mode-jack.js  |      1 +
 .../components/ace-builds/src-min/mode-jade.js  |      1 +
 .../components/ace-builds/src-min/mode-java.js  |      1 +
 .../ace-builds/src-min/mode-javascript.js       |      1 +
 .../components/ace-builds/src-min/mode-json.js  |      1 +
 .../ace-builds/src-min/mode-jsoniq.js           |      1 +
 .../components/ace-builds/src-min/mode-jsp.js   |      1 +
 .../components/ace-builds/src-min/mode-jsx.js   |      1 +
 .../components/ace-builds/src-min/mode-julia.js |      1 +
 .../components/ace-builds/src-min/mode-latex.js |      1 +
 .../components/ace-builds/src-min/mode-less.js  |      1 +
 .../ace-builds/src-min/mode-liquid.js           |      1 +
 .../components/ace-builds/src-min/mode-lisp.js  |      1 +
 .../ace-builds/src-min/mode-livescript.js       |      1 +
 .../ace-builds/src-min/mode-logiql.js           |      1 +
 .../components/ace-builds/src-min/mode-lsl.js   |      1 +
 .../components/ace-builds/src-min/mode-lua.js   |      1 +
 .../ace-builds/src-min/mode-luapage.js          |      1 +
 .../ace-builds/src-min/mode-lucene.js           |      1 +
 .../ace-builds/src-min/mode-makefile.js         |      1 +
 .../ace-builds/src-min/mode-markdown.js         |      1 +
 .../ace-builds/src-min/mode-matlab.js           |      1 +
 .../components/ace-builds/src-min/mode-mel.js   |      1 +
 .../ace-builds/src-min/mode-mushcode.js         |      1 +
 .../src-min/mode-mushcode_high_rules.js         |      1 +
 .../components/ace-builds/src-min/mode-mysql.js |      1 +
 .../components/ace-builds/src-min/mode-nix.js   |      1 +
 .../ace-builds/src-min/mode-objectivec.js       |      1 +
 .../components/ace-builds/src-min/mode-ocaml.js |      1 +
 .../ace-builds/src-min/mode-pascal.js           |      1 +
 .../components/ace-builds/src-min/mode-perl.js  |      1 +
 .../components/ace-builds/src-min/mode-pgsql.js |      1 +
 .../components/ace-builds/src-min/mode-php.js   |      1 +
 .../ace-builds/src-min/mode-plain_text.js       |      1 +
 .../ace-builds/src-min/mode-powershell.js       |      1 +
 .../ace-builds/src-min/mode-prolog.js           |      1 +
 .../ace-builds/src-min/mode-properties.js       |      1 +
 .../ace-builds/src-min/mode-protobuf.js         |      1 +
 .../ace-builds/src-min/mode-python.js           |      1 +
 .../app/components/ace-builds/src-min/mode-r.js |      1 +
 .../components/ace-builds/src-min/mode-rdoc.js  |      1 +
 .../components/ace-builds/src-min/mode-rhtml.js |      1 +
 .../components/ace-builds/src-min/mode-ruby.js  |      1 +
 .../components/ace-builds/src-min/mode-rust.js  |      1 +
 .../components/ace-builds/src-min/mode-sass.js  |      1 +
 .../components/ace-builds/src-min/mode-scad.js  |      1 +
 .../components/ace-builds/src-min/mode-scala.js |      1 +
 .../ace-builds/src-min/mode-scheme.js           |      1 +
 .../components/ace-builds/src-min/mode-scss.js  |      1 +
 .../components/ace-builds/src-min/mode-sh.js    |      1 +
 .../components/ace-builds/src-min/mode-sjs.js   |      1 +
 .../ace-builds/src-min/mode-smarty.js           |      1 +
 .../ace-builds/src-min/mode-snippets.js         |      1 +
 .../ace-builds/src-min/mode-soy_template.js     |      1 +
 .../components/ace-builds/src-min/mode-space.js |      1 +
 .../components/ace-builds/src-min/mode-sql.js   |      1 +
 .../ace-builds/src-min/mode-stylus.js           |      1 +
 .../components/ace-builds/src-min/mode-svg.js   |      1 +
 .../components/ace-builds/src-min/mode-tcl.js   |      1 +
 .../components/ace-builds/src-min/mode-tex.js   |      1 +
 .../components/ace-builds/src-min/mode-text.js  |      0
 .../ace-builds/src-min/mode-textile.js          |      1 +
 .../ace-builds/src-min/mode-tmsnippet.js        |      1 +
 .../components/ace-builds/src-min/mode-toml.js  |      1 +
 .../components/ace-builds/src-min/mode-twig.js  |      1 +
 .../ace-builds/src-min/mode-typescript.js       |      1 +
 .../ace-builds/src-min/mode-vbscript.js         |      1 +
 .../ace-builds/src-min/mode-velocity.js         |      1 +
 .../ace-builds/src-min/mode-verilog.js          |      1 +
 .../components/ace-builds/src-min/mode-vhdl.js  |      1 +
 .../components/ace-builds/src-min/mode-xml.js   |      1 +
 .../ace-builds/src-min/mode-xquery.js           |      1 +
 .../components/ace-builds/src-min/mode-yaml.js  |      1 +
 .../ace-builds/src-min/snippets/abap.js         |      1 +
 .../ace-builds/src-min/snippets/actionscript.js |      1 +
 .../ace-builds/src-min/snippets/ada.js          |      1 +
 .../ace-builds/src-min/snippets/apache_conf.js  |      1 +
 .../ace-builds/src-min/snippets/asciidoc.js     |      1 +
 .../ace-builds/src-min/snippets/assembly_x86.js |      1 +
 .../ace-builds/src-min/snippets/autohotkey.js   |      1 +
 .../ace-builds/src-min/snippets/batchfile.js    |      1 +
 .../ace-builds/src-min/snippets/c9search.js     |      1 +
 .../ace-builds/src-min/snippets/c_cpp.js        |      1 +
 .../ace-builds/src-min/snippets/cirru.js        |      1 +
 .../ace-builds/src-min/snippets/clojure.js      |      1 +
 .../ace-builds/src-min/snippets/cobol.js        |      1 +
 .../ace-builds/src-min/snippets/coffee.js       |      1 +
 .../ace-builds/src-min/snippets/coldfusion.js   |      1 +
 .../ace-builds/src-min/snippets/csharp.js       |      1 +
 .../ace-builds/src-min/snippets/css.js          |      1 +
 .../ace-builds/src-min/snippets/curly.js        |      1 +
 .../components/ace-builds/src-min/snippets/d.js |      1 +
 .../ace-builds/src-min/snippets/dart.js         |      1 +
 .../ace-builds/src-min/snippets/diff.js         |      1 +
 .../ace-builds/src-min/snippets/django.js       |      1 +
 .../ace-builds/src-min/snippets/dot.js          |      1 +
 .../ace-builds/src-min/snippets/ejs.js          |      1 +
 .../ace-builds/src-min/snippets/erlang.js       |      1 +
 .../ace-builds/src-min/snippets/forth.js        |      1 +
 .../ace-builds/src-min/snippets/ftl.js          |      1 +
 .../ace-builds/src-min/snippets/gherkin.js      |      1 +
 .../ace-builds/src-min/snippets/glsl.js         |      1 +
 .../ace-builds/src-min/snippets/golang.js       |      1 +
 .../ace-builds/src-min/snippets/groovy.js       |      1 +
 .../ace-builds/src-min/snippets/haml.js         |      1 +
 .../ace-builds/src-min/snippets/handlebars.js   |      1 +
 .../ace-builds/src-min/snippets/haskell.js      |      1 +
 .../ace-builds/src-min/snippets/haxe.js         |      1 +
 .../ace-builds/src-min/snippets/html.js         |      1 +
 .../src-min/snippets/html_completions.js        |      1 +
 .../ace-builds/src-min/snippets/html_ruby.js    |      1 +
 .../ace-builds/src-min/snippets/ini.js          |      1 +
 .../ace-builds/src-min/snippets/jack.js         |      1 +
 .../ace-builds/src-min/snippets/jade.js         |      1 +
 .../ace-builds/src-min/snippets/java.js         |      1 +
 .../ace-builds/src-min/snippets/javascript.js   |      1 +
 .../ace-builds/src-min/snippets/json.js         |      1 +
 .../ace-builds/src-min/snippets/jsoniq.js       |      1 +
 .../ace-builds/src-min/snippets/jsp.js          |      1 +
 .../ace-builds/src-min/snippets/jsx.js          |      1 +
 .../ace-builds/src-min/snippets/julia.js        |      1 +
 .../ace-builds/src-min/snippets/latex.js        |      1 +
 .../ace-builds/src-min/snippets/less.js         |      1 +
 .../ace-builds/src-min/snippets/liquid.js       |      1 +
 .../ace-builds/src-min/snippets/lisp.js         |      1 +
 .../ace-builds/src-min/snippets/livescript.js   |      1 +
 .../ace-builds/src-min/snippets/logiql.js       |      1 +
 .../ace-builds/src-min/snippets/lsl.js          |      1 +
 .../ace-builds/src-min/snippets/lua.js          |      1 +
 .../ace-builds/src-min/snippets/luapage.js      |      1 +
 .../ace-builds/src-min/snippets/lucene.js       |      1 +
 .../ace-builds/src-min/snippets/makefile.js     |      1 +
 .../ace-builds/src-min/snippets/markdown.js     |      1 +
 .../ace-builds/src-min/snippets/matlab.js       |      1 +
 .../ace-builds/src-min/snippets/mel.js          |      1 +
 .../ace-builds/src-min/snippets/mushcode.js     |      1 +
 .../src-min/snippets/mushcode_high_rules.js     |      1 +
 .../ace-builds/src-min/snippets/mysql.js        |      1 +
 .../ace-builds/src-min/snippets/nix.js          |      1 +
 .../ace-builds/src-min/snippets/objectivec.js   |      1 +
 .../ace-builds/src-min/snippets/ocaml.js        |      1 +
 .../ace-builds/src-min/snippets/pascal.js       |      1 +
 .../ace-builds/src-min/snippets/perl.js         |      1 +
 .../ace-builds/src-min/snippets/pgsql.js        |      1 +
 .../ace-builds/src-min/snippets/php.js          |      1 +
 .../ace-builds/src-min/snippets/plain_text.js   |      1 +
 .../ace-builds/src-min/snippets/powershell.js   |      1 +
 .../ace-builds/src-min/snippets/prolog.js       |      1 +
 .../ace-builds/src-min/snippets/properties.js   |      1 +
 .../ace-builds/src-min/snippets/protobuf.js     |      1 +
 .../ace-builds/src-min/snippets/python.js       |      1 +
 .../components/ace-builds/src-min/snippets/r.js |      1 +
 .../ace-builds/src-min/snippets/rdoc.js         |      1 +
 .../ace-builds/src-min/snippets/rhtml.js        |      1 +
 .../ace-builds/src-min/snippets/ruby.js         |      1 +
 .../ace-builds/src-min/snippets/rust.js         |      1 +
 .../ace-builds/src-min/snippets/sass.js         |      1 +
 .../ace-builds/src-min/snippets/scad.js         |      1 +
 .../ace-builds/src-min/snippets/scala.js        |      1 +
 .../ace-builds/src-min/snippets/scheme.js       |      1 +
 .../ace-builds/src-min/snippets/scss.js         |      1 +
 .../ace-builds/src-min/snippets/sh.js           |      1 +
 .../ace-builds/src-min/snippets/sjs.js          |      1 +
 .../ace-builds/src-min/snippets/smarty.js       |      1 +
 .../ace-builds/src-min/snippets/snippets.js     |      1 +
 .../ace-builds/src-min/snippets/soy_template.js |      1 +
 .../ace-builds/src-min/snippets/space.js        |      1 +
 .../ace-builds/src-min/snippets/sql.js          |      1 +
 .../ace-builds/src-min/snippets/stylus.js       |      1 +
 .../ace-builds/src-min/snippets/svg.js          |      1 +
 .../ace-builds/src-min/snippets/tcl.js          |      1 +
 .../ace-builds/src-min/snippets/tex.js          |      1 +
 .../ace-builds/src-min/snippets/text.js         |      1 +
 .../ace-builds/src-min/snippets/textile.js      |      1 +
 .../ace-builds/src-min/snippets/toml.js         |      1 +
 .../ace-builds/src-min/snippets/twig.js         |      1 +
 .../ace-builds/src-min/snippets/typescript.js   |      1 +
 .../ace-builds/src-min/snippets/vbscript.js     |      1 +
 .../ace-builds/src-min/snippets/velocity.js     |      1 +
 .../ace-builds/src-min/snippets/verilog.js      |      1 +
 .../ace-builds/src-min/snippets/vhdl.js         |      1 +
 .../ace-builds/src-min/snippets/xml.js          |      1 +
 .../ace-builds/src-min/snippets/xquery.js       |      1 +
 .../ace-builds/src-min/snippets/yaml.js         |      1 +
 .../ace-builds/src-min/theme-ambiance.js        |      1 +
 .../ace-builds/src-min/theme-chaos.js           |      1 +
 .../ace-builds/src-min/theme-chrome.js          |      1 +
 .../ace-builds/src-min/theme-clouds.js          |      1 +
 .../ace-builds/src-min/theme-clouds_midnight.js |      1 +
 .../ace-builds/src-min/theme-cobalt.js          |      1 +
 .../ace-builds/src-min/theme-crimson_editor.js  |      1 +
 .../components/ace-builds/src-min/theme-dawn.js |      1 +
 .../ace-builds/src-min/theme-dreamweaver.js     |      1 +
 .../ace-builds/src-min/theme-eclipse.js         |      1 +
 .../ace-builds/src-min/theme-github.js          |      1 +
 .../ace-builds/src-min/theme-idle_fingers.js    |      1 +
 .../ace-builds/src-min/theme-katzenmilch.js     |      1 +
 .../components/ace-builds/src-min/theme-kr.js   |      1 +
 .../ace-builds/src-min/theme-kuroir.js          |      1 +
 .../ace-builds/src-min/theme-merbivore.js       |      1 +
 .../ace-builds/src-min/theme-merbivore_soft.js  |      1 +
 .../ace-builds/src-min/theme-mono_industrial.js |      1 +
 .../ace-builds/src-min/theme-monokai.js         |      1 +
 .../ace-builds/src-min/theme-pastel_on_dark.js  |      1 +
 .../ace-builds/src-min/theme-solarized_dark.js  |      1 +
 .../ace-builds/src-min/theme-solarized_light.js |      1 +
 .../ace-builds/src-min/theme-terminal.js        |      1 +
 .../ace-builds/src-min/theme-textmate.js        |      1 +
 .../ace-builds/src-min/theme-tomorrow.js        |      1 +
 .../ace-builds/src-min/theme-tomorrow_night.js  |      1 +
 .../src-min/theme-tomorrow_night_blue.js        |      1 +
 .../src-min/theme-tomorrow_night_bright.js      |      1 +
 .../src-min/theme-tomorrow_night_eighties.js    |      1 +
 .../ace-builds/src-min/theme-twilight.js        |      1 +
 .../ace-builds/src-min/theme-vibrant_ink.js     |      1 +
 .../ace-builds/src-min/theme-xcode.js           |      1 +
 .../ace-builds/src-min/worker-coffee.js         |      1 +
 .../components/ace-builds/src-min/worker-css.js |      1 +
 .../ace-builds/src-min/worker-html.js           |      1 +
 .../ace-builds/src-min/worker-javascript.js     |      1 +
 .../ace-builds/src-min/worker-json.js           |      1 +
 .../components/ace-builds/src-min/worker-lua.js |      1 +
 .../components/ace-builds/src-min/worker-php.js |      1 +
 .../ace-builds/src-min/worker-xquery.js         |      1 +
 .../components/ace-builds/src-noconflict/ace.js |  17601 +
 .../ace-builds/src-noconflict/ext-chromevox.js  |    542 +
 .../src-noconflict/ext-elastic_tabstops_lite.js |    305 +
 .../ace-builds/src-noconflict/ext-emmet.js      |   1114 +
 .../src-noconflict/ext-error_marker.js          |      5 +
 .../src-noconflict/ext-keybinding_menu.js       |    211 +
 .../src-noconflict/ext-language_tools.js        |   1669 +
 .../ace-builds/src-noconflict/ext-modelist.js   |    176 +
 .../ace-builds/src-noconflict/ext-old_ie.js     |    505 +
 .../ace-builds/src-noconflict/ext-searchbox.js  |    426 +
 .../src-noconflict/ext-settings_menu.js         |    639 +
 .../ace-builds/src-noconflict/ext-spellcheck.js |     73 +
 .../ace-builds/src-noconflict/ext-split.js      |    276 +
 .../src-noconflict/ext-static_highlight.js      |    183 +
 .../ace-builds/src-noconflict/ext-statusbar.js  |     51 +
 .../ace-builds/src-noconflict/ext-textarea.js   |    483 +
 .../ace-builds/src-noconflict/ext-themelist.js  |     91 +
 .../ace-builds/src-noconflict/ext-whitespace.js |    214 +
 .../src-noconflict/keybinding-emacs.js          |   1058 +
 .../ace-builds/src-noconflict/keybinding-vim.js |   1762 +
 .../ace-builds/src-noconflict/mode-abap.js      |    261 +
 .../src-noconflict/mode-actionscript.js         |    257 +
 .../ace-builds/src-noconflict/mode-ada.js       |    118 +
 .../src-noconflict/mode-apache_conf.js          |    346 +
 .../ace-builds/src-noconflict/mode-asciidoc.js  |    373 +
 .../src-noconflict/mode-assembly_x86.js         |    217 +
 .../src-noconflict/mode-autohotkey.js           |    225 +
 .../ace-builds/src-noconflict/mode-batchfile.js |    213 +
 .../ace-builds/src-noconflict/mode-c9search.js  |    308 +
 .../ace-builds/src-noconflict/mode-c_cpp.js     |    816 +
 .../ace-builds/src-noconflict/mode-cirru.js     |    233 +
 .../ace-builds/src-noconflict/mode-clojure.js   |    300 +
 .../ace-builds/src-noconflict/mode-cobol.js     |    125 +
 .../ace-builds/src-noconflict/mode-coffee.js    |    444 +
 .../src-noconflict/mode-coldfusion.js           |   2052 +
 .../ace-builds/src-noconflict/mode-csharp.js    |    801 +
 .../ace-builds/src-noconflict/mode-css.js       |    815 +
 .../ace-builds/src-noconflict/mode-curly.js     |   2475 +
 .../ace-builds/src-noconflict/mode-d.js         |    492 +
 .../ace-builds/src-noconflict/mode-dart.js      |   1023 +
 .../ace-builds/src-noconflict/mode-diff.js      |    170 +
 .../ace-builds/src-noconflict/mode-django.js    |   2504 +
 .../ace-builds/src-noconflict/mode-dot.js       |    361 +
 .../ace-builds/src-noconflict/mode-ejs.js       |   2869 +
 .../ace-builds/src-noconflict/mode-erlang.js    |    987 +
 .../ace-builds/src-noconflict/mode-forth.js     |    280 +
 .../ace-builds/src-noconflict/mode-ftl.js       |   1061 +
 .../ace-builds/src-noconflict/mode-gherkin.js   |    160 +
 .../ace-builds/src-noconflict/mode-glsl.js      |    894 +
 .../ace-builds/src-noconflict/mode-golang.js    |    704 +
 .../ace-builds/src-noconflict/mode-groovy.js    |   1125 +
 .../ace-builds/src-noconflict/mode-haml.js      |    498 +
 .../src-noconflict/mode-handlebars.js           |   2485 +
 .../ace-builds/src-noconflict/mode-haskell.js   |    362 +
 .../ace-builds/src-noconflict/mode-haxe.js      |    687 +
 .../ace-builds/src-noconflict/mode-html.js      |   2413 +
 .../src-noconflict/mode-html_completions.js     |    309 +
 .../ace-builds/src-noconflict/mode-html_ruby.js |   2874 +
 .../ace-builds/src-noconflict/mode-ini.js       |    185 +
 .../ace-builds/src-noconflict/mode-jack.js      |    683 +
 .../ace-builds/src-noconflict/mode-jade.js      |   2081 +
 .../ace-builds/src-noconflict/mode-java.js      |   1083 +
 .../src-noconflict/mode-javascript.js           |    963 +
 .../ace-builds/src-noconflict/mode-json.js      |    655 +
 .../ace-builds/src-noconflict/mode-jsoniq.js    |   2791 +
 .../ace-builds/src-noconflict/mode-jsp.js       |   1583 +
 .../ace-builds/src-noconflict/mode-jsx.js       |    712 +
 .../ace-builds/src-noconflict/mode-julia.js     |    286 +
 .../ace-builds/src-noconflict/mode-latex.js     |    190 +
 .../ace-builds/src-noconflict/mode-less.js      |    884 +
 .../ace-builds/src-noconflict/mode-liquid.js    |   1063 +
 .../ace-builds/src-noconflict/mode-lisp.js      |    137 +
 .../src-noconflict/mode-livescript.js           |    289 +
 .../ace-builds/src-noconflict/mode-logiql.js    |    699 +
 .../ace-builds/src-noconflict/mode-lsl.js       |    920 +
 .../ace-builds/src-noconflict/mode-lua.js       |    457 +
 .../ace-builds/src-noconflict/mode-luapage.js   |   2884 +
 .../ace-builds/src-noconflict/mode-lucene.js    |     68 +
 .../ace-builds/src-noconflict/mode-makefile.js  |    332 +
 .../ace-builds/src-noconflict/mode-markdown.js  |   2776 +
 .../ace-builds/src-noconflict/mode-matlab.js    |    230 +
 .../ace-builds/src-noconflict/mode-mel.js       |    600 +
 .../ace-builds/src-noconflict/mode-mushcode.js  |    705 +
 .../src-noconflict/mode-mushcode_high_rules.js  |    569 +
 .../ace-builds/src-noconflict/mode-mysql.js     |    185 +
 .../ace-builds/src-noconflict/mode-nix.js       |    965 +
 .../src-noconflict/mode-objectivec.js           |    699 +
 .../ace-builds/src-noconflict/mode-ocaml.js     |    445 +
 .../ace-builds/src-noconflict/mode-pascal.js    |    233 +
 .../ace-builds/src-noconflict/mode-perl.js      |    359 +
 .../ace-builds/src-noconflict/mode-pgsql.js     |   1356 +
 .../ace-builds/src-noconflict/mode-php.js       |   2526 +
 .../src-noconflict/mode-plain_text.js           |     56 +
 .../src-noconflict/mode-powershell.js           |    695 +
 .../ace-builds/src-noconflict/mode-prolog.js    |    354 +
 .../src-noconflict/mode-properties.js           |    104 +
 .../ace-builds/src-noconflict/mode-protobuf.js  |    915 +
 .../ace-builds/src-noconflict/mode-python.js    |    295 +
 .../ace-builds/src-noconflict/mode-r.js         |    337 +
 .../ace-builds/src-noconflict/mode-rdoc.js      |    210 +
 .../ace-builds/src-noconflict/mode-rhtml.js     |   2715 +
 .../ace-builds/src-noconflict/mode-ruby.js      |    444 +
 .../ace-builds/src-noconflict/mode-rust.js      |    245 +
 .../ace-builds/src-noconflict/mode-sass.js      |    443 +
 .../ace-builds/src-noconflict/mode-scad.js      |    747 +
 .../ace-builds/src-noconflict/mode-scala.js     |   1113 +
 .../ace-builds/src-noconflict/mode-scheme.js    |    143 +
 .../ace-builds/src-noconflict/mode-scss.js      |    909 +
 .../ace-builds/src-noconflict/mode-sh.js        |    320 +
 .../ace-builds/src-noconflict/mode-sjs.js       |   1184 +
 .../ace-builds/src-noconflict/mode-smarty.js    |   2546 +
 .../ace-builds/src-noconflict/mode-snippets.js  |    198 +
 .../src-noconflict/mode-soy_template.js         |   2759 +
 .../ace-builds/src-noconflict/mode-space.js     |    160 +
 .../ace-builds/src-noconflict/mode-sql.js       |    124 +
 .../ace-builds/src-noconflict/mode-stylus.js    |    447 +
 .../ace-builds/src-noconflict/mode-svg.js       |   1658 +
 .../ace-builds/src-noconflict/mode-tcl.js       |    361 +
 .../ace-builds/src-noconflict/mode-tex.js       |    187 +
 .../ace-builds/src-noconflict/mode-text.js      |      0
 .../ace-builds/src-noconflict/mode-textile.js   |    171 +
 .../ace-builds/src-noconflict/mode-toml.js      |    177 +
 .../ace-builds/src-noconflict/mode-twig.js      |   2258 +
 .../src-noconflict/mode-typescript.js           |   1048 +
 .../ace-builds/src-noconflict/mode-vbscript.js  |    250 +
 .../ace-builds/src-noconflict/mode-velocity.js  |   1652 +
 .../ace-builds/src-noconflict/mode-verilog.js   |    127 +
 .../ace-builds/src-noconflict/mode-vhdl.js      |    139 +
 .../ace-builds/src-noconflict/mode-xml.js       |    967 +
 .../ace-builds/src-noconflict/mode-xquery.js    |   2825 +
 .../ace-builds/src-noconflict/mode-yaml.js      |    287 +
 .../ace-builds/src-noconflict/snippets/abap.js  |      7 +
 .../src-noconflict/snippets/actionscript.js     |    164 +
 .../ace-builds/src-noconflict/snippets/ada.js   |      7 +
 .../src-noconflict/snippets/apache_conf.js      |      7 +
 .../src-noconflict/snippets/asciidoc.js         |      7 +
 .../src-noconflict/snippets/assembly_x86.js     |      7 +
 .../src-noconflict/snippets/autohotkey.js       |      7 +
 .../src-noconflict/snippets/batchfile.js        |      7 +
 .../src-noconflict/snippets/c9search.js         |      7 +
 .../ace-builds/src-noconflict/snippets/c_cpp.js |    138 +
 .../ace-builds/src-noconflict/snippets/cirru.js |      7 +
 .../src-noconflict/snippets/clojure.js          |     97 +
 .../ace-builds/src-noconflict/snippets/cobol.js |      7 +
 .../src-noconflict/snippets/coffee.js           |    102 +
 .../src-noconflict/snippets/coldfusion.js       |      7 +
 .../src-noconflict/snippets/csharp.js           |      7 +
 .../ace-builds/src-noconflict/snippets/css.js   |    974 +
 .../ace-builds/src-noconflict/snippets/curly.js |      7 +
 .../ace-builds/src-noconflict/snippets/d.js     |      7 +
 .../ace-builds/src-noconflict/snippets/dart.js  |     90 +
 .../ace-builds/src-noconflict/snippets/diff.js  |     18 +
 .../src-noconflict/snippets/django.js           |    115 +
 .../ace-builds/src-noconflict/snippets/dot.js   |      7 +
 .../ace-builds/src-noconflict/snippets/ejs.js   |      7 +
 .../src-noconflict/snippets/erlang.js           |    167 +
 .../ace-builds/src-noconflict/snippets/forth.js |      7 +
 .../ace-builds/src-noconflict/snippets/ftl.js   |      7 +
 .../src-noconflict/snippets/gherkin.js          |      7 +
 .../ace-builds/src-noconflict/snippets/glsl.js  |      7 +
 .../src-noconflict/snippets/golang.js           |      7 +
 .../src-noconflict/snippets/groovy.js           |      7 +
 .../ace-builds/src-noconflict/snippets/haml.js  |     27 +
 .../src-noconflict/snippets/handlebars.js       |      7 +
 .../src-noconflict/snippets/haskell.js          |     89 +
 .../ace-builds/src-noconflict/snippets/haxe.js  |      7 +
 .../ace-builds/src-noconflict/snippets/html.js  |    835 +
 .../src-noconflict/snippets/html_completions.js |      7 +
 .../src-noconflict/snippets/html_ruby.js        |      7 +
 .../ace-builds/src-noconflict/snippets/ini.js   |      7 +
 .../ace-builds/src-noconflict/snippets/jack.js  |      7 +
 .../ace-builds/src-noconflict/snippets/jade.js  |      7 +
 .../ace-builds/src-noconflict/snippets/java.js  |    247 +
 .../src-noconflict/snippets/javascript.js       |    793 +
 .../ace-builds/src-noconflict/snippets/json.js  |      7 +
 .../src-noconflict/snippets/jsoniq.js           |      7 +
 .../ace-builds/src-noconflict/snippets/jsp.js   |    106 +
 .../ace-builds/src-noconflict/snippets/jsx.js   |      7 +
 .../ace-builds/src-noconflict/snippets/julia.js |      7 +
 .../ace-builds/src-noconflict/snippets/latex.js |      7 +
 .../ace-builds/src-noconflict/snippets/less.js  |      7 +
 .../src-noconflict/snippets/liquid.js           |      7 +
 .../ace-builds/src-noconflict/snippets/lisp.js  |      7 +
 .../src-noconflict/snippets/livescript.js       |      7 +
 .../src-noconflict/snippets/logiql.js           |      7 +
 .../ace-builds/src-noconflict/snippets/lsl.js   |    894 +
 .../ace-builds/src-noconflict/snippets/lua.js   |     28 +
 .../src-noconflict/snippets/luapage.js          |      7 +
 .../src-noconflict/snippets/lucene.js           |      7 +
 .../src-noconflict/snippets/makefile.js         |     11 +
 .../src-noconflict/snippets/markdown.js         |     95 +
 .../src-noconflict/snippets/matlab.js           |      7 +
 .../ace-builds/src-noconflict/snippets/mel.js   |      7 +
 .../src-noconflict/snippets/mushcode.js         |      7 +
 .../snippets/mushcode_high_rules.js             |      7 +
 .../ace-builds/src-noconflict/snippets/mysql.js |      7 +
 .../ace-builds/src-noconflict/snippets/nix.js   |      7 +
 .../src-noconflict/snippets/objectivec.js       |      7 +
 .../ace-builds/src-noconflict/snippets/ocaml.js |      7 +
 .../src-noconflict/snippets/pascal.js           |      7 +
 .../ace-builds/src-noconflict/snippets/perl.js  |    354 +
 .../ace-builds/src-noconflict/snippets/pgsql.js |      7 +
 .../ace-builds/src-noconflict/snippets/php.js   |    384 +
 .../src-noconflict/snippets/plain_text.js       |      7 +
 .../src-noconflict/snippets/powershell.js       |      7 +
 .../src-noconflict/snippets/prolog.js           |      7 +
 .../src-noconflict/snippets/properties.js       |      7 +
 .../src-noconflict/snippets/protobuf.js         |      7 +
 .../src-noconflict/snippets/python.js           |    165 +
 .../ace-builds/src-noconflict/snippets/r.js     |    128 +
 .../ace-builds/src-noconflict/snippets/rdoc.js  |      7 +
 .../ace-builds/src-noconflict/snippets/rhtml.js |      7 +
 .../ace-builds/src-noconflict/snippets/ruby.js  |    935 +
 .../ace-builds/src-noconflict/snippets/rust.js  |      7 +
 .../ace-builds/src-noconflict/snippets/sass.js  |      7 +
 .../ace-builds/src-noconflict/snippets/scad.js  |      7 +
 .../ace-builds/src-noconflict/snippets/scala.js |      7 +
 .../src-noconflict/snippets/scheme.js           |      7 +
 .../ace-builds/src-noconflict/snippets/scss.js  |      7 +
 .../ace-builds/src-noconflict/snippets/sh.js    |     90 +
 .../ace-builds/src-noconflict/snippets/sjs.js   |      7 +
 .../src-noconflict/snippets/smarty.js           |      7 +
 .../src-noconflict/snippets/snippets.js         |     16 +
 .../src-noconflict/snippets/soy_template.js     |      7 +
 .../ace-builds/src-noconflict/snippets/space.js |      7 +
 .../ace-builds/src-noconflict/snippets/sql.js   |     33 +
 .../src-noconflict/snippets/stylus.js           |      7 +
 .../ace-builds/src-noconflict/snippets/svg.js   |      7 +
 .../ace-builds/src-noconflict/snippets/tcl.js   |     99 +
 .../ace-builds/src-noconflict/snippets/tex.js   |    197 +
 .../ace-builds/src-noconflict/snippets/text.js  |      7 +
 .../src-noconflict/snippets/textile.js          |     37 +
 .../ace-builds/src-noconflict/snippets/toml.js  |      7 +
 .../ace-builds/src-noconflict/snippets/twig.js  |      7 +
 .../src-noconflict/snippets/typescript.js       |      7 +
 .../src-noconflict/snippets/vbscript.js         |      7 +
 .../src-noconflict/snippets/velocity.js         |     36 +
 .../src-noconflict/snippets/verilog.js          |      7 +
 .../ace-builds/src-noconflict/snippets/vhdl.js  |      7 +
 .../ace-builds/src-noconflict/snippets/xml.js   |      7 +
 .../src-noconflict/snippets/xquery.js           |      7 +
 .../ace-builds/src-noconflict/snippets/yaml.js  |      7 +
 .../ace-builds/src-noconflict/theme-ambiance.js |    202 +
 .../ace-builds/src-noconflict/theme-chaos.js    |    179 +
 .../ace-builds/src-noconflict/theme-chrome.js   |    158 +
 .../ace-builds/src-noconflict/theme-clouds.js   |    126 +
 .../src-noconflict/theme-clouds_midnight.js     |    127 +
 .../ace-builds/src-noconflict/theme-cobalt.js   |    143 +
 .../src-noconflict/theme-crimson_editor.js      |    148 +
 .../ace-builds/src-noconflict/theme-dawn.js     |    139 +
 .../src-noconflict/theme-dreamweaver.js         |    168 +
 .../ace-builds/src-noconflict/theme-eclipse.js  |    125 +
 .../ace-builds/src-noconflict/theme-github.js   |    131 +
 .../src-noconflict/theme-idle_fingers.js        |    127 +
 .../src-noconflict/theme-katzenmilch.js         |    150 +
 .../ace-builds/src-noconflict/theme-kr.js       |    135 +
 .../ace-builds/src-noconflict/theme-kuroir.js   |     89 +
 .../src-noconflict/theme-merbivore.js           |    126 +
 .../src-noconflict/theme-merbivore_soft.js      |    127 +
 .../src-noconflict/theme-mono_industrial.js     |    138 +
 .../ace-builds/src-noconflict/theme-monokai.js  |    136 +
 .../src-noconflict/theme-pastel_on_dark.js      |    139 +
 .../src-noconflict/theme-solarized_dark.js      |    119 +
 .../src-noconflict/theme-solarized_light.js     |    122 +
 .../ace-builds/src-noconflict/theme-terminal.js |    145 +
 .../ace-builds/src-noconflict/theme-textmate.js |    160 +
 .../ace-builds/src-noconflict/theme-tomorrow.js |    139 +
 .../src-noconflict/theme-tomorrow_night.js      |    139 +
 .../src-noconflict/theme-tomorrow_night_blue.js |    137 +
 .../theme-tomorrow_night_bright.js              |    152 +
 .../theme-tomorrow_night_eighties.js            |    139 +
 .../ace-builds/src-noconflict/theme-twilight.js |    140 +
 .../src-noconflict/theme-vibrant_ink.js         |    125 +
 .../ace-builds/src-noconflict/theme-xcode.js    |    119 +
 .../ace-builds/src-noconflict/worker-coffee.js  |   7578 +
 .../ace-builds/src-noconflict/worker-css.js     |   8323 +
 .../ace-builds/src-noconflict/worker-html.js    |  12225 +
 .../src-noconflict/worker-javascript.js         |  10117 +
 .../ace-builds/src-noconflict/worker-json.js    |   2296 +
 .../ace-builds/src-noconflict/worker-lua.js     |   3532 +
 .../ace-builds/src-noconflict/worker-php.js     |   6900 +
 .../ace-builds/src-noconflict/worker-xquery.js  |  21888 +
 webapp/app/components/ace-builds/src/ace.js     |  17601 +
 .../components/ace-builds/src/ext-chromevox.js  |    542 +
 .../ace-builds/src/ext-elastic_tabstops_lite.js |    305 +
 .../app/components/ace-builds/src/ext-emmet.js  |   1114 +
 .../ace-builds/src/ext-error_marker.js          |      5 +
 .../ace-builds/src/ext-keybinding_menu.js       |    211 +
 .../ace-builds/src/ext-language_tools.js        |   1669 +
 .../components/ace-builds/src/ext-modelist.js   |    176 +
 .../app/components/ace-builds/src/ext-old_ie.js |    505 +
 .../components/ace-builds/src/ext-options.js    |    252 +
 .../components/ace-builds/src/ext-searchbox.js  |    426 +
 .../ace-builds/src/ext-settings_menu.js         |    639 +
 .../components/ace-builds/src/ext-spellcheck.js |     73 +
 .../app/components/ace-builds/src/ext-split.js  |    276 +
 .../ace-builds/src/ext-static_highlight.js      |    183 +
 .../components/ace-builds/src/ext-statusbar.js  |     51 +
 .../components/ace-builds/src/ext-textarea.js   |    483 +
 .../components/ace-builds/src/ext-themelist.js  |     91 +
 .../components/ace-builds/src/ext-whitespace.js |    214 +
 .../ace-builds/src/keybinding-emacs.js          |   1058 +
 .../components/ace-builds/src/keybinding-vim.js |   1762 +
 .../app/components/ace-builds/src/mode-abap.js  |    261 +
 .../ace-builds/src/mode-actionscript.js         |    257 +
 .../app/components/ace-builds/src/mode-ada.js   |    118 +
 .../ace-builds/src/mode-apache_conf.js          |    346 +
 .../components/ace-builds/src/mode-asciidoc.js  |    373 +
 .../ace-builds/src/mode-assembly_x86.js         |    217 +
 .../ace-builds/src/mode-autohotkey.js           |    225 +
 .../components/ace-builds/src/mode-batchfile.js |    213 +
 .../components/ace-builds/src/mode-c9search.js  |    308 +
 .../app/components/ace-builds/src/mode-c_cpp.js |    816 +
 .../app/components/ace-builds/src/mode-cirru.js |    233 +
 .../components/ace-builds/src/mode-clojure.js   |    300 +
 .../app/components/ace-builds/src/mode-cobol.js |    125 +
 .../components/ace-builds/src/mode-coffee.js    |    444 +
 .../ace-builds/src/mode-coldfusion.js           |   2052 +
 .../components/ace-builds/src/mode-csharp.js    |    801 +
 .../app/components/ace-builds/src/mode-css.js   |    815 +
 .../app/components/ace-builds/src/mode-curly.js |   2475 +
 webapp/app/components/ace-builds/src/mode-d.js  |    492 +
 .../app/components/ace-builds/src/mode-dart.js  |   1023 +
 .../app/components/ace-builds/src/mode-diff.js  |    170 +
 .../components/ace-builds/src/mode-django.js    |   2504 +
 .../app/components/ace-builds/src/mode-dot.js   |    361 +
 .../app/components/ace-builds/src/mode-ejs.js   |   2869 +
 .../components/ace-builds/src/mode-erlang.js    |    987 +
 .../app/components/ace-builds/src/mode-forth.js |    280 +
 .../app/components/ace-builds/src/mode-ftl.js   |   1061 +
 .../components/ace-builds/src/mode-gherkin.js   |    160 +
 .../app/components/ace-builds/src/mode-glsl.js  |    894 +
 .../components/ace-builds/src/mode-golang.js    |    704 +
 .../components/ace-builds/src/mode-groovy.js    |   1125 +
 .../app/components/ace-builds/src/mode-haml.js  |    498 +
 .../ace-builds/src/mode-handlebars.js           |   2485 +
 .../components/ace-builds/src/mode-haskell.js   |    362 +
 .../app/components/ace-builds/src/mode-haxe.js  |    687 +
 .../app/components/ace-builds/src/mode-html.js  |   2413 +
 .../ace-builds/src/mode-html_completions.js     |    309 +
 .../components/ace-builds/src/mode-html_ruby.js |   2874 +
 .../app/components/ace-builds/src/mode-ini.js   |    185 +
 .../app/components/ace-builds/src/mode-jack.js  |    683 +
 .../app/components/ace-builds/src/mode-jade.js  |   2081 +
 .../app/components/ace-builds/src/mode-java.js  |   1083 +
 .../ace-builds/src/mode-javascript.js           |    963 +
 .../app/components/ace-builds/src/mode-json.js  |    655 +
 .../components/ace-builds/src/mode-jsoniq.js    |   2791 +
 .../app/components/ace-builds/src/mode-jsp.js   |   1583 +
 .../app/components/ace-builds/src/mode-jsx.js   |    712 +
 .../app/components/ace-builds/src/mode-julia.js |    286 +
 .../app/components/ace-builds/src/mode-latex.js |    190 +
 .../app/components/ace-builds/src/mode-less.js  |    884 +
 .../components/ace-builds/src/mode-liquid.js    |   1063 +
 .../app/components/ace-builds/src/mode-lisp.js  |    137 +
 .../ace-builds/src/mode-livescript.js           |    289 +
 .../components/ace-builds/src/mode-logiql.js    |    699 +
 .../app/components/ace-builds/src/mode-lsl.js   |    920 +
 .../app/components/ace-builds/src/mode-lua.js   |    457 +
 .../components/ace-builds/src/mode-luapage.js   |   2884 +
 .../components/ace-builds/src/mode-lucene.js    |     68 +
 .../components/ace-builds/src/mode-makefile.js  |    332 +
 .../components/ace-builds/src/mode-markdown.js  |   2776 +
 .../components/ace-builds/src/mode-matlab.js    |    230 +
 .../app/components/ace-builds/src/mode-mel.js   |    600 +
 .../components/ace-builds/src/mode-mushcode.js  |    705 +
 .../ace-builds/src/mode-mushcode_high_rules.js  |    569 +
 .../app/components/ace-builds/src/mode-mysql.js |    185 +
 .../app/components/ace-builds/src/mode-nix.js   |    965 +
 .../ace-builds/src/mode-objectivec.js           |    699 +
 .../app/components/ace-builds/src/mode-ocaml.js |    445 +
 .../components/ace-builds/src/mode-pascal.js    |    233 +
 .../app/components/ace-builds/src/mode-perl.js  |    359 +
 .../app/components/ace-builds/src/mode-pgsql.js |   1356 +
 .../app/components/ace-builds/src/mode-php.js   |   2526 +
 .../ace-builds/src/mode-plain_text.js           |     56 +
 .../ace-builds/src/mode-powershell.js           |    695 +
 .../components/ace-builds/src/mode-prolog.js    |    354 +
 .../ace-builds/src/mode-properties.js           |    104 +
 .../components/ace-builds/src/mode-protobuf.js  |    915 +
 .../components/ace-builds/src/mode-python.js    |    295 +
 webapp/app/components/ace-builds/src/mode-r.js  |    337 +
 .../app/components/ace-builds/src/mode-rdoc.js  |    210 +
 .../app/components/ace-builds/src/mode-rhtml.js |   2715 +
 .../app/components/ace-builds/src/mode-ruby.js  |    444 +
 .../app/components/ace-builds/src/mode-rust.js  |    245 +
 .../app/components/ace-builds/src/mode-sass.js  |    443 +
 .../app/components/ace-builds/src/mode-scad.js  |    747 +
 .../app/components/ace-builds/src/mode-scala.js |   1113 +
 .../components/ace-builds/src/mode-scheme.js    |    143 +
 .../app/components/ace-builds/src/mode-scss.js  |    909 +
 webapp/app/components/ace-builds/src/mode-sh.js |    320 +
 .../app/components/ace-builds/src/mode-sjs.js   |   1184 +
 .../components/ace-builds/src/mode-smarty.js    |   2546 +
 .../components/ace-builds/src/mode-snippets.js  |    198 +
 .../ace-builds/src/mode-soy_template.js         |   2759 +
 .../app/components/ace-builds/src/mode-space.js |    160 +
 .../app/components/ace-builds/src/mode-sql.js   |    124 +
 .../components/ace-builds/src/mode-stylus.js    |    447 +
 .../app/components/ace-builds/src/mode-svg.js   |   1658 +
 .../app/components/ace-builds/src/mode-tcl.js   |    361 +
 .../app/components/ace-builds/src/mode-tex.js   |    187 +
 .../app/components/ace-builds/src/mode-text.js  |      0
 .../components/ace-builds/src/mode-textile.js   |    171 +
 .../components/ace-builds/src/mode-tmsnippet.js |    200 +
 .../app/components/ace-builds/src/mode-toml.js  |    177 +
 .../app/components/ace-builds/src/mode-twig.js  |   2258 +
 .../ace-builds/src/mode-typescript.js           |   1048 +
 .../components/ace-builds/src/mode-vbscript.js  |    250 +
 .../components/ace-builds/src/mode-velocity.js  |   1652 +
 .../components/ace-builds/src/mode-verilog.js   |    127 +
 .../app/components/ace-builds/src/mode-vhdl.js  |    139 +
 .../app/components/ace-builds/src/mode-xml.js   |    967 +
 .../components/ace-builds/src/mode-xquery.js    |   2825 +
 .../app/components/ace-builds/src/mode-yaml.js  |    287 +
 .../components/ace-builds/src/snippets/abap.js  |      7 +
 .../ace-builds/src/snippets/actionscript.js     |    164 +
 .../components/ace-builds/src/snippets/ada.js   |      7 +
 .../ace-builds/src/snippets/apache_conf.js      |      7 +
 .../ace-builds/src/snippets/asciidoc.js         |      7 +
 .../ace-builds/src/snippets/assembly_x86.js     |      7 +
 .../ace-builds/src/snippets/autohotkey.js       |      7 +
 .../ace-builds/src/snippets/batchfile.js        |      7 +
 .../ace-builds/src/snippets/c9search.js         |      7 +
 .../components/ace-builds/src/snippets/c_cpp.js |    138 +
 .../components/ace-builds/src/snippets/cirru.js |      7 +
 .../ace-builds/src/snippets/clojure.js          |     97 +
 .../components/ace-builds/src/snippets/cobol.js |      7 +
 .../ace-builds/src/snippets/coffee.js           |    102 +
 .../ace-builds/src/snippets/coldfusion.js       |      7 +
 .../ace-builds/src/snippets/csharp.js           |      7 +
 .../components/ace-builds/src/snippets/css.js   |    974 +
 .../components/ace-builds/src/snippets/curly.js |      7 +
 .../app/components/ace-builds/src/snippets/d.js |      7 +
 .../components/ace-builds/src/snippets/dart.js  |     90 +
 .../components/ace-builds/src/snippets/diff.js  |     18 +
 .../ace-builds/src/snippets/django.js           |    115 +
 .../components/ace-builds/src/snippets/dot.js   |      7 +
 .../components/ace-builds/src/snippets/ejs.js   |      7 +
 .../ace-builds/src/snippets/erlang.js           |    167 +
 .../components/ace-builds/src/snippets/forth.js |      7 +
 .../components/ace-builds/src/snippets/ftl.js   |      7 +
 .../ace-builds/src/snippets/gherkin.js          |      7 +
 .../components/ace-builds/src/snippets/glsl.js  |      7 +
 .../ace-builds/src/snippets/golang.js           |      7 +
 .../ace-builds/src/snippets/groovy.js           |      7 +
 .../components/ace-builds/src/snippets/haml.js  |     27 +
 .../ace-builds/src/snippets/handlebars.js       |      7 +
 .../ace-builds/src/snippets/haskell.js          |     89 +
 .../components/ace-builds/src/snippets/haxe.js  |      7 +
 .../components/ace-builds/src/snippets/html.js  |    835 +
 .../ace-builds/src/snippets/html_completions.js |      7 +
 .../ace-builds/src/snippets/html_ruby.js        |      7 +
 .../components/ace-builds/src/snippets/ini.js   |      7 +
 .../components/ace-builds/src/snippets/jack.js  |      7 +
 .../components/ace-builds/src/snippets/jade.js  |      7 +
 .../components/ace-builds/src/snippets/java.js  |    247 +
 .../ace-builds/src/snippets/javascript.js       |    793 +
 .../components/ace-builds/src/snippets/json.js  |      7 +
 .../ace-builds/src/snippets/jsoniq.js           |      7 +
 .../components/ace-builds/src/snippets/jsp.js   |    106 +
 .../components/ace-builds/src/snippets/jsx.js   |      7 +
 .../components/ace-builds/src/snippets/julia.js |      7 +
 .../components/ace-builds/src/snippets/latex.js |      7 +
 .../components/ace-builds/src/snippets/less.js  |      7 +
 .../ace-builds/src/snippets/liquid.js           |      7 +
 .../components/ace-builds/src/snippets/lisp.js  |      7 +
 .../ace-builds/src/snippets/livescript.js       |      7 +
 .../ace-builds/src/snippets/logiql.js           |      7 +
 .../components/ace-builds/src/snippets/lsl.js   |    894 +
 .../components/ace-builds/src/snippets/lua.js   |     28 +
 .../ace-builds/src/snippets/luapage.js          |      7 +
 .../ace-builds/src/snippets/lucene.js           |      7 +
 .../ace-builds/src/snippets/makefile.js         |     11 +
 .../ace-builds/src/snippets/markdown.js         |     95 +
 .../ace-builds/src/snippets/matlab.js           |      7 +
 .../components/ace-builds/src/snippets/mel.js   |      7 +
 .../ace-builds/src/snippets/mushcode.js         |      7 +
 .../src/snippets/mushcode_high_rules.js         |      7 +
 .../components/ace-builds/src/snippets/mysql.js |      7 +
 .../components/ace-builds/src/snippets/nix.js   |      7 +
 .../ace-builds/src/snippets/objectivec.js       |      7 +
 .../components/ace-builds/src/snippets/ocaml.js |      7 +
 .../ace-builds/src/snippets/pascal.js           |      7 +
 .../components/ace-builds/src/snippets/perl.js  |    354 +
 .../components/ace-builds/src/snippets/pgsql.js |      7 +
 .../components/ace-builds/src/snippets/php.js   |    384 +
 .../ace-builds/src/snippets/plain_text.js       |      7 +
 .../ace-builds/src/snippets/powershell.js       |      7 +
 .../ace-builds/src/snippets/prolog.js           |      7 +
 .../ace-builds/src/snippets/properties.js       |      7 +
 .../ace-builds/src/snippets/protobuf.js         |      7 +
 .../ace-builds/src/snippets/python.js           |    165 +
 .../app/components/ace-builds/src/snippets/r.js |    128 +
 .../components/ace-builds/src/snippets/rdoc.js  |      7 +
 .../components/ace-builds/src/snippets/rhtml.js |      7 +
 .../components/ace-builds/src/snippets/ruby.js  |    935 +
 .../components/ace-builds/src/snippets/rust.js  |      7 +
 .../components/ace-builds/src/snippets/sass.js  |      7 +
 .../components/ace-builds/src/snippets/scad.js  |      7 +
 .../components/ace-builds/src/snippets/scala.js |      7 +
 .../ace-builds/src/snippets/scheme.js           |      7 +
 .../components/ace-builds/src/snippets/scss.js  |      7 +
 .../components/ace-builds/src/snippets/sh.js    |     90 +
 .../components/ace-builds/src/snippets/sjs.js   |      7 +
 .../ace-builds/src/snippets/smarty.js           |      7 +
 .../ace-builds/src/snippets/snippets.js         |     16 +
 .../ace-builds/src/snippets/soy_template.js     |      7 +
 .../components/ace-builds/src/snippets/space.js |      7 +
 .../components/ace-builds/src/snippets/sql.js   |     33 +
 .../ace-builds/src/snippets/stylus.js           |      7 +
 .../components/ace-builds/src/snippets/svg.js   |      7 +
 .../components/ace-builds/src/snippets/tcl.js   |     99 +
 .../components/ace-builds/src/snippets/tex.js   |    197 +
 .../components/ace-builds/src/snippets/text.js  |      7 +
 .../ace-builds/src/snippets/textile.js          |     37 +
 .../components/ace-builds/src/snippets/toml.js  |      7 +
 .../components/ace-builds/src/snippets/twig.js  |      7 +
 .../ace-builds/src/snippets/typescript.js       |      7 +
 .../ace-builds/src/snippets/vbscript.js         |      7 +
 .../ace-builds/src/snippets/velocity.js         |     36 +
 .../ace-builds/src/snippets/verilog.js          |      7 +
 .../components/ace-builds/src/snippets/vhdl.js  |      7 +
 .../components/ace-builds/src/snippets/xml.js   |      7 +
 .../ace-builds/src/snippets/xquery.js           |      7 +
 .../components/ace-builds/src/snippets/yaml.js  |      7 +
 .../components/ace-builds/src/theme-ambiance.js |    202 +
 .../components/ace-builds/src/theme-chaos.js    |    179 +
 .../components/ace-builds/src/theme-chrome.js   |    158 +
 .../components/ace-builds/src/theme-clouds.js   |    126 +
 .../ace-builds/src/theme-clouds_midnight.js     |    127 +
 .../components/ace-builds/src/theme-cobalt.js   |    143 +
 .../ace-builds/src/theme-crimson_editor.js      |    148 +
 .../app/components/ace-builds/src/theme-dawn.js |    139 +
 .../ace-builds/src/theme-dreamweaver.js         |    168 +
 .../components/ace-builds/src/theme-eclipse.js  |    125 +
 .../components/ace-builds/src/theme-github.js   |    131 +
 .../ace-builds/src/theme-idle_fingers.js        |    127 +
 .../ace-builds/src/theme-katzenmilch.js         |    150 +
 .../app/components/ace-builds/src/theme-kr.js   |    135 +
 .../components/ace-builds/src/theme-kuroir.js   |     89 +
 .../ace-builds/src/theme-merbivore.js           |    126 +
 .../ace-builds/src/theme-merbivore_soft.js      |    127 +
 .../ace-builds/src/theme-mono_industrial.js     |    138 +
 .../components/ace-builds/src/theme-monokai.js  |    136 +
 .../ace-builds/src/theme-pastel_on_dark.js      |    139 +
 .../ace-builds/src/theme-solarized_dark.js      |    119 +
 .../ace-builds/src/theme-solarized_light.js     |    122 +
 .../components/ace-builds/src/theme-terminal.js |    145 +
 .../components/ace-builds/src/theme-textmate.js |    160 +
 .../components/ace-builds/src/theme-tomorrow.js |    139 +
 .../ace-builds/src/theme-tomorrow_night.js      |    139 +
 .../ace-builds/src/theme-tomorrow_night_blue.js |    137 +
 .../src/theme-tomorrow_night_bright.js          |    152 +
 .../src/theme-tomorrow_night_eighties.js        |    139 +
 .../components/ace-builds/src/theme-twilight.js |    140 +
 .../ace-builds/src/theme-vibrant_ink.js         |    125 +
 .../components/ace-builds/src/theme-xcode.js    |    119 +
 .../components/ace-builds/src/worker-coffee.js  |   7578 +
 .../app/components/ace-builds/src/worker-css.js |   8323 +
 .../components/ace-builds/src/worker-html.js    |  12225 +
 .../ace-builds/src/worker-javascript.js         |  10117 +
 .../components/ace-builds/src/worker-json.js    |   2296 +
 .../app/components/ace-builds/src/worker-lua.js |   3532 +
 .../app/components/ace-builds/src/worker-php.js |   6900 +
 .../components/ace-builds/src/worker-xquery.js  |  21888 +
 .../components/ace-builds/static_highlight.js   |   7506 +
 .../components/ace-builds/textarea/editor.html  |    115 +
 .../ace-builds/textarea/images/ace.png          |    Bin 0 -> 24364 bytes
 .../ace-builds/textarea/images/background.png   |    Bin 0 -> 3445 bytes
 .../textarea/images/body_background.png         |    Bin 0 -> 3013 bytes
 .../ace-builds/textarea/images/bottombar.png    |    Bin 0 -> 1186 bytes
 .../textarea/images/fork_on_github.png          |    Bin 0 -> 18528 bytes
 .../ace-builds/textarea/images/logo.png         |    Bin 0 -> 52537 bytes
 .../ace-builds/textarea/images/logo_half.png    |    Bin 0 -> 19990 bytes
 .../ace-builds/textarea/src/ace-bookmarklet.js  |  18050 +
 .../ace-builds/textarea/src/ext-chromevox.js    |    542 +
 .../textarea/src/ext-elastic_tabstops_lite.js   |    305 +
 .../ace-builds/textarea/src/ext-emmet.js        |   1114 +
 .../ace-builds/textarea/src/ext-error_marker.js |      5 +
 .../textarea/src/ext-keybinding_menu.js         |    211 +
 .../textarea/src/ext-language_tools.js          |   1669 +
 .../ace-builds/textarea/src/ext-modelist.js     |    176 +
 .../ace-builds/textarea/src/ext-old_ie.js       |    505 +
 .../ace-builds/textarea/src/ext-searchbox.js    |    426 +
 .../textarea/src/ext-settings_menu.js           |    639 +
 .../ace-builds/textarea/src/ext-spellcheck.js   |     73 +
 .../ace-builds/textarea/src/ext-split.js        |    276 +
 .../textarea/src/ext-static_highlight.js        |    183 +
 .../ace-builds/textarea/src/ext-statusbar.js    |     51 +
 .../ace-builds/textarea/src/ext-textarea.js     |      5 +
 .../ace-builds/textarea/src/ext-themelist.js    |     91 +
 .../ace-builds/textarea/src/ext-whitespace.js   |    214 +
 .../ace-builds/textarea/src/mode-abap.js        |    261 +
 .../textarea/src/mode-actionscript.js           |    257 +
 .../ace-builds/textarea/src/mode-ada.js         |    118 +
 .../ace-builds/textarea/src/mode-apache_conf.js |    346 +
 .../ace-builds/textarea/src/mode-asciidoc.js    |    373 +
 .../textarea/src/mode-assembly_x86.js           |    217 +
 .../ace-builds/textarea/src/mode-autohotkey.js  |    225 +
 .../ace-builds/textarea/src/mode-batchfile.js   |    213 +
 .../ace-builds/textarea/src/mode-c9search.js    |    308 +
 .../ace-builds/textarea/src/mode-c_cpp.js       |    816 +
 .../ace-builds/textarea/src/mode-cirru.js       |    233 +
 .../ace-builds/textarea/src/mode-clojure.js     |    300 +
 .../ace-builds/textarea/src/mode-cobol.js       |    125 +
 .../ace-builds/textarea/src/mode-coffee.js      |    444 +
 .../ace-builds/textarea/src/mode-coldfusion.js  |   2052 +
 .../ace-builds/textarea/src/mode-csharp.js      |    801 +
 .../ace-builds/textarea/src/mode-css.js         |    815 +
 .../ace-builds/textarea/src/mode-curly.js       |   2475 +
 .../ace-builds/textarea/src/mode-d.js           |    492 +
 .../ace-builds/textarea/src/mode-dart.js        |   1023 +
 .../ace-builds/textarea/src/mode-diff.js        |    170 +
 .../ace-builds/textarea/src/mode-django.js      |   2504 +
 .../ace-builds/textarea/src/mode-dot.js         |    361 +
 .../ace-builds/textarea/src/mode-ejs.js         |   2869 +
 .../ace-builds/textarea/src/mode-erlang.js      |    987 +
 .../ace-builds/textarea/src/mode-forth.js       |    280 +
 .../ace-builds/textarea/src/mode-ftl.js         |   1061 +
 .../ace-builds/textarea/src/mode-gherkin.js     |    160 +
 .../ace-builds/textarea/src/mode-glsl.js        |    894 +
 .../ace-builds/textarea/src/mode-golang.js      |    704 +
 .../ace-builds/textarea/src/mode-groovy.js      |   1125 +
 .../ace-builds/textarea/src/mode-haml.js        |    498 +
 .../ace-builds/textarea/src/mode-handlebars.js  |   2485 +
 .../ace-builds/textarea/src/mode-haskell.js     |    362 +
 .../ace-builds/textarea/src/mode-haxe.js        |    687 +
 .../ace-builds/textarea/src/mode-html.js        |   2413 +
 .../textarea/src/mode-html_completions.js       |    309 +
 .../ace-builds/textarea/src/mode-html_ruby.js   |   2874 +
 .../ace-builds/textarea/src/mode-ini.js         |    185 +
 .../ace-builds/textarea/src/mode-jack.js        |    683 +
 .../ace-builds/textarea/src/mode-jade.js        |   2081 +
 .../ace-builds/textarea/src/mode-java.js        |   1083 +
 .../ace-builds/textarea/src/mode-javascript.js  |    963 +
 .../ace-builds/textarea/src/mode-json.js        |    655 +
 .../ace-builds/textarea/src/mode-jsoniq.js      |   2791 +
 .../ace-builds/textarea/src/mode-jsp.js         |   1583 +
 .../ace-builds/textarea/src/mode-jsx.js         |    712 +
 .../ace-builds/textarea/src/mode-julia.js       |    286 +
 .../ace-builds/textarea/src/mode-latex.js       |    190 +
 .../ace-builds/textarea/src/mode-less.js        |    884 +
 .../ace-builds/textarea/src/mode-liquid.js      |   1063 +
 .../ace-builds/textarea/src/mode-lisp.js        |    137 +
 .../ace-builds/textarea/src/mode-livescript.js  |    289 +
 .../ace-builds/textarea/src/mode-logiql.js      |    699 +
 .../ace-builds/textarea/src/mode-lsl.js         |    920 +
 .../ace-builds/textarea/src/mode-lua.js         |    457 +
 .../ace-builds/textarea/src/mode-luapage.js     |   2884 +
 .../ace-builds/textarea/src/mode-lucene.js      |     68 +
 .../ace-builds/textarea/src/mode-makefile.js    |    332 +
 .../ace-builds/textarea/src/mode-markdown.js    |   2776 +
 .../ace-builds/textarea/src/mode-matlab.js      |    230 +
 .../ace-builds/textarea/src/mode-mel.js         |    600 +
 .../ace-builds/textarea/src/mode-mushcode.js    |    705 +
 .../textarea/src/mode-mushcode_high_rules.js    |    569 +
 .../ace-builds/textarea/src/mode-mysql.js       |    185 +
 .../ace-builds/textarea/src/mode-nix.js         |    965 +
 .../ace-builds/textarea/src/mode-objectivec.js  |    699 +
 .../ace-builds/textarea/src/mode-ocaml.js       |    445 +
 .../ace-builds/textarea/src/mode-pascal.js      |    233 +
 .../ace-builds/textarea/src/mode-perl.js        |    359 +
 .../ace-builds/textarea/src/mode-pgsql.js       |   1356 +
 .../ace-builds/textarea/src/mode-php.js         |   2526 +
 .../ace-builds/textarea/src/mode-plain_text.js  |     56 +
 .../ace-builds/textarea/src/mode-powershell.js  |    695 +
 .../ace-builds/textarea/src/mode-prolog.js      |    354 +
 .../ace-builds/textarea/src/mode-properties.js  |    104 +
 .../ace-builds/textarea/src/mode-protobuf.js    |    915 +
 .../ace-builds/textarea/src/mode-python.js      |    295 +
 .../ace-builds/textarea/src/mode-r.js           |    337 +
 .../ace-builds/textarea/src/mode-rdoc.js        |    210 +
 .../ace-builds/textarea/src/mode-rhtml.js       |   2715 +
 .../ace-builds/textarea/src/mode-ruby.js        |    444 +
 .../ace-builds/textarea/src/mode-rust.js        |    245 +
 .../ace-builds/textarea/src/mode-sass.js        |    443 +
 .../ace-builds/textarea/src/mode-scad.js        |    747 +
 .../ace-builds/textarea/src/mode-scala.js       |   1113 +
 .../ace-builds/textarea/src/mode-scheme.js      |    143 +
 .../ace-builds/textarea/src/mode-scss.js        |    909 +
 .../ace-builds/textarea/src/mode-sh.js          |    320 +
 .../ace-builds/textarea/src/mode-sjs.js         |   1184 +
 .../ace-builds/textarea/src/mode-smarty.js      |   2546 +
 .../ace-builds/textarea/src/mode-snippets.js    |    198 +
 .../textarea/src/mode-soy_template.js           |   2759 +
 .../ace-builds/textarea/src/mode-space.js       |    160 +
 .../ace-builds/textarea/src/mode-sql.js         |    124 +
 .../ace-builds/textarea/src/mode-stylus.js      |    447 +
 .../ace-builds/textarea/src/mode-svg.js         |   1658 +
 .../ace-builds/textarea/src/mode-tcl.js         |    361 +
 .../ace-builds/textarea/src/mode-tex.js         |    187 +
 .../ace-builds/textarea/src/mode-text.js        |      0
 .../ace-builds/textarea/src/mode-textile.js     |    171 +
 .../ace-builds/textarea/src/mode-toml.js        |    177 +
 .../ace-builds/textarea/src/mode-twig.js        |   2258 +
 .../ace-builds/textarea/src/mode-typescript.js  |   1048 +
 .../ace-builds/textarea/src/mode-vbscript.js    |    250 +
 .../ace-builds/textarea/src/mode-velocity.js    |   1652 +
 .../ace-builds/textarea/src/mode-verilog.js     |    127 +
 .../ace-builds/textarea/src/mode-vhdl.js        |    139 +
 .../ace-builds/textarea/src/mode-xml.js         |    967 +
 .../ace-builds/textarea/src/mode-xquery.js      |   2825 +
 .../ace-builds/textarea/src/mode-yaml.js        |    287 +
 .../ace-builds/textarea/src/snippets/abap.js    |      7 +
 .../textarea/src/snippets/actionscript.js       |    164 +
 .../ace-builds/textarea/src/snippets/ada.js     |      7 +
 .../textarea/src/snippets/apache_conf.js        |      7 +
 .../textarea/src/snippets/asciidoc.js           |      7 +
 .../textarea/src/snippets/assembly_x86.js       |      7 +
 .../textarea/src/snippets/autohotkey.js         |      7 +
 .../textarea/src/snippets/batchfile.js          |      7 +
 .../textarea/src/snippets/c9search.js           |      7 +
 .../ace-builds/textarea/src/snippets/c_cpp.js   |    138 +
 .../ace-builds/textarea/src/snippets/cirru.js   |      7 +
 .../ace-builds/textarea/src/snippets/clojure.js |     97 +
 .../ace-builds/textarea/src/snippets/cobol.js   |      7 +
 .../ace-builds/textarea/src/snippets/coffee.js  |    102 +
 .../textarea/src/snippets/coldfusion.js         |      7 +
 .../ace-builds/textarea/src/snippets/csharp.js  |      7 +
 .../ace-builds/textarea/src/snippets/css.js     |    974 +
 .../ace-builds/textarea/src/snippets/curly.js   |      7 +
 .../ace-builds/textarea/src/snippets/d.js       |      7 +
 .../ace-builds/textarea/src/snippets/dart.js    |     90 +
 .../ace-builds/textarea/src/snippets/diff.js    |     18 +
 .../ace-builds/textarea/src/snippets/django.js  |    115 +
 .../ace-builds/textarea/src/snippets/dot.js     |      7 +
 .../ace-builds/textarea/src/snippets/ejs.js     |      7 +
 .../ace-builds/textarea/src/snippets/erlang.js  |    167 +
 .../ace-builds/textarea/src/snippets/forth.js   |      7 +
 .../ace-builds/textarea/src/snippets/ftl.js     |      7 +
 .../ace-builds/textarea/src/snippets/gherkin.js |      7 +
 .../ace-builds/textarea/src/snippets/glsl.js    |      7 +
 .../ace-builds/textarea/src/snippets/golang.js  |      7 +
 .../ace-builds/textarea/src/snippets/groovy.js  |      7 +
 .../ace-builds/textarea/src/snippets/haml.js    |     27 +
 .../textarea/src/snippets/handlebars.js         |      7 +
 .../ace-builds/textarea/src/snippets/haskell.js |     89 +
 .../ace-builds/textarea/src/snippets/haxe.js    |      7 +
 .../ace-builds/textarea/src/snippets/html.js    |    835 +
 .../textarea/src/snippets/html_completions.js   |      7 +
 .../textarea/src/snippets/html_ruby.js          |      7 +
 .../ace-builds/textarea/src/snippets/ini.js     |      7 +
 .../ace-builds/textarea/src/snippets/jack.js    |      7 +
 .../ace-builds/textarea/src/snippets/jade.js    |      7 +
 .../ace-builds/textarea/src/snippets/java.js    |    247 +
 .../textarea/src/snippets/javascript.js         |    793 +
 .../ace-builds/textarea/src/snippets/json.js    |      7 +
 .../ace-builds/textarea/src/snippets/jsoniq.js  |      7 +
 .../ace-builds/textarea/src/snippets/jsp.js     |    106 +
 .../ace-builds/textarea/src/snippets/jsx.js     |      7 +
 .../ace-builds/textarea/src/snippets/julia.js   |      7 +
 .../ace-builds/textarea/src/snippets/latex.js   |      7 +
 .../ace-builds/textarea/src/snippets/less.js    |      7 +
 .../ace-builds/textarea/src/snippets/liquid.js  |      7 +
 .../ace-builds/textarea/src/snippets/lisp.js    |      7 +
 .../textarea/src/snippets/livescript.js         |      7 +
 .../ace-builds/textarea/src/snippets/logiql.js  |      7 +
 .../ace-builds/textarea/src/snippets/lsl.js     |    894 +
 .../ace-builds/textarea/src/snippets/lua.js     |     28 +
 .../ace-builds/textarea/src/snippets/luapage.js |      7 +
 .../ace-builds/textarea/src/snippets/lucene.js  |      7 +
 .../textarea/src/snippets/makefile.js           |     11 +
 .../textarea/src/snippets/markdown.js           |     95 +
 .../ace-builds/textarea/src/snippets/matlab.js  |      7 +
 .../ace-builds/textarea/src/snippets/mel.js     |      7 +
 .../textarea/src/snippets/mushcode.js           |      7 +
 .../src/snippets/mushcode_high_rules.js         |      7 +
 .../ace-builds/textarea/src/snippets/mysql.js   |      7 +
 .../ace-builds/textarea/src/snippets/nix.js     |      7 +
 .../textarea/src/snippets/objectivec.js         |      7 +
 .../ace-builds/textarea/src/snippets/ocaml.js   |      7 +
 .../ace-builds/textarea/src/snippets/pascal.js  |      7 +
 .../ace-builds/textarea/src/snippets/perl.js    |    354 +
 .../ace-builds/textarea/src/snippets/pgsql.js   |      7 +
 .../ace-builds/textarea/src/snippets/php.js     |    384 +
 .../textarea/src/snippets/plain_text.js         |      7 +
 .../textarea/src/snippets/powershell.js         |      7 +
 .../ace-builds/textarea/src/snippets/prolog.js  |      7 +
 .../textarea/src/snippets/properties.js         |      7 +
 .../textarea/src/snippets/protobuf.js           |      7 +
 .../ace-builds/textarea/src/snippets/python.js  |    165 +
 .../ace-builds/textarea/src/snippets/r.js       |    128 +
 .../ace-builds/textarea/src/snippets/rdoc.js    |      7 +
 .../ace-builds/textarea/src/snippets/rhtml.js   |      7 +
 .../ace-builds/textarea/src/snippets/ruby.js    |    935 +
 .../ace-builds/textarea/src/snippets/rust.js    |      7 +
 .../ace-builds/textarea/src/snippets/sass.js    |      7 +
 .../ace-builds/textarea/src/snippets/scad.js    |      7 +
 .../ace-builds/textarea/src/snippets/scala.js   |      7 +
 .../ace-builds/textarea/src/snippets/scheme.js  |      7 +
 .../ace-builds/textarea/src/snippets/scss.js    |      7 +
 .../ace-builds/textarea/src/snippets/sh.js      |     90 +
 .../ace-builds/textarea/src/snippets/sjs.js     |      7 +
 .../ace-builds/textarea/src/snippets/smarty.js  |      7 +
 .../textarea/src/snippets/snippets.js           |     16 +
 .../textarea/src/snippets/soy_template.js       |      7 +
 .../ace-builds/textarea/src/snippets/space.js   |      7 +
 .../ace-builds/textarea/src/snippets/sql.js     |     33 +
 .../ace-builds/textarea/src/snippets/stylus.js  |      7 +
 .../ace-builds/textarea/src/snippets/svg.js     |      7 +
 .../ace-builds/textarea/src/snippets/tcl.js     |     99 +
 .../ace-builds/textarea/src/snippets/tex.js     |    197 +
 .../ace-builds/textarea/src/snippets/text.js    |      7 +
 .../ace-builds/textarea/src/snippets/textile.js |     37 +
 .../ace-builds/textarea/src/snippets/toml.js    |      7 +
 .../ace-builds/textarea/src/snippets/twig.js    |      7 +
 .../textarea/src/snippets/typescript.js         |      7 +
 .../textarea/src/snippets/vbscript.js           |      7 +
 .../textarea/src/snippets/velocity.js           |     36 +
 .../ace-builds/textarea/src/snippets/verilog.js |      7 +
 .../ace-builds/textarea/src/snippets/vhdl.js    |      7 +
 .../ace-builds/textarea/src/snippets/xml.js     |      7 +
 .../ace-builds/textarea/src/snippets/xquery.js  |      7 +
 .../ace-builds/textarea/src/snippets/yaml.js    |      7 +
 .../ace-builds/textarea/src/theme-ambiance.js   |    202 +
 .../ace-builds/textarea/src/theme-chaos.js      |    179 +
 .../ace-builds/textarea/src/theme-chrome.js     |    158 +
 .../ace-builds/textarea/src/theme-clouds.js     |    126 +
 .../textarea/src/theme-clouds_midnight.js       |    127 +
 .../ace-builds/textarea/src/theme-cobalt.js     |    143 +
 .../textarea/src/theme-crimson_editor.js        |    148 +
 .../ace-builds/textarea/src/theme-dawn.js       |    139 +
 .../textarea/src/theme-dreamweaver.js           |    168 +
 .../ace-builds/textarea/src/theme-eclipse.js    |    125 +
 .../ace-builds/textarea/src/theme-github.js     |    131 +
 .../textarea/src/theme-idle_fingers.js          |    127 +
 .../textarea/src/theme-katzenmilch.js           |    150 +
 .../ace-builds/textarea/src/theme-kr.js         |    135 +
 .../ace-builds/textarea/src/theme-kuroir.js     |     89 +
 .../ace-builds/textarea/src/theme-merbivore.js  |    126 +
 .../textarea/src/theme-merbivore_soft.js        |    127 +
 .../textarea/src/theme-mono_industrial.js       |    138 +
 .../ace-builds/textarea/src/theme-monokai.js    |    136 +
 .../textarea/src/theme-pastel_on_dark.js        |    139 +
 .../textarea/src/theme-solarized_dark.js        |    119 +
 .../textarea/src/theme-solarized_light.js       |    122 +
 .../ace-builds/textarea/src/theme-terminal.js   |    145 +
 .../ace-builds/textarea/src/theme-textmate.js   |    160 +
 .../ace-builds/textarea/src/theme-tomorrow.js   |    139 +
 .../textarea/src/theme-tomorrow_night.js        |    139 +
 .../textarea/src/theme-tomorrow_night_blue.js   |    137 +
 .../textarea/src/theme-tomorrow_night_bright.js |    152 +
 .../src/theme-tomorrow_night_eighties.js        |    139 +
 .../ace-builds/textarea/src/theme-twilight.js   |    140 +
 .../textarea/src/theme-vibrant_ink.js           |    125 +
 .../ace-builds/textarea/src/theme-xcode.js      |    119 +
 .../components/ace-builds/textarea/style.css    |    230 +
 .../app/components/ace/css/AdminLTE-fonts.css   |     16 +
 webapp/app/components/ace/css/AdminLTE.css      |   3505 +
 ...1aHBYDBqgeIAH2zlPytCVloACqnDHJwh-em2k8.woff2 |    Bin 0 -> 11400 bytes
 ...t5NIMarvNW-hkYWvT3rGVtsTkPsbDajuO5ueQw.woff2 |    Bin 0 -> 11044 bytes
 .../app/components/angular-animate/.bower.json  |     20 +
 webapp/app/components/angular-animate/README.md |     77 +
 .../angular-animate/angular-animate.js          |   1702 +
 .../angular-animate/angular-animate.min.js      |     28 +
 .../angular-animate/angular-animate.min.js.map  |      8 +
 .../app/components/angular-animate/bower.json   |      9 +
 .../app/components/angular-animate/package.json |     26 +
 .../app/components/angular-base64/.bower.json   |     25 +
 webapp/app/components/angular-base64/LICENSE    |     34 +
 webapp/app/components/angular-base64/README.md  |     32 +
 .../components/angular-base64/angular-base64.js |    166 +
 .../angular-base64/angular-base64.min.js        |      1 +
 webapp/app/components/angular-base64/bower.json |     15 +
 .../components/angular-bootstrap/.bower.json    |     23 +
 .../app/components/angular-bootstrap/bower.json |     11 +
 .../angular-bootstrap/ui-bootstrap-tpls.js      |   3677 +
 .../angular-bootstrap/ui-bootstrap-tpls.min.js  |      9 +
 .../angular-bootstrap/ui-bootstrap.js           |   3427 +
 .../angular-bootstrap/ui-bootstrap.min.js       |      9 +
 .../angular-chosen-localytics/.bower.json       |     23 +
 .../angular-chosen-localytics/LICENSE           |     21 +
 .../angular-chosen-localytics/README.md         |    116 +
 .../angular-chosen-localytics/bower.json        |     13 +
 .../chosen-spinner.css                          |     12 +
 .../angular-chosen-localytics/chosen.js         |    109 +
 .../angular-chosen-localytics/spinner.gif       |    Bin 0 -> 1849 bytes
 .../app/components/angular-cookies/.bower.json  |     18 +
 webapp/app/components/angular-cookies/README.md |     54 +
 .../angular-cookies/angular-cookies.js          |    197 +
 .../angular-cookies/angular-cookies.min.js      |      8 +
 .../angular-cookies/angular-cookies.min.js.map  |      8 +
 .../app/components/angular-cookies/bower.json   |      8 +
 .../angular-mocks-unstable/.bower.json          |     18 +
 .../components/angular-mocks-unstable/README.md |      2 +
 .../angular-mocks-unstable/angular-mocks.js     |   1886 +
 .../angular-mocks-unstable/bower.json           |      8 +
 .../angular-resource-unstable/.bower.json       |     18 +
 .../angular-resource-unstable/README.md         |      7 +
 .../angular-resource.js                         |    537 +
 .../angular-resource.min.js                     |     11 +
 .../angular-resource-unstable/bower.json        |      8 +
 .../app/components/angular-resource/.bower.json |     18 +
 .../app/components/angular-resource/README.md   |     54 +
 .../angular-resource/angular-resource.js        |    607 +
 .../angular-resource/angular-resource.min.js    |     13 +
 .../angular-resource.min.js.map                 |      8 +
 .../app/components/angular-resource/bower.json  |      8 +
 webapp/app/components/angular-route/.bower.json |     18 +
 webapp/app/components/angular-route/README.md   |     54 +
 .../components/angular-route/angular-route.js   |    927 +
 .../angular-route/angular-route.min.js          |     14 +
 .../angular-route/angular-route.min.js.map      |      8 +
 webapp/app/components/angular-route/bower.json  |      8 +
 .../components/angular-sweetalert/.bower.json   |     42 +
 .../app/components/angular-sweetalert/README.md |     27 +
 .../components/angular-sweetalert/SweetAlert.js |     47 +
 .../angular-sweetalert/SweetAlert.min.js        |      1 +
 .../components/angular-sweetalert/bower.json    |     32 +
 .../components/angular-tree-control/.bower.json |     28 +
 .../components/angular-tree-control/.gitignore  |      6 +
 .../components/angular-tree-control/.travis.yml |     10 +
 .../app/components/angular-tree-control/LICENSE |     20 +
 .../components/angular-tree-control/README.md   |    184 +
 .../angular-tree-control.js                     |    194 +
 .../components/angular-tree-control/bower.json  |     18 +
 .../css/tree-control-attribute.css              |    109 +
 .../angular-tree-control/css/tree-control.css   |    109 +
 .../demo/tree-control-attribute.html            |    105 +
 .../angular-tree-control/demo/tree-control.html |    106 +
 .../angular-tree-control/images/file.png        |    Bin 0 -> 263 bytes
 .../images/folder-closed.png                    |    Bin 0 -> 281 bytes
 .../angular-tree-control/images/folder.png      |    Bin 0 -> 289 bytes
 .../images/node-closed-2.png                    |    Bin 0 -> 2866 bytes
 .../images/node-closed-light.png                |    Bin 0 -> 2863 bytes
 .../angular-tree-control/images/node-closed.png |    Bin 0 -> 2868 bytes
 .../images/node-opened-2.png                    |    Bin 0 -> 2864 bytes
 .../images/node-opened-light.png                |    Bin 0 -> 2856 bytes
 .../angular-tree-control/images/node-opened.png |    Bin 0 -> 2864 bytes
 .../angular-tree-control/images/sample.png      |    Bin 0 -> 73892 bytes
 .../app/components/angular-ui-ace/.bower.json   |     19 +
 .../app/components/angular-ui-ace/.travis.yml   |      5 +
 .../app/components/angular-ui-ace/CHANGELOG.md  |     36 +
 webapp/app/components/angular-ui-ace/bower.json |      9 +
 webapp/app/components/angular-ui-ace/ui-ace.js  |     77 +
 .../app/components/angular-ui-ace/ui-ace.min.js |      7 +
 .../app/components/angular-unstable/.bower.json |     16 +
 .../app/components/angular-unstable/README.md   |      7 +
 .../app/components/angular-unstable/angular.js  |  16876 +
 .../components/angular-unstable/angular.min.js  |    178 +
 .../app/components/angular-unstable/bower.json  |      7 +
 webapp/app/components/angular/.bower.json       |     16 +
 webapp/app/components/angular/README.md         |     48 +
 webapp/app/components/angular/angular-csp.css   |     18 +
 webapp/app/components/angular/angular.js        |  21168 +
 webapp/app/components/angular/angular.min.js    |    207 +
 .../app/components/angular/angular.min.js.gzip  |    Bin 0 -> 38065 bytes
 .../app/components/angular/angular.min.js.map   |      8 +
 webapp/app/components/angular/bower.json        |      7 +
 .../components/angularLocalStorage/.bower.json  |     29 +
 .../app/components/angularLocalStorage/LICENSE  |     21 +
 .../components/angularLocalStorage/README.md    |     70 +
 .../components/angularLocalStorage/bower.json   |     19 +
 .../examples/assets/foundation.min.css          |      1 +
 .../examples/assets/hero.jpg                    |    Bin 0 -> 41631 bytes
 .../examples/assets/jquery-1.9.1.min.js         |      5 +
 .../examples/assets/prettify.css                |     27 +
 .../examples/assets/prettify.js                 |   1280 +
 .../examples/assets/style.css                   |     31 +
 .../examples/assets/style.less                  |     22 +
 .../angularLocalStorage/examples/index.html     |     71 +
 .../components/angularLocalStorage/package.json |     10 +
 .../src/angularLocalStorage.js                  |    159 +
 .../angularjs-nvd3-directives/.bower.json       |     42 +
 .../angularjs-nvd3-directives/Gruntfile.js      |    100 +
 .../angularjs-nvd3-directives/LICENSE           |    167 +
 .../angularjs-nvd3-directives/LICENSE.md        |     87 +
 .../angularjs-nvd3-directives/README.md         |     41 +
 .../angularjs-nvd3-directives/bower.json        |     33 +
 .../dist/angularjs-nvd3-directives.js           |   2904 +
 .../examples/bulletChart.html                   |     52 +
 .../examples/cumulativeLineChart.html           |     82 +
 .../examples/discreteBar.html                   |     50 +
 .../discreteBar.with.automatic.resize.html      |     50 +
 .../examples/discreteBar.with.event.html        |     61 +
 .../examples/historicalBarChart.html            |     40 +
 .../examples/issue.30.html                      |     46 +
 .../examples/issue.37.html                      |     60 +
 .../examples/issue.49.html                      |     94 +
 .../examples/issue51.html                       |    164 +
 .../examples/js/angular.js                      |  20311 +
 .../angularjs-nvd3-directives/examples/js/d3.js |   9275 +
 .../examples/js/moment.js                       |   2314 +
 .../examples/js/nv.d3.js                        |  14303 +
 .../examples/legendDirective.html               |     53 +
 .../examples/lineChart.d3.native.html           |     89 +
 .../examples/lineChart.html                     |     48 +
 .../examples/lineChart.tickValue.html           |     86 +
 .../examples/lineChart.ticks.html               |     63 +
 .../lineChart.with.automatic.resize.html        |     48 +
 .../examples/lineChart.with.configuration.html  |     54 +
 .../examples/lineChart.with.ngRepeat.html       |     63 +
 .../examples/linePlusBarChart.html              |     65 +
 .../linePlusBarChart.with.automatic.resize.html |     61 +
 .../examples/lineWithFocusChart.html            |     50 +
 .../examples/liveData.example.html              |     99 +
 .../examples/multiBarChart.clipping.nvd3.html   |     62 +
 .../examples/multiBarChart.clippingData.html    |     68 +
 .../examples/multiBarChart.html                 |     53 +
 .../examples/multiBarHorizontalChart.html       |     67 +
 .../mutiBarChart.with.automatic.resize.html     |     49 +
 .../examples/nvd3.callback.html                 |     57 +
 .../examples/objectEquality.html                |    114 +
 .../examples/pie.donut.chart.html               |     79 +
 .../examples/pieChart.html                      |     84 +
 .../pieChart.with.automatic.resize.html         |     85 +
 .../examples/refresh.example.html               |    153 +
 .../examples/scatterChart.html                  |     91 +
 .../scatterChart.with.automatic.resize.html     |     60 +
 .../examples/sparklineChart.html                |     49 +
 .../examples/stackedAreaChart.html              |     58 +
 .../stackedAreaChart.with.automatic.resize.html |     57 +
 .../examples/stylesheets/bootstrap.min.css      |    866 +
 .../examples/stylesheets/nv.d3.css              |    769 +
 .../examples/ticks.d3.html                      |     71 +
 .../angularjs-nvd3-directives/package.json      |     52 +
 .../src/directives/intro.js                     |      2 +
 .../src/directives/legendDirectives.js          |    239 +
 .../src/directives/nvD3AxisConfiguration.js     |    297 +
 .../src/directives/nvD3Events.js                |    159 +
 .../src/directives/nvD3LegendConfiguration.js   |     25 +
 .../src/directives/nvd3Directives.js            |   2180 +
 .../src/directives/outro.js                     |      1 +
 .../app/components/animate.css/.animate-config  |     97 +
 webapp/app/components/animate.css/.bower.json   |     15 +
 webapp/app/components/animate.css/.gitignore    |      3 +
 webapp/app/components/animate.css/Gruntfile.js  |     83 +
 webapp/app/components/animate.css/README.md     |     65 +
 webapp/app/components/animate.css/animate.css   |   2738 +
 .../app/components/animate.css/animate.min.css  |      1 +
 webapp/app/components/animate.css/bower.json    |      5 +
 webapp/app/components/animate.css/package.json  |     12 +
 .../app/components/animate.css/source/_base.css |     22 +
 .../source/attention_seekers/bounce.css         |      9 +
 .../source/attention_seekers/flash.css          |      8 +
 .../source/attention_seekers/pulse.css          |     11 +
 .../source/attention_seekers/shake.css          |      9 +
 .../source/attention_seekers/swing.css          |     12 +
 .../source/attention_seekers/tada.css           |     11 +
 .../source/attention_seekers/wobble.css         |     15 +
 .../source/bouncing_entrances/bounceIn.css      |     23 +
 .../source/bouncing_entrances/bounceInDown.css  |     23 +
 .../source/bouncing_entrances/bounceInLeft.css  |     23 +
 .../source/bouncing_entrances/bounceInRight.css |     23 +
 .../source/bouncing_entrances/bounceInUp.css    |     23 +
 .../source/bouncing_exits/bounceOut.css         |     23 +
 .../source/bouncing_exits/bounceOutDown.css     |     19 +
 .../source/bouncing_exits/bounceOutLeft.css     |     19 +
 .../source/bouncing_exits/bounceOutRight.css    |     19 +
 .../source/bouncing_exits/bounceOutUp.css       |     19 +
 .../source/fading_entrances/fadeIn.css          |      8 +
 .../source/fading_entrances/fadeInDown.css      |     15 +
 .../source/fading_entrances/fadeInDownBig.css   |     15 +
 .../source/fading_entrances/fadeInLeft.css      |     15 +
 .../source/fading_entrances/fadeInLeftBig.css   |     15 +
 .../source/fading_entrances/fadeInRight.css     |     15 +
 .../source/fading_entrances/fadeInRightBig.css  |     15 +
 .../source/fading_entrances/fadeInUp.css        |     15 +
 .../source/fading_entrances/fadeInUpBig.css     |     15 +
 .../animate.css/source/fading_exits/fadeOut.css |      8 +
 .../source/fading_exits/fadeOutDown.css         |     15 +
 .../source/fading_exits/fadeOutDownBig.css      |     15 +
 .../source/fading_exits/fadeOutLeft.css         |     15 +
 .../source/fading_exits/fadeOutLeftBig.css      |     15 +
 .../source/fading_exits/fadeOutRight.css        |     15 +
 .../source/fading_exits/fadeOutRightBig.css     |     15 +
 .../source/fading_exits/fadeOutUp.css           |     15 +
 .../source/fading_exits/fadeOutUpBig.css        |     15 +
 .../animate.css/source/flippers/flip.css        |     27 +
 .../animate.css/source/flippers/flipInX.css     |     24 +
 .../animate.css/source/flippers/flipInY.css     |     24 +
 .../animate.css/source/flippers/flipOutX.css    |     15 +
 .../animate.css/source/flippers/flipOutY.css    |     15 +
 .../source/lightspeed/lightSpeedIn.css          |     11 +
 .../source/lightspeed/lightSpeedOut.css         |     10 +
 .../source/rotating_entrances/rotateIn.css      |     17 +
 .../rotating_entrances/rotateInDownLeft.css     |     17 +
 .../rotating_entrances/rotateInDownRight.css    |     17 +
 .../rotating_entrances/rotateInUpLeft.css       |     17 +
 .../rotating_entrances/rotateInUpRight.css      |     17 +
 .../source/rotating_exits/rotateOut.css         |     17 +
 .../source/rotating_exits/rotateOutDownLeft.css |     17 +
 .../rotating_exits/rotateOutDownRight.css       |     17 +
 .../source/rotating_exits/rotateOutUpLeft.css   |     17 +
 .../source/rotating_exits/rotateOutUpRight.css  |     17 +
 .../animate.css/source/sliders/slideInDown.css  |     14 +
 .../animate.css/source/sliders/slideInLeft.css  |     14 +
 .../animate.css/source/sliders/slideInRight.css |     14 +
 .../animate.css/source/sliders/slideOutLeft.css |     14 +
 .../source/sliders/slideOutRight.css            |     14 +
 .../animate.css/source/sliders/slideOutUp.css   |     14 +
 .../animate.css/source/specials/hinge.css       |     11 +
 .../animate.css/source/specials/rollIn.css      |     10 +
 .../animate.css/source/specials/rollOut.css     |     17 +
 .../components/bootstrap-sweetalert/.bower.json |     35 +
 .../bootstrap-sweetalert/.editorconfig          |     11 +
 .../components/bootstrap-sweetalert/.gitignore  |      3 +
 .../bootstrap-sweetalert/Gruntfile.coffee       |     35 +
 .../app/components/bootstrap-sweetalert/LICENSE |     22 +
 .../components/bootstrap-sweetalert/README.md   |     31 +
 .../assets/bootstrap-sweetalert.png             |    Bin 0 -> 52320 bytes
 .../bootstrap-sweetalert/assets/docs.css        |    123 +
 .../bootstrap-sweetalert/assets/thumbs-up.jpg   |    Bin 0 -> 30052 bytes
 .../bootstrap-sweetalert/bootstrap/mixins.less  |     13 +
 .../bootstrap/variables.less                    |     42 +
 .../components/bootstrap-sweetalert/bower.json  |     24 +
 .../components/bootstrap-sweetalert/index.html  |    308 +
 .../lib/sweet-alert-animations.less             |    255 +
 .../lib/sweet-alert-combine.less                |      3 +
 .../bootstrap-sweetalert/lib/sweet-alert.css    |    564 +
 .../bootstrap-sweetalert/lib/sweet-alert.html   |     39 +
 .../bootstrap-sweetalert/lib/sweet-alert.js     |    706 +
 .../bootstrap-sweetalert/lib/sweet-alert.less   |    254 +
 .../bootstrap-sweetalert/lib/sweet-alert.min.js |      1 +
 .../bootstrap-sweetalert/package.json           |     29 +
 webapp/app/components/bootstrap/.bower.json     |     24 +
 webapp/app/components/bootstrap/CNAME           |      1 +
 webapp/app/components/bootstrap/CONTRIBUTING.md |     66 +
 webapp/app/components/bootstrap/Gruntfile.js    |    217 +
 webapp/app/components/bootstrap/LICENSE         |    176 +
 webapp/app/components/bootstrap/README.md       |    141 +
 webapp/app/components/bootstrap/_config.yml     |     23 +
 .../app/components/bootstrap/_includes/ads.html |      1 +
 .../components/bootstrap/_includes/footer.html  |     34 +
 .../components/bootstrap/_includes/header.html  |     43 +
 .../bootstrap/_includes/nav-components.html     |    137 +
 .../components/bootstrap/_includes/nav-css.html |     87 +
 .../bootstrap/_includes/nav-customize.html      |     40 +
 .../_includes/nav-getting-started.html          |     44 +
 .../bootstrap/_includes/nav-javascript.html     |     88 +
 .../bootstrap/_includes/nav-main.html           |     32 +
 .../bootstrap/_includes/old-bs-docs.html        |      8 +
 .../bootstrap/_includes/social-buttons.html     |     16 +
 .../components/bootstrap/_layouts/default.html  |     76 +
 .../app/components/bootstrap/_layouts/home.html |     46 +
 .../components/bootstrap/assets/css/docs.css    |   1084 +
 .../bootstrap/assets/css/pygments-manni.css     |     66 +
 .../ico/apple-touch-icon-114-precomposed.png    |    Bin 0 -> 2939 bytes
 .../ico/apple-touch-icon-144-precomposed.png    |    Bin 0 -> 3614 bytes
 .../ico/apple-touch-icon-57-precomposed.png     |    Bin 0 -> 1577 bytes
 .../ico/apple-touch-icon-72-precomposed.png     |    Bin 0 -> 1932 bytes
 .../components/bootstrap/assets/ico/favicon.png |    Bin 0 -> 889 bytes
 .../bootstrap/assets/js/application.js          |     83 +
 .../bootstrap/assets/js/customizer.js           |    290 +
 .../components/bootstrap/assets/js/filesaver.js |    169 +
 .../components/bootstrap/assets/js/holder.js    |    419 +
 .../components/bootstrap/assets/js/html5shiv.js |      8 +
 .../components/bootstrap/assets/js/jquery.js    |      6 +
 .../app/components/bootstrap/assets/js/jszip.js |   1425 +
 .../app/components/bootstrap/assets/js/less.js  |      9 +
 .../components/bootstrap/assets/js/raw-files.js |      3 +
 .../bootstrap/assets/js/respond.min.js          |      6 +
 .../components/bootstrap/assets/js/uglify.js    |     14 +
 webapp/app/components/bootstrap/bower.json      |     11 +
 .../app/components/bootstrap/browserstack.json  |     37 +
 webapp/app/components/bootstrap/components.html |   2978 +
 webapp/app/components/bootstrap/composer.json   |     20 +
 webapp/app/components/bootstrap/css.html        |   2441 +
 webapp/app/components/bootstrap/customize.html  |   1446 +
 .../bootstrap/dist/css/bootstrap-theme.css      |    384 +
 .../bootstrap/dist/css/bootstrap-theme.min.css  |      1 +
 .../components/bootstrap/dist/css/bootstrap.css |   6805 +
 .../bootstrap/dist/css/bootstrap.min.css        |      9 +
 .../bootstrap/dist/css/bootswatch.min.css       |      7 +
 .../dist/fonts/glyphicons-halflings-regular.eot |    Bin 0 -> 14079 bytes
 .../dist/fonts/glyphicons-halflings-regular.svg |    228 +
 .../dist/fonts/glyphicons-halflings-regular.ttf |    Bin 0 -> 29512 bytes
 .../fonts/glyphicons-halflings-regular.woff     |    Bin 0 -> 16448 bytes
 .../components/bootstrap/dist/js/bootstrap.js   |   1999 +
 .../bootstrap/dist/js/bootstrap.min.js          |      6 +
 .../bootstrap/examples/carousel/carousel.css    |    128 +
 .../bootstrap/examples/carousel/index.html      |    201 +
 .../components/bootstrap/examples/grid/grid.css |     28 +
 .../bootstrap/examples/grid/index.html          |    119 +
 .../examples/jumbotron-narrow/index.html        |     78 +
 .../jumbotron-narrow/jumbotron-narrow.css       |     79 +
 .../bootstrap/examples/jumbotron/index.html     |    111 +
 .../bootstrap/examples/jumbotron/jumbotron.css  |      5 +
 .../bootstrap/examples/justified-nav/index.html |     79 +
 .../examples/justified-nav/justified-nav.css    |     88 +
 .../examples/navbar-fixed-top/index.html        |     86 +
 .../navbar-fixed-top/navbar-fixed-top.css       |      4 +
 .../examples/navbar-static-top/index.html       |     87 +
 .../navbar-static-top/navbar-static-top.css     |      7 +
 .../bootstrap/examples/navbar/index.html        |     83 +
 .../bootstrap/examples/navbar/navbar.css        |      7 +
 .../examples/non-responsive/index.html          |     96 +
 .../examples/non-responsive/non-responsive.css  |    117 +
 .../bootstrap/examples/offcanvas/index.html     |    127 +
 .../bootstrap/examples/offcanvas/offcanvas.css  |     48 +
 .../bootstrap/examples/offcanvas/offcanvas.js   |      5 +
 .../bootstrap/examples/screenshots/carousel.jpg |    Bin 0 -> 82379 bytes
 .../bootstrap/examples/screenshots/grid.jpg     |    Bin 0 -> 100105 bytes
 .../examples/screenshots/jumbotron-narrow.jpg   |    Bin 0 -> 68675 bytes
 .../examples/screenshots/jumbotron.jpg          |    Bin 0 -> 77911 bytes
 .../examples/screenshots/justified-nav.jpg      |    Bin 0 -> 79170 bytes
 .../examples/screenshots/navbar-fixed.jpg       |    Bin 0 -> 38093 bytes
 .../examples/screenshots/navbar-static.jpg      |    Bin 0 -> 39712 bytes
 .../bootstrap/examples/screenshots/navbar.jpg   |    Bin 0 -> 41029 bytes
 .../examples/screenshots/non-responsive.jpg     |    Bin 0 -> 61233 bytes
 .../examples/screenshots/offcanvas.jpg          |    Bin 0 -> 112583 bytes
 .../bootstrap/examples/screenshots/sign-in.jpg  |    Bin 0 -> 13648 bytes
 .../examples/screenshots/starter-template.jpg   |    Bin 0 -> 22620 bytes
 .../screenshots/sticky-footer-navbar.jpg        |    Bin 0 -> 39387 bytes
 .../examples/screenshots/sticky-footer.jpg      |    Bin 0 -> 25056 bytes
 .../bootstrap/examples/screenshots/theme.jpg    |    Bin 0 -> 95407 bytes
 .../bootstrap/examples/signin/index.html        |     46 +
 .../bootstrap/examples/signin/signin.css        |     40 +
 .../examples/starter-template/index.html        |     63 +
 .../starter-template/starter-template.css       |      7 +
 .../examples/sticky-footer-navbar/index.html    |     86 +
 .../sticky-footer-navbar.css                    |     46 +
 .../bootstrap/examples/sticky-footer/index.html |     51 +
 .../examples/sticky-footer/sticky-footer.css    |     39 +
 .../bootstrap/examples/theme/index.html         |    387 +
 .../bootstrap/examples/theme/theme.css          |     14 +
 .../fonts/glyphicons-halflings-regular.eot      |    Bin 0 -> 14079 bytes
 .../fonts/glyphicons-halflings-regular.svg      |    228 +
 .../fonts/glyphicons-halflings-regular.ttf      |    Bin 0 -> 29512 bytes
 .../fonts/glyphicons-halflings-regular.woff     |    Bin 0 -> 16448 bytes
 .../components/bootstrap/getting-started.html   |    866 +
 webapp/app/components/bootstrap/index.html      |     15 +
 webapp/app/components/bootstrap/javascript.html |   1912 +
 webapp/app/components/bootstrap/js/affix.js     |    126 +
 webapp/app/components/bootstrap/js/alert.js     |     98 +
 webapp/app/components/bootstrap/js/button.js    |    109 +
 webapp/app/components/bootstrap/js/carousel.js  |    217 +
 webapp/app/components/bootstrap/js/collapse.js  |    179 +
 webapp/app/components/bootstrap/js/dropdown.js  |    154 +
 webapp/app/components/bootstrap/js/modal.js     |    246 +
 webapp/app/components/bootstrap/js/popover.js   |    117 +
 webapp/app/components/bootstrap/js/scrollspy.js |    158 +
 webapp/app/components/bootstrap/js/tab.js       |    135 +
 .../components/bootstrap/js/tests/index.html    |     52 +
 .../components/bootstrap/js/tests/phantom.js    |     63 +
 .../app/components/bootstrap/js/tests/server.js |     14 +
 .../components/bootstrap/js/tests/unit/affix.js |     25 +
 .../components/bootstrap/js/tests/unit/alert.js |     62 +
 .../bootstrap/js/tests/unit/button.js           |    116 +
 .../bootstrap/js/tests/unit/carousel.js         |     87 +
 .../bootstrap/js/tests/unit/collapse.js         |    164 +
 .../bootstrap/js/tests/unit/dropdown.js         |    219 +
 .../components/bootstrap/js/tests/unit/modal.js |    196 +
 .../bootstrap/js/tests/unit/phantom.js          |     69 +
 .../bootstrap/js/tests/unit/popover.js          |    133 +
 .../bootstrap/js/tests/unit/scrollspy.js        |     37 +
 .../components/bootstrap/js/tests/unit/tab.js   |     86 +
 .../bootstrap/js/tests/unit/tooltip.js          |    437 +
 .../bootstrap/js/tests/unit/transition.js       |     13 +
 .../bootstrap/js/tests/vendor/jquery.js         |      6 +
 .../bootstrap/js/tests/vendor/qunit.css         |    232 +
 .../bootstrap/js/tests/vendor/qunit.js          |   1510 +
 webapp/app/components/bootstrap/js/tooltip.js   |    386 +
 .../app/components/bootstrap/js/transition.js   |     56 +
 .../app/components/bootstrap/less/alerts.less   |     67 +
 .../app/components/bootstrap/less/badges.less   |     51 +
 .../components/bootstrap/less/bootstrap.less    |     59 +
 .../components/bootstrap/less/breadcrumbs.less  |     23 +
 .../bootstrap/less/button-groups.less           |    248 +
 .../app/components/bootstrap/less/buttons.less  |    160 +
 .../app/components/bootstrap/less/carousel.less |    209 +
 webapp/app/components/bootstrap/less/close.less |     33 +
 webapp/app/components/bootstrap/less/code.less  |     56 +
 .../bootstrap/less/component-animations.less    |     29 +
 .../components/bootstrap/less/dropdowns.less    |    193 +
 webapp/app/components/bootstrap/less/forms.less |    353 +
 .../components/bootstrap/less/glyphicons.less   |    232 +
 webapp/app/components/bootstrap/less/grid.less  |    346 +
 .../components/bootstrap/less/input-groups.less |    127 +
 .../components/bootstrap/less/jumbotron.less    |     40 +
 .../app/components/bootstrap/less/labels.less   |     58 +
 .../components/bootstrap/less/list-group.less   |     88 +
 webapp/app/components/bootstrap/less/media.less |     56 +
 .../app/components/bootstrap/less/mixins.less   |    723 +
 .../app/components/bootstrap/less/modals.less   |    141 +
 .../app/components/bootstrap/less/navbar.less   |    621 +
 webapp/app/components/bootstrap/less/navs.less  |    229 +
 .../components/bootstrap/less/normalize.less    |    396 +
 webapp/app/components/bootstrap/less/pager.less |     55 +
 .../components/bootstrap/less/pagination.less   |     83 +
 .../app/components/bootstrap/less/panels.less   |    148 +
 .../app/components/bootstrap/less/popovers.less |    133 +
 webapp/app/components/bootstrap/less/print.less |    100 +
 .../bootstrap/less/progress-bars.less           |     95 +
 .../bootstrap/less/responsive-utilities.less    |    220 +
 .../components/bootstrap/less/scaffolding.less  |    130 +
 .../app/components/bootstrap/less/tables.less   |    236 +
 webapp/app/components/bootstrap/less/theme.less |    232 +
 .../components/bootstrap/less/thumbnails.less   |     31 +
 .../app/components/bootstrap/less/tooltip.less  |     95 +
 webapp/app/components/bootstrap/less/type.less  |    238 +
 .../components/bootstrap/less/utilities.less    |     42 +
 .../components/bootstrap/less/variables.less    |    620 +
 webapp/app/components/bootstrap/less/wells.less |     29 +
 webapp/app/components/bootstrap/package.json    |     34 +
 webapp/app/components/chosen/.bower.json        |     14 +
 webapp/app/components/chosen/chosen-sprite.png  |    Bin 0 -> 646 bytes
 .../app/components/chosen/chosen-sprite@2x.png  |    Bin 0 -> 872 bytes
 webapp/app/components/chosen/chosen.css         |    430 +
 webapp/app/components/chosen/chosen.jquery.js   |   1166 +
 .../app/components/chosen/chosen.jquery.min.js  |      2 +
 webapp/app/components/chosen/chosen.min.css     |      3 +
 webapp/app/components/chosen/chosen.proto.js    |   1189 +
 .../app/components/chosen/chosen.proto.min.js   |      2 +
 .../app/components/chosen/docsupport/chosen.png |    Bin 0 -> 3467 bytes
 .../components/chosen/docsupport/oss-credit.png |    Bin 0 -> 6709 bytes
 .../app/components/chosen/docsupport/prism.css  |    108 +
 .../app/components/chosen/docsupport/prism.js   |      9 +
 .../app/components/chosen/docsupport/style.css  |    201 +
 webapp/app/components/chosen/index.html         |   1479 +
 webapp/app/components/chosen/index.proto.html   |   1485 +
 webapp/app/components/chosen/options.html       |    229 +
 webapp/app/components/d3/.bower.json            |     34 +
 webapp/app/components/d3/CONTRIBUTING.md        |     25 +
 webapp/app/components/d3/LICENSE                |     26 +
 webapp/app/components/d3/README.md              |      9 +
 webapp/app/components/d3/bower.json             |     24 +
 webapp/app/components/d3/composer.json          |     19 +
 webapp/app/components/d3/d3.js                  |   9294 +
 webapp/app/components/d3/d3.min.js              |      5 +
 .../font-awesome/css/font-awesome.css           |   1672 +
 .../font-awesome/css/font-awesome.min.css       |      4 +
 .../font-awesome/fonts/FontAwesome.otf          |    Bin 0 -> 85908 bytes
 .../font-awesome/fonts/fontawesome-webfont.eot  |    Bin 0 -> 56006 bytes
 .../font-awesome/fonts/fontawesome-webfont.svg  |    520 +
 .../font-awesome/fonts/fontawesome-webfont.ttf  |    Bin 0 -> 112160 bytes
 .../font-awesome/fonts/fontawesome-webfont.woff |    Bin 0 -> 65452 bytes
 .../font-awesome/less/bordered-pulled.less      |     16 +
 .../app/components/font-awesome/less/core.less  |     11 +
 .../font-awesome/less/fixed-width.less          |      6 +
 .../font-awesome/less/font-awesome.less         |     17 +
 .../app/components/font-awesome/less/icons.less |    552 +
 .../components/font-awesome/less/larger.less    |     13 +
 .../app/components/font-awesome/less/list.less  |     19 +
 .../components/font-awesome/less/mixins.less    |     25 +
 .../app/components/font-awesome/less/path.less  |     14 +
 .../font-awesome/less/rotated-flipped.less      |     20 +
 .../components/font-awesome/less/spinning.less  |     29 +
 .../components/font-awesome/less/stacked.less   |     20 +
 .../components/font-awesome/less/variables.less |    561 +
 .../font-awesome/scss/_bordered-pulled.scss     |     16 +
 .../app/components/font-awesome/scss/_core.scss |     11 +
 .../font-awesome/scss/_fixed-width.scss         |      6 +
 .../components/font-awesome/scss/_icons.scss    |    552 +
 .../components/font-awesome/scss/_larger.scss   |     13 +
 .../app/components/font-awesome/scss/_list.scss |     19 +
 .../components/font-awesome/scss/_mixins.scss   |     25 +
 .../app/components/font-awesome/scss/_path.scss |     14 +
 .../font-awesome/scss/_rotated-flipped.scss     |     20 +
 .../components/font-awesome/scss/_spinning.scss |     29 +
 .../components/font-awesome/scss/_stacked.scss  |     20 +
 .../font-awesome/scss/_variables.scss           |    561 +
 .../font-awesome/scss/font-awesome.scss         |     17 +
 .../fuelux/data/fuelux.tree-sample-demo-data.js |    178 +
 .../app/components/fuelux/fuelux.spinner.min.js |      8 +
 webapp/app/components/fuelux/fuelux.tree.min.js |      8 +
 .../app/components/fuelux/fuelux.wizard.min.js  |      8 +
 webapp/app/components/jquery/.bower.json        |     37 +
 webapp/app/components/jquery/MIT-LICENSE.txt    |     21 +
 webapp/app/components/jquery/bower.json         |     27 +
 webapp/app/components/jquery/dist/jquery.js     |  10308 +
 webapp/app/components/jquery/dist/jquery.min.js |      5 +
 .../app/components/jquery/dist/jquery.min.map   |      1 +
 webapp/app/components/jquery/src/ajax.js        |    807 +
 webapp/app/components/jquery/src/ajax/jsonp.js  |     89 +
 webapp/app/components/jquery/src/ajax/load.js   |     75 +
 .../app/components/jquery/src/ajax/parseJSON.js |     51 +
 .../app/components/jquery/src/ajax/parseXML.js  |     31 +
 webapp/app/components/jquery/src/ajax/script.js |     93 +
 .../app/components/jquery/src/ajax/var/nonce.js |      5 +
 .../components/jquery/src/ajax/var/rquery.js    |      3 +
 webapp/app/components/jquery/src/ajax/xhr.js    |    196 +
 webapp/app/components/jquery/src/attributes.js  |     11 +
 .../components/jquery/src/attributes/attr.js    |    271 +
 .../components/jquery/src/attributes/classes.js |    157 +
 .../components/jquery/src/attributes/prop.js    |    134 +
 .../components/jquery/src/attributes/support.js |     62 +
 .../app/components/jquery/src/attributes/val.js |    178 +
 webapp/app/components/jquery/src/callbacks.js   |    205 +
 webapp/app/components/jquery/src/core.js        |    534 +
 webapp/app/components/jquery/src/core/access.js |     60 +
 webapp/app/components/jquery/src/core/init.js   |    132 +
 .../app/components/jquery/src/core/parseHTML.js |     39 +
 webapp/app/components/jquery/src/core/ready.js  |    152 +
 .../jquery/src/core/var/rsingleTag.js           |      4 +
 webapp/app/components/jquery/src/css.js         |    504 +
 .../components/jquery/src/css/addGetHookIf.js   |     32 +
 webapp/app/components/jquery/src/css/curCSS.js  |    117 +
 .../components/jquery/src/css/defaultDisplay.js |     69 +
 .../jquery/src/css/hiddenVisibleSelectors.js    |     20 +
 webapp/app/components/jquery/src/css/support.js |    149 +
 webapp/app/components/jquery/src/css/swap.js    |     28 +
 .../components/jquery/src/css/var/cssExpand.js  |      3 +
 .../components/jquery/src/css/var/isHidden.js   |     13 +
 .../components/jquery/src/css/var/rmargin.js    |      3 +
 .../components/jquery/src/css/var/rnumnonpx.js  |      5 +
 webapp/app/components/jquery/src/data.js        |    335 +
 .../app/components/jquery/src/data/accepts.js   |     21 +
 .../app/components/jquery/src/data/support.js   |     25 +
 webapp/app/components/jquery/src/deferred.js    |    150 +
 webapp/app/components/jquery/src/deprecated.js  |     13 +
 webapp/app/components/jquery/src/dimensions.js  |     50 +
 webapp/app/components/jquery/src/effects.js     |    656 +
 .../app/components/jquery/src/effects/Tween.js  |    114 +
 .../jquery/src/effects/animatedSelector.js      |     13 +
 .../components/jquery/src/effects/support.js    |     55 +
 webapp/app/components/jquery/src/event.js       |   1037 +
 webapp/app/components/jquery/src/event/alias.js |     39 +
 .../app/components/jquery/src/event/support.js  |     26 +
 webapp/app/components/jquery/src/exports/amd.js |     24 +
 .../app/components/jquery/src/exports/global.js |     32 +
 webapp/app/components/jquery/src/intro.js       |     44 +
 webapp/app/components/jquery/src/jquery.js      |     37 +
 .../app/components/jquery/src/manipulation.js   |    744 +
 .../jquery/src/manipulation/_evalUrl.js         |     18 +
 .../jquery/src/manipulation/support.js          |     76 +
 .../src/manipulation/var/rcheckableType.js      |      3 +
 webapp/app/components/jquery/src/offset.js      |    211 +
 webapp/app/components/jquery/src/outro.js       |      1 +
 webapp/app/components/jquery/src/queue.js       |    142 +
 webapp/app/components/jquery/src/queue/delay.js |     22 +
 .../components/jquery/src/selector-sizzle.js    |     14 +
 webapp/app/components/jquery/src/selector.js    |      1 +
 webapp/app/components/jquery/src/serialize.js   |    110 +
 .../components/jquery/src/sizzle/dist/sizzle.js |   2044 +
 .../jquery/src/sizzle/dist/sizzle.min.js        |      3 +
 .../jquery/src/sizzle/dist/sizzle.min.map       |      1 +
 webapp/app/components/jquery/src/support.js     |     58 +
 webapp/app/components/jquery/src/traversing.js  |    200 +
 .../jquery/src/traversing/findFilter.js         |    100 +
 .../jquery/src/traversing/var/rneedsContext.js  |      6 +
 .../app/components/jquery/src/var/class2type.js |      4 +
 webapp/app/components/jquery/src/var/concat.js  |      5 +
 .../app/components/jquery/src/var/deletedIds.js |      3 +
 webapp/app/components/jquery/src/var/hasOwn.js  |      5 +
 webapp/app/components/jquery/src/var/indexOf.js |      5 +
 webapp/app/components/jquery/src/var/pnum.js    |      3 +
 webapp/app/components/jquery/src/var/push.js    |      5 +
 .../app/components/jquery/src/var/rnotwhite.js  |      3 +
 webapp/app/components/jquery/src/var/slice.js   |      5 +
 .../components/jquery/src/var/strundefined.js   |      3 +
 webapp/app/components/jquery/src/var/support.js |      4 +
 .../app/components/jquery/src/var/toString.js   |      5 +
 webapp/app/components/jquery/src/wrap.js        |     75 +
 webapp/app/components/less.js/.bower.json       |     14 +
 webapp/app/components/less.js/.gitattributes    |      9 +
 webapp/app/components/less.js/.gitignore        |      8 +
 webapp/app/components/less.js/.npmignore        |      1 +
 webapp/app/components/less.js/CHANGELOG.md      |    186 +
 webapp/app/components/less.js/CONTRIBUTING.md   |     49 +
 webapp/app/components/less.js/LICENSE           |    179 +
 webapp/app/components/less.js/Makefile          |    102 +
 webapp/app/components/less.js/README.md         |     20 +
 .../components/less.js/benchmark/benchmark.less |   3979 +
 .../less.js/benchmark/less-benchmark.js         |     47 +
 webapp/app/components/less.js/build/amd.js      |      6 +
 webapp/app/components/less.js/build/header.js   |      9 +
 .../components/less.js/build/require-rhino.js   |      7 +
 webapp/app/components/less.js/build/require.js  |      7 +
 .../app/components/less.js/dist/less-1.1.0.js   |   2695 +
 .../components/less.js/dist/less-1.1.0.min.js   |     16 +
 .../app/components/less.js/dist/less-1.1.1.js   |   2710 +
 .../components/less.js/dist/less-1.1.1.min.js   |     16 +
 .../app/components/less.js/dist/less-1.1.2.js   |   2712 +
 .../components/less.js/dist/less-1.1.2.min.js   |     16 +
 .../app/components/less.js/dist/less-1.1.3.js   |   2721 +
 .../components/less.js/dist/less-1.1.3.min.js   |     16 +
 .../app/components/less.js/dist/less-1.1.4.js   |   2769 +
 .../components/less.js/dist/less-1.1.4.min.js   |     16 +
 .../app/components/less.js/dist/less-1.1.5.js   |   2805 +
 .../components/less.js/dist/less-1.1.5.min.js   |      9 +
 .../app/components/less.js/dist/less-1.1.6.js   |   3004 +
 .../components/less.js/dist/less-1.1.6.min.js   |      9 +
 .../app/components/less.js/dist/less-1.2.0.js   |   3293 +
 .../components/less.js/dist/less-1.2.0.min.js   |      9 +
 .../app/components/less.js/dist/less-1.2.1.js   |   3318 +
 .../components/less.js/dist/less-1.2.1.min.js   |      9 +
 .../app/components/less.js/dist/less-1.2.2.js   |   3337 +
 .../components/less.js/dist/less-1.2.2.min.js   |      9 +
 .../app/components/less.js/dist/less-1.3.0.js   |   3478 +
 .../components/less.js/dist/less-1.3.0.min.js   |      9 +
 .../app/components/less.js/dist/less-1.3.1.js   |   4011 +
 .../components/less.js/dist/less-1.3.1.min.js   |      9 +
 .../app/components/less.js/dist/less-1.3.2.js   |   4401 +
 .../components/less.js/dist/less-1.3.2.min.js   |      9 +
 .../app/components/less.js/dist/less-1.3.3.js   |   4413 +
 .../components/less.js/dist/less-1.3.3.min.js   |      9 +
 .../components/less.js/dist/less-1.4.0-beta.js  |   5830 +
 .../less.js/dist/less-1.4.0-beta.min.js         |     11 +
 .../app/components/less.js/dist/less-1.4.0.js   |   5830 +
 .../components/less.js/dist/less-1.4.0.min.js   |     11 +
 .../app/components/less.js/dist/less-1.4.1.js   |   5837 +
 .../components/less.js/dist/less-1.4.1.min.js   |     11 +
 .../app/components/less.js/dist/less-1.4.2.js   |   5837 +
 .../components/less.js/dist/less-1.4.2.min.js   |     11 +
 .../components/less.js/dist/less-rhino-1.1.3.js |   2460 +
 .../components/less.js/dist/less-rhino-1.1.5.js |   2481 +
 .../components/less.js/dist/less-rhino-1.3.1.js |   3725 +
 .../components/less.js/dist/less-rhino-1.3.2.js |   3990 +
 .../components/less.js/dist/less-rhino-1.3.3.js |   4002 +
 .../components/less.js/dist/less-rhino-1.4.0.js |   4273 +
 .../app/components/less.js/lib/less/browser.js  |    549 +
 .../app/components/less.js/lib/less/colors.js   |    152 +
 webapp/app/components/less.js/lib/less/env.js   |    105 +
 .../less.js/lib/less/extend-visitor.js          |    391 +
 .../components/less.js/lib/less/functions.js    |    532 +
 .../less.js/lib/less/import-visitor.js          |    107 +
 webapp/app/components/less.js/lib/less/index.js |    223 +
 .../less.js/lib/less/join-selector-visitor.js   |     37 +
 .../components/less.js/lib/less/lessc_helper.js |     71 +
 .../app/components/less.js/lib/less/parser.js   |   1611 +
 webapp/app/components/less.js/lib/less/rhino.js |    126 +
 webapp/app/components/less.js/lib/less/tree.js  |     45 +
 .../components/less.js/lib/less/tree/alpha.js   |     21 +
 .../less.js/lib/less/tree/anonymous.js          |     28 +
 .../less.js/lib/less/tree/assignment.js         |     23 +
 .../components/less.js/lib/less/tree/call.js    |     60 +
 .../components/less.js/lib/less/tree/color.js   |    156 +
 .../components/less.js/lib/less/tree/comment.js |     15 +
 .../less.js/lib/less/tree/condition.js          |     49 +
 .../less.js/lib/less/tree/dimension.js          |    305 +
 .../less.js/lib/less/tree/directive.js          |     44 +
 .../components/less.js/lib/less/tree/element.js |     85 +
 .../less.js/lib/less/tree/expression.js         |     48 +
 .../components/less.js/lib/less/tree/extend.js  |     43 +
 .../components/less.js/lib/less/tree/import.js  |    103 +
 .../less.js/lib/less/tree/javascript.js         |     52 +
 .../components/less.js/lib/less/tree/keyword.js |     20 +
 .../components/less.js/lib/less/tree/media.js   |    137 +
 .../components/less.js/lib/less/tree/mixin.js   |    232 +
 .../less.js/lib/less/tree/negative.js           |     22 +
 .../less.js/lib/less/tree/operation.js          |     52 +
 .../components/less.js/lib/less/tree/paren.js   |     20 +
 .../components/less.js/lib/less/tree/quoted.js  |     45 +
 .../components/less.js/lib/less/tree/rule.js    |     62 +
 .../components/less.js/lib/less/tree/ruleset.js |    441 +
 .../less.js/lib/less/tree/selector.js           |     62 +
 .../less.js/lib/less/tree/unicode-descriptor.js |     14 +
 .../app/components/less.js/lib/less/tree/url.js |     31 +
 .../components/less.js/lib/less/tree/value.js   |     27 +
 .../less.js/lib/less/tree/variable.js           |     39 +
 .../app/components/less.js/lib/less/visitor.js  |     54 +
 webapp/app/components/less.js/package.json      |     73 +
 .../less.js/test/browser-test-prepare.js        |     46 +
 .../components/less.js/test/browser/common.js   |    126 +
 .../test/browser/css/relative-urls/urls.css     |     36 +
 .../test/browser/css/rootpath-relative/urls.css |     36 +
 .../less.js/test/browser/css/rootpath/urls.css  |     36 +
 .../less.js/test/browser/css/urls.css           |     49 +
 .../less.js/test/browser/jasmine-html.js        |    681 +
 .../components/less.js/test/browser/jasmine.css |     82 +
 .../components/less.js/test/browser/jasmine.js  |   2600 +
 .../less.js/test/browser/less/imports/urls.less |      4 +
 .../test/browser/less/imports/urls2.less        |      4 +
 .../test/browser/less/relative-urls/urls.less   |     33 +
 .../browser/less/rootpath-relative/urls.less    |     33 +
 .../test/browser/less/rootpath/urls.less        |     33 +
 .../less.js/test/browser/less/urls.less         |     49 +
 .../less.js/test/browser/phantom-runner.js      |    141 +
 .../less.js/test/browser/runner-browser.js      |      7 +
 .../less.js/test/browser/runner-errors.js       |      5 +
 .../less.js/test/browser/runner-legacy.js       |      6 +
 .../less.js/test/browser/runner-main.js         |     15 +
 .../less.js/test/browser/runner-production.js   |      7 +
 .../test/browser/runner-relative-urls.js        |      4 +
 .../test/browser/runner-rootpath-relative.js    |      5 +
 .../less.js/test/browser/runner-rootpath.js     |      4 +
 .../less.js/test/browser/template.htm           |     10 +
 .../components/less.js/test/css/charsets.css    |      1 +
 .../app/components/less.js/test/css/colors.css  |     80 +
 .../components/less.js/test/css/comments.css    |     64 +
 .../test/css/compression/compression.css        |      2 +
 .../app/components/less.js/test/css/css-3.css   |    117 +
 .../components/less.js/test/css/css-escapes.css |     24 +
 webapp/app/components/less.js/test/css/css.css  |     95 +
 .../less.js/test/css/debug/linenumbers-all.css  |     43 +
 .../test/css/debug/linenumbers-comments.css     |     35 +
 .../test/css/debug/linenumbers-mediaquery.css   |     35 +
 .../less.js/test/css/extend-chaining.css        |     72 +
 .../less.js/test/css/extend-clearfix.css        |     19 +
 .../less.js/test/css/extend-exact.css           |     37 +
 .../less.js/test/css/extend-media.css           |     24 +
 .../components/less.js/test/css/extend-nest.css |     57 +
 .../less.js/test/css/extend-selector.css        |     72 +
 .../app/components/less.js/test/css/extend.css  |     76 +
 .../components/less.js/test/css/functions.css   |    128 +
 .../components/less.js/test/css/ie-filters.css  |      9 +
 .../less.js/test/css/import-interpolation.css   |      6 +
 .../components/less.js/test/css/import-once.css |      3 +
 .../app/components/less.js/test/css/import.css  |     38 +
 .../components/less.js/test/css/javascript.css  |     23 +
 .../components/less.js/test/css/lazy-eval.css   |      3 +
 .../less.js/test/css/legacy/legacy.css          |      7 +
 .../app/components/less.js/test/css/media.css   |    203 +
 .../components/less.js/test/css/mixins-args.css |    113 +
 .../less.js/test/css/mixins-closure.css         |      9 +
 .../less.js/test/css/mixins-guards.css          |     76 +
 .../less.js/test/css/mixins-important.css       |     38 +
 .../less.js/test/css/mixins-named-args.css      |     27 +
 .../less.js/test/css/mixins-nested.css          |     14 +
 .../less.js/test/css/mixins-pattern.css         |     47 +
 .../app/components/less.js/test/css/mixins.css  |    121 +
 .../components/less.js/test/css/operations.css  |     49 +
 .../app/components/less.js/test/css/parens.css  |     33 +
 .../components/less.js/test/css/rulesets.css    |     33 +
 .../app/components/less.js/test/css/scope.css   |     35 +
 .../components/less.js/test/css/selectors.css   |    141 +
 .../less.js/test/css/static-urls/urls.css       |     42 +
 .../app/components/less.js/test/css/strings.css |     40 +
 webapp/app/components/less.js/test/css/urls.css |     59 +
 .../components/less.js/test/css/variables.css   |     45 +
 .../components/less.js/test/css/whitespace.css  |     42 +
 .../less.js/test/data/data-uri-fail.png         |    Bin 0 -> 52420 bytes
 .../app/components/less.js/test/data/image.jpg  |      1 +
 .../app/components/less.js/test/data/page.html  |      1 +
 webapp/app/components/less.js/test/less-test.js |    214 +
 .../components/less.js/test/less/charsets.less  |      3 +
 .../components/less.js/test/less/colors.less    |     92 +
 .../components/less.js/test/less/comments.less  |     77 +
 .../test/less/compression/compression.less      |     16 +
 .../app/components/less.js/test/less/css-3.less |    117 +
 .../less.js/test/less/css-escapes.less          |     33 +
 .../app/components/less.js/test/less/css.less   |    108 +
 .../less.js/test/less/debug/import/test.less    |     25 +
 .../less.js/test/less/debug/linenumbers.less    |     23 +
 .../test/less/errors/add-mixed-units.less       |      3 +
 .../test/less/errors/add-mixed-units.txt        |      2 +
 .../test/less/errors/add-mixed-units2.less      |      3 +
 .../test/less/errors/add-mixed-units2.txt       |      2 +
 .../less/errors/bad-variable-declaration1.less  |      1 +
 .../less/errors/bad-variable-declaration1.txt   |      2 +
 .../test/less/errors/color-operation-error.less |      3 +
 .../test/less/errors/color-operation-error.txt  |      2 +
 .../test/less/errors/comment-in-selector.less   |      1 +
 .../test/less/errors/comment-in-selector.txt    |      2 +
 .../test/less/errors/divide-mixed-units.less    |      3 +
 .../test/less/errors/divide-mixed-units.txt     |      4 +
 .../test/less/errors/extend-no-selector.less    |      3 +
 .../test/less/errors/extend-no-selector.txt     |      3 +
 .../test/less/errors/extend-not-at-end.less     |      3 +
 .../test/less/errors/extend-not-at-end.txt      |      3 +
 .../test/less/errors/import-missing.less        |      6 +
 .../less.js/test/less/errors/import-missing.txt |      3 +
 .../test/less/errors/import-no-semi.less        |      1 +
 .../less.js/test/less/errors/import-no-semi.txt |      2 +
 .../test/less/errors/import-subfolder1.less     |      1 +
 .../test/less/errors/import-subfolder1.txt      |      3 +
 .../test/less/errors/import-subfolder2.less     |      1 +
 .../test/less/errors/import-subfolder2.txt      |      2 +
 .../less/errors/imports/import-subfolder1.less  |      1 +
 .../less/errors/imports/import-subfolder2.less  |      1 +
 .../test/less/errors/imports/import-test.less   |      4 +
 .../imports/subfolder/mixin-not-defined.less    |      1 +
 .../subfolder/parse-error-curly-bracket.less    |      1 +
 .../test/less/errors/javascript-error.less      |      3 +
 .../test/less/errors/javascript-error.txt       |      4 +
 .../errors/mixed-mixin-definition-args-1.less   |      6 +
 .../errors/mixed-mixin-definition-args-1.txt    |      4 +
 .../errors/mixed-mixin-definition-args-2.less   |      6 +
 .../errors/mixed-mixin-definition-args-2.txt    |      4 +
 .../test/less/errors/mixin-not-defined.less     |     11 +
 .../test/less/errors/mixin-not-defined.txt      |      3 +
 .../test/less/errors/mixin-not-matched.less     |      6 +
 .../test/less/errors/mixin-not-matched.txt      |      3 +
 .../test/less/errors/mixin-not-matched2.less    |      6 +
 .../test/less/errors/mixin-not-matched2.txt     |      3 +
 .../test/less/errors/multiply-mixed-units.less  |      7 +
 .../test/less/errors/multiply-mixed-units.txt   |      4 +
 .../test/less/errors/parens-error-1.less        |      3 +
 .../less.js/test/less/errors/parens-error-1.txt |      4 +
 .../test/less/errors/parens-error-2.less        |      3 +
 .../less.js/test/less/errors/parens-error-2.txt |      4 +
 .../test/less/errors/parens-error-3.less        |      3 +
 .../less.js/test/less/errors/parens-error-3.txt |      4 +
 .../less/errors/parse-error-curly-bracket.less  |      1 +
 .../less/errors/parse-error-curly-bracket.txt   |      2 +
 .../errors/parse-error-missing-bracket.less     |      2 +
 .../less/errors/parse-error-missing-bracket.txt |      3 +
 .../less/errors/parse-error-with-import.less    |     13 +
 .../less/errors/parse-error-with-import.txt     |      4 +
 .../test/less/errors/property-ie5-hack.less     |      3 +
 .../test/less/errors/property-ie5-hack.txt      |      4 +
 .../test/less/errors/property-in-root.less      |      4 +
 .../test/less/errors/property-in-root.txt       |      4 +
 .../test/less/errors/property-in-root2.less     |      1 +
 .../test/less/errors/property-in-root2.txt      |      4 +
 .../test/less/errors/property-in-root3.less     |      4 +
 .../test/less/errors/property-in-root3.txt      |      3 +
 .../test/less/errors/recursive-variable.less    |      1 +
 .../test/less/errors/recursive-variable.txt     |      2 +
 .../less.js/test/less/extend-chaining.less      |     79 +
 .../less.js/test/less/extend-clearfix.less      |     19 +
 .../less.js/test/less/extend-exact.less         |     46 +
 .../less.js/test/less/extend-media.less         |     24 +
 .../less.js/test/less/extend-nest.less          |     65 +
 .../less.js/test/less/extend-selector.less      |     84 +
 .../components/less.js/test/less/extend.less    |     81 +
 .../components/less.js/test/less/functions.less |    142 +
 .../less.js/test/less/ie-filters.less           |     15 +
 .../less.js/test/less/import-interpolation.less |      8 +
 .../less.js/test/less/import-once.less          |      4 +
 .../components/less.js/test/less/import.less    |     21 +
 .../less/import/deeper/import-once-test-a.less  |      1 +
 .../import/import-and-relative-paths-test.less  |      6 +
 .../test/less/import/import-charset-test.less   |      1 +
 .../test/less/import/import-interpolation.less  |      1 +
 .../test/less/import/import-interpolation2.less |      5 +
 .../test/less/import/import-once-test-c.less    |      6 +
 .../less.js/test/less/import/import-test-a.less |      3 +
 .../less.js/test/less/import/import-test-b.less |      8 +
 .../less.js/test/less/import/import-test-c.less |      6 +
 .../less.js/test/less/import/import-test-d.css  |      1 +
 .../less.js/test/less/import/import-test-e.less |      2 +
 .../less.js/test/less/import/imports/font.less  |      8 +
 .../less.js/test/less/import/imports/logo.less  |      5 +
 .../less.js/test/less/import/urls.less          |      1 +
 .../less.js/test/less/javascript.less           |     29 +
 .../components/less.js/test/less/lazy-eval.less |      6 +
 .../less.js/test/less/legacy/legacy.less        |      7 +
 .../app/components/less.js/test/less/media.less |    210 +
 .../less.js/test/less/mixins-args.less          |    205 +
 .../less.js/test/less/mixins-closure.less       |     26 +
 .../less.js/test/less/mixins-guards.less        |    137 +
 .../less.js/test/less/mixins-important.less     |     22 +
 .../less.js/test/less/mixins-named-args.less    |     36 +
 .../less.js/test/less/mixins-nested.less        |     22 +
 .../less.js/test/less/mixins-pattern.less       |     99 +
 .../components/less.js/test/less/mixins.less    |    114 +
 .../less.js/test/less/operations.less           |     62 +
 .../components/less.js/test/less/parens.less    |     41 +
 .../components/less.js/test/less/rulesets.less  |     30 +
 .../app/components/less.js/test/less/scope.less |     79 +
 .../components/less.js/test/less/selectors.less |    143 +
 .../less.js/test/less/static-urls/urls.less     |     33 +
 .../components/less.js/test/less/strings.less   |     51 +
 .../app/components/less.js/test/less/urls.less  |     57 +
 .../components/less.js/test/less/variables.less |     83 +
 .../less.js/test/less/whitespace.less           |     44 +
 webapp/app/components/messenger/.bower.json     |     52 +
 webapp/app/components/messenger/CHANGELOG.md    |    102 +
 webapp/app/components/messenger/CONTRIBUTORS    |      8 +
 webapp/app/components/messenger/LICENSE         |      8 +
 webapp/app/components/messenger/README.md       |     10 +
 webapp/app/components/messenger/bower.json      |     43 +
 .../messenger/build/css/messenger-spinner.css   |    235 +
 .../messenger/build/css/messenger-theme-air.css |    438 +
 .../build/css/messenger-theme-block.css         |     96 +
 .../build/css/messenger-theme-flat.css          |    462 +
 .../build/css/messenger-theme-future.css        |    496 +
 .../messenger/build/css/messenger-theme-ice.css |    113 +
 .../messenger/build/css/messenger.css           |    101 +
 .../messenger/build/js/messenger-theme-flat.js  |     33 +
 .../build/js/messenger-theme-future.js          |     33 +
 .../components/messenger/build/js/messenger.js  |   1263 +
 .../messenger/build/js/messenger.min.js         |      2 +
 .../messenger/docs/images/messenger.gif         |    Bin 0 -> 452663 bytes
 .../messenger/docs/images/messenger.m4v         |    Bin 0 -> 122299 bytes
 webapp/app/components/messenger/docs/intro.md   |    316 +
 .../messenger/docs/welcome/iframe-demo.html     |    120 +
 .../messenger/docs/welcome/images/bg_hr.png     |    Bin 0 -> 943 bytes
 .../docs/welcome/images/blacktocat.png          |    Bin 0 -> 1428 bytes
 .../docs/welcome/images/icon_download.png       |    Bin 0 -> 1162 bytes
 .../docs/welcome/images/messenger_preview.png   |    Bin 0 -> 7656 bytes
 .../docs/welcome/images/sprite_download.png     |    Bin 0 -> 16799 bytes
 .../messenger/docs/welcome/index.html           |    436 +
 .../messenger/docs/welcome/javascripts/demo.js  |     41 +
 .../docs/welcome/javascripts/execute.coffee     |     19 +
 .../welcome/javascripts/location-sel.coffee     |     41 +
 .../docs/welcome/javascripts/location-sel.js    |     63 +
 .../messenger/docs/welcome/javascripts/main.js  |      1 +
 .../docs/welcome/javascripts/theme-sel.coffee   |     30 +
 .../docs/welcome/javascripts/theme-sel.js       |     57 +
 .../messenger/docs/welcome/lib/executr/LICENSE  |      8 +
 .../docs/welcome/lib/executr/README.md          |     93 +
 .../messenger/docs/welcome/lib/executr/build.sh |      4 +
 .../welcome/lib/executr/build/css/executr.css   |     38 +
 .../welcome/lib/executr/build/js/executr.js     |    217 +
 .../docs/welcome/lib/executr/demo.html          |     39 +
 .../welcome/lib/executr/lib/CodeMirror/LICENSE  |     23 +
 .../lib/executr/lib/CodeMirror/codemirror.css   |    240 +
 .../lib/executr/lib/CodeMirror/codemirror.js    |   4786 +
 .../lib/CodeMirror/mode/coffeescript/LICENSE    |     22 +
 .../mode/coffeescript/coffeescript.js           |    346 +
 .../lib/CodeMirror/mode/coffeescript/index.html |    728 +
 .../lib/CodeMirror/mode/javascript/index.html   |     88 +
 .../CodeMirror/mode/javascript/javascript.js    |    422 +
 .../CodeMirror/mode/javascript/typescript.html  |     48 +
 .../welcome/lib/executr/lib/coffee-script.js    |      8 +
 .../welcome/lib/executr/lib/js2coffee.min.js    |   2867 +
 .../welcome/lib/executr/lib/underscore.min.js   |     32 +
 .../lib/executr/src/coffee/executr.coffee       |    207 +
 .../welcome/lib/executr/src/css/executr.css     |     38 +
 .../docs/welcome/lib/jasmine-1.3.1/MIT.LICENSE  |     20 +
 .../welcome/lib/jasmine-1.3.1/jasmine-html.js   |    681 +
 .../docs/welcome/lib/jasmine-1.3.1/jasmine.css  |     82 +
 .../docs/welcome/lib/jasmine-1.3.1/jasmine.js   |   2600 +
 .../docs/welcome/lib/licenses/backbone          |     22 +
 .../messenger/docs/welcome/lib/licenses/jasmine |     20 +
 .../messenger/docs/welcome/lib/licenses/sinon   |     27 +
 .../docs/welcome/lib/licenses/underscore        |     22 +
 .../messenger/docs/welcome/lib/shims.js         |    276 +
 .../messenger/docs/welcome/lib/sinon-1.6.0.js   |   4223 +
 .../messenger/docs/welcome/stylesheets/demo.css |     64 +
 .../docs/welcome/stylesheets/location-sel.css   |     28 +
 .../docs/welcome/stylesheets/location-sel.sass  |     56 +
 .../docs/welcome/stylesheets/pygment_trac.css   |     70 +
 .../docs/welcome/stylesheets/stylesheet.css     |    429 +
 .../docs/welcome/stylesheets/theme-sel.css      |      8 +
 .../docs/welcome/stylesheets/theme-sel.sass     |     10 +
 webapp/app/components/moment/.bower.json        |     30 +
 webapp/app/components/moment/LICENSE            |     22 +
 webapp/app/components/moment/bower.json         |     20 +
 webapp/app/components/moment/lang/ar-ma.js      |     56 +
 webapp/app/components/moment/lang/ar.js         |     56 +
 webapp/app/components/moment/lang/bg.js         |     86 +
 webapp/app/components/moment/lang/br.js         |    107 +
 webapp/app/components/moment/lang/bs.js         |    139 +
 webapp/app/components/moment/lang/ca.js         |     66 +
 webapp/app/components/moment/lang/cs.js         |    155 +
 webapp/app/components/moment/lang/cv.js         |     59 +
 webapp/app/components/moment/lang/cy.js         |     77 +
 webapp/app/components/moment/lang/da.js         |     56 +
 webapp/app/components/moment/lang/de.js         |     71 +
 webapp/app/components/moment/lang/el.js         |     79 +
 webapp/app/components/moment/lang/en-au.js      |     62 +
 webapp/app/components/moment/lang/en-ca.js      |     59 +
 webapp/app/components/moment/lang/en-gb.js      |     63 +
 webapp/app/components/moment/lang/eo.js         |     65 +
 webapp/app/components/moment/lang/es.js         |     66 +
 webapp/app/components/moment/lang/et.js         |     60 +
 webapp/app/components/moment/lang/eu.js         |     60 +
 webapp/app/components/moment/lang/fa.js         |     97 +
 webapp/app/components/moment/lang/fi.js         |    103 +
 webapp/app/components/moment/lang/fo.js         |     56 +
 webapp/app/components/moment/lang/fr-ca.js      |     54 +
 webapp/app/components/moment/lang/fr.js         |     58 +
 webapp/app/components/moment/lang/gl.js         |     71 +
 webapp/app/components/moment/lang/he.js         |     77 +
 webapp/app/components/moment/lang/hi.js         |    105 +
 webapp/app/components/moment/lang/hr.js         |    140 +
 webapp/app/components/moment/lang/hu.js         |     98 +
 webapp/app/components/moment/lang/id.js         |     67 +
 webapp/app/components/moment/lang/is.js         |    124 +
 webapp/app/components/moment/lang/it.js         |     59 +
 webapp/app/components/moment/lang/ja.js         |     58 +
 webapp/app/components/moment/lang/ka.js         |    108 +
 webapp/app/components/moment/lang/ko.js         |     56 +
 webapp/app/components/moment/lang/lt.js         |    118 +
 webapp/app/components/moment/lang/lv.js         |     77 +
 webapp/app/components/moment/lang/ml.js         |     64 +
 webapp/app/components/moment/lang/mr.js         |    104 +
 webapp/app/components/moment/lang/ms-my.js      |     66 +
 webapp/app/components/moment/lang/nb.js         |     57 +
 webapp/app/components/moment/lang/ne.js         |    105 +
 webapp/app/components/moment/lang/nl.js         |     67 +
 webapp/app/components/moment/lang/nn.js         |     56 +
 webapp/app/components/moment/lang/pl.js         |     98 +
 webapp/app/components/moment/lang/pt-br.js      |     56 +
 webapp/app/components/moment/lang/pt.js         |     60 +
 webapp/app/components/moment/lang/ro.js         |     56 +
 webapp/app/components/moment/lang/ru.js         |    163 +
 webapp/app/components/moment/lang/sk.js         |    156 +
 webapp/app/components/moment/lang/sl.js         |    144 +
 webapp/app/components/moment/lang/sq.js         |     57 +
 webapp/app/components/moment/lang/sv.js         |     63 +
 webapp/app/components/moment/lang/th.js         |     58 +
 webapp/app/components/moment/lang/tl-ph.js      |     58 +
 webapp/app/components/moment/lang/tr.js         |     93 +
 webapp/app/components/moment/lang/tzm-la.js     |     55 +
 webapp/app/components/moment/lang/tzm.js        |     55 +
 webapp/app/components/moment/lang/uk.js         |    157 +
 webapp/app/components/moment/lang/uz.js         |     55 +
 webapp/app/components/moment/lang/vn.js         |     62 +
 webapp/app/components/moment/lang/zh-cn.js      |    108 +
 webapp/app/components/moment/lang/zh-tw.js      |     84 +
 webapp/app/components/moment/min/langs.js       |   5187 +
 webapp/app/components/moment/min/langs.min.js   |      3 +
 .../components/moment/min/moment-with-langs.js  |   7063 +
 .../moment/min/moment-with-langs.min.js         |      9 +
 webapp/app/components/moment/min/moment.min.js  |      6 +
 webapp/app/components/moment/moment.js          |   2314 +
 webapp/app/components/moment/readme.md          |    299 +
 webapp/app/components/ng-grid/.bower.json       |     41 +
 webapp/app/components/ng-grid/CHANGELOG.md      |    271 +
 webapp/app/components/ng-grid/CONTRIBUTING.md   |    145 +
 webapp/app/components/ng-grid/LICENSE.md        |     21 +
 webapp/app/components/ng-grid/README.md         |    124 +
 webapp/app/components/ng-grid/bower.json        |     29 +
 .../components/ng-grid/build/ng-grid.debug.js   |   3979 +
 webapp/app/components/ng-grid/build/ng-grid.js  |   3591 +
 .../app/components/ng-grid/build/ng-grid.min.js |      3 +
 .../components/ng-grid/ng-grid-2.0.11.debug.js  |   3979 +
 .../components/ng-grid/ng-grid-2.0.11.min.js    |      3 +
 webapp/app/components/ng-grid/ng-grid.css       |    447 +
 webapp/app/components/ng-grid/ng-grid.min.css   |      1 +
 webapp/app/components/ng-grid/package.json      |     40 +
 webapp/app/components/ng-grid/plugins/README.md |     45 +
 .../ng-grid/plugins/ng-grid-csv-export.js       |     91 +
 .../ng-grid/plugins/ng-grid-flexible-height.js  |     40 +
 .../ng-grid/plugins/ng-grid-layout.js           |     22 +
 .../ng-grid/plugins/ng-grid-pdf-export.js       |     44 +
 .../ng-grid/plugins/ng-grid-reorderable.js      |     68 +
 .../ng-grid/plugins/ng-grid-wysiwyg-export.js   |     39 +
 .../components/ng-grid/plugins/playground.html  |     85 +
 webapp/app/components/nvd3/.bower.json          |     47 +
 webapp/app/components/nvd3/GruntFile.js         |    106 +
 webapp/app/components/nvd3/LICENSE.md           |     49 +
 webapp/app/components/nvd3/Makefile             |     72 +
 webapp/app/components/nvd3/README.md            |     87 +
 webapp/app/components/nvd3/bower.json           |     35 +
 webapp/app/components/nvd3/build.bat            |      6 +
 webapp/app/components/nvd3/nv.d3.css            |    769 +
 webapp/app/components/nvd3/nv.d3.js             |  14365 +
 webapp/app/components/nvd3/nv.d3.min.css        |      1 +
 webapp/app/components/nvd3/nv.d3.min.js         |      6 +
 webapp/app/components/nvd3/package.json         |     13 +
 webapp/app/components/nvd3/src/nv.d3.css        |    769 +
 webapp/app/htaccess.dist                        |      3 +
 webapp/app/image/Himg.png                       |    Bin 0 -> 913426 bytes
 webapp/app/image/ajax-loader.gif                |    Bin 0 -> 847 bytes
 webapp/app/image/cube1.png                      |    Bin 0 -> 24538 bytes
 webapp/app/image/database.png                   |    Bin 0 -> 3147 bytes
 webapp/app/image/default_avatar.png             |    Bin 0 -> 977 bytes
 webapp/app/image/favicon.ico                    |    Bin 0 -> 16958 bytes
 webapp/app/image/forkme_right_red.png           |    Bin 0 -> 7927 bytes
 webapp/app/image/icon_table.png                 |    Bin 0 -> 484 bytes
 webapp/app/image/logo.png                       |    Bin 0 -> 114812 bytes
 webapp/app/image/table-icon.png                 |    Bin 0 -> 489 bytes
 webapp/app/image/waiting.gif                    |    Bin 0 -> 1787 bytes
 webapp/app/index.html                           |    144 +
 webapp/app/js/app.js                            |      2 +
 webapp/app/js/config.js                         |    135 +
 webapp/app/js/controllers/access.js             |    110 +
 webapp/app/js/controllers/admin.js              |    198 +
 webapp/app/js/controllers/auth.js               |     33 +
 webapp/app/js/controllers/cube.js               |     87 +
 webapp/app/js/controllers/cubeEdit.js           |    554 +
 webapp/app/js/controllers/cubeSchema.js         |    286 +
 webapp/app/js/controllers/cubes.js              |    448 +
 webapp/app/js/controllers/index.js              |     26 +
 webapp/app/js/controllers/job.js                |    207 +
 webapp/app/js/controllers/page.js               |    226 +
 webapp/app/js/controllers/projectMeta.js        |     78 +
 webapp/app/js/controllers/projects.js           |     81 +
 webapp/app/js/controllers/query.js              |    469 +
 webapp/app/js/controllers/sourceMeta.js         |    209 +
 webapp/app/js/directives/directives.js          |    178 +
 webapp/app/js/factories/graph.js                |     59 +
 webapp/app/js/filters/filter.js                 |     87 +
 webapp/app/js/listeners.js                      |     84 +
 webapp/app/js/services/access.js                |      8 +
 webapp/app/js/services/admin.js                 |     11 +
 webapp/app/js/services/auth.js                  |      7 +
 webapp/app/js/services/cache.js                 |      5 +
 webapp/app/js/services/cubeDesc.js              |      5 +
 webapp/app/js/services/cubes.js                 |     17 +
 webapp/app/js/services/graph.js                 |     28 +
 webapp/app/js/services/jobs.js                  |      9 +
 webapp/app/js/services/message.js               |     39 +
 webapp/app/js/services/ngLoading.js             |     34 +
 webapp/app/js/services/projects.js              |      8 +
 webapp/app/js/services/query.js                 |     11 +
 webapp/app/js/services/tables.js                |     10 +
 webapp/app/js/services/tree.js                  |    301 +
 webapp/app/js/services/users.js                 |     41 +
 webapp/app/js/utils/utils.js                    |     50 +
 webapp/app/less/app.less                        |    538 +
 webapp/app/less/build.less                      |      5 +
 webapp/app/less/component.less                  |    840 +
 webapp/app/less/home.less                       |     85 +
 webapp/app/less/layout.less                     |     27 +
 webapp/app/less/navbar.less                     |    208 +
 webapp/app/partials/admin/admin.html            |    150 +
 webapp/app/partials/common/access.html          |     88 +
 .../cubeDesigner/advanced_settings.html         |    113 +
 .../app/partials/cubeDesigner/dimensions.html   |    357 +
 webapp/app/partials/cubeDesigner/filter.html    |     40 +
 .../app/partials/cubeDesigner/incremental.html  |    103 +
 webapp/app/partials/cubeDesigner/info.html      |     93 +
 webapp/app/partials/cubeDesigner/measures.html  |    191 +
 webapp/app/partials/cubeDesigner/overview.html  |     58 +
 webapp/app/partials/cubes/cube_detail.html      |     95 +
 webapp/app/partials/cubes/cube_edit.html        |     16 +
 webapp/app/partials/cubes/cube_json_edit.html   |     43 +
 webapp/app/partials/cubes/cube_schema.html      |     50 +
 webapp/app/partials/cubes/cubes.html            |    146 +
 webapp/app/partials/directives/loading.html     |      6 +
 webapp/app/partials/directives/noResult.html    |      3 +
 webapp/app/partials/directives/pagination.html  |      9 +
 webapp/app/partials/directives/typeahead.html   |     12 +
 webapp/app/partials/footer.html                 |     10 +
 webapp/app/partials/header.html                 |     59 +
 webapp/app/partials/home.html                   |     47 +
 webapp/app/partials/jobs/job_merge.html         |    114 +
 webapp/app/partials/jobs/job_refresh.html       |     81 +
 webapp/app/partials/jobs/job_steps.html         |    145 +
 webapp/app/partials/jobs/job_submit.html        |     66 +
 webapp/app/partials/jobs/jobs.html              |    154 +
 webapp/app/partials/login.html                  |     55 +
 .../app/partials/projects/project_create.html   |     38 +
 .../app/partials/projects/project_detail.html   |     46 +
 .../partials/projects/project_table_tree.html   |     21 +
 webapp/app/partials/projects/projects.html      |     66 +
 webapp/app/partials/query/query.html            |    218 +
 webapp/app/partials/query/query_detail.html     |    254 +
 webapp/app/partials/tables/source_metadata.html |    167 +
 .../app/partials/tables/source_table_tree.html  |     31 +
 webapp/app/routes.json                          |    107 +
 webapp/bower.json                               |     36 +
 webapp/grunt.json                               |     63 +
 webapp/package.json                             |     22 +
 webapp/template/nginx_default.conf              |     95 +
 webapp/validate-commit-msg.js                   |    106 +
 3655 files changed, 1665281 insertions(+), 3 deletions(-)
----------------------------------------------------------------------



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

Posted by lu...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/docs/README.md
----------------------------------------------------------------------
diff --git a/docs/README.md b/docs/README.md
new file mode 100644
index 0000000..e69de29

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/examples/.gitignore
----------------------------------------------------------------------
diff --git a/examples/.gitignore b/examples/.gitignore
new file mode 100644
index 0000000..91fe24e
--- /dev/null
+++ b/examples/.gitignore
@@ -0,0 +1,2 @@
+/yadesk00.remote*
+

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/examples/ReleaseNotes.md
----------------------------------------------------------------------
diff --git a/examples/ReleaseNotes.md b/examples/ReleaseNotes.md
new file mode 100644
index 0000000..fd27193
--- /dev/null
+++ b/examples/ReleaseNotes.md
@@ -0,0 +1,10 @@
+Model: Query
+
+Release 0.3.1-SNAPSHOT
+======================
+>Upgrade Optiq to 0.4.17
+	Upgrade Optiq framework from 0.4.16 to 0.4.17
+
+
+Release 0.3.0
+=============
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/examples/test_case_data/localmeta/cube/test_kylin_cube_ii.json
----------------------------------------------------------------------
diff --git a/examples/test_case_data/localmeta/cube/test_kylin_cube_ii.json b/examples/test_case_data/localmeta/cube/test_kylin_cube_ii.json
new file mode 100644
index 0000000..8e8e474
--- /dev/null
+++ b/examples/test_case_data/localmeta/cube/test_kylin_cube_ii.json
@@ -0,0 +1,39 @@
+{
+  "uuid" : "daa53e80-41be-49a5-89ca-9fb7294db186",
+  "name" : "test_kylin_cube_ii",
+  "owner" : null,
+  "version" : null,
+  "cost" : 10,
+  "status" : "READY",
+  "segments" : [ {
+    "name" : "19700101000000_20140901000000",
+    "status" : "READY",
+    "dictionaries" : {
+      "TEST_KYLIN_FACT/LSTG_FORMAT_NAME" : "/dict/TEST_KYLIN_FACT/LSTG_FORMAT_NAME/bd9f6b22-36ba-4e6b-92aa-0d585faf0b39.dict",
+      "TEST_KYLIN_FACT/TRANS_ID" : "/dict/TEST_KYLIN_FACT/TRANS_ID/c8d19f95-b6cd-4219-a114-54aaddcb2909.dict",
+      "TEST_KYLIN_FACT/LSTG_SITE_ID" : "/dict/TEST_KYLIN_FACT/LSTG_SITE_ID/7df5789b-0280-453c-b406-b75cad6770d1.dict",
+      "TEST_KYLIN_FACT/SLR_SEGMENT_CD" : "/dict/TEST_KYLIN_FACT/SLR_SEGMENT_CD/8300bf83-053e-48bb-8c87-88c8d483afd1.dict",
+      "TEST_KYLIN_FACT/SELLER_ID" : "/dict/TEST_KYLIN_FACT/SELLER_ID/6e285e1e-ed16-4012-9f1e-f950dd6927ce.dict",
+      "TEST_KYLIN_FACT/ITEM_COUNT" : "/dict/TEST_KYLIN_FACT/ITEM_COUNT/73c9bfe1-6496-4ff8-9467-6cbee2924c16.dict",
+      "TEST_KYLIN_FACT/CAL_DT" : "/dict/TEST_KYLIN_FACT/CAL_DT/48433f91-0d68-495f-b7f2-295414591275.dict",
+      "TEST_KYLIN_FACT/LEAF_CATEG_ID" : "/dict/TEST_KYLIN_FACT/LEAF_CATEG_ID/96b7c577-b209-45b3-a848-4d2d7af5c0cc.dict",
+      "TEST_KYLIN_FACT/PRICE" : "/dict/TEST_KYLIN_FACT/PRICE/927dde3f-6999-4434-b57c-adfa73160334.dict"
+    },
+    "snapshots" : { },
+    "storage_location_identifier" : "test_III",
+    "date_range_start" : 0,
+    "date_range_end" : 0,
+    "size_kb" : 0,
+    "source_records" : 0,
+    "source_records_size" : 0,
+    "last_build_time" : 0,
+    "last_build_job_id" : null,
+    "binary_signature" : null
+  } ],
+  "last_modified" : 1414999085798,
+  "descriptor" : "test_kylin_cube_ii",
+  "create_time" : null,
+  "size_kb" : 0,
+  "source_records_count" : 0,
+  "source_records_size" : 0
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/examples/test_case_data/localmeta/cube/test_kylin_cube_with_slr_1_new_segment.json
----------------------------------------------------------------------
diff --git a/examples/test_case_data/localmeta/cube/test_kylin_cube_with_slr_1_new_segment.json b/examples/test_case_data/localmeta/cube/test_kylin_cube_with_slr_1_new_segment.json
new file mode 100644
index 0000000..1f6865c
--- /dev/null
+++ b/examples/test_case_data/localmeta/cube/test_kylin_cube_with_slr_1_new_segment.json
@@ -0,0 +1,43 @@
+{
+  "uuid" : "111ca32a-a33e-4b69-83dd-0bb8b1f8c53b",
+  "last_modified" : 1404098141020,
+  "name" : "test_kylin_cube_with_slr_1_new_segment",
+  "owner" : null,
+  "version" : null,
+  "descriptor" : "test_kylin_cube_with_slr_desc",
+  "cost" : 50,
+  "segments" : [ {
+    "name" : "20130331080000_20131212080000",
+    "storage_location_identifier" : "KYLIN-CUBE-TEST_KYLIN_CUBE_WITH_SLR_READY-F24668F6-DCFF-4CB6-A89B-77F1119DF8FA",
+    "date_range_start" : 1364688000000,
+    "date_range_end" : 1386806400000,
+    "status" : "NEW",
+    "size_kb" : 0,
+    "source_records" : 0,
+    "source_records_size" : 0,
+    "last_build_time" : 0,
+    "last_build_job_id" : null,
+    "binary_signature" : null,
+    "dictionaries" : {
+	    "TEST_CATEGORY_GROUPINGS/CATEG_LVL2_NAME" : "/dict/TEST_CATEGORY_GROUPINGS/CATEG_LVL2_NAME/16d8185c-ee6b-4f8c-a919-756d9809f937.dict",
+	    "TEST_KYLIN_FACT/SLR_SEGMENT_CD" : "/dict/TEST_SELLER_TYPE_DIM/SELLER_TYPE_CD/c6bf9b51-6e90-4337-8082-4e2fdf78307f.dict",
+	    "TEST_KYLIN_FACT/LSTG_SITE_ID" : "/dict/TEST_SITES/SITE_ID/652bd393-678a-4f16-a504-fd8ce1229355.dict",
+	    "TEST_SELLER_TYPE_DIM/SELLER_TYPE_CD" : "/dict/TEST_SELLER_TYPE_DIM/SELLER_TYPE_CD/2a44ff38-f64b-42e7-9fcf-66afccac8047.dict",
+	    "TEST_KYLIN_FACT/CAL_DT" : "/dict/PREDEFINED/date(yyyy-mm-dd)/64ac4f82-f2af-476e-85b9-f0805001014e.dict",
+	    "TEST_CATEGORY_GROUPINGS/CATEG_LVL3_NAME" : "/dict/TEST_CATEGORY_GROUPINGS/CATEG_LVL3_NAME/ad09f2d5-054a-4e1b-a776-7cc07399a6c1.dict",
+	    "TEST_CAL_DT/CAL_DT" : "/dict/TEST_CAL_DT/CAL_DT/ed0c3451-593c-494c-9019-64f63fcb0b8e.dict",
+	    "TEST_KYLIN_FACT/LEAF_CATEG_ID" : "/dict/TEST_CATEGORY_GROUPINGS/LEAF_CATEG_ID/4243889f-bc81-4807-a975-7041bbbf35e7.dict",
+	    "TEST_SITES/SITE_ID" : "/dict/TEST_SITES/SITE_ID/ff7e8943-ac0f-4e66-b9ed-510f6a0b875d.dict",
+	    "TEST_CATEGORY_GROUPINGS/META_CATEG_NAME" : "/dict/TEST_CATEGORY_GROUPINGS/META_CATEG_NAME/aceae914-4246-4251-a0c2-692fe7a300df.dict"
+	  },
+	 "snapshots" : {
+	    "TEST_SELLER_TYPE_DIM" : "/table_snapshot/TEST_SELLER_TYPE_DIM.csv/4fe75ccd-9b24-4cdf-ac9d-b4038e947f89.snapshot",
+	    "TEST_CAL_DT" : "/table_snapshot/TEST_CAL_DT.csv/8ff1339e-f804-47f3-b42c-1d4fa4ff0cf7.snapshot",
+	    "TEST_CATEGORY_GROUPINGS" : "/table_snapshot/TEST_CATEGORY_GROUPINGS.csv/e172b442-ae10-447e-9071-c7dbb2bb38cc.snapshot",
+	    "TEST_SITES" : "/table_snapshot/TEST_SITES.csv/28130338-fcf4-429e-91b0-cd8dfd397280.snapshot"
+	  }
+  } ],
+  "status" : "DISABLED",
+  "create_time" : null,
+  "notify_list" : null
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/examples/test_case_data/localmeta/cube/test_kylin_cube_with_slr_empty.json
----------------------------------------------------------------------
diff --git a/examples/test_case_data/localmeta/cube/test_kylin_cube_with_slr_empty.json b/examples/test_case_data/localmeta/cube/test_kylin_cube_with_slr_empty.json
new file mode 100644
index 0000000..70d89ea
--- /dev/null
+++ b/examples/test_case_data/localmeta/cube/test_kylin_cube_with_slr_empty.json
@@ -0,0 +1,10 @@
+{
+  "uuid" : "1eaca32a-a33e-4b69-83dd-0bb8b1f8c53b",
+  "last_modified" : 0,
+  "name" : "test_kylin_cube_with_slr_empty",
+  "owner" : null,
+  "version" : null,
+  "descriptor" : "test_kylin_cube_with_slr_desc",
+  "segments" : [ ],
+  "create_time" : null
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/examples/test_case_data/localmeta/cube/test_kylin_cube_with_slr_left_join_empty.json
----------------------------------------------------------------------
diff --git a/examples/test_case_data/localmeta/cube/test_kylin_cube_with_slr_left_join_empty.json b/examples/test_case_data/localmeta/cube/test_kylin_cube_with_slr_left_join_empty.json
new file mode 100644
index 0000000..e708a86
--- /dev/null
+++ b/examples/test_case_data/localmeta/cube/test_kylin_cube_with_slr_left_join_empty.json
@@ -0,0 +1,11 @@
+{
+  "uuid" : "1eaca32a-a33e-4b69-83dd-0bb8b1f8dddd",
+  "last_modified" : 0,
+  "name" : "test_kylin_cube_with_slr_left_join_empty",
+  "owner" : null,
+  "version" : null,
+  "descriptor" : "test_kylin_cube_with_slr_left_join_desc",
+  "segments" : [ ],
+  "status" : "DISABLED",
+  "create_time" : null
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/examples/test_case_data/localmeta/cube/test_kylin_cube_with_slr_left_join_ready.json
----------------------------------------------------------------------
diff --git a/examples/test_case_data/localmeta/cube/test_kylin_cube_with_slr_left_join_ready.json b/examples/test_case_data/localmeta/cube/test_kylin_cube_with_slr_left_join_ready.json
new file mode 100644
index 0000000..90acf8d
--- /dev/null
+++ b/examples/test_case_data/localmeta/cube/test_kylin_cube_with_slr_left_join_ready.json
@@ -0,0 +1,43 @@
+{
+  "uuid" : "kkkka32a-a33e-4b69-83dd-0bb8b1f8c53b",
+  "last_modified" : 1404097095621,
+  "name" : "test_kylin_cube_with_slr_left_join_ready",
+  "owner" : null,
+  "version" : null,
+  "descriptor" : "test_kylin_cube_with_slr_left_join_desc",
+  "cost" : 50,
+  "segments" : [ {
+	"name" : null,
+	"storage_location_identifier" : "KYLIN-CUBE-TEST_KYLIN_CUBE_WITH_SLR_LEFT_JOIN_READY-BCF2F125-9B0B-40DD-9509-95EC59B31333",
+	"date_range_start" : null,
+	"date_range_end" : null,
+	"status" : "READY",
+	"size_kb" : 7690,
+	"source_records" : 10000,
+	"source_records_size" : 610288,
+	"last_build_time" : 1404097095455,
+	"last_build_job_id" : "bcf2f125-9b0b-40dd-9509-95ec59b31333",
+	"binary_signature" : null,
+      "dictionaries" : {
+        "TEST_CATEGORY_GROUPINGS/CATEG_LVL2_NAME" : "/dict/TEST_CATEGORY_GROUPINGS/CATEG_LVL2_NAME/c12ae49d-9dbe-4a58-b169-19afac317696.dict",
+        "TEST_KYLIN_FACT/SLR_SEGMENT_CD" : "/dict/TEST_SELLER_TYPE_DIM/SELLER_TYPE_CD/14fe66b3-5956-498c-bd93-40182cac5510.dict",
+        "TEST_KYLIN_FACT/LSTG_SITE_ID" : "/dict/TEST_SITES/SITE_ID/0bec6bb3-1b0d-469c-8289-b8c4ca5d5001.dict",
+        "TEST_SELLER_TYPE_DIM/SELLER_TYPE_CD" : "/dict/TEST_SELLER_TYPE_DIM/SELLER_TYPE_CD/2a44ff38-f64b-42e7-9fcf-66afccac8047.dict",
+        "TEST_KYLIN_FACT/CAL_DT" : "/dict/PREDEFINED/date(yyyy-mm-dd)/64ac4f82-f2af-476e-85b9-f0805001014e.dict",
+        "TEST_CATEGORY_GROUPINGS/CATEG_LVL3_NAME" : "/dict/TEST_CATEGORY_GROUPINGS/CATEG_LVL3_NAME/eacccee7-d120-4f4d-97d0-c99a5b83ec32.dict",
+        "TEST_CAL_DT/CAL_DT" : "/dict/TEST_CAL_DT/CAL_DT/ed0c3451-593c-494c-9019-64f63fcb0b8e.dict",
+        "TEST_KYLIN_FACT/LEAF_CATEG_ID" : "/dict/TEST_CATEGORY_GROUPINGS/LEAF_CATEG_ID/8b4b1c06-fb74-486b-a2ad-74420afebcda.dict",
+        "TEST_SITES/SITE_ID" : "/dict/TEST_SITES/SITE_ID/ff7e8943-ac0f-4e66-b9ed-510f6a0b875d.dict",
+        "TEST_CATEGORY_GROUPINGS/META_CATEG_NAME" : "/dict/TEST_CATEGORY_GROUPINGS/META_CATEG_NAME/c2af25cf-6c79-45e6-a6f6-6d2a8ecc6592.dict"
+      },
+      "snapshots" : {
+        "TEST_SELLER_TYPE_DIM" : "/table_snapshot/TEST_SELLER_TYPE_DIM.csv/b43dd3f1-9a34-4721-8abc-df90962e94d8.snapshot",
+        "TEST_CAL_DT" : "/table_snapshot/TEST_CAL_DT.csv/4af48c94-86de-4e22-a4fd-c49b06cbaa4f.snapshot",
+        "TEST_CATEGORY_GROUPINGS" : "/table_snapshot/TEST_CATEGORY_GROUPINGS.csv/89715f33-15c9-4745-83f9-f2b9817d9100.snapshot",
+        "TEST_SITES" : "/table_snapshot/TEST_SITES.csv/7d70b82e-43f6-4999-b012-0e91a9bb8408.snapshot"
+      }
+  } ],
+  "status" : "READY",
+  "create_time" : null,
+  "notify_list" : null
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/examples/test_case_data/localmeta/cube/test_kylin_cube_with_slr_ready.json
----------------------------------------------------------------------
diff --git a/examples/test_case_data/localmeta/cube/test_kylin_cube_with_slr_ready.json b/examples/test_case_data/localmeta/cube/test_kylin_cube_with_slr_ready.json
new file mode 100644
index 0000000..8b71f9b
--- /dev/null
+++ b/examples/test_case_data/localmeta/cube/test_kylin_cube_with_slr_ready.json
@@ -0,0 +1,45 @@
+{
+  "uuid" : "1eaca32a-a33e-4b69-83dd-0bb8b1f8c53b",
+  "last_modified" : 1404098141020,
+  "name" : "test_kylin_cube_with_slr_ready",
+  "owner" : null,
+  "version" : null,
+  "descriptor" : "test_kylin_cube_with_slr_desc",
+  "cost" : 50,
+  "segments" : [ {
+    "name" : "20130331080000_20131212080000",
+    "storage_location_identifier" : "KYLIN-CUBE-TEST_KYLIN_CUBE_WITH_SLR_READY-F24668F6-DCFF-4CB6-A89B-77F1119DF8FA",
+    "date_range_start" : 1364688000000,
+    "date_range_end" : 1386806400000,
+    "status" : "READY",
+    "size_kb" : 7801,
+    "source_records" : 10000,
+    "source_records_size" : 608012,
+    "last_build_time" : 1404098140902,
+    "last_build_job_id" : "f24668f6-dcff-4cb6-a89b-77f1119df8fa",
+    "binary_signature" : null,
+
+  "dictionaries" : {
+    "TEST_CATEGORY_GROUPINGS/CATEG_LVL2_NAME" : "/dict/TEST_CATEGORY_GROUPINGS/CATEG_LVL2_NAME/16d8185c-ee6b-4f8c-a919-756d9809f937.dict",
+    "TEST_KYLIN_FACT/SLR_SEGMENT_CD" : "/dict/TEST_SELLER_TYPE_DIM/SELLER_TYPE_CD/c6bf9b51-6e90-4337-8082-4e2fdf78307f.dict",
+    "TEST_KYLIN_FACT/LSTG_SITE_ID" : "/dict/TEST_SITES/SITE_ID/652bd393-678a-4f16-a504-fd8ce1229355.dict",
+    "TEST_SELLER_TYPE_DIM/SELLER_TYPE_CD" : "/dict/TEST_SELLER_TYPE_DIM/SELLER_TYPE_CD/2a44ff38-f64b-42e7-9fcf-66afccac8047.dict",
+    "TEST_KYLIN_FACT/CAL_DT" : "/dict/PREDEFINED/date(yyyy-mm-dd)/64ac4f82-f2af-476e-85b9-f0805001014e.dict",
+    "TEST_CATEGORY_GROUPINGS/CATEG_LVL3_NAME" : "/dict/TEST_CATEGORY_GROUPINGS/CATEG_LVL3_NAME/ad09f2d5-054a-4e1b-a776-7cc07399a6c1.dict",
+    "TEST_CAL_DT/CAL_DT" : "/dict/TEST_CAL_DT/CAL_DT/ed0c3451-593c-494c-9019-64f63fcb0b8e.dict",
+    "TEST_KYLIN_FACT/LEAF_CATEG_ID" : "/dict/TEST_CATEGORY_GROUPINGS/LEAF_CATEG_ID/4243889f-bc81-4807-a975-7041bbbf35e7.dict",
+    "TEST_SITES/SITE_ID" : "/dict/TEST_SITES/SITE_ID/ff7e8943-ac0f-4e66-b9ed-510f6a0b875d.dict",
+    "TEST_CATEGORY_GROUPINGS/META_CATEG_NAME" : "/dict/TEST_CATEGORY_GROUPINGS/META_CATEG_NAME/aceae914-4246-4251-a0c2-692fe7a300df.dict"
+  },
+  "snapshots" : {
+    "TEST_SELLER_TYPE_DIM" : "/table_snapshot/TEST_SELLER_TYPE_DIM.csv/4fe75ccd-9b24-4cdf-ac9d-b4038e947f89.snapshot",
+    "TEST_CAL_DT" : "/table_snapshot/TEST_CAL_DT.csv/8ff1339e-f804-47f3-b42c-1d4fa4ff0cf7.snapshot",
+    "TEST_CATEGORY_GROUPINGS" : "/table_snapshot/TEST_CATEGORY_GROUPINGS.csv/e172b442-ae10-447e-9071-c7dbb2bb38cc.snapshot",
+    "TEST_SITES" : "/table_snapshot/TEST_SITES.csv/28130338-fcf4-429e-91b0-cd8dfd397280.snapshot"
+  }
+
+  } ],
+  "status" : "READY",
+  "create_time" : null,
+  "notify_list" : null
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/examples/test_case_data/localmeta/cube/test_kylin_cube_with_slr_ready_2_segments.json
----------------------------------------------------------------------
diff --git a/examples/test_case_data/localmeta/cube/test_kylin_cube_with_slr_ready_2_segments.json b/examples/test_case_data/localmeta/cube/test_kylin_cube_with_slr_ready_2_segments.json
new file mode 100644
index 0000000..dba3bb2
--- /dev/null
+++ b/examples/test_case_data/localmeta/cube/test_kylin_cube_with_slr_ready_2_segments.json
@@ -0,0 +1,73 @@
+{
+  "uuid" : "1eaca32a-a33e-4b69-83dd-0bb8b1f8c53c",
+  "last_modified" : 1404098141020,
+  "name" : "test_kylin_cube_with_slr_ready_2_segments",
+  "owner" : null,
+  "version" : null,
+  "descriptor" : "test_kylin_cube_with_slr_desc",
+  "cost" : 50,
+  "segments" : [ {
+    "name" : "19691231160000_20131112000000",
+    "storage_location_identifier" : "KYLIN-CUBE-TEST_KYLIN_CUBE_WITH_SLR_READY-F24668F6-DCFF-4CB6-A89B-77F1119DF8FA",
+    "date_range_start" : 1384240200000,
+    "date_range_end" : 1384243200000,
+    "status" : "READY",
+    "size_kb" : 7801,
+    "source_records" : 10000,
+    "source_records_size" : 608012,
+    "last_build_time" : 1404098140902,
+    "last_build_job_id" : "f24668f6-dcff-4cb6-a89b-77f1119df8fa",
+    "binary_signature" : null,
+      "dictionaries" : {
+        "TEST_CATEGORY_GROUPINGS/CATEG_LVL2_NAME" : "/dict/TEST_CATEGORY_GROUPINGS/CATEG_LVL2_NAME/16d8185c-ee6b-4f8c-a919-756d9809f937.dict",
+        "TEST_KYLIN_FACT/SLR_SEGMENT_CD" : "/dict/TEST_SELLER_TYPE_DIM/SELLER_TYPE_CD/c6bf9b51-6e90-4337-8082-4e2fdf78307f.dict",
+        "TEST_KYLIN_FACT/LSTG_SITE_ID" : "/dict/TEST_SITES/SITE_ID/652bd393-678a-4f16-a504-fd8ce1229355.dict",
+        "TEST_SELLER_TYPE_DIM/SELLER_TYPE_CD" : "/dict/TEST_SELLER_TYPE_DIM/SELLER_TYPE_CD/2a44ff38-f64b-42e7-9fcf-66afccac8047.dict",
+        "TEST_KYLIN_FACT/CAL_DT" : "/dict/PREDEFINED/date(yyyy-mm-dd)/64ac4f82-f2af-476e-85b9-f0805001014e.dict",
+        "TEST_CATEGORY_GROUPINGS/CATEG_LVL3_NAME" : "/dict/TEST_CATEGORY_GROUPINGS/CATEG_LVL3_NAME/ad09f2d5-054a-4e1b-a776-7cc07399a6c1.dict",
+        "TEST_CAL_DT/CAL_DT" : "/dict/TEST_CAL_DT/CAL_DT/ed0c3451-593c-494c-9019-64f63fcb0b8e.dict",
+        "TEST_KYLIN_FACT/LEAF_CATEG_ID" : "/dict/TEST_CATEGORY_GROUPINGS/LEAF_CATEG_ID/4243889f-bc81-4807-a975-7041bbbf35e7.dict",
+        "TEST_SITES/SITE_ID" : "/dict/TEST_SITES/SITE_ID/ff7e8943-ac0f-4e66-b9ed-510f6a0b875d.dict",
+        "TEST_CATEGORY_GROUPINGS/META_CATEG_NAME" : "/dict/TEST_CATEGORY_GROUPINGS/META_CATEG_NAME/aceae914-4246-4251-a0c2-692fe7a300df.dict"
+      },
+      "snapshots" : {
+        "TEST_SELLER_TYPE_DIM" : "/table_snapshot/TEST_SELLER_TYPE_DIM.csv/4fe75ccd-9b24-4cdf-ac9d-b4038e947f89.snapshot",
+        "TEST_CAL_DT" : "/table_snapshot/TEST_CAL_DT.csv/8ff1339e-f804-47f3-b42c-1d4fa4ff0cf7.snapshot",
+        "TEST_CATEGORY_GROUPINGS" : "/table_snapshot/TEST_CATEGORY_GROUPINGS.csv/e172b442-ae10-447e-9071-c7dbb2bb38cc.snapshot",
+        "TEST_SITES" : "/table_snapshot/TEST_SITES.csv/28130338-fcf4-429e-91b0-cd8dfd397280.snapshot"
+      }
+  }, {
+    "name" : "20131112000000_20131212000000",
+    "storage_location_identifier" : "KYLIN-CUBE-TEST_KYLIN_CUBE_WITH_SLR_READY-F24668F6-DCFF-4CB6-A89B-77F1119DF8FB",
+    "date_range_start" : 1384243200000,
+    "date_range_end" : 1386835200000,
+    "status" : "READY",
+    "size_kb" : 7801,
+    "source_records" : 10000,
+    "source_records_size" : 608012,
+    "last_build_time" : 1404098140902,
+    "last_build_job_id" : "f24668f6-dcff-4cb6-a89b-77f1119df8fb",
+    "binary_signature" : null,
+      "dictionaries" : {
+        "TEST_CATEGORY_GROUPINGS/CATEG_LVL2_NAME" : "/dict/TEST_CATEGORY_GROUPINGS/CATEG_LVL2_NAME/16d8185c-ee6b-4f8c-a919-756d9809f937.dict",
+        "TEST_KYLIN_FACT/SLR_SEGMENT_CD" : "/dict/TEST_SELLER_TYPE_DIM/SELLER_TYPE_CD/c6bf9b51-6e90-4337-8082-4e2fdf78307f.dict",
+        "TEST_KYLIN_FACT/LSTG_SITE_ID" : "/dict/TEST_SITES/SITE_ID/652bd393-678a-4f16-a504-fd8ce1229355.dict",
+        "TEST_SELLER_TYPE_DIM/SELLER_TYPE_CD" : "/dict/TEST_SELLER_TYPE_DIM/SELLER_TYPE_CD/2a44ff38-f64b-42e7-9fcf-66afccac8047.dict",
+        "TEST_KYLIN_FACT/CAL_DT" : "/dict/PREDEFINED/date(yyyy-mm-dd)/64ac4f82-f2af-476e-85b9-f0805001014e.dict",
+        "TEST_CATEGORY_GROUPINGS/CATEG_LVL3_NAME" : "/dict/TEST_CATEGORY_GROUPINGS/CATEG_LVL3_NAME/ad09f2d5-054a-4e1b-a776-7cc07399a6c1.dict",
+        "TEST_CAL_DT/CAL_DT" : "/dict/TEST_CAL_DT/CAL_DT/ed0c3451-593c-494c-9019-64f63fcb0b8e.dict",
+        "TEST_KYLIN_FACT/LEAF_CATEG_ID" : "/dict/TEST_CATEGORY_GROUPINGS/LEAF_CATEG_ID/4243889f-bc81-4807-a975-7041bbbf35e7.dict",
+        "TEST_SITES/SITE_ID" : "/dict/TEST_SITES/SITE_ID/ff7e8943-ac0f-4e66-b9ed-510f6a0b875d.dict",
+        "TEST_CATEGORY_GROUPINGS/META_CATEG_NAME" : "/dict/TEST_CATEGORY_GROUPINGS/META_CATEG_NAME/aceae914-4246-4251-a0c2-692fe7a300df.dict"
+      },
+      "snapshots" : {
+        "TEST_SELLER_TYPE_DIM" : "/table_snapshot/TEST_SELLER_TYPE_DIM.csv/4fe75ccd-9b24-4cdf-ac9d-b4038e947f89.snapshot",
+        "TEST_CAL_DT" : "/table_snapshot/TEST_CAL_DT.csv/8ff1339e-f804-47f3-b42c-1d4fa4ff0cf7.snapshot",
+        "TEST_CATEGORY_GROUPINGS" : "/table_snapshot/TEST_CATEGORY_GROUPINGS.csv/e172b442-ae10-447e-9071-c7dbb2bb38cc.snapshot",
+        "TEST_SITES" : "/table_snapshot/TEST_SITES.csv/28130338-fcf4-429e-91b0-cd8dfd397280.snapshot"
+      }
+  } ],
+  "status" : "READY",
+  "create_time" : null,
+  "notify_list" : null
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/examples/test_case_data/localmeta/cube/test_kylin_cube_without_slr_empty.json
----------------------------------------------------------------------
diff --git a/examples/test_case_data/localmeta/cube/test_kylin_cube_without_slr_empty.json b/examples/test_case_data/localmeta/cube/test_kylin_cube_without_slr_empty.json
new file mode 100644
index 0000000..6bd566a
--- /dev/null
+++ b/examples/test_case_data/localmeta/cube/test_kylin_cube_without_slr_empty.json
@@ -0,0 +1,10 @@
+{
+  "uuid" : "daa53e80-41be-49a5-90ca-9fb7294db186",
+  "last_modified" : 0,
+  "name" : "test_kylin_cube_without_slr_empty",
+  "owner" : null,
+  "version" : null,
+  "descriptor" : "test_kylin_cube_without_slr_desc",
+  "segments" : [ ],
+  "create_time" : null
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/examples/test_case_data/localmeta/cube/test_kylin_cube_without_slr_left_join_empty.json
----------------------------------------------------------------------
diff --git a/examples/test_case_data/localmeta/cube/test_kylin_cube_without_slr_left_join_empty.json b/examples/test_case_data/localmeta/cube/test_kylin_cube_without_slr_left_join_empty.json
new file mode 100644
index 0000000..b4b5fd7
--- /dev/null
+++ b/examples/test_case_data/localmeta/cube/test_kylin_cube_without_slr_left_join_empty.json
@@ -0,0 +1,11 @@
+{
+  "uuid" : "dddd3e80-41be-49a5-90ca-9fb7294db186",
+  "last_modified" : 0,
+  "name" : "test_kylin_cube_without_slr_left_join_empty",
+  "owner" : null,
+  "version" : null,
+  "descriptor" : "test_kylin_cube_without_slr_left_join_desc",
+  "segments" : [ ],
+  "status" : "DISABLED",
+  "create_time" : null
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/examples/test_case_data/localmeta/cube/test_kylin_cube_without_slr_left_join_ready.json
----------------------------------------------------------------------
diff --git a/examples/test_case_data/localmeta/cube/test_kylin_cube_without_slr_left_join_ready.json b/examples/test_case_data/localmeta/cube/test_kylin_cube_without_slr_left_join_ready.json
new file mode 100644
index 0000000..c84a0bd
--- /dev/null
+++ b/examples/test_case_data/localmeta/cube/test_kylin_cube_without_slr_left_join_ready.json
@@ -0,0 +1,43 @@
+{
+  "uuid" : "mmmm3e80-41be-49a5-90ca-9fb7294db186",
+  "last_modified" : 1404097288087,
+  "name" : "test_kylin_cube_without_slr_left_join_ready",
+  "owner" : null,
+  "version" : null,
+  "descriptor" : "test_kylin_cube_without_slr_left_join_desc",
+  "cost" : 10,
+  "segments" : [ {
+    "name" : null,
+    "storage_location_identifier" : "KYLIN-CUBE-TEST_KYLIN_CUBE_WITHOUT_SLR_LEFT_JOIN_READY-1EF30C45-17A4-4350-A032-A33B3B25E5B8",
+    "date_range_start" : null,
+    "date_range_end" : null,
+    "status" : "READY",
+    "size_kb" : 6060,
+    "source_records" : 10000,
+    "source_records_size" : 610288,
+    "last_build_time" : 1404097287905,
+    "last_build_job_id" : "1ef30c45-17a4-4350-a032-a33b3b25e5b8",
+    "binary_signature" : null,
+      "dictionaries" : {
+        "TEST_CATEGORY_GROUPINGS/CATEG_LVL2_NAME" : "/dict/TEST_CATEGORY_GROUPINGS/CATEG_LVL2_NAME/eaed91b0-4182-4ee5-a733-1047a622ee29.dict",
+        "TEST_KYLIN_FACT/SLR_SEGMENT_CD" : "/dict/TEST_SELLER_TYPE_DIM/SELLER_TYPE_CD/14fe66b3-5956-498c-bd93-40182cac5510.dict",
+        "TEST_KYLIN_FACT/LSTG_SITE_ID" : "/dict/TEST_SITES/SITE_ID/0bec6bb3-1b0d-469c-8289-b8c4ca5d5001.dict",
+        "TEST_SELLER_TYPE_DIM/SELLER_TYPE_CD" : "/dict/TEST_SELLER_TYPE_DIM/SELLER_TYPE_CD/2a44ff38-f64b-42e7-9fcf-66afccac8047.dict",
+        "TEST_KYLIN_FACT/CAL_DT" : "/dict/PREDEFINED/date(yyyy-mm-dd)/64ac4f82-f2af-476e-85b9-f0805001014e.dict",
+        "TEST_CATEGORY_GROUPINGS/CATEG_LVL3_NAME" : "/dict/TEST_CATEGORY_GROUPINGS/CATEG_LVL3_NAME/8f92faa4-7d2d-455c-8623-6e1d1b272afe.dict",
+        "TEST_CAL_DT/CAL_DT" : "/dict/TEST_CAL_DT/CAL_DT/aa634e3e-22d3-4cc2-9de5-085e9ac35c1f.dict",
+        "TEST_KYLIN_FACT/LEAF_CATEG_ID" : "/dict/TEST_CATEGORY_GROUPINGS/LEAF_CATEG_ID/2602386c-debb-4968-8d2f-b52b8215e385.dict",
+        "TEST_SITES/SITE_ID" : "/dict/TEST_SITES/SITE_ID/ff7e8943-ac0f-4e66-b9ed-510f6a0b875d.dict",
+        "TEST_CATEGORY_GROUPINGS/META_CATEG_NAME" : "/dict/TEST_CATEGORY_GROUPINGS/META_CATEG_NAME/0410d2c4-4686-40bc-ba14-170042a2de94.dict"
+      },
+      "snapshots" : {
+        "TEST_SELLER_TYPE_DIM" : "/table_snapshot/TEST_SELLER_TYPE_DIM.csv/b43dd3f1-9a34-4721-8abc-df90962e94d8.snapshot",
+        "TEST_CAL_DT" : "/table_snapshot/TEST_CAL_DT.csv/e5f004b5-0be1-4ccf-8950-20f8f86c1999.snapshot",
+        "TEST_CATEGORY_GROUPINGS" : "/table_snapshot/TEST_CATEGORY_GROUPINGS.csv/89715f33-15c9-4745-83f9-f2b9817d9100.snapshot",
+        "TEST_SITES" : "/table_snapshot/TEST_SITES.csv/7d70b82e-43f6-4999-b012-0e91a9bb8408.snapshot"
+      }
+  } ],
+  "status" : "READY",
+  "create_time" : null,
+  "notify_list" : null
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/examples/test_case_data/localmeta/cube/test_kylin_cube_without_slr_left_join_ready_2_segments.json
----------------------------------------------------------------------
diff --git a/examples/test_case_data/localmeta/cube/test_kylin_cube_without_slr_left_join_ready_2_segments.json b/examples/test_case_data/localmeta/cube/test_kylin_cube_without_slr_left_join_ready_2_segments.json
new file mode 100644
index 0000000..39a2dab
--- /dev/null
+++ b/examples/test_case_data/localmeta/cube/test_kylin_cube_without_slr_left_join_ready_2_segments.json
@@ -0,0 +1,73 @@
+{
+  "uuid" : "1eaca32a-a33e-4b69-83dd-0bb8b1f8c5ff",
+  "last_modified" : 1404098141020,
+  "name" : "test_kylin_cube_without_slr_left_join_ready_2_segments",
+  "owner" : null,
+  "version" : null,
+  "descriptor" : "test_kylin_cube_without_slr_left_join_desc",
+  "cost" : 50,
+  "segments" : [ {
+    "name" : "19691231160000_20131112000000",
+    "storage_location_identifier" : "KYLIN-CUBE-TEST_KYLIN_CUBE_WITH_SLR_READY-F24668F6-DCFF-4CB6-A89B-77F1119DF8FA",
+    "date_range_start" : 1384240200000,
+    "date_range_end" : 1384243200000,
+    "status" : "READY",
+    "size_kb" : 7801,
+    "source_records" : 10000,
+    "source_records_size" : 608012,
+    "last_build_time" : 1404098140902,
+    "last_build_job_id" : "f24668f6-dcff-4cb6-a89b-77f1119df8fa",
+    "binary_signature" : null,
+      "dictionaries" : {
+        "TEST_CATEGORY_GROUPINGS/CATEG_LVL2_NAME" : "/dict/TEST_CATEGORY_GROUPINGS/CATEG_LVL2_NAME/16d8185c-ee6b-4f8c-a919-756d9809f937.dict",
+        "TEST_KYLIN_FACT/SLR_SEGMENT_CD" : "/dict/TEST_SELLER_TYPE_DIM/SELLER_TYPE_CD/c6bf9b51-6e90-4337-8082-4e2fdf78307f.dict",
+        "TEST_KYLIN_FACT/LSTG_SITE_ID" : "/dict/TEST_SITES/SITE_ID/652bd393-678a-4f16-a504-fd8ce1229355.dict",
+        "TEST_SELLER_TYPE_DIM/SELLER_TYPE_CD" : "/dict/TEST_SELLER_TYPE_DIM/SELLER_TYPE_CD/2a44ff38-f64b-42e7-9fcf-66afccac8047.dict",
+        "TEST_KYLIN_FACT/CAL_DT" : "/dict/PREDEFINED/date(yyyy-mm-dd)/64ac4f82-f2af-476e-85b9-f0805001014e.dict",
+        "TEST_CATEGORY_GROUPINGS/CATEG_LVL3_NAME" : "/dict/TEST_CATEGORY_GROUPINGS/CATEG_LVL3_NAME/ad09f2d5-054a-4e1b-a776-7cc07399a6c1.dict",
+        "TEST_CAL_DT/CAL_DT" : "/dict/TEST_CAL_DT/CAL_DT/ed0c3451-593c-494c-9019-64f63fcb0b8e.dict",
+        "TEST_KYLIN_FACT/LEAF_CATEG_ID" : "/dict/TEST_CATEGORY_GROUPINGS/LEAF_CATEG_ID/4243889f-bc81-4807-a975-7041bbbf35e7.dict",
+        "TEST_SITES/SITE_ID" : "/dict/TEST_SITES/SITE_ID/ff7e8943-ac0f-4e66-b9ed-510f6a0b875d.dict",
+        "TEST_CATEGORY_GROUPINGS/META_CATEG_NAME" : "/dict/TEST_CATEGORY_GROUPINGS/META_CATEG_NAME/aceae914-4246-4251-a0c2-692fe7a300df.dict"
+      },
+      "snapshots" : {
+        "TEST_SELLER_TYPE_DIM" : "/table_snapshot/TEST_SELLER_TYPE_DIM.csv/4fe75ccd-9b24-4cdf-ac9d-b4038e947f89.snapshot",
+        "TEST_CAL_DT" : "/table_snapshot/TEST_CAL_DT.csv/8ff1339e-f804-47f3-b42c-1d4fa4ff0cf7.snapshot",
+        "TEST_CATEGORY_GROUPINGS" : "/table_snapshot/TEST_CATEGORY_GROUPINGS.csv/e172b442-ae10-447e-9071-c7dbb2bb38cc.snapshot",
+        "TEST_SITES" : "/table_snapshot/TEST_SITES.csv/28130338-fcf4-429e-91b0-cd8dfd397280.snapshot"
+      }
+  }, {
+    "name" : "20131112000000_20131212000000",
+    "storage_location_identifier" : "KYLIN-CUBE-TEST_KYLIN_CUBE_WITH_SLR_READY-F24668F6-DCFF-4CB6-A89B-77F1119DF8FB",
+    "date_range_start" : 1384243200000,
+    "date_range_end" : 1386835200000,
+    "status" : "READY",
+    "size_kb" : 7801,
+    "source_records" : 10000,
+    "source_records_size" : 608012,
+    "last_build_time" : 1404098140902,
+    "last_build_job_id" : "f24668f6-dcff-4cb6-a89b-77f1119df8fb",
+    "binary_signature" : null,
+      "dictionaries" : {
+        "TEST_CATEGORY_GROUPINGS/CATEG_LVL2_NAME" : "/dict/TEST_CATEGORY_GROUPINGS/CATEG_LVL2_NAME/16d8185c-ee6b-4f8c-a919-756d9809f937.dict",
+        "TEST_KYLIN_FACT/SLR_SEGMENT_CD" : "/dict/TEST_SELLER_TYPE_DIM/SELLER_TYPE_CD/c6bf9b51-6e90-4337-8082-4e2fdf78307f.dict",
+        "TEST_KYLIN_FACT/LSTG_SITE_ID" : "/dict/TEST_SITES/SITE_ID/652bd393-678a-4f16-a504-fd8ce1229355.dict",
+        "TEST_SELLER_TYPE_DIM/SELLER_TYPE_CD" : "/dict/TEST_SELLER_TYPE_DIM/SELLER_TYPE_CD/2a44ff38-f64b-42e7-9fcf-66afccac8047.dict",
+        "TEST_KYLIN_FACT/CAL_DT" : "/dict/PREDEFINED/date(yyyy-mm-dd)/64ac4f82-f2af-476e-85b9-f0805001014e.dict",
+        "TEST_CATEGORY_GROUPINGS/CATEG_LVL3_NAME" : "/dict/TEST_CATEGORY_GROUPINGS/CATEG_LVL3_NAME/ad09f2d5-054a-4e1b-a776-7cc07399a6c1.dict",
+        "TEST_CAL_DT/CAL_DT" : "/dict/TEST_CAL_DT/CAL_DT/ed0c3451-593c-494c-9019-64f63fcb0b8e.dict",
+        "TEST_KYLIN_FACT/LEAF_CATEG_ID" : "/dict/TEST_CATEGORY_GROUPINGS/LEAF_CATEG_ID/4243889f-bc81-4807-a975-7041bbbf35e7.dict",
+        "TEST_SITES/SITE_ID" : "/dict/TEST_SITES/SITE_ID/ff7e8943-ac0f-4e66-b9ed-510f6a0b875d.dict",
+        "TEST_CATEGORY_GROUPINGS/META_CATEG_NAME" : "/dict/TEST_CATEGORY_GROUPINGS/META_CATEG_NAME/aceae914-4246-4251-a0c2-692fe7a300df.dict"
+      },
+      "snapshots" : {
+        "TEST_SELLER_TYPE_DIM" : "/table_snapshot/TEST_SELLER_TYPE_DIM.csv/4fe75ccd-9b24-4cdf-ac9d-b4038e947f89.snapshot",
+        "TEST_CAL_DT" : "/table_snapshot/TEST_CAL_DT.csv/8ff1339e-f804-47f3-b42c-1d4fa4ff0cf7.snapshot",
+        "TEST_CATEGORY_GROUPINGS" : "/table_snapshot/TEST_CATEGORY_GROUPINGS.csv/e172b442-ae10-447e-9071-c7dbb2bb38cc.snapshot",
+        "TEST_SITES" : "/table_snapshot/TEST_SITES.csv/28130338-fcf4-429e-91b0-cd8dfd397280.snapshot"
+      }
+  } ],
+  "status" : "READY",
+  "create_time" : null,
+  "notify_list" : null
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/examples/test_case_data/localmeta/cube/test_kylin_cube_without_slr_ready.json
----------------------------------------------------------------------
diff --git a/examples/test_case_data/localmeta/cube/test_kylin_cube_without_slr_ready.json b/examples/test_case_data/localmeta/cube/test_kylin_cube_without_slr_ready.json
new file mode 100644
index 0000000..ec88d1a
--- /dev/null
+++ b/examples/test_case_data/localmeta/cube/test_kylin_cube_without_slr_ready.json
@@ -0,0 +1,43 @@
+{
+  "uuid" : "daa53e80-41be-49a5-90ca-9fb7294db186",
+  "last_modified" : 1404098303976,
+  "name" : "test_kylin_cube_without_slr_ready",
+  "owner" : null,
+  "version" : null,
+  "descriptor" : "test_kylin_cube_without_slr_desc",
+  "cost" : 10,
+  "segments" : [ {
+    "name" : null,
+    "storage_location_identifier" : "KYLIN-CUBE-TEST_KYLIN_CUBE_WITHOUT_SLR_READY-619D9B4A-DA56-48CA-9C58-29DC8323B200",
+    "date_range_start" : null,
+    "date_range_end" : null,
+    "status" : "READY",
+    "size_kb" : 5600,
+    "source_records" : 10000,
+    "source_records_size" : 608012,
+    "last_build_time" : 1404098303829,
+    "last_build_job_id" : "619d9b4a-da56-48ca-9c58-29dc8323b200",
+    "binary_signature" : null,
+      "dictionaries" : {
+        "TEST_CATEGORY_GROUPINGS/CATEG_LVL2_NAME" : "/dict/TEST_CATEGORY_GROUPINGS/CATEG_LVL2_NAME/16d8185c-ee6b-4f8c-a919-756d9809f937.dict",
+        "TEST_KYLIN_FACT/SLR_SEGMENT_CD" : "/dict/TEST_SELLER_TYPE_DIM/SELLER_TYPE_CD/c6bf9b51-6e90-4337-8082-4e2fdf78307f.dict",
+        "TEST_KYLIN_FACT/LSTG_SITE_ID" : "/dict/TEST_SITES/SITE_ID/652bd393-678a-4f16-a504-fd8ce1229355.dict",
+        "TEST_SELLER_TYPE_DIM/SELLER_TYPE_CD" : "/dict/TEST_SELLER_TYPE_DIM/SELLER_TYPE_CD/2a44ff38-f64b-42e7-9fcf-66afccac8047.dict",
+        "TEST_KYLIN_FACT/CAL_DT" : "/dict/PREDEFINED/date(yyyy-mm-dd)/64ac4f82-f2af-476e-85b9-f0805001014e.dict",
+        "TEST_CATEGORY_GROUPINGS/CATEG_LVL3_NAME" : "/dict/TEST_CATEGORY_GROUPINGS/CATEG_LVL3_NAME/ad09f2d5-054a-4e1b-a776-7cc07399a6c1.dict",
+        "TEST_CAL_DT/CAL_DT" : "/dict/TEST_CAL_DT/CAL_DT/aa634e3e-22d3-4cc2-9de5-085e9ac35c1f.dict",
+        "TEST_KYLIN_FACT/LEAF_CATEG_ID" : "/dict/TEST_CATEGORY_GROUPINGS/LEAF_CATEG_ID/494683c1-52d1-4d9a-a3f7-8c53b30fd898.dict",
+        "TEST_SITES/SITE_ID" : "/dict/TEST_SITES/SITE_ID/ff7e8943-ac0f-4e66-b9ed-510f6a0b875d.dict",
+        "TEST_CATEGORY_GROUPINGS/META_CATEG_NAME" : "/dict/TEST_CATEGORY_GROUPINGS/META_CATEG_NAME/b298089f-9656-4693-b9b2-8fea46f06dd5.dict"
+      },
+      "snapshots" : {
+        "TEST_SELLER_TYPE_DIM" : "/table_snapshot/TEST_SELLER_TYPE_DIM.csv/4fe75ccd-9b24-4cdf-ac9d-b4038e947f89.snapshot",
+        "TEST_CAL_DT" : "/table_snapshot/TEST_CAL_DT.csv/9a1fbc0d-8790-4f32-99e1-7c60a9d124d4.snapshot",
+        "TEST_CATEGORY_GROUPINGS" : "/table_snapshot/TEST_CATEGORY_GROUPINGS.csv/e172b442-ae10-447e-9071-c7dbb2bb38cc.snapshot",
+        "TEST_SITES" : "/table_snapshot/TEST_SITES.csv/28130338-fcf4-429e-91b0-cd8dfd397280.snapshot"
+      }
+  } ],
+  "status" : "READY",
+  "create_time" : null,
+  "notify_list" : null
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/examples/test_case_data/localmeta/cube_desc/test_kylin_cube_ii.json
----------------------------------------------------------------------
diff --git a/examples/test_case_data/localmeta/cube_desc/test_kylin_cube_ii.json b/examples/test_case_data/localmeta/cube_desc/test_kylin_cube_ii.json
new file mode 100644
index 0000000..f79dcf1
--- /dev/null
+++ b/examples/test_case_data/localmeta/cube_desc/test_kylin_cube_ii.json
@@ -0,0 +1,130 @@
+{
+  "uuid" : "ac13878c-8767-4454-8aab-1007e274fa23",
+  "name" : "test_kylin_cube_ii",
+  "description" : null,
+  "dimensions" : [ {
+    "id" : 1,
+    "name" : "TRANS_ID",
+    "join" : null,
+    "hierarchy" : null,
+    "table" : "TEST_KYLIN_FACT",
+    "column" : "TRANS_ID",
+    "datatype" : "bigint",
+    "derived" : null
+  }, {
+    "id" : 2,
+    "name" : "CAL_DT",
+    "join" : {
+      "type" : "left",
+      "primary_key" : [ "CAL_DT" ],
+      "foreign_key" : [ "CAL_DT" ]
+    },
+    "hierarchy" : null,
+    "table" : "TEST_CAL_DT",
+    "column" : "{FK}",
+    "datatype" : "date",
+    "derived" : [ "CAL_DT", "WEEK_BEG_DT", "YEAR_BEG_DT", "QTR_BEG_DT", "MONTH_BEG_DT" ]
+  }, {
+    "id" : 3,
+    "name" : "LSTG_FORMAT_NAME",
+    "join" : null,
+    "hierarchy" : null,
+    "table" : "TEST_KYLIN_FACT",
+    "column" : "LSTG_FORMAT_NAME",
+    "datatype" : "string",
+    "derived" : null
+  }, {
+    "id" : 4,
+    "name" : "LEAF_CATEG_ID",
+    "join" : {
+      "type" : "left",
+      "primary_key" : [ "LEAF_CATEG_ID", "SITE_ID" ],
+      "foreign_key" : [ "LEAF_CATEG_ID", "LSTG_SITE_ID" ]
+    },
+    "hierarchy" : null,
+    "table" : "TEST_CATEGORY_GROUPINGS",
+    "column" : "{FK}",
+    "datatype" : null,
+    "derived" : [ "USER_DEFINED_FIELD1", "USER_DEFINED_FIELD3", "UPD_DATE", "UPD_USER" ]
+  }, {
+    "id" : 5,
+    "name" : "LSTG_SITE_ID",
+    "join" : {
+      "type" : "left",
+      "primary_key" : [ "SITE_ID" ],
+      "foreign_key" : [ "LSTG_SITE_ID" ]
+    },
+    "hierarchy" : null,
+    "table" : "TEST_SITES",
+    "column" : "{FK}",
+    "datatype" : "string",
+    "derived" : [ "SITE_NAME", "CRE_USER" ]
+  }, {
+    "id" : 6,
+    "name" : "SLR_SEGMENT_CD",
+    "join" : {
+      "type" : "left",
+      "primary_key" : [ "SELLER_TYPE_CD" ],
+      "foreign_key" : [ "SLR_SEGMENT_CD" ]
+    },
+    "hierarchy" : null,
+    "table" : "TEST_SELLER_TYPE_DIM",
+    "column" : "{FK}",
+    "datatype" : "string",
+    "derived" : [ "SELLER_TYPE_DESC" ]
+  }, {
+    "id" : 7,
+    "name" : "PRICE",
+    "join" : null,
+    "hierarchy" : null,
+    "table" : "TEST_KYLIN_FACT",
+    "column" : "PRICE",
+    "datatype" : "decimal",
+    "derived" : null
+  }, {
+    "id" : 8,
+    "name" : "ITEM_COUNT",
+    "join" : null,
+    "hierarchy" : null,
+    "table" : "TEST_KYLIN_FACT",
+    "column" : "ITEM_COUNT",
+    "datatype" : "bigint",
+    "derived" : null
+  }, {
+    "id" : 9,
+    "name" : "SELLER_ID",
+    "join" : null,
+    "hierarchy" : null,
+    "table" : "TEST_KYLIN_FACT",
+    "column" : "SELLER_ID",
+    "datatype" : "string",
+    "derived" : null
+  } ],
+  "measures" : null,
+  "rowkey":{
+    "rowkey_columns": [
+      { "column": "TRANS_ID",         "dictionary": "string" },
+      { "column": "CAL_DT",           "dictionary": "date(yyyy-mm-dd)" },
+      { "column": "LSTG_FORMAT_NAME", "dictionary": "string" },
+      { "column": "LEAF_CATEG_ID",    "dictionary": "string" },
+      { "column": "LSTG_SITE_ID",     "dictionary": "string" },
+      { "column": "SLR_SEGMENT_CD",   "dictionary": "string" },
+      { "column": "PRICE",            "dictionary": "string" },
+      { "column": "ITEM_COUNT",       "dictionary": "string" },
+      { "column": "SELLER_ID",        "dictionary": "string" }
+    ],
+	"aggregation_groups": null
+  },
+  "signature" : null,
+  "capacity" : "MEDIUM",
+  "last_modified" : 1408328222841,
+  "fact_table" : "TEST_KYLIN_FACT",
+  "filter_condition" : null,
+  "cube_partition_desc" : {
+    "partition_date_column" : null,
+    "partition_date_start" : 0,
+    "cube_partition_type" : "APPEND"
+  },
+  "hbase_mapping" : null,
+  "notify_list" : null
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/examples/test_case_data/localmeta/cube_desc/test_kylin_cube_with_slr_desc.json
----------------------------------------------------------------------
diff --git a/examples/test_case_data/localmeta/cube_desc/test_kylin_cube_with_slr_desc.json b/examples/test_case_data/localmeta/cube_desc/test_kylin_cube_with_slr_desc.json
new file mode 100644
index 0000000..e7f8d79
--- /dev/null
+++ b/examples/test_case_data/localmeta/cube_desc/test_kylin_cube_with_slr_desc.json
@@ -0,0 +1,176 @@
+{
+  "uuid": "a24ca905-1fc6-4f67-985c-38fa5aeafd92",
+  "name": "test_kylin_cube_with_slr_desc",
+  "fact_table": "test_kylin_fact",
+  "filter_condition": null,
+  "cube_partition_desc": {
+    "partition_date_column": "test_kylin_fact.cal_dt",
+    "partition_date_start": 0,
+    "cube_partition_type": "APPEND"
+  },
+  "dimensions": [
+    {
+      "id": "1",
+      "name": "cal_dt",
+      "datatype": "date",
+      "table": "test_cal_dt",
+      "column": "{FK}",
+      "derived": ["week_beg_dt"],
+      "join": {
+        "type": "inner",
+        "primary_key": ["cal_dt"],
+        "foreign_key": ["cal_dt"]
+      }
+    },
+    {
+      "id": "2",
+      "name": "category",
+      "table": "test_category_groupings",
+      "column": "{FK}",
+      "derived": ["USER_DEFINED_FIELD1", "USER_DEFINED_FIELD3", "UPD_DATE", "UPD_USER"],
+      "join": {
+        "type": "inner",
+        "primary_key": ["leaf_categ_id", "site_id"],
+        "foreign_key": ["leaf_categ_id", "lstg_site_id"]
+      },
+      "hierarchy": [
+        {
+          "level": "1",
+          "column": "meta_categ_name"
+        },
+        {
+          "level": "2",
+          "column": "categ_lvl2_name"
+        },
+        {
+          "level": "3",
+          "column": "categ_lvl3_name"
+        }
+      ]
+    },
+    {
+      "id": "3",
+      "name": "lstg_format_name",
+      "datatype": "string",
+      "table": "test_kylin_fact",
+      "column": "lstg_format_name"
+    },
+    {
+      "id": "4",
+      "name": "site_id",
+      "datatype": "string",
+      "table": "test_sites",
+      "column": "{FK}",
+      "derived": ["site_name", "cre_user"],
+      "join": {
+        "type": "inner",
+        "primary_key": ["site_id"],
+        "foreign_key": ["lstg_site_id"]
+      }
+    },
+    {
+      "id": "5",
+      "name": "seller_type_cd",
+      "datatype": "string",
+      "table": "test_seller_type_dim",
+      "column": "{FK}",
+      "derived": ["seller_type_desc"],
+      "join": {
+        "type": "inner",
+        "primary_key": ["seller_type_cd"],
+        "foreign_key": ["slr_segment_cd"]
+      }
+    },
+    {
+      "id": "6",
+      "name": "seller_id",
+      "datatype": "string",
+      "table": "test_kylin_fact",
+      "column": "seller_id"
+    }
+  ],
+  "measures": [
+    {
+      "id": "1",
+      "name": "gmv_sum",
+      "function": {
+        "expression": "sum",
+        "returntype": "decimal",
+        "parameter": {
+          "type": "column",
+          "value": "price"
+        }
+      }
+    },
+    {
+      "id": "2",
+      "name": "gmv_min",
+      "function": {
+        "expression": "min",
+        "returntype": "decimal",
+        "parameter": {
+          "type": "column",
+          "value": "price"
+        }
+      }
+    },
+    {
+      "id": "3",
+      "name": "gmv_max",
+      "function": {
+        "expression": "max",
+        "returntype": "decimal",
+        "parameter": {
+          "type": "column",
+          "value": "price"
+        }
+      }
+    },
+    {
+      "id": "4",
+      "name": "trans_cnt",
+      "function": {
+        "expression": "count",
+        "returntype": "bigint",
+        "parameter": {
+          "type": "constant",
+          "value": "1"
+        }
+      }
+    }
+  ],
+  "rowkey":{
+    "rowkey_columns": [
+      { "column": "seller_id",        "length": 18,  "mandatory": "true" },
+      { "column": "cal_dt",           "dictionary": "date(yyyy-mm-dd)" },
+      { "column": "leaf_categ_id",    "dictionary": "string" },
+      { "column": "meta_categ_name",  "dictionary": "string" },
+      { "column": "categ_lvl2_name",  "dictionary": "string" },
+      { "column": "categ_lvl3_name",  "dictionary": "string" },
+      { "column": "lstg_format_name", "length": 12  },
+      { "column": "lstg_site_id",     "dictionary": "string" },
+      { "column": "slr_segment_cd",   "dictionary": "string" }
+    ],
+	"aggregation_groups": [
+	  ["leaf_categ_id","meta_categ_name","categ_lvl2_name","categ_lvl3_name","cal_dt"]
+	]
+  },
+  "hbase_mapping": {
+    "column_family": [
+      {
+        "name": "f1",
+        "columns": [
+          {
+            "qualifier": "m",
+            "measure_refs": [
+              "gmv_sum",
+              "gmv_min",
+              "gmv_max",
+              "trans_cnt"
+            ]
+          }
+        ]
+      }
+    ]
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/examples/test_case_data/localmeta/cube_desc/test_kylin_cube_with_slr_left_join_desc.json
----------------------------------------------------------------------
diff --git a/examples/test_case_data/localmeta/cube_desc/test_kylin_cube_with_slr_left_join_desc.json b/examples/test_case_data/localmeta/cube_desc/test_kylin_cube_with_slr_left_join_desc.json
new file mode 100644
index 0000000..74f3996
--- /dev/null
+++ b/examples/test_case_data/localmeta/cube_desc/test_kylin_cube_with_slr_left_join_desc.json
@@ -0,0 +1,186 @@
+{
+  "uuid": "bbbba905-1fc6-4f67-985c-38fa5aeafd92",
+  "name": "test_kylin_cube_with_slr_left_join_desc",
+  "fact_table": "test_kylin_fact",
+  "cube_partition_desc": {
+    "partition_date_column": "test_kylin_fact.cal_dt",
+    "partition_date_start": 0,
+    "cube_partition_type": "APPEND"
+  },
+  "filter_condition": null,
+  "dimensions": [
+    {
+      "id": "1",
+      "name": "cal_dt",
+      "datatype": "date",
+      "table": "test_cal_dt",
+      "column": "{FK}",
+      "derived": ["week_beg_dt"],
+      "join": {
+        "type": "left",
+        "primary_key": ["cal_dt"],
+        "foreign_key": ["cal_dt"]
+      }
+    },
+    {
+      "id": "2",
+      "name": "category",
+      "table": "test_category_groupings",
+      "join": {
+        "type": "left",
+        "primary_key": ["leaf_categ_id", "site_id"],
+        "foreign_key": ["leaf_categ_id", "lstg_site_id"]
+      },
+      "hierarchy": [
+        {
+          "level": "1",
+          "column": "meta_categ_name"
+        },
+        {
+          "level": "2",
+          "column": "categ_lvl2_name"
+        },
+        {
+          "level": "3",
+          "column": "categ_lvl3_name"
+        }
+      ]
+    },
+    {
+      "id": "3",
+      "name": "category_derived",
+      "table": "test_category_groupings",
+      "column": "{FK}",
+      "derived": ["USER_DEFINED_FIELD1", "USER_DEFINED_FIELD3", "UPD_DATE", "UPD_USER"],
+      "join": {
+        "type": "left",
+        "primary_key": ["leaf_categ_id", "site_id"],
+        "foreign_key": ["leaf_categ_id", "lstg_site_id"]
+      }
+    },
+    {
+      "id": "4",
+      "name": "lstg_format_name",
+      "datatype": "string",
+      "table": "test_kylin_fact",
+      "column": "lstg_format_name"
+    },
+    {
+      "id": "5",
+      "name": "site_id",
+      "datatype": "string",
+      "table": "test_sites",
+      "column": "{FK}",
+      "derived": ["site_name", "cre_user"],
+      "join": {
+        "type": "left",
+        "primary_key": ["site_id"],
+        "foreign_key": ["lstg_site_id"]
+      }
+    },
+    {
+      "id": "6",
+      "name": "seller_type_cd",
+      "datatype": "string",
+      "table": "test_seller_type_dim",
+      "column": "{FK}",
+      "derived": ["seller_type_desc"],
+      "join": {
+        "type": "left",
+        "primary_key": ["seller_type_cd"],
+        "foreign_key": ["slr_segment_cd"]
+      }
+    },
+    {
+      "id": "7",
+      "name": "seller_id",
+      "datatype": "string",
+      "table": "test_kylin_fact",
+      "column": "seller_id"
+    }
+  ],
+  "measures": [
+    {
+      "id": "1",
+      "name": "gmv_sum",
+      "function": {
+        "expression": "sum",
+        "returntype": "decimal",
+        "parameter": {
+          "type": "column",
+          "value": "price"
+        }
+      }
+    },
+    {
+      "id": "2",
+      "name": "gmv_min",
+      "function": {
+        "expression": "min",
+        "returntype": "decimal",
+        "parameter": {
+          "type": "column",
+          "value": "price"
+        }
+      }
+    },
+    {
+      "id": "3",
+      "name": "gmv_max",
+      "function": {
+        "expression": "max",
+        "returntype": "decimal",
+        "parameter": {
+          "type": "column",
+          "value": "price"
+        }
+      }
+    },
+    {
+      "id": "4",
+      "name": "trans_cnt",
+      "function": {
+        "expression": "count",
+        "returntype": "bigint",
+        "parameter": {
+          "type": "constant",
+          "value": "1"
+        }
+      }
+    }
+  ],
+  "rowkey":{
+    "rowkey_columns": [
+      { "column": "seller_id",        "length": 18,  "mandatory": "true" },
+      { "column": "cal_dt",           "dictionary": "date(yyyy-mm-dd)" },
+      { "column": "leaf_categ_id",    "dictionary": "string" },
+      { "column": "meta_categ_name",  "dictionary": "string" },
+      { "column": "categ_lvl2_name",  "dictionary": "string" },
+      { "column": "categ_lvl3_name",  "dictionary": "string" },
+      { "column": "lstg_format_name",  "length": 12  },
+      { "column": "lstg_site_id",     "dictionary": "string" },
+      { "column": "slr_segment_cd",   "dictionary": "string" }
+    ],
+	"aggregation_groups": [
+	  ["leaf_categ_id","meta_categ_name","categ_lvl2_name","categ_lvl3_name","cal_dt"]
+	]
+  },
+  "hbase_mapping": {
+    "column_family": [
+      {
+        "name": "f1",
+        "columns": [
+          {
+            "qualifier": "m",
+            "measure_refs": [
+              "gmv_sum",
+              "gmv_min",
+              "gmv_max",
+              "trans_cnt"
+            ]
+          }
+        ]
+      }
+    ]
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/examples/test_case_data/localmeta/cube_desc/test_kylin_cube_without_slr_desc.json
----------------------------------------------------------------------
diff --git a/examples/test_case_data/localmeta/cube_desc/test_kylin_cube_without_slr_desc.json b/examples/test_case_data/localmeta/cube_desc/test_kylin_cube_without_slr_desc.json
new file mode 100644
index 0000000..4814020
--- /dev/null
+++ b/examples/test_case_data/localmeta/cube_desc/test_kylin_cube_without_slr_desc.json
@@ -0,0 +1,285 @@
+{
+  "uuid": "9ac9b7a8-3929-4dff-b59d-2100aadc8dbf",
+  "name": "test_kylin_cube_without_slr_desc",
+  "capacity": "SMALL",
+  "fact_table": "test_kylin_fact",
+  "cube_partition_desc": {
+    "partition_date_column": null,
+    "partition_date_start": null,
+    "cube_partition_type": "APPEND"
+  },
+  "filter_condition": null,
+  "dimensions": [
+    {
+      "id": "1",
+      "name": "cal_dt",
+      "datatype": "date",
+      "table": "test_cal_dt",
+      "column": "{FK}",
+      "derived": [
+        "week_beg_dt"
+      ],
+      "join": {
+        "type": "inner",
+        "primary_key": [
+          "cal_dt"
+        ],
+        "foreign_key": [
+          "cal_dt"
+        ]
+      }
+    },
+    {
+      "id": "2",
+      "name": "category",
+      "table": "test_category_groupings",
+      "column": "{FK}",
+      "derived": [
+        "USER_DEFINED_FIELD1",
+        "USER_DEFINED_FIELD3",
+        "UPD_DATE",
+        "UPD_USER"
+      ],
+      "join": {
+        "type": "inner",
+        "primary_key": [
+          "leaf_categ_id",
+          "site_id"
+        ],
+        "foreign_key": [
+          "leaf_categ_id",
+          "lstg_site_id"
+        ]
+      },
+      "hierarchy": [
+        {
+          "level": "1",
+          "column": "meta_categ_name"
+        },
+        {
+          "level": "2",
+          "column": "categ_lvl2_name"
+        },
+        {
+          "level": "3",
+          "column": "categ_lvl3_name"
+        }
+      ]
+    },
+    {
+      "id": "3",
+      "name": "lstg_format_name",
+      "datatype": "string",
+      "table": "test_kylin_fact",
+      "column": "lstg_format_name"
+    },
+    {
+      "id": "4",
+      "name": "site_id",
+      "datatype": "string",
+      "table": "test_sites",
+      "column": "{FK}",
+      "derived": [
+        "site_name",
+        "cre_user"
+      ],
+      "join": {
+        "type": "inner",
+        "primary_key": [
+          "site_id"
+        ],
+        "foreign_key": [
+          "lstg_site_id"
+        ]
+      }
+    },
+    {
+      "id": "5",
+      "name": "seller_type_cd",
+      "datatype": "string",
+      "table": "test_seller_type_dim",
+      "column": "{FK}",
+      "derived": [
+        "seller_type_desc"
+      ],
+      "join": {
+        "type": "inner",
+        "primary_key": [
+          "seller_type_cd"
+        ],
+        "foreign_key": [
+          "slr_segment_cd"
+        ]
+      }
+    }
+  ],
+  "measures": [
+    {
+      "id": "1",
+      "name": "gmv_sum",
+      "function": {
+        "expression": "sum",
+        "returntype": "decimal",
+        "parameter": {
+          "type": "column",
+          "value": "price"
+        }
+      }
+    },
+    {
+      "id": "2",
+      "name": "gmv_min",
+      "function": {
+        "expression": "min",
+        "returntype": "decimal",
+        "parameter": {
+          "type": "column",
+          "value": "price"
+        }
+      }
+    },
+    {
+      "id": "3",
+      "name": "gmv_max",
+      "function": {
+        "expression": "max",
+        "returntype": "decimal",
+        "parameter": {
+          "type": "column",
+          "value": "price"
+        }
+      }
+    },
+    {
+      "id": "4",
+      "name": "trans_cnt",
+      "function": {
+        "expression": "count",
+        "returntype": "bigint",
+        "parameter": {
+          "type": "constant",
+          "value": "1"
+        }
+      }
+    },
+    {
+      "id": "5",
+      "name": "seller_cnt_hll",
+      "function": {
+        "expression": "count_distinct",
+        "returntype": "hllc(10)",
+        "parameter": {
+          "type": "column",
+          "value": "seller_id"
+        }
+      }
+    },
+    {
+      "id": "6",
+      "name": "seller_cnt_long",
+      "function": {
+        "expression": "count_distinct",
+        "returntype": "bigint",
+        "parameter": {
+          "type": "column",
+          "value": "seller_id"
+        }
+      },
+      "dependent_measure_ref" : "seller_cnt_hll"
+    },
+    {
+      "id": "7",
+      "name": "seller_format_cnt",
+      "function": {
+        "expression": "count_distinct",
+        "returntype": "hllc(10)",
+        "parameter": {
+          "type": "column",
+          "value": "seller_id, lstg_format_name"
+        }
+      }
+    }
+  ],
+  "rowkey": {
+    "rowkey_columns": [
+      {
+        "column": "cal_dt",
+        "dictionary": "date(yyyy-mm-dd)",
+        "mandatory": "true"
+      },
+      {
+        "column": "leaf_categ_id",
+        "dictionary": "string"
+      },
+      {
+        "column": "meta_categ_name",
+        "dictionary": "string"
+      },
+      {
+        "column": "categ_lvl2_name",
+        "dictionary": "string"
+      },
+      {
+        "column": "categ_lvl3_name",
+        "dictionary": "string"
+      },
+      {
+        "column": "lstg_format_name",
+        "length": 12
+      },
+      {
+        "column": "lstg_site_id",
+        "dictionary": "string"
+      },
+      {
+        "column": "slr_segment_cd",
+        "dictionary": "string"
+      }
+    ],
+    "aggregation_groups": [
+      [
+        "lstg_format_name",
+        "lstg_site_id",
+        "slr_segment_cd"
+      ],
+      [
+        "leaf_categ_id",
+        "meta_categ_name",
+        "categ_lvl3_name",
+        "categ_lvl2_name",
+        "lstg_format_name"
+      ]
+    ]
+  },
+  "hbase_mapping": {
+    "column_family": [
+      {
+        "name": "f1",
+        "columns": [
+          {
+            "qualifier": "m",
+            "measure_refs": [
+              "gmv_sum",
+              "gmv_min",
+              "gmv_max",
+              "trans_cnt",
+              "seller_cnt_long"
+            ]
+          }
+        ]
+      },
+      {
+        "name": "f2",
+        "columns": [
+          {
+            "qualifier": "m",
+            "measure_refs": [
+                "seller_cnt_hll",
+                "seller_format_cnt"
+            ]
+          }
+        ]
+      }
+    ]
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/examples/test_case_data/localmeta/cube_desc/test_kylin_cube_without_slr_left_join_desc.json
----------------------------------------------------------------------
diff --git a/examples/test_case_data/localmeta/cube_desc/test_kylin_cube_without_slr_left_join_desc.json b/examples/test_case_data/localmeta/cube_desc/test_kylin_cube_without_slr_left_join_desc.json
new file mode 100644
index 0000000..d79ae72
--- /dev/null
+++ b/examples/test_case_data/localmeta/cube_desc/test_kylin_cube_without_slr_left_join_desc.json
@@ -0,0 +1,283 @@
+{
+  "uuid": "9ac9b7a8-3929-4dff-b59d-2100aadc8dbf",
+  "name": "test_kylin_cube_without_slr_left_join_desc",
+  "fact_table": "test_kylin_fact",
+  "cube_partition_desc": {
+    "partition_date_column": "test_kylin_fact.cal_dt",
+    "partition_date_start": 0,
+    "cube_partition_type": "UPDATE_INSERT"
+  },
+  "dimensions": [
+    {
+      "id": "1",
+      "name": "cal_dt",
+      "datatype": "date",
+      "table": "test_cal_dt",
+      "column": "{FK}",
+      "derived": [
+        "week_beg_dt"
+      ],
+      "join": {
+        "type": "left",
+        "primary_key": [
+          "cal_dt"
+        ],
+        "foreign_key": [
+          "cal_dt"
+        ]
+      }
+    },
+    {
+      "id": "2",
+      "name": "category",
+      "table": "test_category_groupings",
+      "column": "{FK}",
+      "derived": [
+        "USER_DEFINED_FIELD1",
+        "USER_DEFINED_FIELD3",
+        "UPD_DATE",
+        "UPD_USER"
+      ],
+      "join": {
+        "type": "left",
+        "primary_key": [
+          "leaf_categ_id",
+          "site_id"
+        ],
+        "foreign_key": [
+          "leaf_categ_id",
+          "lstg_site_id"
+        ]
+      },
+      "hierarchy": [
+        {
+          "level": "1",
+          "column": "meta_categ_name"
+        },
+        {
+          "level": "2",
+          "column": "categ_lvl2_name"
+        },
+        {
+          "level": "3",
+          "column": "categ_lvl3_name"
+        }
+      ]
+    },
+    {
+      "id": "3",
+      "name": "lstg_format_name",
+      "datatype": "string",
+      "table": "test_kylin_fact",
+      "column": "lstg_format_name"
+    },
+    {
+      "id": "4",
+      "name": "site_id",
+      "datatype": "string",
+      "table": "test_sites",
+      "column": "{FK}",
+      "derived": [
+        "site_name",
+        "cre_user"
+      ],
+      "join": {
+        "type": "left",
+        "primary_key": [
+          "site_id"
+        ],
+        "foreign_key": [
+          "lstg_site_id"
+        ]
+      }
+    },
+    {
+      "id": "5",
+      "name": "seller_type_cd",
+      "datatype": "string",
+      "table": "test_seller_type_dim",
+      "column": "{FK}",
+      "derived": [
+        "seller_type_desc"
+      ],
+      "join": {
+        "type": "left",
+        "primary_key": [
+          "seller_type_cd"
+        ],
+        "foreign_key": [
+          "slr_segment_cd"
+        ]
+      }
+    }
+  ],
+  "measures": [
+    {
+      "id": "1",
+      "name": "gmv_sum",
+      "function": {
+        "expression": "sum",
+        "returntype": "decimal",
+        "parameter": {
+          "type": "column",
+          "value": "price"
+        }
+      }
+    },
+    {
+      "id": "2",
+      "name": "gmv_min",
+      "function": {
+        "expression": "min",
+        "returntype": "decimal",
+        "parameter": {
+          "type": "column",
+          "value": "price"
+        }
+      }
+    },
+    {
+      "id": "3",
+      "name": "gmv_max",
+      "function": {
+        "expression": "max",
+        "returntype": "decimal",
+        "parameter": {
+          "type": "column",
+          "value": "price"
+        }
+      }
+    },
+    {
+      "id": "4",
+      "name": "trans_cnt",
+      "function": {
+        "expression": "count",
+        "returntype": "bigint",
+        "parameter": {
+          "type": "constant",
+          "value": "1"
+        }
+      }
+    },
+    {
+      "id": "5",
+      "name": "seller_cnt_hll",
+      "function": {
+        "expression": "count_distinct",
+        "returntype": "hllc(10)",
+        "parameter": {
+          "type": "column",
+          "value": "seller_id"
+        }
+      }
+    },
+    {
+      "id": "6",
+      "name": "seller_cnt_long",
+      "function": {
+        "expression": "count_distinct",
+        "returntype": "bigint",
+        "parameter": {
+          "type": "column",
+          "value": "seller_id"
+        },
+        "dependent_measure_ref" : "seller_cnt_hll"
+      }
+    },
+    {
+      "id": "7",
+      "name": "seller_format_cnt",
+      "function": {
+        "expression": "count_distinct",
+        "returntype": "hllc(10)",
+        "parameter": {
+          "type": "column",
+          "value": "seller_id, lstg_format_name"
+        }
+      }
+    }
+  ],
+  "rowkey": {
+    "rowkey_columns": [
+      {
+        "column": "cal_dt",
+        "dictionary": "date(yyyy-mm-dd)",
+        "mandatory": "true"
+      },
+      {
+        "column": "leaf_categ_id",
+        "dictionary": "string"
+      },
+      {
+        "column": "meta_categ_name",
+        "dictionary": "string"
+      },
+      {
+        "column": "categ_lvl2_name",
+        "dictionary": "string"
+      },
+      {
+        "column": "categ_lvl3_name",
+        "dictionary": "string"
+      },
+      {
+        "column": "lstg_format_name",
+        "dictionary": "string"
+      },
+      {
+        "column": "lstg_site_id",
+        "dictionary": "string"
+      },
+      {
+        "column": "slr_segment_cd",
+        "dictionary": "string"
+      }
+    ],
+    "aggregation_groups": [
+      [
+        "lstg_format_name",
+        "lstg_site_id",
+        "slr_segment_cd"
+      ],
+      [
+        "leaf_categ_id",
+        "meta_categ_name",
+        "categ_lvl3_name",
+        "categ_lvl2_name",
+        "lstg_format_name"
+      ]
+    ]
+  },
+  "hbase_mapping": {
+    "column_family": [
+      {
+        "name": "f1",
+        "columns": [
+          {
+            "qualifier": "m",
+            "measure_refs": [
+              "gmv_sum",
+              "gmv_min",
+              "gmv_max",
+              "trans_cnt",
+              "seller_cnt_long"
+            ]
+          }
+        ]
+      },
+      {
+        "name": "f2",
+        "columns": [
+          {
+            "qualifier": "m",
+            "measure_refs": [
+              "seller_cnt_hll",
+              "seller_format_cnt"
+             ]
+          }
+        ]
+      }
+    ]
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/examples/test_case_data/localmeta/data/.gitignore
----------------------------------------------------------------------
diff --git a/examples/test_case_data/localmeta/data/.gitignore b/examples/test_case_data/localmeta/data/.gitignore
new file mode 100644
index 0000000..87d7b53
--- /dev/null
+++ b/examples/test_case_data/localmeta/data/.gitignore
@@ -0,0 +1 @@
+*.csv.back


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

Posted by lu...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/metadata/src/main/java/com/kylinolap/metadata/model/cube/CubeDesc.java
----------------------------------------------------------------------
diff --git a/metadata/src/main/java/com/kylinolap/metadata/model/cube/CubeDesc.java b/metadata/src/main/java/com/kylinolap/metadata/model/cube/CubeDesc.java
new file mode 100644
index 0000000..f6c638a
--- /dev/null
+++ b/metadata/src/main/java/com/kylinolap/metadata/model/cube/CubeDesc.java
@@ -0,0 +1,827 @@
+/*
+ * 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.metadata.model.cube;
+
+import java.security.MessageDigest;
+import java.security.NoSuchAlgorithmException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.LinkedHashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.Set;
+
+import org.apache.commons.lang.ArrayUtils;
+import org.apache.commons.net.util.Base64;
+
+import com.fasterxml.jackson.annotation.JsonAutoDetect;
+import com.fasterxml.jackson.annotation.JsonAutoDetect.Visibility;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.fasterxml.jackson.core.JsonProcessingException;
+import com.google.common.collect.Lists;
+import com.google.common.collect.Maps;
+import com.kylinolap.common.KylinConfig;
+import com.kylinolap.common.persistence.ResourceStore;
+import com.kylinolap.common.persistence.RootPersistentEntity;
+import com.kylinolap.common.util.Array;
+import com.kylinolap.common.util.JsonUtil;
+import com.kylinolap.metadata.MetadataConstances;
+import com.kylinolap.metadata.MetadataManager;
+import com.kylinolap.metadata.model.schema.ColumnDesc;
+import com.kylinolap.metadata.model.schema.DataType;
+import com.kylinolap.metadata.model.schema.TableDesc;
+
+/**
+ * Created with IntelliJ IDEA. User: lukhan Date: 9/24/13 Time: 10:40 AM To
+ * change this template use File | Settings | File Templates.
+ */
+@JsonAutoDetect(fieldVisibility = Visibility.NONE, getterVisibility = Visibility.NONE, isGetterVisibility = Visibility.NONE, setterVisibility = Visibility.NONE)
+public class CubeDesc extends RootPersistentEntity {
+
+    public static enum CubeCapacity {
+        SMALL, MEDIUM, LARGE;
+    }
+
+    public static enum DeriveType {
+        LOOKUP, PK_FK
+    }
+
+    public static class DeriveInfo {
+        public DeriveType type;
+        public DimensionDesc dimension;
+        public TblColRef[] columns;
+        public boolean isOneToOne; // only used when ref from derived to host
+
+        DeriveInfo(DeriveType type, DimensionDesc dimension, TblColRef[] columns, boolean isOneToOne) {
+            this.type = type;
+            this.dimension = dimension;
+            this.columns = columns;
+            this.isOneToOne = isOneToOne;
+        }
+
+        @Override
+        public String toString() {
+            return "DeriveInfo [type=" + type + ", dimension=" + dimension + ", columns=" + Arrays.toString(columns) + ", isOneToOne=" + isOneToOne + "]";
+        }
+
+    }
+
+    private KylinConfig config;
+
+    @JsonProperty("name")
+    private String name;
+    @JsonProperty("description")
+    private String description;
+    @JsonProperty("fact_table")
+    private String factTable;
+    @JsonProperty("null_string")
+    private String[] nullStrings;
+    @JsonProperty("filter_condition")
+    private String filterCondition;
+    @JsonProperty("cube_partition_desc")
+    CubePartitionDesc cubePartitionDesc;
+    @JsonProperty("dimensions")
+    private List<DimensionDesc> dimensions;
+    @JsonProperty("measures")
+    private List<MeasureDesc> measures;
+    @JsonProperty("rowkey")
+    private RowKeyDesc rowkey;
+    @JsonProperty("hbase_mapping")
+    private HBaseMappingDesc hbaseMapping;
+    @JsonProperty("signature")
+    private String signature;
+    @JsonProperty("capacity")
+    private CubeCapacity capacity = CubeCapacity.MEDIUM;
+    @JsonProperty("notify_list")
+    private List<String> notifyList;
+
+    private Map<String, Map<String, TblColRef>> columnMap = new HashMap<String, Map<String, TblColRef>>();
+    private LinkedHashSet<TblColRef> allColumns = new LinkedHashSet<TblColRef>();
+    private LinkedHashSet<TblColRef> dimensionColumns = new LinkedHashSet<TblColRef>();
+    private Map<TblColRef, DeriveInfo> derivedToHostMap = Maps.newHashMap();
+    private Map<Array<TblColRef>, List<DeriveInfo>> hostToDerivedMap = Maps.newHashMap();
+
+    /**
+     * Error messages during resolving json metadata
+     */
+    private List<String> errors = new ArrayList<String>();
+
+    /**
+     * @return all columns this cube can support, including derived
+     */
+    public Set<TblColRef> listAllColumns() {
+        return allColumns;
+    }
+
+    /**
+     * @return dimension columns including derived, BUT NOT measures
+     */
+    public Set<TblColRef> listDimensionColumnsIncludingDerived() {
+        return dimensionColumns;
+    }
+
+    /**
+     * @return dimension columns excluding derived and measures
+     */
+    public List<TblColRef> listDimensionColumnsExcludingDerived() {
+        List<TblColRef> result = new ArrayList<TblColRef>();
+        for (TblColRef col : dimensionColumns) {
+            if (isDerived(col) == false)
+                result.add(col);
+        }
+        return result;
+    }
+
+    /**
+     * Find FunctionDesc by Full Expression.
+     * 
+     * @return
+     */
+    public FunctionDesc findFunctionOnCube(FunctionDesc manualFunc) {
+        for (MeasureDesc m : measures) {
+            if (m.getFunction().equals(manualFunc))
+                return m.getFunction();
+        }
+        return null;
+    }
+
+    public TblColRef findColumnRef(String table, String column) {
+        Map<String, TblColRef> cols = columnMap.get(table);
+        if (cols == null)
+            return null;
+        else
+            return cols.get(column);
+    }
+
+    public DimensionDesc findDimensionByColumn(TblColRef col) {
+        for (DimensionDesc dim : dimensions) {
+            if (ArrayUtils.contains(dim.getColumnRefs(), col))
+                return dim;
+        }
+        return null;
+    }
+
+    public DimensionDesc findDimensionByTable(String lookupTableName) {
+        lookupTableName = lookupTableName.toUpperCase();
+        for (DimensionDesc dim : dimensions)
+            if (dim.getTable() != null && dim.getTable().equals(lookupTableName))
+                return dim;
+        return null;
+    }
+
+    public DimensionDesc findDimensionByName(String dimName) {
+        dimName = dimName.toUpperCase();
+        for (DimensionDesc dim : dimensions) {
+            if (dimName.equals(dim.getName()))
+                return dim;
+        }
+        return null;
+    }
+
+    public TblColRef findPKByFK(TblColRef fk) {
+        assert isFactTable(fk.getTable());
+
+        TblColRef candidate = null;
+
+        for (DimensionDesc dim : dimensions) {
+            JoinDesc join = dim.getJoin();
+            if (join == null)
+                continue;
+
+            int find = ArrayUtils.indexOf(join.getForeignKeyColumns(), fk);
+            if (find >= 0) {
+                candidate = join.getPrimaryKeyColumns()[find];
+                if (join.getForeignKeyColumns().length == 1) { // is single
+                                                               // column join?
+                    break;
+                }
+            }
+        }
+        return candidate;
+    }
+
+    /**
+     * Get all functions from each measure.
+     * 
+     * @return
+     */
+    public List<FunctionDesc> listAllFunctions() {
+        List<FunctionDesc> functions = new ArrayList<FunctionDesc>();
+        for (MeasureDesc m : measures) {
+            functions.add(m.getFunction());
+        }
+        return functions;
+    }
+
+    public List<TableDesc> listTables() {
+        MetadataManager metaMgr = MetadataManager.getInstance(config);
+        HashSet<String> tableNames = new HashSet<String>();
+        List<TableDesc> result = new ArrayList<TableDesc>();
+
+        tableNames.add(factTable.toUpperCase());
+        for (DimensionDesc dim : dimensions) {
+            String table = dim.getTable();
+            if (table != null)
+                tableNames.add(table.toUpperCase());
+        }
+
+        for (String tableName : tableNames) {
+            result.add(metaMgr.getTableDesc(tableName));
+        }
+
+        return result;
+    }
+
+    public boolean isFactTable(String factTable) {
+        return this.factTable.equalsIgnoreCase(factTable);
+    }
+
+    public boolean isDerived(TblColRef col) {
+        return derivedToHostMap.containsKey(col);
+    }
+
+    public DeriveInfo getHostInfo(TblColRef derived) {
+        return derivedToHostMap.get(derived);
+    }
+
+    public Map<Array<TblColRef>, List<DeriveInfo>> getHostToDerivedInfo(List<TblColRef> rowCols, Collection<TblColRef> wantedCols) {
+        Map<Array<TblColRef>, List<DeriveInfo>> result = new HashMap<Array<TblColRef>, List<DeriveInfo>>();
+        for (Entry<Array<TblColRef>, List<DeriveInfo>> entry : hostToDerivedMap.entrySet()) {
+            Array<TblColRef> hostCols = entry.getKey();
+            boolean hostOnRow = rowCols.containsAll(Arrays.asList(hostCols.data));
+            if (!hostOnRow)
+                continue;
+
+            List<DeriveInfo> wantedInfo = new ArrayList<DeriveInfo>();
+            for (DeriveInfo info : entry.getValue()) {
+                if (wantedCols == null || Collections.disjoint(wantedCols, Arrays.asList(info.columns)) == false) // has
+                                                                                                                  // any
+                                                                                                                  // wanted
+                                                                                                                  // columns?
+                    wantedInfo.add(info);
+            }
+
+            if (wantedInfo.size() > 0)
+                result.put(hostCols, wantedInfo);
+        }
+        return result;
+    }
+
+    public String getResourcePath() {
+        return getCubeDescResourcePath(name);
+    }
+
+    public static String getCubeDescResourcePath(String descName) {
+        return ResourceStore.CUBE_DESC_RESOURCE_ROOT + "/" + descName + MetadataConstances.FILE_SURFIX;
+    }
+
+    // ============================================================================
+
+    public HBaseMappingDesc getHBaseMapping() {
+        return hbaseMapping;
+    }
+
+    public void setHBaseMapping(HBaseMappingDesc hbaseMapping) {
+        this.hbaseMapping = hbaseMapping;
+    }
+
+    public KylinConfig getConfig() {
+        return config;
+    }
+
+    public void setConfig(KylinConfig config) {
+        this.config = config;
+    }
+
+    public String getName() {
+        return name;
+    }
+
+    public void setName(String name) {
+        this.name = name;
+    }
+
+    public String getDescription() {
+        return description;
+    }
+
+    public void setDescription(String description) {
+        this.description = description;
+    }
+
+    public String getFactTable() {
+        return factTable;
+    }
+
+    public void setFactTable(String factTable) {
+        this.factTable = factTable;
+    }
+
+    public String[] getNullStrings() {
+        return nullStrings;
+    }
+
+    public String getFilterCondition() {
+        return filterCondition;
+    }
+
+    public void setFilterCondition(String filterCondition) {
+        this.filterCondition = filterCondition;
+    }
+
+    public CubePartitionDesc getCubePartitionDesc() {
+        return cubePartitionDesc;
+    }
+
+    public void setCubePartitionDesc(CubePartitionDesc cubePartitionDesc) {
+        this.cubePartitionDesc = cubePartitionDesc;
+    }
+
+    public List<DimensionDesc> getDimensions() {
+        return dimensions;
+    }
+
+    public void setDimensions(List<DimensionDesc> dimensions) {
+        this.dimensions = dimensions;
+    }
+
+    public List<MeasureDesc> getMeasures() {
+        return measures;
+    }
+
+    public void setMeasures(List<MeasureDesc> measures) {
+        this.measures = measures;
+    }
+
+    public RowKeyDesc getRowkey() {
+        return rowkey;
+    }
+
+    public void setRowkey(RowKeyDesc rowkey) {
+        this.rowkey = rowkey;
+    }
+
+    public String getSignature() {
+        return signature;
+    }
+
+    public void setSignature(String signature) {
+        this.signature = signature;
+    }
+
+    public CubeCapacity getCapacity() {
+        return capacity;
+    }
+
+    public void setCapacity(CubeCapacity capacity) {
+        this.capacity = capacity;
+    }
+
+    public List<String> getNotifyList() {
+        return notifyList;
+    }
+
+    public void setNotifyList(List<String> notifyList) {
+        this.notifyList = notifyList;
+    }
+
+    @Override
+    public boolean equals(Object o) {
+        if (this == o)
+            return true;
+        if (o == null || getClass() != o.getClass())
+            return false;
+
+        CubeDesc cubeDesc = (CubeDesc) o;
+
+        if (!name.equals(cubeDesc.name))
+            return false;
+        if (!factTable.equals(cubeDesc.factTable))
+            return false;
+
+        return true;
+    }
+
+    @Override
+    public int hashCode() {
+        int result = 0;
+        result = 31 * result + name.hashCode();
+        result = 31 * result + factTable.hashCode();
+        return result;
+    }
+
+    @Override
+    public String toString() {
+        return "CubeDesc [name=" + name + ", factTable=" + factTable + ", cubePartitionDesc=" + cubePartitionDesc + ", dimensions=" + dimensions + ", measures=" + measures + ", rowkey=" + rowkey + ", hbaseMapping=" + hbaseMapping + "]";
+    }
+
+    public String calculateSignature() {
+        MessageDigest md = null;
+        try {
+            md = MessageDigest.getInstance("MD5");
+            StringBuilder sigString = new StringBuilder();
+            sigString.append(this.name).append("|").append(this.factTable).append("|").append(JsonUtil.writeValueAsString(this.cubePartitionDesc)).append("|").append(JsonUtil.writeValueAsString(this.dimensions)).append("|").append(JsonUtil.writeValueAsString(this.measures)).append("|").append(JsonUtil.writeValueAsString(this.rowkey)).append("|").append(JsonUtil.writeValueAsString(this.hbaseMapping));
+
+            byte[] signature = md.digest(sigString.toString().getBytes());
+            return new String(Base64.encodeBase64(signature));
+        } catch (NoSuchAlgorithmException e) {
+            throw new RuntimeException("Failed to calculate signature");
+        } catch (JsonProcessingException e) {
+            throw new RuntimeException("Failed to calculate signature");
+        }
+    }
+
+    public Map<String, TblColRef> buildColumnNameAbbreviation() {
+        Map<String, TblColRef> r = new HashMap<String, TblColRef>();
+        for (TblColRef col : listDimensionColumnsExcludingDerived()) {
+            r.put(col.getName(), col);
+        }
+        return r;
+    }
+
+    public void init(KylinConfig config, Map<String, TableDesc> tables) {
+        this.errors.clear();
+        this.config = config;
+
+        if (factTable != null)
+            factTable = factTable.toUpperCase();
+
+        for (DimensionDesc dim : dimensions) {
+            dim.init(tables);
+        }
+
+        sortDimAndMeasure();
+
+        initJoinColumns(tables);
+        initDimensionColumns(tables);
+        initMeasureColumns(tables);
+
+        rowkey.init(this);
+        if (hbaseMapping != null) {
+            hbaseMapping.init(this);
+        }
+
+        initMeasureReferenceToColumnFamily();
+
+        if (null != this.cubePartitionDesc) {
+            this.cubePartitionDesc.init(columnMap);
+        }
+
+        // check all dimension columns are presented on rowkey
+        List<TblColRef> dimCols = listDimensionColumnsExcludingDerived();
+        if (rowkey.getRowKeyColumns().length != dimCols.size()) {
+            addError("RowKey columns count (" + rowkey.getRowKeyColumns().length + ") does not match dimension columns count (" + dimCols.size() + "). ");
+        }
+    }
+
+    private void initDimensionColumns(Map<String, TableDesc> tables) {
+        // fill back ColRefDesc
+        for (DimensionDesc dim : dimensions) {
+            TableDesc dimTable = tables.get(dim.getTable());
+            JoinDesc join = dim.getJoin();
+
+            ArrayList<TblColRef> dimColList = new ArrayList<TblColRef>();
+            ArrayList<TblColRef> hostColList = new ArrayList<TblColRef>();
+
+            // dimension column
+            if (dim.getColumn() != null) {
+                if ("{FK}".equals(dim.getColumn())) {
+                    for (TblColRef ref : join.getForeignKeyColumns()) {
+                        TblColRef inited = initDimensionColRef(ref);
+                        dimColList.add(inited);
+                        hostColList.add(inited);
+                    }
+                } else {
+                    TblColRef ref = initDimensionColRef(dimTable, dim.getColumn());
+                    dimColList.add(ref);
+                    hostColList.add(ref);
+                }
+            }
+            // hierarchy columns
+            if (dim.getHierarchy() != null) {
+                for (HierarchyDesc hier : dim.getHierarchy()) {
+                    TblColRef ref = initDimensionColRef(dimTable, hier.getColumn());
+                    hier.setColumnRef(ref);
+                    dimColList.add(ref);
+                }
+                if (hostColList.isEmpty()) { // the last hierarchy could serve
+                                             // as host when col is
+                                             // unspecified
+                    hostColList.add(dimColList.get(dimColList.size() - 1));
+                }
+            }
+            TblColRef[] dimCols = (TblColRef[]) dimColList.toArray(new TblColRef[dimColList.size()]);
+            dim.setColumnRefs(dimCols);
+
+            // lookup derived columns
+            TblColRef[] hostCols = (TblColRef[]) hostColList.toArray(new TblColRef[hostColList.size()]);
+            String[] derived = dim.getDerived();
+            if (derived != null) {
+                String[][] split = splitDerivedColumnAndExtra(derived);
+                String[] derivedNames = split[0];
+                String[] derivedExtra = split[1];
+                TblColRef[] derivedCols = new TblColRef[derivedNames.length];
+                for (int i = 0; i < derivedNames.length; i++) {
+                    derivedCols[i] = initDimensionColRef(dimTable, derivedNames[i]);
+                }
+                initDerivedMap(hostCols, DeriveType.LOOKUP, dim, derivedCols, derivedExtra);
+            }
+
+            // FK derived column
+            if (join != null) {
+                TblColRef[] fk = join.getForeignKeyColumns();
+                TblColRef[] pk = join.getPrimaryKeyColumns();
+                for (int i = 0; i < fk.length; i++) {
+                    int find = ArrayUtils.indexOf(hostCols, fk[i]);
+                    if (find >= 0) {
+                        TblColRef derivedCol = initDimensionColRef(pk[i]);
+                        initDerivedMap(hostCols[find], DeriveType.PK_FK, dim, derivedCol);
+                    }
+                }
+                for (int i = 0; i < pk.length; i++) {
+                    int find = ArrayUtils.indexOf(hostCols, pk[i]);
+                    if (find >= 0) {
+                        TblColRef derivedCol = initDimensionColRef(fk[i]);
+                        initDerivedMap(hostCols[find], DeriveType.PK_FK, dim, derivedCol);
+                    }
+                }
+            }
+        }
+    }
+
+    private String[][] splitDerivedColumnAndExtra(String[] derived) {
+        String[] cols = new String[derived.length];
+        String[] extra = new String[derived.length];
+        for (int i = 0; i < derived.length; i++) {
+            String str = derived[i];
+            int cut = str.indexOf(":");
+            if (cut >= 0) {
+                cols[i] = str.substring(0, cut);
+                extra[i] = str.substring(cut + 1).trim();
+            } else {
+                cols[i] = str;
+                extra[i] = "";
+            }
+        }
+        return new String[][] { cols, extra };
+    }
+
+    private void initDerivedMap(TblColRef hostCol, DeriveType type, DimensionDesc dimension, TblColRef derivedCol) {
+        initDerivedMap(new TblColRef[] { hostCol }, type, dimension, new TblColRef[] { derivedCol }, null);
+    }
+
+    private void initDerivedMap(TblColRef[] hostCols, DeriveType type, DimensionDesc dimension, TblColRef[] derivedCols, String[] extra) {
+        if (hostCols.length == 0 || derivedCols.length == 0)
+            throw new IllegalStateException("host/derived columns must not be empty");
+
+        Array<TblColRef> hostColArray = new Array<TblColRef>(hostCols);
+        List<DeriveInfo> infoList = hostToDerivedMap.get(hostColArray);
+        if (infoList == null) {
+            hostToDerivedMap.put(hostColArray, infoList = new ArrayList<DeriveInfo>());
+        }
+        infoList.add(new DeriveInfo(type, dimension, derivedCols, false));
+
+        for (int i = 0; i < derivedCols.length; i++) {
+            TblColRef derivedCol = derivedCols[i];
+            boolean isOneToOne = type == DeriveType.PK_FK || ArrayUtils.contains(hostCols, derivedCol) || (extra != null && extra[i].contains("1-1"));
+            derivedToHostMap.put(derivedCol, new DeriveInfo(type, dimension, hostCols, isOneToOne));
+        }
+    }
+
+    private TblColRef initDimensionColRef(TableDesc table, String colName) {
+        ColumnDesc col = table.findColumnByName(colName);
+        if (col == null)
+            throw new IllegalArgumentException("No column '" + colName + "' found in table " + table);
+
+        TblColRef ref = new TblColRef(col);
+        return initDimensionColRef(ref);
+    }
+
+    private TblColRef initDimensionColRef(TblColRef ref) {
+        TblColRef existing = findColumnRef(ref.getTable(), ref.getName());
+        if (existing != null) {
+            return existing;
+        }
+
+        allColumns.add(ref);
+        dimensionColumns.add(ref);
+
+        Map<String, TblColRef> cols = columnMap.get(ref.getTable());
+        if (cols == null) {
+            columnMap.put(ref.getTable(), cols = new HashMap<String, TblColRef>());
+        }
+        cols.put(ref.getName(), ref);
+        return ref;
+    }
+
+    private void initJoinColumns(Map<String, TableDesc> tables) {
+        // join columns may or may not present in cube;
+        // here we don't modify 'allColumns' and 'dimensionColumns';
+        // initDimensionColumns() will do the update
+        for (DimensionDesc dim : dimensions) {
+            TableDesc dimTable = tables.get(dim.getTable());
+
+            JoinDesc join = dim.getJoin();
+            if (join == null)
+                continue;
+
+            // primary key
+            String[] pks = join.getPrimaryKey();
+            TblColRef[] pkCols = new TblColRef[pks.length];
+            for (int i = 0; i < pks.length; i++) {
+                ColumnDesc col = dimTable.findColumnByName(pks[i]);
+                if (col == null) {
+                    addError("Can't find column " + pks[i] + " in table " + dimTable.getName());
+                }
+                TblColRef colRef = new TblColRef(col);
+                pks[i] = colRef.getName();
+                pkCols[i] = colRef;
+            }
+            join.setPrimaryKeyColumns(pkCols);
+            // foreign key
+            TableDesc factTable = tables.get(this.factTable);
+            if (factTable == null) {
+                addError("Fact table does not exist:" + this.factTable);
+            }
+            String[] fks = join.getForeignKey();
+            TblColRef[] fkCols = new TblColRef[fks.length];
+            for (int i = 0; i < fks.length; i++) {
+                ColumnDesc col = factTable.findColumnByName(fks[i]);
+                if (col == null) {
+                    addError("Can't find column " + fks[i] + " in table " + this.factTable);
+                }
+                TblColRef colRef = new TblColRef(col);
+                fks[i] = colRef.getName();
+                fkCols[i] = colRef;
+            }
+            join.setForeignKeyColumns(fkCols);
+            // Validate join in dimension
+            if (pkCols.length != fkCols.length) {
+                addError("Primary keys(" + dim.getTable() + ")" + Arrays.toString(pks) + " are not consistent with Foreign keys(" + this.factTable + ") " + Arrays.toString(fks));
+            }
+            for (int i = 0; i < fkCols.length; i++) {
+                if (!fkCols[i].getDatatype().equals(pkCols[i].getDatatype())) {
+                    addError("Primary key " + dim.getTable() + "." + pkCols[i].getName() + "." + pkCols[i].getDatatype() + " are not consistent with Foreign key " + this.factTable + "." + fkCols[i].getName() + "." + fkCols[i].getDatatype());
+                }
+            }
+
+        }
+    }
+
+    private void initMeasureColumns(Map<String, TableDesc> tables) {
+        if (measures == null || measures.isEmpty()) {
+            return;
+        }
+
+        TableDesc factTable = tables.get(getFactTable());
+        for (MeasureDesc m : measures) {
+            m.setName(m.getName().toUpperCase());
+
+            if (m.getDependentMeasureRef() != null) {
+                m.setDependentMeasureRef(m.getDependentMeasureRef().toUpperCase());
+            }
+
+            FunctionDesc f = m.getFunction();
+            f.setExpression(f.getExpression().toUpperCase());
+            f.setReturnDataType(DataType.getInstance(f.getReturnType()));
+
+            ParameterDesc p = f.getParameter();
+            p.normalizeColumnValue();
+
+            if (p.isColumnType()) {
+                ArrayList<TblColRef> colRefs = Lists.newArrayList();
+                for (String cName : p.getValue().split("\\s*,\\s*")) {
+                    ColumnDesc sourceColumn = factTable.findColumnByName(cName);
+                    TblColRef colRef = new TblColRef(sourceColumn);
+                    colRefs.add(colRef);
+                    allColumns.add(colRef);
+                }
+                if (colRefs.isEmpty() == false)
+                    p.setColRefs(colRefs);
+            }
+        }
+    }
+
+    private void initMeasureReferenceToColumnFamily() {
+        if (measures == null || measures.size() == 0)
+            return;
+
+        Map<String, MeasureDesc> measureCache = new HashMap<String, MeasureDesc>();
+        for (MeasureDesc m : measures)
+            measureCache.put(m.getName(), m);
+
+        for (HBaseColumnFamilyDesc cf : getHBaseMapping().getColumnFamily()) {
+            for (HBaseColumnDesc c : cf.getColumns()) {
+                MeasureDesc[] measureDescs = new MeasureDesc[c.getMeasureRefs().length];
+                for (int i = 0; i < c.getMeasureRefs().length; i++) {
+                    measureDescs[i] = measureCache.get(c.getMeasureRefs()[i]);
+                }
+                c.setMeasures(measureDescs);
+                c.setColumnFamilyName(cf.getName());
+            }
+        }
+    }
+
+    private void sortDimAndMeasure() {
+        sortDimensionsByID();
+        sortMeasuresByID();
+        for (DimensionDesc dim : dimensions) {
+            sortHierarchiesByLevel(dim.getHierarchy());
+        }
+    }
+
+    private void sortDimensionsByID() {
+        Collections.sort(dimensions, new Comparator<DimensionDesc>() {
+            @Override
+            public int compare(DimensionDesc d1, DimensionDesc d2) {
+                Integer id1 = d1.getId();
+                Integer id2 = d2.getId();
+                return id1.compareTo(id2);
+            }
+        });
+    }
+
+    private void sortMeasuresByID() {
+        if (measures == null) {
+            measures = Lists.newArrayList();
+        }
+
+        Collections.sort(measures, new Comparator<MeasureDesc>() {
+            @Override
+            public int compare(MeasureDesc m1, MeasureDesc m2) {
+                Integer id1 = m1.getId();
+                Integer id2 = m2.getId();
+                return id1.compareTo(id2);
+            }
+        });
+    }
+
+    private void sortHierarchiesByLevel(HierarchyDesc[] hierarchies) {
+        if (hierarchies != null) {
+            Arrays.sort(hierarchies, new Comparator<HierarchyDesc>() {
+                @Override
+                public int compare(HierarchyDesc h1, HierarchyDesc h2) {
+                    Integer level1 = Integer.parseInt(h1.getLevel());
+                    Integer level2 = Integer.parseInt(h2.getLevel());
+                    return level1.compareTo(level2);
+                }
+            });
+        }
+    }
+
+    public boolean hasHolisticCountDistinctMeasures() {
+        for (MeasureDesc measure : measures) {
+            if (measure.getFunction().isHolisticCountDistinct()) {
+                return true;
+            }
+        }
+        return false;
+    }
+
+    /**
+     * Add error info and thrown exception out
+     * 
+     * @param message
+     */
+    public void addError(String message) {
+        addError(message, false);
+    }
+
+    /**
+     * @param message
+     *            error message
+     * @param silent
+     *            if throw exception
+     */
+    public void addError(String message, boolean silent) {
+        if (!silent) {
+            throw new IllegalStateException(message);
+        } else {
+            this.errors.add(message);
+        }
+    }
+
+    public List<String> getError() {
+        return this.errors;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/metadata/src/main/java/com/kylinolap/metadata/model/cube/CubePartitionDesc.java
----------------------------------------------------------------------
diff --git a/metadata/src/main/java/com/kylinolap/metadata/model/cube/CubePartitionDesc.java b/metadata/src/main/java/com/kylinolap/metadata/model/cube/CubePartitionDesc.java
new file mode 100644
index 0000000..270370f
--- /dev/null
+++ b/metadata/src/main/java/com/kylinolap/metadata/model/cube/CubePartitionDesc.java
@@ -0,0 +1,86 @@
+/*
+ * 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.metadata.model.cube;
+
+import java.util.Map;
+
+import com.fasterxml.jackson.annotation.JsonAutoDetect;
+import com.fasterxml.jackson.annotation.JsonAutoDetect.Visibility;
+import com.fasterxml.jackson.annotation.JsonProperty;
+
+/**
+ * @author xduo
+ * 
+ */
+@JsonAutoDetect(fieldVisibility = Visibility.NONE, getterVisibility = Visibility.NONE, isGetterVisibility = Visibility.NONE, setterVisibility = Visibility.NONE)
+public class CubePartitionDesc {
+
+    public static enum CubePartitionType {
+        APPEND, UPDATE_INSERT
+    }
+
+    @JsonProperty("partition_date_column")
+    private String partitionDateColumn;
+    @JsonProperty("partition_date_start")
+    private long partitionDateStart = 0L;
+    @JsonProperty("cube_partition_type")
+    private CubePartitionType cubePartitionType = CubePartitionType.APPEND;
+
+    private TblColRef partitionDateColumnRef;
+
+    public void init(Map<String, Map<String, TblColRef>> columnMap) {
+        if (null != partitionDateColumn) {
+            String[] columns = partitionDateColumn.split("\\.");
+
+            if (null != columns && columns.length == 2) {
+                Map<String, TblColRef> cols = columnMap.get(columns[0].toUpperCase());
+                if (cols != null)
+                    partitionDateColumnRef = cols.get(columns[1].toUpperCase());
+
+            }
+        }
+    }
+
+    public String getPartitionDateColumn() {
+        return partitionDateColumn;
+    }
+
+    public void setPartitionDateColumn(String partitionDateColumn) {
+        this.partitionDateColumn = partitionDateColumn;
+    }
+
+    public long getPartitionDateStart() {
+        return partitionDateStart;
+    }
+
+    public void setPartitionDateStart(long partitionDateStart) {
+        this.partitionDateStart = partitionDateStart;
+    }
+
+    public CubePartitionType getCubePartitionType() {
+        return cubePartitionType;
+    }
+
+    public void setCubePartitionType(CubePartitionType cubePartitionType) {
+        this.cubePartitionType = cubePartitionType;
+    }
+
+    public TblColRef getPartitionDateColumnRef() {
+        return partitionDateColumnRef;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/metadata/src/main/java/com/kylinolap/metadata/model/cube/DimensionDesc.java
----------------------------------------------------------------------
diff --git a/metadata/src/main/java/com/kylinolap/metadata/model/cube/DimensionDesc.java b/metadata/src/main/java/com/kylinolap/metadata/model/cube/DimensionDesc.java
new file mode 100644
index 0000000..142fc27
--- /dev/null
+++ b/metadata/src/main/java/com/kylinolap/metadata/model/cube/DimensionDesc.java
@@ -0,0 +1,209 @@
+/*
+ * 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.metadata.model.cube;
+
+import java.util.Arrays;
+import java.util.Map;
+
+import com.fasterxml.jackson.annotation.JsonAutoDetect;
+import com.fasterxml.jackson.annotation.JsonAutoDetect.Visibility;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.kylinolap.common.util.StringUtil;
+import com.kylinolap.metadata.model.schema.TableDesc;
+
+/**
+ * Created with IntelliJ IDEA. User: lukhan Date: 9/24/13 Time: 10:40 AM To
+ * change this template use File | Settings | File Templates.
+ */
+@JsonAutoDetect(fieldVisibility = Visibility.NONE, getterVisibility = Visibility.NONE, isGetterVisibility = Visibility.NONE, setterVisibility = Visibility.NONE)
+public class DimensionDesc {
+
+    @JsonProperty("id")
+    private int id;
+    @JsonProperty("name")
+    private String name;
+    @JsonProperty("join")
+    private JoinDesc join;
+    @JsonProperty("hierarchy")
+    private HierarchyDesc[] hierarchy;
+    @JsonProperty("table")
+    private String table;
+    @JsonProperty("column")
+    private String column;
+    @JsonProperty("datatype")
+    private String datatype;
+    @JsonProperty("derived")
+    private String[] derived;
+
+    // computed
+    private TblColRef[] columnRefs;
+    private TblColRef[] derivedColRefs;
+
+    public boolean isHierarchyColumn(TblColRef col) {
+        if (hierarchy == null)
+            return false;
+
+        for (HierarchyDesc hier : hierarchy) {
+            if (hier.getColumnRef().equals(col))
+                return true;
+        }
+        return false;
+    }
+
+    public String getDatatype() {
+        return datatype;
+    }
+
+    public void setDatatype(String datatype) {
+        this.datatype = datatype;
+    }
+
+    public String getTable() {
+        return table.toUpperCase();
+    }
+
+    public void setTable(String table) {
+        this.table = table;
+    }
+
+    public int getId() {
+        return id;
+    }
+
+    public void setId(int id) {
+        this.id = id;
+    }
+
+    public JoinDesc getJoin() {
+        return join;
+    }
+
+    public void setJoin(JoinDesc join) {
+        this.join = join;
+    }
+
+    public String getName() {
+        return name;
+    }
+
+    public void setName(String name) {
+        this.name = name;
+    }
+
+    public TblColRef[] getColumnRefs() {
+        return this.columnRefs;
+    }
+
+    public void setColumnRefs(TblColRef[] colRefs) {
+        this.columnRefs = colRefs;
+    }
+
+    public String getColumn() {
+        return this.column;
+    }
+
+    public void setColumn(String column) {
+        this.column = column;
+        if (this.column != null)
+            this.column = this.column.toUpperCase();
+    }
+
+    public HierarchyDesc[] getHierarchy() {
+        return hierarchy;
+    }
+
+    public void setHierarchy(HierarchyDesc[] hierarchy) {
+        this.hierarchy = hierarchy;
+    }
+
+    public String[] getDerived() {
+        return derived;
+    }
+
+    public void setDerived(String[] derived) {
+        this.derived = derived;
+    }
+
+    public TblColRef[] getDerivedColRefs() {
+        return derivedColRefs;
+    }
+
+    public void setDerivedColRefs(TblColRef[] derivedColRefs) {
+        this.derivedColRefs = derivedColRefs;
+    }
+
+    @Override
+    public boolean equals(Object o) {
+        if (this == o)
+            return true;
+        if (o == null || getClass() != o.getClass())
+            return false;
+
+        DimensionDesc that = (DimensionDesc) o;
+
+        if (id != that.id)
+            return false;
+        if (!name.equals(that.name))
+            return false;
+
+        return true;
+    }
+
+    @Override
+    public int hashCode() {
+        int result = id;
+        result = 31 * result + name.hashCode();
+        return result;
+    }
+
+    @Override
+    public String toString() {
+        return "DimensionDesc [name=" + name + ", join=" + join + ", hierarchy=" + Arrays.toString(hierarchy) + ", table=" + table + ", column=" + column + ", datatype=" + datatype + ", derived=" + Arrays.toString(derived) + "]";
+    }
+
+    public void init(Map<String, TableDesc> tables) {
+        if (name != null)
+            name = name.toUpperCase();
+        if (table != null)
+            table = table.toUpperCase();
+        if (column != null)
+            column = column.toUpperCase();
+
+        TableDesc tableDesc = tables.get(table);
+        if (tableDesc == null)
+            throw new IllegalStateException("Can't find table " + table + " on dimension " + name);
+
+        if (hierarchy != null && hierarchy.length == 0)
+            hierarchy = null;
+        if (derived != null && derived.length == 0)
+            derived = null;
+
+        if (join != null) {
+            StringUtil.toUpperCaseArray(join.getForeignKey(), join.getForeignKey());
+            StringUtil.toUpperCaseArray(join.getPrimaryKey(), join.getPrimaryKey());
+        }
+
+        if (hierarchy != null) {
+            for (HierarchyDesc h : hierarchy)
+                h.setColumn(h.getColumn().toUpperCase());
+        }
+
+        if (derived != null) {
+            StringUtil.toUpperCaseArray(derived, derived);
+        }
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/metadata/src/main/java/com/kylinolap/metadata/model/cube/FunctionDesc.java
----------------------------------------------------------------------
diff --git a/metadata/src/main/java/com/kylinolap/metadata/model/cube/FunctionDesc.java b/metadata/src/main/java/com/kylinolap/metadata/model/cube/FunctionDesc.java
new file mode 100644
index 0000000..6b3f92f
--- /dev/null
+++ b/metadata/src/main/java/com/kylinolap/metadata/model/cube/FunctionDesc.java
@@ -0,0 +1,193 @@
+/*
+ * 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.metadata.model.cube;
+
+import com.fasterxml.jackson.annotation.JsonAutoDetect;
+import com.fasterxml.jackson.annotation.JsonAutoDetect.Visibility;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.kylinolap.metadata.model.schema.DataType;
+
+/**
+ * Created with IntelliJ IDEA. User: lukhan Date: 9/26/13 Time: 1:30 PM To
+ * change this template use File | Settings | File Templates.
+ */
+@JsonAutoDetect(fieldVisibility = Visibility.NONE, getterVisibility = Visibility.NONE, isGetterVisibility = Visibility.NONE, setterVisibility = Visibility.NONE)
+public class FunctionDesc {
+
+    public static final String FUNC_SUM = "SUM";
+    public static final String FUNC_MIN = "MIN";
+    public static final String FUNC_MAX = "MAX";
+    public static final String FUNC_COUNT = "COUNT";
+    public static final String FUNC_COUNT_DISTINCT = "COUNT_DISTINCT";
+
+    public static final String PARAMTER_TYPE_CONSTANT = "constant";
+    public static final String PARAMETER_TYPE_COLUMN = "column";
+
+    @JsonProperty("expression")
+    private String expression;
+    @JsonProperty("parameter")
+    private ParameterDesc parameter;
+    @JsonProperty("returntype")
+    private String returnType;
+
+    private DataType returnDataType;
+    private boolean isAppliedOnDimension = false;
+
+    public String getRewriteFieldName() {
+        if (isSum()) {
+            return getParameter().getValue();
+        } else if (isCount()) {
+            return "COUNT__"; // ignores parameter, count(*), count(1),
+                              // count(col) are all the same
+        } else {
+            return getFullExpression().replaceAll("[(), ]", "_");
+        }
+    }
+
+    public boolean needRewrite() {
+        return !isSum() && !isHolisticCountDistinct() && !isAppliedOnDimension();
+    }
+
+    public boolean isMin() {
+        return FUNC_MIN.equalsIgnoreCase(expression);
+    }
+
+    public boolean isMax() {
+        return FUNC_MAX.equalsIgnoreCase(expression);
+    }
+
+    public boolean isSum() {
+        return FUNC_SUM.equalsIgnoreCase(expression);
+    }
+
+    public boolean isCount() {
+        return FUNC_COUNT.equalsIgnoreCase(expression);
+    }
+
+    public boolean isCountDistinct() {
+        return FUNC_COUNT_DISTINCT.equalsIgnoreCase(expression);
+    }
+
+    public boolean isHolisticCountDistinct() {
+        if (isCountDistinct() && returnDataType != null && returnDataType.isBigInt()) {
+            return true;
+        } else {
+            return false;
+        }
+    }
+
+    /**
+     * Get Full Expression such as sum(amount), count(1), count(*)...
+     */
+    public String getFullExpression() {
+        StringBuilder sb = new StringBuilder(expression);
+        sb.append("(");
+        if (parameter != null) {
+            sb.append(parameter.getValue());
+        }
+        sb.append(")");
+        return sb.toString();
+    }
+
+    public boolean isAppliedOnDimension() {
+        return isAppliedOnDimension;
+    }
+
+    public void setAppliedOnDimension(boolean isAppliedOnDimension) {
+        this.isAppliedOnDimension = isAppliedOnDimension;
+    }
+
+    public String getExpression() {
+        return expression;
+    }
+
+    public void setExpression(String expression) {
+        this.expression = expression;
+    }
+
+    public ParameterDesc getParameter() {
+        return parameter;
+    }
+
+    public void setParameter(ParameterDesc parameter) {
+        this.parameter = parameter;
+    }
+
+    public DataType getReturnDataType() {
+        return returnDataType;
+    }
+
+    void setReturnDataType(DataType returnDataType) {
+        this.returnDataType = returnDataType;
+    }
+
+    public String getSQLType() {
+        if (isCountDistinct())
+            return "any";
+        else if (isSum() || isMax() || isMin())
+            return parameter.getColRefs().get(0).getType().getName();
+        else
+            return returnType;
+    }
+
+    public String getReturnType() {
+        return returnType;
+    }
+
+    public void setReturnType(String returnType) {
+        this.returnType = returnType;
+    }
+
+    @Override
+    public int hashCode() {
+        final int prime = 31;
+        int result = 1;
+        result = prime * result + ((expression == null) ? 0 : expression.hashCode());
+        result = prime * result + ((parameter == null) ? 0 : parameter.hashCode());
+        return result;
+    }
+
+    @Override
+    public boolean equals(Object obj) {
+        if (this == obj)
+            return true;
+        if (obj == null)
+            return false;
+        if (getClass() != obj.getClass())
+            return false;
+        FunctionDesc other = (FunctionDesc) obj;
+        if (expression == null) {
+            if (other.expression != null)
+                return false;
+        } else if (!expression.equals(other.expression))
+            return false;
+        // NOTE: don't check the parameter of count()
+        if (isCount() == false) {
+            if (parameter == null) {
+                if (other.parameter != null)
+                    return false;
+            } else if (!parameter.equals(other.parameter))
+                return false;
+        }
+        return true;
+    }
+
+    @Override
+    public String toString() {
+        return "FunctionDesc [expression=" + expression + ", parameter=" + parameter + ", returnType=" + returnType + "]";
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/metadata/src/main/java/com/kylinolap/metadata/model/cube/HBaseColumnDesc.java
----------------------------------------------------------------------
diff --git a/metadata/src/main/java/com/kylinolap/metadata/model/cube/HBaseColumnDesc.java b/metadata/src/main/java/com/kylinolap/metadata/model/cube/HBaseColumnDesc.java
new file mode 100644
index 0000000..5f80eb7
--- /dev/null
+++ b/metadata/src/main/java/com/kylinolap/metadata/model/cube/HBaseColumnDesc.java
@@ -0,0 +1,117 @@
+/*
+ * 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.metadata.model.cube;
+
+import java.util.Arrays;
+
+import com.fasterxml.jackson.annotation.JsonAutoDetect;
+import com.fasterxml.jackson.annotation.JsonAutoDetect.Visibility;
+import com.fasterxml.jackson.annotation.JsonProperty;
+
+/**
+ * Created with IntelliJ IDEA. User: lukhan Date: 9/30/13 Time: 10:57 AM To
+ * change this template use File | Settings | File Templates.
+ */
+@JsonAutoDetect(fieldVisibility = Visibility.NONE, getterVisibility = Visibility.NONE, isGetterVisibility = Visibility.NONE, setterVisibility = Visibility.NONE)
+public class HBaseColumnDesc {
+
+    @JsonProperty("qualifier")
+    private String qualifier;
+    @JsonProperty("measure_refs")
+    private String[] measureRefs;
+
+    // these two will be assemble in runtime.
+    private MeasureDesc[] measures;
+    private String columnFamilyName;
+
+    public String getQualifier() {
+        return qualifier;
+    }
+
+    public void setQualifier(String qualifier) {
+        this.qualifier = qualifier;
+    }
+
+    public String[] getMeasureRefs() {
+        return measureRefs;
+    }
+
+    public void setMeasureRefs(String[] measureRefs) {
+        this.measureRefs = measureRefs;
+    }
+
+    public MeasureDesc[] getMeasures() {
+        return measures;
+    }
+
+    public int findMeasureIndex(FunctionDesc function) {
+        for (int i = 0; i < measures.length; i++) {
+            if (measures[i].getFunction().equals(function)) {
+                return i;
+            }
+        }
+        return -1;
+    }
+
+    public void setMeasures(MeasureDesc[] measures) {
+        this.measures = measures;
+    }
+
+    public String getColumnFamilyName() {
+        return columnFamilyName;
+    }
+
+    public void setColumnFamilyName(String columnFamilyName) {
+        this.columnFamilyName = columnFamilyName;
+    }
+
+    @Override
+    public int hashCode() {
+        final int prime = 31;
+        int result = 1;
+        result = prime * result + ((columnFamilyName == null) ? 0 : columnFamilyName.hashCode());
+        result = prime * result + ((qualifier == null) ? 0 : qualifier.hashCode());
+        return result;
+    }
+
+    @Override
+    public boolean equals(Object obj) {
+        if (this == obj)
+            return true;
+        if (obj == null)
+            return false;
+        if (getClass() != obj.getClass())
+            return false;
+        HBaseColumnDesc other = (HBaseColumnDesc) obj;
+        if (columnFamilyName == null) {
+            if (other.columnFamilyName != null)
+                return false;
+        } else if (!columnFamilyName.equals(other.columnFamilyName))
+            return false;
+        if (qualifier == null) {
+            if (other.qualifier != null)
+                return false;
+        } else if (!qualifier.equals(other.qualifier))
+            return false;
+        return true;
+    }
+
+    @Override
+    public String toString() {
+        return "HBaseColumnDesc [qualifier=" + qualifier + ", measureRefs=" + Arrays.toString(measureRefs) + "]";
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/metadata/src/main/java/com/kylinolap/metadata/model/cube/HBaseColumnFamilyDesc.java
----------------------------------------------------------------------
diff --git a/metadata/src/main/java/com/kylinolap/metadata/model/cube/HBaseColumnFamilyDesc.java b/metadata/src/main/java/com/kylinolap/metadata/model/cube/HBaseColumnFamilyDesc.java
new file mode 100644
index 0000000..4560ec9
--- /dev/null
+++ b/metadata/src/main/java/com/kylinolap/metadata/model/cube/HBaseColumnFamilyDesc.java
@@ -0,0 +1,57 @@
+/*
+ * 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.metadata.model.cube;
+
+import java.util.Arrays;
+
+import com.fasterxml.jackson.annotation.JsonAutoDetect;
+import com.fasterxml.jackson.annotation.JsonAutoDetect.Visibility;
+import com.fasterxml.jackson.annotation.JsonProperty;
+
+/**
+ * Created with IntelliJ IDEA. User: lukhan Date: 9/30/13 Time: 10:41 AM To
+ * change this template use File | Settings | File Templates.
+ */
+@JsonAutoDetect(fieldVisibility = Visibility.NONE, getterVisibility = Visibility.NONE, isGetterVisibility = Visibility.NONE, setterVisibility = Visibility.NONE)
+public class HBaseColumnFamilyDesc {
+
+    @JsonProperty("name")
+    private String name;
+    @JsonProperty("columns")
+    private HBaseColumnDesc[] columns;
+
+    public String getName() {
+        return name;
+    }
+
+    public void setName(String name) {
+        this.name = name;
+    }
+
+    public HBaseColumnDesc[] getColumns() {
+        return columns;
+    }
+
+    public void setColumns(HBaseColumnDesc[] columns) {
+        this.columns = columns;
+    }
+
+    @Override
+    public String toString() {
+        return "HBaseColumnFamilyDesc [name=" + name + ", columns=" + Arrays.toString(columns) + "]";
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/metadata/src/main/java/com/kylinolap/metadata/model/cube/HBaseMappingDesc.java
----------------------------------------------------------------------
diff --git a/metadata/src/main/java/com/kylinolap/metadata/model/cube/HBaseMappingDesc.java b/metadata/src/main/java/com/kylinolap/metadata/model/cube/HBaseMappingDesc.java
new file mode 100644
index 0000000..69488c5
--- /dev/null
+++ b/metadata/src/main/java/com/kylinolap/metadata/model/cube/HBaseMappingDesc.java
@@ -0,0 +1,92 @@
+/*
+ * 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.metadata.model.cube;
+
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.LinkedList;
+
+import com.fasterxml.jackson.annotation.JsonAutoDetect;
+import com.fasterxml.jackson.annotation.JsonAutoDetect.Visibility;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.kylinolap.common.util.StringUtil;
+
+/**
+ * Created with IntelliJ IDEA. User: lukhan Date: 9/24/13 Time: 10:44 AM To
+ * change this template use File | Settings | File Templates.
+ */
+@JsonAutoDetect(fieldVisibility = Visibility.NONE, getterVisibility = Visibility.NONE, isGetterVisibility = Visibility.NONE, setterVisibility = Visibility.NONE)
+public class HBaseMappingDesc {
+
+    @JsonProperty("column_family")
+    private HBaseColumnFamilyDesc[] columnFamily;
+
+    // point to the cube instance which contain this HBaseMappingDesc instance.
+    private CubeDesc cubeRef;
+
+    public Collection<HBaseColumnDesc> findHBaseColumnByFunction(FunctionDesc function) {
+        Collection<HBaseColumnDesc> result = new LinkedList<HBaseColumnDesc>();
+        HBaseMappingDesc hbaseMapping = cubeRef.getHBaseMapping();
+        if (hbaseMapping == null || hbaseMapping.getColumnFamily() == null) {
+            return result;
+        }
+        for (HBaseColumnFamilyDesc cf : hbaseMapping.getColumnFamily()) {
+            for (HBaseColumnDesc c : cf.getColumns()) {
+                for (MeasureDesc m : c.getMeasures()) {
+                    if (m.getFunction().equals(function)) {
+                        result.add(c);
+                    }
+                }
+            }
+        }
+        return result;
+    }
+
+    public CubeDesc getCubeRef() {
+        return cubeRef;
+    }
+
+    public void setCubeRef(CubeDesc cubeRef) {
+        this.cubeRef = cubeRef;
+    }
+
+    public HBaseColumnFamilyDesc[] getColumnFamily() {
+        return columnFamily;
+    }
+
+    public void setColumnFamily(HBaseColumnFamilyDesc[] columnFamily) {
+        this.columnFamily = columnFamily;
+    }
+
+    public void init(CubeDesc cubeDesc) {
+        cubeRef = cubeDesc;
+
+        for (HBaseColumnFamilyDesc cf : columnFamily) {
+            cf.setName(cf.getName().toUpperCase());
+
+            for (HBaseColumnDesc c : cf.getColumns()) {
+                c.setQualifier(c.getQualifier().toUpperCase());
+                StringUtil.toUpperCaseArray(c.getMeasureRefs(), c.getMeasureRefs());
+            }
+        }
+    }
+
+    @Override
+    public String toString() {
+        return "HBaseMappingDesc [columnFamily=" + Arrays.toString(columnFamily) + "]";
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/metadata/src/main/java/com/kylinolap/metadata/model/cube/HierarchyDesc.java
----------------------------------------------------------------------
diff --git a/metadata/src/main/java/com/kylinolap/metadata/model/cube/HierarchyDesc.java b/metadata/src/main/java/com/kylinolap/metadata/model/cube/HierarchyDesc.java
new file mode 100644
index 0000000..37af9d9
--- /dev/null
+++ b/metadata/src/main/java/com/kylinolap/metadata/model/cube/HierarchyDesc.java
@@ -0,0 +1,65 @@
+/*
+ * 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.metadata.model.cube;
+
+import com.fasterxml.jackson.annotation.JsonAutoDetect;
+import com.fasterxml.jackson.annotation.JsonAutoDetect.Visibility;
+import com.fasterxml.jackson.annotation.JsonProperty;
+
+/**
+ * Created with IntelliJ IDEA. User: lukhan Date: 9/24/13 Time: 10:46 AM To
+ * change this template use File | Settings | File Templates.
+ */
+@JsonAutoDetect(fieldVisibility = Visibility.NONE, getterVisibility = Visibility.NONE, isGetterVisibility = Visibility.NONE, setterVisibility = Visibility.NONE)
+public class HierarchyDesc {
+
+    @JsonProperty("level")
+    private String level;
+    @JsonProperty("column")
+    private String column;
+
+    private TblColRef columnRef;
+
+    public String getLevel() {
+        return level;
+    }
+
+    public void setLevel(String level) {
+        this.level = level;
+    }
+
+    public TblColRef getColumnRef() {
+        return columnRef;
+    }
+
+    public void setColumnRef(TblColRef column) {
+        this.columnRef = column;
+    }
+
+    public String getColumn() {
+        return column;
+    }
+
+    public void setColumn(String columnName) {
+        this.column = columnName;
+    }
+
+    @Override
+    public String toString() {
+        return "HierarchyDesc [level=" + level + ", column=" + column + "]";
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/metadata/src/main/java/com/kylinolap/metadata/model/cube/JoinDesc.java
----------------------------------------------------------------------
diff --git a/metadata/src/main/java/com/kylinolap/metadata/model/cube/JoinDesc.java b/metadata/src/main/java/com/kylinolap/metadata/model/cube/JoinDesc.java
new file mode 100644
index 0000000..44ce795
--- /dev/null
+++ b/metadata/src/main/java/com/kylinolap/metadata/model/cube/JoinDesc.java
@@ -0,0 +1,134 @@
+/*
+ * 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.metadata.model.cube;
+
+import java.util.Arrays;
+
+import com.fasterxml.jackson.annotation.JsonAutoDetect;
+import com.fasterxml.jackson.annotation.JsonAutoDetect.Visibility;
+import com.fasterxml.jackson.annotation.JsonProperty;
+
+/**
+ * Created with IntelliJ IDEA. User: lukhan Date: 10/14/13 Time: 2:16 PM To
+ * change this template use File | Settings | File Templates.
+ */
+@JsonAutoDetect(fieldVisibility = Visibility.NONE, getterVisibility = Visibility.NONE, isGetterVisibility = Visibility.NONE, setterVisibility = Visibility.NONE)
+public class JoinDesc {
+
+    // inner, left, right, outer...
+    @JsonProperty("type")
+    private String type;
+    @JsonProperty("primary_key")
+    private String[] primaryKey;
+    @JsonProperty("foreign_key")
+    private String[] foreignKey;
+
+    private TblColRef[] primaryKeyColumns;
+    private TblColRef[] foreignKeyColumns;
+
+    public void swapPKFK() {
+        String[] t = primaryKey;
+        primaryKey = foreignKey;
+        foreignKey = t;
+
+        TblColRef[] tt = primaryKeyColumns;
+        primaryKeyColumns = foreignKeyColumns;
+        foreignKeyColumns = tt;
+    }
+
+    public String getType() {
+        return type;
+    }
+
+    public void setType(String type) {
+        this.type = type;
+    }
+
+    public String[] getPrimaryKey() {
+        return primaryKey;
+    }
+
+    public void setPrimaryKey(String[] primaryKey) {
+        this.primaryKey = primaryKey;
+    }
+
+    public String[] getForeignKey() {
+        return foreignKey;
+    }
+
+    public void setForeignKey(String[] foreignKey) {
+        this.foreignKey = foreignKey;
+    }
+
+    public TblColRef[] getPrimaryKeyColumns() {
+        return primaryKeyColumns;
+    }
+
+    public void setPrimaryKeyColumns(TblColRef[] primaryKeyColumns) {
+        this.primaryKeyColumns = primaryKeyColumns;
+    }
+
+    public TblColRef[] getForeignKeyColumns() {
+        return foreignKeyColumns;
+    }
+
+    public void setForeignKeyColumns(TblColRef[] foreignKeyColumns) {
+        this.foreignKeyColumns = foreignKeyColumns;
+    }
+
+    @Override
+    public int hashCode() {
+        final int prime = 31;
+        int result = 1;
+        result = prime * result + Arrays.hashCode(primaryKeyColumns);
+        result = prime * result + Arrays.hashCode(foreignKeyColumns);
+        result = prime * result + this.type.hashCode();
+        return result;
+    }
+
+    @Override
+    public boolean equals(Object obj) {
+        if (this == obj)
+            return true;
+        if (obj == null)
+            return false;
+        if (getClass() != obj.getClass())
+            return false;
+        JoinDesc other = (JoinDesc) obj;
+
+        if (!this.columnsEqualIgnoringOrder(foreignKeyColumns, other.foreignKeyColumns))
+            return false;
+        if (!this.columnsEqualIgnoringOrder(primaryKeyColumns, other.primaryKeyColumns))
+            return false;
+
+        if (!this.type.equalsIgnoreCase(other.getType()))
+            return false;
+        return true;
+    }
+
+    private boolean columnsEqualIgnoringOrder(TblColRef[] a, TblColRef[] b) {
+        if (a.length != b.length)
+            return false;
+
+        return Arrays.asList(a).containsAll(Arrays.asList(b));
+    }
+
+    @Override
+    public String toString() {
+        return "JoinDesc [type=" + type + ", primary_key=" + Arrays.toString(primaryKey) + ", foreign_key=" + Arrays.toString(foreignKey) + "]";
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/metadata/src/main/java/com/kylinolap/metadata/model/cube/MeasureDesc.java
----------------------------------------------------------------------
diff --git a/metadata/src/main/java/com/kylinolap/metadata/model/cube/MeasureDesc.java b/metadata/src/main/java/com/kylinolap/metadata/model/cube/MeasureDesc.java
new file mode 100644
index 0000000..3ee07a4
--- /dev/null
+++ b/metadata/src/main/java/com/kylinolap/metadata/model/cube/MeasureDesc.java
@@ -0,0 +1,100 @@
+/*
+ * 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.metadata.model.cube;
+
+import com.fasterxml.jackson.annotation.JsonAutoDetect;
+import com.fasterxml.jackson.annotation.JsonAutoDetect.Visibility;
+import com.fasterxml.jackson.annotation.JsonProperty;
+
+/**
+ * Created with IntelliJ IDEA. User: lukhan Date: 9/24/13 Time: 10:41 AM To
+ * change this template use File | Settings | File Templates.
+ */
+
+@JsonAutoDetect(fieldVisibility = Visibility.NONE, getterVisibility = Visibility.NONE, isGetterVisibility = Visibility.NONE, setterVisibility = Visibility.NONE)
+public class MeasureDesc {
+
+    @JsonProperty("id")
+    private int id;
+    @JsonProperty("name")
+    private String name;
+    @JsonProperty("function")
+    private FunctionDesc function;
+    @JsonProperty("dependent_measure_ref")
+    private String dependentMeasureRef;
+
+    public int getId() {
+        return id;
+    }
+
+    public void setId(int id) {
+        this.id = id;
+    }
+
+    public String getName() {
+        return name;
+    }
+
+    public void setName(String name) {
+        this.name = name;
+    }
+
+    public FunctionDesc getFunction() {
+        return function;
+    }
+
+    public void setFunction(FunctionDesc function) {
+        this.function = function;
+    }
+
+    public String getDependentMeasureRef() {
+        return dependentMeasureRef;
+    }
+
+    public void setDependentMeasureRef(String dependentMeasureRef) {
+        this.dependentMeasureRef = dependentMeasureRef;
+    }
+
+    public boolean isHolisticCountDistinct() {
+        return function.isHolisticCountDistinct();
+    }
+
+    @Override
+    public boolean equals(Object o) {
+        if (this == o)
+            return true;
+        if (o == null || getClass() != o.getClass())
+            return false;
+
+        MeasureDesc that = (MeasureDesc) o;
+
+        if (id != that.id)
+            return false;
+
+        return true;
+    }
+
+    @Override
+    public int hashCode() {
+        return id;
+    }
+
+    @Override
+    public String toString() {
+        return "MeasureDesc [name=" + name + ", function=" + function + "]";
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/metadata/src/main/java/com/kylinolap/metadata/model/cube/ParameterDesc.java
----------------------------------------------------------------------
diff --git a/metadata/src/main/java/com/kylinolap/metadata/model/cube/ParameterDesc.java b/metadata/src/main/java/com/kylinolap/metadata/model/cube/ParameterDesc.java
new file mode 100644
index 0000000..b43df48
--- /dev/null
+++ b/metadata/src/main/java/com/kylinolap/metadata/model/cube/ParameterDesc.java
@@ -0,0 +1,122 @@
+/*
+ * 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.metadata.model.cube;
+
+import java.io.UnsupportedEncodingException;
+import java.util.Arrays;
+import java.util.List;
+
+import org.apache.hadoop.util.StringUtils;
+
+import com.fasterxml.jackson.annotation.JsonAutoDetect;
+import com.fasterxml.jackson.annotation.JsonAutoDetect.Visibility;
+import com.fasterxml.jackson.annotation.JsonProperty;
+
+/**
+ * Created with IntelliJ IDEA. User: lukhan Date: 9/30/13 Time: 2:46 PM To
+ * change this template use File | Settings | File Templates.
+ */
+@JsonAutoDetect(fieldVisibility = Visibility.NONE, getterVisibility = Visibility.NONE, isGetterVisibility = Visibility.NONE, setterVisibility = Visibility.NONE)
+public class ParameterDesc {
+
+    public static final String COLUMN_TYPE = "column";
+
+    @JsonProperty("type")
+    private String type;
+    @JsonProperty("value")
+    private String value;
+
+    private List<TblColRef> colRefs;
+
+    public String getType() {
+        return type;
+    }
+
+    public void setType(String type) {
+        this.type = type;
+    }
+
+    public byte[] getBytes() throws UnsupportedEncodingException {
+        return value.getBytes("UTF-8");
+    }
+
+    public String getValue() {
+        return value;
+    }
+
+    public void setValue(String value) {
+        this.value = value;
+    }
+
+    public List<TblColRef> getColRefs() {
+        return colRefs;
+    }
+
+    public void setColRefs(List<TblColRef> colRefs) {
+        this.colRefs = colRefs;
+    }
+
+    public boolean isColumnType() {
+        return COLUMN_TYPE.equals(type);
+    }
+
+    public void normalizeColumnValue() {
+        if (isColumnType()) {
+            String values[] = value.split("\\s*,\\s*");
+            for (int i = 0; i < values.length; i++)
+                values[i] = values[i].toUpperCase();
+            Arrays.sort(values);
+            value = StringUtils.join(",", values);
+        }
+    }
+
+    @Override
+    public int hashCode() {
+        final int prime = 31;
+        int result = 1;
+        result = prime * result + ((type == null) ? 0 : type.hashCode());
+        result = prime * result + ((value == null) ? 0 : value.hashCode());
+        return result;
+    }
+
+    @Override
+    public boolean equals(Object obj) {
+        if (this == obj)
+            return true;
+        if (obj == null)
+            return false;
+        if (getClass() != obj.getClass())
+            return false;
+        ParameterDesc other = (ParameterDesc) obj;
+        if (type == null) {
+            if (other.type != null)
+                return false;
+        } else if (!type.equals(other.type))
+            return false;
+        if (value == null) {
+            if (other.value != null)
+                return false;
+        } else if (!value.equals(other.value))
+            return false;
+        return true;
+    }
+
+    @Override
+    public String toString() {
+        return "ParameterDesc [type=" + type + ", value=" + value + "]";
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/metadata/src/main/java/com/kylinolap/metadata/model/cube/RowKeyColDesc.java
----------------------------------------------------------------------
diff --git a/metadata/src/main/java/com/kylinolap/metadata/model/cube/RowKeyColDesc.java b/metadata/src/main/java/com/kylinolap/metadata/model/cube/RowKeyColDesc.java
new file mode 100644
index 0000000..e489f08
--- /dev/null
+++ b/metadata/src/main/java/com/kylinolap/metadata/model/cube/RowKeyColDesc.java
@@ -0,0 +1,84 @@
+/*
+ * 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.metadata.model.cube;
+
+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 RowKeyColDesc {
+
+    @JsonProperty("column")
+    private String column;
+    @JsonProperty("length")
+    private int length;
+    @JsonProperty("dictionary")
+    private String dictionary;
+    @JsonProperty("mandatory")
+    private boolean mandatory = false;
+
+    // computed
+    private int bitIndex;
+    private TblColRef colRef;
+
+    public String getDictionary() {
+        return dictionary;
+    }
+
+    public String getColumn() {
+        return column;
+    }
+
+    void setColumn(String column) {
+        this.column = column;
+    }
+
+    public int getLength() {
+        return length;
+    }
+
+    public boolean isMandatory() {
+        return mandatory;
+    }
+
+    public int getBitIndex() {
+        return bitIndex;
+    }
+
+    void setBitIndex(int index) {
+        this.bitIndex = index;
+    }
+
+    public TblColRef getColRef() {
+        return colRef;
+    }
+
+    void setColRef(TblColRef colRef) {
+        this.colRef = colRef;
+    }
+
+    @Override
+    public String toString() {
+        return "RowKeyColDesc [column=" + column + ", length=" + length + ", dictionary=" + dictionary + ", mandatory=" + mandatory + "]";
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/metadata/src/main/java/com/kylinolap/metadata/model/cube/RowKeyDesc.java
----------------------------------------------------------------------
diff --git a/metadata/src/main/java/com/kylinolap/metadata/model/cube/RowKeyDesc.java b/metadata/src/main/java/com/kylinolap/metadata/model/cube/RowKeyDesc.java
new file mode 100644
index 0000000..97372f7
--- /dev/null
+++ b/metadata/src/main/java/com/kylinolap/metadata/model/cube/RowKeyDesc.java
@@ -0,0 +1,290 @@
+/*
+ * 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.metadata.model.cube;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.commons.lang.StringUtils;
+
+import com.fasterxml.jackson.annotation.JsonAutoDetect;
+import com.fasterxml.jackson.annotation.JsonAutoDetect.Visibility;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.kylinolap.common.util.StringUtil;
+
+/**
+ * Created by lukhan on 1/2/14.
+ */
+@JsonAutoDetect(fieldVisibility = Visibility.NONE, getterVisibility = Visibility.NONE, isGetterVisibility = Visibility.NONE, setterVisibility = Visibility.NONE)
+public class RowKeyDesc {
+
+    public static class HierarchyMask {
+        public long fullMask;
+        public long[] allMasks;
+    }
+
+    public static class AggrGroupMask {
+        public AggrGroupMask(int size) {
+            groupOneBitMasks = new long[size];
+        }
+
+        public long groupMask;
+        public long groupOneBitMasks[];
+        public long uniqueMask;
+        public long leftoverMask;
+    }
+
+    @JsonProperty("rowkey_columns")
+    private RowKeyColDesc[] rowkeyColumns;
+    @JsonProperty("aggregation_groups")
+    private String[][] aggregationGroups;
+
+    // computed content
+    private CubeDesc cubeRef;
+    private Map<TblColRef, RowKeyColDesc> columnMap;
+
+    private long fullMask;
+    private long mandatoryColumnMask;
+    private AggrGroupMask[] aggrGroupMasks;
+    private long aggrGroupFullMask;
+    private long tailMask;
+
+    private List<HierarchyMask> hierarchyMasks;
+
+    public RowKeyColDesc[] getRowKeyColumns() {
+        return rowkeyColumns;
+    }
+
+    // search a specific row key col
+    public int getRowKeyIndexByColumnName(String columnName) {
+        if (this.rowkeyColumns == null)
+            return -1;
+
+        for (int i = 0; i < this.rowkeyColumns.length; ++i) {
+            RowKeyColDesc desc = this.rowkeyColumns[i];
+            if (desc.getColumn().equalsIgnoreCase(columnName)) {
+                return i;
+            }
+        }
+        return -1;
+    }
+
+    public int getNCuboidBuildLevels() {
+        // N aggregation columns requires N levels of cuboid build
+        // - N columns requires N-1 levels build
+        // - zero tail cuboid needs one more additional level
+        Set<String> aggDims = new HashSet<String>();
+        for (String[] aggrGroup : aggregationGroups) {
+            for (String dim : aggrGroup) {
+                aggDims.add(dim);
+            }
+        }
+        return aggDims.size();
+    }
+
+    public String[][] getAggregationGroups() {
+        return aggregationGroups;
+    }
+
+    public CubeDesc getCubeRef() {
+        return cubeRef;
+    }
+
+    public void setCubeRef(CubeDesc cubeRef) {
+        this.cubeRef = cubeRef;
+    }
+
+    public long getFullMask() {
+        return fullMask;
+    }
+
+    public long getMandatoryColumnMask() {
+        return mandatoryColumnMask;
+    }
+
+    public long getAggrGroupFullMask() {
+        return aggrGroupFullMask;
+    }
+
+    public AggrGroupMask[] getAggrGroupMasks() {
+        return aggrGroupMasks;
+    }
+
+    public List<HierarchyMask> getHierarchyMasks() {
+        return hierarchyMasks;
+    }
+
+    public long getTailMask() {
+        return tailMask;
+    }
+
+    public int getColumnBitIndex(TblColRef col) {
+        return getColDesc(col).getBitIndex();
+    }
+
+    public int getColumnLength(TblColRef col) {
+        return getColDesc(col).getLength();
+    }
+
+    public String getDictionary(TblColRef col) {
+        return getColDesc(col).getDictionary();
+    }
+
+    private RowKeyColDesc getColDesc(TblColRef col) {
+        RowKeyColDesc desc = columnMap.get(col);
+        if (desc == null)
+            throw new NullPointerException("Column " + col + " does not exist in row key desc");
+        return desc;
+    }
+    
+    public boolean isUseDictionary(TblColRef col) {
+        String useDictionary = getDictionary(col);
+        return !StringUtils.isBlank(useDictionary) && !"false".equals(useDictionary);
+    }
+
+    public boolean isUseDictionary() {
+        for (RowKeyColDesc col : getRowKeyColumns()) {
+            if (isUseDictionary(col.getColRef())) {
+                return true;
+            }
+        }
+        return false;
+    }
+
+    public void init(CubeDesc cube) {
+        setCubeRef(cube);
+        Map<String, TblColRef> colNameAbbr = cube.buildColumnNameAbbreviation();
+
+        buildRowKey(colNameAbbr);
+        buildAggregationGroups(colNameAbbr);
+        buildHierarchyMasks();
+    }
+
+    @Override
+    public String toString() {
+        return "RowKeyDesc [rowkeyColumns=" + Arrays.toString(rowkeyColumns) + ", aggregationGroups=" + Arrays.toString(aggregationGroups) + "]";
+    }
+
+    private void buildRowKey(Map<String, TblColRef> colNameAbbr) {
+        columnMap = new HashMap<TblColRef, RowKeyColDesc>();
+        mandatoryColumnMask = 0;
+
+        for (int i = 0; i < rowkeyColumns.length; i++) {
+            RowKeyColDesc col = rowkeyColumns[i];
+            col.setColumn(col.getColumn().toUpperCase());
+            col.setBitIndex(rowkeyColumns.length - i - 1);
+            col.setColRef(colNameAbbr.get(col.getColumn()));
+            if (col.getColRef() == null)
+                throw new IllegalArgumentException("Cannot find rowkey column " + col.getColumn() + " in cube " + cubeRef);
+
+            columnMap.put(col.getColRef(), col);
+
+            if (col.isMandatory()) {
+                mandatoryColumnMask |= 1L << col.getBitIndex();
+            }
+        }
+    }
+
+    private void buildAggregationGroups(Map<String, TblColRef> colNameAbbr) {
+        if (aggregationGroups == null) {
+            aggregationGroups = new String[0][];
+        }
+
+        for (int i = 0; i < aggregationGroups.length; i++) {
+            StringUtil.toUpperCaseArray(aggregationGroups[i], this.aggregationGroups[i]);
+        }
+
+        for (int i = 0; i < this.rowkeyColumns.length; i++) {
+            int index = rowkeyColumns[i].getBitIndex();
+            this.fullMask |= 1L << index;
+        }
+
+        this.aggrGroupMasks = new AggrGroupMask[aggregationGroups.length];
+        for (int i = 0; i < this.aggregationGroups.length; i++) {
+            String[] aggGrp = this.aggregationGroups[i];
+            AggrGroupMask mask = new AggrGroupMask(aggGrp.length);
+
+            for (int j = 0; j < aggGrp.length; j++) {
+                TblColRef aggCol = colNameAbbr.get(aggGrp[j].toUpperCase());
+                if (aggCol == null) {
+                    throw new IllegalArgumentException("Can't find aggregation column " + aggGrp[j] + " in  cube " + this.cubeRef.getName());
+                }
+                Integer index = getColumnBitIndex(aggCol);
+                mask.groupMask |= 1L << index;
+                mask.groupOneBitMasks[j] = 1L << index;
+                this.aggrGroupFullMask |= 1L << index;
+            }
+            this.aggrGroupMasks[i] = mask;
+        }
+
+        this.tailMask = fullMask ^ mandatoryColumnMask ^ aggrGroupFullMask;
+
+        // unique mask = (bits in this group) - (bits in following groups)
+        // leftover mask = (tail bits) + (bits in following groups) - (bits in
+        // this group)
+        for (int i = 0; i < aggrGroupMasks.length; i++) {
+            AggrGroupMask mask = aggrGroupMasks[i];
+
+            mask.uniqueMask = mask.groupMask;
+            for (int j = i + 1; j < aggrGroupMasks.length; j++) {
+                mask.uniqueMask &= ~aggrGroupMasks[j].groupMask;
+            }
+
+            mask.leftoverMask = tailMask;
+            for (int j = i + 1; j < aggrGroupMasks.length; j++) {
+                mask.leftoverMask |= aggrGroupMasks[j].groupMask;
+            }
+            mask.leftoverMask &= ~mask.groupMask;
+        }
+    }
+
+    private void buildHierarchyMasks() {
+        this.hierarchyMasks = new ArrayList<HierarchyMask>();
+
+        for (DimensionDesc dimension : this.cubeRef.getDimensions()) {
+            HierarchyDesc[] hierarchies = dimension.getHierarchy();
+            if (hierarchies == null || hierarchies.length == 0)
+                continue;
+
+            HierarchyMask mask = new HierarchyMask();
+            ArrayList<Long> allMaskList = new ArrayList<Long>();
+            for (int i = 0; i < hierarchies.length; i++) {
+                TblColRef hColumn = hierarchies[i].getColumnRef();
+                Integer index = getColumnBitIndex(hColumn);
+                long bit = 1L << index;
+
+                if ((tailMask & bit) > 0)
+                    continue; // ignore levels in tail, they don't participate
+                              // aggregation group combination anyway
+
+                mask.fullMask |= bit;
+                allMaskList.add(mask.fullMask);
+            }
+
+            mask.allMasks = new long[allMaskList.size()];
+            for (int i = 0; i < allMaskList.size(); i++)
+                mask.allMasks[i] = allMaskList.get(i);
+
+            this.hierarchyMasks.add(mask);
+        }
+    }
+
+}


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

Posted by lu...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/query/src/main/java/com/kylinolap/query/relnode/OLAPLimitRel.java
----------------------------------------------------------------------
diff --git a/query/src/main/java/com/kylinolap/query/relnode/OLAPLimitRel.java b/query/src/main/java/com/kylinolap/query/relnode/OLAPLimitRel.java
new file mode 100644
index 0000000..205803b
--- /dev/null
+++ b/query/src/main/java/com/kylinolap/query/relnode/OLAPLimitRel.java
@@ -0,0 +1,134 @@
+/*
+ * 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.query.relnode;
+
+import java.util.List;
+
+import net.hydromatic.optiq.rules.java.EnumerableConvention;
+import net.hydromatic.optiq.rules.java.EnumerableRel;
+import net.hydromatic.optiq.rules.java.EnumerableRelImplementor;
+import net.hydromatic.optiq.rules.java.JavaRules.EnumerableLimitRel;
+
+import org.eigenbase.rel.RelNode;
+import org.eigenbase.rel.RelWriter;
+import org.eigenbase.rel.SingleRel;
+import org.eigenbase.relopt.RelOptCluster;
+import org.eigenbase.relopt.RelOptCost;
+import org.eigenbase.relopt.RelOptPlanner;
+import org.eigenbase.relopt.RelTrait;
+import org.eigenbase.relopt.RelTraitSet;
+import org.eigenbase.rex.RexLiteral;
+import org.eigenbase.rex.RexNode;
+
+import com.google.common.base.Preconditions;
+
+/**
+ * 
+ * @author xjiang
+ * 
+ */
+public class OLAPLimitRel extends SingleRel implements OLAPRel, EnumerableRel {
+
+    private final RexNode localOffset; // avoid same name in parent class
+    private final RexNode localFetch; // avoid same name in parent class
+    private ColumnRowType columnRowType;
+    private OLAPContext context;
+
+    public OLAPLimitRel(RelOptCluster cluster, RelTraitSet traitSet, RelNode child, RexNode offset, RexNode fetch) {
+        super(cluster, traitSet, child);
+        Preconditions.checkArgument(getConvention() == OLAPRel.CONVENTION);
+        Preconditions.checkArgument(getConvention() == child.getConvention());
+        this.localOffset = offset;
+        this.localFetch = fetch;
+    }
+
+    @Override
+    public OLAPLimitRel copy(RelTraitSet traitSet, List<RelNode> inputs) {
+        return new OLAPLimitRel(getCluster(), traitSet, sole(inputs), localOffset, localFetch);
+    }
+
+    @Override
+    public RelOptCost computeSelfCost(RelOptPlanner planner) {
+        return super.computeSelfCost(planner).multiplyBy(.05);
+    }
+
+    @Override
+    public RelWriter explainTerms(RelWriter pw) {
+        return super.explainTerms(pw).itemIf("offset", localOffset, localOffset != null).itemIf("fetch", localFetch, localFetch != null);
+    }
+
+    @Override
+    public void implementOLAP(OLAPImplementor implementor) {
+        implementor.visitChild(getChild(), this);
+
+        this.columnRowType = buildColumnRowType();
+
+        this.context = implementor.getContext();
+        Number limitValue = (Number) (((RexLiteral) localFetch).getValue());
+        int limit = limitValue.intValue();
+        this.context.storageContext.setLimit(limit);
+    }
+
+    private ColumnRowType buildColumnRowType() {
+        OLAPRel olapChild = (OLAPRel) getChild();
+        ColumnRowType inputColumnRowType = olapChild.getColumnRowType();
+        return inputColumnRowType;
+    }
+
+    @Override
+    public void implementRewrite(RewriteImplementor implementor) {
+        implementor.visitChild(this, getChild());
+
+        this.rowType = this.deriveRowType();
+        this.columnRowType = buildColumnRowType();
+    }
+
+    @Override
+    public Result implement(EnumerableRelImplementor implementor, Prefer pref) {
+        OLAPRel childRel = (OLAPRel) getChild();
+        childRel.replaceTraitSet(EnumerableConvention.INSTANCE);
+
+        EnumerableLimitRel enumLimit = new EnumerableLimitRel(getCluster(), getCluster().traitSetOf(EnumerableConvention.INSTANCE), getChild(), localOffset, localFetch);
+        Result res = enumLimit.implement(implementor, pref);
+
+        childRel.replaceTraitSet(OLAPRel.CONVENTION);
+
+        return res;
+    }
+
+    @Override
+    public OLAPContext getContext() {
+        return context;
+    }
+
+    @Override
+    public ColumnRowType getColumnRowType() {
+        return columnRowType;
+    }
+
+    @Override
+    public boolean hasSubQuery() {
+        OLAPRel olapChild = (OLAPRel) getChild();
+        return olapChild.hasSubQuery();
+    }
+
+    @Override
+    public RelTraitSet replaceTraitSet(RelTrait trait) {
+        RelTraitSet oldTraitSet = this.traitSet;
+        this.traitSet = this.traitSet.replace(trait);
+        return oldTraitSet;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/query/src/main/java/com/kylinolap/query/relnode/OLAPProjectRel.java
----------------------------------------------------------------------
diff --git a/query/src/main/java/com/kylinolap/query/relnode/OLAPProjectRel.java b/query/src/main/java/com/kylinolap/query/relnode/OLAPProjectRel.java
new file mode 100644
index 0000000..5f45e3e
--- /dev/null
+++ b/query/src/main/java/com/kylinolap/query/relnode/OLAPProjectRel.java
@@ -0,0 +1,299 @@
+/*
+ * 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.query.relnode;
+
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import net.hydromatic.optiq.rules.java.EnumerableConvention;
+import net.hydromatic.optiq.rules.java.EnumerableRel;
+import net.hydromatic.optiq.rules.java.EnumerableRelImplementor;
+import net.hydromatic.optiq.rules.java.JavaRules.EnumerableCalcRel;
+
+import org.eigenbase.rel.ProjectRelBase;
+import org.eigenbase.rel.RelCollation;
+import org.eigenbase.rel.RelNode;
+import org.eigenbase.relopt.RelOptCluster;
+import org.eigenbase.relopt.RelOptCost;
+import org.eigenbase.relopt.RelOptPlanner;
+import org.eigenbase.relopt.RelTrait;
+import org.eigenbase.relopt.RelTraitSet;
+import org.eigenbase.reltype.RelDataType;
+import org.eigenbase.reltype.RelDataTypeFactory.FieldInfoBuilder;
+import org.eigenbase.reltype.RelDataTypeField;
+import org.eigenbase.reltype.RelDataTypeFieldImpl;
+import org.eigenbase.rex.RexCall;
+import org.eigenbase.rex.RexInputRef;
+import org.eigenbase.rex.RexLiteral;
+import org.eigenbase.rex.RexNode;
+import org.eigenbase.rex.RexProgram;
+import org.eigenbase.sql.SqlOperator;
+import org.eigenbase.sql.fun.SqlCaseOperator;
+import org.eigenbase.sql.fun.SqlStdOperatorTable;
+import org.eigenbase.sql.validate.SqlUserDefinedFunction;
+
+import com.google.common.base.Preconditions;
+import com.google.common.collect.ImmutableList;
+import com.kylinolap.metadata.model.cube.TblColRef;
+import com.kylinolap.metadata.model.cube.TblColRef.InnerDataTypeEnum;
+
+/**
+ * 
+ * @author xjiang
+ * 
+ */
+public class OLAPProjectRel extends ProjectRelBase implements OLAPRel, EnumerableRel {
+
+    private OLAPContext context;
+    private List<RexNode> rewriteProjects;
+    private boolean rewriting;
+    private ColumnRowType columnRowType;
+    private boolean hasJoin;
+    private boolean afterJoin;
+    private boolean afterAggregate;
+
+    public OLAPProjectRel(RelOptCluster cluster, RelTraitSet traitSet, RelNode child, List<RexNode> exps, RelDataType rowType, int flags) {
+        super(cluster, traitSet, child, exps, rowType, flags);
+        Preconditions.checkArgument(getConvention() == OLAPRel.CONVENTION);
+        Preconditions.checkArgument(child.getConvention() == OLAPRel.CONVENTION);
+        this.rewriteProjects = exps;
+        this.hasJoin = false;
+        this.afterJoin = false;
+        this.rowType = getRowType();
+    }
+
+    @Override
+    public List<RexNode> getChildExps() {
+        return rewriteProjects;
+    }
+
+    @Override
+    public List<RexNode> getProjects() {
+        return rewriteProjects;
+    }
+
+    @Override
+    public RelOptCost computeSelfCost(RelOptPlanner planner) {
+        return super.computeSelfCost(planner).multiplyBy(.05);
+    }
+
+    @Override
+    public ProjectRelBase copy(RelTraitSet traitSet, RelNode child, List<RexNode> exps, RelDataType rowType) {
+        return new OLAPProjectRel(getCluster(), traitSet, child, exps, rowType, this.flags);
+    }
+
+    @Override
+    public void implementOLAP(OLAPImplementor implementor) {
+        implementor.visitChild(getChild(), this);
+
+        this.context = implementor.getContext();
+        this.hasJoin = context.hasJoin;
+        this.afterJoin = context.afterJoin;
+        this.afterAggregate = context.afterAggregate;
+
+        this.columnRowType = buildColumnRowType();
+    }
+
+    private ColumnRowType buildColumnRowType() {
+        List<TblColRef> columns = new ArrayList<TblColRef>();
+        List<Set<TblColRef>> sourceColumns = new ArrayList<Set<TblColRef>>();
+        OLAPRel olapChild = (OLAPRel) getChild();
+        ColumnRowType inputColumnRowType = olapChild.getColumnRowType();
+        for (int i = 0; i < this.rewriteProjects.size(); i++) {
+            RexNode rex = this.rewriteProjects.get(i);
+            RelDataTypeField columnField = this.rowType.getFieldList().get(i);
+            String fieldName = columnField.getName();
+            Set<TblColRef> sourceCollector = new HashSet<TblColRef>();
+            TblColRef column = translateRexNode(rex, inputColumnRowType, fieldName, sourceCollector);
+            columns.add(column);
+            sourceColumns.add(sourceCollector);
+        }
+        return new ColumnRowType(columns, sourceColumns);
+    }
+
+    private TblColRef translateRexNode(RexNode rexNode, ColumnRowType inputColumnRowType, String fieldName, Set<TblColRef> sourceCollector) {
+        TblColRef column = null;
+        if (rexNode instanceof RexInputRef) {
+            RexInputRef inputRef = (RexInputRef) rexNode;
+            column = translateRexInputRef(inputRef, inputColumnRowType, fieldName, sourceCollector);
+        } else if (rexNode instanceof RexLiteral) {
+            RexLiteral literal = (RexLiteral) rexNode;
+            column = translateRexLiteral(literal);
+        } else if (rexNode instanceof RexCall) {
+            RexCall call = (RexCall) rexNode;
+            column = translateRexCall(call, inputColumnRowType, fieldName, sourceCollector);
+        } else {
+            throw new IllegalStateException("Unsupport RexNode " + rexNode);
+        }
+        return column;
+    }
+
+    private TblColRef translateRexInputRef(RexInputRef inputRef, ColumnRowType inputColumnRowType, String fieldName, Set<TblColRef> sourceCollector) {
+        int index = inputRef.getIndex();
+        // check it for rewrite count
+        if (index < inputColumnRowType.size()) {
+            TblColRef column = inputColumnRowType.getColumnByIndex(index);
+            if (!column.isInnerColumn() && !this.rewriting && !this.afterAggregate) {
+                context.allColumns.add(column);
+                sourceCollector.add(column);
+            }
+            return column;
+        } else {
+            throw new IllegalStateException("Can't find " + inputRef + " from child columnrowtype " + inputColumnRowType + " with fieldname " + fieldName);
+        }
+    }
+
+    private TblColRef translateRexLiteral(RexLiteral literal) {
+        if (RexLiteral.isNullLiteral(literal)) {
+            return TblColRef.newInnerColumn("null", InnerDataTypeEnum.LITERAL);
+        } else {
+            return TblColRef.newInnerColumn(literal.getValue().toString(), InnerDataTypeEnum.LITERAL);
+        }
+
+    }
+
+    private TblColRef translateRexCall(RexCall call, ColumnRowType inputColumnRowType, String fieldName, Set<TblColRef> sourceCollector) {
+        SqlOperator operator = call.getOperator();
+        if (operator == SqlStdOperatorTable.EXTRACT_DATE) {
+            List<RexNode> extractDateOps = call.getOperands();
+            RexCall reinterpret = (RexCall) extractDateOps.get(1);
+            List<RexNode> reinterpretOps = reinterpret.getOperands();
+            RexInputRef inputRef = (RexInputRef) reinterpretOps.get(0);
+            return translateRexInputRef(inputRef, inputColumnRowType, fieldName, sourceCollector);
+        } else if (operator instanceof SqlUserDefinedFunction) {
+            if (operator.getName().equals("QUARTER")) {
+                List<RexNode> quaterOps = call.getOperands();
+                RexInputRef inputRef = (RexInputRef) quaterOps.get(0);
+                return translateRexInputRef(inputRef, inputColumnRowType, fieldName, sourceCollector);
+            }
+        } else if (operator instanceof SqlCaseOperator) {
+            for (RexNode operand : call.getOperands()) {
+                if (operand instanceof RexInputRef) {
+                    RexInputRef inputRef = (RexInputRef) operand;
+                    return translateRexInputRef(inputRef, inputColumnRowType, fieldName, sourceCollector);
+                }
+            }
+        }
+
+        for (RexNode operand : call.getOperands()) {
+            translateRexNode(operand, inputColumnRowType, fieldName, sourceCollector);
+        }
+        return TblColRef.newInnerColumn(fieldName, InnerDataTypeEnum.LITERAL);
+    }
+
+    @Override
+    public Result implement(EnumerableRelImplementor implementor, Prefer pref) {
+        EnumerableCalcRel enumCalcRel;
+
+        RelNode child = getChild();
+        if (child instanceof OLAPFilterRel) {
+            // merge project & filter
+            OLAPFilterRel filter = (OLAPFilterRel) getChild();
+            RexProgram program = RexProgram.create(filter.getChild().getRowType(), this.rewriteProjects, filter.getCondition(), this.rowType, getCluster().getRexBuilder());
+
+            enumCalcRel = new EnumerableCalcRel(getCluster(), getCluster().traitSetOf(EnumerableConvention.INSTANCE), filter.getChild(), this.rowType, program, ImmutableList.<RelCollation> of());
+        } else {
+            // keep project for tablescan
+            RexProgram program = RexProgram.create(child.getRowType(), this.rewriteProjects, null, this.rowType, getCluster().getRexBuilder());
+
+            enumCalcRel = new EnumerableCalcRel(getCluster(), getCluster().traitSetOf(EnumerableConvention.INSTANCE), child, this.rowType, program, ImmutableList.<RelCollation> of());
+        }
+
+        return enumCalcRel.implement(implementor, pref);
+    }
+
+    @Override
+    public ColumnRowType getColumnRowType() {
+        return columnRowType;
+    }
+
+    @Override
+    public void implementRewrite(RewriteImplementor implementor) {
+        implementor.visitChild(this, getChild());
+
+        this.rewriting = true;
+
+        // project before join or is just after OLAPToEnumerableConverter
+        if (!RewriteImplementor.needRewrite(this.context) || (this.hasJoin && !this.afterJoin) || this.afterAggregate) {
+            this.columnRowType = this.buildColumnRowType();
+            return;
+        }
+
+        // find missed rewrite fields
+        int paramIndex = this.rowType.getFieldList().size();
+        List<RelDataTypeField> newFieldList = new LinkedList<RelDataTypeField>();
+        List<RexNode> newExpList = new LinkedList<RexNode>();
+        ColumnRowType inputColumnRowType = ((OLAPRel) getChild()).getColumnRowType();
+
+        for (Map.Entry<String, RelDataType> rewriteField : this.context.rewriteFields.entrySet()) {
+            String rewriteFieldName = rewriteField.getKey();
+            int rowIndex = this.columnRowType.getIndexByName(rewriteFieldName);
+            if (rowIndex < 0) {
+                int inputIndex = inputColumnRowType.getIndexByName(rewriteFieldName);
+                if (inputIndex >= 0) {
+                    // new field
+                    RelDataType fieldType = rewriteField.getValue();
+                    RelDataTypeField newField = new RelDataTypeFieldImpl(rewriteFieldName, paramIndex++, fieldType);
+                    newFieldList.add(newField);
+                    // new project
+                    RelDataTypeField inputField = getChild().getRowType().getFieldList().get(inputIndex);
+                    RexInputRef newFieldRef = new RexInputRef(inputField.getIndex(), inputField.getType());
+                    newExpList.add(newFieldRef);
+                }
+            }
+        }
+
+        if (!newFieldList.isEmpty()) {
+            // rebuild projects
+            List<RexNode> newProjects = new ArrayList<RexNode>(this.rewriteProjects);
+            newProjects.addAll(newExpList);
+            this.rewriteProjects = newProjects;
+
+            // rebuild row type
+            FieldInfoBuilder fieldInfo = getCluster().getTypeFactory().builder();
+            fieldInfo.addAll(this.rowType.getFieldList());
+            fieldInfo.addAll(newFieldList);
+            this.rowType = getCluster().getTypeFactory().createStructType(fieldInfo);
+        }
+
+        // rebuild columns
+        this.columnRowType = this.buildColumnRowType();
+
+        this.rewriting = false;
+    }
+
+    @Override
+    public OLAPContext getContext() {
+        return context;
+    }
+
+    @Override
+    public boolean hasSubQuery() {
+        OLAPRel olapChild = (OLAPRel) getChild();
+        return olapChild.hasSubQuery();
+    }
+
+    @Override
+    public RelTraitSet replaceTraitSet(RelTrait trait) {
+        RelTraitSet oldTraitSet = this.traitSet;
+        this.traitSet = this.traitSet.replace(trait);
+        return oldTraitSet;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/query/src/main/java/com/kylinolap/query/relnode/OLAPRel.java
----------------------------------------------------------------------
diff --git a/query/src/main/java/com/kylinolap/query/relnode/OLAPRel.java b/query/src/main/java/com/kylinolap/query/relnode/OLAPRel.java
new file mode 100644
index 0000000..c773db4
--- /dev/null
+++ b/query/src/main/java/com/kylinolap/query/relnode/OLAPRel.java
@@ -0,0 +1,156 @@
+/*
+ * 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.query.relnode;
+
+import java.util.Stack;
+
+import net.hydromatic.optiq.rules.java.EnumerableRel;
+import net.hydromatic.optiq.rules.java.EnumerableRelImplementor;
+
+import org.eigenbase.rel.RelNode;
+import org.eigenbase.relopt.Convention;
+import org.eigenbase.relopt.RelTrait;
+import org.eigenbase.relopt.RelTraitSet;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * 
+ * @author xjiang
+ * 
+ */
+public interface OLAPRel extends RelNode {
+
+    public static final Logger logger = LoggerFactory.getLogger(OLAPRel.class);
+
+    // Calling convention for relational operations that occur in OLAP.
+    public static final Convention CONVENTION = new Convention.Impl("OLAP", OLAPRel.class);
+
+    /**
+     * get olap context
+     */
+    public OLAPContext getContext();
+
+    /**
+     * get the row type of ColumnDesc
+     * 
+     * @return
+     */
+    public ColumnRowType getColumnRowType();
+
+    /**
+     * whether has sub query
+     */
+    public boolean hasSubQuery();
+
+    /**
+     * replace RelTraitSet
+     */
+    public RelTraitSet replaceTraitSet(RelTrait trait);
+
+    /**
+     * visitor pattern for olap query analysis
+     */
+    public static class OLAPImplementor {
+
+        private RelNode parentNode = null;
+        private int ctxSeq = 0;
+        private Stack<OLAPContext> ctxStack = new Stack<OLAPContext>();
+
+        public void visitChild(RelNode input, RelNode parentNode) {
+            this.parentNode = parentNode;
+            ((OLAPRel) input).implementOLAP(this);
+        }
+
+        public RelNode getParentNode() {
+            return parentNode;
+        }
+
+        public OLAPContext getContext() {
+            if (ctxStack.isEmpty()) {
+                return null;
+            }
+            return ctxStack.peek();
+        }
+
+        public void freeContext() {
+            ctxStack.pop();
+        }
+
+        public void allocateContext() {
+            OLAPContext context = new OLAPContext(ctxSeq++);
+            ctxStack.push(context);
+            OLAPContext.registerContext(context);
+        }
+    }
+
+    public void implementOLAP(OLAPImplementor implementor);
+
+    /**
+     * visitor pattern for query rewrite
+     */
+
+    public static class RewriteImplementor {
+        private OLAPContext parentContext;
+
+        public void visitChild(RelNode parent, RelNode child) {
+            if (parent instanceof OLAPRel) {
+                OLAPRel olapRel = (OLAPRel) parent;
+                this.parentContext = olapRel.getContext();
+            }
+            OLAPRel olapChild = (OLAPRel) child;
+            olapChild.implementRewrite(this);
+        }
+
+        public OLAPContext getParentContext() {
+            return parentContext;
+        }
+
+        public static boolean needRewrite(OLAPContext ctx) {
+            boolean hasFactTable = ctx.hasJoin || ctx.firstTableScan.getCubeTable().equals(ctx.cubeDesc.getFactTable());
+            boolean hasRewriteFields = !ctx.rewriteFields.isEmpty();
+            return hasRewriteFields && hasFactTable;
+        }
+    }
+
+    public void implementRewrite(RewriteImplementor rewriter);
+
+    /**
+     * implementor for java generation
+     */
+    public static class JavaImplementor extends EnumerableRelImplementor {
+
+        private OLAPContext parentContext;
+
+        public JavaImplementor(EnumerableRelImplementor enumImplementor) {
+            super(enumImplementor.getRexBuilder());
+        }
+
+        public OLAPContext getParentContext() {
+            return parentContext;
+        }
+
+        @Override
+        public EnumerableRel.Result visitChild(EnumerableRel parent, int ordinal, EnumerableRel child, EnumerableRel.Prefer prefer) {
+            if (parent instanceof OLAPRel) {
+                OLAPRel olapRel = (OLAPRel) parent;
+                this.parentContext = olapRel.getContext();
+            }
+            return super.visitChild(parent, ordinal, child, prefer);
+        }
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/query/src/main/java/com/kylinolap/query/relnode/OLAPSortRel.java
----------------------------------------------------------------------
diff --git a/query/src/main/java/com/kylinolap/query/relnode/OLAPSortRel.java b/query/src/main/java/com/kylinolap/query/relnode/OLAPSortRel.java
new file mode 100644
index 0000000..9d71eaf
--- /dev/null
+++ b/query/src/main/java/com/kylinolap/query/relnode/OLAPSortRel.java
@@ -0,0 +1,153 @@
+/*
+ * 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.query.relnode;
+
+import net.hydromatic.optiq.rules.java.EnumerableConvention;
+import net.hydromatic.optiq.rules.java.EnumerableRel;
+import net.hydromatic.optiq.rules.java.EnumerableRelImplementor;
+import net.hydromatic.optiq.rules.java.JavaRules.EnumerableSortRel;
+
+import org.eigenbase.rel.RelCollation;
+import org.eigenbase.rel.RelFieldCollation;
+import org.eigenbase.rel.RelNode;
+import org.eigenbase.rel.SortRel;
+import org.eigenbase.relopt.RelOptCluster;
+import org.eigenbase.relopt.RelOptCost;
+import org.eigenbase.relopt.RelOptPlanner;
+import org.eigenbase.relopt.RelTrait;
+import org.eigenbase.relopt.RelTraitSet;
+import org.eigenbase.rex.RexNode;
+
+import com.google.common.base.Preconditions;
+import com.kylinolap.metadata.model.cube.MeasureDesc;
+import com.kylinolap.metadata.model.cube.TblColRef;
+import com.kylinolap.storage.StorageContext;
+
+/**
+ * @author xjiang
+ * 
+ */
+public class OLAPSortRel extends SortRel implements EnumerableRel, OLAPRel {
+
+    private ColumnRowType columnRowType;
+    private OLAPContext context;
+
+    public OLAPSortRel(RelOptCluster cluster, RelTraitSet traitSet, RelNode child, RelCollation collation, RexNode offset, RexNode fetch) {
+        super(cluster, traitSet, child, collation, offset, fetch);
+        Preconditions.checkArgument(getConvention() == OLAPRel.CONVENTION);
+        Preconditions.checkArgument(getConvention() == child.getConvention());
+    }
+
+    @Override
+    public OLAPSortRel copy(RelTraitSet traitSet, RelNode newInput, RelCollation newCollation, RexNode offset, RexNode fetch) {
+        return new OLAPSortRel(getCluster(), traitSet, newInput, newCollation, offset, fetch);
+    }
+
+    @Override
+    public RelOptCost computeSelfCost(RelOptPlanner planner) {
+        return super.computeSelfCost(planner).multiplyBy(.05);
+    }
+
+    @Override
+    public void implementOLAP(OLAPImplementor implementor) {
+        implementor.visitChild(getChild(), this);
+
+        this.context = implementor.getContext();
+        this.columnRowType = buildColumnRowType();
+    }
+
+    private ColumnRowType buildColumnRowType() {
+        OLAPRel olapChild = (OLAPRel) getChild();
+        ColumnRowType inputColumnRowType = olapChild.getColumnRowType();
+        return inputColumnRowType;
+    }
+
+    @Override
+    public void implementRewrite(RewriteImplementor implementor) {
+        implementor.visitChild(this, getChild());
+
+        for (RelFieldCollation fieldCollation : this.collation.getFieldCollations()) {
+            int index = fieldCollation.getFieldIndex();
+            StorageContext.OrderEnum order = getOrderEnum(fieldCollation.getDirection());
+            OLAPRel olapChild = (OLAPRel) this.getChild();
+            TblColRef orderCol = olapChild.getColumnRowType().getAllColumns().get(index);
+            MeasureDesc measure = findMeasure(orderCol);
+            if (measure != null) {
+                this.context.storageContext.addSort(measure, order);
+            }
+            this.context.storageContext.markSort();
+        }
+
+        this.rowType = this.deriveRowType();
+        this.columnRowType = buildColumnRowType();
+    }
+
+    private StorageContext.OrderEnum getOrderEnum(RelFieldCollation.Direction direction) {
+        if (direction == RelFieldCollation.Direction.DESCENDING) {
+            return StorageContext.OrderEnum.DESCENDING;
+        } else {
+            return StorageContext.OrderEnum.ASCENDING;
+        }
+    }
+
+    private MeasureDesc findMeasure(TblColRef col) {
+        for (MeasureDesc measure : this.context.cubeDesc.getMeasures()) {
+            if (col.getName().equals(measure.getFunction().getRewriteFieldName())) {
+                return measure;
+            }
+        }
+        return null;
+    }
+
+    @Override
+    public Result implement(EnumerableRelImplementor implementor, Prefer pref) {
+        OLAPRel childRel = (OLAPRel) getChild();
+        childRel.replaceTraitSet(EnumerableConvention.INSTANCE);
+
+        EnumerableSortRel enumSort = new EnumerableSortRel(getCluster(), getCluster().traitSetOf(EnumerableConvention.INSTANCE, collation), getChild(), collation, offset, fetch);
+
+        Result res = enumSort.implement(implementor, pref);
+
+        childRel.replaceTraitSet(OLAPRel.CONVENTION);
+
+        return res;
+    }
+
+    @Override
+    public OLAPContext getContext() {
+        return context;
+    }
+
+    @Override
+    public ColumnRowType getColumnRowType() {
+        return columnRowType;
+    }
+
+    @Override
+    public boolean hasSubQuery() {
+        OLAPRel olapChild = (OLAPRel) getChild();
+        return olapChild.hasSubQuery();
+    }
+
+    @Override
+    public RelTraitSet replaceTraitSet(RelTrait trait) {
+        RelTraitSet oldTraitSet = this.traitSet;
+        this.traitSet = this.traitSet.replace(trait);
+        return oldTraitSet;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/query/src/main/java/com/kylinolap/query/relnode/OLAPTableScan.java
----------------------------------------------------------------------
diff --git a/query/src/main/java/com/kylinolap/query/relnode/OLAPTableScan.java b/query/src/main/java/com/kylinolap/query/relnode/OLAPTableScan.java
new file mode 100644
index 0000000..9c50c36
--- /dev/null
+++ b/query/src/main/java/com/kylinolap/query/relnode/OLAPTableScan.java
@@ -0,0 +1,246 @@
+/*
+ * 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.query.relnode;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+
+import net.hydromatic.linq4j.expressions.Blocks;
+import net.hydromatic.linq4j.expressions.Expressions;
+import net.hydromatic.linq4j.expressions.Primitive;
+import net.hydromatic.optiq.rules.java.EnumerableRel;
+import net.hydromatic.optiq.rules.java.EnumerableRelImplementor;
+import net.hydromatic.optiq.rules.java.PhysType;
+import net.hydromatic.optiq.rules.java.PhysTypeImpl;
+
+import org.eigenbase.rel.RelNode;
+import org.eigenbase.rel.RelWriter;
+import org.eigenbase.rel.TableAccessRelBase;
+import org.eigenbase.rel.rules.*;
+import org.eigenbase.relopt.RelOptCluster;
+import org.eigenbase.relopt.RelOptCost;
+import org.eigenbase.relopt.RelOptPlanner;
+import org.eigenbase.relopt.RelOptTable;
+import org.eigenbase.relopt.RelTrait;
+import org.eigenbase.relopt.RelTraitSet;
+import org.eigenbase.reltype.RelDataType;
+import org.eigenbase.reltype.RelDataTypeFactory;
+import org.eigenbase.reltype.RelDataTypeField;
+
+import com.google.common.base.Preconditions;
+import com.kylinolap.metadata.model.cube.TblColRef;
+import com.kylinolap.metadata.model.schema.ColumnDesc;
+import com.kylinolap.query.optrule.OLAPAggregateRule;
+import com.kylinolap.query.optrule.OLAPFilterRule;
+import com.kylinolap.query.optrule.OLAPJoinRule;
+import com.kylinolap.query.optrule.OLAPLimitRule;
+import com.kylinolap.query.optrule.OLAPProjectRule;
+import com.kylinolap.query.optrule.OLAPSortRule;
+import com.kylinolap.query.optrule.OLAPToEnumerableConverterRule;
+import com.kylinolap.query.schema.OLAPSchema;
+import com.kylinolap.query.schema.OLAPTable;
+
+/**
+ * @author xjiang
+ */
+public class OLAPTableScan extends TableAccessRelBase implements OLAPRel, EnumerableRel {
+
+    private final OLAPTable olapTable;
+    private final String cubeTable;
+    private final int[] fields;
+    private ColumnRowType columnRowType;
+    private OLAPContext context;
+
+    public OLAPTableScan(RelOptCluster cluster, RelOptTable table, OLAPTable olapTable, int[] fields) {
+        super(cluster, cluster.traitSetOf(OLAPRel.CONVENTION), table);
+        this.olapTable = olapTable;
+        this.fields = fields;
+        this.cubeTable = olapTable.getTableName();
+        this.rowType = getRowType();
+    }
+
+    public OLAPTable getOlapTable() {
+        return olapTable;
+    }
+
+    public String getCubeTable() {
+        return cubeTable;
+    }
+
+    public int[] getFields() {
+        return fields;
+    }
+
+    @Override
+    public OLAPContext getContext() {
+        return context;
+    }
+
+    @Override
+    public RelNode copy(RelTraitSet traitSet, List<RelNode> inputs) {
+        Preconditions.checkArgument(inputs.isEmpty());
+        return new OLAPTableScan(getCluster(), table, olapTable, fields);
+    }
+
+    @Override
+    public void register(RelOptPlanner planner) {
+        // force clear the query context before traversal relational operators
+        OLAPContext.clearThreadLocalContexts();
+
+        // register OLAP rules
+        planner.addRule(OLAPToEnumerableConverterRule.INSTANCE);
+        planner.addRule(OLAPFilterRule.INSTANCE);
+        planner.addRule(OLAPProjectRule.INSTANCE);
+        planner.addRule(OLAPAggregateRule.INSTANCE);
+        planner.addRule(OLAPJoinRule.INSTANCE);
+        planner.addRule(OLAPLimitRule.INSTANCE);
+        planner.addRule(OLAPSortRule.INSTANCE);
+
+        // since join is the entry point, we can't push filter past join
+        planner.removeRule(PushFilterPastJoinRule.FILTER_ON_JOIN);
+        planner.removeRule(PushFilterPastJoinRule.JOIN);
+
+        // TODO : since we don't have statistic of table, the optimization of join is too cost
+        planner.removeRule(SwapJoinRule.INSTANCE);
+        planner.removeRule(PushJoinThroughJoinRule.LEFT);
+        planner.removeRule(PushJoinThroughJoinRule.RIGHT);
+
+        // for columns in having clause will enable table scan filter rule
+        // cause kylin does not depend on MPP
+        planner.removeRule(PushFilterPastProjectRule.INSTANCE);
+        // distinct count will be split into a separated query that is joined with the left query
+        planner.removeRule(RemoveDistinctAggregateRule.INSTANCE);
+    }
+
+    @Override
+    public RelDataType deriveRowType() {
+        final List<RelDataTypeField> fieldList = table.getRowType().getFieldList();
+        final RelDataTypeFactory.FieldInfoBuilder builder = getCluster().getTypeFactory().builder();
+        for (int field : fields) {
+            builder.add(fieldList.get(field));
+        }
+        return getCluster().getTypeFactory().createStructType(builder);
+    }
+
+    @Override
+    public RelOptCost computeSelfCost(RelOptPlanner planner) {
+        return super.computeSelfCost(planner).multiplyBy(.05);
+    }
+
+    @Override
+    public RelWriter explainTerms(RelWriter pw) {
+        return super.explainTerms(pw).item("fields", Primitive.asList(fields));
+    }
+
+    @Override
+    public void implementOLAP(OLAPImplementor implementor) {
+        // create context in case of non-join
+        if (implementor.getContext() == null || !(implementor.getParentNode() instanceof OLAPJoinRel)) {
+            implementor.allocateContext();
+        }
+        columnRowType = buildColumnRowType();
+        context = implementor.getContext();
+
+        if (context.olapSchema == null) {
+            OLAPSchema schema = olapTable.getSchema();
+            context.olapSchema = schema;
+            context.storageContext.setConnUrl(schema.getStorageUrl());
+        }
+
+        if (context.firstTableScan == null) {
+            context.firstTableScan = this;
+        }
+
+        context.olapRowType = rowType;
+    }
+
+    private ColumnRowType buildColumnRowType() {
+        List<TblColRef> columns = new ArrayList<TblColRef>();
+        for (ColumnDesc sourceColumn : olapTable.getExposedColumns()) {
+            TblColRef colRef = new TblColRef(sourceColumn);
+            columns.add(colRef);
+        }
+        return new ColumnRowType(columns);
+    }
+
+    @Override
+    public Result implement(EnumerableRelImplementor implementor, Prefer pref) {
+        if (!(implementor instanceof JavaImplementor))
+            throw new IllegalStateException("implementor is not JavaImplementor");
+        JavaImplementor javaImplementor = (JavaImplementor) implementor;
+
+        int ctxId = this.context.id;
+        if (javaImplementor.getParentContext() != null) {
+            ctxId = javaImplementor.getParentContext().id;
+        }
+
+        PhysType physType = PhysTypeImpl.of(javaImplementor.getTypeFactory(), this.rowType, pref.preferArray());
+
+        String execFunction = genExecFunc();
+
+        return javaImplementor.result(physType, Blocks.toBlock(Expressions.call(table.getExpression(OLAPTable.class), execFunction, javaImplementor.getRootExpression(), Expressions.constant(ctxId))));
+    }
+
+    private String genExecFunc() {
+        // if the table to scan is not the fact table of cube, then it's a
+        // lookup table
+        if (context.hasJoin == false && cubeTable.equals(context.cubeDesc.getFactTable()) == false) {
+            return "executeLookupTableQuery";
+        } else {
+            return "executeCubeQuery";
+        }
+
+    }
+
+    @Override
+    public ColumnRowType getColumnRowType() {
+        return columnRowType;
+    }
+
+    /**
+     * Because OLAPTableScan is reused for the same table, we can't use
+     * this.context and have to use parent context
+     */
+    @Override
+    public void implementRewrite(RewriteImplementor implementor) {
+        Map<String, RelDataType> rewriteFields = this.context.rewriteFields;
+        if (implementor.getParentContext() != null) {
+            rewriteFields = implementor.getParentContext().rewriteFields;
+        }
+
+        for (Map.Entry<String, RelDataType> rewriteField : rewriteFields.entrySet()) {
+            String fieldName = rewriteField.getKey();
+            RelDataTypeField field = rowType.getField(fieldName, true);
+            if (field != null) {
+                RelDataType fieldType = field.getType();
+                rewriteField.setValue(fieldType);
+            }
+        }
+    }
+
+    @Override
+    public boolean hasSubQuery() {
+        return false;
+    }
+
+    @Override
+    public RelTraitSet replaceTraitSet(RelTrait trait) {
+        RelTraitSet oldTraitSet = this.traitSet;
+        this.traitSet = this.traitSet.replace(trait);
+        return oldTraitSet;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/query/src/main/java/com/kylinolap/query/relnode/OLAPToEnumerableConverter.java
----------------------------------------------------------------------
diff --git a/query/src/main/java/com/kylinolap/query/relnode/OLAPToEnumerableConverter.java b/query/src/main/java/com/kylinolap/query/relnode/OLAPToEnumerableConverter.java
new file mode 100644
index 0000000..b38282e
--- /dev/null
+++ b/query/src/main/java/com/kylinolap/query/relnode/OLAPToEnumerableConverter.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.query.relnode;
+
+import java.util.List;
+
+import net.hydromatic.linq4j.expressions.Blocks;
+import net.hydromatic.linq4j.expressions.Expressions;
+import net.hydromatic.optiq.rules.java.EnumerableRel;
+import net.hydromatic.optiq.rules.java.EnumerableRelImplementor;
+import net.hydromatic.optiq.rules.java.PhysType;
+import net.hydromatic.optiq.rules.java.PhysTypeImpl;
+
+import org.eigenbase.rel.RelNode;
+import org.eigenbase.rel.convert.ConverterRelImpl;
+import org.eigenbase.relopt.ConventionTraitDef;
+import org.eigenbase.relopt.RelOptCluster;
+import org.eigenbase.relopt.RelOptCost;
+import org.eigenbase.relopt.RelOptPlanner;
+import org.eigenbase.relopt.RelOptTable;
+import org.eigenbase.relopt.RelTraitSet;
+import org.eigenbase.reltype.RelDataType;
+
+import com.kylinolap.cube.CubeInstance;
+import com.kylinolap.query.relnode.OLAPRel.JavaImplementor;
+import com.kylinolap.query.relnode.OLAPRel.OLAPImplementor;
+import com.kylinolap.query.relnode.OLAPRel.RewriteImplementor;
+import com.kylinolap.query.routing.CubeNotFoundException;
+import com.kylinolap.query.routing.QueryRouter;
+import com.kylinolap.query.schema.OLAPTable;
+
+/**
+ * @author xjiang
+ */
+public class OLAPToEnumerableConverter extends ConverterRelImpl implements EnumerableRel {
+
+    public OLAPToEnumerableConverter(RelOptCluster cluster, RelTraitSet traits, RelNode input) {
+        super(cluster, ConventionTraitDef.INSTANCE, traits, input);
+    }
+
+    @Override
+    public RelNode copy(RelTraitSet traitSet, List<RelNode> inputs) {
+        return new OLAPToEnumerableConverter(getCluster(), traitSet, sole(inputs));
+    }
+
+    @Override
+    public RelOptCost computeSelfCost(RelOptPlanner planner) {
+        return super.computeSelfCost(planner).multiplyBy(.05);
+    }
+
+    @Override
+    public Result implement(EnumerableRelImplementor enumImplementor, Prefer pref) {
+        // post-order travel children
+        OLAPImplementor olapImplementor = new OLAPRel.OLAPImplementor();
+        olapImplementor.visitChild(getChild(), this);
+
+        // find cube from olap context
+        try {
+            for (OLAPContext context : OLAPContext.getThreadLocalContexts()) {
+                CubeInstance cube = QueryRouter.findCube(context);
+                context.cubeInstance = cube;
+                context.cubeDesc = cube.getDescriptor();
+            }
+        } catch (CubeNotFoundException e) {
+            OLAPContext ctx0 = (OLAPContext) OLAPContext.getThreadLocalContexts().toArray()[0];
+            if (ctx0 != null && ctx0.olapSchema.hasStarSchemaUrl()) {
+                // generate hive result
+                return buildHiveResult(enumImplementor, pref, ctx0);
+            } else {
+                throw e;
+            }
+        }
+
+        // rewrite query if necessary
+        RewriteImplementor rewriteImplementor = new RewriteImplementor();
+        rewriteImplementor.visitChild(this, getChild());
+
+        // build java implementation
+        EnumerableRel child = (EnumerableRel) getChild();
+        JavaImplementor javaImplementor = new JavaImplementor(enumImplementor);
+        return javaImplementor.visitChild(this, 0, child, pref);
+
+    }
+
+    private Result buildHiveResult(EnumerableRelImplementor enumImplementor, Prefer pref, OLAPContext context) {
+        RelDataType hiveRowType = getRowType();
+
+        context.olapRowType = hiveRowType;
+        PhysType physType = PhysTypeImpl.of(enumImplementor.getTypeFactory(), hiveRowType, pref.preferArray());
+
+        RelOptTable factTable = context.firstTableScan.getTable();
+        Result result = enumImplementor.result(physType, Blocks.toBlock(Expressions.call(factTable.getExpression(OLAPTable.class), "executeHiveQuery", enumImplementor.getRootExpression())));
+        return result;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/query/src/main/java/com/kylinolap/query/routing/CubeNotFoundException.java
----------------------------------------------------------------------
diff --git a/query/src/main/java/com/kylinolap/query/routing/CubeNotFoundException.java b/query/src/main/java/com/kylinolap/query/routing/CubeNotFoundException.java
new file mode 100644
index 0000000..b8b09ff
--- /dev/null
+++ b/query/src/main/java/com/kylinolap/query/routing/CubeNotFoundException.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.query.routing;
+
+/**
+ * @author xjiang
+ * 
+ */
+public class CubeNotFoundException extends RuntimeException {
+
+    private static final long serialVersionUID = 4372584597304243555L;
+
+    public CubeNotFoundException(String message, Throwable t) {
+        super(message, t);
+    }
+
+    public CubeNotFoundException(String message) {
+        super(message);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/query/src/main/java/com/kylinolap/query/routing/QueryRouter.java
----------------------------------------------------------------------
diff --git a/query/src/main/java/com/kylinolap/query/routing/QueryRouter.java b/query/src/main/java/com/kylinolap/query/routing/QueryRouter.java
new file mode 100644
index 0000000..7745c2a
--- /dev/null
+++ b/query/src/main/java/com/kylinolap/query/routing/QueryRouter.java
@@ -0,0 +1,311 @@
+/*
+ * 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.query.routing;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.commons.lang3.StringUtils;
+import org.eigenbase.reltype.RelDataType;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.kylinolap.cube.CubeInstance;
+import com.kylinolap.cube.CubeManager;
+import com.kylinolap.cube.project.ProjectManager;
+import com.kylinolap.metadata.model.cube.CubeDesc;
+import com.kylinolap.metadata.model.cube.DimensionDesc;
+import com.kylinolap.metadata.model.cube.FunctionDesc;
+import com.kylinolap.metadata.model.cube.JoinDesc;
+import com.kylinolap.metadata.model.cube.ParameterDesc;
+import com.kylinolap.metadata.model.cube.TblColRef;
+import com.kylinolap.query.relnode.OLAPContext;
+
+/**
+ * @author xjiang
+ */
+public class QueryRouter {
+
+    private static final Logger logger = LoggerFactory.getLogger(QueryRouter.class);
+
+    public static CubeInstance findCube(OLAPContext olapContext) throws CubeNotFoundException {
+
+        CubeInstance bestCube = null;
+        // NOTE: since some query has no groups and projections are the superset of groups, we choose projections.
+        ProjectManager projectManager = ProjectManager.getInstance(olapContext.olapSchema.getConfig());
+
+        if (olapContext.isSimpleQuery()) {
+            // if simple query like "select X from fact table", just return the cube with most dimensions
+            // Note that this will only succeed to get best cube if the current simple query is on fact table.
+            // Simple query on look up table is handled in OLAPTableScan.genExecFunc
+            // In other words, for simple query on lookup tables, bestCube here will be assigned null in this method
+            bestCube = findCubeWithMostDimensions(projectManager, olapContext);
+        }
+
+        if (bestCube == null) {
+            bestCube = findBestMatchCube(projectManager, olapContext);
+        }
+
+        if (bestCube == null) {
+            throw new CubeNotFoundException("Can't find cube for fact table " + olapContext.firstTableScan.getCubeTable() //
+                    + " in project " + olapContext.olapSchema.getProjectName() + " with dimensions " //
+                    + getDimensionColumns(olapContext) + " and measures " + olapContext.aggregations //
+                    + ". Also please check whether join types match what defined in Cube.");
+        }
+
+        return bestCube;
+    }
+
+    private static CubeInstance findCubeWithMostDimensions(ProjectManager projectManager, OLAPContext olapContext) {
+        List<CubeInstance> candidates = projectManager.getOnlineCubesByFactTable(olapContext.olapSchema.getProjectName(), olapContext.firstTableScan.getCubeTable());
+        if (candidates.isEmpty()) {
+            return null;
+        }
+
+        CubeInstance cubeWithMostColumns = candidates.get(0);
+        for (CubeInstance instance : candidates) {
+            int currentDimCount = instance.getDescriptor().listDimensionColumnsIncludingDerived().size();
+            int maxDimCount = cubeWithMostColumns.getDescriptor().listDimensionColumnsIncludingDerived().size();
+
+            if ((currentDimCount > maxDimCount) || ((currentDimCount == maxDimCount) && (instance.getCost() < cubeWithMostColumns.getCost())))
+                cubeWithMostColumns = instance;
+        }
+        return cubeWithMostColumns;
+    }
+
+    private static void sortByCost(List<CubeInstance> matchCubes) {
+        // sort cube candidates, 0) the cost indicator, 1) the lesser header
+        // columns the better, 2) the lesser body columns the better
+        Collections.sort(matchCubes, new Comparator<CubeInstance>() {
+            @Override
+            public int compare(CubeInstance c1, CubeInstance c2) {
+                int comp = 0;
+                comp = c1.getCost() - c2.getCost();
+                if (comp != 0) {
+                    return comp;
+                }
+
+                CubeDesc schema1 = c1.getDescriptor();
+                CubeDesc schema2 = c2.getDescriptor();
+
+                comp = schema1.listDimensionColumnsIncludingDerived().size() - schema2.listDimensionColumnsIncludingDerived().size();
+                if (comp != 0)
+                    return comp;
+
+                comp = schema1.getMeasures().size() - schema2.getMeasures().size();
+                return comp;
+            }
+        });
+    }
+
+    private static Collection<TblColRef> getDimensionColumns(OLAPContext olapContext) {
+        Collection<TblColRef> dimensionColumns = new HashSet<TblColRef>();
+        dimensionColumns.addAll(olapContext.allColumns);
+        for (TblColRef measureColumn : olapContext.metricsColumns) {
+            dimensionColumns.remove(measureColumn);
+        }
+        return dimensionColumns;
+    }
+
+    static List<CubeInstance> findMatchCubesForTableScanQuery(CubeManager cubeMgr, String factTableName, Collection<TblColRef> dimensionColumns, Collection<FunctionDesc> functions) throws CubeNotFoundException {
+        return null;
+    }
+
+    static CubeInstance findBestMatchCube(ProjectManager projectManager, OLAPContext olapContext) throws CubeNotFoundException {
+
+        // retrieve members from olapContext
+        String factTableName = olapContext.firstTableScan.getCubeTable();
+        String projectName = olapContext.olapSchema.getProjectName();
+        Collection<TblColRef> dimensionColumns = getDimensionColumns(olapContext);
+        Collection<FunctionDesc> functions = olapContext.aggregations;
+        Collection<TblColRef> metricsColumns = olapContext.metricsColumns;
+        Collection<JoinDesc> joins = olapContext.joins;
+        Map<String, RelDataType> rewriteFields = olapContext.rewriteFields;
+
+        // find cubes by table
+        List<CubeInstance> candidates = projectManager.getCubesByTable(projectName, factTableName);
+        logger.info("Find candidates by table " + factTableName + " and project=" + projectName + " : " + StringUtils.join(candidates, ","));
+
+        // match dimensions & aggregations & joins
+        Iterator<CubeInstance> it = candidates.iterator();
+        List<CubeInstance> backups = new ArrayList<CubeInstance>();
+
+        while (it.hasNext()) {
+            CubeInstance cube = it.next();
+            boolean isOnline = cube.isReady();
+
+            boolean matchDimensions = isMatchedWithDimensions(dimensionColumns, cube);
+            boolean matchAggregation = isMatchedWithAggregations(functions, cube);
+            boolean matchJoin = isMatchedWithJoins(joins, cube);
+
+            // Some cubes are not "perfectly" match, but still save them in case of usage
+            if (isOnline && matchDimensions && !matchAggregation && matchJoin) {
+                // sometimes metrics are indeed dimensions
+                // e.g. select min(cal_dt) from ..., where cal_dt is actually a dimension
+                if (isWeaklyMatchedWithAggregations(functions, metricsColumns, cube)) {
+                    logger.info("Weak matched cube " + cube);
+                    backups.add(cube);
+                }
+            }
+
+            if (!isOnline || !matchDimensions || !matchAggregation || !matchJoin) {
+                logger.info("Remove cube " + cube.getName() + " because " + " isOnlne=" + isOnline + ",matchDimensions=" + matchDimensions + ",matchAggregation=" + matchAggregation + ",matchJoin=" + matchJoin);
+                it.remove();
+            }
+        }
+
+        // normal case:
+        if (!candidates.isEmpty()) {
+            return getCheapestCube(candidates);
+        }
+        // consider backup
+        else if (!backups.isEmpty()) {
+            CubeInstance cube = getCheapestCube(backups);
+            // Using backup cubes indicates that previous judgment on dimensions/metrics is incorrect
+            adjustOLAPContext(dimensionColumns, functions, metricsColumns, cube, rewriteFields, olapContext);
+            logger.info("Use weak matched cube " + cube.getName());
+            return cube;
+        }
+        return null;
+    }
+
+    private static CubeInstance getCheapestCube(List<CubeInstance> candidates) {
+        sortByCost(candidates);
+        CubeInstance bestCube = null;
+        if (!candidates.isEmpty()) {
+            bestCube = candidates.iterator().next();
+        }
+        return bestCube;
+    }
+
+    private static boolean isMatchedWithDimensions(Collection<TblColRef> dimensionColumns, CubeInstance cube) {
+        CubeDesc cubeDesc = cube.getDescriptor();
+        boolean matchAgg = cubeDesc.listDimensionColumnsIncludingDerived().containsAll(dimensionColumns);
+        return matchAgg;
+    }
+
+    private static boolean isMatchedWithAggregations(Collection<FunctionDesc> aggregations, CubeInstance cube) {
+        CubeDesc cubeDesc = cube.getDescriptor();
+        boolean matchAgg = cubeDesc.listAllFunctions().containsAll(aggregations);
+        return matchAgg;
+    }
+
+    private static boolean isMatchedWithJoins(Collection<JoinDesc> joins, CubeInstance cube) throws CubeNotFoundException {
+        CubeDesc cubeDesc = cube.getDescriptor();
+
+        List<JoinDesc> cubeJoins = new ArrayList<JoinDesc>(cubeDesc.getDimensions().size());
+        for (DimensionDesc d : cubeDesc.getDimensions()) {
+            if (d.getJoin() != null) {
+                cubeJoins.add(d.getJoin());
+            }
+        }
+        for (JoinDesc j : joins) {
+            // optiq engine can't decide which one is fk or pk
+            String pTable = j.getPrimaryKeyColumns()[0].getTable();
+            String factTable = cubeDesc.getFactTable();
+            if (factTable.equals(pTable)) {
+                j.swapPKFK();
+            }
+
+            // check primary key, all PK column should refer to same tale, the Fact Table of cube.
+            // Using first column's table name to check.
+            String fTable = j.getForeignKeyColumns()[0].getTable();
+            if (!factTable.equals(fTable)) {
+                logger.info("Fact Table" + factTable + " not matched in join: " + j + " on cube " + cube.getName());
+                return false;
+            }
+
+            // The hashcode() function of JoinDesc has been overwritten,
+            // which takes into consideration: pk,fk,jointype
+            if (!cubeJoins.contains(j)) {
+                logger.info("Query joins don't macth on cube " + cube.getName());
+                return false;
+            }
+        }
+        return true;
+    }
+
+    private static boolean isWeaklyMatchedWithAggregations(Collection<FunctionDesc> aggregations, Collection<TblColRef> metricColumns, CubeInstance cube) {
+        CubeDesc cubeDesc = cube.getDescriptor();
+        Collection<FunctionDesc> cubeFuncs = cubeDesc.listAllFunctions();
+
+        boolean matched = true;
+        for (FunctionDesc functionDesc : aggregations) {
+            if (cubeFuncs.contains(functionDesc))
+                continue;
+
+            // only inverted-index cube does not have count, and let calcite handle in this case
+            if (functionDesc.isCount())
+                continue;
+
+            if (functionDesc.isCountDistinct()) // calcite can not handle distinct count
+                matched = false;
+
+            TblColRef col = findTblColByMetrics(metricColumns, functionDesc);
+            if (col == null || !cubeDesc.listDimensionColumnsIncludingDerived().contains(col)) {
+                matched = false;
+            }
+        }
+        return matched;
+    }
+
+    private static void adjustOLAPContext(Collection<TblColRef> dimensionColumns, Collection<FunctionDesc> aggregations, //
+            Collection<TblColRef> metricColumns, CubeInstance cube, Map<String, RelDataType> rewriteFields, OLAPContext olapContext) {
+        CubeDesc cubeDesc = cube.getDescriptor();
+        Collection<FunctionDesc> cubeFuncs = cubeDesc.listAllFunctions();
+
+        Iterator<FunctionDesc> it = aggregations.iterator();
+        while (it.hasNext()) {
+            FunctionDesc functionDesc = it.next();
+            if (!cubeFuncs.contains(functionDesc)) {
+                // try to convert the metric to dimension to see if it works
+                TblColRef col = findTblColByMetrics(metricColumns, functionDesc);
+                functionDesc.setAppliedOnDimension(true);
+                rewriteFields.remove(functionDesc.getRewriteFieldName());
+                if (col != null) {
+                    metricColumns.remove(col);
+                    dimensionColumns.add(col);
+                    olapContext.storageContext.addOtherMandatoryColumns(col);
+                }
+                logger.info("Adjust OLAPContext for " + functionDesc);
+            }
+        }
+    }
+
+    private static TblColRef findTblColByMetrics(Collection<TblColRef> dimensionColumns, FunctionDesc func) {
+        if (func.isCount())
+            return null; // count is not about any column but the whole row
+
+        ParameterDesc parameter = func.getParameter();
+        if (parameter == null)
+            return null;
+
+        String columnName = parameter.getValue();
+        for (TblColRef col : dimensionColumns) {
+            String name = col.getName();
+            if (name != null && name.equals(columnName))
+                return col;
+        }
+        return null;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/query/src/main/java/com/kylinolap/query/schema/OLAPSchema.java
----------------------------------------------------------------------
diff --git a/query/src/main/java/com/kylinolap/query/schema/OLAPSchema.java b/query/src/main/java/com/kylinolap/query/schema/OLAPSchema.java
new file mode 100644
index 0000000..0bb7689
--- /dev/null
+++ b/query/src/main/java/com/kylinolap/query/schema/OLAPSchema.java
@@ -0,0 +1,123 @@
+/*
+ * 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.query.schema;
+
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import net.hydromatic.optiq.Table;
+import net.hydromatic.optiq.impl.AbstractSchema;
+
+import com.kylinolap.common.KylinConfig;
+import com.kylinolap.cube.CubeManager;
+import com.kylinolap.cube.project.ProjectInstance;
+import com.kylinolap.cube.project.ProjectManager;
+import com.kylinolap.metadata.MetadataManager;
+import com.kylinolap.metadata.model.schema.TableDesc;
+
+/**
+ * @author xjiang
+ */
+public class OLAPSchema extends AbstractSchema {
+
+//    private static final Logger logger = LoggerFactory.getLogger(OLAPSchema.class);
+
+    private KylinConfig config;
+    private String projectName;
+    private String schemaName;
+    private String storageUrl;
+    private String starSchemaUrl;
+    private String starSchemaUser;
+    private String starSchemaPassword;
+
+    private void init() {
+        this.config = KylinConfig.getInstanceFromEnv();
+        this.storageUrl = config.getStorageUrl();
+        this.starSchemaUrl = config.getHiveUrl();
+        this.starSchemaUser = config.getHiveUser();
+        this.starSchemaPassword = config.getHivePassword();
+    }
+
+    public OLAPSchema(String project, String schemaName) {
+        this.projectName = ProjectInstance.getNormalizedProjectName(project);
+        this.schemaName = schemaName;
+        init();
+    }
+
+    @Override
+    protected Map<String, Table> getTableMap() {
+        return buildTableMap();
+    }
+
+    private Map<String, Table> buildTableMap() {
+        Map<String, Table> olapTables = new HashMap<String, Table>();
+        List<TableDesc> projectTables = getProjectManager().listExposedTables(projectName);
+
+        for (TableDesc tableDesc : projectTables) {
+            final String tableName = tableDesc.getName();
+            final OLAPTable table = new OLAPTable(this, tableDesc);
+            olapTables.put(tableName, table);
+//            logger.debug("Project " + projectName + " exposes table " + tableName);
+        }
+
+        return olapTables;
+    }
+
+    public String getSchemaName() {
+        return schemaName;
+    }
+
+    public String getStorageUrl() {
+        return storageUrl;
+    }
+
+    public boolean hasStarSchemaUrl() {
+        return starSchemaUrl != null && !starSchemaUrl.isEmpty();
+    }
+
+    public String getStarSchemaUrl() {
+        return starSchemaUrl;
+    }
+
+    public String getStarSchemaUser() {
+        return starSchemaUser;
+    }
+
+    public String getStarSchemaPassword() {
+        return starSchemaPassword;
+    }
+
+    public MetadataManager getMetadataManager() {
+        return MetadataManager.getInstance(config);
+    }
+
+    public KylinConfig getConfig() {
+        return config;
+    }
+
+    public String getProjectName() {
+        return this.projectName;
+    }
+
+    public CubeManager getCubeManager() {
+        return CubeManager.getInstance(config);
+    }
+
+    public ProjectManager getProjectManager() {
+        return ProjectManager.getInstance(config);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/query/src/main/java/com/kylinolap/query/schema/OLAPSchemaFactory.java
----------------------------------------------------------------------
diff --git a/query/src/main/java/com/kylinolap/query/schema/OLAPSchemaFactory.java b/query/src/main/java/com/kylinolap/query/schema/OLAPSchemaFactory.java
new file mode 100644
index 0000000..f3dcb8e
--- /dev/null
+++ b/query/src/main/java/com/kylinolap/query/schema/OLAPSchemaFactory.java
@@ -0,0 +1,127 @@
+/*
+ * 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.query.schema;
+
+import java.io.File;
+import java.io.FileWriter;
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import net.hydromatic.optiq.Schema;
+import net.hydromatic.optiq.SchemaFactory;
+import net.hydromatic.optiq.SchemaPlus;
+
+import org.eigenbase.util14.ConversionUtil;
+
+import com.kylinolap.common.KylinConfig;
+import com.kylinolap.cube.project.ProjectInstance;
+import com.kylinolap.cube.project.ProjectManager;
+import com.kylinolap.metadata.model.schema.DatabaseDesc;
+import com.kylinolap.metadata.model.schema.TableDesc;
+
+/**
+ * @author xjiang
+ */
+public class OLAPSchemaFactory implements SchemaFactory {
+
+    static {
+        /*
+         * Tricks Optiq to work with Unicode.
+         * 
+         * Sets default char set for string literals in SQL and row types of
+         * RelNode. This is more a label used to compare row type equality. For
+         * both SQL string and row record, they are passed to Optiq in String
+         * object and does not require additional codec.
+         * 
+         * Ref SaffronProperties.defaultCharset
+         * SqlUtil.translateCharacterSetName() NlsString constructor()
+         */
+        System.setProperty("saffron.default.charset", ConversionUtil.NATIVE_UTF16_CHARSET_NAME);
+        System.setProperty("saffron.default.nationalcharset", ConversionUtil.NATIVE_UTF16_CHARSET_NAME);
+        System.setProperty("saffron.default.collation.name", ConversionUtil.NATIVE_UTF16_CHARSET_NAME + "$en_US");
+    }
+
+    private final static String SCHEMA_PROJECT = "project";
+
+    @Override
+    public Schema create(SchemaPlus parentSchema, String schemaName, Map<String, Object> operand) {
+        String project = (String) operand.get(SCHEMA_PROJECT);
+        Schema newSchema = new OLAPSchema(project, schemaName);
+        return newSchema;
+    }
+
+    public static File createTempOLAPJson(String project, KylinConfig config) {
+        project = ProjectInstance.getNormalizedProjectName(project);
+
+        List<TableDesc> tables = ProjectManager.getInstance(config).listExposedTables(project);
+        // "database" in TableDesc correspond to our schema
+        HashMap<String, Integer> schemaCounts = DatabaseDesc.extractDatabaseOccurenceCounts(tables);
+
+        String majoritySchemaName = "";
+        int majoritySchemaCount = 0;
+        for (Map.Entry<String, Integer> e : schemaCounts.entrySet()) {
+            if (e.getValue() >= majoritySchemaCount) {
+                majoritySchemaCount = e.getValue();
+                majoritySchemaName = e.getKey();
+            }
+        }
+
+        try {
+            File tmp = File.createTempFile("olap_model_", ".json");
+
+            FileWriter out = new FileWriter(tmp);
+            out.write("{\n");
+            out.write("    \"version\": \"1.0\",\n");
+            out.write("    \"defaultSchema\": \"" + majoritySchemaName + "\",\n");
+            out.write("    \"schemas\": [\n");
+
+            int counter = 0;
+            for (String schemaName : schemaCounts.keySet()) {
+                out.write("        {\n");
+                out.write("            \"type\": \"custom\",\n");
+                out.write("            \"name\": \"" + schemaName + "\",\n");
+                out.write("            \"factory\": \"com.kylinolap.query.schema.OLAPSchemaFactory\",\n");
+                out.write("            \"operand\": {\n");
+                out.write("                \"" + SCHEMA_PROJECT + "\": \"" + project + "\"\n");
+                out.write("            },\n");
+                out.write("           \"functions\": [\n");
+                out.write("                 {\n");
+                out.write("                     \"name\": \"QUARTER\",\n");
+                out.write("                     \"className\": \"com.kylinolap.query.sqlfunc.QuarterFunc\"\n");
+                out.write("                 }\n");
+                out.write("            ]\n");
+                out.write("        }\n");
+
+                if (++counter != schemaCounts.size()) {
+                    out.write(",\n");
+                }
+            }
+
+            out.write("    ]\n");
+            out.write("}\n");
+            out.close();
+
+            tmp.deleteOnExit();
+            return tmp;
+
+        } catch (IOException e) {
+            throw new RuntimeException(e);
+        }
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/query/src/main/java/com/kylinolap/query/schema/OLAPTable.java
----------------------------------------------------------------------
diff --git a/query/src/main/java/com/kylinolap/query/schema/OLAPTable.java b/query/src/main/java/com/kylinolap/query/schema/OLAPTable.java
new file mode 100644
index 0000000..c56c087
--- /dev/null
+++ b/query/src/main/java/com/kylinolap/query/schema/OLAPTable.java
@@ -0,0 +1,235 @@
+/*
+ * 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.query.schema;
+
+import java.util.ArrayList;
+import java.util.BitSet;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+
+import net.hydromatic.linq4j.Enumerable;
+import net.hydromatic.linq4j.Enumerator;
+import net.hydromatic.linq4j.QueryProvider;
+import net.hydromatic.linq4j.Queryable;
+import net.hydromatic.optiq.DataContext;
+import net.hydromatic.optiq.SchemaPlus;
+import net.hydromatic.optiq.Statistic;
+import net.hydromatic.optiq.Statistics;
+import net.hydromatic.optiq.TranslatableTable;
+import net.hydromatic.optiq.impl.AbstractTableQueryable;
+import net.hydromatic.optiq.impl.java.AbstractQueryableTable;
+
+import org.eigenbase.rel.RelNode;
+import org.eigenbase.relopt.RelOptTable;
+import org.eigenbase.relopt.RelOptTable.ToRelContext;
+import org.eigenbase.reltype.RelDataType;
+import org.eigenbase.reltype.RelDataTypeFactory;
+import org.eigenbase.sql.type.SqlTypeName;
+import org.eigenbase.sql.type.SqlTypeUtil;
+
+import com.kylinolap.cube.project.ProjectManager;
+import com.kylinolap.metadata.model.cube.FunctionDesc;
+import com.kylinolap.metadata.model.cube.MeasureDesc;
+import com.kylinolap.metadata.model.schema.ColumnDesc;
+import com.kylinolap.metadata.model.schema.TableDesc;
+import com.kylinolap.query.enumerator.OLAPQuery;
+import com.kylinolap.query.enumerator.OLAPQuery.EnumeratorTypeEnum;
+import com.kylinolap.query.relnode.OLAPTableScan;
+
+/**
+ * 
+ * @author xjiang
+ * 
+ */
+public class OLAPTable extends AbstractQueryableTable implements TranslatableTable {
+
+    private static Map<String, SqlTypeName> SQLTYPE_MAPPING = new HashMap<String, SqlTypeName>();
+
+    static {
+        SQLTYPE_MAPPING.put("char", SqlTypeName.CHAR);
+        SQLTYPE_MAPPING.put("varchar", SqlTypeName.VARCHAR);
+        SQLTYPE_MAPPING.put("boolean", SqlTypeName.BOOLEAN);
+        SQLTYPE_MAPPING.put("integer", SqlTypeName.INTEGER);
+        SQLTYPE_MAPPING.put("tinyint", SqlTypeName.TINYINT);
+        SQLTYPE_MAPPING.put("smallint", SqlTypeName.SMALLINT);
+        SQLTYPE_MAPPING.put("bigint", SqlTypeName.BIGINT);
+        SQLTYPE_MAPPING.put("decimal", SqlTypeName.DECIMAL);
+        SQLTYPE_MAPPING.put("numeric", SqlTypeName.DECIMAL);
+        SQLTYPE_MAPPING.put("float", SqlTypeName.FLOAT);
+        SQLTYPE_MAPPING.put("real", SqlTypeName.REAL);
+        SQLTYPE_MAPPING.put("double", SqlTypeName.DOUBLE);
+        SQLTYPE_MAPPING.put("date", SqlTypeName.DATE);
+        SQLTYPE_MAPPING.put("time", SqlTypeName.TIME);
+        SQLTYPE_MAPPING.put("timestamp", SqlTypeName.TIMESTAMP);
+        SQLTYPE_MAPPING.put("any", SqlTypeName.ANY);
+
+        // try {
+        // Class.forName("org.apache.hive.jdbc.HiveDriver");
+        // } catch (ClassNotFoundException e) {
+        // e.printStackTrace();
+        // }
+    }
+
+    private final OLAPSchema olapSchema;
+    private final TableDesc sourceTable;
+    private RelDataType rowType;
+    private List<ColumnDesc> exposedColumns;
+
+    public OLAPTable(OLAPSchema schema, TableDesc tableDesc) {
+        super(Object[].class);
+        this.olapSchema = schema;
+        this.sourceTable = tableDesc;
+        this.rowType = null;
+    }
+
+    public OLAPSchema getSchema() {
+        return this.olapSchema;
+    }
+
+    public TableDesc getSourceTable() {
+        return this.sourceTable;
+    }
+
+    public String getTableName() {
+        return this.sourceTable.getName();
+    }
+
+    public List<ColumnDesc> getExposedColumns() {
+        return exposedColumns;
+    }
+
+    @Override
+    public RelDataType getRowType(RelDataTypeFactory typeFactory) {
+        if (this.rowType == null) {
+            // always build exposedColumns and rowType together
+            this.exposedColumns = listSourceColumns();
+            this.rowType = deduceRowType(typeFactory);
+        }
+        return this.rowType;
+    }
+
+    private RelDataType deduceRowType(RelDataTypeFactory typeFactory) {
+        RelDataTypeFactory.FieldInfoBuilder fieldInfo = typeFactory.builder();
+        for (ColumnDesc column : exposedColumns) {
+            RelDataType sqlType = createSqlType(typeFactory, column);
+            sqlType = SqlTypeUtil.addCharsetAndCollation(sqlType, typeFactory);
+            fieldInfo.add(column.getName(), sqlType);
+        }
+        return typeFactory.createStructType(fieldInfo);
+    }
+
+    private RelDataType createSqlType(RelDataTypeFactory typeFactory, ColumnDesc column) {
+        SqlTypeName sqlTypeName = SQLTYPE_MAPPING.get(column.getTypeName());
+        if (sqlTypeName == null)
+            throw new IllegalArgumentException("Unrecognized column type " + column.getTypeName() + " from " + column);
+
+        int precision = column.getTypePrecision();
+        int scale = column.getTypeScale();
+
+        RelDataType result;
+        if (precision >= 0 && scale >= 0)
+            result = typeFactory.createSqlType(sqlTypeName, precision, scale);
+        else if (precision >= 0)
+            result = typeFactory.createSqlType(sqlTypeName, precision);
+        else
+            result = typeFactory.createSqlType(sqlTypeName);
+
+        // due to left join and uncertain data quality, dimension value can be
+        // null
+        if (column.isNullable()) {
+            result = typeFactory.createTypeWithNullability(result, true);
+        } else {
+            result = typeFactory.createTypeWithNullability(result, false);
+        }
+
+        return result;
+    }
+
+    private List<ColumnDesc> listSourceColumns() {
+        ProjectManager projectMgr = olapSchema.getProjectManager();
+        List<ColumnDesc> exposedColumns = projectMgr.listExposedColumns(olapSchema.getProjectName(), sourceTable.getName());
+
+        List<MeasureDesc> countMeasures = projectMgr.listEffectiveRewriteMeasures(olapSchema.getProjectName(), sourceTable.getName());
+        HashSet<String> metFields = new HashSet<String>();
+        for (MeasureDesc m : countMeasures) {
+            FunctionDesc func = m.getFunction();
+            String fieldName = func.getRewriteFieldName();
+            if (metFields.contains(fieldName) == false) {
+                metFields.add(fieldName);
+                ColumnDesc fakeCountCol = new ColumnDesc();
+                fakeCountCol.setName(fieldName);
+                fakeCountCol.setDatatype(func.getSQLType());
+                fakeCountCol.setNullable(false);
+                fakeCountCol.init(sourceTable);
+                exposedColumns.add(fakeCountCol);
+            }
+        }
+
+        return exposedColumns;
+    }
+
+    @Override
+    public RelNode toRel(ToRelContext context, RelOptTable relOptTable) {
+        int fieldCount = relOptTable.getRowType().getFieldCount();
+        int[] fields = identityList(fieldCount);
+        return new OLAPTableScan(context.getCluster(), relOptTable, this, fields);
+    }
+
+    private int[] identityList(int n) {
+        int[] integers = new int[n];
+        for (int i = 0; i < n; i++) {
+            integers[i] = i;
+        }
+        return integers;
+    }
+
+    @Override
+    public <T> Queryable<T> asQueryable(QueryProvider queryProvider, SchemaPlus schema, String tableName) {
+        return new AbstractTableQueryable<T>(queryProvider, schema, this, tableName) {
+            @SuppressWarnings("unchecked")
+            public Enumerator<T> enumerator() {
+                final OLAPQuery query = new OLAPQuery(EnumeratorTypeEnum.CUBE, 0);
+                return (Enumerator<T>) query.enumerator();
+            }
+        };
+    }
+
+    @Override
+    public Statistic getStatistic() {
+        List<BitSet> keys = new ArrayList<BitSet>();
+        return Statistics.of(100, keys);
+    }
+
+    @Override
+    public String toString() {
+        return "OLAPTable {" + getTableName() + "}";
+    }
+
+    public Enumerable<Object[]> executeCubeQuery(DataContext optiqContext, int ctxSeq) {
+        return new OLAPQuery(optiqContext, EnumeratorTypeEnum.CUBE, ctxSeq);
+    }
+
+    public Enumerable<Object[]> executeLookupTableQuery(DataContext optiqContext, int ctxSeq) {
+        return new OLAPQuery(optiqContext, EnumeratorTypeEnum.LOOKUP_TABLE, ctxSeq);
+    }
+
+    public Enumerable<Object[]> executeHiveQuery(DataContext optiqContext, int ctxSeq) {
+        return new OLAPQuery(optiqContext, EnumeratorTypeEnum.HIVE, ctxSeq);
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/query/src/main/java/com/kylinolap/query/sqlfunc/HLLDistinctCountAggFunc.java
----------------------------------------------------------------------
diff --git a/query/src/main/java/com/kylinolap/query/sqlfunc/HLLDistinctCountAggFunc.java b/query/src/main/java/com/kylinolap/query/sqlfunc/HLLDistinctCountAggFunc.java
new file mode 100644
index 0000000..e8f952a
--- /dev/null
+++ b/query/src/main/java/com/kylinolap/query/sqlfunc/HLLDistinctCountAggFunc.java
@@ -0,0 +1,151 @@
+/*
+ * 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.query.sqlfunc;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.kylinolap.common.hll.HyperLogLogPlusCounter;
+
+/**
+ * @author xjiang
+ */
+public class HLLDistinctCountAggFunc {
+
+    private static final Logger logger = LoggerFactory.getLogger(HLLDistinctCountAggFunc.class);
+
+    public static HyperLogLogPlusCounter init() {
+        return null;
+    }
+
+    public static HyperLogLogPlusCounter initAdd(Object v) {
+        if (v instanceof Long) { // holistic case
+            long l = (Long) v;
+            return new FixedValueHLLCMockup(l);
+        } else {
+            HyperLogLogPlusCounter c = (HyperLogLogPlusCounter) v;
+            return new HyperLogLogPlusCounter(c);
+        }
+    }
+
+    public static HyperLogLogPlusCounter add(HyperLogLogPlusCounter counter, Object v) {
+        if (v instanceof Long) { // holistic case
+            long l = (Long) v;
+            if (counter == null) {
+                return new FixedValueHLLCMockup(l);
+            } else {
+                if (!(counter instanceof FixedValueHLLCMockup))
+                    throw new IllegalStateException("counter is not FixedValueHLLCMockup");
+
+                ((FixedValueHLLCMockup) counter).set(l);
+                return counter;
+            }
+        } else {
+            HyperLogLogPlusCounter c = (HyperLogLogPlusCounter) v;
+            if (counter == null) {
+                return new HyperLogLogPlusCounter(c);
+            } else {
+                counter.merge(c);
+                return counter;
+            }
+        }
+    }
+
+    public static HyperLogLogPlusCounter merge(HyperLogLogPlusCounter counter0, Object counter1) {
+        return add(counter0, counter1);
+    }
+
+    public static long result(HyperLogLogPlusCounter counter) {
+        return counter == null ? 0L : counter.getCountEstimate();
+    }
+
+    private static class FixedValueHLLCMockup extends HyperLogLogPlusCounter {
+
+        private Long value = null;
+
+        FixedValueHLLCMockup(long value) {
+            this.value = value;
+        }
+
+        public void set(long value) {
+            if (this.value == null) {
+                this.value = value;
+            } else {
+                long oldValue = Math.abs(this.value.longValue());
+                long take = Math.max(oldValue, value);
+                logger.warn("Error to aggregate holistic count distinct, old value " + oldValue + ", new value " + value + ", taking " + take);
+                this.value = -take; // make it obvious that this value is wrong
+            }
+        }
+
+        @Override
+        public void clear() {
+            this.value = null;
+        }
+
+        @Override
+        protected void add(long hash) {
+            throw new UnsupportedOperationException();
+        }
+
+        @Override
+        public void merge(HyperLogLogPlusCounter another) {
+            throw new UnsupportedOperationException();
+        }
+
+        @Override
+        public long getCountEstimate() {
+            return value;
+        }
+
+        @Override
+        public void writeRegisters(ByteBuffer out) throws IOException {
+            throw new UnsupportedOperationException();
+        }
+
+        @Override
+        public void readRegisters(ByteBuffer in) throws IOException {
+            throw new UnsupportedOperationException();
+        }
+
+        @Override
+        public int hashCode() {
+            final int prime = 31;
+            int result = super.hashCode();
+            result = prime * result + (int) (value ^ (value >>> 32));
+            return result;
+        }
+
+        @Override
+        public boolean equals(Object obj) {
+            if (this == obj)
+                return true;
+            if (!super.equals(obj))
+                return false;
+            if (getClass() != obj.getClass())
+                return false;
+            FixedValueHLLCMockup other = (FixedValueHLLCMockup) obj;
+            if (!value.equals(other.value))
+                return false;
+            return true;
+        }
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/query/src/main/java/com/kylinolap/query/sqlfunc/QuarterBase.java
----------------------------------------------------------------------
diff --git a/query/src/main/java/com/kylinolap/query/sqlfunc/QuarterBase.java b/query/src/main/java/com/kylinolap/query/sqlfunc/QuarterBase.java
new file mode 100644
index 0000000..f0ff187
--- /dev/null
+++ b/query/src/main/java/com/kylinolap/query/sqlfunc/QuarterBase.java
@@ -0,0 +1,20 @@
+package com.kylinolap.query.sqlfunc;
+
+import net.hydromatic.optiq.runtime.SqlFunctions;
+import net.hydromatic.optiq.runtime.SqlFunctions.TimeUnitRange;
+
+/**
+ * @author xjiang
+ * 
+ */
+public abstract class QuarterBase {
+    
+    /**
+     * According to jvm spec, it return self method before parent.
+     * So, we keep Date in parent and int in child
+     */
+    public static long eval(int date) {
+        long month = SqlFunctions.unixDateExtract(TimeUnitRange.MONTH, date);
+        return month / 4 + 1;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/query/src/main/java/com/kylinolap/query/sqlfunc/QuarterFunc.java
----------------------------------------------------------------------
diff --git a/query/src/main/java/com/kylinolap/query/sqlfunc/QuarterFunc.java b/query/src/main/java/com/kylinolap/query/sqlfunc/QuarterFunc.java
new file mode 100644
index 0000000..1d72a20
--- /dev/null
+++ b/query/src/main/java/com/kylinolap/query/sqlfunc/QuarterFunc.java
@@ -0,0 +1,32 @@
+/*
+ * 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.query.sqlfunc;
+
+import java.sql.Date;
+
+/**
+ * @author xjiang
+ * 
+ */
+public class QuarterFunc extends QuarterBase {
+    private QuarterFunc() {
+    }
+
+    public static long eval(Date date) {
+        throw new UnsupportedOperationException();
+    }
+}


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

Posted by lu...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/dictionary/src/test/resources/dict/dw_category_grouping_names.dat
----------------------------------------------------------------------
diff --git a/dictionary/src/test/resources/dict/dw_category_grouping_names.dat b/dictionary/src/test/resources/dict/dw_category_grouping_names.dat
new file mode 100644
index 0000000..129c48d
--- /dev/null
+++ b/dictionary/src/test/resources/dict/dw_category_grouping_names.dat
@@ -0,0 +1,448529 @@
+Auto-Ersatz- & -Reparaturteile
+V?tements b?b?s, tout-petits
+Herrenschuhe
+Great Britain
+Fashion Jewelry
+Sachb?cher & Ratgeber
+Watches
+Postcards
+Herrenschuhe
+Women's Shoes
+Car, Truck Parts
+Rasenpflegeger?te
+T?l?, vid?o & audio domestique
+Uomo: Abbigliamento
+Damenmode
+Car Electronics
+Heavy Equipment
+Feste & Besondere Anl?sse
+Fashion Jewellery
+Bad & K?che
+Gardening
+Zeitschriften
+Kids' Clothing, Shoes & Accs
+Kids' Clothing, Shoes & Accs
+Film Photography
+V?tements pour hommes
+Kindermode, Schuhe & Access.
+Baby Clothing
+Tenue de danse
+Team Sports
+Auto-Ersatz- & -Reparaturteile
+Cycling
+Home Improvement
+Studium & Wissen
+Watches
+Womens' Clothing
+Tennis & sports de raquette
+MP3 Players
+Jewellery & Watches
+Makeup
+Kinderm?bel & Wohnen
+Digital Clocks & Clock Radios
+Loose Diamonds & Gemstones
+Damenmode
+Cleaning & Laundry Supplies
+Automobile
+Exonumia
+Handys ohne Vertrag
+Sciences, culture
+Weird Stuff
+Orologi da Polso: alla Moda
+K?rperpflege
+Dameskleding
+Cyclisme
+Movie Memorabilia
+CD's
+Standmodelle
+Lampes, plafonniers-?ventails
+Vinyl
+Reise & Regionales
+Games
+Animals
+Tv-componenten & Accessoires
+Pottery & China
+Paper
+????
+Damenmode
+Kleinger?te K?che
+Losse onderdelen PC
+DVD, Blu-ray & Heimkino
+Retro Home computers
+Men's Clothing
+Film DVD's
+Antiquarische B?cher
+Parfums
+Parts & Accessories
+TV & Home Audio Accessories
+TV- & Heim-Audio-Teile
+Produkcja i Przemys?
+Food & Beverages
+Other
+Digitalkameras
+Fragrances
+Nederlandse niet-euromunten
+Reise & Regionales
+Ricambi Auto
+Livres anciens - avant 1900
+Sports d'hiver
+Reit- & Fahrsport
+Baby Clothing
+Parfums
+Automobiles & camions
+Modelbouw
+Donna: Scarpe
+Gastro & Nahrungsmittelgewerbe
+Cz??ci samochodowe
+Cards
+Damenmode
+Printing & Personalization
+Vintage
+Veranstaltungs- & DJ-Equipment
+Kunststoffindustrie & Chemie
+Telefonia fissa
+Car Electronics
+Computer Components & Parts
+Fan Apparel & Souvenirs
+Drucker, Scanner & Zubeh?r
+Enterprise Networking, Servers
+Calzado mujer
+Damenmode
+Meubles, d?coration, Xx?me
+Bijoux de mode
+Handys ohne Vertrag
+R?seau d'entreprise, serveurs
+Playmobil
+Auto-Anbau- & -Zubeh?rteile
+Vehicle Electronics & GPS
+France
+Transport
+Hand Tools & Equipment
+Printing & Graphic Arts
+Winter Sports
+Transport
+Vinyles 33T
+Damenmode
+Redes de empresa y servidores
+Produktions- & Industriebedarf
+Damenmode
+Fotostudio-Zubeh?r
+Tools
+Accessoires de jeux vid?o
+????
+Wedding
+Jewellery Boxes & Supplies
+Composants PC
+Papier- en schrijfwaren
+Echtschmuck
+Nonfiction
+Europe
+Auto-Tuning & -Styling
+Imprimantes, scanneurs & acc.
+Petit Electro - Cuisine
+Vehicle Electronics & GPS
+Motorrad- & Kraftradteile
+Pottery
+TV, Video & Home Audio
+Auto-Hi-Fi & Navigation
+Damenmode
+Bolsos
+iPods & MP3 Players
+Cards
+Armband- & Taschenuhren
+Auto-Ersatz- & -Reparaturteile
+??/????
+Storage
+Sonstige
+Baugewerbe
+Metalware
+Handys ohne Vertrag
+Firmennetzwerk, Server
+Varie
+DVDs & Blu-rays
+Diecast & Toy Vehicles
+Motorrad- & Kraftradteile
+Damenschuhe
+Enterprise Networking, Servers
+Science Fiction & Horror
+Modellautos
+V?tements, accessoires
+Comics
+Historical Memorabilia
+Outdoor Toys
+Auto-Tuning & -Styling
+Metallbearbeitung & Schlosser.
+Drucker, Scanner & Zubeh?r
+Cars & Trucks
+Elektronik & Elektrotechnik
+Cars
+Film Photography
+Pi?ces & accessoires
+Militaria
+Vinyl en platen
+Gewerbe
+Engagement & Wedding
+Travaux de r?novation
+Dolls
+Computer-Komponenten & -Teile
+Accessories
+Automobilia
+Fashion Jewelry
+Armband- & Taschenuhren
+Festivals & Konzerte
+Miscellaneous
+Kleidung, Schuhe & Accessoires
+Coins
+Feste & Besondere Anl?sse
+Album e figurine
+Damen-Designerbekleidung
+Makeup
+Heavy Equipment
+Concerts
+R?seau d'entreprise, serveurs
+Baby Clothing
+Fashion Jewellery
+LCD / TFT Monitors
+Wereldkunst
+Sonstige
+Performance & DJ Equipment
+Laptop & Desktop Accessories
+Pi?ces & accessoires
+Herrenschuhe
+F?r Bastler
+Motorr?der
+Skin Care
+DVDs & Blu-rays
+Herrenmode
+M?bel
+Baby Clothing
+Reit- & Fahrsport
+Dekoration
+Echtschmuck
+Cars & Trucks
+Golf
+Am?rique latine
+Sports de plein air
+Parfums
+Automobilia
+DIY Materials
+Dolls
+Decorative Arts
+M?nzen Deutschland ab 1945
+Cleaning, Housekeeping
+Gitarren
+Shaving & Hair Removal
+Clothes, Shoes & Accessories
+Motorcycles
+Billets de spectacles
+LEGO
+CDs & DVDs
+Trading Cards
+Ringtones, Logos & Software
+Action Figures
+Kindermode, Schuhe & Access.
+Home Audio & HiFi Separates
+Haarpflege
+Kindermode, Schuhe & Access.
+Gesichtspflege
+Bijoux d'?poque, anciens
+Edelsteine
+Kindermode, Schuhe & Access.
+Jeux
+Clothes, Shoes & Accessories
+Clothes, Shoes & Accessories
+Litograf?as y Grabados
+Kleidung & Accessoires
+V?tements, accessoires
+Non-Fiction
+Meubles
+Voiture ancienne: pi?ces
+GSM's
+Fan Apparel & Souvenirs
+Kindermode, Schuhe & Access.
+Pro Audio
+??????/??
+Radsport
+Dolls
+Cards
+Hobby & K?nstlerbedarf
+Sachb?cher & Ratgeber
+Herenschoenen
+????
+DVDs & Blu-rays
+Reise & Regionales
+V?tements pour enfants
+Giochi Nintendo
+Outdoor Sports
+??
+Education & Professional
+Half Video Games
+Cookware, Dining & Bar
+Decorative Collectibles
+Fantaisie/mythe/magie
+Yard, Garden & Outdoor Living
+Non-Fiction
+Sonstige Branchen & Produkte
+Pro Audio Equipment
+Model Trains
+Fine Jewelry
+Enterprise Networking, Servers
+Fine Jewellery
+Decorative Collectibles
+Kitchenware
+Medizin & Labor
+Cross Stitch
+Familienplanung
+DVD, Blu-ray & Heimkino
+Baugewerbe
+Bi?uteria z pere?
+Histoire - philat?lie & poste
+Vehicle Electronics & GPS
+Video Game Accessories
+Yard, Garden & Outdoor Living
+Gastro & Nahrungsmittelgewerbe
+Bettausstattung
+Handys ohne Vertrag
+Cell Phone Accessories
+Motorcycles
+Mat?riel ?lectrique & d'essai
+Advertising
+Design & Stil
+Locaux: Mat?riel, Fournitures
+Tragbare Disc-Player & Radios
+Damenmode
+Cz??ci samochodowe: Tuning
+Kamine & Kaminzubeh?r
+Antiquarian & Collectible
+Kurzreisen
+??GSM??
+Motorrad- & Kraftradteile
+Parfums
+Cultural, Religious
+Clothes, Shoes & Accessories
+Radio Control
+Accessoires
+Electrical & Test Equipment
+Kleidung, Schuhe & Accessoires
+Damenmode
+Couture, Broderie, Tricot
+Mobile Accessories
+Kleidung, Schuhe & Accessoires
+Kleinger?te K?che
+Antiquarische B?cher
+Auto-Ersatz- & -Reparaturteile
+Baugewerbe
+Cars & Trucks
+??/????
+Herrenmode
+Gastro & Nahrungsmittelgewerbe
+Gro?handel & Sonderposten
+Arcade, Jukeboxes & Pinball
+Dameskleding
+Non-Fiction
+Llantas y Neum?ticos
+Motorr?der
+Furniture
+Breweriana
+Business & Management
+Damenmode
+Kindermode, Schuhe & Access.
+Cycling
+???
+Studium & Wissen
+CDs
+Dollhouse Miniatures
+Trading Cards
+Zeitschriften
+Laptop & Desktop Accessories
+Bijoux artisanaux
+Software
+Puppen
+Kindermode, Schuhe & Access.
+????????
+Test Auctions
+Photographie argentique
+Comics
+Autographs-Original
+Lotti e Stock Infanzia
+Damenmode
+Herrenmode
+Kindermode, Schuhe & Access.
+RC-Modellbau
+Kindermode, Schuhe & Access.
+Vintage & Antique Jewelry
+Albums BD
+Armband- & Taschenuhren
+Printers, Scanners & Supplies
+Technik & Photographica
+Oorlog
+Audio
+Men's Jewelry
+Pro Audio Equipment
+M?nzen Dt. Reich 1871-1945
+Kunststoffindustrie & Chemie
+Herrenmode
+Studium & Wissen
+Theatre Memorabilia
+Portable Audio & Headphones
+Poup?es
+Golf
+Digital Cameras
+K?nstlerbedarf
+Gadgets & Other Electronics
+Car Parts
+Odzie? m?ska
+Bettwaren, -w?sche & Matratzen
+Women's Clothing
+Heating/ Cooling/ Air
+Cards
+Postcards
+Postcards
+Mobiliar & Interieur
+Photographic Accessories
+V?tements b?b?s, tout-petits
+Skisport & Snowboarding
+Bistrot
+Cz??ci samochodowe
+Tools, Hardware & Locks
+Dekoration
+Kochen & Genie?en
+Handys mit Vertrag
+Digitalkameras
+P?yty kompaktowe
+Fantasy, Mythical & Magic
+T?l?, vid?o & audio domestique
+Gro?handel & Sonderposten
+Greeting Cards & Party Supply
+Gro?handel & Sonderposten
+Pi?ces & accessoires
+Fan Apparel & Souvenirs
+Handys mit Vertrag
+TV- & Heim-Audio-Zubeh?r
+Nintendo overig
+Handy- & PDA-Zubeh?r
+Modellbaus?tze
+Nonfiction Books
+Reise & Regionales
+Drucker, Scanner & Zubeh?r
+Computer Components & Parts
+Miscellaneous
+Transportation
+Fan Shop
+Sewing/ Fabric/ Textiles
+Restaurant & Catering
+Agrar, Forst & Kommune
+Diecast & Toy Vehicles
+Computer Components & Parts
+Damenmode
+Th?mes
+V?tements & souvenirs de fan
+Haushalt
+Radsport
+Science-Fiction
+Antiquarische B?cher
+Drives, Storage & Blank Media
+Rollenspiele & Tabletops
+V?tements, accessoires
+Collections
+Haushalt
+Car Electronics
+Automobilia
+Odzie? pozosta?a
+Sauna & Schwimmbad
+Beeldende kunst
+Makeup
+Kleinger?te K?che
+Guitar
+Saggistica
+Electrical & Test Equipment
+Manik?re & Pedik?re
+Damenmode
+Needlecrafts & Yarn
+Komunikacja i ??czno??
+Kleidung, Schuhe & Accessoires
+Fan Apparel & Souvenirs
+Echtschmuck
+Back?fen & Herde
+Dolls & Toys
+Baugewerbe
+Autographs
+Weitere Ballsportarten
+Home Audio & HiFi Separates
+Massage
+Bedding
+Motorcycles
+BD
+Tennis & Racquet Sports
+Women's Clothing
+Football Memorabilia
+Reptilien
+????
+????DIY
+Elektronika i elektrotechnika
+Parfums
+B?rom?bel
+Paper
+Stamps
+Timeshare
+Kids' Clothes, Shoes & Accs.
+Cz??ci motocyklowe
+Skisport & Snowboarding
+Decorative Arts
+USB
+Boats
+Z?une & Sichtschutzw?nde
+Pianos, Keyboards & Organs
+Kindermode, Schuhe & Access.
+????
+Livres de bandes dessin?es
+V?tements pour enfants
+Bew?ssern
+Games
+Porzellan & Keramik
+Gefrierger?te & K?hlschr?nke
+Gartenger?te
+Clothes, Shoes & Accessories
+Bijoux de mode
+Fan Shop
+Dolls
+Fernseher
+Damenschuhe
+Manuals & Guides
+Computer-Komponenten & -Teile
+Figuren
+Games
+Comics
+Kids' Clothing, Shoes & Accs
+Women's Shoes
+PC- & Videospiele
+Trains/ Railway Models
+Hard Disk Drives
+Fitness
+Rugs & Carpets
+Memorabilia
+B?rotechnik
+Damenmode
+Stationery
+Animation Art, Characters
+Relojes de Pulsera
+Golf
+Accessori laptop e desktop
+DVDs & Blu-ray Discs
+S?mereien & Zwiebeln
+Kleinger?te K?che
+Damenmode
+Porzellan & Keramik
+Weiterer Wassersport
+DVDs & Blu-ray Discs
+MRO & Industrial Supply
+Batteries
+Armband- & Taschenuhren
+Arts, Artisanat, D?coration
+Damenmode
+Gitarren
+DVDs & Blu-rays
+Water Sports
+Test Auctions
+Kleidung, Schuhe & Accessoires
+Cards
+Guitar
+Radsport
+Fashion Jewellery
+Femmes: Chaussures
+Fine Jewellery
+Men's Clothing
+Trading Card Games
+Hobby & K?nstlerbedarf
+Comic Books
+Football Shirts
+P?riodiques, Magazines BD
+Audio- & Video-Rarit?ten
+Film Photography
+Automobile
+Baugewerbe
+Kindermode, Schuhe & Access.
+Chaussures pour hommes
+Kleidung, Schuhe & Accessoires
+Computer-Komponenten & -Teile
+Holiday & Seasonal
+Office
+Motorrad- & Kraftradteile
+Heim-Audio & HiFi
+Echtschmuck
+Soins du Corps & Hygi?ne
+Animals
+Networking e reti home
+Vintage Sports Memorabilia
+Personnages de t?l? & de films
+Pottery
+Trading Cards
+Serien & Lizenzprodukte
+Motorcycle Parts & Accessories
+Logiciels
+Services & Real Estate
+Miniatures
+T?l?phones ? domicile
+Casino
+LCD TVs
+Weitere Ballsportarten
+Muziek luisteren
+Guitare
+Cartes de collection
+Sonnenschutz
+Kleinger?te K?che
+Antiquarische B?cher
+Teich- & Bachlaufsysteme
+Auto-Tuning & -Styling
+Baby Feeding
+TV, Video & Home Audio
+Auto-Anbau- & -Zubeh?rteile
+Mariage & tenue habill?e
+Herrenschmuck
+Silver
+Damenmode
+Fine Jewellery
+Men's Accessories
+Damenmode
+Kinderkleding- meisjes
+Teich- & Bachlaufsysteme
+Kitchen, Dining & Bar
+Sports motoris?s
+Vins
+Zabawki
+Porcelain/ China
+Laptops & Netbooks
+Anlageobjekte/Versteigerungen
+Damenmode
+Auto-Ersatz- & -Reparaturteile
+Souvenirs ?missions t?l?vis?es
+Militaria
+Overige games & Consoles
+Armband- & Taschenuhren
+Bijoux pour hommes
+Moto e Scooter
+Reading Accessories
+Car, Truck Parts
+TV, Movie & Character Toys
+Damenmode
+Auto-Ersatz- & -Reparaturteile
+Lighting & Studio
+Geb?udebaus?tze
+Camera & Photo Accessories
+Cardmaking & Scrapbooking
+Modernariato
+Model Trains
+Film DVD's
+Accessori cellulari e palmari
+Car Parts
+Plants/ Seeds/ Bulbs
+Vintage-Mode
+Kleidung, Schuhe & Accessoires
+Telescopios y prism?ticos
+Records
+Kleidung, Schuhe & Accessoires
+Brass
+Pauschalreisen
+RC-Modellbau
+Fashion Jewellery
+Budownictwo i akcesoria
+???
+Kindermode, Schuhe & Access.
+Women's Clothing
+Objets de collection de d?cora
+Playstation 2
+Gastro & Nahrungsmittelgewerbe
+Building Materials, DIY
+Nursery Decor
+Pi?ces & accessoires
+Jeux d'int?rieur
+Auto-Ersatz- & -Reparaturteile
+Cz??ci samochodowe: Tuning
+Transportation
+Other Jewellery
+Koffer, Taschen & Accessoires
+Diecast & Toy Vehicles
+Revues, manuels, catalogues
+Audio portatifs & ?couteurs
+Transport
+Fish & Aquarium
+Bambini 2 - 16 anni
+Exercise & Fitness
+Music Memorabilia
+Barbie
+Hair Care & Styling
+Laptop & Desktop Accessories
+Printers, Scanners & Supplies
+Autres Sports
+Auto-Tuning & -Styling
+?iEI
+Kunst & Kultur
+M?nzen Europa
+Clothing: Girls Jr
+Antikspielzeug
+Teleskope & Ferngl?ser
+?????????
+Bottles & Insulators
+Werkzeuge & Werkstattbedarf
+????
+Engagement & Wedding
+Skisport & Snowboarding
+Kindermode, Schuhe & Access.
+Kids' Clothes, Shoes & Accs.
+Zubeh?r
+Damenmode
+Auto-Ersatz- & -Reparaturteile
+K?rperpflege
+Eisenwaren
+Pflanzen, B?ume & Str?ucher
+Men's Clothing
+Computer-Komponenten & -Teile
+iPods & MP3-Player
+Modellautos
+M?bler
+Motorradkleidung
+Rennbahnen & Slotcars
+Sachb?cher & Ratgeber
+Asian Art
+Loose Diamonds & Gemstones
+Damenmode
+Kost?me & Verkleidungen
+Werkzeug
+Skisport & Snowboarding
+Car, Truck Parts
+Imprimantes, scanneurs & acc.
+Cookware, Dining & Bar
+Gadgets & autres ?lectroniques
+Cucina
+Disneyana
+Education & Professional
+Weitere Sportarten
+Componenti e parti
+Vintage
+Camping & Outdoor
+Kindermode, Schuhe & Access.
+Historische Wertpapiere
+Furniture
+Technik & Ger?te
+Vintage
+Kabel & Steckverbinder
+Women's Clothing
+Scolaire, dictionnaires
+Parts & Accessories
+Records
+Historical Memorabilia
+CD
+Makeup
+Motorr?der
+Cards
+Kinderwagen & -tragen
+Figurines d'action
+Kleidung, Schuhe & Accessoires
+Kunst & Kultur
+Cuisine, restaurants & bar
+Manik?re & Pedik?re
+Test Auctions
+Pet Accessories
+Handy- & PDA-Zubeh?r
+Fu?ball-Fanshop
+Guitare
+Zahnpflege
+Holzspielzeug
+Damenmode
+Outdoor Sports
+Auto-Anbau- & -Zubeh?rteile
+Guitar
+Haushalt & K?che
+iPods & MP3 Players
+Historical Memorabilia
+Transportation
+Construction Toys & Kits
+Boats
+Postcards
+Herrenschuhe
+Children's & Young Adults
+Advertising
+Auto-Ersatz- & -Reparaturteile
+Verre, cristal
+Fu?ball
+Golf
+Business & Management
+Bath
+Motorrad- & Kraftradteile
+Manufacturiers & m?tallurgie
+Onderdelen Personenauto's
+Men's Jewelry
+Auto-Ersatz- & -Reparaturteile
+Filmposter
+DVDs & Blu-rays
+Cell Phone Accessories
+Echtschmuck
+TV-Receiver & Set-Top-Boxen
+Jewellery Design & Repair
+Desktops & All-In-Ones
+Football Programmes
+Filters
+Girls' Clothing
+Nederland
+Motorrad- & Kraftradteile
+Software
+Fishing
+Speakers, Webcams & Multimedia
+Haarpflege
+Pro Audio Equipment
+Damenmode
+Womens' Handbags & Bags
+Auto-Tuning & -Styling
+Drucker, Scanner & Zubeh?r
+Edelsteine
+Motorrad- & Kraftradteile
+Software
+Gesichtspflege
+Men's Shoes
+Laufwerke & Speichermedien
+Nachschlagewerke
+Books & Manuscripts
+Skisport & Snowboarding
+Yard, Garden & Outdoor Living
+Horse Riding
+Kochen & Genie?en
+Men's Accessories
+CDs
+Skisport & Snowboarding
+Ansichtskarten
+Wholesale Lots
+Calendriers, tickets, affiches
+Fish & Aquariums
+Kids' Clothes, Shoes & Accs.
+VHS-Kassetten
+Kids' Clothing, Shoes & Accs
+Golf
+Transport
+Auto-Ersatz- & -Reparaturteile
+United States
+Fleisch & Wurst
+Yard, Garden & Outdoor Living
+Computer-Komponenten & -Teile
+Auto-Anbau- & -Zubeh?rteile
+Golf
+CD
+Animation Art & Characters
+Kids' Clothes, Shoes & Accs.
+V?tements, accessoires
+Non-Fiction
+H?rb?cher & H?rspiele
+Fan Apparel & Souvenirs
+Construction
+T?l?, vid?o & audio domestique
+Damenmode
+Modelleisenbahn
+KFZ-Handwerk
+Antiquarische B?cher
+Enthaarung & Rasur
+Kindermode, Schuhe & Access.
+Test Auctions
+Geheugenkaarten
+R?seaux
+Hi-Fi, son, mat?riel audio
+Monedas Antiguas
+Bettausstattung
+Coins
+Transportation
+Testbereich
+Damenmode
+Damenmode
+Computer-Komponenten & -Teile
+Luxus-Accessoires
+Handys ohne Vertrag
+Non-Fiction
+Computer-Komponenten & -Teile
+Medizin & Labor
+Car, Truck Parts
+Dekoration
+Fine Jewellery
+Kochen & Genie?en
+Trading Card Games
+Men's Accessories
+Cards
+Transportation
+Postcards
+DIY Materials
+Memorabilia
+Chasse
+Jantes & Pneus
+Componenti e parti
+PC- & Videospiele
+Handys mit Vertrag
+Kindermode, Schuhe & Access.
+Ladenausstattung & -werbung
+Digitalkameras
+Logiciels
+Hobby & K?nstlerbedarf
+Sachb?cher & Ratgeber
+Funktechnik
+Gardening
+Jongenskleding
+Non-Fiction Books
+Pi?ces Etrang?res
+Fashion Jewellery
+Home Networking & Connectivity
+Football Memorabilia
+Input Peripherals
+Kindermode, Schuhe & Access.
+Commonwealth/ British Colonial
+Reit- & Fahrsport
+Kitchen, Dining & Bar
+Digital Cameras
+Modelleisenbahn
+Kindermode, Schuhe & Access.
+Dekoration
+??/??
+Diecast
+Lenses & Filters
+TV, Video & Home Audio
+Accessoires pour femmes
+Bad & K?che
+Heavy Equipment Attachments
+Nachschlagewerke
+Fashion Jewelry
+Beleuchtung
+Swimming
+Ansichtskarten
+DVD's
+Food & Beverages
+Bureau
+TV & Film Character Toys
+Business & Management
+Beleuchtung
+Automobilia
+Historische Wertpapiere
+Fan Apparel & Souvenirs
+Women's Clothing
+Heim-Audio & HiFi
+Feste & Besondere Anl?sse
+Test Auctions
+Souvenirs de jeu usag?s
+Malerei
+?berraschungseier
+Oprogramowanie
+Gesichtspflege
+Pro-Audio Equipment
+Budownictwo
+Kleidung, Schuhe & Accessoires
+Sports de plein air
+Echtschmuck
+Motorcycle Parts & Accessories
+Maison & jardin
+Hunting
+Other Formats
+Powersports
+Baby Clothing
+Claviers, souris, pointeurs
+Piel?gnacja jamy ustnej
+Wholesale Lots
+B?rotechnik
+Historische Wertpapiere
+Elektronik & Elektrotechnik
+Software
+Records
+Fan Apparel & Souvenirs
+Antiques 20th Century
+Design & Stil
+Exotische Instrumente
+MRO & Industrial Supply
+Auto's
+Cz??ci samochodowe
+Sports d'?quipe
+Odzie? damska
+Autogramme & Autographen
+Radio Control & Control Line
+Fan Apparel & Souvenirs
+P?yty kompaktowe
+Heim-Audio & HiFi
+Car Tuning & Styling
+Kindermode, Schuhe & Access.
+%F%l%S%2!<%`
+Multipurpose Batteries & Power
+Reise & Regionales
+Advertising
+Antique Furniture
+Music Memorabilia
+Shaving & Hair Removal
+Parts & Accessories
+Vintage Clothing & Accessories
+Comics
+Manuscripts
+Motorrad- & Kraftradteile
+Laufwerke & Speichermedien
+Herrenschuhe
+Accessoires t?l. mobiles, PDA
+Radsport
+Camping
+Porzellan & Keramik
+Yard, Garden & Outdoor Living
+Cell Phone Accessories
+Tatouages & Art corporel
+Furniture
+Men's Clothing
+Boy's Accessories
+Antique Furniture
+Antiquities
+Animals
+Portable Audio & Headphones
+Armband- & Taschenuhren
+Handys ohne Vertrag
+Obr?bka drewna i stolarstwo
+Heim-Audio & HiFi
+Portables, netbooks
+Automobile
+Handys ohne Vertrag
+Kindermode, Schuhe & Access.
+Reise & Regionales
+Golf
+Belletristik
+Religion & Spirituality
+Laufwerke & Speichermedien
+Pi?ces Euro
+Men's Clothing
+Antique Furniture
+V?hicules miniatures
+Industrial Tools
+Outdoor Sports
+Fitness
+Zubeh?r
+Elektronik & Elektrotechnik
+Holiday & Seasonal
+Accessories
+Yard, Garden & Outdoor Living
+Fu?ball
+Animals
+Aliment & boissons
+Women's Shoes
+Literatur & Kochb?cher
+Automobilia
+Vinyl
+Tools
+Action Figures
+Gesichtspflege
+Vorsorge & Entspannung
+????
+Hunde
+TV- & Heim-Audio-Zubeh?r
+Bain & soins du corps
+Computer Components & Parts
+Europe
+Coins
+T?l?, vid?o & audio domestique
+Fragrances
+Manuals & Resources
+Produktions- & Industriebedarf
+Audio- & Video-Rarit?ten
+Motorcycle Parts & Accessories
+Books
+Sports Cars
+Europe
+Preschool Toys & Pretend Play
+Non Fiction
+Computer-Komponenten & -Teile
+????
+Animals
+Reit- & Fahrsport
+Korsord & korsordsrelaterat
+Studium & Wissen
+Cat?gorie test
+Cars & Trucks
+Women's Shoes
+M?nzen Dt. Reich 1871-1945
+Damenmode
+Computer Components & Parts
+Fan Shop
+Vintage
+Children's Books
+Veranstaltungs- & DJ-Equipment
+Autoreifen & Felgen
+Motorrad- & Kraftradteile
+Damenmode
+Computer-Komponenten & -Teile
+Kinderkleding- meisjes
+Outdoor Sports
+Standmodelle
+Magazines
+Unisex Jewellery
+Klein- & H?ngeaufbewahrung
+Guitar
+Reit- & Fahrsport
+Kleinger?te K?che
+????
+Cz??ci samochodowe
+Kunststoffindustrie & Chemie
+Damenmode
+Printers & Scanners
+Zubeh?r & Aufbewahrung
+Reise & Regionales
+Auto-Anbau- & -Zubeh?rteile
+Medizin & Labor
+V?tements/accessoires d'?poque
+Herrenschuhe
+Kochen & Genie?en
+Herrenschuhe
+M?bel
+Damenmode
+Poligrafia
+L?sungsb?cher & Ratgeber
+D?coration de maison
+Holzbearbeitung & Tischlerei
+Back?fen & Herde
+Herrenschmuck
+Skin Care
+Fu?ball
+Coins: World
+Action- & Spielfiguren
+Engagement & Wedding
+Home Improvement
+Books/Music/Kids Book
+Damenmode
+Games
+Greeting Cards & Party Supply
+Animals
+Reit- & Fahrsport
+Pauschalreisen
+DVDs & Blu-rays
+Make-up
+Kleidung, Schuhe & Accessoires
+Accessoires de voyage
+Body Jewellery
+Animals
+Parts & Accessories
+Sachb?cher & Ratgeber
+Software
+Damen-Designerbekleidung
+Lodging
+Automobilia
+Magazines
+CDs
+Home Networking & Connectivity
+Vintage & Antique Jewelry
+Laptops
+Trading Cards/ CCG
+Armband- & Taschenuhren
+Damenmode
+VHS Tapes
+Kitchen, Dining & Bar
+Kindermode, Schuhe & Access.
+Herrenschuhe
+Education & Professional
+DVDs & Blu-rays
+Makeup
+Fan Apparel & Souvenirs
+Baby & Toddler Clothing
+Heimnetzwerk & Zubeh?r
+Packing & Shipping
+Bad & K?che
+Damenmode
+Piercings
+B?rotechnik
+CDs
+Bears
+??DVD
+Ladenausstattung & -werbung
+Kindermode, Schuhe & Access.
+Packing & Shipping
+PC- & Videospiele
+Motorr?der
+M?bel
+??/????
+Gsm-accessoires & Onderdelen
+Handys mit Vertrag
+Religious
+Fisch & Meeresfr?chte
+Loisirs Vie Pratique
+Bricolage
+Soins de sant?, sciences vie
+Damenmode
+Baby Clothing
+Werkzeuge & Werkstattbedarf
+Multipurpose Batteries & Power
+Malerei
+Objektive & Filter
+Cards
+Grues: logistiq., manutention
+Wedding
+Cards
+Women's Clothing
+Bad & K?che
+Holiday & Seasonal
+Handys ohne Vertrag
+Cycling
+Deutschland
+Dekoration
+Steiff
+Inline Skates
+CDs
+Auto-Ersatz- & -Reparaturteile
+Clothing: Men
+Parts & Accessories
+Accessoires t?l. mobiles, PDA
+Women's Clothing
+Bettwaren, -w?sche & Matratzen
+Piercing-/K?rperschmuck
+Weitere Ballsportarten
+Books & Manuscripts
+Model Kits
+TV-Receiver & Set-Top-Boxen
+Antikspielzeug
+Handys ohne Vertrag
+Camera & Photo Accessories
+Packing & Shipping
+Vinyles 33T
+Indian Coins
+Heavy Equipment
+Installation
+Sammleruhren
+Music Memorabilia
+V?tements & souvenirs de fan
+Studium & Wissen
+Cz??ci motocyklowe
+???
+Accessories
+Analoge Fotografie
+Unisex kleding
+Action Figures
+Camping & Outdoor
+Wedding Supplies
+Dekoration
+Video Games
+Outdoor Sports
+DVDs & Blu-rays
+Services & Real Estate
+Komunikacja i ??czno??
+Clothing & Accessories
+?????
+Outdoor Toys & Structures
+Edelsteine
+Asiatische Lebensmittel
+Herrenschuhe
+Cyclisme
+Advertising
+Objets de collection sports
+Clothing & Accessories
+Furniture
+Damenmode
+Classic Cars
+Dolls
+Echtschmuck
+Damenmode
+Bathroom
+Fournitures pour chien
+Handy- & PDA-Zubeh?r
+Antiquarische B?cher
+Restaurant & Catering
+Veranstaltungs- & DJ-Equipment
+Speakers, Webcams & Multimedia
+CD
+Damenmode
+Other Tickets, Travel
+Antiquarische B?cher
+Make Up e Cosmetici
+Logiciels
+TV- & Heim-Audio-Zubeh?r
+Motive
+Modeschmuck
+Skisport & Snowboarding
+Fan Apparel & Souvenirs
+Vins Alcools
+Men's Clothing
+Objektive & Filter
+Schule & Ausbildung
+Vintage Computing
+D?coration de maison
+Games
+Makeup
+Nonfiction
+Antiquarische B?cher
+Clothes, Shoes & Accessories
+Dolls' House Miniatures
+Video Game Accessories
+Jet-Ski, Acquascooter
+Drives, Storage & Blank Media
+Car Accessories & Parts
+Skisport & Snowboarding
+Cards
+Playmobil
+Fashion Jewelry
+Motocyclettes
+Restaurant & Catering
+Transportation
+Reklame & Werbung
+Nintendo DS
+Jewellery
+Radsport
+????
+Motorcycle Parts & Accessories
+Exercise & Fitness
+Handy- & PDA-Zubeh?r
+Urz. wskazuj?ce i wej?ciowe
+Fashion Jewelry
+Diecast & Vehicles
+Software
+Dancewear & Accessories
+Instruments d'?criture, plumes
+Sachb?cher & Ratgeber
+Vintage & Antique Jewelry
+Coins: US
+Sony
+CDs
+Mat?riel l?ger & outils
+Sewing (1930-Now)
+????
+Tasteninstrumente
+Kleinger?te Haushalt
+Electrical & Test Equipment
+B?rom?bel
+Studium & Wissen
+BD
+Greeting Cards & Party Supply
+Herrenschmuck
+Software
+??????
+Computer-Komponenten & -Teile
+Weitere Ballsportarten
+??
+Medizin & Labor
+Other
+Kids' Clothes, Shoes & Accs.
+Shaving & Hair Removal
+Baustoffe
+Art du XIX?me, et avant
+Pi?ces & accessoires
+Laufwerke & Speichermedien
+Damenmode
+iPods y reproductores MP3
+TV Memorabilia
+Motorrad- & Kraftradteile
+Rollenspiele & Tabletops
+????
+Veranstaltungs- & DJ-Equipment
+Software
+Fashion Jewellery
+Cardmaking & Scrapbooking
+Elektronik & Elektrotechnik
+Radio Communication
+????
+Fu?ball-Fanshop
+Kleidung, Schuhe & Accessoires
+Bielizna damska,st. k?pielowe
+Handys ohne Vertrag
+Loose Diamonds & Gemstones
+Cars
+Objets Collectors
+Photographie argentique
+Men's Accessories
+V?hicules miniatures
+Radsport
+Coins: US
+Fashion Jewelry
+Spielzeug f?r drau?en
+Poterie & porcelaine
+Men's Shoes
+Elektronik & Elektrotechnik
+Damen-Accessoires
+Non-Fiction
+MRO & Industrial Supply
+Kids' Clothing, Shoes & Accs
+Budownictwo
+Home Networking & Connectivity
+Ansichtkaarten en foto's
+Building Materials & Supplies
+MP3 Player Accessories
+Outdoor Sports
+Linge & textile (avant 1930)
+Cars & Trucks
+Papier, B?ro- & Schreibwaren
+Kinder- & Jugendliteratur
+Mobile Phone Accessories
+Bettwaren, -w?sche & Matratzen
+Gesichtspflege
+GPS & In-Car Technology
+Beads & Jewellery Making
+Transport
+Golf
+Playmobil
+Armband- & Taschenuhren
+Damenmode
+Fan Apparel & Souvenirs
+Fancy Dress & Period Costume
+Tools
+Trading Card Games
+T?l?phones mobiles
+Akcesoria dla artyst?w
+Sonstige Branchen & Produkte
+Teppiche & Teppichb?den
+Edelsteine
+Electrical & Test Equipment
+Tattoos & Body Art
+Sports de plein air
+Computer-Komponenten & -Teile
+Kindermode, Schuhe & Access.
+DVD - Films
+TV- & Heim-Audio-Zubeh?r
+Modelarstwo
+Echtschmuck
+Hobby & K?nstlerbedarf
+Rugs & Carpets
+Records
+Watches
+Armband- & Taschenuhren
+Car, Truck Parts
+Haarpflege
+Kleidung, Schuhe & Accessoires
+Handys ohne Vertrag
+???????
+Team Sports
+Cardmaking & Scrapbooking
+Damentaschen
+Fashion Jewellery
+Varie
+Echtschmuck
+Auto-Tuning & -Styling
+Kitchen, Dining, Bar
+Tapis & moquettes
+Music Memorabilia
+Ricambi Auto
+Handy- & PDA-Zubeh?r
+Computer Components & Parts
+Kleidung, Schuhe & Accessoires
+Comics
+Needlecrafts & Yarn
+Composants & pi?ces d'ordi
+Soins de sant?, sciences vie
+V?tements & souvenirs de fan
+Damenmode
+Kamera- & Fotozubeh?r
+??????/??
+Cycling
+Makeup
+Drucker, Scanner & Zubeh?r
+Ansichtskarten
+Non-Fiction
+Gewerbe
+Lighting & Studio
+Non-Fiction
+Sport
+Pauschalreisen
+Complementos Mujer
+Scrapbooking & Paper Crafts
+Revues, manuels, catalogues
+Mobiliar & Interieur
+Blitzger?te & Zubeh?r
+Kindermode, Schuhe & Access.
+Genealogy
+Militaria
+Kurzreisen
+Reise & Regionales
+Motorradkleidung
+Porcellana e Ceramica
+Soins de la peau/du visage
+Computer-Komponenten & -Teile
+Music Memorabilia
+?????
+Hand Tools & Equipment
+TV, Video & Home Audio
+Kleinger?te K?che
+Action Figures
+M?nchen & Region
+Damenmode
+Reit- & Fahrsport
+Animation
+Cables & Connectors
+Produktions- & Industriebedarf
+DVDs & Blu-rays
+Odzie? damska
+SCUBA & Snorkelling
+Audio portatifs & ?couteurs
+Kindermode, Schuhe & Access.
+K?rperpflege
+Kids' Clothing, Shoes & Accs
+Antikpuppen
+Odzie? damska
+Bootsport
+Damenmode
+Sewing
+Women's Accessories
+Refrigerators, Freezers
+Rollenspiele & Tabletops
+Elektronik & Elektrotechnik
+Comics
+Motorcycles
+Edelsteine
+Mu?ecas
+?????
+Wohnwagen & Wohnmobile
+Cross Stitch
+Cycling
+Volumi, Artbook e Riviste
+Rasenpflegeger?te
+Herrenmode
+Zubeh?r
+Women's Accessories
+Clothes, Shoes & Accessories
+Brass
+Electrical & Test Equipment
+Echtschmuck
+Massage
+Furniture
+Trading Cards/ CCG
+Dienstleistungen
+Handy- & PDA-Zubeh?r
+Modelleisenbahn Spur H0
+Cz??ci samochodowe
+Kleidung, Schuhe & Accessoires
+Telefony kom?rkowe
+Jewellery Storage & Cleaners
+Cat?gorie test
+Echtschmuck
+Tobakiana
+Date-Lined Ceramics
+????
+Landline Phones
+Komunikacja i ??czno??
+Home Arts & Crafts
+Furniture
+Decorative Arts
+Kleidung, Schuhe & Accessoires
+Basketball
+Animals
+Cz??ci samochodowe
+Software
+Zubeh?r
+C?bles & connecteurs
+TV, Video & Home Audio
+Geschenk- & Werbeartikel
+Meubels en wonen
+Games
+Mp3 & Mp4-spelare
+Zegarki nowych marek
+Cookware, Dining & Bar
+Reise & Regionales
+Malerei
+??/???
+Electrical & Test Equipment
+Weine
+Animals
+Oggetti sportivi
+Gadgets & Other Electronics
+Women's Clothing
+Damenmode
+DVDs & Blu-ray Discs
+Religion & Spirituality
+Handys ohne Vertrag
+????
+Computer-Komponenten & -Teile
+Telefony kom?rkowe
+Radsport
+Postcards
+Kamera- & Fotozubeh?r
+Clothing:GirlToddler
+Transportation
+Test Auctions
+Manufacturing & Metalworking
+Kleidung, Schuhe & Accessoires
+Pozosta?e zegarki na r?k?
+K?nstlerbedarf
+Auto-Tuning & -Styling
+Moto de collection: pi?ces
+Restaurant & service traiteur
+Chitarre e Bassi
+Cz??ci samochodowe: Tuning
+Home Surveillance
+Figuren, Poppetjes & Robots
+Men's Clothing
+Zubeh?r
+Kleidung, Schuhe & Accessoires
+TV- & Heim-Audio-Zubeh?r
+Beads
+Stationery
+Damenmode
+LEGO
+Grafik, Drucke
+DVD, Blu-ray & Heimkino
+Kochen & Genie?en
+Kleidung, Schuhe & Accessoires
+Video Game Consoles
+Cards
+Men's Clothing
+Adult Unisex
+Fu?ball
+Armband- & Taschenuhren
+Furniture
+Fine Jewellery
+Pierres pr?cieuses & Perles
+Damenmode
+Badminton
+Women's Clothing
+Cars & Trucks
+Europa
+Fragrances
+Wedding Supplies
+Kindermode, Schuhe & Access.
+Laufwerke & Speichermedien
+Handcrafted & Finished Pieces
+Brett- & Gesellschaftsspiele
+Latin America
+Software
+Slot Cars
+Men's Shoes
+Fan Apparel & Souvenirs
+Serviettes papier
+iPod- & MP3-Player-Zubeh?r
+Antikspielzeug
+Handy- & PDA-Zubeh?r
+Zubeh?r
+Antique Furniture
+Games
+Kochen & Genie?en
+Vintage Computing
+Drives, Storage & Blank Media
+CDs
+Sewing/ Fabric/ Textiles
+Cardmaking & Scrapbooking
+RC-Modellbau
+Teppiche & Teppichb?den
+Damenschuhe
+Staubsauger
+Vintage Clothing & Accessories
+Indoor Games
+Auto-Ersatz- & -Reparaturteile
+Tekstylia domowe
+Mobile Phones
+Kleidung, Schuhe & Accessoires
+Needlecrafts & Yarn
+Bettwaren, -w?sche & Matratzen
+Skanery
+Professional Broadcasting
+Echtschmuck
+Auto-Tuning & -Styling
+Cz??ci motocyklowe
+Sport
+Accessori cellulari e palmari
+M?bel
+Herrenmode
+M?bel
+DVDs & Blu-rays
+TV, Video & Home Audio
+Women's Clothing
+Fernseher
+Games
+Grafik, Drucke
+Golf
+Fu?ball-Fanshop
+Health Care
+Garten- & Gew?chsh?user
+Zubeh?r & Hardware
+DVDs & Blu-rays
+Home Improvement
+Budownictwo
+Objektive & Filter
+Logiciels
+Handy-Zubeh?r
+Impresoras y esc?neres
+Computer-Komponenten & -Teile
+Bielizna damska,st. k?pielowe
+Kleidung, Schuhe & Accessoires
+Echtschmuck
+Volkskunst
+Art. ko?cielne i dewocjonalia
+Digitalkameror
+Men's Jewelry
+Kindermode, Schuhe & Access.
+Unisex Clothing, Shoes & Accs
+DIY Tools
+Locaux: mat?riel, fournitures
+Hilfsmittel
+V?tements, accessoires vintage
+Antique Furniture
+Kindermode, Schuhe & Access.
+Damenmode
+Damenmode
+Software
+GPS & In-Car Technology
+Clothes, Shoes & Accessories
+Luggage & Travel Accessories
+Wand & Boden
+Fancy Dress & Period Costume
+Loisirs, vie pratique
+CDs
+Cards
+Bi?uteria z?ota
+Home Improvement
+Pi?ces - ?chelle mondiale
+Watches
+Weitere Uhren
+Habits Sp?cifiques
+Kinderkleding- jongens
+Color TVs
+Zigarren & Tabakwaren
+Auto-Ersatz- & -Reparaturteile
+Holzspielzeug
+Budownictwo i akcesoria
+Advertising
+Obuwie m?skie
+Football Shirts
+Auto-Ersatz- & -Reparaturteile
+Disneyana
+iPads, Tablets, eReaders
+Voeding
+Heimnetzwerk & Zubeh?r
+Woodcrafts
+Books & Manuscripts
+Vehicle Electronics & GPS
+Education & Professional
+Software
+Silver
+Appliances
+Hunting
+Home Improvement
+Decorative Collectibles
+Historische Wertpapiere
+Vintage & Antique Jewellery
+Geschenk- & Werbeartikel
+Welt der Tiere
+Haushalt
+DVDs & Blu-rays
+Meubles, d?coration, Xx?me
+Motorrad- & Kraftradteile
+Rasenm?her
+Golf
+Monitors, Projectors & Accs.
+Autre
+Handys mit Vertrag
+Music Memorabilia
+Fashion Jewellery
+Cell Phone Accessories
+??/??/??
+Reise & Regionales
+Basteln & Kreativit?t
+Auto-Ersatz- & -Reparaturteile
+Other
+Bath & Body
+Office
+Lecteur, stockage, sup. vierge
+Computer Components & Parts
+Donna: Scarpe
+Camera & Photo Accessories
+Medizin & Labor
+Accommodation
+Paper Money: World
+PC- & Videospiele
+Clothes, Shoes & Accessories
+Dekoration
+Decorative Collectibles
+Video Games
+Cz??ci samochodowe
+Photographie argentique
+Emballage & exp?dition
+Reclame- & Caf?voorwerpen
+Cartes de jeux de collection
+Video, Film
+Parts & Accessories
+Handys ohne Vertrag
+Automobilia
+Cell Phone Accessories
+Kaffee & Espressomaschinen
+Haushalt
+Damen-Accessoires
+Cars
+Camera & Photo Accessories
+Skin Care
+Gommoni
+Odzie? m?ska
+Hunting
+?bernachtungen
+Electrical & Test Equipment
+Prints
+Archery
+Comics
+Television Memorabilia
+Banknotes
+Grafik, Drucke
+??????
+Reise & Regionales
+Cell Phones & Smartphones
+Non-Fiction
+Damenbekleidung
+Auto-Tuning & -Styling
+Bottles, Jars, Jugs
+Kindermode, Schuhe & Access.
+Weitere Sammelgebiete
+Machinerie lourde
+
+Software
+Damenmode
+Ordinateur: composants, pi?ces
+Echtschmuck
+??
+Kleinger?te K?che
+Luxus-Accessoires
+Baby Books
+Folkloreschmuck
+Test Auctions
+V?tements pour femmes
+TV, Video & Home Audio
+Handcrafted, Artisan Jewelry
+Mobile Phones
+B?rotechnik
+Zubeh?r & Hardware
+Land
+Fournitures tout usage
+Drives, Storage & Blank Media
+Katzen
+Skisport & Snowboarding
+Modelleisenbahn Spur H0
+P??ki i rega?y
+Outdoor Sports
+Rhein-Neckar
+Software
+Men's Accessories
+Computer-Komponenten & -Teile
+Basteln & Kreativit?t
+TV, Video & Home Audio
+Ethnic, Regional & Tribal
+Baby & Toddler Clothing
+Model Trains
+DVD, Blu-ray & Heimkino
+Accessoires pour femmes
+Vintage & Antique Jewellery
+Schule & Ausbildung
+Bootonderdelen en -accessoires
+Militaria
+Cz??ci samochodowe
+Figuren
+Non-Fiction
+Perles, Fabrication de bijoux
+Reise & Regionales
+Vinilos
+Commonwealth/ British Colonial
+Music Memorabilia
+Horse Racing Memorabilia
+Sonnenschutz & Solarium
+Piercings
+Antiquities
+Back?fen & Herde
+Kindermode, Schuhe & Access.
+Trading Cards
+Kleinger?te K?che
+??????/??
+Skin Care
+Postcards
+R?seau domicile, connectivit?
+Percussion
+Accesorios para m?viles y PDAs
+Objets de Disney
+Camping, Hiking
+Cards
+Herren-Designerbekleidung
+Xbox 360
+Parts & Accessories
+Car, Truck Parts
+Sp?cialit?
+Records
+Cards
+Fitness
+Group Buy Listings
+Cartes de collection
+Cartes de collection
+Hunting
+Automobile
+Sachb?cher & Ratgeber
+Accessoires ordinateur
+Computer Components & Parts
+Personenauto's
+??????/??
+Echtschmuck
+Cars
+Fotografie
+Deutschland
+Audio Books
+Frankrijk
+World Coins
+Make-up
+????
+Woodwind
+Outdoor Sports
+Fragrances
+Veranstaltungs- & DJ-Equipment
+Olympic Memorabilia
+Loose Beads
+Tennis
+Zubeh?r
+Festivals & Konzerte
+Music Memorabilia
+V?tements & souvenirs de fan
+Tobacciana
+Men's Jewellery
+Handy- & PDA-Zubeh?r
+Glass
+Auto-Hi-Fi & Navigation
+Games
+Glass
+Advertising
+Lingerie & Strandkledij
+Versandmaterialien
+Cross Stitch
+Grafik, Drucke
+Diecast & Vehicles
+Nederland
+Home Arts & Crafts
+Scuba, Snorkeling
+Lampen & Licht
+Femmes: V?tements
+Abbigliamento Moto
+Echtschmuck
+Parts & Accessories
+Pocket Watches
+Auto-Ersatz- & -Reparaturteile
+Baby & Toddler Clothing
+Test Auctions
+Gastro & Nahrungsmittelgewerbe
+Yard, Garden & Outdoor Living
+?ducatifs
+Silver
+K?nstlerbedarf
+Major Appliances
+Cricket
+Damenmode
+Veranstaltungs- & DJ-Equipment
+Fu?ball
+Music Memorabilia
+Mobile Phones
+TV, Video & Home Audio
+GPS Accessories
+Romans, litt?rature
+Gro?handel & Sonderposten
+Yard, Garden & Outdoor Living
+Software
+Gadgets & Other Electronics
+Brett- & Gesellschaftsspiele
+Damenmode
+String
+Handys ohne Vertrag
+Festivals & Konzerte
+Asia
+Dekoration
+Dolls
+Hochzeit & Besondere Anl?sse
+Fabric/ Textiles
+Parts & Accessories
+Weine
+Handys ohne Vertrag
+Handys mit Vertrag
+Transport
+Cz??ci motocyklowe
+Action- & Spielfiguren
+?
+Kids' Clothing, Shoes & Accs
+Objektive & Filter
+Cz??ci samochodowe
+Skisport & Snowboarding
+Sewing & Fabric
+Cz??ci motocyklowe
+Mobiliar & Interieur
+Stamping & Embossing
+CDs
+Zubeh?r
+Katzen
+DVD - Spectacles musicaux
+Photographic Images
+Beads & Jewelry Making
+Revues, manuels, catalogues
+Breweriana
+Cars
+Deutschland
+TV, Movie & Character Toys
+DVD, Blu-ray & Heimkino
+Outdoor Sports
+Fu?ball
+Praktisch & Educatief
+Other
+Cell Phone Accessories
+Handy- & PDA-Zubeh?r
+Motorrad- & Kraftradteile
+Fashion Jewelry
+Art, Architecture & Design
+Yard, Garden & Outdoor Living
+Kurzreisen
+D?coration de maison
+Voiliers
+Men's Jewellery
+Papiergeld Deutschland
+Modeschmuck
+Profumi
+Computer Components & Parts
+Automobiles & camions
+UMDs
+Auto-Tuning & -Styling
+Motorrad- & Kraftradteile
+Kurzreisen
+Kurzreisen
+IT, Communication
+K?nstlerbedarf
+Laptop & Desktop Accessories
+K?rperpflege
+Parts & Accessories
+Cleaning, Housekeeping
+Biographies & Autobiographies
+Transport
+M?nzen International
+Video Games
+Home Decor
+Telefony kom?rkowe
+Kleidung, Schuhe & Accessoires
+Advertising
+Esoterik, Mystik & Magie
+PC's
+Handys mit Vertrag
+Parfums
+Bath
+Handy- & PDA-Zubeh?r
+Girl's Clothing
+Conexi?n de redes
+Handys ohne Vertrag
+Antikschmuck
+Dienstleistungen
+Europa
+Poterie & porcelaine
+Kleidung, Schuhe & Accessoires
+Modelleisenbahn
+????
+Porcelain/ China
+Back?fen & Herde
+Funktechnik
+Records
+Canada
+Car, Truck Parts
+Zeitschriften
+Glass
+Advertising
+Kindermode, Schuhe & Access.
+Auto-Ersatz- & -Reparaturteile
+AGD
+Jeux
+Metaphysical & New Age
+Militaria
+Kolekcje sportowe
+Haushalt & K?che
+Damenmode
+Bijouterie de qualit?
+Bettausstattung
+Laufwerke & Speichermedien
+???
+Odzie? damska
+Computer-Komponenten & -Teile
+Sicherheit
+Computer Components
+Lingerie
+Couture & tissus
+Mobile Phones
+Maquillage/cosm?tiques
+Motorrad- & Kraftradteile
+Sachb?cher & Ratgeber
+Damesschoenen
+???????
+Non-Fiction
+Damenmode
+Kindermode, Schuhe & Access.
+Bad- en strandmode
+Motorcycles
+Mobile Accessories
+Deutschland
+Music Memorabilia
+Heavy Equipment Parts & Accs
+Domotique
+Puzzles
+Handys ohne Vertrag
+Vehicle Electronics & GPS
+Fan Apparel & Souvenirs
+Mini Moto Ricambi
+Vintage & Antique Jewellery
+Festnetztelefone & Zubeh?r
+??
+Computer-Komponenten & -Teile
+Handys ohne Vertrag
+Herren-Accessoires
+Gadgets & Other Electronics
+V?gel
+D?tail & services
+Varie
+??/??/??
+Restaurant & Catering
+Bootonderdelen en -accessoires
+Baby & Toddler Clothing
+Cars
+Ansichtkaarten en foto's
+Parfums
+Kleinger?te K?che
+Puppen
+Computer-Komponenten & -Teile
+Cars
+Automobilia
+Vintage & Antique Jewelry
+Decorative Collectibles
+??????/??
+Outdoor Sports
+Herenhorloges
+Luggage Accessories
+Peripherals
+Fantasy
+Glass & Mosaics
+Mode Fille
+Other Sound & Vision
+Mobile Phones
+Zestawy odzie?y
+Cultures & Ethnicities
+Cz??ci samochodowe: Tuning
+Reisekoffer & -taschen
+Mode B?b? Mixte
+Vintage & Antique Jewellery
+Mat?riel param?dical
+GPS & In-Car Technology
+Bettwaren, -w?sche & Matratzen
+Damenmode
+??
+Transportation
+Studium & Wissen
+Banknotes
+Educational Toys
+Damenmode
+Paintings
+Livres Toutes Disciplines FR
+Auto-Ersatz- & -Reparaturteile
+Agrar, Forst & Kommune
+Stoneware
+Home Improvement
+Fiction
+Artist Jewelry
+Skin Care
+Art from Dealers & Resellers
+Serien & Lizenzprodukte
+Lighting, Fans
+Lecteur, stockage, sup. vierge
+Teleskope & Ferngl?ser
+Dolls
+Gutscheine
+V?tements pour hommes
+Laufwerke & Speichermedien
+Cultures & Ethnicities
+Fu?ball-Fanshop
+Security
+Music Memorabilia
+Software
+Computer Components & Parts
+Cuisine, restaurants & bar
+Spielzeugautos
+Other
+Animals
+Profumi
+Musik & Instrumente
+Echtschmuck
+Home Improvement
+R?seau d'entreprise, serveurs
+Games
+Modelleisenbahn
+Papier- en schrijfwaren
+Zubeh?r & Hardware
+Exercise & Fitness
+Home Arts & Crafts
+Papier, B?ro- & Schreibwaren
+Varie
+Glass
+Decorative Collectibles
+Damenmode
+Biographies
+Mobile Phones
+Fragrances
+Indian Coins
+Parfums
+Motorrad- & Kraftradteile
+Instrument Accessories
+Games
+Couture, Broderie, Tricot
+Bigiotteria
+Laptop Accessories
+Other
+Militaria
+Armband- & Taschenuhren
+Wholesale, Large & Small Lots
+Reise & Regionales
+Maternity/ Pregnancy
+Objektive & Filter
+C?bles & connecteurs
+Hardware
+Feste & Besondere Anl?sse
+TV, Movie & Character Toys
+Dameskleding
+Wholesale Lots
+Movie Memorabilia
+Sauna & Schwimmbad
+Damen-Accessoires
+Handy- & PDA-Zubeh?r
+Kunst & Kultur
+Gry
+Motorrad- & Kraftradteile
+%''%K%a!<%7%g%s
+??/?????
+Golf
+Electrical & Test Equipment
+Autoaccessoires en onderdelen
+Luxus-Accessoires
+Kinderm?bel & Wohnen
+Pro Audio Equipment
+Television Memorabilia
+Produktions- & Industriebedarf
+Babykleding - unisex
+Herrenmode
+Auto-Tuning & -Styling
+Damenmode
+Gewerbe
+Radio-Controlled
+Nail Care
+Laufwerke & Speichermedien
+CDs
+Women's Clothing
+Mobile Phones
+Kleidung, Schuhe & Accessoires
+Heim-Audio & HiFi
+Cards
+Fashion Jewellery
+Laptop & Desktop Accessories
+Animation Art & Characters
+Football Shirts
+P?yty kompaktowe
+Baby & Toddler Clothing
+Baby & Toddler Clothing
+Publicit?
+Damenmode
+Stage Lighting & Effects
+V?tements & souvenirs de fan
+Computer Components & Parts
+Pi?ces & access. de machinerie
+Rollenspiele & Tabletops
+Cartes de collection
+Automobile
+Computer Components & Parts
+Tickets & Experiences
+DIY Tools
+Dekoration
+Auto-Ersatz- & -Reparaturteile
+Ansichtskarten
+TV-Receiver & Set-Top-Boxen
+Music Memorabilia
+Damenmode
+Transportation
+Furniture
+Light Equipment & Tools
+Women's Clothing
+Holiday & Seasonal
+Maquettes
+Auto-Ersatz- & -Reparaturteile
+Computer Components & Parts
+Jeux de soci?t?
+Damenmode
+Vehicle Electronics & GPS
+Tools
+Yard, Garden & Outdoor Living
+Auto - Tuning & Styling
+Motocyclettes
+Women's Clothing
+Cars
+Damenmode
+Kitchen & Home
+Accessories
+Wholesale, Large & Small Lots
+Ride-On Toys
+Cartes de collection
+Heim-Audio & HiFi
+B?rotechnik
+Fische & Aquarien
+Composants: TV, son
+Education & Professional
+Weitere Sportarten
+Fine Jewellery
+Tijdschriften en kranten
+Fantasy/ Myth/ Magic
+iPods & MP3-Player
+Historische Wertpapiere
+Scolaire, dictionnaires
+Teich- & Bachlaufsysteme
+Cycling
+Music Memorabilia
+Fu?ball
+Guitar
+Armband- & Taschenuhren
+Restaurants
+Musicassette
+??????
+DVD, Blu-ray & Heimkino
+Pi?ces & accessoires
+Enterprise Networking, Servers
+Trading Cards/ CCG
+Enterprise Networking, Servers
+Armband- & Taschenuhren
+Women's Clothing
+Percussions
+Weitere Ballsportarten
+Historische Baustoffe
+Greeting Cards & Party Supply
+Pianos, Keyboards & Organs
+Women's Accessories
+Kleidung, Schuhe & Accessoires
+Echtschmuck
+Pottery
+?berraschungseier
+Science Fiction & Horror
+Bureau
+Uomo: Scarpe
+M?bel
+Pottery & China
+Reise & Regionales
+Handy- & PDA-Zubeh?r
+Kochen & Genie?en
+Wedding Supplies
+Angelsport
+Motorcycle Parts & Accessories
+Asian Antiques
+Hilfsmittel
+Loisirs, vie pratique
+Cromos
+Medikamente von Apothekern
+Motorrad- & Kraftradteile
+Cyclisme
+Kitchen, Dining & Bar
+Manga
+M?bel
+Flags
+Echtschmuck
+Metalworking/ Milling/ Welding
+Beleuchtung
+Revues, manuels, catalogues
+Guitar
+Auto-Ersatz- & -Reparaturteile
+Bath
+Famille des cuivres
+Souvenirs musicaux
+Commonwealth/ British Colonial
+Fische & Aquarien
+Porzellan & Keramik
+Decorative Arts
+Motorrad- & Kraftradteile
+Sicherheit
+Militaria
+TV, Video & Home Audio
+Damenmode
+Digitalkameras
+Handcrafted, Artisan Jewelry
+F?tes & saisonniers
+Women's Clothing
+????
+Crafts & Hobbies
+?bernachtungen
+Echtschmuck
+Rollenspiele & Tabletops
+Women's Clothing
+Wand & Boden
+Damenschuhe
+Needlecrafts & Yarn
+Fragrances
+??/??
+??CD
+Meubles
+Kleinger?te K?che
+Mat?riel ?lectrique & d'essai
+Vehicle Electronics & GPS
+Trading Cards
+Cars
+Cavi e connettori
+Computer-Komponenten & -Teile
+Healthcare, Lab & Life Science
+Sewing (1930-Now)
+Videojuegos
+B?rotechnik
+Software
+Streekromans
+Kurzreisen
+Bijoux de mode
+Motorcycles
+Sports d'hiver
+Weine
+Parts & Accessories
+FMCG
+Militaria
+Sports d'?quipe
+Kurzreisen
+Tourisme
+Srebra
+Digital Cameras
+Golf
+Information Products
+Shaving & Hair Removal
+Auto-Anbau- & -Zubeh?rteile
+Women's Clothing
+Science fiction & horreur
+MP3 Player Accessories
+Model Trains
+Cycling
+Obuwie m?skie
+Heimnetzwerk & Zubeh?r
+Home Furnishings
+Mobile Phone & PDA Accessories
+Quilting
+Arts Graphiques
+Dolls
+Computer-Komponenten & -Teile
+Sachb?cher & Ratgeber
+Litt?rature & Fiction NL
+Beads & Jewelry Making
+Computers
+Herrenmode
+Auto-Anbau- & -Zubeh?rteile
+??
+Plakate & Kunstdrucke
+Wand & Boden
+Parts & Accessories
+TV, Video & Home Audio
+Magazines
+Cz??ci samochodowe
+Kleidung, Schuhe & Accessoires
+A/V Accessories & Cables
+Damenmode
+Fragrances
+Philately/ Postal History
+M?nzen International
+Agrar, Forst & Kommune
+Cars
+Anlageobjekte
+Handys ohne Vertrag
+Hunting
+Outdoor Sports
+V?tements b?b?s, tout-petits
+Baby Bathing & Grooming
+Teleskope & Ferngl?ser
+Gitarren
+Imprimantes, scanneurs & acc.
+Cycling
+Veranstaltungs- & DJ-Equipment
+Comics
+Tragbare Disc-Player & Radios
+Dog Supplies
+Kost?me & Verkleidungen
+Auto-Ersatz- & -Reparaturteile
+????
+Damenmode
+Fragrances
+Laptop & Desktop Accessories
+Manufacturiers & m?tallurgie
+Motorcycle Parts & Accessories
+??/??
+Kinder- & Jugendliteratur
+Gitarren
+Lotti e Stock Libri
+Angelsport
+Skateboarding
+Vintage Movie & Photography
+DVD
+Kleidung, Schuhe & Accessoires
+Herrenschuhe
+Kindermode, Schuhe & Access.
+Trading Cards
+Tauchen
+Clothing, Helmets & Protection
+Motive
+Decorative Collectibles
+Alte Berufe
+Saisonales & Feste
+Mobile Phones
+Odzie? pozosta?a
+Cars
+Cars & Trucks
+Herrenmode
+Guitar
+Orologi da Polso: alla Moda
+Cards
+Werkzeuge
+Antikspielzeug
+Software
+Drucker, Scanner & Zubeh?r
+Vintage & Antique Jewelry
+Elektromaterial
+Kindermode, Schuhe & Access.
+Heimnetzwerk & Zubeh?r
+Militaria
+Caravanas y Remolques
+Kleinger?te Pflege & Wellness
+Verre, cristal
+T?l?phones mobiles
+Parfums
+Motorr?der
+Moviles
+Jeux
+Sporting Goods
+Baugewerbe
+Fitness & Gym
+????/??
+Fitness
+TV & Film Character Toys
+Hobby & K?nstlerbedarf
+Kurzreisen
+Auto-Tuning & -Styling
+Sicherheitstechnik
+Objets de collection de d?cora
+Skin Care
+GPS & In-Car Technology
+Kinderschoenen
+Gitarren
+Hair Care & Styling
+Digitale camera's
+Kindermode, Schuhe & Access.
+Games
+Fu?ball-Fanshop
+Beleuchtung
+Handys mit Vertrag
+Watches
+Modeschmuck
+Baugewerbe
+Makeup
+M?quinas recreativas
+Notebook- & Desktop-Zubeh?r
+Pi?ces & accessoires
+Fotografie & Fotokunst
+Audio portatifs & ?couteurs
+MRE, fournitures industrielles
+Printers, Scanners & Supplies
+Bijoux d'?poque, anciens
+Writing Instruments
+Baugewerbe
+Lernspielzeug
+Claviers, souris & pointeurs
+Zubeh?r
+Lampes, plafonniers-?ventails
+Damenmode
+Kleidung, Schuhe & Accessoires
+Fiction Books
+Baby & Toddler Clothing
+Cars & Trucks
+Accessoires de cellulaires
+Car Seats & Accessories
+Reiseaccessoires
+Outdoor Sports
+Motorrad- & Kraftradteile
+Robots, Automates
+Wholesale Lots
+Boy's Clothing
+Soins de sant?, sciences vie
+Vehicle Electronics & GPS
+V?tements b?b?s, tout-petits
+Women's Clothing
+Cycling
+Advertising
+Druckerei & Copyshop
+Herrenschuhe
+Damenmode
+Feeding
+Cars & Trucks
+Camping & Outdoor
+Laptop & Desktop Accessories
+Restaurant & Catering
+Fashion Jewellery
+Laufwerke & Speichermedien
+Laufwerke & Speichermedien
+??/??
+TV, Video & Home Audio
+Animals
+Oferty hurtowe i wyprzeda?e
+Meubles
+Men's Clothing
+Glass
+Computer-Klassiker
+Games
+Cartes de jeux de collection
+Kindermode, Schuhe & Access.
+CDs
+Toys & Activities
+Damenmode
+Kochen & Genie?en
+Skisport & Snowboarding
+Souvenirs militaires
+??GSM??
+Test Auctions
+Echtschmuck
+Wholesale Lots
+Auto-Ersatz- & -Reparaturteile
+Budownictwo
+Hard Disk Drives
+VHS-video?s
+Pi?ces d?tach?es
+????
+Plants/ Seeds/ Bulbs
+Science & Medicine (Pre-1930)
+Sun Protection & Tanning
+????
+Audio Books
+Damenmode
+Nonfiction
+Sonstige Branchen & Produkte
+Damenmode
+Damenmode
+Pens & Writing Equipment
+AGD
+??
+Antikspielzeug
+Agriculture
+Advertising
+Dierenvoeding
+Design & Stil
+Trains/ Railway Models
+Cat Supplies
+Holzbearbeitung & Tischlerei
+Videojuegos
+Gastronomie & F?tes
+Antyki, sztuka mi?dzynarodowa
+Gastro & Nahrungsmittelgewerbe
+Militaria
+??????
+Action- & Spielfiguren
+Kamera- & Fotozubeh?r
+??CD
+Auto-Ersatz- & -Reparaturteile
+PC- & Videospiele
+Music Memorabilia
+Heim-Audio & HiFi
+KFZ-Handwerk
+Parfums
+Accesorios c?maras/fotograf?a
+Poppetjes en figuurtjes
+Middle East
+Gesichtspflege
+Festivals & Konzerte
+Movie Memorabilia
+Ponds & Water Features
+Car Manuals & Literature
+Cz??ci samochodowe
+Cycling
+Kids' Clothing, Shoes & Accs
+Computer-Komponenten & -Teile
+Fournitures pour chien
+Produktions- & Industriebedarf
+Boats Parts & Accessories
+Damenmode
+Memorabilia
+Diecast & Vehicles
+Geschirrsp?ler
+Fan Apparel & Souvenirs
+Acciones y Papeles
+Diamond Jewellery
+Skisport & Snowboarding
+Herrenschuhe
+Football Shirts
+Parts & Accessories
+Bath & Body
+Software
+Bricolage
+Motorrad- & Kraftradteile
+Grafik, Drucke
+Men's Wrist Watches
+Festnetztelefone & Zubeh?r
+Cards
+Dischi Vinile 33 giri - 12''
+B?rotechnik
+Pens & Writing Equipment
+Cigarette/ Tea/ Gum Cards
+Pflanzen, B?ume & Str?ucher
+Scrapbooking
+T?
+Hochzeit & Besondere Anl?sse
+Loose Diamonds & Gemstones
+Damentaschen
+Videocassette e VHS
+Damenmode
+Festivals & Konzerte
+Yard, Garden & Outdoor Living
+Children & Teens
+Guitars
+TV- & Heim-Audio-Zubeh?r
+Video Games
+Damenmode
+Movie Memorabilia
+Computer Components & Parts
+Handy-Attrappen
+Loose Diamonds & Gemstones
+Armband- & Taschenuhren
+Instruments
+Enterprise Networking, Servers
+Education & Professional
+Trading Cards
+Tee
+Hair Removal & Shaving
+Computer-Komponenten & -Teile
+Perfumy
+Auto-Anbau- & -Zubeh?rteile
+Light Equipment & Tools
+TV-Receiver & Set-Top-Boxen
+Autographs-Original
+Herrenmode
+Veranstaltungs- & DJ-Equipment
+Figurines, Statues
+TV- & Heim-Audio-Zubeh?r
+Ricambi Auto
+Football Memorabilia
+Guitare
+Animation Art & Characters
+Homeware, Kitchenware
+B?rotechnik
+Livres de bandes dessin?es
+Motor Sport Memorabilia
+Coins: US
+Cz??ci motocyklowe
+Cat Supplies
+Maps, Atlases & Globes
+Makeup
+Women's Clothing
+Casino
+Pi?ces France
+Fan Apparel & Souvenirs
+Livres de b?b?s & faire-part
+Computer-Komponenten & -Teile
+Reise & Regionales
+Ansichtskarten
+M?nzen Europa
+Cuisine: Arts de la table
+Woodwind
+Emballage & exp?dition
+K?nstlerbedarf
+Tragbare Disc-Player & Radios
+???????
+Notebook- & Desktop-Zubeh?r
+Versandmaterialien
+MRO & Industrial Supply
+Cultures & Ethnicities
+Fan Apparel & Souvenirs
+Fanartikelen
+Handy- & PDA-Zubeh?r
+Car Seats, Accessories
+Women's Clothing
+Chaussures de femme
+Deodorants
+?????/??
+Zubeh?r & Hardware
+Odzie? m?ska
+Clothing, Merchandise, Media
+M?bel
+Dollhouse Miniatures
+Parfum, eau en after shave
+Damenmode
+T?l?, vid?o & audio domestique
+Kindermode, Schuhe & Access.
+DVDs & Blu-rays
+Kleinger?te Pflege & Wellness
+Auto-Hi-Fi & Navigation
+Cars
+Monitore, Projektoren & Zub.
+Hobby & K?nstlerbedarf
+Casino
+Sammleruhren
+Cards
+Pens & Writing Equipment
+Ruilkaartspellen
+H?rb?cher & H?rspiele
+Babyspeelgoed
+Kurzreisen
+Electrical
+Piercing-/K?rperschmuck
+Radio, Gramophone, Phone
+Action figures e statiche
+Diecast & Vehicles
+Elektromaterial
+Maquettes
+Echtschmuck
+Pottery
+Kindermode, Schuhe & Access.
+Fan Apparel & Souvenirs
+Vintage
+Parts & Accessories
+Porte-clefs
+Cartes postales
+Photocopier Accessories, Parts
+Art?culos de Escritorio
+Drives, Storage & Blank Media
+Polizei & Beh?rden
+Nutrition
+Souvenirs & Travel Memorabilia
+Nautika & Maritimes
+Tabak, Feuerzeuge & Pfeifen
+Metallbearbeitung & Schlosser.
+Heavy Equipment Parts & Accs
+Kitchen, Dining & Bar
+Handys mit Vertrag
+Mat?riel ?lectrique & d'essai
+Sonstige
+Winter Sports
+Kleidung, Schuhe & Accessoires
+Welt der Tiere
+Kleidung, Schuhe & Accessoires
+Analoge Fotografie
+Num?riques - Accessoires
+Tools
+Motorrad- & Kraftradteile
+Zubeh?r
+Ricambi Auto d'Epoca
+Baustoffe
+Papier, B?ro- & Schreibwaren
+Armband- & Taschenuhren
+Parts & Accessories
+Drives, Storage & Blank Media
+Fishing
+Relojes de Pulsera
+Autoaccessoires en onderdelen
+Meble designerskie
+Movie Memorabilia
+Memorabilia
+Analoge Fotografie
+Klassische Konsolen
+Film Photography
+Modelleisenbahn Spur H0
+Parts & Accessories
+Tv & Homecinema
+Fine Jewellery
+Preschool Toys & Pretend Play
+TV- & Heim-Audio-Zubeh?r
+Weitere Sportarten
+Bettausstattung
+??
+Auto-Ersatz- & -Reparaturteile
+Handys ohne Vertrag
+Jogging
+Auto-Ersatz- & -Reparaturteile
+Gitarren
+Damenmode
+Mode Homme
+Vintage & Antique Jewellery
+Records
+Desktops & All-In-Ones
+Damenmode
+CD's
+Swimming
+Przemys? tekstylny i pokrewne
+TV, Video & Home Audio
+Pi?ces & accessoires
+Memorabilia
+Baby & Toddler Clothing
+Kinderfahrzeuge
+Notebook- & Desktop-Zubeh?r
+Car, Truck Parts
+Auto-Tuning & -Styling
+Jardin, Ext?rieur
+Accessoires de cellulaires
+Audio portatifs & ?couteurs
+Internationale Antiq. & Kunst
+Printers, Scanners & Supplies
+Notebook- & Desktop-Zubeh?r
+Vinyles
+Jeux
+Guitar
+Auto-Ersatz- & -Reparaturteile
+Papiergeld Welt
+Fantasy, Mythical & Magic
+Fu?ball
+Fournitures tout usage
+Telefony kom?rkowe
+Non-Fiction
+Tastaturen, M?use & Pointing
+Wholesale Lots
+Reise & Regionales
+Klussen
+Auto-Ersatz- & -Reparaturteile
+Modelleisenbahn
+Polizei & Beh?rden
+TV- & Heim-Audio-Zubeh?r
+Pi?ces - Canada
+Maternity Clothing
+Postcards
+Dolls
+Verre
+Computer-Komponenten & -Teile
+Tickets & Toegangskaarten
+Stamps
+Damenmode
+Food & Beverages
+Monitore, Projektoren & Zub.
+iPods & MP3-Player
+Cars & Trucks
+Fan Apparel & Souvenirs
+Fashion Jewellery
+Klimaanlagen & Heizger?te
+Golf
+Weitere Ballsportarten
+Diecast & Vehicles
+Music Memorabilia
+Wholesale Lots
+Travaux de r?novation
+????
+Silver
+Video Games
+Fragrances
+Kabel & Steckverbinder
+Desktops & All-in-One-PCs
+Digitalkameras
+Gro?handel & Sonderposten
+Mobile Phones
+Livres Toutes Disciplines NL
+Kids' Clothes, Shoes & Accs.
+Kochen & Genie?en
+Elektrisches Spielzeug
+Lamps, Lighting & Ceiling Fans
+Kochen & Genie?en
+Sewing & Fabric
+??
+Kurzreisen
+Antykwariat
+Birofilistyka
+Other Formats
+Homeware, Kitchenware
+Femmes: V?tements
+Motorrad- & Kraftradteile
+Deutschland
+Parts & Accessories
+Art du Monde
+Haushalt
+Beads
+Shaving & Hair Removal
+Coches: repuestos
+H?rb?cher & H?rspiele
+Glass
+Cell Phone Accessories
+Coffee Makers (Automatic)
+Books & Manuscripts
+Kitchenalia
+Non-Fiction
+Weitere Sportarten
+Holiday & Seasonal
+Test Auctions
+Articoli per radioamatori
+Herrenschuhe
+Inlineskating
+iPod- & MP3-Player-Zubeh?r
+Fitness
+Fan Apparel & Souvenirs
+Personnages de t?l? & de films
+Manufacturing & Metalworking
+Cards
+Equip. y Material Oficinas
+Zubeh?r
+Kleinger?te Haushalt
+Needlecrafts & Yarn
+Trading Cards
+Kindermode, Schuhe & Access.
+Parts & Accessories
+Clothing, Helmets & Protection
+Home Furnishings
+Accessories
+Industrial Tools
+B?b?-Fille
+Equestrian
+Deutschland
+Bi?uteria do piercingu
+Baby Health & Safety
+Lotes de Cine, DVD y Pel?culas
+Kids' Clothes, Shoes & Accs.
+Kindermode, Schuhe & Access.
+Retail & Services
+Utensili da modellismo
+Restaurant & service traiteur
+Computer-Komponenten & -Teile
+V?tements & souvenirs de fan
+Damenschuhe
+Produktions- & Industriebedarf
+Home Furniture
+Historical Memorabilia
+Vacances
+Decorative Collectibles
+Cars & Trucks
+Kleidung, Schuhe & Accessoires
+Analoge Fotografie
+Cards
+Sheet Music & Song Books
+Auto-Ersatz- & -Reparaturteile
+???????
+Polskie banknoty
+Other Crafts
+Cz??ci samochodowe
+Loose Diamonds
+Herren-Designer-Accessoires
+Autographs
+????
+Romanhefte
+Cz??ci samochodowe: Tuning
+Auto-Ersatz- & -Reparaturteile
+Auto-Ersatz- & -Reparaturteile
+Software
+Health Care
+Manufacturing & Metalworking
+Action Figures
+Souvenirs musicaux
+Autres Nintendo
+Model Railroads & Trains
+Pi?ces Euro
+Magazine Back Issues
+Motorcycle Parts & Accessories
+Articles religieux
+Woodwind
+Verschonen & Verzorgen
+Kindermode, Schuhe & Access.
+Radsport
+Cz??ci samochodowe
+Baby & Toddler Clothing
+?lectronique, pile, remontoire
+Cycling
+V?tements pour enfants
+Snooker & Pool
+Sony Consoles
+Echtschmuck
+Tuin & Terras
+Electrical & Test Equipment
+TV series en programma's DVD's
+Comics
+Body Jewellery
+Bad & K?che
+Elettronica ed Elettricit?
+Baby Clothing
+?????/??
+Baby & Toddler Clothing
+??/??
+Cars
+Mat?riel de radiocommunication
+Music
+Radio Communication Equipment
+Back?fen & Herde
+Diapering
+Science & Medicine (Pre-1930)
+Kurzreisen
+Auto-Ersatz- & -Reparaturteile
+Equitation
+Kurzreisen
+Radsport
+Homeware, Kitchenware
+Bullion
+Gardening
+V?tements pour femmes
+CD
+Animation
+Studium & Wissen
+Radio Communication Equipment
+Malerei
+Software
+Cartes de collection
+Music Memorabilia
+Healthcare, Lab & Life Science
+Furniture
+Sicherheitstechnik
+Sport
+Auto-Anbau- & -Zubeh?rteile
+Digitalkameras
+Asian Antiques
+Auto-Kindersitze & Zubeh?r
+Handys mit Vertrag
+Technik & Ger?te
+iPods & MP3-Player
+Appliances
+Echtschmuck
+Auto-Anbau- & -Zubeh?rteile
+Damenmode
+Gesichtspflege
+Auto-Ersatz- & -Reparaturteile
+Fan Apparel & Souvenirs
+Handys ohne Vertrag
+Bistrot
+Het aanzien van...
+Natural & Alternative Remedies
+International
+Garten- & Gew?chsh?user
+Non-Fiction
+Trading Cards
+R?seau d'entreprise, serveurs
+Beanbag Plush
+Computer-Komponenten & -Teile
+V?tements & souvenirs de fan
+Kfz-Services & -Reparaturen
+????
+Klein- & Nagetiere
+VHS Tapes
+Damenmode
+Cellulari e smartphone
+Philately/ Postal History
+Packing & Posting Supplies
+Feste & Besondere Anl?sse
+Damenmode
+Herrenschuhe
+Cars
+Asian Art
+????
+Pi?ces & accessoires
+DVD, Blu-ray & Heimkino
+Scrapbooking & Paper Crafts
+Kindermode, Schuhe & Access.
+Advertising
+R?seau d'entreprise, serveurs
+Equipment
+Damenmode
+Televisori e Proiettori
+Dessins anim?s & personnages
+Computer-Komponenten & -Teile
+Damenmode
+Auto - Tuning & Styling
+Fashion Jewellery
+Journaux, revues, magazines
+Printers, Scanners & Supplies
+Papiergeld Deutschland
+Stroje ?lubne
+Console
+Telefony kom?rkowe
+Damenschuhe
+??????????????????
+Comics
+Software
+Non-Fiction Books
+Claviers, souris & pointeurs
+Coins
+Auto-Ersatz- & -Reparaturteile
+Installation
+Handy- & PDA-Zubeh?r
+Women's Clothing
+Fan Apparel & Souvenirs
+Mobile Phones
+Kochen & Genie?en
+M?nzen ?sterreich
+Souvenirs & Travel Memorabilia
+Manufacturing & Metalworking
+Skisport & Snowboarding
+Vintage & Antique Jewellery
+Building Materials, DIY
+Perfumes & Fragrances
+Medizin & Labor
+Dolls
+Souvenirs ?missions t?l?vis?es
+Bettwaren, -w?sche & Matratzen
+Motorrad- & Kraftradteile
+Food & Beverages
+Cuisine, restaurants & bar
+Golf
+Bureau
+Gemstone Jewellery
+Comics
+Medizin & Labor
+Zubeh?r
+Exercise & Fitness
+Golf Memorabilia
+Reise & Regionales
+Akcesoria dla artyst?w
+DVDs & Blu-rays
+Kleidung, Schuhe & Accessoires
+Baby Gear
+Frankrijk
+Kochen & Genie?en
+Zubeh?r
+Pi?ces D?tach?es
+Handys ohne Vertrag
+Kids' Clothing, Shoes & Accs
+Glass & Mosaics
+Rugby Union
+Cardmaking & Scrapbooking
+Sciences, culture
+Heimnetzwerk & Zubeh?r
+Percussion
+Kids' Clothing, Shoes & Accs
+Theater Memorabilia
+T?l?phones mobiles
+Skin Care
+Fragrances
+Gry
+Nose Pins
+Animals
+Firmennetzwerk, Server
+Fabriqu?s ? la main & finis
+Standmodelle
+Handys mit Vertrag
+Volkskunst
+Cultures & ?thnicit?s
+Advertising
+Handys ohne Vertrag
+Technik & Ger?te
+Babykleding - meisjes
+Fabric
+T?l?phones ? domicile
+Papiergeld Deutschland
+Home Arts & Crafts
+Laufwerke & Speichermedien
+????
+Revues, Manuels & Catalogues
+Chasse
+Radio & Lecteurs CD Portables
+Personnages de t?l? & de films
+Malerei
+Gewerbe
+Pi?ces & accessoires
+Music Memorabilia
+Baugewerbe
+Scalextric & Slot Car
+Decorative Ornaments/ Plates
+Auto-Ersatz- & -Reparaturteile
+Appliances
+Cars
+Telefony kom?rkowe
+Herrenmode
+Sonstige
+SUVs
+Construction Toys & Kits
+????
+Kids' Clothing, Shoes & Accs
+Modelleisenbahn Spur H0
+Clocks
+Motorcycle Parts, Accessories
+Militaria
+W?rzen & verfeinern
+BD
+Laufwerke & Speichermedien
+Auto-Tuning & -Styling
+Cycling
+Disneyana
+Records
+Fashion Jewelry
+Echtschmuck
+Auto-Tuning & -Styling
+Fragrances
+Auto-Ersatz- & -Reparaturteile
+Kitchen & Home
+Flakons & Seifen
+Stromschutz & Stromverteilung
+Dispositivi archiviazione dati
+Religion & Spirituality
+????/??
+Women's Clothing
+Men's Clothing
+Test Auctions
+Metallobjekte
+Wedding & Formal Occasion
+Dekoration
+Handys mit Vertrag
+Computer Components & Parts
+Auto-Motorsport
+CD
+Budownictwo
+B?rotechnik
+Kurzreisen
+Kurzreisen
+Meubles de pouponni?re
+FMCG
+Asian Antiques
+Software
+Objets du XX?me, r?cents
+Cars & Trucks
+Computer Components & Parts
+Sport
+Speelautomaten
+Radio Equipment
+Mat?riel ?lectrique & d'essai
+K?rperpflege
+Fernseher
+Teich- & Bachlaufsysteme
+Womens' Handbags & Bags
+Echtschmuck
+Vinyl
+Comics
+PC- & Videospiele
+Gadgets
+Pet Supplies
+Kochen & Genie?en
+Women's Clothing
+Performance & DJ Equipment
+Sonstige
+Literie
+Handys ohne Vertrag
+Automobile
+Kaffee & Espressomaschinen
+Cz??ci samochodowe
+iPod- & MP3-Player-Zubeh?r
+Heim-Audio & HiFi
+Camping & Outdoor
+Handy- & PDA-Zubeh?r
+Skisport & Snowboarding
+France
+Bears
+Marcofilia
+Reise & Regionales
+Augenoptik
+Sauna & Schwimmbad
+Pottery & China
+Pottery, Porcelain
+Car Parts
+Fan Apparel & Souvenirs
+Handy- & PDA-Zubeh?r
+Mobiliar & Interieur
+Bad & K?che
+Kitchen, Dining, Bar
+DVDs & Blu-ray Discs
+Worldwide
+Skisport & Snowboarding
+BD
+Heavy Equipment Parts & Accs
+Cellulari e smartphone
+Modelleisenbahn
+Sports
+Fashion & Imitation Jewellery
+Outils
+Souvenirs musicaux
+Galanteria damska i dodatki
+Cars
+Body Piercing & Jewellery
+Elektronik & Elektrotechnik
+Cars
+Cables & Connectors
+Handys ohne Vertrag
+Sachb?cher & Ratgeber
+Veranstaltungs- & DJ-Equipment
+Tobacciana
+Auto-Hi-Fi & Navigation
+Men's Accessories
+Kids' Clothing, Shoes & Accs
+S?curit?, Domotique
+Automobile
+Computer-Komponenten & -Teile
+Celebrations & Occasions
+Herren-Accessoires
+Skisport & Snowboarding
+????
+Kochen & Genie?en
+Art, Architecture & Design
+Holzbearbeitung & Tischlerei
+Sporting Goods
+Children & Young Adults
+Silver
+Housekeeping & Organization
+B?rom?bel
+?vrigt
+Fashion Jewelry
+Speakers, Webcams & Multimedia
+Auto Epoca e Moderne
+Gastro & Nahrungsmittelgewerbe
+Healthcare, Lab & Life Science
+Beads
+Tilicartes
+MP3 Players
+Comic Books
+Parfum
+Kamera- & Fotozubeh?r
+????
+Fan Apparel & Souvenirs
+Laptop & Desktop Accessories
+Armband- & Taschenuhren
+Trading Card Games
+Mobile Phones
+Men's Jewellery
+Cards
+Parfums
+Automobile
+Verre & mosa?ques
+Weed/ Pest Control
+Jeux vid?o
+Handys ohne Vertrag
+Cz??ci samochodowe
+Srebra
+Parts & Accessories
+Hunde
+Nonfiction
+Dekoration
+TV, Video & Home Audio
+Maquillage/cosm?tiques
+V?tements pour enfants
+Fashion Jewellery
+Golf
+DVDs & Blu-rays
+Light Equipment & Tools
+Cell Phone Accessories
+Fernseher
+MRO & Industrial Supply
+Mobile Phones
+Urmakeriverktyg
+Yard, Garden & Outdoor Living
+Bijoux d'?poque, anciens
+Women's Accessories
+Test Auctions
+Vehicle Electronics & GPS
+Studium & Wissen
+Vintage Clothing & Accessories
+Cookware, Dining & Bar
+Men's Accessories
+Reit- & Fahrsport
+Women's Accessories
+Motorradkleidung
+Muziek
+Szk?o i kryszta?y
+Makeup
+Vinyl
+Serien & Lizenzprodukte
+Golf
+Fiction
+Vintage & Antique Jewellery
+Kleidung, Schuhe & Accessoires
+Reit- & Fahrsport
+Handy- & PDA-Zubeh?r
+Agrar, Forst & Kommune
+Computer Components & Parts
+Golf
+Mobile Phones
+Art. do przechowywania
+Travaux de r?novation
+TV- & Heim-Audio-Zubeh?r
+Weitere Uhren
+S??es & Salziges
+Gardening
+Plants/ Seeds/ Bulbs
+Loose Gemstones
+Fashion Jewellery
+Werkzeug
+Engagement & Wedding
+Kurzreisen
+Lots en gros
+Water Sports
+Paper Money: World
+Drives, Storage & Blank Media
+PC-Spiele
+Furniture
+DVDs & Blu-ray Discs
+Heim-Audio & HiFi
+Porzellan & Keramik
+Light Equipment & Tools
+Fiction
+Damenschuhe
+Echtschmuck
+B?rom?bel
+St?mplar
+Herrenschuhe
+Cultures & Ethnicities
+Damenmode
+Armband- & Taschenuhren
+Dekoration
+Handy- & PDA-Zubeh?r
+Parfum, eau en after shave
+Kochen & Genie?en
+G?n?alogie
+Internationale Antiq. & Kunst
+Pubblicitario
+??/??
+Fanartikel & Merchandise
+Fishing
+Stripboeken
+Test Auctions
+Other Crafts
+Kabel & Steckverbinder
+Cables & Connectors
+Golf
+Soins du corps
+Music Memorabilia
+????
+AGD
+Gitarren
+Cars
+Medizin & Labor
+Bath
+Werkzeuge & Werkstattbedarf
+Tools
+?????????
+Makeup
+Souvenirs & Travel Memorabilia
+Clothing, Helmets & Protection
+Glass
+Sports de Balle & Ballon
+Fabriqu?s ? la main & finis
+Produktions- & Industriebedarf
+Vintage
+Computer-Komponenten & -Teile
+Gastro & Nahrungsmittelgewerbe
+Software
+Souvenirs historiques
+Cucina
+Test Auctions
+Metallbearbeitung & Schlosser.
+?????
+Plants/ Seeds/ Bulbs
+VHS-video?s
+Anlageobjekte/Versteigerungen
+V?tements pour enfants
+Vinyl
+Stampanti, scanner e forniture
+Notebook- & Desktop-Zubeh?r
+Apple iPod Accessories
+Celebrations & Occasions
+Produktions- & Industriebedarf
+Kindermode, Schuhe & Access.
+Porcellana e Ceramica
+Makeup
+Obr?bka metali i ?lusarstwo
+Reklame & Werbung
+Action Figures
+Papier, B?ro- & Schreibwaren
+Wholesale, Bulk Lots
+Kleidung, Schuhe & Accessoires
+Kindermode, Schuhe & Access.
+Damenmode
+Echtschmuck
+Bedding
+V?tements & souvenirs de fan
+Weitere Wintersportarten
+Papier, B?ro- & Schreibwaren
+Modeschmuck
+Notebook- & Desktop-Zubeh?r
+Druckerei & Copyshop
+Teich- & Bachlaufsysteme
+Fragrances
+Teenskleidung M?dchen
+Coins: US
+Odzie? m?ska
+Antiquarische B?cher
+Baugewerbe
+Herrenmode
+Kindermode, Schuhe & Access.
+Thematics
+Skisport & Snowboarding
+Automobilia
+Perles, Fabrication de bijoux
+Sports de plein air
+Men's Shoes
+Wedding Supplies
+Bureau
+Handys ohne Vertrag
+Cellulari e smartphone
+Weitere Sammelgebiete
+Elektronika i elektrotechnika
+Motive
+Digital Cameras
+??????
+Clothing, Merchandise, Media
+Other Baby & Mom
+Reit- & Fahrsport
+Tragbare Disc-Player & Radios
+Modelleisenbahn Spur H0
+Card Games
+Golf
+Wholesale Lots
+?????
+Zubeh?r
+Voetbal
+Scooters
+Glass
+Music Memorabilia
+Animals
+Central & South America
+Cables & Connectors
+Papier, B?ro- & Schreibwaren
+Reit- & Fahrsport
+Software
+Camera & Photo Accessories
+Fashion Jewelry
+Kids' Clothes, Shoes & Accs.
+Herren-Accessoires
+Loose Diamonds & Gemstones
+Technik & Ger?te
+B?b?-Gar?on
+Bi?uteria z pere?
+Medizin & Labor
+K?nstlerbedarf
+??
+Car, Truck Parts
+Baby & Toddler Clothing
+Parts & Accessories
+Cars & Trucks
+Nail Care, Manicure & Pedicure
+Kindermode, Schuhe & Access.
+Notebook- & Desktop-Zubeh?r
+Heim-Audio & HiFi
+63517
+Komputery i cz??ci Apple
+Fische & Aquarien
+Englischsprachige B?cher
+Reise & Regionales
+Cz??ci samochodowe
+Kaffee & Espressomaschinen
+Heavy Equipment
+DVDs & Blu-rays
+Romanhefte
+Regional Movies
+Radsport
+Damenmode
+B?rotechnik
+Spezielle Fahrzeug-Teile
+Modelarstwo
+Boat Parts, Accessories
+Bath
+Feste & Besondere Anl?sse
+DVDs & Blu-rays
+Car Wheels, Tyres & Trims
+Hard Disk Drives
+Tuinieren en klussen
+Kleidung, Schuhe & Accessoires
+Auto-Anbau- & -Zubeh?rteile
+Pet Memorials & Urns
+C?bles & connecteurs
+Men's Clothing
+Militaria
+Vintage
+Hair Removal & Shaving
+Software
+Middle East
+Drucker, Scanner & Zubeh?r
+Geschenk- & Werbeartikel
+Fan Apparel & Souvenirs
+Armband- & Taschenuhren
+Hunde
+Dekoracje
+Cables & Connectors
+Plakate & Kunstdrucke
+Guitar
+Relojes de Pulsera
+Clothing, Helmets & Protection
+Audio
+Armband- & Taschenuhren
+Internationale Antiq. & Kunst
+PC- & Videospiele
+Heimnetzwerk & Zubeh?r
+Kleidung, Schuhe & Accessoires
+Herren-Designerbekleidung
+??????/??
+Winter Sports
+Fische & Aquarien
+Equipos de DJ y espect?culos
+Cookware, Dining & Bar
+Clothing, Merchandise, Media
+Jewel Watches
+BD
+Turismos
+Spielzeug
+Damenmode
+Livres de bandes dessin?es
+Trading Cards
+iPods & MP3-Player
+Antiquarische B?cher
+Jouet pr?scolaire, jeu de r?le
+Health Care
+Non-Fiction
+Motorrad- & Kraftradteile
+UK (Great Britain)
+Postcards
+Kleidung, Schuhe & Accessoires
+Zegarki designerskie na r?k?
+Fashion Jewelry
+Vinyles 33T
+Auto-Ersatz- & -Reparaturteile
+Jeux vid?o
+Damenmode
+Unknown
+Radio Communication
+Action Figures
+Deutschland
+Chitarre e Bassi
+Damenmode
+DVDs & Blu-rays
+Cz??ci samochodowe: Tuning
+Computer-Komponenten & -Teile
+Handy- & PDA-Zubeh?r
+F?ves
+Acciones y Papeles
+Comics
+Damen-Accessoires
+Chauffage & Climatisation
+Fragrances
+Armband- & Taschenuhren
+Damenmode
+Lettres, Vieux papiers
+Pokemon
+V?tements & souvenirs de fan
+Mobile Phones
+Jongenskleding
+Medizin & Labor
+Home Decor
+Mobile Phone & PDA Accessories
+DVDs & Blu-rays
+Damenmode
+Drives, Storage & Media
+Kinderwagen & -tragen
+??
+D?coration de maison
+Serier USA
+??????/??
+Home Decor
+Trading Card Games
+Stamps
+Puppen
+B?ro, Papier & Schreiben
+Action Figures
+Fishing
+Damenmode
+Zubeh?r & Hardware
+Disneyana
+Nonfiction
+????
+Bijoux pour hommes
+Video Equipment
+Men's Clothing
+Motorrad- & Kraftradteile
+Damenmode
+Electrical & Test Equipment
+Gewerbe
+Dyski i nap?dy
+Herrenmode
+Versandmaterialien
+Analoge Fotografie
+Camping & Outdoor
+Consommables
+Film & Fernsehen
+Make-up
+Motorrad- & Kraftradteile
+??
+Accessori foto e videocamere
+Zubeh?r
+Team Sports
+Diecast & Vehicles
+Mobile Accessories
+Reit- & Fahrsport
+Europa
+Militaria
+Bijouterie de qualit?
+Studium & Wissen
+CD Single & Maxi
+Laufwerke & Speichermedien
+V?tements pour enfants
+Handys ohne Vertrag
+Portable Audio & Headphones
+Holiday & Seasonal
+Fische & Aquarien
+???
+Women's Shoes
+Pinbacks, Bobbles, Lunchboxes
+Motorrad- & Kraftradteile
+Sports d'hiver
+Fan Shop
+V?tements pour enfants
+PC- & Videospiele
+Audio Books
+Test Auctions
+Scrapbooking & Paper Crafts
+Games
+Groothandel & Restpartijen
+Parts & Accessories
+Weitere Formate
+Cartes de jeux de collection
+Damenmode
+Firmennetzwerk, Server
+Mobile Phones
+Design & Stil
+Perles au d?tail
+Unknown
+Silver
+Cycling
+Beleuchtung
+Locaux: mat?riel, fournitures
+Uomo: Accessori
+Paper Money: World
+Edelsteine
+Science-Fiction
+Publicit?
+Tennis & sports de raquette
+Sicherheitstechnik
+Kinderkleding- jongens
+Kleidung, Schuhe & Accessoires
+Books & Manuscripts
+Autographs-Original
+Anlageobjekte/Versteigerungen
+Kunst & Kultur
+Meters, Testers, Probes
+Soins de sant?, sciences vie
+Haushaltsbatterien & Strom
+Embroidery
+Adult Only
+Hunde
+Damenmode
+Software
+Classic Toys
+Mobiliar & Interieur
+Motorrad- & Kraftradteile
+Auto's
+Hobby & K?nstlerbedarf
+Auto-Ersatz- & -Reparaturteile
+Wargames & Role-Playing
+Damenmode
+Damenmode
+Papier, B?ro- & Schreibwaren
+Vehicle Electronics & GPS
+Accessoires Image, Son
+Damenmode
+Computing
+Cassettes vid?o
+B?b?-Fille
+Makeup
+Edelsteine
+Holzspielzeug
+Dolls & Toys
+Automobiles & camions
+Hi-Fi, son, mat?riel audio
+Damenschuhe
+TV, Video & Home Audio
+V?tements b?b?s, tout-petits
+Echtschmuck
+Militaria
+Figurines d'action
+Hand Tools & Equipment
+Fu?ball-Fanshop
+Women's Clothing
+Pi?ces & accessoires
+Movie Memorabilia
+Art du Monde
+Needlecrafts & Yarn
+Vinyles
+V?tements & bijoux hors s?rie
+Obr?bka metali i ?lusarstwo
+Damenmode
+Handys ohne Vertrag
+C?bles & Alimentations
+Pflanzen, B?ume & Str?ucher
+Damenmode
+Heimnetzwerk & Zubeh?r
+Kost?me & Verkleidungen
+Software
+Herrenmode
+Skin Care
+Glass
+Test ench?res
+Auto-Ersatz- & -Reparaturteile
+Chaussures pour hommes
+Aparatos y M?quinas Antiguas
+Education & Professional
+Cell Phone Accessories
+Kochen & Genie?en
+Maquillage/cosm?tiques
+Non-Fiction
+Lenses & Filters
+Pi?ces & accessoires
+Composants & pi?ces d'ordi
+Film-Fanartikel
+Cards
+Baustoffe
+Damenmode
+Lighting & Studio
+Markenbekleidung Jungen
+Orologi da Polso: alla Moda
+Europese niet-euromunten
+Notebooks & Netbooks
+Perles, Fabrication de bijoux
+Brass
+Manufacturing & Metalworking
+Diamants, gemmes non sertis
+Monitore, Projektoren & Zub.
+??????
+Computer-Komponenten & -Teile
+VHS Tapes
+Echtschmuck
+Machinerie lourde
+Werkzeuge & Werkstattbedarf
+Bears
+Grafik, Drucke
+Damenmode
+Video Games
+Jeunesse
+Health Care
+TV, Video & Home Audio
+Baugewerbe
+Fu?ball
+Laptop & Desktop Accessories
+Echtschmuck
+Women's Clothing
+Golf
+Art du XIX?me, et avant
+Echtschmuck
+Enterprise Networking, Servers
+Radsport
+Reise & Regionales
+Car Parts
+Fan Apparel & Souvenirs
+Home Networking & Connectivity
+Auto - Hi-Fi & Navigation
+Echtschmuck
+Grundst?cke
+Vintage
+Fragrances
+Diecast & Toy Vehicles
+DVDs & Blu-rays
+Romans Policiers Suspense
+H?rb?cher & H?rspiele
+?bernachtungen
+Uomo: Accessori
+Golf
+Kindermode, Schuhe & Access.
+Jeux de construction
+Olympic Memorabilia
+Gewerbe
+Wedding & Formal Occasion
+Sheet Music & Song Books
+Diaporama
+Papier, B?ro- & Schreibwaren
+Men's Jewelry
+Building Toys
+Zubeh?r
+Accessories
+Vanity/ Perfume/ Grooming
+Auto-Tuning & -Styling
+M?nzen Deutschland ab 1945
+CDs
+Parts & Accessories
+Hand Tools & Equipment
+Fine Jewellery
+V?tements/accessoires d'?poque
+Knitting
+Cables & Connectors
+Aircraft & Aviation
+Books & Manuscripts
+MRO & Industrial Supply
+Fashion Jewellery
+BD
+Software
+Vinyles 45T
+Angelsport
+Silver
+Pflanzen, B?ume & Str?ucher
+Sporting Goods
+K?nstlerbedarf
+Indian, Hindi VCDs
+Puppen
+K?nstlerbedarf
+Kleinger?te K?che
+PC- & Videospiele
+Sports
+Italia
+Postcards
+Transportation
+Kochen & Genie?en
+Sport
+Pottery & China
+Mobile Phones
+Fu?ball
+Kindermode, Schuhe & Access.
+Vintage
+Medizin & Labor
+V?tements b?b?s, tout-petits
+Computer-Komponenten & -Teile
+Knives, Swords & Blades
+Dog Supplies
+Diecast & Toy Vehicles
+Th?mes
+T?l?, vid?o & audio domestique
+Cuisine
+Tauchen
+Women's Clothing
+Women's Clothing
+Memorabilia
+Computer-Klassiker
+Nonfiction
+Haushalt
+Test Auctions
+Software
+Sculptures
+Kunststoffindustrie & Chemie
+Animation
+Mobile Phones
+Objets publicitaires
+Dolls
+Kochen & Genie?en
+Produkcja i Przemys?
+Pet Accessories
+Parts & Accessories
+Dolls
+Herrenmode
+Fragrances
+Aktionen
+F?tes & saisonniers
+Europa
+Herrenmode
+LCD / TFT Monitors
+???
+Home Networking & Connectivity
+Drives, Storage & Blank Media
+Software
+Teleskope & Ferngl?ser
+Fashion Jewelry
+Agriculture
+M?nzen Deutschland ab 1945
+Handy- & PDA-Zubeh?r
+Apple iPods
+Advertising
+Watches
+Scrapbooking & Paper Crafts
+??
+Monitors, Projectors & Accs.
+Decorative Collectibles
+Mobile Phones
+Land
+Damenmode
+Partitions/Livres de chansons
+RC-Modellbau
+Bi?uteria do piercingu
+Bijoux d'?poque, anciens
+Software
+?lectronique & GPS de v?hicule
+Food & Beverages
+??
+Damenmode
+Parts & Accessories
+Plakate & Kunstdrucke
+Ansichtkaarten en foto's
+Cross Stitch
+Motorrad- & Kraftradteile
+Damenmode
+Skisport & Snowboarding
+Spa, Kosmetik- & Friseursalon
+Artisanat ? l'aiguille & fil
+Glass
+Literie
+Batteries, percussions
+Fan Apparel & Souvenirs
+Glass
+Creative Services
+Sewing & Fabric
+Makeup
+Clothes, Shoes & Accessories
+Watches
+Yard, Garden & Outdoor Living
+Imprimantes, scanneurs & acc.
+Revues, manuels, catalogues
+Antiquit?s asiatiques
+Notes
+T?l?phones fixes, r?pondeurs
+Bijoux de mode
+Sega
+Clothes, Shoes & Accessories
+Electrical & Test Equipment
+Bijoux de mode
+Camcorder
+Modellautos
+Accesorios, tablets e eBooks
+Bureau
+Kultur & Events
+Europa
+Feste & Besondere Anl?sse
+Auto - Hi-Fi & Navigation
+Gartenger?te
+Fantaisie/mythe/magie
+Affiches
+Desktop PCs
+Golf
+Tasteninstrumente
+Golf
+Major Appliances
+Pi?ces Europe (pre-euro)
+Motocyclettes
+?ducatifs
+??????
+Vintage
+Cards
+Auto-Ersatz- & -Reparaturteile
+Damenmode
+Kleinger?te Pflege & Wellness
+Elettrodomestici
+D?coration de maison
+Historical Memorabilia
+Herrenschuhe
+Action Figures
+Druckerei & Copyshop
+Hunting
+Automobilia
+Baby Clothing
+Fabric
+LEGO
+Decorative Collectibles
+M?taphysique & Nouvel ?ge
+Audio - Composants & Syst?mes
+Herrenschuhe
+Antikspielzeug
+Automobile
+Reit- & Fahrsport
+??
+Vini rossi: Toscana
+Cycling
+Coins: World
+Cartes de jeux de collection
+Cz??ci samochodowe
+??/??
+Bijoux de mode
+Adult Unisex
+Elektrowerkzeuge
+Publicit?
+Computer Components & Parts
+Verre, cristal
+Souvenirs & Travel Memorabilia
+A/V Accessories & Cables
+Varia
+Internationale Antiq. & Kunst
+Laptops & Netbooks
+Glas?gon
+World Coins
+Nintendo Gamecube
+Models & Kits
+Pottery & China
+Handys ohne Vertrag
+Textbooks, Education
+????/??
+Radsport
+Bijoux de mode
+Echtschmuck
+Computer-Komponenten & -Teile
+Clothes, Shoes & Accessories
+Golf
+Services & Real Estate
+Damenmode
+Sachb?cher & Ratgeber
+DVD, Blu-ray & Heimkino
+Bew?ssern
+Staubsauger
+Kinderkleding- meisjes
+Vintage
+M?bel
+Porzellan & Keramik
+Fu?ball
+Dispositivi archiviazione dati
+Linens & Textiles (1930-Now)
+Comics
+???
+Accessories
+Scrapbooking
+Staubsauger
+Sauna & Schwimmbad
+Middle East
+Bijoux de mode
+Computer-Komponenten & -Teile
+Portable Audio & Headphones
+Agrar, Forst & Kommune
+Disneyana
+Software
+Soins de sant?, sciences vie
+Electronic Pets
+Linge & textile (avant 1930)
+Porzellan & Keramik
+Radsport
+V?tements pour enfants
+Budownictwo
+Fashion Jewelry
+Action Figures
+Zapatillas
+Skiing & Snowboarding
+Fotocamere digitali
+Dolls
+Medizin & Labor
+Car Accessories
+Bricolage e fai da te
+Construction
+Car Parts
+Elektrowerkzeuge
+Mobile & Smart Phones
+Outdoor Sports
+Science fiction & horreur
+Odzie? pozosta?a
+Festnetztelefone & Zubeh?r
+Echtschmuck
+CDs & DVDs
+Decorative Collectibles
+Fragrances
+Exercise & Fitness
+Loose Diamonds & Gemstones
+Hochzeit & Besondere Anl?sse
+Baby & Toddler Clothing
+Tickets, Experiences
+Wholesale Lots
+Vintage & Antique Jewellery
+??CD
+Handys ohne Vertrag
+CD Players
+Animals
+Fine Jewellery
+Wand & Boden
+Printing & Graphic Arts
+Echtschmuck
+Dvd's - Films
+Computer-Komponenten & -Teile
+Historische Wertpapiere
+Music Memorabilia
+Fitness
+Software
+Jogging
+Damesschoenen
+Enfants: v?tements, access.
+Radio Communication
+Pokemon
+Photographic Images
+Home Improvement
+Kindermode, Schuhe & Access.
+Produktions- & Industriebedarf
+Ansichtskarten
+Power Protection, Distribution
+Autoreifen & Felgen
+Team Sports
+Anlageobjekte/Versteigerungen
+Commonwealth/c. britannique
+Objets de collection de d?cora
+Trading Cards
+MP3 Player Accessories
+Automobilia
+Papiergeld Deutschland
+Glass
+iPods & MP3-Player
+Pre-School & Young Children
+Kamera- & Fotozubeh?r
+Kleidung, Schuhe & Accessoires
+Familienplanung
+Men's Jewelry
+Women's Clothing
+Otros deportes
+Poligrafia
+Baby Clothing
+Fische & Aquarien
+Heavy Equipment Attachments
+Software
+????
+Nakrycia sto?owe
+Bricolage: Outils
+Software
+Fashion Jewelry
+Home Arts & Crafts
+Damenmode
+Damenmode
+Audio- & Video-Rarit?ten
+Souvenirs & Travel Memorabilia
+Transport
+Brett- & Gesellschaftsspiele
+Guitar
+Animals
+Enterprise Networking, Servers
+Damenschuhe
+Cz??ci samochodowe
+Pneumatici, Cerchi e Ruote
+Women's Wrist Watches
+Alkoholfreie Getr?nke
+Mobile Phones
+Vinos y Gastronom?a
+Sports de plein air
+Hobby & K?nstlerbedarf
+Handy- & PDA-Zubeh?r
+Calzado mujer
+Personnages de t?l? & de films
+????
+Weitere Ballsportarten
+Other Video Games & Consoles
+Garden & Patio Furniture
+Portable Audio & Headphones
+Jeux
+Journaux, revues, magazines
+Parts & Accessories
+Fu?ball-Fanshop
+Diecast & Toy Vehicles
+Colliers
+Women's Shoes & Sandals
+Computer-Komponenten & -Teile
+Sciences, culture
+Camping & Outdoor
+Stamping & Embossing
+Car Parts
+Kleinkindspielzeug
+Gro?handel & Sonderposten
+Dekoration
+Kinderm?bel & Wohnen
+Cromos
+Fitness
+Mobility, Disability & Medical
+Remorques
+Kurzreisen
+Reise & Regionales
+Papier, B?ro- & Schreibwaren
+Men's Jewellery
+VHS Tapes
+Golf
+Textbooks
+Malerei
+Yard, Garden & Outdoor Living
+Mobile Phone & PDA Accessories
+Mobile Phone Accessories
+%F%l%S%2!<%`
+Feste & Besondere Anl?sse
+Bijoux de mode
+Skisport & Snowboarding
+Golf
+Teich- & Bachlaufsysteme
+Fashion Jewellery
+Animals
+Printers, Scanners & Supplies
+Auto-Tuning & -Styling
+Kids' Clothes, Shoes & Accs.
+Tuning & Styling
+Spielzeug
+Bistrot
+Kochen & Genie?en
+Antiquarische B?cher
+Camcorders
+Skisport & Snowboarding
+Literatur & Kochb?cher
+Fragrances
+Schulbedarf
+M?bel
+Passenger Vehicles
+Auto-Ersatz- & -Reparaturteile
+Tools
+Spezielle Fahrzeug-Teile
+Kindermode, Schuhe & Access.
+Kleidung, Schuhe & Accessoires
+Hunting
+PC- & Videospiele
+Watches
+Nachschlagewerke
+Decorative Ornaments/ Plates
+Piriphiriques PC
+Music Memorabilia
+Records
+Computer & Internet
+Adult Only
+Marine/ Maritime
+Machinerie lourde
+Mobiele Telefoon Accessoires
+Memory Cards & Readers
+Militaria
+Weine
+Paper Money: World
+Pen Drives
+Kinderwagen & -tragen
+Spielzeug f?r drau?en
+Home Decor
+Kleidung, Schuhe & Accessoires
+Maquillage/cosm?tiques
+Skisport & Snowboarding
+Clocks
+Bolsos
+Homeware, Kitchenware
+Damenmode
+Medizin & Labor
+Football Shirts
+Womens' Shoes
+Studium & Wissen
+Zestawy mebli
+Rollos, Gardinen & Vorh?nge
+Romans, litt?rature
+Militaria
+Wand & Boden
+Skisport & Snowboarding
+Festnetztelefone & Zubeh?r
+Travaux de r?novation
+Pi?ces & accessoires
+Music Memorabilia
+Haushaltsstrom
+Fan Shop
+Werkzeuge
+Europe
+V?tements & souvenirs de fan
+B?rom?bel
+Cards
+Cuisine, restaurants & bar
+Transport
+Auto-Tuning & -Styling
+Zubeh?r & Hardware
+Zubeh?r
+Thema's
+Autographs
+Sonnenschutz
+Tragbare Disc-Player & Radios
+W?osy
+Lichaam & Hygi?ne
+Sicurezza e Antifurti
+Akcesoria do konsol
+Home Improvement
+Cartes Postales
+W?rzen & verfeinern
+Xbox 360
+Laufwerke & Speichermedien
+Pet Supplies
+?bernachtungen
+Kamera- & Fotozubeh?r
+Shaving & Hair Removal
+Holiday & Seasonal
+Drives, Storage & Blank Media
+Hobby & K?nstlerbedarf
+Fietsen en fietssport
+Furniture
+Vintage Vehicles
+Handcrafted & Finished Pieces
+Television Memorabilia
+Livres Toutes Disciplines FR
+Software
+Femmes: Chaussures
+DVDs & Blu-ray Discs
+Personenauto's
+Piercing-/K?rperschmuck
+Pen Drives
+Pflanzen, B?ume & Str?ucher
+Dekoration
+Baugewerbe
+Festivals & Konzerte
+Kochen & Genie?en
+Car Parts
+Coins
+Folkloreschmuck
+Fische & Aquarien
+Fashion Jewellery
+Cars & Trucks
+Damenmode
+Cycling
+Hobby & K?nstlerbedarf
+Schulbedarf
+Handys mit Vertrag
+Enthaarung & Rasur
+Souvenirs historiques
+Werkzeuge
+Motor Sport Memorabilia
+Wyposa?enie
+Computer-Komponenten & -Teile
+Celebrations & Occasions
+Souvenirs musicaux
+Cultures & Ethnicities
+Herrenschuhe
+Outremer
+Nakrycia sto?owe
+Bi?uteria do piercingu
+Animals
+Skisport & Snowboarding
+Noten & Songbooks
+Kabel & Steckverbinder
+Gesichtspflege
+Mobile Phone & PDA Accessories
+Zubeh?r
+Motorcycles
+Weitere Sportarten
+Water Sports
+Team Sports
+Architectural Antiques
+Reise & Regionales
+Indian, Hindi DVDs
+Damenmode
+Reit- & Fahrsport
+Parts & Accessories
+Kleidung, Schuhe & Accessoires
+Sciences, culture
+Pflege
+Damenmode
+S?mereien & Zwiebeln
+Sport d'acqua
+Kids & Teens at Home
+Dessins anim?s & personnages
+Fashion Jewellery
+Golf
+Conexi?n de redes
+Auto-Anbau- & -Zubeh?rteile
+Echtschmuck
+Manuali, Corsi, Libri di testo
+Srebra
+Eisenwaren
+Auto-Ersatz- & -Reparaturteile
+V?tements & souvenirs de fan
+Vanity, Perfume & Shaving
+Fashion Jewellery
+Cz??ci samochodowe
+Art. do przechowywania
+Instruments
+Rocks, Fossils & Minerals
+Geschenk- & Werbeartikel
+??????????
+Akcesoria kuchenne
+Men's Clothing
+Adult Only
+Tools
+Modellautos
+Playmobil
+Manik?re & Pedik?re
+Fashion Jewelry
+Sewing (1930-Now)
+Fu?ball-Fanshop
+Weitere Uhren
+Fantasy, Magic
+Football
+Elektronik & Elektrotechnik
+Makeup
+Fan Apparel & Souvenirs
+Caravanas y Remolques
+Paper
+Parts & Accessories
+Professional, Medical, Dental
+Heim-Audio & HiFi
+TV-Receiver & Set-Top-Boxen
+Souvenirs militaires
+Painting, Drawing & Art
+Trains/ Railway Models
+Holzbearbeitung & Tischlerei
+Jeux de construction
+M?bel
+Staubsauger
+Scrapbooking & Paper Crafts
+CDs
+????
+Fan Apparel & Souvenirs
+Mariage & tenue habill?e
+Bath
+Analoge Fotografie
+Argento
+B?rotechnik
+Costume Jewellery
+Armband- & Taschenuhren
+Fernseher
+Pi?ces - ?chelle mondiale
+Obr?bka metali i ?lusarstwo
+Automobile
+Building Materials, DIY
+Meble
+Cross Stitch
+Barware
+Spezielle Fahrzeug-Teile
+Damenmode
+Cars
+Militaria
+Men's Clothing
+Accessori cellulari e palmari
+Collectibles & Memorabilia
+Cartes de collection
+Home Improvement
+Construction Toys & Kits
+Donna: Scarpe
+Enterprise Networking, Servers
+Radsport
+Armband- & Taschenuhren
+F?tes & saisonniers
+Vinyles 33T
+Ricambi Moto d'Epoca

<TRUNCATED>

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

Posted by lu...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/job/src/main/java/com/kylinolap/job/hadoop/invertedindex/IIBulkLoadJob.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/com/kylinolap/job/hadoop/invertedindex/IIBulkLoadJob.java b/job/src/main/java/com/kylinolap/job/hadoop/invertedindex/IIBulkLoadJob.java
new file mode 100644
index 0000000..022963d
--- /dev/null
+++ b/job/src/main/java/com/kylinolap/job/hadoop/invertedindex/IIBulkLoadJob.java
@@ -0,0 +1,83 @@
+/*
+ * 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.job.hadoop.invertedindex;
+
+import org.apache.commons.cli.Options;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.hbase.mapreduce.LoadIncrementalHFiles;
+import org.apache.hadoop.util.ToolRunner;
+
+import com.kylinolap.common.KylinConfig;
+import com.kylinolap.common.util.HadoopUtil;
+import com.kylinolap.cube.CubeInstance;
+import com.kylinolap.cube.CubeManager;
+import com.kylinolap.cube.CubeSegment;
+import com.kylinolap.cube.CubeSegmentStatusEnum;
+import com.kylinolap.job.hadoop.AbstractHadoopJob;
+import com.kylinolap.metadata.model.invertedindex.InvertedIndexDesc;
+
+/**
+ * @author ysong1
+ * 
+ */
+public class IIBulkLoadJob extends AbstractHadoopJob {
+
+    @Override
+    public int run(String[] args) throws Exception {
+        Options options = new Options();
+
+        try {
+            options.addOption(OPTION_INPUT_PATH);
+            options.addOption(OPTION_HTABLE_NAME);
+            options.addOption(OPTION_CUBE_NAME);
+            parseOptions(options, args);
+
+            String tableName = getOptionValue(OPTION_HTABLE_NAME);
+            String input = getOptionValue(OPTION_INPUT_PATH);
+            String cubeName = getOptionValue(OPTION_CUBE_NAME);
+
+            FileSystem fs = FileSystem.get(getConf());
+            FsPermission permission = new FsPermission((short) 0777);
+            fs.setPermission(new Path(input, InvertedIndexDesc.HBASE_FAMILY), permission);
+
+            int hbaseExitCode = ToolRunner.run(new LoadIncrementalHFiles(getConf()), new String[] { input, tableName });
+
+            CubeManager mgr = CubeManager.getInstance(KylinConfig.getInstanceFromEnv());
+            CubeInstance cube = mgr.getCube(cubeName);
+            CubeSegment seg = cube.getFirstSegment();
+            seg.setStorageLocationIdentifier(tableName);
+            seg.setStatus(CubeSegmentStatusEnum.READY);
+            mgr.updateCube(cube);
+
+            return hbaseExitCode;
+
+        } catch (Exception e) {
+            printUsage(options);
+            e.printStackTrace(System.err);
+            return 2;
+        }
+    }
+
+    public static void main(String[] args) throws Exception {
+        IIBulkLoadJob job = new IIBulkLoadJob();
+        job.setConf(HadoopUtil.newHBaseConfiguration(KylinConfig.getInstanceFromEnv().getStorageUrl()));
+        int exitCode = ToolRunner.run(job, args);
+        System.exit(exitCode);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/job/src/main/java/com/kylinolap/job/hadoop/invertedindex/IICreateHFileJob.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/com/kylinolap/job/hadoop/invertedindex/IICreateHFileJob.java b/job/src/main/java/com/kylinolap/job/hadoop/invertedindex/IICreateHFileJob.java
new file mode 100644
index 0000000..0c1afd0
--- /dev/null
+++ b/job/src/main/java/com/kylinolap/job/hadoop/invertedindex/IICreateHFileJob.java
@@ -0,0 +1,96 @@
+/*
+ * 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.job.hadoop.invertedindex;
+
+import java.io.File;
+
+import org.apache.commons.cli.Options;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.client.HTable;
+import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
+import org.apache.hadoop.hbase.mapreduce.HFileOutputFormat;
+import org.apache.hadoop.mapreduce.Job;
+import org.apache.hadoop.mapreduce.lib.input.SequenceFileInputFormat;
+import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
+import org.apache.hadoop.util.ToolRunner;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.kylinolap.common.KylinConfig;
+import com.kylinolap.common.util.HadoopUtil;
+import com.kylinolap.job.hadoop.AbstractHadoopJob;
+
+/**
+ * @author yangli9
+ * 
+ */
+public class IICreateHFileJob extends AbstractHadoopJob {
+
+    protected static final Logger log = LoggerFactory.getLogger(IICreateHFileJob.class);
+
+    public int run(String[] args) throws Exception {
+        Options options = new Options();
+
+        try {
+            options.addOption(OPTION_JOB_NAME);
+            options.addOption(OPTION_CUBE_NAME);
+            options.addOption(OPTION_INPUT_PATH);
+            options.addOption(OPTION_OUTPUT_PATH);
+            options.addOption(OPTION_HTABLE_NAME);
+            parseOptions(options, args);
+
+            Path output = new Path(getOptionValue(OPTION_OUTPUT_PATH));
+
+            job = Job.getInstance(getConf(), getOptionValue(OPTION_JOB_NAME));
+
+            File JarFile = new File(KylinConfig.getInstanceFromEnv().getKylinJobJarPath());
+            if (JarFile.exists()) {
+                job.setJar(KylinConfig.getInstanceFromEnv().getKylinJobJarPath());
+            } else {
+                job.setJarByClass(this.getClass());
+            }
+
+            addInputDirs(getOptionValue(OPTION_INPUT_PATH), job);
+            FileOutputFormat.setOutputPath(job, output);
+
+            job.setInputFormatClass(SequenceFileInputFormat.class);
+            job.setMapperClass(IICreateHFileMapper.class);
+            job.setMapOutputKeyClass(ImmutableBytesWritable.class);
+            job.setMapOutputValueClass(KeyValue.class);
+
+            String tableName = getOptionValue(OPTION_HTABLE_NAME);
+            HTable htable = new HTable(getConf(), tableName);
+            HFileOutputFormat.configureIncrementalLoad(job, htable);
+
+            this.deletePath(job.getConfiguration(), output);
+
+            return waitForCompletion(job);
+        } catch (Exception e) {
+            printUsage(options);
+            log.error(e.getLocalizedMessage(), e);
+            return 2;
+        }
+    }
+
+    public static void main(String[] args) throws Exception {
+        IICreateHFileJob job = new IICreateHFileJob();
+        job.setConf(HadoopUtil.newHBaseConfiguration(KylinConfig.getInstanceFromEnv().getStorageUrl()));
+        int exitCode = ToolRunner.run(job, args);
+        System.exit(exitCode);
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/job/src/main/java/com/kylinolap/job/hadoop/invertedindex/IICreateHFileMapper.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/com/kylinolap/job/hadoop/invertedindex/IICreateHFileMapper.java b/job/src/main/java/com/kylinolap/job/hadoop/invertedindex/IICreateHFileMapper.java
new file mode 100644
index 0000000..2ceaa1c
--- /dev/null
+++ b/job/src/main/java/com/kylinolap/job/hadoop/invertedindex/IICreateHFileMapper.java
@@ -0,0 +1,53 @@
+/*
+ * 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.job.hadoop.invertedindex;
+
+import static com.kylinolap.metadata.model.invertedindex.InvertedIndexDesc.*;
+
+import java.io.IOException;
+
+import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.KeyValue.Type;
+import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
+import org.apache.hadoop.mapreduce.Mapper;
+
+/**
+ * @author yangli9
+ * 
+ */
+public class IICreateHFileMapper extends Mapper<ImmutableBytesWritable, ImmutableBytesWritable, ImmutableBytesWritable, KeyValue> {
+
+    long timestamp;
+
+    @Override
+    protected void setup(Context context) throws IOException, InterruptedException {
+        timestamp = System.currentTimeMillis();
+    }
+
+    @Override
+    protected void map(ImmutableBytesWritable key, ImmutableBytesWritable value, Context context) throws IOException, InterruptedException {
+
+        KeyValue kv = new KeyValue(key.get(), key.getOffset(), key.getLength(), //
+                HBASE_FAMILY_BYTES, 0, HBASE_FAMILY_BYTES.length, //
+                HBASE_QUALIFIER_BYTES, 0, HBASE_QUALIFIER_BYTES.length, //
+                timestamp, Type.Put, //
+                value.get(), value.getOffset(), value.getLength());
+
+        context.write(key, kv);
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/job/src/main/java/com/kylinolap/job/hadoop/invertedindex/IICreateHTableJob.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/com/kylinolap/job/hadoop/invertedindex/IICreateHTableJob.java b/job/src/main/java/com/kylinolap/job/hadoop/invertedindex/IICreateHTableJob.java
new file mode 100644
index 0000000..04fd274
--- /dev/null
+++ b/job/src/main/java/com/kylinolap/job/hadoop/invertedindex/IICreateHTableJob.java
@@ -0,0 +1,118 @@
+/*
+ * 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.job.hadoop.invertedindex;
+
+import org.apache.commons.cli.Options;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.HBaseConfiguration;
+import org.apache.hadoop.hbase.HColumnDescriptor;
+import org.apache.hadoop.hbase.HTableDescriptor;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.HBaseAdmin;
+import org.apache.hadoop.hbase.io.compress.Compression.Algorithm;
+import org.apache.hadoop.hbase.io.encoding.DataBlockEncoding;
+import org.apache.hadoop.hbase.security.User;
+import org.apache.hadoop.util.ToolRunner;
+
+import com.kylinolap.common.KylinConfig;
+import com.kylinolap.common.util.BytesUtil;
+import com.kylinolap.common.util.HadoopUtil;
+import com.kylinolap.cube.CubeInstance;
+import com.kylinolap.cube.CubeManager;
+import com.kylinolap.cube.invertedindex.IIKeyValueCodec;
+import com.kylinolap.job.hadoop.AbstractHadoopJob;
+import com.kylinolap.metadata.model.invertedindex.InvertedIndexDesc;
+
+/**
+ * @author George Song (ysong1)
+ */
+public class IICreateHTableJob extends AbstractHadoopJob {
+
+    @Override
+    public int run(String[] args) throws Exception {
+        Options options = new Options();
+
+        try {
+            options.addOption(OPTION_CUBE_NAME);
+            options.addOption(OPTION_HTABLE_NAME);
+            parseOptions(options, args);
+
+            String tableName = getOptionValue(OPTION_HTABLE_NAME);
+            String cubeName = getOptionValue(OPTION_CUBE_NAME);
+
+            CubeInstance cubeInstance = CubeManager.getInstance(KylinConfig.getInstanceFromEnv()).getCube(cubeName);
+            int sharding = cubeInstance.getInvertedIndexDesc().getSharding();
+
+            HTableDescriptor tableDesc = new HTableDescriptor(TableName.valueOf(tableName));
+            HColumnDescriptor cf = new HColumnDescriptor(InvertedIndexDesc.HBASE_FAMILY);
+            cf.setMaxVersions(1);
+            cf.setCompressionType(Algorithm.LZO);
+            cf.setDataBlockEncoding(DataBlockEncoding.FAST_DIFF);
+            tableDesc.addFamily(cf);
+
+            Configuration conf = HBaseConfiguration.create(getConf());
+            if (User.isHBaseSecurityEnabled(conf)) {
+                // add coprocessor for bulk load
+                tableDesc.addCoprocessor("org.apache.hadoop.hbase.security.access.SecureBulkLoadEndpoint");
+            }
+
+            // drop the table first
+            HBaseAdmin admin = new HBaseAdmin(conf);
+            if (admin.tableExists(tableName)) {
+                admin.disableTable(tableName);
+                admin.deleteTable(tableName);
+            }
+
+            // create table
+            byte[][] splitKeys = getSplits(sharding);
+            if (splitKeys.length == 0)
+                splitKeys = null;
+            admin.createTable(tableDesc, splitKeys);
+            if (splitKeys != null) {
+                for (int i = 0; i < splitKeys.length; i++) {
+                    System.out.println("split key " + i + ": " + BytesUtil.toHex(splitKeys[i]));
+                }
+            }
+            System.out.println("create hbase table " + tableName + " done.");
+            admin.close();
+
+            return 0;
+        } catch (Exception e) {
+            printUsage(options);
+            e.printStackTrace(System.err);
+            log.error(e.getLocalizedMessage(), e);
+            return 2;
+        }
+    }
+
+    //one region for one shard
+    private byte[][] getSplits(int shard) {
+        byte[][] result = new byte[shard - 1][];
+        for (int i = 1; i < shard; ++i) {
+            byte[] split = new byte[IIKeyValueCodec.SHARD_LEN];
+            BytesUtil.writeUnsigned(i, split, 0, IIKeyValueCodec.SHARD_LEN);
+            result[i - 1] = split;
+        }
+        return result;
+    }
+
+    public static void main(String[] args) throws Exception {
+        IICreateHTableJob job = new IICreateHTableJob();
+        job.setConf(HadoopUtil.newHBaseConfiguration(KylinConfig.getInstanceFromEnv().getStorageUrl()));
+        int exitCode = ToolRunner.run(job, args);
+        System.exit(exitCode);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/job/src/main/java/com/kylinolap/job/hadoop/invertedindex/IIDistinctColumnsCombiner.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/com/kylinolap/job/hadoop/invertedindex/IIDistinctColumnsCombiner.java b/job/src/main/java/com/kylinolap/job/hadoop/invertedindex/IIDistinctColumnsCombiner.java
new file mode 100644
index 0000000..83219d2
--- /dev/null
+++ b/job/src/main/java/com/kylinolap/job/hadoop/invertedindex/IIDistinctColumnsCombiner.java
@@ -0,0 +1,54 @@
+/*
+ * 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.job.hadoop.invertedindex;
+
+import java.io.IOException;
+import java.util.HashSet;
+
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.io.ShortWritable;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.mapreduce.Reducer;
+
+import com.kylinolap.common.util.ByteArray;
+
+/**
+ * @author yangli9
+ */
+public class IIDistinctColumnsCombiner extends Reducer<ShortWritable, Text, ShortWritable, Text> {
+
+    private Text outputValue = new Text();
+
+    @Override
+    protected void setup(Context context) throws IOException {
+    }
+
+    @Override
+    public void reduce(ShortWritable key, Iterable<Text> values, Context context) throws IOException, InterruptedException {
+
+        HashSet<ByteArray> set = new HashSet<ByteArray>();
+        for (Text textValue : values) {
+            ByteArray value = new ByteArray(Bytes.copy(textValue.getBytes(), 0, textValue.getLength()));
+            set.add(value);
+        }
+
+        for (ByteArray value : set) {
+            outputValue.set(value.data);
+            context.write(key, outputValue);
+        }
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/job/src/main/java/com/kylinolap/job/hadoop/invertedindex/IIDistinctColumnsJob.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/com/kylinolap/job/hadoop/invertedindex/IIDistinctColumnsJob.java b/job/src/main/java/com/kylinolap/job/hadoop/invertedindex/IIDistinctColumnsJob.java
new file mode 100644
index 0000000..bc12db2
--- /dev/null
+++ b/job/src/main/java/com/kylinolap/job/hadoop/invertedindex/IIDistinctColumnsJob.java
@@ -0,0 +1,153 @@
+/*
+ * 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.job.hadoop.invertedindex;
+
+import java.io.File;
+import java.io.IOException;
+
+import org.apache.commons.cli.Options;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.io.NullWritable;
+import org.apache.hadoop.io.ShortWritable;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.mapreduce.Job;
+import org.apache.hadoop.mapreduce.lib.input.FileInputFormat;
+import org.apache.hadoop.mapreduce.lib.input.SequenceFileInputFormat;
+import org.apache.hadoop.mapreduce.lib.input.TextInputFormat;
+import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
+import org.apache.hadoop.mapreduce.lib.output.SequenceFileOutputFormat;
+import org.apache.hadoop.util.ToolRunner;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.kylinolap.common.KylinConfig;
+import com.kylinolap.job.constant.BatchConstants;
+import com.kylinolap.job.hadoop.AbstractHadoopJob;
+import com.kylinolap.metadata.MetadataManager;
+import com.kylinolap.metadata.model.schema.ColumnDesc;
+import com.kylinolap.metadata.model.schema.TableDesc;
+
+/**
+ * @author yangli9
+ */
+public class IIDistinctColumnsJob extends AbstractHadoopJob {
+    protected static final Logger log = LoggerFactory.getLogger(IIDistinctColumnsJob.class);
+
+    @Override
+    public int run(String[] args) throws Exception {
+        Options options = new Options();
+
+        try {
+            options.addOption(OPTION_JOB_NAME);
+            options.addOption(OPTION_TABLE_NAME);
+            options.addOption(OPTION_INPUT_PATH);
+            options.addOption(OPTION_INPUT_FORMAT);
+            options.addOption(OPTION_INPUT_DELIM);
+            options.addOption(OPTION_OUTPUT_PATH);
+            parseOptions(options, args);
+
+            job = Job.getInstance(getConf(), getOptionValue(OPTION_JOB_NAME));
+            String tableName = getOptionValue(OPTION_TABLE_NAME).toUpperCase();
+            Path input = new Path(getOptionValue(OPTION_INPUT_PATH));
+            String inputFormat = getOptionValue(OPTION_INPUT_FORMAT);
+            String inputDelim = getOptionValue(OPTION_INPUT_DELIM);
+            Path output = new Path(getOptionValue(OPTION_OUTPUT_PATH));
+
+            // ----------------------------------------------------------------------------
+
+            System.out.println("Starting: " + job.getJobName());
+
+            setupMapInput(input, inputFormat, inputDelim);
+            setupReduceOutput(output);
+
+            // pass table and columns
+            MetadataManager metaMgr = MetadataManager.getInstance(KylinConfig.getInstanceFromEnv());
+            TableDesc table = metaMgr.getTableDesc(tableName);
+            job.getConfiguration().set(BatchConstants.TABLE_NAME, tableName);
+            job.getConfiguration().set(BatchConstants.TABLE_COLUMNS, getColumns(table));
+
+            return waitForCompletion(job);
+
+        } catch (Exception e) {
+            printUsage(options);
+            log.error(e.getLocalizedMessage(), e);
+            return 2;
+        }
+
+    }
+
+    private String getColumns(TableDesc table) {
+        StringBuilder buf = new StringBuilder();
+        for (ColumnDesc col : table.getColumns()) {
+            if (buf.length() > 0)
+                buf.append(",");
+            buf.append(col.getName());
+        }
+        return buf.toString();
+    }
+
+    private void setupMapInput(Path input, String inputFormat, String inputDelim) throws IOException {
+        FileInputFormat.setInputPaths(job, input);
+
+        File JarFile = new File(KylinConfig.getInstanceFromEnv().getKylinJobJarPath());
+        if (JarFile.exists()) {
+            job.setJar(KylinConfig.getInstanceFromEnv().getKylinJobJarPath());
+        } else {
+            job.setJarByClass(this.getClass());
+        }
+
+        if ("textinputformat".equalsIgnoreCase(inputFormat) || "text".equalsIgnoreCase(inputFormat)) {
+            job.setInputFormatClass(TextInputFormat.class);
+        } else {
+            job.setInputFormatClass(SequenceFileInputFormat.class);
+        }
+
+        if ("t".equals(inputDelim)) {
+            inputDelim = "\t";
+        } else if ("177".equals(inputDelim)) {
+            inputDelim = "\177";
+        }
+        if (inputDelim != null) {
+            job.getConfiguration().set(BatchConstants.INPUT_DELIM, inputDelim);
+        }
+
+        job.setMapperClass(IIDistinctColumnsMapper.class);
+        job.setCombinerClass(IIDistinctColumnsCombiner.class);
+        job.setMapOutputKeyClass(ShortWritable.class);
+        job.setMapOutputValueClass(Text.class);
+    }
+
+    private void setupReduceOutput(Path output) throws IOException {
+        job.setReducerClass(IIDistinctColumnsReducer.class);
+        job.setOutputFormatClass(SequenceFileOutputFormat.class);
+        job.setOutputKeyClass(NullWritable.class);
+        job.setOutputValueClass(Text.class);
+
+        FileOutputFormat.setOutputPath(job, output);
+        job.getConfiguration().set(BatchConstants.OUTPUT_PATH, output.toString());
+
+        job.setNumReduceTasks(1);
+
+        deletePath(job.getConfiguration(), output);
+    }
+
+    public static void main(String[] args) throws Exception {
+        IIDistinctColumnsJob job = new IIDistinctColumnsJob();
+        int exitCode = ToolRunner.run(job, args);
+        System.exit(exitCode);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/job/src/main/java/com/kylinolap/job/hadoop/invertedindex/IIDistinctColumnsMapper.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/com/kylinolap/job/hadoop/invertedindex/IIDistinctColumnsMapper.java b/job/src/main/java/com/kylinolap/job/hadoop/invertedindex/IIDistinctColumnsMapper.java
new file mode 100644
index 0000000..b7456bf
--- /dev/null
+++ b/job/src/main/java/com/kylinolap/job/hadoop/invertedindex/IIDistinctColumnsMapper.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.job.hadoop.invertedindex;
+
+import java.io.IOException;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.io.ShortWritable;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.mapreduce.Mapper;
+
+import com.kylinolap.cube.common.BytesSplitter;
+import com.kylinolap.cube.common.SplittedBytes;
+import com.kylinolap.job.constant.BatchConstants;
+
+/**
+ * @author yangli9
+ */
+public class IIDistinctColumnsMapper<KEYIN> extends Mapper<KEYIN, Text, ShortWritable, Text> {
+
+    private String[] columns;
+    private int delim;
+    private BytesSplitter splitter;
+
+    private ShortWritable outputKey = new ShortWritable();
+    private Text outputValue = new Text();
+
+    @Override
+    protected void setup(Context context) throws IOException {
+        Configuration conf = context.getConfiguration();
+        this.columns = conf.get(BatchConstants.TABLE_COLUMNS).split(",");
+        String inputDelim = conf.get(BatchConstants.INPUT_DELIM);
+        this.delim = inputDelim == null ? -1 : inputDelim.codePointAt(0);
+        this.splitter = new BytesSplitter(200, 4096);
+    }
+
+    @Override
+    public void map(KEYIN key, Text value, Context context) throws IOException, InterruptedException {
+        if (delim == -1) {
+            delim = splitter.detectDelim(value, columns.length);
+        }
+
+        int nParts = splitter.split(value.getBytes(), value.getLength(), (byte) delim);
+        SplittedBytes[] parts = splitter.getSplitBuffers();
+
+        if (nParts != columns.length) {
+            throw new RuntimeException("Got " + parts.length + " from -- " + value.toString() + " -- but only " + columns.length + " expected");
+        }
+
+        for (short i = 0; i < nParts; i++) {
+            outputKey.set(i);
+            outputValue.set(parts[i].value, 0, parts[i].length);
+            context.write(outputKey, outputValue);
+        }
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/job/src/main/java/com/kylinolap/job/hadoop/invertedindex/IIDistinctColumnsReducer.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/com/kylinolap/job/hadoop/invertedindex/IIDistinctColumnsReducer.java b/job/src/main/java/com/kylinolap/job/hadoop/invertedindex/IIDistinctColumnsReducer.java
new file mode 100644
index 0000000..f170057
--- /dev/null
+++ b/job/src/main/java/com/kylinolap/job/hadoop/invertedindex/IIDistinctColumnsReducer.java
@@ -0,0 +1,73 @@
+/*
+ * 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.job.hadoop.invertedindex;
+
+import java.io.IOException;
+import java.util.HashSet;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.io.NullWritable;
+import org.apache.hadoop.io.ShortWritable;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.mapreduce.Reducer;
+
+import com.kylinolap.common.util.ByteArray;
+import com.kylinolap.job.constant.BatchConstants;
+
+/**
+ * @author yangli9
+ */
+public class IIDistinctColumnsReducer extends Reducer<ShortWritable, Text, NullWritable, Text> {
+
+    private String[] columns;
+
+    @Override
+    protected void setup(Context context) throws IOException {
+        Configuration conf = context.getConfiguration();
+        this.columns = conf.get(BatchConstants.TABLE_COLUMNS).split(",");
+    }
+
+    @Override
+    public void reduce(ShortWritable key, Iterable<Text> values, Context context) throws IOException, InterruptedException {
+        String columnName = columns[key.get()];
+
+        HashSet<ByteArray> set = new HashSet<ByteArray>();
+        for (Text textValue : values) {
+            ByteArray value = new ByteArray(Bytes.copy(textValue.getBytes(), 0, textValue.getLength()));
+            set.add(value);
+        }
+
+        Configuration conf = context.getConfiguration();
+        FileSystem fs = FileSystem.get(conf);
+        String outputPath = conf.get(BatchConstants.OUTPUT_PATH);
+        FSDataOutputStream out = fs.create(new Path(outputPath, columnName));
+
+        try {
+            for (ByteArray value : set) {
+                out.write(value.data);
+                out.write('\n');
+            }
+        } finally {
+            out.close();
+        }
+
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/job/src/main/java/com/kylinolap/job/hadoop/invertedindex/InvertedIndexJob.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/com/kylinolap/job/hadoop/invertedindex/InvertedIndexJob.java b/job/src/main/java/com/kylinolap/job/hadoop/invertedindex/InvertedIndexJob.java
new file mode 100644
index 0000000..6681db6
--- /dev/null
+++ b/job/src/main/java/com/kylinolap/job/hadoop/invertedindex/InvertedIndexJob.java
@@ -0,0 +1,163 @@
+/*
+ * 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.job.hadoop.invertedindex;
+
+import java.io.File;
+import java.io.IOException;
+
+import org.apache.commons.cli.Options;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
+import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.mapreduce.Job;
+import org.apache.hadoop.mapreduce.lib.input.FileInputFormat;
+import org.apache.hadoop.mapreduce.lib.input.SequenceFileInputFormat;
+import org.apache.hadoop.mapreduce.lib.input.TextInputFormat;
+import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
+import org.apache.hadoop.mapreduce.lib.output.SequenceFileOutputFormat;
+import org.apache.hadoop.util.ToolRunner;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.kylinolap.common.KylinConfig;
+import com.kylinolap.cube.CubeInstance;
+import com.kylinolap.cube.CubeManager;
+import com.kylinolap.cube.CubeSegment;
+import com.kylinolap.job.constant.BatchConstants;
+import com.kylinolap.job.hadoop.AbstractHadoopJob;
+
+/**
+ * @author yangli9
+ */
+public class InvertedIndexJob extends AbstractHadoopJob {
+    protected static final Logger log = LoggerFactory.getLogger(InvertedIndexJob.class);
+
+    @Override
+    public int run(String[] args) throws Exception {
+        Options options = new Options();
+
+        try {
+            options.addOption(OPTION_JOB_NAME);
+            options.addOption(OPTION_CUBE_NAME);
+            options.addOption(OPTION_INPUT_PATH);
+            options.addOption(OPTION_INPUT_FORMAT);
+            options.addOption(OPTION_INPUT_DELIM);
+            options.addOption(OPTION_OUTPUT_PATH);
+            parseOptions(options, args);
+
+            job = Job.getInstance(getConf(), getOptionValue(OPTION_JOB_NAME));
+            String cubeName = getOptionValue(OPTION_CUBE_NAME);
+            Path input = new Path(getOptionValue(OPTION_INPUT_PATH));
+            String inputFormat = getOptionValue(OPTION_INPUT_FORMAT);
+            String inputDelim = getOptionValue(OPTION_INPUT_DELIM);
+            Path output = new Path(getOptionValue(OPTION_OUTPUT_PATH));
+
+            // ----------------------------------------------------------------------------
+
+            System.out.println("Starting: " + job.getJobName());
+            
+            CubeInstance cube = getCube(cubeName);
+
+            setupMapInput(input, inputFormat, inputDelim);
+            setupReduceOutput(output, cube.getInvertedIndexDesc().getSharding());
+            attachMetadata(cube);
+
+            return waitForCompletion(job);
+
+        } catch (Exception e) {
+            printUsage(options);
+            log.error(e.getLocalizedMessage(), e);
+            return 2;
+        }
+
+    }
+
+    /**
+     * @param cubeName
+     * @return
+     */
+    private CubeInstance getCube(String cubeName) {
+        CubeManager mgr = CubeManager.getInstance(KylinConfig.getInstanceFromEnv());
+        CubeInstance cube = mgr.getCube(cubeName);
+        if (cube == null)
+            throw new IllegalArgumentException("No Inverted Index Cubefound by name " + cubeName);
+        return cube;
+    }
+
+    private void attachMetadata(CubeInstance cube) throws IOException {
+
+        Configuration conf = job.getConfiguration();
+        attachKylinPropsAndMetadata(cube, conf);
+
+        CubeSegment seg = cube.getFirstSegment();
+        conf.set(BatchConstants.CFG_CUBE_NAME, cube.getName());
+        conf.set(BatchConstants.CFG_CUBE_SEGMENT_NAME, seg.getName());
+    }
+
+    private void setupMapInput(Path input, String inputFormat, String inputDelim) throws IOException {
+        FileInputFormat.setInputPaths(job, input);
+
+        File JarFile = new File(KylinConfig.getInstanceFromEnv().getKylinJobJarPath());
+        if (JarFile.exists()) {
+            job.setJar(KylinConfig.getInstanceFromEnv().getKylinJobJarPath());
+        } else {
+            job.setJarByClass(this.getClass());
+        }
+
+        if ("textinputformat".equalsIgnoreCase(inputFormat) || "text".equalsIgnoreCase(inputFormat)) {
+            job.setInputFormatClass(TextInputFormat.class);
+        } else {
+            job.setInputFormatClass(SequenceFileInputFormat.class);
+        }
+
+        if ("t".equals(inputDelim)) {
+            inputDelim = "\t";
+        } else if ("177".equals(inputDelim)) {
+            inputDelim = "\177";
+        }
+        if (inputDelim != null) {
+            job.getConfiguration().set(BatchConstants.INPUT_DELIM, inputDelim);
+        }
+
+        job.setMapperClass(InvertedIndexMapper.class);
+        job.setMapOutputKeyClass(LongWritable.class);
+        job.setMapOutputValueClass(ImmutableBytesWritable.class);
+        job.setPartitionerClass(InvertedIndexPartitioner.class);
+    }
+
+    private void setupReduceOutput(Path output, short sharding) throws IOException {
+        job.setReducerClass(InvertedIndexReducer.class);
+        job.setOutputFormatClass(SequenceFileOutputFormat.class);
+        job.setOutputKeyClass(ImmutableBytesWritable.class);
+        job.setOutputValueClass(ImmutableBytesWritable.class);
+        
+        job.setNumReduceTasks(sharding);
+
+        FileOutputFormat.setOutputPath(job, output);
+
+        job.getConfiguration().set(BatchConstants.OUTPUT_PATH, output.toString());
+
+        deletePath(job.getConfiguration(), output);
+    }
+
+    public static void main(String[] args) throws Exception {
+        InvertedIndexJob job = new InvertedIndexJob();
+        int exitCode = ToolRunner.run(job, args);
+        System.exit(exitCode);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/job/src/main/java/com/kylinolap/job/hadoop/invertedindex/InvertedIndexMapper.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/com/kylinolap/job/hadoop/invertedindex/InvertedIndexMapper.java b/job/src/main/java/com/kylinolap/job/hadoop/invertedindex/InvertedIndexMapper.java
new file mode 100644
index 0000000..f555c40
--- /dev/null
+++ b/job/src/main/java/com/kylinolap/job/hadoop/invertedindex/InvertedIndexMapper.java
@@ -0,0 +1,95 @@
+/*
+ * 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.job.hadoop.invertedindex;
+
+import java.io.IOException;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.mapreduce.Mapper;
+
+import com.kylinolap.common.KylinConfig;
+import com.kylinolap.cube.CubeInstance;
+import com.kylinolap.cube.CubeManager;
+import com.kylinolap.cube.CubeSegment;
+import com.kylinolap.cube.CubeSegmentStatusEnum;
+import com.kylinolap.cube.common.BytesSplitter;
+import com.kylinolap.cube.common.SplittedBytes;
+import com.kylinolap.cube.invertedindex.TableRecord;
+import com.kylinolap.cube.invertedindex.TableRecordInfo;
+import com.kylinolap.job.constant.BatchConstants;
+import com.kylinolap.job.hadoop.AbstractHadoopJob;
+
+/**
+ * @author yangli9
+ * 
+ */
+public class InvertedIndexMapper<KEYIN> extends Mapper<KEYIN, Text, LongWritable, ImmutableBytesWritable> {
+
+    private TableRecordInfo info;
+    private TableRecord rec;
+    private int delim;
+    private BytesSplitter splitter;
+
+    private LongWritable outputKey;
+    private ImmutableBytesWritable outputValue;
+
+    @Override
+    protected void setup(Context context) throws IOException {
+        Configuration conf = context.getConfiguration();
+        String inputDelim = conf.get(BatchConstants.INPUT_DELIM);
+        this.delim = inputDelim == null ? -1 : inputDelim.codePointAt(0);
+        this.splitter = new BytesSplitter(200, 4096);
+
+        KylinConfig config = AbstractHadoopJob.loadKylinPropsAndMetadata(conf);
+        CubeManager mgr = CubeManager.getInstance(config);
+        CubeInstance cube = mgr.getCube(conf.get(BatchConstants.CFG_CUBE_NAME));
+        CubeSegment seg = cube.getSegment(conf.get(BatchConstants.CFG_CUBE_SEGMENT_NAME), CubeSegmentStatusEnum.NEW);
+        this.info = new TableRecordInfo(seg);
+        this.rec = new TableRecord(this.info);
+
+        outputKey = new LongWritable();
+        outputValue = new ImmutableBytesWritable(rec.getBytes());
+    }
+
+    @Override
+    public void map(KEYIN key, Text value, Context context) throws IOException, InterruptedException {
+        if (delim == -1) {
+            delim = splitter.detectDelim(value, info.getColumnCount());
+        }
+
+        int nParts = splitter.split(value.getBytes(), value.getLength(), (byte) delim);
+        SplittedBytes[] parts = splitter.getSplitBuffers();
+
+        if (nParts != info.getColumnCount()) {
+            throw new RuntimeException("Got " + parts.length + " from -- " + value.toString() + " -- but only " + info.getColumnCount() + " expected");
+        }
+
+        rec.reset();
+        for (int i = 0; i < nParts; i++) {
+            rec.setValueString(i, Bytes.toString(parts[i].value, 0, parts[i].length));
+        }
+
+        outputKey.set(rec.getTimestamp());
+        // outputValue's backing bytes array is the same as rec
+
+        context.write(outputKey, outputValue);
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/job/src/main/java/com/kylinolap/job/hadoop/invertedindex/InvertedIndexPartitioner.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/com/kylinolap/job/hadoop/invertedindex/InvertedIndexPartitioner.java b/job/src/main/java/com/kylinolap/job/hadoop/invertedindex/InvertedIndexPartitioner.java
new file mode 100644
index 0000000..bd06d74
--- /dev/null
+++ b/job/src/main/java/com/kylinolap/job/hadoop/invertedindex/InvertedIndexPartitioner.java
@@ -0,0 +1,73 @@
+/*
+ * 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.job.hadoop.invertedindex;
+
+import java.io.IOException;
+
+import org.apache.hadoop.conf.Configurable;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
+import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.mapreduce.Partitioner;
+
+import com.kylinolap.common.KylinConfig;
+import com.kylinolap.cube.CubeInstance;
+import com.kylinolap.cube.CubeManager;
+import com.kylinolap.cube.CubeSegment;
+import com.kylinolap.cube.CubeSegmentStatusEnum;
+import com.kylinolap.cube.invertedindex.TableRecord;
+import com.kylinolap.cube.invertedindex.TableRecordInfo;
+import com.kylinolap.job.constant.BatchConstants;
+import com.kylinolap.job.hadoop.AbstractHadoopJob;
+
+/**
+ * @author yangli9
+ * 
+ */
+public class InvertedIndexPartitioner extends Partitioner<LongWritable, ImmutableBytesWritable> implements Configurable {
+
+    private Configuration conf;
+    private TableRecordInfo info;
+    private TableRecord rec;
+
+    @Override
+    public int getPartition(LongWritable key, ImmutableBytesWritable value, int numPartitions) {
+        rec.setBytes(value.get(), value.getOffset(), value.getLength());
+        return rec.getShard();
+    }
+
+    @Override
+    public void setConf(Configuration conf) {
+        this.conf = conf;
+        try {
+            KylinConfig config = AbstractHadoopJob.loadKylinPropsAndMetadata(conf);
+            CubeManager mgr = CubeManager.getInstance(config);
+            CubeInstance cube = mgr.getCube(conf.get(BatchConstants.CFG_CUBE_NAME));
+            CubeSegment seg = cube.getSegment(conf.get(BatchConstants.CFG_CUBE_SEGMENT_NAME), CubeSegmentStatusEnum.NEW);
+            this.info = new TableRecordInfo(seg);
+            this.rec = new TableRecord(this.info);
+        } catch (IOException e) {
+            throw new RuntimeException("", e);
+        }
+    }
+
+    @Override
+    public Configuration getConf() {
+        return conf;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/job/src/main/java/com/kylinolap/job/hadoop/invertedindex/InvertedIndexReducer.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/com/kylinolap/job/hadoop/invertedindex/InvertedIndexReducer.java b/job/src/main/java/com/kylinolap/job/hadoop/invertedindex/InvertedIndexReducer.java
new file mode 100644
index 0000000..09954a1
--- /dev/null
+++ b/job/src/main/java/com/kylinolap/job/hadoop/invertedindex/InvertedIndexReducer.java
@@ -0,0 +1,94 @@
+/*
+ * 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.job.hadoop.invertedindex;
+
+import java.io.IOException;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
+import org.apache.hadoop.hbase.util.Pair;
+import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.mapreduce.Reducer;
+
+import com.kylinolap.common.KylinConfig;
+import com.kylinolap.cube.CubeInstance;
+import com.kylinolap.cube.CubeManager;
+import com.kylinolap.cube.CubeSegment;
+import com.kylinolap.cube.CubeSegmentStatusEnum;
+import com.kylinolap.cube.invertedindex.IIKeyValueCodec;
+import com.kylinolap.cube.invertedindex.TableRecord;
+import com.kylinolap.cube.invertedindex.TableRecordInfo;
+import com.kylinolap.cube.invertedindex.Slice;
+import com.kylinolap.cube.invertedindex.SliceBuilder;
+import com.kylinolap.job.constant.BatchConstants;
+import com.kylinolap.job.hadoop.AbstractHadoopJob;
+
+/**
+ * @author yangli9
+ */
+public class InvertedIndexReducer extends Reducer<LongWritable, ImmutableBytesWritable, ImmutableBytesWritable, ImmutableBytesWritable> {
+
+    private TableRecordInfo info;
+    private TableRecord rec;
+    private SliceBuilder builder;
+    private IIKeyValueCodec kv;
+
+    @Override
+    protected void setup(Context context) throws IOException {
+        Configuration conf = context.getConfiguration();
+        KylinConfig config = AbstractHadoopJob.loadKylinPropsAndMetadata(conf);
+        CubeManager mgr = CubeManager.getInstance(config);
+        CubeInstance cube = mgr.getCube(conf.get(BatchConstants.CFG_CUBE_NAME));
+        CubeSegment seg = cube.getSegment(conf.get(BatchConstants.CFG_CUBE_SEGMENT_NAME), CubeSegmentStatusEnum.NEW);
+        info = new TableRecordInfo(seg);
+        rec = new TableRecord(info);
+        builder = null;
+        kv = new IIKeyValueCodec(info);
+    }
+
+    @Override
+    public void reduce(LongWritable key, Iterable<ImmutableBytesWritable> values, Context context) //
+            throws IOException, InterruptedException {
+        for (ImmutableBytesWritable v : values) {
+            rec.setBytes(v.get(), v.getOffset(), v.getLength());
+
+            if (builder == null) {
+                builder = new SliceBuilder(info, rec.getShard());
+            }
+            System.out.println(rec.getShard() + " - " + rec);
+
+            Slice slice = builder.append(rec);
+            if (slice != null) {
+                output(slice, context);
+            }
+        }
+    }
+
+    @Override
+    protected void cleanup(Context context) throws IOException, InterruptedException {
+        Slice slice = builder.close();
+        if (slice != null) {
+            output(slice, context);
+        }
+    }
+
+    private void output(Slice slice, Context context) throws IOException, InterruptedException {
+        for (Pair<ImmutableBytesWritable, ImmutableBytesWritable> pair : kv.encodeKeyValue(slice)) {
+            context.write(pair.getFirst(), pair.getSecond());
+        }
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/job/src/main/java/com/kylinolap/job/hadoop/invertedindex/RandomKeyDistributionJob.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/com/kylinolap/job/hadoop/invertedindex/RandomKeyDistributionJob.java b/job/src/main/java/com/kylinolap/job/hadoop/invertedindex/RandomKeyDistributionJob.java
new file mode 100644
index 0000000..096ae86
--- /dev/null
+++ b/job/src/main/java/com/kylinolap/job/hadoop/invertedindex/RandomKeyDistributionJob.java
@@ -0,0 +1,114 @@
+/*
+ * 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.job.hadoop.invertedindex;
+
+import org.apache.commons.cli.Option;
+import org.apache.commons.cli.OptionBuilder;
+import org.apache.commons.cli.Options;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.io.NullWritable;
+import org.apache.hadoop.mapreduce.Job;
+import org.apache.hadoop.mapreduce.lib.input.SequenceFileInputFormat;
+import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
+import org.apache.hadoop.mapreduce.lib.output.SequenceFileOutputFormat;
+import org.apache.hadoop.util.ToolRunner;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.kylinolap.job.constant.BatchConstants;
+import com.kylinolap.job.hadoop.AbstractHadoopJob;
+
+/**
+ * @author ysong1
+ * 
+ */
+@SuppressWarnings("static-access")
+public class RandomKeyDistributionJob extends AbstractHadoopJob {
+
+    protected static final Logger log = LoggerFactory.getLogger(RandomKeyDistributionJob.class);
+
+    static final Option OPTION_KEY_CLASS = OptionBuilder.withArgName("keyclass").hasArg().isRequired(true).withDescription("Key Class").create("keyclass");
+    static final Option OPTION_REGION_MB = OptionBuilder.withArgName("regionmb").hasArg().isRequired(true).withDescription("MB per Region").create("regionmb");
+
+    @Override
+    public int run(String[] args) throws Exception {
+        Options options = new Options();
+
+        try {
+            options.addOption(OPTION_INPUT_PATH);
+            options.addOption(OPTION_OUTPUT_PATH);
+            options.addOption(OPTION_JOB_NAME);
+            options.addOption(OPTION_KEY_CLASS);
+            options.addOption(OPTION_REGION_MB);
+
+            parseOptions(options, args);
+
+            // start job
+            String jobName = getOptionValue(OPTION_JOB_NAME);
+            job = Job.getInstance(getConf(), jobName);
+
+            job.setJarByClass(this.getClass());
+            addInputDirs(getOptionValue(OPTION_INPUT_PATH), job);
+
+            Path output = new Path(getOptionValue(OPTION_OUTPUT_PATH));
+            FileOutputFormat.setOutputPath(job, output);
+
+            String keyClass = getOptionValue(OPTION_KEY_CLASS);
+            Class<?> keyClz = Class.forName(keyClass);
+
+            int regionMB = Integer.parseInt(getOptionValue(OPTION_REGION_MB));
+
+            // Mapper
+            job.setInputFormatClass(SequenceFileInputFormat.class);
+            job.setMapperClass(RandomKeyDistributionMapper.class);
+            job.setMapOutputKeyClass(keyClz);
+            job.setMapOutputValueClass(NullWritable.class);
+
+            // Reducer - only one
+            job.setReducerClass(RandomKeyDistributionReducer.class);
+            job.setOutputFormatClass(SequenceFileOutputFormat.class);
+            job.setOutputKeyClass(keyClz);
+            job.setOutputValueClass(NullWritable.class);
+            job.setNumReduceTasks(1);
+
+            this.deletePath(job.getConfiguration(), output);
+
+            // total map input MB
+            double totalMapInputMB = this.getTotalMapInputMB();
+            int regionCount = Math.max(1, (int) (totalMapInputMB / regionMB));
+            int mapSampleNumber = 1000;
+            System.out.println("Total Map Input MB: " + totalMapInputMB);
+            System.out.println("Region Count: " + regionCount);
+
+            // set job configuration
+            job.getConfiguration().set(BatchConstants.MAPPER_SAMPLE_NUMBER, String.valueOf(mapSampleNumber));
+            job.getConfiguration().set(BatchConstants.REGION_NUMBER, String.valueOf(regionCount));
+
+            return waitForCompletion(job);
+        } catch (Exception e) {
+            printUsage(options);
+            log.error(e.getLocalizedMessage(), e);
+            return 2;
+        }
+    }
+
+    public static void main(String[] args) throws Exception {
+        int exitCode = ToolRunner.run(new RandomKeyDistributionJob(), args);
+        System.exit(exitCode);
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/job/src/main/java/com/kylinolap/job/hadoop/invertedindex/RandomKeyDistributionMapper.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/com/kylinolap/job/hadoop/invertedindex/RandomKeyDistributionMapper.java b/job/src/main/java/com/kylinolap/job/hadoop/invertedindex/RandomKeyDistributionMapper.java
new file mode 100644
index 0000000..c434f69
--- /dev/null
+++ b/job/src/main/java/com/kylinolap/job/hadoop/invertedindex/RandomKeyDistributionMapper.java
@@ -0,0 +1,66 @@
+/*
+ * 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.job.hadoop.invertedindex;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.io.NullWritable;
+import org.apache.hadoop.io.Writable;
+import org.apache.hadoop.mapreduce.Mapper;
+import org.apache.hadoop.util.ReflectionUtils;
+
+import com.kylinolap.common.util.RandomSampler;
+import com.kylinolap.job.constant.BatchConstants;
+
+/**
+ * @author ysong1
+ * 
+ */
+public class RandomKeyDistributionMapper<KEY extends Writable, VALUE> extends Mapper<KEY, VALUE, KEY, NullWritable> {
+
+    private Configuration conf;
+    private int sampleNumber;
+    private List<KEY> allKeys;
+
+    @Override
+    protected void setup(Context context) throws IOException {
+        conf = context.getConfiguration();
+        allKeys = new ArrayList<KEY>();
+        sampleNumber = Integer.parseInt(conf.get(BatchConstants.MAPPER_SAMPLE_NUMBER));
+    }
+
+    @SuppressWarnings("unchecked")
+    @Override
+    public void map(KEY key, VALUE value, Context context) throws IOException, InterruptedException {
+        KEY keyCopy = (KEY) ReflectionUtils.newInstance(key.getClass(), conf);
+        ReflectionUtils.copy(conf, key, keyCopy);
+        allKeys.add(keyCopy);
+    }
+
+    @Override
+    protected void cleanup(Context context) throws IOException, InterruptedException {
+        RandomSampler<KEY> sampler = new RandomSampler<KEY>();
+        List<KEY> sampleResult = sampler.sample(allKeys, sampleNumber);
+        for (KEY k : sampleResult) {
+            context.write(k, NullWritable.get());
+        }
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/job/src/main/java/com/kylinolap/job/hadoop/invertedindex/RandomKeyDistributionReducer.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/com/kylinolap/job/hadoop/invertedindex/RandomKeyDistributionReducer.java b/job/src/main/java/com/kylinolap/job/hadoop/invertedindex/RandomKeyDistributionReducer.java
new file mode 100644
index 0000000..f5475f2
--- /dev/null
+++ b/job/src/main/java/com/kylinolap/job/hadoop/invertedindex/RandomKeyDistributionReducer.java
@@ -0,0 +1,63 @@
+/*
+ * 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.job.hadoop.invertedindex;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.io.NullWritable;
+import org.apache.hadoop.io.Writable;
+import org.apache.hadoop.mapreduce.Reducer;
+import org.apache.hadoop.util.ReflectionUtils;
+
+import com.kylinolap.job.constant.BatchConstants;
+
+/**
+ * @author ysong1
+ * 
+ */
+public class RandomKeyDistributionReducer<KEY extends Writable> extends Reducer<KEY, NullWritable, KEY, NullWritable> {
+
+    private Configuration conf;
+    private int regionNumber;
+    private List<KEY> allSplits;
+
+    @Override
+    protected void setup(Context context) throws IOException {
+        conf = context.getConfiguration();
+        allSplits = new ArrayList<KEY>();
+        regionNumber = Integer.parseInt(context.getConfiguration().get(BatchConstants.REGION_NUMBER));
+    }
+
+    @SuppressWarnings("unchecked")
+    @Override
+    public void reduce(KEY key, Iterable<NullWritable> values, Context context) throws IOException, InterruptedException {
+        KEY keyCopy = (KEY) ReflectionUtils.newInstance(key.getClass(), conf);
+        ReflectionUtils.copy(conf, key, keyCopy);
+        allSplits.add(keyCopy);
+    }
+
+    @Override
+    protected void cleanup(Context context) throws IOException, InterruptedException {
+        int stepLength = allSplits.size() / regionNumber;
+        for (int i = stepLength; i < allSplits.size(); i += stepLength) {
+            context.write(allSplits.get(i), NullWritable.get());
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/job/src/main/java/com/kylinolap/job/tools/CubeMigrationCLI.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/com/kylinolap/job/tools/CubeMigrationCLI.java b/job/src/main/java/com/kylinolap/job/tools/CubeMigrationCLI.java
new file mode 100644
index 0000000..28237ca
--- /dev/null
+++ b/job/src/main/java/com/kylinolap/job/tools/CubeMigrationCLI.java
@@ -0,0 +1,418 @@
+package com.kylinolap.job.tools;
+
+import com.kylinolap.common.KylinConfig;
+import com.kylinolap.common.persistence.JsonSerializer;
+import com.kylinolap.common.persistence.ResourceStore;
+import com.kylinolap.common.persistence.Serializer;
+import com.kylinolap.cube.*;
+import com.kylinolap.cube.project.ProjectInstance;
+import com.kylinolap.dict.DictionaryInfo;
+import com.kylinolap.dict.DictionaryManager;
+import com.kylinolap.dict.lookup.SnapshotManager;
+import com.kylinolap.dict.lookup.SnapshotTable;
+import com.kylinolap.job.JobInstance;
+import com.kylinolap.metadata.model.cube.CubeDesc;
+import com.kylinolap.metadata.model.schema.TableDesc;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.HBaseConfiguration;
+import org.apache.hadoop.hbase.HTableDescriptor;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.HBaseAdmin;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+
+/**
+ * Created by honma on 9/3/14.
+ * <p/>
+ * This tool serves for the purpose of migrating cubes. e.g. upgrade cube from
+ * dev env to test(prod) env, or vice versa.
+ * <p/>
+ * Note that different envs are assumed to share the same hadoop cluster,
+ * including hdfs, hbase and hive.
+ */
+public class CubeMigrationCLI {
+
+    private static final Logger logger = LoggerFactory.getLogger(CubeMigrationCLI.class);
+
+    private static List<Opt> operations;
+    private static KylinConfig srcConfig;
+    private static KylinConfig dstConfig;
+    private static ResourceStore srcStore;
+    private static ResourceStore dstStore;
+    private static FileSystem hdfsFS;
+    private static HBaseAdmin hbaseAdmin;
+
+    public static void main(String[] args) throws IOException, InterruptedException {
+
+        if (args.length != 6) {
+            usage();
+            System.exit(1);
+        }
+
+        moveCube(args[0], args[1], args[2], args[3], args[4], args[5]);
+    }
+
+    private static void usage() {
+        System.out.println("Usage: CubeMigrationCLI srcKylinConfigUri dstKylinConfigUri cubeName projectName overwriteIfExists realExecute");
+        System.out.println(
+                " srcKylinConfigUri: The KylinConfig of the cube’s source \n" +
+                        "dstKylinConfigUri: The KylinConfig of the cube’s new home \n" +
+                        "cubeName: the name of cube to be migrated. \n" +
+                        "projectName: The target project in the target environment.(Make sure it exist) \n" +
+                        "overwriteIfExists: overwrite cube if it already exists in the target environment. \n" +
+                        "realExecute: if false, just print the operations to take, if true, do the real migration. \n");
+
+    }
+
+    public static void moveCube(KylinConfig srcCfg, KylinConfig dstCfg, String cubeName, String projectName, String overwriteIfExists, String realExecute) throws IOException, InterruptedException {
+
+        srcConfig = srcCfg;
+        srcStore = ResourceStore.getStore(srcConfig);
+        dstConfig = dstCfg;
+        dstStore = ResourceStore.getStore(dstConfig);
+
+        CubeManager cubeManager = CubeManager.getInstance(srcConfig);
+        CubeInstance cube = cubeManager.getCube(cubeName);
+        logger.info("cube to be moved is : " + cubeName);
+
+        if (cube.getStatus() != CubeStatusEnum.READY)
+            throw new IllegalStateException("Cannot migrate cube that is not in READY state.");
+
+        for (CubeSegment segment : cube.getSegments()) {
+            if (segment.getStatus() != CubeSegmentStatusEnum.READY) {
+                throw new IllegalStateException("At least one segment is not in READY state");
+            }
+        }
+
+        checkAndGetHbaseUrl();
+
+        Configuration conf = HBaseConfiguration.create();
+        hbaseAdmin = new HBaseAdmin(conf);
+
+        hdfsFS = FileSystem.get(new Configuration());
+
+        operations = new ArrayList<Opt>();
+
+        copyFilesInMetaStore(cube, overwriteIfExists);
+        renameFoldersInHdfs(cube);
+        changeHtableHost(cube);
+        addCubeIntoProject(cubeName, projectName);
+
+        if (realExecute.equalsIgnoreCase("true")) {
+            doOpts();
+        } else {
+            showOpts();
+        }
+    }
+
+    public static void moveCube(String srcCfgUri, String dstCfgUri, String cubeName, String projectName, String overwriteIfExists, String realExecute) throws IOException, InterruptedException {
+
+        moveCube(KylinConfig.createInstanceFromUri(srcCfgUri), KylinConfig.createInstanceFromUri(dstCfgUri), cubeName, projectName, overwriteIfExists, realExecute);
+    }
+
+    private static String checkAndGetHbaseUrl() {
+        String srcMetadataUrl = srcConfig.getMetadataUrl();
+        String dstMetadataUrl = dstConfig.getMetadataUrl();
+
+        logger.info("src metadata url is " + srcMetadataUrl);
+        logger.info("dst metadata url is " + dstMetadataUrl);
+
+        int srcIndex = srcMetadataUrl.toLowerCase().indexOf("hbase:");
+        int dstIndex = dstMetadataUrl.toLowerCase().indexOf("hbase:");
+        if (srcIndex < 0 || dstIndex < 0)
+            throw new IllegalStateException("Both metadata urls should be hbase metadata url");
+
+        String srcHbaseUrl = srcMetadataUrl.substring(srcIndex).trim();
+        String dstHbaseUrl = dstMetadataUrl.substring(dstIndex).trim();
+        if (!srcHbaseUrl.equalsIgnoreCase(dstHbaseUrl)) {
+            throw new IllegalStateException("hbase url not equal! ");
+        }
+
+        logger.info("hbase url is " + srcHbaseUrl.trim());
+        return srcHbaseUrl.trim();
+    }
+
+    private static void renameFoldersInHdfs(CubeInstance cube) {
+        for (CubeSegment segment : cube.getSegments()) {
+
+            String jobUuid = segment.getLastBuildJobID();
+            String src = JobInstance.getJobWorkingDir(jobUuid, srcConfig.getHdfsWorkingDirectory());
+            String tgt = JobInstance.getJobWorkingDir(jobUuid, dstConfig.getHdfsWorkingDirectory());
+
+            operations.add(new Opt(OptType.RENAME_FOLDER_IN_HDFS, new Object[] { src, tgt }));
+        }
+
+    }
+
+    private static void changeHtableHost(CubeInstance cube) {
+        for (CubeSegment segment : cube.getSegments()) {
+            operations.add(new Opt(OptType.CHANGE_HTABLE_HOST,
+                    new Object[] { segment.getStorageLocationIdentifier() }));
+        }
+    }
+
+    private static void copyFilesInMetaStore(CubeInstance cube, String overwriteIfExists) throws IOException {
+
+        List<String> metaItems = new ArrayList<String>();
+        List<String> dictAndSnapshot = new ArrayList<String>();
+        listCubeRelatedResources(cube, metaItems, dictAndSnapshot);
+
+        if (dstStore.exists(cube.getResourcePath()) && !overwriteIfExists.equalsIgnoreCase("true"))
+            throw new IllegalStateException("The cube named " + cube.getName() + " already exists on target metadata store. Use overwriteIfExists to overwrite it");
+
+        for (String item : metaItems) {
+            operations.add(new Opt(OptType.COPY_FILE_IN_META, new Object[] { item }));
+        }
+
+        for (String item : dictAndSnapshot) {
+            operations.add(new Opt(OptType.COPY_DICT_OR_SNAPSHOT, new Object[] { item, cube.getName() }));
+        }
+    }
+
+    private static void addCubeIntoProject(String cubeName, String projectName) throws IOException {
+        String projectResPath = ProjectInstance.concatResourcePath(projectName);
+        if (!dstStore.exists(projectResPath))
+            throw new IllegalStateException("The target project " + projectName + "does not exist");
+
+        operations.add(new Opt(OptType.ADD_INTO_PROJECT, new Object[] { cubeName, projectName }));
+    }
+
+    private static void listCubeRelatedResources(CubeInstance cube, List<String> metaResource, List<String> dictAndSnapshot) throws IOException {
+
+        CubeDesc cubeDesc = cube.getDescriptor();
+        metaResource.add(cube.getResourcePath());
+        metaResource.add(cubeDesc.getResourcePath());
+
+        for (TableDesc tableDesc : cubeDesc.listTables()) {
+            metaResource.add(tableDesc.getResourcePath());
+        }
+
+        for (CubeSegment segment : cube.getSegments()) {
+            dictAndSnapshot.addAll(segment.getSnapshotPaths());
+            dictAndSnapshot.addAll(segment.getDictionaryPaths());
+        }
+    }
+
+    private static enum OptType {
+        COPY_FILE_IN_META, COPY_DICT_OR_SNAPSHOT, RENAME_FOLDER_IN_HDFS, ADD_INTO_PROJECT, CHANGE_HTABLE_HOST
+    }
+
+    private static class Opt {
+        private OptType type;
+        private Object[] params;
+
+        private Opt(OptType type, Object[] params) {
+            this.type = type;
+            this.params = params;
+        }
+
+        public String toString() {
+            StringBuilder sb = new StringBuilder();
+            sb.append(type).append(":");
+            for (Object s : params)
+                sb.append(s).append(", ");
+            return sb.toString();
+        }
+
+    }
+
+    private static void showOpts() {
+        for (int i = 0; i < operations.size(); ++i) {
+            showOpt(operations.get(i));
+        }
+    }
+
+    private static void showOpt(Opt opt) {
+        logger.info("Operation: " + opt.toString());
+    }
+
+    private static void doOpts() throws IOException, InterruptedException {
+        int index = 0;
+        try {
+            for (; index < operations.size(); ++index) {
+                logger.info("Operation index :" + index);
+                doOpt(operations.get(index));
+            }
+        } catch (Exception e) {
+            logger.error("error met", e);
+            logger.info("Try undoing previous changes");
+            // undo:
+            for (int i = index; i >= 0; --i) {
+                try {
+                    undo(operations.get(i));
+                } catch (Exception ee) {
+                    logger.error("error met ", e);
+                    logger.info("Continue undoing...");
+                }
+            }
+
+            throw new RuntimeException("Cube moving failed");
+        }
+    }
+
+    private static void doOpt(Opt opt) throws IOException, InterruptedException {
+        logger.info("Executing operation: " + opt.toString());
+
+        switch (opt.type) {
+        case CHANGE_HTABLE_HOST: {
+            String tableName = (String) opt.params[0];
+            HTableDescriptor desc = hbaseAdmin.getTableDescriptor(TableName.valueOf(tableName));
+            hbaseAdmin.disableTable(tableName);
+            desc.setValue(CubeManager.getHtableMetadataKey(), dstConfig.getMetadataUrlPrefix());
+            hbaseAdmin.modifyTable(tableName, desc);
+            hbaseAdmin.enableTable(tableName);
+            logger.info("CHANGE_HTABLE_HOST is completed");
+            break;
+        }
+        case COPY_FILE_IN_META: {
+            String item = (String) opt.params[0];
+            InputStream inputStream = srcStore.getResource(item);
+            long ts = srcStore.getResourceTimestamp(item);
+            dstStore.putResource(item, inputStream, ts);
+            inputStream.close();
+            logger.info("Item " + item + " is copied");
+            break;
+        }
+        case COPY_DICT_OR_SNAPSHOT: {
+            String item = (String) opt.params[0];
+
+            if (item.toLowerCase().endsWith(".dict")) {
+                DictionaryManager dstDictMgr = DictionaryManager.getInstance(dstConfig);
+                DictionaryManager srcDicMgr = DictionaryManager.getInstance(srcConfig);
+                DictionaryInfo dictSrc = srcDicMgr.getDictionaryInfo(item);
+
+                long ts = dictSrc.getLastModified();
+                dictSrc.setLastModified(0);//to avoid resource store write conflict
+                DictionaryInfo dictSaved = dstDictMgr.trySaveNewDict(dictSrc.getDictionaryObject(), dictSrc);
+                dictSrc.setLastModified(ts);
+
+                if (dictSaved == dictSrc) {
+                    //no dup found, already saved to dest
+                    logger.info("Item " + item + " is copied");
+                } else {
+                    //dictSrc is rejected because of duplication
+                    //modify cube's dictionary path
+                    String cubeName = (String) opt.params[1];
+                    String cubeResPath = CubeInstance.concatResourcePath(cubeName);
+                    Serializer<CubeInstance> cubeSerializer = new JsonSerializer<CubeInstance>(CubeInstance.class);
+                    CubeInstance cube = dstStore.getResource(cubeResPath, CubeInstance.class, cubeSerializer);
+                    for (CubeSegment segment : cube.getSegments()) {
+                        for (Map.Entry<String, String> entry : segment.getDictionaries().entrySet()) {
+                            if (entry.getValue().equalsIgnoreCase(item)) {
+                                entry.setValue(dictSaved.getResourcePath());
+                            }
+                        }
+                    }
+                    dstStore.putResource(cubeResPath, cube, cubeSerializer);
+                    logger.info("Item " + item + " is dup, instead " + dictSaved.getResourcePath() + " is reused");
+                }
+
+            } else if (item.toLowerCase().endsWith(".snapshot")) {
+                SnapshotManager dstSnapMgr = SnapshotManager.getInstance(dstConfig);
+                SnapshotManager srcSnapMgr = SnapshotManager.getInstance(srcConfig);
+                SnapshotTable snapSrc = srcSnapMgr.getSnapshotTable(item);
+
+                long ts = snapSrc.getLastModified();
+                snapSrc.setLastModified(0);
+                SnapshotTable snapSaved = dstSnapMgr.trySaveNewSnapshot(snapSrc);
+                snapSrc.setLastModified(ts);
+
+
+                if (snapSaved == snapSrc) {
+                    //no dup found, already saved to dest
+                    logger.info("Item " + item + " is copied");
+
+                } else {
+                    String cubeName = (String) opt.params[1];
+                    String cubeResPath = CubeInstance.concatResourcePath(cubeName);
+                    Serializer<CubeInstance> cubeSerializer = new JsonSerializer<CubeInstance>(CubeInstance.class);
+                    CubeInstance cube = dstStore.getResource(cubeResPath, CubeInstance.class, cubeSerializer);
+                    for (CubeSegment segment : cube.getSegments()) {
+                        for (Map.Entry<String, String> entry : segment.getSnapshots().entrySet()) {
+                            if (entry.getValue().equalsIgnoreCase(item)) {
+                                entry.setValue(snapSaved.getResourcePath());
+                            }
+                        }
+                    }
+                    dstStore.putResource(cubeResPath, cube, cubeSerializer);
+                    logger.info("Item " + item + " is dup, instead " + snapSaved.getResourcePath() + " is reused");
+
+                }
+
+            } else {
+                logger.error("unknown item found: " + item);
+                logger.info("ignore it");
+            }
+
+            break;
+        }
+        case RENAME_FOLDER_IN_HDFS: {
+            String srcPath = (String) opt.params[0];
+            String dstPath = (String) opt.params[1];
+            hdfsFS.rename(new Path(srcPath), new Path(dstPath));
+            logger.info("HDFS Folder renamed from " + srcPath + " to " + dstPath);
+            break;
+        }
+        case ADD_INTO_PROJECT: {
+            String cubeName = (String) opt.params[0];
+            String projectName = (String) opt.params[1];
+            String projectResPath = ProjectInstance.concatResourcePath(projectName);
+            Serializer<ProjectInstance> projectSerializer = new JsonSerializer<ProjectInstance>(ProjectInstance.class);
+            ProjectInstance project = dstStore.getResource(projectResPath, ProjectInstance.class, projectSerializer);
+            project.removeCube(cubeName);
+            project.addCube(cubeName);
+            dstStore.putResource(projectResPath, project, projectSerializer);
+            logger.info("Project instance for " + projectName + " is corrected");
+            break;
+        }
+        }
+    }
+
+    private static void undo(Opt opt) throws IOException, InterruptedException {
+        logger.info("Undo operation: " + opt.toString());
+
+        switch (opt.type) {
+        case CHANGE_HTABLE_HOST: {
+            String tableName = (String) opt.params[0];
+            HTableDescriptor desc = hbaseAdmin.getTableDescriptor(TableName.valueOf(tableName));
+            hbaseAdmin.disableTable(tableName);
+            desc.setValue(CubeManager.getHtableMetadataKey(), srcConfig.getMetadataUrlPrefix());
+            hbaseAdmin.modifyTable(tableName, desc);
+            hbaseAdmin.enableTable(tableName);
+            break;
+        }
+        case COPY_FILE_IN_META: {
+            // no harm
+            logger.info("Undo for COPY_FILE_IN_META is ignored");
+            break;
+        }
+        case COPY_DICT_OR_SNAPSHOT: {
+            // no harm
+            logger.info("Undo for COPY_DICT_OR_SNAPSHOT is ignored");
+            break;
+        }
+        case RENAME_FOLDER_IN_HDFS: {
+            String srcPath = (String) opt.params[1];
+            String dstPath = (String) opt.params[0];
+
+            if (hdfsFS.exists(new Path(srcPath)) && !hdfsFS.exists(new Path(dstPath))) {
+                hdfsFS.rename(new Path(srcPath), new Path(dstPath));
+                logger.info("HDFS Folder renamed from " + srcPath + " to " + dstPath);
+            }
+            break;
+        }
+        case ADD_INTO_PROJECT: {
+            logger.info("Undo for ADD_INTO_PROJECT is ignored");
+            break;
+        }
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/job/src/main/java/com/kylinolap/job/tools/DefaultSslProtocolSocketFactory.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/com/kylinolap/job/tools/DefaultSslProtocolSocketFactory.java b/job/src/main/java/com/kylinolap/job/tools/DefaultSslProtocolSocketFactory.java
new file mode 100644
index 0000000..8c81e73
--- /dev/null
+++ b/job/src/main/java/com/kylinolap/job/tools/DefaultSslProtocolSocketFactory.java
@@ -0,0 +1,148 @@
+/*
+ * 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.job.tools;
+
+import java.io.IOException;
+import java.net.InetAddress;
+import java.net.Socket;
+import java.net.UnknownHostException;
+
+import javax.net.ssl.SSLContext;
+import javax.net.ssl.TrustManager;
+
+import org.apache.commons.httpclient.ConnectTimeoutException;
+import org.apache.commons.httpclient.HttpClientError;
+import org.apache.commons.httpclient.params.HttpConnectionParams;
+import org.apache.commons.httpclient.protocol.ControllerThreadSocketFactory;
+import org.apache.commons.httpclient.protocol.SecureProtocolSocketFactory;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * @author xduo
+ * 
+ */
+public class DefaultSslProtocolSocketFactory implements SecureProtocolSocketFactory {
+    /** Log object for this class. */
+    private static Logger LOG = LoggerFactory.getLogger(DefaultSslProtocolSocketFactory.class);
+    private SSLContext sslcontext = null;
+
+    /**
+     * Constructor for DefaultSslProtocolSocketFactory.
+     */
+    public DefaultSslProtocolSocketFactory() {
+        super();
+    }
+
+    /**
+     * @see SecureProtocolSocketFactory#createSocket(java.lang.String,int,java.net.InetAddress,int)
+     */
+    public Socket createSocket(String host, int port, InetAddress clientHost, int clientPort) throws IOException, UnknownHostException {
+        return getSSLContext().getSocketFactory().createSocket(host, port, clientHost, clientPort);
+    }
+
+    /**
+     * Attempts to get a new socket connection to the given host within the
+     * given time limit.
+     * 
+     * <p>
+     * To circumvent the limitations of older JREs that do not support connect
+     * timeout a controller thread is executed. The controller thread attempts
+     * to create a new socket within the given limit of time. If socket
+     * constructor does not return until the timeout expires, the controller
+     * terminates and throws an {@link ConnectTimeoutException}
+     * </p>
+     * 
+     * @param host
+     *            the host name/IP
+     * @param port
+     *            the port on the host
+     * @param localAddress
+     *            the local host name/IP to bind the socket to
+     * @param localPort
+     *            the port on the local machine
+     * @param params
+     *            {@link HttpConnectionParams Http connection parameters}
+     * 
+     * @return Socket a new socket
+     * 
+     * @throws IOException
+     *             if an I/O error occurs while creating the socket
+     * @throws UnknownHostException
+     *             if the IP address of the host cannot be determined
+     * @throws ConnectTimeoutException
+     *             DOCUMENT ME!
+     * @throws IllegalArgumentException
+     *             DOCUMENT ME!
+     */
+    public Socket createSocket(final String host, final int port, final InetAddress localAddress, final int localPort, final HttpConnectionParams params) throws IOException, UnknownHostException, ConnectTimeoutException {
+        if (params == null) {
+            throw new IllegalArgumentException("Parameters may not be null");
+        }
+
+        int timeout = params.getConnectionTimeout();
+
+        if (timeout == 0) {
+            return createSocket(host, port, localAddress, localPort);
+        } else {
+            // To be eventually deprecated when migrated to Java 1.4 or above
+            return ControllerThreadSocketFactory.createSocket(this, host, port, localAddress, localPort, timeout);
+        }
+    }
+
+    /**
+     * @see SecureProtocolSocketFactory#createSocket(java.lang.String,int)
+     */
+    public Socket createSocket(String host, int port) throws IOException, UnknownHostException {
+        return getSSLContext().getSocketFactory().createSocket(host, port);
+    }
+
+    /**
+     * @see SecureProtocolSocketFactory#createSocket(java.net.Socket,java.lang.String,int,boolean)
+     */
+    public Socket createSocket(Socket socket, String host, int port, boolean autoClose) throws IOException, UnknownHostException {
+        return getSSLContext().getSocketFactory().createSocket(socket, host, port, autoClose);
+    }
+
+    public boolean equals(Object obj) {
+        return ((obj != null) && obj.getClass().equals(DefaultX509TrustManager.class));
+    }
+
+    public int hashCode() {
+        return DefaultX509TrustManager.class.hashCode();
+    }
+
+    private static SSLContext createEasySSLContext() {
+        try {
+            SSLContext context = SSLContext.getInstance("TLS");
+            context.init(null, new TrustManager[] { new DefaultX509TrustManager(null) }, null);
+
+            return context;
+        } catch (Exception e) {
+            LOG.error(e.getMessage(), e);
+            throw new HttpClientError(e.toString());
+        }
+    }
+
+    private SSLContext getSSLContext() {
+        if (this.sslcontext == null) {
+            this.sslcontext = createEasySSLContext();
+        }
+
+        return this.sslcontext;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/job/src/main/java/com/kylinolap/job/tools/DefaultX509TrustManager.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/com/kylinolap/job/tools/DefaultX509TrustManager.java b/job/src/main/java/com/kylinolap/job/tools/DefaultX509TrustManager.java
new file mode 100644
index 0000000..8dedaa6
--- /dev/null
+++ b/job/src/main/java/com/kylinolap/job/tools/DefaultX509TrustManager.java
@@ -0,0 +1,112 @@
+/*
+ * 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.job.tools;
+
+import java.security.KeyStore;
+import java.security.KeyStoreException;
+import java.security.NoSuchAlgorithmException;
+import java.security.cert.CertificateException;
+import java.security.cert.X509Certificate;
+
+import javax.net.ssl.KeyManagerFactory;
+import javax.net.ssl.TrustManager;
+import javax.net.ssl.TrustManagerFactory;
+import javax.net.ssl.X509TrustManager;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * @author xduo
+ * 
+ */
+public class DefaultX509TrustManager implements X509TrustManager {
+
+    /** Log object for this class. */
+    private static Logger LOG = LoggerFactory.getLogger(DefaultX509TrustManager.class);
+    private X509TrustManager standardTrustManager = null;
+
+    /**
+     * Constructor for DefaultX509TrustManager.
+     * 
+     */
+    public DefaultX509TrustManager(KeyStore keystore) throws NoSuchAlgorithmException, KeyStoreException {
+        super();
+
+        TrustManagerFactory factory = TrustManagerFactory.getInstance(KeyManagerFactory.getDefaultAlgorithm());
+        factory.init(keystore);
+
+        TrustManager[] trustmanagers = factory.getTrustManagers();
+
+        if (trustmanagers.length == 0) {
+            throw new NoSuchAlgorithmException("SunX509 trust manager not supported");
+        }
+
+        this.standardTrustManager = (X509TrustManager) trustmanagers[0];
+    }
+
+    public X509Certificate[] getAcceptedIssuers() {
+        return this.standardTrustManager.getAcceptedIssuers();
+    }
+
+    public boolean isClientTrusted(X509Certificate[] certificates) {
+        return true;
+        // return this.standardTrustManager.isClientTrusted(certificates);
+    }
+
+    public boolean isServerTrusted(X509Certificate[] certificates) {
+        if ((certificates != null) && LOG.isDebugEnabled()) {
+            LOG.debug("Server certificate chain:");
+
+            for (int i = 0; i < certificates.length; i++) {
+                if (LOG.isDebugEnabled()) {
+                    LOG.debug("X509Certificate[" + i + "]=" + certificates[i]);
+                }
+            }
+        }
+
+        if ((certificates != null) && (certificates.length == 1)) {
+            X509Certificate certificate = certificates[0];
+
+            try {
+                certificate.checkValidity();
+            } catch (CertificateException e) {
+                LOG.error(e.toString());
+
+                return false;
+            }
+
+            return true;
+        } else {
+            return true;
+            // return this.standardTrustManager.isServerTrusted(certificates);
+        }
+    }
+
+    @Override
+    public void checkClientTrusted(X509Certificate[] chain, String authType) throws CertificateException {
+        // TODO Auto-generated method stub
+
+    }
+
+    @Override
+    public void checkServerTrusted(X509Certificate[] chain, String authType) throws CertificateException {
+        // TODO Auto-generated method stub
+
+    }
+
+}


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

Posted by lu...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/server/src/main/java/com/kylinolap/rest/controller/CubeController.java
----------------------------------------------------------------------
diff --git a/server/src/main/java/com/kylinolap/rest/controller/CubeController.java b/server/src/main/java/com/kylinolap/rest/controller/CubeController.java
new file mode 100644
index 0000000..d952ef4
--- /dev/null
+++ b/server/src/main/java/com/kylinolap/rest/controller/CubeController.java
@@ -0,0 +1,468 @@
+/*
+ * 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.rest.controller;
+
+import java.io.IOException;
+import java.net.UnknownHostException;
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.UUID;
+
+import org.apache.commons.lang.StringUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.springframework.beans.factory.annotation.Autowired;
+import org.springframework.security.access.AccessDeniedException;
+import org.springframework.security.core.Authentication;
+import org.springframework.security.core.context.SecurityContextHolder;
+import org.springframework.security.core.userdetails.UserDetails;
+import org.springframework.stereotype.Controller;
+import org.springframework.web.bind.annotation.PathVariable;
+import org.springframework.web.bind.annotation.RequestBody;
+import org.springframework.web.bind.annotation.RequestMapping;
+import org.springframework.web.bind.annotation.RequestMethod;
+import org.springframework.web.bind.annotation.RequestParam;
+import org.springframework.web.bind.annotation.ResponseBody;
+
+import com.codahale.metrics.annotation.Metered;
+import com.fasterxml.jackson.core.JsonParseException;
+import com.fasterxml.jackson.core.JsonProcessingException;
+import com.fasterxml.jackson.databind.JsonMappingException;
+import com.kylinolap.common.util.JsonUtil;
+import com.kylinolap.cube.CubeBuildTypeEnum;
+import com.kylinolap.cube.CubeInstance;
+import com.kylinolap.cube.CubeSegment;
+import com.kylinolap.cube.exception.CubeIntegrityException;
+import com.kylinolap.cube.project.ProjectInstance;
+import com.kylinolap.job.JobInstance;
+import com.kylinolap.job.exception.InvalidJobInstanceException;
+import com.kylinolap.job.exception.JobException;
+import com.kylinolap.metadata.model.cube.CubeDesc;
+import com.kylinolap.rest.exception.BadRequestException;
+import com.kylinolap.rest.exception.ForbiddenException;
+import com.kylinolap.rest.exception.InternalErrorException;
+import com.kylinolap.rest.exception.NotFoundException;
+import com.kylinolap.rest.request.CubeRequest;
+import com.kylinolap.rest.request.JobBuildRequest;
+import com.kylinolap.rest.response.GeneralResponse;
+import com.kylinolap.rest.response.HBaseResponse;
+import com.kylinolap.rest.service.CubeService;
+import com.kylinolap.rest.service.JobService;
+import com.kylinolap.storage.hbase.observer.CoprocessorEnabler;
+
+/**
+ * CubeController is defined as Restful API entrance for UI.
+ * 
+ * @author jianliu
+ */
+@Controller
+@RequestMapping(value = "/cubes")
+public class CubeController extends BasicController {
+    private static final Logger logger = LoggerFactory.getLogger(CubeController.class);
+
+    @Autowired
+    private CubeService cubeService;
+
+    @Autowired
+    private JobService jobService;
+
+    @RequestMapping(value = "", method = { RequestMethod.GET })
+    @ResponseBody
+    @Metered(name = "listCubes")
+    public List<CubeInstance> getCubes(@RequestParam(value = "cubeName", required = false) String cubeName, @RequestParam(value = "projectName", required = false) String projectName, @RequestParam("limit") Integer limit, @RequestParam("offset") Integer offset) {
+        return cubeService.getCubes(cubeName, projectName, (null == limit) ? 20 : limit, offset);
+    }
+
+    /**
+     * Get hive SQL of the cube
+     * 
+     * @param cubeName
+     *            Cube Name
+     * @return
+     * @throws UnknownHostException
+     * @throws IOException
+     */
+    @RequestMapping(value = "/{cubeName}/segs/{segmentName}/sql", method = { RequestMethod.GET })
+    @ResponseBody
+    public GeneralResponse getSql(@PathVariable String cubeName, @PathVariable String segmentName) {
+        String sql = null;
+        try {
+            sql = cubeService.getJobManager().previewFlatHiveQL(cubeName, segmentName);
+        } catch (JobException e) {
+            logger.error(e.getLocalizedMessage(), e);
+            throw new InternalErrorException(e.getLocalizedMessage());
+        } catch (UnknownHostException e) {
+            logger.error(e.getLocalizedMessage(), e);
+            throw new InternalErrorException(e.getLocalizedMessage());
+        }
+
+        GeneralResponse repsonse = new GeneralResponse();
+        repsonse.setProperty("sql", sql);
+
+        return repsonse;
+    }
+
+    /**
+     * Update cube notify list
+     * 
+     * @param cubeName
+     * @param notifyList
+     * @throws IOException
+     * @throws CubeIntegrityException
+     */
+    @RequestMapping(value = "/{cubeName}/notify_list", method = { RequestMethod.PUT })
+    @ResponseBody
+    public void updateNotifyList(@PathVariable String cubeName, @RequestBody List<String> notifyList) {
+        CubeInstance cube = cubeService.getCubeManager().getCube(cubeName);
+
+        if (cube == null) {
+            throw new InternalErrorException("Cannot find cube " + cubeName);
+        }
+
+        try {
+            cubeService.updateCubeNotifyList(cube, notifyList);
+        } catch (Exception e) {
+            logger.error(e.getLocalizedMessage(), e);
+            throw new InternalErrorException(e.getLocalizedMessage());
+        }
+
+    }
+
+    @RequestMapping(value = "/{cubeName}/cost", method = { RequestMethod.PUT })
+    @ResponseBody
+    @Metered(name = "updateCubeCost")
+    public CubeInstance updateCubeCost(@PathVariable String cubeName, @RequestParam(value = "cost") int cost) {
+        try {
+            return cubeService.updateCubeCost(cubeName, cost);
+        } catch (Exception e) {
+            String message = "Failed to update cube cost: " + cubeName + " : " + cost;
+            logger.error(message, e);
+            throw new InternalErrorException(message + " Caused by: " + e.getMessage(), e);
+        }
+    }
+
+    @RequestMapping(value = "/{cubeName}/coprocessor", method = { RequestMethod.PUT })
+    @ResponseBody
+    public Map<String, Boolean> updateCubeCoprocessor(@PathVariable String cubeName, @RequestParam(value = "force") String force) {
+        try {
+            CoprocessorEnabler.updateCubeOverride(cubeName, force);
+            return CoprocessorEnabler.getCubeOverrides();
+        } catch (Exception e) {
+            String message = "Failed to update cube coprocessor: " + cubeName + " : " + force;
+            logger.error(message, e);
+            throw new InternalErrorException(message + " Caused by: " + e.getMessage(), e);
+        }
+    }
+
+    /**
+     * Force rebuild a cube's lookup table snapshot
+     * 
+     * @throws IOException
+     */
+    @RequestMapping(value = "/{cubeName}/segs/{segmentName}/refresh_lookup", method = { RequestMethod.PUT })
+    @ResponseBody
+    public CubeInstance rebuildLookupSnapshot(@PathVariable String cubeName, @PathVariable String segmentName, @RequestParam(value = "lookupTable") String lookupTable) {
+        try {
+            return cubeService.rebuildLookupSnapshot(cubeName, segmentName, lookupTable);
+        } catch (IOException e) {
+            logger.error(e.getLocalizedMessage(), e);
+            throw new InternalErrorException(e.getLocalizedMessage());
+        }
+    }
+
+    /**
+     * Send a rebuild cube job
+     * 
+     * @param cubeName
+     *            Cube ID
+     * @return
+     * @throws SchedulerException
+     * @throws IOException
+     * @throws InvalidJobInstanceException
+     */
+    @RequestMapping(value = "/{cubeName}/rebuild", method = { RequestMethod.PUT })
+    @ResponseBody
+    public JobInstance rebuild(@PathVariable String cubeName, @RequestBody JobBuildRequest jobBuildRequest) {
+        JobInstance jobInstance = null;
+        try {
+            CubeInstance cube = jobService.getCubeManager().getCube(cubeName);
+            
+            String submitter = SecurityContextHolder.getContext().getAuthentication().getName();
+
+            String jobId = jobService.submitJob(cube, jobBuildRequest.getStartTime(), jobBuildRequest.getEndTime(), CubeBuildTypeEnum.valueOf(jobBuildRequest.getBuildType()),submitter);
+            jobInstance = jobService.getJobInstance(jobId);
+        } catch (JobException e) {
+            logger.error(e.getLocalizedMessage(), e);
+            throw new InternalErrorException(e.getLocalizedMessage());
+        } catch (IOException e) {
+            logger.error(e.getLocalizedMessage(), e);
+            throw new InternalErrorException(e.getLocalizedMessage());
+        } catch (InvalidJobInstanceException e) {
+            logger.error(e.getLocalizedMessage(), e);
+            throw new InternalErrorException(e.getLocalizedMessage());
+        }
+
+        return jobInstance;
+    }
+
+    @RequestMapping(value = "/{cubeName}/disable", method = { RequestMethod.PUT })
+    @ResponseBody
+    @Metered(name = "disableCube")
+    public CubeInstance disableCube(@PathVariable String cubeName) {
+        try {
+            CubeInstance cube = cubeService.getCubeManager().getCube(cubeName);
+
+            if (cube == null) {
+                throw new InternalErrorException("Cannot find cube " + cubeName);
+            }
+
+            return cubeService.disableCube(cube);
+        } catch (Exception e) {
+            String message = "Failed to disable cube: " + cubeName;
+            logger.error(message, e);
+            throw new InternalErrorException(message + " Caused by: " + e.getMessage(), e);
+        }
+    }
+
+    @RequestMapping(value = "/{cubeName}/purge", method = { RequestMethod.PUT })
+    @ResponseBody
+    @Metered(name = "purgeCube")
+    public CubeInstance purgeCube(@PathVariable String cubeName) {
+        try {
+            CubeInstance cube = cubeService.getCubeManager().getCube(cubeName);
+
+            if (cube == null) {
+                throw new InternalErrorException("Cannot find cube " + cubeName);
+            }
+
+            return cubeService.purgeCube(cube);
+        } catch (Exception e) {
+            String message = "Failed to purge cube: " + cubeName;
+            logger.error(message, e);
+            throw new InternalErrorException(message + " Caused by: " + e.getMessage(), e);
+        }
+    }
+
+
+    @RequestMapping(value = "/{cubeName}/enable", method = { RequestMethod.PUT })
+    @ResponseBody
+    @Metered(name = "enableCube")
+    public CubeInstance enableCube(@PathVariable String cubeName) {
+        try {
+            CubeInstance cube = cubeService.getCubeManager().getCube(cubeName);
+            if (null == cube) {
+                throw new InternalErrorException("Cannot find cube " + cubeName);
+            }
+
+            return cubeService.enableCube(cube);
+        } catch (Exception e) {
+            String message = "Failed to enable cube: " + cubeName;
+            logger.error(message, e);
+            throw new InternalErrorException(message + " Caused by: " + e.getMessage(), e);
+        }
+    }
+
+    @RequestMapping(value = "/{cubeName}", method = { RequestMethod.DELETE })
+    @ResponseBody
+    @Metered(name = "deleteCube")
+    public void deleteCube(@PathVariable String cubeName) {
+        CubeInstance cube = cubeService.getCubeManager().getCube(cubeName);
+        if (null == cube) {
+            throw new NotFoundException("Cube with name " + cubeName + " not found..");
+        }
+
+        try {
+            cubeService.deleteCube(cube);
+        } catch (Exception e) {
+            logger.error(e.getLocalizedMessage(), e);
+            throw new InternalErrorException("Failed to delete cube. " + " Caused by: " + e.getMessage(), e);
+        }
+    }
+
+    /**
+     * Get available table list of the input database
+     * 
+     * @return Table metadata array
+     * @throws IOException
+     */
+    @RequestMapping(value = "", method = { RequestMethod.POST })
+    @ResponseBody
+    @Metered(name = "saveCube")
+    public CubeRequest saveCubeDesc(@RequestBody CubeRequest cubeRequest) {
+        CubeDesc desc = deserializeCubeDesc(cubeRequest);
+        if (desc == null) {
+            return cubeRequest;
+        }
+
+        String name = CubeService.getCubeNameFromDesc(desc.getName());
+        if (StringUtils.isEmpty(name)) {
+            logger.info("Cube name should not be empty.");
+            throw new BadRequestException("Cube name should not be empty.");
+        }
+
+        try {
+            desc.setUuid(UUID.randomUUID().toString());
+            String projectName = (null == cubeRequest.getProject()) ? ProjectInstance.DEFAULT_PROJECT_NAME : cubeRequest.getProject();
+            cubeService.createCubeAndDesc(name, projectName, desc);
+        } catch (Exception e) {
+            logger.error("Failed to deal with the request.", e);
+            throw new InternalErrorException(e.getLocalizedMessage(),e);
+        }
+
+        cubeRequest.setUuid(desc.getUuid());
+        cubeRequest.setSuccessful(true);
+        return cubeRequest;
+    }
+
+    /**
+     * Get available table list of the input database
+     * 
+     * @return Table metadata array
+     * @throws JsonProcessingException 
+     * @throws IOException
+     */
+    @RequestMapping(value = "", method = { RequestMethod.PUT })
+    @ResponseBody
+    @Metered(name = "updateCube")
+    public CubeRequest updateCubeDesc(@RequestBody CubeRequest cubeRequest) throws JsonProcessingException {
+        CubeDesc desc = deserializeCubeDesc(cubeRequest);
+
+        if (desc == null) {
+            return cubeRequest;
+        }
+
+        // Check if the cube is editable
+        if (!cubeService.isCubeDescEditable(desc)) {
+            String error = "Cube desc " + desc.getName().toUpperCase() + " is not editable.";
+            updateRequest(cubeRequest, false, error);
+            return cubeRequest;
+        }
+
+        try {
+            CubeInstance cube = cubeService.getCubeManager().getCube(cubeRequest.getCubeName());
+            String projectName = (null == cubeRequest.getProject()) ? ProjectInstance.DEFAULT_PROJECT_NAME : cubeRequest.getProject();
+            desc = cubeService.updateCubeAndDesc(cube, desc, projectName);
+
+        } catch (AccessDeniedException accessDeniedException) {
+            throw new ForbiddenException("You don't have right to update this cube.");
+        } catch (Exception e) {
+            logger.error("Failed to deal with the request.", e);
+            throw new InternalErrorException("Failed to deal with the request: " + e.getMessage());
+        }
+
+        if (desc.getError().isEmpty()) {
+            cubeRequest.setSuccessful(true);
+        } else {
+            logger.warn("Cube " + desc.getName() + " fail to create because " + desc.getError());
+            updateRequest(cubeRequest, false, omitMessage(desc.getError()));
+        }
+        String descData = JsonUtil.writeValueAsIndentString(desc);
+        cubeRequest.setCubeDescData(descData);
+
+        return cubeRequest;
+    }
+
+    /**
+     * Get available table list of the input database
+     * 
+     * @return true
+     * @throws IOException
+     */
+    @RequestMapping(value = "/{cubeName}/hbase", method = { RequestMethod.GET })
+    @ResponseBody
+    @Metered(name = "getHBaseInfo")
+    public List<HBaseResponse> getHBaseInfo(@PathVariable String cubeName) {
+        List<HBaseResponse> hbase = new ArrayList<HBaseResponse>();
+
+        CubeInstance cube = cubeService.getCubeManager().getCube(cubeName);
+        if (null == cube) {
+            throw new InternalErrorException("Cannot find cube " + cubeName);
+        }
+
+        List<CubeSegment> segments = cube.getSegments();
+
+        for (CubeSegment segment : segments) {
+            String tableName = segment.getStorageLocationIdentifier();
+            HBaseResponse hr = null;
+
+            // Get info of given table.
+            try {
+                hr = cubeService.getHTableInfo(tableName);
+            } catch (IOException e) {
+                logger.error("Failed to calcuate size of HTable \"" + tableName + "\".", e);
+            }
+
+            if (null == hr) {
+                logger.info("Failed to calcuate size of HTable \"" + tableName + "\".");
+                hr = new HBaseResponse();
+            }
+
+            hr.setTableName(tableName);
+            hbase.add(hr);
+        }
+
+        return hbase;
+    }
+
+    private CubeDesc deserializeCubeDesc(CubeRequest cubeRequest) {
+        CubeDesc desc = null;
+        try {
+            logger.debug("Saving cube " + cubeRequest.getCubeDescData());
+            desc = JsonUtil.readValue(cubeRequest.getCubeDescData(), CubeDesc.class);
+        } catch (JsonParseException e) {
+            logger.error("The cube definition is not valid.", e);
+            updateRequest(cubeRequest, false, e.getMessage());
+        } catch (JsonMappingException e) {
+            logger.error("The cube definition is not valid.", e);
+            updateRequest(cubeRequest, false, e.getMessage());
+        } catch (IOException e) {
+            logger.error("Failed to deal with the request.", e);
+            throw new InternalErrorException("Failed to deal with the request:"+e.getMessage(), e);
+        }
+        return desc;
+    }
+
+    /**
+     * @param error
+     * @return
+     */
+    private String omitMessage(List<String> errors) {
+        StringBuffer buffer = new StringBuffer();
+        for (Iterator<String> iterator = errors.iterator(); iterator.hasNext();) {
+            String string = (String) iterator.next();
+            buffer.append(string);
+            buffer.append("\n");
+        }
+        return buffer.toString();
+    }
+
+    private void updateRequest(CubeRequest request, boolean success, String message) {
+        request.setCubeDescData("");
+        request.setSuccessful(success);
+        request.setMessage(message);
+    }
+
+    public void setCubeService(CubeService cubeService) {
+        this.cubeService = cubeService;
+    }
+
+    public void setJobService(JobService jobService) {
+        this.jobService = jobService;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/server/src/main/java/com/kylinolap/rest/controller/CubeDescController.java
----------------------------------------------------------------------
diff --git a/server/src/main/java/com/kylinolap/rest/controller/CubeDescController.java b/server/src/main/java/com/kylinolap/rest/controller/CubeDescController.java
new file mode 100644
index 0000000..173c3b6
--- /dev/null
+++ b/server/src/main/java/com/kylinolap/rest/controller/CubeDescController.java
@@ -0,0 +1,67 @@
+/*
+ * 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.rest.controller;
+
+import java.io.IOException;
+
+import org.springframework.beans.factory.annotation.Autowired;
+import org.springframework.stereotype.Controller;
+import org.springframework.web.bind.annotation.PathVariable;
+import org.springframework.web.bind.annotation.RequestMapping;
+import org.springframework.web.bind.annotation.RequestMethod;
+import org.springframework.web.bind.annotation.ResponseBody;
+
+import com.kylinolap.cube.CubeInstance;
+import com.kylinolap.metadata.model.cube.CubeDesc;
+import com.kylinolap.rest.service.CubeService;
+
+/**
+ * @author xduo
+ * 
+ */
+@Controller
+@RequestMapping(value = "/cube_desc")
+public class CubeDescController {
+
+    @Autowired
+    private CubeService cubeService;
+
+    /**
+     * Get detail information of the "Cube ID"
+     * 
+     * @param cubeDescName
+     *            Cube ID
+     * @return
+     * @throws IOException
+     */
+    @RequestMapping(value = "/{cubeName}", method = { RequestMethod.GET })
+    @ResponseBody
+    public CubeDesc[] getCube(@PathVariable String cubeName) {
+        CubeInstance cubeInstance = cubeService.getCubeManager().getCube(cubeName);
+        CubeDesc cSchema = cubeInstance.getDescriptor();
+        if (cSchema != null) {
+            return new CubeDesc[] { cSchema };
+        } else {
+            return null;
+        }
+    }
+
+    public void setCubeService(CubeService cubeService) {
+        this.cubeService = cubeService;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/server/src/main/java/com/kylinolap/rest/controller/JobController.java
----------------------------------------------------------------------
diff --git a/server/src/main/java/com/kylinolap/rest/controller/JobController.java b/server/src/main/java/com/kylinolap/rest/controller/JobController.java
new file mode 100644
index 0000000..5f26ad9
--- /dev/null
+++ b/server/src/main/java/com/kylinolap/rest/controller/JobController.java
@@ -0,0 +1,226 @@
+/*
+ * 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.rest.controller;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.TimeZone;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.springframework.beans.factory.InitializingBean;
+import org.springframework.beans.factory.annotation.Autowired;
+import org.springframework.stereotype.Controller;
+import org.springframework.web.bind.annotation.PathVariable;
+import org.springframework.web.bind.annotation.RequestMapping;
+import org.springframework.web.bind.annotation.RequestMethod;
+import org.springframework.web.bind.annotation.ResponseBody;
+
+import com.kylinolap.common.KylinConfig;
+import com.kylinolap.job.JobInstance;
+import com.kylinolap.job.JobManager;
+import com.kylinolap.job.constant.JobStatusEnum;
+import com.kylinolap.rest.constant.Constant;
+import com.kylinolap.rest.exception.InternalErrorException;
+import com.kylinolap.rest.request.JobListRequest;
+import com.kylinolap.rest.service.JobService;
+
+/**
+ * @author ysong1
+ * @author Jack
+ * 
+ */
+@Controller
+@RequestMapping(value = "jobs")
+public class JobController extends BasicController implements InitializingBean {
+    private static final Logger logger = LoggerFactory.getLogger(JobController.class);
+
+    @Autowired
+    private JobService jobService;
+
+    /*
+     * (non-Javadoc)
+     * 
+     * @see
+     * org.springframework.beans.factory.InitializingBean#afterPropertiesSet()
+     */
+    @Override
+    public void afterPropertiesSet() throws Exception {
+        String timeZone = jobService.getKylinConfig().getTimeZone();
+        TimeZone tzone = TimeZone.getTimeZone(timeZone);
+        TimeZone.setDefault(tzone);
+
+        String serverMode = KylinConfig.getInstanceFromEnv().getServerMode();
+
+        if (Constant.SERVER_MODE_JOB.equals(serverMode.toLowerCase()) || Constant.SERVER_MODE_ALL.equals(serverMode.toLowerCase())) {
+            logger.info("Initializing Job Engine ....");
+
+            new Thread(new Runnable() {
+                @Override
+                public void run() {
+                    JobManager jobManager = null;
+                    try {
+                        jobManager = jobService.getJobManager();
+                        jobManager.startJobEngine();
+                        metricsService.registerJobMetrics(jobManager);
+                    } catch (Exception e) {
+                        throw new RuntimeException(e);
+                    }
+                }
+            }).start();
+        }
+    }
+
+    /**
+     * get all cube jobs
+     * 
+     * @param cubeName
+     *            Cube ID
+     * @return
+     * @throws IOException
+     */
+    @RequestMapping(value = "", method = { RequestMethod.GET })
+    @ResponseBody
+    public List<JobInstance> list(JobListRequest jobRequest) {
+
+        List<JobInstance> jobInstanceList = Collections.emptyList();
+        List<JobStatusEnum> statusList = new ArrayList<JobStatusEnum>();
+
+        if (null != jobRequest.getStatus()) {
+            for (int status : jobRequest.getStatus()) {
+                statusList.add(JobStatusEnum.getByCode(status));
+            }
+        }
+
+        try {
+            jobInstanceList = jobService.listAllJobs(jobRequest.getCubeName(), jobRequest.getProjectName(), statusList, jobRequest.getLimit(), jobRequest.getOffset());
+        } catch (Exception e) {
+            logger.error(e.getLocalizedMessage(), e);
+            throw new InternalErrorException(e);
+        }
+        return jobInstanceList;
+    }
+
+    /**
+     * Get a cube job
+     * 
+     * @param cubeName
+     *            Cube ID
+     * @return
+     * @throws IOException
+     */
+    @RequestMapping(value = "/{jobId}", method = { RequestMethod.GET })
+    @ResponseBody
+    public JobInstance get(@PathVariable String jobId) {
+        JobInstance jobInstance = null;
+        try {
+            jobInstance = jobService.getJobInstance(jobId);
+        } catch (Exception e) {
+            logger.error(e.getLocalizedMessage(), e);
+            throw new InternalErrorException(e);
+        }
+
+        return jobInstance;
+    }
+
+    /**
+     * Get a job step output
+     * 
+     * @param cubeName
+     *            Cube ID
+     * @return
+     * @throws IOException
+     */
+    @RequestMapping(value = "/{jobId}/steps/{stepId}/output", method = { RequestMethod.GET })
+    @ResponseBody
+    public Map<String, String> getStepOutput(@PathVariable String jobId, @PathVariable int stepId) {
+        Map<String, String> result = new HashMap<String, String>();
+        result.put("jobId", jobId);
+        result.put("stepId", String.valueOf(stepId));
+        long start = System.currentTimeMillis();
+        String output = "";
+
+        try {
+            output = jobService.getJobManager().getJobStepOutput(jobId, stepId);
+        } catch (Exception e) {
+            logger.error(e.getLocalizedMessage(), e);
+            throw new InternalErrorException(e);
+        }
+
+        result.put("cmd_output", output);
+        long end = System.currentTimeMillis();
+        logger.info("Complete fetching step " + jobId + ":" + stepId + " output in " + (end - start) + " seconds");
+        return result;
+    }
+
+    /**
+     * Resume a cube job
+     * 
+     * @param String
+     *            Job ID
+     * @return
+     * @throws IOException
+     */
+    @RequestMapping(value = "/{jobId}/resume", method = { RequestMethod.PUT })
+    @ResponseBody
+    public JobInstance resume(@PathVariable String jobId) {
+        JobInstance jobInstance = null;
+        try {
+            jobInstance = jobService.getJobInstance(jobId);
+            jobService.resumeJob(jobInstance);
+            jobInstance = jobService.getJobInstance(jobId);
+        } catch (Exception e) {
+            logger.error(e.getLocalizedMessage(), e);
+            throw new InternalErrorException(e);
+        }
+
+        return jobInstance;
+    }
+
+    /**
+     * Cancel a job
+     * 
+     * @param String
+     *            Job ID
+     * @return
+     * @throws IOException
+     */
+    @RequestMapping(value = "/{jobId}/cancel", method = { RequestMethod.PUT })
+    @ResponseBody
+    public JobInstance cancel(@PathVariable String jobId) {
+
+        JobInstance jobInstance = null;
+        try {
+            jobInstance = jobService.getJobInstance(jobId);
+            jobService.cancelJob(jobInstance);
+        } catch (Exception e) {
+            logger.error(e.getLocalizedMessage(), e);
+            throw new InternalErrorException(e);
+        }
+
+        return jobInstance;
+    }
+
+    public void setJobService(JobService jobService) {
+        this.jobService = jobService;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/server/src/main/java/com/kylinolap/rest/controller/ProjectController.java
----------------------------------------------------------------------
diff --git a/server/src/main/java/com/kylinolap/rest/controller/ProjectController.java b/server/src/main/java/com/kylinolap/rest/controller/ProjectController.java
new file mode 100644
index 0000000..3b15dc9
--- /dev/null
+++ b/server/src/main/java/com/kylinolap/rest/controller/ProjectController.java
@@ -0,0 +1,118 @@
+/*
+ * 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.rest.controller;
+
+import java.io.IOException;
+import java.util.List;
+
+import org.apache.commons.lang.StringUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.springframework.beans.factory.annotation.Autowired;
+import org.springframework.stereotype.Controller;
+import org.springframework.web.bind.annotation.PathVariable;
+import org.springframework.web.bind.annotation.RequestBody;
+import org.springframework.web.bind.annotation.RequestMapping;
+import org.springframework.web.bind.annotation.RequestMethod;
+import org.springframework.web.bind.annotation.RequestParam;
+import org.springframework.web.bind.annotation.ResponseBody;
+
+import com.codahale.metrics.annotation.Metered;
+import com.kylinolap.cube.project.ProjectInstance;
+import com.kylinolap.rest.exception.InternalErrorException;
+import com.kylinolap.rest.request.CreateProjectRequest;
+import com.kylinolap.rest.request.UpdateProjectRequest;
+import com.kylinolap.rest.service.ProjectService;
+
+/**
+ * @author xduo
+ */
+@Controller
+@RequestMapping(value = "/projects")
+public class ProjectController extends BasicController {
+    private static final Logger logger = LoggerFactory.getLogger(ProjectController.class);
+
+    @Autowired
+    private ProjectService projectService;
+
+    /**
+     * Get available project list
+     * 
+     * @return Table metadata array
+     * @throws IOException
+     */
+    @RequestMapping(value = "", method = { RequestMethod.GET })
+    @ResponseBody
+    public List<ProjectInstance> getProjects(@RequestParam(value = "limit", required = false) Integer limit, @RequestParam(value = "offset", required = false) Integer offset) {
+        return projectService.listAllProjects(limit, offset);
+    }
+
+    @RequestMapping(value = "", method = { RequestMethod.POST })
+    @ResponseBody
+    @Metered(name = "saveProject")
+    public ProjectInstance saveProject(@RequestBody CreateProjectRequest projectRequest) {
+        if (StringUtils.isEmpty(projectRequest.getName())) {
+            throw new InternalErrorException("A project name must be given to create a project");
+        }
+
+        ProjectInstance createdProj = null;
+        try {
+            createdProj = projectService.createProject(projectRequest);
+        } catch (Exception e) {
+            logger.error("Failed to deal with the request.", e);
+            throw new InternalErrorException(e.getLocalizedMessage());
+        }
+
+        return createdProj;
+    }
+
+    @RequestMapping(value = "", method = { RequestMethod.PUT })
+    @ResponseBody
+    @Metered(name = "updateProject")
+    public ProjectInstance updateProject(@RequestBody UpdateProjectRequest projectRequest) {
+        if (StringUtils.isEmpty(projectRequest.getFormerProjectName())) {
+            throw new InternalErrorException("A project name must be given to update a project");
+        }
+
+        ProjectInstance updatedProj = null;
+        try {
+            updatedProj = projectService.updateProject(projectRequest);
+        } catch (Exception e) {
+            logger.error("Failed to deal with the request.", e);
+            throw new InternalErrorException(e.getLocalizedMessage());
+        }
+
+        return updatedProj;
+    }
+
+    @RequestMapping(value = "/{projectName}", method = { RequestMethod.DELETE })
+    @ResponseBody
+    @Metered(name = "deleteProject")
+    public void deleteProject(@PathVariable String projectName) {
+        try {
+            projectService.deleteProject(projectName);
+        } catch (Exception e) {
+            logger.error(e.getLocalizedMessage(), e);
+            throw new InternalErrorException("Failed to delete project. " + " Caused by: " + e.getMessage(), e);
+        }
+    }
+
+    public void setProjectService(ProjectService projectService) {
+        this.projectService = projectService;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/server/src/main/java/com/kylinolap/rest/controller/QueryController.java
----------------------------------------------------------------------
diff --git a/server/src/main/java/com/kylinolap/rest/controller/QueryController.java b/server/src/main/java/com/kylinolap/rest/controller/QueryController.java
new file mode 100644
index 0000000..e080aff
--- /dev/null
+++ b/server/src/main/java/com/kylinolap/rest/controller/QueryController.java
@@ -0,0 +1,266 @@
+/*
+ * Copyright 2013-2014 eBay Software Foundation
+ *
+ * Licensed under the <dependency>
+    <groupId>com.ryantenney.metrics</groupId>
+    <artifactId>metrics-spring</artifactId>
+    <version>3.0.0</version>
+</dependency>che 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.rest.controller;
+
+import java.io.IOException;
+import java.sql.SQLException;
+import java.util.ArrayList;
+import java.util.Date;
+import java.util.List;
+
+import javax.servlet.http.HttpServletResponse;
+
+import net.sf.ehcache.Cache;
+import net.sf.ehcache.CacheManager;
+import net.sf.ehcache.Element;
+
+import org.apache.commons.io.IOUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.springframework.beans.factory.annotation.Autowired;
+import org.springframework.security.access.AccessDeniedException;
+import org.springframework.security.core.context.SecurityContextHolder;
+import org.springframework.stereotype.Controller;
+import org.springframework.web.bind.annotation.PathVariable;
+import org.springframework.web.bind.annotation.RequestBody;
+import org.springframework.web.bind.annotation.RequestMapping;
+import org.springframework.web.bind.annotation.RequestMethod;
+import org.springframework.web.bind.annotation.ResponseBody;
+import org.supercsv.io.CsvListWriter;
+import org.supercsv.io.ICsvListWriter;
+import org.supercsv.prefs.CsvPreference;
+
+import com.codahale.metrics.annotation.Timed;
+import com.kylinolap.common.KylinConfig;
+import com.kylinolap.cube.CubeInstance;
+import com.kylinolap.rest.constant.Constant;
+import com.kylinolap.rest.exception.ForbiddenException;
+import com.kylinolap.rest.exception.InternalErrorException;
+import com.kylinolap.rest.model.Query;
+import com.kylinolap.rest.model.SelectedColumnMeta;
+import com.kylinolap.rest.model.TableMeta;
+import com.kylinolap.rest.request.MetaRequest;
+import com.kylinolap.rest.request.PrepareSqlRequest;
+import com.kylinolap.rest.request.SQLRequest;
+import com.kylinolap.rest.request.SaveSqlRequest;
+import com.kylinolap.rest.response.SQLResponse;
+import com.kylinolap.rest.service.QueryService;
+import com.kylinolap.rest.util.QueryUtil;
+
+/**
+ * Handle query requests.
+ * 
+ * @author xduo
+ */
+@Controller
+public class QueryController extends BasicController {
+
+    private static final Logger logger = LoggerFactory.getLogger(QueryController.class);
+
+    public static final String SUCCESS_QUERY_CACHE = "SuccessQueryCache";
+    public static final String EXCEPTION_QUERY_CACHE = "ExceptionQueryCache";
+
+    @Autowired
+    private QueryService queryService;
+
+    @Autowired
+    private CacheManager cacheManager;
+
+    @RequestMapping(value = "/query", method = RequestMethod.POST)
+    @ResponseBody
+    @Timed(name = "query")
+    public SQLResponse query(@RequestBody SQLRequest sqlRequest) {
+        long startTimestamp = System.currentTimeMillis();
+
+        SQLResponse response = doQuery(sqlRequest);
+        response.setDuration(System.currentTimeMillis() - startTimestamp);
+
+        queryService.logQuery(sqlRequest, response, new Date(startTimestamp), new Date(System.currentTimeMillis()));
+
+        return response;
+    }
+
+    @RequestMapping(value = "/query/prestate", method = RequestMethod.POST, produces = "application/json")
+    @ResponseBody
+    @Timed(name = "query")
+    public SQLResponse prepareQuery(@RequestBody PrepareSqlRequest sqlRequest) {
+        long startTimestamp = System.currentTimeMillis();
+
+        SQLResponse response = doQuery(sqlRequest);
+        response.setDuration(System.currentTimeMillis() - startTimestamp);
+
+        queryService.logQuery(sqlRequest, response, new Date(startTimestamp), new Date(System.currentTimeMillis()));
+
+        if (response.getIsException()) {
+            String errorMsg = response.getExceptionMessage();
+            throw new InternalErrorException(QueryUtil.makeErrorMsgUserFriendly(errorMsg));
+        }
+
+        return response;
+    }
+
+    @RequestMapping(value = "/saved_queries", method = RequestMethod.POST)
+    @ResponseBody
+    @Timed(name = "saveQuery")
+    public void saveQuery(@RequestBody SaveSqlRequest sqlRequest) throws IOException {
+        String creator = SecurityContextHolder.getContext().getAuthentication().getName();
+        Query newQuery = new Query(sqlRequest.getName(), sqlRequest.getProject(), sqlRequest.getSql(), sqlRequest.getDescription());
+
+        queryService.saveQuery(creator, newQuery);
+    }
+
+    @RequestMapping(value = "/saved_queries/{id}", method = RequestMethod.DELETE)
+    @ResponseBody
+    @Timed(name = "removeQuery")
+    public void removeQuery(@PathVariable String id) throws IOException {
+        String creator = SecurityContextHolder.getContext().getAuthentication().getName();
+        queryService.removeQuery(creator, id);
+    }
+
+    @RequestMapping(value = "/saved_queries", method = RequestMethod.GET)
+    @ResponseBody
+    @Timed(name = "getQueries")
+    public List<Query> getQueries() throws IOException {
+        String creator = SecurityContextHolder.getContext().getAuthentication().getName();
+        return queryService.getQueries(creator);
+    }
+
+    @RequestMapping(value = "/query/format/{format}", method = RequestMethod.GET)
+    @ResponseBody
+    @Timed(name = "downloadResult")
+    public void downloadQueryResult(@PathVariable String format, SQLRequest sqlRequest, HttpServletResponse response) {
+        SQLResponse result = doQuery(sqlRequest);
+        response.setContentType("text/" + format + ";charset=utf-8");
+        response.setHeader("Content-Disposition", "attachment; filename=\"result." + format + "\"");
+        ICsvListWriter csvWriter = null;
+
+        try {
+            csvWriter = new CsvListWriter(response.getWriter(), CsvPreference.STANDARD_PREFERENCE);
+
+            List<String> headerList = new ArrayList<String>();
+
+            for (SelectedColumnMeta column : result.getColumnMetas()) {
+                headerList.add(column.getName());
+            }
+
+            String[] headers = new String[headerList.size()];
+            csvWriter.writeHeader(headerList.toArray(headers));
+
+            for (List<String> row : result.getResults()) {
+                csvWriter.write(row);
+            }
+        } catch (IOException e) {
+            logger.error("", e);
+        } finally {
+            IOUtils.closeQuietly(csvWriter);
+        }
+    }
+
+    @RequestMapping(value = "/tables_and_columns", method = RequestMethod.GET)
+    @ResponseBody
+    public List<TableMeta> getMetadata(MetaRequest metaRequest) {
+        try {
+            return queryService.getMetadata(metaRequest.getProject());
+        } catch (SQLException e) {
+            logger.error(e.getLocalizedMessage(), e);
+            throw new InternalErrorException(e.getLocalizedMessage(), e);
+        }
+    }
+
+    private SQLResponse doQuery(SQLRequest sqlRequest) {
+        String sql = sqlRequest.getSql();
+        String project = sqlRequest.getProject();
+        logger.info("Using project: " + project);
+        logger.info("The original query:  " + sql);
+
+        String serverMode = KylinConfig.getInstanceFromEnv().getServerMode();
+        if (!(Constant.SERVER_MODE_QUERY.equals(serverMode.toLowerCase()) || Constant.SERVER_MODE_ALL.equals(serverMode.toLowerCase()))) {
+            throw new InternalErrorException("Query is not allowed in " + serverMode + " mode.");
+        }
+
+        if (sql.toLowerCase().contains("select")) {
+            SQLResponse sqlResponse = searchQueryInCache(sqlRequest);
+            try {
+                if (null == sqlResponse) {
+                    sqlResponse = queryService.query(sqlRequest);
+
+                    long durationThreshold = KylinConfig.getInstanceFromEnv().getQueryDurationCacheThreshold();
+                    long scancountThreshold = KylinConfig.getInstanceFromEnv().getQueryScanCountCacheThreshold();
+                    if (!sqlResponse.getIsException() && (sqlResponse.getDuration() > durationThreshold || sqlResponse.getTotalScanCount() > scancountThreshold)) {
+                        cacheManager.getCache(SUCCESS_QUERY_CACHE).put(new Element(sqlRequest, sqlResponse));
+                    }
+                }
+
+                checkQueryAuth(sqlResponse);
+
+                return sqlResponse;
+            } catch (AccessDeniedException ade) {
+                // Access exception is bind with each user, it will not be
+                // cached.
+                logger.error("Exception when execute sql", ade);
+                throw new ForbiddenException(ade.getLocalizedMessage());
+            } catch (Exception e) {
+                SQLResponse exceptionRes = new SQLResponse(null, null, 0, true, e.getMessage());
+                Cache exceptionCache = cacheManager.getCache(EXCEPTION_QUERY_CACHE);
+                exceptionCache.put(new Element(sqlRequest, exceptionRes));
+
+                logger.error("Exception when execute sql", e);
+                throw new InternalErrorException(QueryUtil.makeErrorMsgUserFriendly(e.getLocalizedMessage()));
+            }
+        } else {
+            logger.debug("Directly return expection as not supported");
+            throw new InternalErrorException(QueryUtil.makeErrorMsgUserFriendly("Not Supported SQL."));
+        }
+    }
+
+    private SQLResponse searchQueryInCache(SQLRequest sqlRequest) {
+        SQLResponse response = null;
+        Cache exceptionCache = cacheManager.getCache(EXCEPTION_QUERY_CACHE);
+        Cache queryCache = cacheManager.getCache(SUCCESS_QUERY_CACHE);
+
+        if (KylinConfig.getInstanceFromEnv().isQueryCacheEnabled() && null != exceptionCache.get(sqlRequest)) {
+            Element element = exceptionCache.get(sqlRequest);
+            response = (SQLResponse) element.getObjectValue();
+            response.setHitCache(true);
+        } else if (KylinConfig.getInstanceFromEnv().isQueryCacheEnabled() && null != queryCache.get(sqlRequest)) {
+            Element element = queryCache.get(sqlRequest);
+            response = (SQLResponse) element.getObjectValue();
+            response.setHitCache(true);
+        }
+
+        return response;
+    }
+
+    private void checkQueryAuth(SQLResponse sqlResponse) throws AccessDeniedException {
+        if (!sqlResponse.getIsException() && KylinConfig.getInstanceFromEnv().isQuerySecureEnabled()) {
+            CubeInstance cubeInstance = this.queryService.getCubeManager().getCube(sqlResponse.getCube());
+            queryService.checkAuthorization(cubeInstance);
+        }
+    }
+
+    public void setQueryService(QueryService queryService) {
+        this.queryService = queryService;
+    }
+
+    public void setCacheManager(CacheManager cacheManager) {
+        this.cacheManager = cacheManager;
+    }
+
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/server/src/main/java/com/kylinolap/rest/controller/TableController.java
----------------------------------------------------------------------
diff --git a/server/src/main/java/com/kylinolap/rest/controller/TableController.java b/server/src/main/java/com/kylinolap/rest/controller/TableController.java
new file mode 100644
index 0000000..0ffdfe5
--- /dev/null
+++ b/server/src/main/java/com/kylinolap/rest/controller/TableController.java
@@ -0,0 +1,214 @@
+/*
+ * 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.rest.controller;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.commons.lang.StringUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.springframework.beans.factory.annotation.Autowired;
+import org.springframework.stereotype.Controller;
+import org.springframework.web.bind.annotation.PathVariable;
+import org.springframework.web.bind.annotation.RequestBody;
+import org.springframework.web.bind.annotation.RequestMapping;
+import org.springframework.web.bind.annotation.RequestMethod;
+import org.springframework.web.bind.annotation.RequestParam;
+import org.springframework.web.bind.annotation.ResponseBody;
+
+import com.codahale.metrics.annotation.Metered;
+import com.kylinolap.metadata.MetadataConstances;
+import com.kylinolap.metadata.model.schema.ColumnDesc;
+import com.kylinolap.metadata.model.schema.TableDesc;
+import com.kylinolap.rest.exception.InternalErrorException;
+import com.kylinolap.rest.request.CardinalityRequest;
+import com.kylinolap.rest.response.TableDescResponse;
+import com.kylinolap.rest.service.CubeService;
+
+/**
+ * @author xduo
+ * 
+ */
+@Controller
+@RequestMapping(value = "/tables")
+public class TableController extends BasicController {
+    private static final Logger logger = LoggerFactory.getLogger(TableController.class);
+
+    @Autowired
+    private CubeService cubeMgmtService;
+
+    /**
+     * Get available table list of the input database
+     * 
+     * @return Table metadata array
+     * @throws IOException
+     */
+    @RequestMapping(value = "", method = { RequestMethod.GET })
+    @ResponseBody
+    @Metered(name = "listSourceTables")
+    public List<TableDesc> getHiveTables(@RequestParam(value = "ext", required = false) boolean withExt,@RequestParam(value = "project", required = false) String project ) {
+        long start = System.currentTimeMillis();
+        List<TableDesc> tables = null;
+        try {
+                tables = cubeMgmtService.getProjectManager().listDefinedTablesInProject(project);
+        } catch (IOException e) {
+            logger.error("Failed to deal with the request.", e);
+            throw new InternalErrorException(e.getLocalizedMessage());
+        }
+
+        if (withExt) {
+            tables = cloneTableDesc(tables);
+        }
+        long end = System.currentTimeMillis();
+        logger.info("Return all table metadata in " + (end - start) + " seconds");
+
+        return tables;
+    }
+
+    /**
+     * Get available table list of the input database
+     * 
+     * @return Table metadata array
+     * @throws IOException
+     */
+    @RequestMapping(value = "/{tableName}", method = { RequestMethod.GET })
+    @ResponseBody
+    public TableDesc getHiveTable(@PathVariable String tableName) {
+        return cubeMgmtService.getMetadataManager().getTableDesc(tableName);
+    }
+
+    /**
+     * Get available table list of the input database
+     * 
+     * @return Table metadata array
+     * @throws IOException
+     */
+    @RequestMapping(value = "/{tableName}/exd-map", method = { RequestMethod.GET })
+    @ResponseBody
+    public Map<String, String> getHiveTableExd(@PathVariable String tableName) {
+        Map<String, String> tableExd = cubeMgmtService.getMetadataManager().getTableDescExd(tableName);
+        return tableExd;
+    }
+
+    @RequestMapping(value = "/reload", method = { RequestMethod.PUT })
+    @ResponseBody
+    public String reloadSourceTable() {
+        cubeMgmtService.getMetadataManager().reload();
+        return "ok";
+    }
+
+    @RequestMapping(value = "/{tables}/{project}", method = { RequestMethod.POST })
+    @ResponseBody
+    public Map<String, String[]> loadHiveTable(@PathVariable String tables,@PathVariable String project){
+        Map<String, String[]> result = new HashMap<String, String[]>();
+        try{ 
+            String[] loadedTables = cubeMgmtService.reloadHiveTable(tables);
+
+            String inputTables[] = tables.split(",");
+            ArrayList<String> unloadedTables = new ArrayList<String>();
+            for (String inputTable : inputTables) {
+                boolean tableLoaded = false;
+                for (String loadedTable : loadedTables) {
+                    int cut = loadedTable.indexOf('.');
+                    String tableName = cut >= 0 ? loadedTable.substring(cut + 1).trim() : loadedTable.trim();
+                    if (inputTable.trim().toUpperCase().equals(tableName)||inputTable.trim().toUpperCase().equals(loadedTable)) {
+                        tableLoaded = true;
+                        break;
+                    }
+                }
+                if(!tableLoaded){
+                    unloadedTables.add(inputTable);
+                }
+            }
+           
+            cubeMgmtService.syncTableToProject(loadedTables, project);
+            result.put("result.loaded", loadedTables);
+            result.put("result.unloaded",unloadedTables.toArray(new String[unloadedTables.size()]) );
+            
+       }catch(IOException e){
+           logger.error("Failed to deal with the request.", e);
+           throw new InternalErrorException("Failed to load table,Please check the table name.");
+       }
+        return result;
+    }
+
+    /**
+     * Regenerate table cardinality
+     * 
+     * @return Table metadata array
+     * @throws IOException
+     */
+    @RequestMapping(value = "/{tableNames}/cardinality", method = { RequestMethod.PUT })
+    @ResponseBody
+    public CardinalityRequest generateCardinality(@PathVariable String tableNames, @RequestBody CardinalityRequest request) {
+        String[] tables = tableNames.split(",");
+        for (String table : tables) {
+            cubeMgmtService.generateCardinality(table.trim(), request.getFormat(), request.getDelimiter());
+        }
+        return request;
+    }
+
+    /**
+     * @param tables
+     * @return
+     */
+    private List<TableDesc> cloneTableDesc(List<TableDesc> tables) {
+        if (null == tables) {
+            return Collections.emptyList();
+        }
+
+        List<TableDesc> descs = new ArrayList<TableDesc>();
+        Iterator<TableDesc> it = tables.iterator();
+        while (it.hasNext()) {
+            TableDesc table = it.next();
+            Map<String, String> exd = cubeMgmtService.getMetadataManager().getTableDescExd(table.getName());
+            if (exd == null) {
+                descs.add(table);
+            } else {
+                // Clone TableDesc
+                TableDescResponse rtableDesc = new TableDescResponse(table);
+                rtableDesc.setDescExd(exd);
+                if (exd.containsKey(MetadataConstances.TABLE_EXD_CARDINALITY)) {
+                    Map<String, Long> cardinality = new HashMap<String, Long>();
+                    String scard = exd.get(MetadataConstances.TABLE_EXD_CARDINALITY);
+                    if (!StringUtils.isEmpty(scard)) {
+                        String[] cards = StringUtils.split(scard, ",");
+                        ColumnDesc[] cdescs = rtableDesc.getColumns();
+                        for (int i = 0; i < cdescs.length; i++) {
+                            ColumnDesc columnDesc = cdescs[i];
+                            if (cards.length > i) {
+                                cardinality.put(columnDesc.getName(), Long.parseLong(cards[i]));
+                            } else {
+                                logger.error("The result cardinality is not identical with hive table metadata, cardinaly : " + scard + " column array length: " + cdescs.length);
+                                break;
+                            }
+                        }
+                        rtableDesc.setCardinality(cardinality);
+                    }
+                }
+                descs.add(rtableDesc);
+            }
+        }
+        return descs;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/server/src/main/java/com/kylinolap/rest/controller/UserController.java
----------------------------------------------------------------------
diff --git a/server/src/main/java/com/kylinolap/rest/controller/UserController.java b/server/src/main/java/com/kylinolap/rest/controller/UserController.java
new file mode 100644
index 0000000..25b6bb2
--- /dev/null
+++ b/server/src/main/java/com/kylinolap/rest/controller/UserController.java
@@ -0,0 +1,66 @@
+/*
+ * 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.rest.controller;
+
+import java.util.List;
+
+import org.springframework.beans.factory.annotation.Autowired;
+import org.springframework.security.core.Authentication;
+import org.springframework.security.core.context.SecurityContextHolder;
+import org.springframework.security.core.userdetails.UserDetails;
+import org.springframework.stereotype.Controller;
+import org.springframework.web.bind.annotation.RequestMapping;
+import org.springframework.web.bind.annotation.RequestMethod;
+
+import com.kylinolap.rest.service.UserService;
+
+/**
+ * Handle user authentication request to protected kylin rest resources by
+ * spring security.
+ * 
+ * @author xduo
+ * 
+ */
+@Controller
+@RequestMapping(value = "/user")
+public class UserController {
+
+    @Autowired
+    UserService userService;
+
+    @RequestMapping(value = "/authentication", method = RequestMethod.POST, produces = "application/json")
+    public UserDetails authenticate() {
+        return authenticatedUser();
+    }
+
+    @RequestMapping(value = "/authentication", method = RequestMethod.GET, produces = "application/json")
+    public UserDetails authenticatedUser() {
+        Authentication authentication = SecurityContextHolder.getContext().getAuthentication();
+
+        if (authentication == null || !(authentication.getPrincipal() instanceof UserDetails)) {
+            return null;
+        }
+
+        return (UserDetails) authentication.getPrincipal();
+    }
+
+    @RequestMapping(value = "/authentication/authorities", method = RequestMethod.GET, produces = "application/json")
+    public List<String> getAuthorities() {
+        return userService.getUserAuthorities();
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/server/src/main/java/com/kylinolap/rest/exception/BadRequestException.java
----------------------------------------------------------------------
diff --git a/server/src/main/java/com/kylinolap/rest/exception/BadRequestException.java b/server/src/main/java/com/kylinolap/rest/exception/BadRequestException.java
new file mode 100644
index 0000000..0bf31c9
--- /dev/null
+++ b/server/src/main/java/com/kylinolap/rest/exception/BadRequestException.java
@@ -0,0 +1,59 @@
+/*
+ * 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.rest.exception;
+
+import org.springframework.http.HttpStatus;
+import org.springframework.web.bind.annotation.ResponseStatus;
+
+
+/**
+ * @author xduo
+ *
+ */
+@ResponseStatus(value = HttpStatus.BAD_REQUEST)
+public class BadRequestException extends RuntimeException {
+    /**
+     * 
+     */
+    private static final long serialVersionUID = -6798154278095441848L;
+
+    public BadRequestException(String s) {
+        super(s);
+    }
+
+    /**
+     * 
+     */
+    public BadRequestException() {
+        super();
+    }
+
+    /**
+     * @param arg0
+     * @param arg1
+     */
+    public BadRequestException(String arg0, Throwable arg1) {
+        super(arg0, arg1);
+    }
+
+    /**
+     * @param arg0
+     */
+    public BadRequestException(Throwable arg0) {
+        super(arg0);
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/server/src/main/java/com/kylinolap/rest/exception/ForbiddenException.java
----------------------------------------------------------------------
diff --git a/server/src/main/java/com/kylinolap/rest/exception/ForbiddenException.java b/server/src/main/java/com/kylinolap/rest/exception/ForbiddenException.java
new file mode 100644
index 0000000..25e53a3
--- /dev/null
+++ b/server/src/main/java/com/kylinolap/rest/exception/ForbiddenException.java
@@ -0,0 +1,39 @@
+/*
+ * 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.rest.exception;
+
+import org.springframework.http.HttpStatus;
+import org.springframework.web.bind.annotation.ResponseStatus;
+
+/**
+ * @author xduo
+ * 
+ */
+@ResponseStatus(value = HttpStatus.FORBIDDEN)
+public class ForbiddenException extends RuntimeException {
+
+    private static final long serialVersionUID = 2741885728370162194L;
+
+    public ForbiddenException() {
+        super();
+    }
+
+    public ForbiddenException(String message) {
+        super(message);
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/server/src/main/java/com/kylinolap/rest/exception/InternalErrorException.java
----------------------------------------------------------------------
diff --git a/server/src/main/java/com/kylinolap/rest/exception/InternalErrorException.java b/server/src/main/java/com/kylinolap/rest/exception/InternalErrorException.java
new file mode 100644
index 0000000..4198fda
--- /dev/null
+++ b/server/src/main/java/com/kylinolap/rest/exception/InternalErrorException.java
@@ -0,0 +1,60 @@
+/*
+ * 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.rest.exception;
+
+import org.springframework.http.HttpStatus;
+import org.springframework.web.bind.annotation.ResponseStatus;
+
+/**
+ * Class to wrap backend exception
+ * 
+ * @author jianliu
+ * 
+ */
+@ResponseStatus(value = HttpStatus.INTERNAL_SERVER_ERROR)
+public class InternalErrorException extends RuntimeException {
+    /**
+     * 
+     */
+    private static final long serialVersionUID = -6798154278095441848L;
+
+    public InternalErrorException(String s) {
+        super(s);
+    }
+
+    /**
+     * 
+     */
+    public InternalErrorException() {
+        super();
+    }
+
+    /**
+     * @param arg0
+     * @param arg1
+     */
+    public InternalErrorException(String arg0, Throwable arg1) {
+        super(arg0, arg1);
+    }
+
+    /**
+     * @param arg0
+     */
+    public InternalErrorException(Throwable arg0) {
+        super(arg0);
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/server/src/main/java/com/kylinolap/rest/exception/NotFoundException.java
----------------------------------------------------------------------
diff --git a/server/src/main/java/com/kylinolap/rest/exception/NotFoundException.java b/server/src/main/java/com/kylinolap/rest/exception/NotFoundException.java
new file mode 100644
index 0000000..ede21bf
--- /dev/null
+++ b/server/src/main/java/com/kylinolap/rest/exception/NotFoundException.java
@@ -0,0 +1,32 @@
+/*
+ * 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.rest.exception;
+
+import org.springframework.http.HttpStatus;
+import org.springframework.web.bind.annotation.ResponseStatus;
+ 
+/**
+ * @author xduo
+ *
+ */
+@ResponseStatus(value = HttpStatus.NOT_FOUND)
+public class NotFoundException extends RuntimeException {
+    private static final long serialVersionUID = 1L;
+
+    public NotFoundException(String s) {
+        super(s);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/server/src/main/java/com/kylinolap/rest/interceptor/CacheIntercaptor.java
----------------------------------------------------------------------
diff --git a/server/src/main/java/com/kylinolap/rest/interceptor/CacheIntercaptor.java b/server/src/main/java/com/kylinolap/rest/interceptor/CacheIntercaptor.java
new file mode 100644
index 0000000..2b763dd
--- /dev/null
+++ b/server/src/main/java/com/kylinolap/rest/interceptor/CacheIntercaptor.java
@@ -0,0 +1,42 @@
+/*
+ * 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.rest.interceptor;
+
+import org.aspectj.lang.JoinPoint;
+import org.aspectj.lang.annotation.After;
+import org.aspectj.lang.annotation.Aspect;
+import org.springframework.stereotype.Component;
+
+import com.kylinolap.common.restclient.Broadcaster;
+
+/**
+ * @author xduo
+ * 
+ */
+@Aspect
+@Component("cacheIntercaptor")
+public class CacheIntercaptor {
+
+    @After("execution(public * com.kylinolap.rest.controller.CubeController.*(..)) || execution(public * com.kylinolap.rest.controller.ProjectController.*(..))")
+    public void flush(JoinPoint joinPoint) {
+        String methodName = joinPoint.getSignature().getName();
+
+        if (methodName.matches("(update|create|save|disable|enable|delete|drop|purge)")) {
+            Broadcaster.flush();
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/server/src/main/java/com/kylinolap/rest/metrics/JobMetrics.java
----------------------------------------------------------------------
diff --git a/server/src/main/java/com/kylinolap/rest/metrics/JobMetrics.java b/server/src/main/java/com/kylinolap/rest/metrics/JobMetrics.java
new file mode 100644
index 0000000..4fde585
--- /dev/null
+++ b/server/src/main/java/com/kylinolap/rest/metrics/JobMetrics.java
@@ -0,0 +1,115 @@
+/*
+ * 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.rest.metrics;
+
+import java.util.HashMap;
+import java.util.Map;
+
+import com.codahale.metrics.Gauge;
+import com.codahale.metrics.Metric;
+import com.codahale.metrics.MetricSet;
+import com.kylinolap.job.JobManager;
+
+/**
+ * @author xduo
+ * 
+ */
+public class JobMetrics implements MetricSet {
+
+    private JobManager jobManager;
+
+    static class JobMetricsHolder {
+        static final JobMetrics INSTANCE = new JobMetrics();
+    }
+
+    public static JobMetrics getInstance() {
+        return JobMetricsHolder.INSTANCE;
+    }
+
+    /*
+     * (non-Javadoc)
+     * 
+     * @see com.codahale.metrics.MetricSet#getMetrics()
+     */
+    @Override
+    public Map<String, Metric> getMetrics() {
+        Map<String, Metric> metricSet = new HashMap<String, Metric>();
+
+        metricSet.put("PercentileJobStepDuration", new Gauge<Double>() {
+            @Override
+            public Double getValue() {
+                return jobManager.getPercentileJobStepDuration(95);
+            }
+        });
+
+        metricSet.put("scheduledJobs", new Gauge<Integer>() {
+            @Override
+            public Integer getValue() {
+                return jobManager.getScheduledJobsSzie();
+            }
+        });
+
+        metricSet.put("EngineThreadPool", new Gauge<Integer>() {
+            @Override
+            public Integer getValue() {
+                return jobManager.getEngineThreadPoolSize();
+            }
+        });
+
+        metricSet.put("MaxJobStep", new Gauge<Double>() {
+            @Override
+            public Double getValue() {
+                return jobManager.getMaxJobStepDuration();
+            }
+        });
+
+        metricSet.put("MinJobStep", new Gauge<Double>() {
+            @Override
+            public Double getValue() {
+                return jobManager.getMinJobStepDuration();
+            }
+        });
+
+        metricSet.put("IdleSlots", new Gauge<Integer>() {
+            @Override
+            public Integer getValue() {
+                return jobManager.getNumberOfIdleSlots();
+            }
+        });
+
+        metricSet.put("JobStepsExecuted", new Gauge<Integer>() {
+            @Override
+            public Integer getValue() {
+                return jobManager.getNumberOfJobStepsExecuted();
+            }
+        });
+
+        metricSet.put("JobStepsRunning", new Gauge<Integer>() {
+            @Override
+            public Integer getValue() {
+                return jobManager.getNumberOfJobStepsRunning();
+            }
+        });
+
+        return metricSet;
+    }
+
+    public void setJobManager(JobManager jobManager) {
+        this.jobManager = jobManager;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/server/src/main/java/com/kylinolap/rest/metrics/KylinInstrumentedFilterContextListener.java
----------------------------------------------------------------------
diff --git a/server/src/main/java/com/kylinolap/rest/metrics/KylinInstrumentedFilterContextListener.java b/server/src/main/java/com/kylinolap/rest/metrics/KylinInstrumentedFilterContextListener.java
new file mode 100644
index 0000000..e54db3e
--- /dev/null
+++ b/server/src/main/java/com/kylinolap/rest/metrics/KylinInstrumentedFilterContextListener.java
@@ -0,0 +1,39 @@
+package com.kylinolap.rest.metrics;
+
+import javax.servlet.ServletContextEvent;
+import javax.servlet.ServletContextListener;
+
+import org.springframework.beans.factory.annotation.Autowired;
+import org.springframework.web.context.support.WebApplicationContextUtils;
+
+import com.codahale.metrics.MetricRegistry;
+import com.codahale.metrics.servlet.InstrumentedFilterContextListener;
+
+public class KylinInstrumentedFilterContextListener implements ServletContextListener {
+
+    @Autowired
+    private MetricRegistry metricRegistry;
+
+    private final InnerKylinInstrumentedFilterContextListener innerKylinInstrumentedFilterContextListener = new InnerKylinInstrumentedFilterContextListener();
+
+    @Override
+    public void contextInitialized(ServletContextEvent event) {
+        WebApplicationContextUtils.getRequiredWebApplicationContext(event.getServletContext()).getAutowireCapableBeanFactory().autowireBean(this);
+
+        innerKylinInstrumentedFilterContextListener.contextInitialized(event);
+    }
+
+    @Override
+    public void contextDestroyed(ServletContextEvent event) {
+    }
+
+    class InnerKylinInstrumentedFilterContextListener extends InstrumentedFilterContextListener {
+
+        @Override
+        protected MetricRegistry getMetricRegistry() {
+            return metricRegistry;
+        }
+
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/server/src/main/java/com/kylinolap/rest/metrics/QueryMetrics.java
----------------------------------------------------------------------
diff --git a/server/src/main/java/com/kylinolap/rest/metrics/QueryMetrics.java b/server/src/main/java/com/kylinolap/rest/metrics/QueryMetrics.java
new file mode 100644
index 0000000..0748314
--- /dev/null
+++ b/server/src/main/java/com/kylinolap/rest/metrics/QueryMetrics.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.rest.metrics;
+
+import java.util.HashMap;
+import java.util.Map;
+
+import com.codahale.metrics.Gauge;
+import com.codahale.metrics.Metric;
+import com.codahale.metrics.MetricSet;
+
+/**
+ * @author xduo
+ * 
+ */
+public class QueryMetrics implements MetricSet {
+
+    private Map<String, Float> metrics = new HashMap<String, Float>();
+
+    private QueryMetrics() {
+        // register query metrics
+        this.increase("duration", (float) 0);
+        this.increase("totalScanCount", (float) 0);
+        this.increase("count", (float) 0);
+    }
+
+    static class QueryMetricsHolder {
+        static final QueryMetrics INSTANCE = new QueryMetrics();
+    }
+
+    public static QueryMetrics getInstance() {
+        return QueryMetricsHolder.INSTANCE;
+    }
+
+    public synchronized void increase(String key, Float value) {
+        if (metrics.containsKey(key)) {
+            metrics.put(key, metrics.get(key) + value);
+        } else {
+            metrics.put(key, value);
+        }
+    }
+
+    public synchronized Float getAndReset(String key) {
+        float value = metrics.get(key);
+        metrics.put(key, (float) 0);
+
+        return value;
+    }
+
+    public synchronized Map<String, Metric> getMetrics() {
+        Map<String, Metric> metricSet = new HashMap<String, Metric>();
+
+        for (final String key : metrics.keySet()) {
+            metricSet.put(key, new Gauge<Float>() {
+                @Override
+                public Float getValue() {
+                    float value = getAndReset(key);
+
+                    return value;
+                }
+            });
+        }
+
+        return metricSet;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/server/src/main/java/com/kylinolap/rest/model/ColumnMeta.java
----------------------------------------------------------------------
diff --git a/server/src/main/java/com/kylinolap/rest/model/ColumnMeta.java b/server/src/main/java/com/kylinolap/rest/model/ColumnMeta.java
new file mode 100644
index 0000000..731684a
--- /dev/null
+++ b/server/src/main/java/com/kylinolap/rest/model/ColumnMeta.java
@@ -0,0 +1,262 @@
+/*
+ * 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.rest.model;
+
+import java.io.Serializable;
+
+/**
+ * Created by lukhan on 2/27/14.
+ */
+public class ColumnMeta implements Serializable {
+    private static final long serialVersionUID = 1L;
+    private String TABLE_CAT;
+    private String TABLE_SCHEM;
+    private String TABLE_NAME;
+    private String COLUMN_NAME;
+    private int DATA_TYPE;
+    private String TYPE_NAME;
+    private int COLUMN_SIZE;
+    private int BUFFER_LENGTH;
+    private int DECIMAL_DIGITS;
+    private int NUM_PREC_RADIX;
+    private int NULLABLE;
+    private String REMARKS;
+    private String COLUMN_DEF;
+    private int SQL_DATA_TYPE;
+    private int SQL_DATETIME_SUB;
+    private int CHAR_OCTET_LENGTH;
+    private int ORDINAL_POSITION;
+    private String IS_NULLABLE;
+    private String SCOPE_CATLOG;
+    private String SCOPE_SCHEMA;
+    private String SCOPE_TABLE;
+    private short SOURCE_DATA_TYPE;
+    private String IS_AUTOINCREMENT;
+
+    public ColumnMeta() {
+    }
+
+    public ColumnMeta(String tABLE_CAT, String tABLE_SCHEM, String tABLE_NAME, String cOLUMN_NAME, int dATA_TYPE, String tYPE_NAME, int cOLUMN_SIZE, int bUFFER_LENGTH, int dECIMAL_DIGITS, int nUM_PREC_RADIX, int nULLABLE, String rEMARKS, String cOLUMN_DEF, int sQL_DATA_TYPE, int sQL_DATETIME_SUB, int cHAR_OCTET_LENGTH, int oRDINAL_POSITION, String iS_NULLABLE, String sCOPE_CATLOG, String sCOPE_SCHEMA, String sCOPE_TABLE, short sOURCE_DATA_TYPE, String iS_AUTOINCREMENT) {
+        super();
+        TABLE_CAT = tABLE_CAT;
+        TABLE_SCHEM = tABLE_SCHEM;
+        TABLE_NAME = tABLE_NAME;
+        COLUMN_NAME = cOLUMN_NAME;
+        DATA_TYPE = dATA_TYPE;
+        TYPE_NAME = tYPE_NAME;
+        COLUMN_SIZE = cOLUMN_SIZE;
+        BUFFER_LENGTH = bUFFER_LENGTH;
+        DECIMAL_DIGITS = dECIMAL_DIGITS;
+        NUM_PREC_RADIX = nUM_PREC_RADIX;
+        NULLABLE = nULLABLE;
+        REMARKS = rEMARKS;
+        COLUMN_DEF = cOLUMN_DEF;
+        SQL_DATA_TYPE = sQL_DATA_TYPE;
+        SQL_DATETIME_SUB = sQL_DATETIME_SUB;
+        CHAR_OCTET_LENGTH = cHAR_OCTET_LENGTH;
+        ORDINAL_POSITION = oRDINAL_POSITION;
+        IS_NULLABLE = iS_NULLABLE;
+        SCOPE_CATLOG = sCOPE_CATLOG;
+        SCOPE_SCHEMA = sCOPE_SCHEMA;
+        SCOPE_TABLE = sCOPE_TABLE;
+        SOURCE_DATA_TYPE = sOURCE_DATA_TYPE;
+        IS_AUTOINCREMENT = iS_AUTOINCREMENT;
+    }
+
+    public String getTABLE_CAT() {
+        return TABLE_CAT;
+    }
+
+    public void setTABLE_CAT(String tABLE_CAT) {
+        TABLE_CAT = tABLE_CAT;
+    }
+
+    public String getTABLE_SCHEM() {
+        return TABLE_SCHEM;
+    }
+
+    public void setTABLE_SCHEM(String tABLE_SCHEM) {
+        TABLE_SCHEM = tABLE_SCHEM;
+    }
+
+    public String getTABLE_NAME() {
+        return TABLE_NAME;
+    }
+
+    public void setTABLE_NAME(String tABLE_NAME) {
+        TABLE_NAME = tABLE_NAME;
+    }
+
+    public String getCOLUMN_NAME() {
+        return COLUMN_NAME;
+    }
+
+    public void setCOLUMN_NAME(String cOLUMN_NAME) {
+        COLUMN_NAME = cOLUMN_NAME;
+    }
+
+    public int getDATA_TYPE() {
+        return DATA_TYPE;
+    }
+
+    public void setDATA_TYPE(int dATA_TYPE) {
+        DATA_TYPE = dATA_TYPE;
+    }
+
+    public String getTYPE_NAME() {
+        return TYPE_NAME;
+    }
+
+    public void setTYPE_NAME(String tYPE_NAME) {
+        TYPE_NAME = tYPE_NAME;
+    }
+
+    public int getCOLUMN_SIZE() {
+        return COLUMN_SIZE;
+    }
+
+    public void setCOLUMN_SIZE(int cOLUMN_SIZE) {
+        COLUMN_SIZE = cOLUMN_SIZE;
+    }
+
+    public int getBUFFER_LENGTH() {
+        return BUFFER_LENGTH;
+    }
+
+    public void setBUFFER_LENGTH(int bUFFER_LENGTH) {
+        BUFFER_LENGTH = bUFFER_LENGTH;
+    }
+
+    public int getDECIMAL_DIGITS() {
+        return DECIMAL_DIGITS;
+    }
+
+    public void setDECIMAL_DIGITS(int dECIMAL_DIGITS) {
+        DECIMAL_DIGITS = dECIMAL_DIGITS;
+    }
+
+    public int getNUM_PREC_RADIX() {
+        return NUM_PREC_RADIX;
+    }
+
+    public void setNUM_PREC_RADIX(int nUM_PREC_RADIX) {
+        NUM_PREC_RADIX = nUM_PREC_RADIX;
+    }
+
+    public int getNULLABLE() {
+        return NULLABLE;
+    }
+
+    public void setNULLABLE(int nULLABLE) {
+        NULLABLE = nULLABLE;
+    }
+
+    public String getREMARKS() {
+        return REMARKS;
+    }
+
+    public void setREMARKS(String rEMARKS) {
+        REMARKS = rEMARKS;
+    }
+
+    public String getCOLUMN_DEF() {
+        return COLUMN_DEF;
+    }
+
+    public void setCOLUMN_DEF(String cOLUMN_DEF) {
+        COLUMN_DEF = cOLUMN_DEF;
+    }
+
+    public int getSQL_DATA_TYPE() {
+        return SQL_DATA_TYPE;
+    }
+
+    public void setSQL_DATA_TYPE(int sQL_DATA_TYPE) {
+        SQL_DATA_TYPE = sQL_DATA_TYPE;
+    }
+
+    public int getSQL_DATETIME_SUB() {
+        return SQL_DATETIME_SUB;
+    }
+
+    public void setSQL_DATETIME_SUB(int sQL_DATETIME_SUB) {
+        SQL_DATETIME_SUB = sQL_DATETIME_SUB;
+    }
+
+    public int getCHAR_OCTET_LENGTH() {
+        return CHAR_OCTET_LENGTH;
+    }
+
+    public void setCHAR_OCTET_LENGTH(int cHAR_OCTET_LENGTH) {
+        CHAR_OCTET_LENGTH = cHAR_OCTET_LENGTH;
+    }
+
+    public int getORDINAL_POSITION() {
+        return ORDINAL_POSITION;
+    }
+
+    public void setORDINAL_POSITION(int oRDINAL_POSITION) {
+        ORDINAL_POSITION = oRDINAL_POSITION;
+    }
+
+    public String getIS_NULLABLE() {
+        return IS_NULLABLE;
+    }
+
+    public void setIS_NULLABLE(String iS_NULLABLE) {
+        IS_NULLABLE = iS_NULLABLE;
+    }
+
+    public String getSCOPE_CATLOG() {
+        return SCOPE_CATLOG;
+    }
+
+    public void setSCOPE_CATLOG(String sCOPE_CATLOG) {
+        SCOPE_CATLOG = sCOPE_CATLOG;
+    }
+
+    public String getSCOPE_SCHEMA() {
+        return SCOPE_SCHEMA;
+    }
+
+    public void setSCOPE_SCHEMA(String sCOPE_SCHEMA) {
+        SCOPE_SCHEMA = sCOPE_SCHEMA;
+    }
+
+    public String getSCOPE_TABLE() {
+        return SCOPE_TABLE;
+    }
+
+    public void setSCOPE_TABLE(String sCOPE_TABLE) {
+        SCOPE_TABLE = sCOPE_TABLE;
+    }
+
+    public short getSOURCE_DATA_TYPE() {
+        return SOURCE_DATA_TYPE;
+    }
+
+    public void setSOURCE_DATA_TYPE(short sOURCE_DATA_TYPE) {
+        SOURCE_DATA_TYPE = sOURCE_DATA_TYPE;
+    }
+
+    public String getIS_AUTOINCREMENT() {
+        return IS_AUTOINCREMENT;
+    }
+
+    public void setIS_AUTOINCREMENT(String iS_AUTOINCREMENT) {
+        this.IS_AUTOINCREMENT = iS_AUTOINCREMENT;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/server/src/main/java/com/kylinolap/rest/model/Query.java
----------------------------------------------------------------------
diff --git a/server/src/main/java/com/kylinolap/rest/model/Query.java b/server/src/main/java/com/kylinolap/rest/model/Query.java
new file mode 100644
index 0000000..9a1a7cf
--- /dev/null
+++ b/server/src/main/java/com/kylinolap/rest/model/Query.java
@@ -0,0 +1,102 @@
+package com.kylinolap.rest.model;
+
+import com.fasterxml.jackson.annotation.JsonIgnoreProperties;
+
+@JsonIgnoreProperties(ignoreUnknown=true)
+public class Query {
+    
+    private String name;
+    private String project;
+    private String sql;
+    private String description;
+
+    public Query(){}
+    
+    public Query(String name, String project, String sql, String description) {
+        super();
+        this.name = name;
+        this.project = project;
+        this.sql = sql;
+        this.description = description;
+    }
+
+    public String getId(){
+        return String.valueOf(this.hashCode());
+    }
+ 
+    public String getName() {
+        return name;
+    }
+
+    public void setName(String name) {
+        this.name = name;
+    }
+
+    public String getProject() {
+        return project;
+    }
+
+    public void setProject(String project) {
+        this.project = project;
+    }
+
+    public String getSql() {
+        return sql;
+    }
+
+    public void setSql(String sql) {
+        this.sql = sql;
+    }
+
+    public String getDescription() {
+        return description;
+    }
+
+    public void setDescription(String description) {
+        this.description = description;
+    }
+
+    @Override
+    public int hashCode() {
+        final int prime = 31;
+        int result = 1;
+        result = prime * result + ((description == null) ? 0 : description.hashCode());
+        result = prime * result + ((name == null) ? 0 : name.hashCode());
+        result = prime * result + ((project == null) ? 0 : project.hashCode());
+        result = prime * result + ((sql == null) ? 0 : sql.hashCode());
+        return result;
+    }
+
+    @Override
+    public boolean equals(Object obj) {
+        if (this == obj)
+            return true;
+        if (obj == null)
+            return false;
+        if (getClass() != obj.getClass())
+            return false;
+        Query other = (Query) obj;
+        if (description == null) {
+            if (other.description != null)
+                return false;
+        } else if (!description.equals(other.description))
+            return false;
+        if (name == null) {
+            if (other.name != null)
+                return false;
+        } else if (!name.equals(other.name))
+            return false;
+        if (project == null) {
+            if (other.project != null)
+                return false;
+        } else if (!project.equals(other.project))
+            return false;
+        if (sql == null) {
+            if (other.sql != null)
+                return false;
+        } else if (!sql.equals(other.sql))
+            return false;
+        return true;
+    }
+ 
+}


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

Posted by lu...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/cube/src/main/java/com/kylinolap/cube/dataGen/FactTableGenerator.java
----------------------------------------------------------------------
diff --git a/cube/src/main/java/com/kylinolap/cube/dataGen/FactTableGenerator.java b/cube/src/main/java/com/kylinolap/cube/dataGen/FactTableGenerator.java
new file mode 100644
index 0000000..de63c27
--- /dev/null
+++ b/cube/src/main/java/com/kylinolap/cube/dataGen/FactTableGenerator.java
@@ -0,0 +1,645 @@
+package com.kylinolap.cube.dataGen;
+
+import java.io.BufferedReader;
+import java.io.BufferedWriter;
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.FileWriter;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.InputStreamReader;
+import java.text.SimpleDateFormat;
+import java.util.*;
+
+import com.kylinolap.common.KylinConfig;
+import com.kylinolap.common.persistence.ResourceStore;
+import com.kylinolap.common.util.Array;
+import com.kylinolap.cube.CubeInstance;
+import com.kylinolap.cube.CubeManager;
+import com.kylinolap.metadata.MetadataManager;
+import com.kylinolap.metadata.model.cube.CubeDesc;
+import com.kylinolap.metadata.model.cube.DimensionDesc;
+import com.kylinolap.metadata.model.cube.JoinDesc;
+import com.kylinolap.metadata.model.cube.MeasureDesc;
+import com.kylinolap.metadata.model.cube.TblColRef;
+import com.kylinolap.metadata.model.schema.ColumnDesc;
+import com.kylinolap.metadata.model.schema.DataType;
+
+/**
+ * Created by hongbin on 5/20/14.
+ */
+public class FactTableGenerator {
+    CubeInstance cube = null;
+    CubeDesc desc = null;
+    ResourceStore store = null;
+    String factTableName = null;
+
+    GenConfig genConf = null;
+
+    Random r = null;
+
+    String cubeName;
+    long randomSeed;
+    int rowCount;
+    int unlinkableRowCount;
+    int unlinkableRowCountMax;
+    double conflictRatio;
+    double linkableRatio;
+
+    // the names of lookup table columns which is in relation with fact
+    // table(appear as fk in fact table)
+    Hashtable<String, LinkedList<String>> lookupTableKeys = new Hashtable<String, LinkedList<String>>();
+
+    // possible values of lookupTableKeys, extracted from existing lookup
+    // tables.
+    // The key is in the format of tablename/columnname
+    HashMap<String, ArrayList<String>> feasibleValues = new HashMap<String, ArrayList<String>>();
+
+    // lookup table name -> sets of all composite keys
+    HashMap<String, HashSet<Array<String>>> lookupTableCompositeKeyValues = new HashMap<String, HashSet<Array<String>>>();
+
+    private void init(String cubeName, int rowCount, double conflictRaio, double linkableRatio, long randomSeed) {
+        this.rowCount = rowCount;
+        this.conflictRatio = conflictRaio;
+        this.cubeName = cubeName;
+        this.randomSeed = randomSeed;
+        this.linkableRatio = linkableRatio;
+
+        this.unlinkableRowCountMax = (int) (this.rowCount * (1 - linkableRatio));
+        this.unlinkableRowCount = 0;
+
+        r = new Random(randomSeed);
+
+        KylinConfig config = KylinConfig.getInstanceFromEnv();
+        cube = CubeManager.getInstance(config).getCube(cubeName);
+        desc = cube.getDescriptor();
+        factTableName = desc.getFactTable();
+        store = ResourceStore.getStore(config);
+    }
+
+    /*
+     * users can specify the value preference for each column
+     */
+    private void loadConfig() {
+        try {
+            InputStream configStream = null;
+            configStream = store.getResource("/data/data_gen_config.json");
+            this.genConf = GenConfig.loadConfig(configStream);
+
+            if (configStream != null)
+                configStream.close();
+        } catch (IOException e) {
+            e.printStackTrace();
+        }
+    }
+
+    private void loadLookupTableValues(String lookupTableName, LinkedList<String> columnNames, int distinctRowCount) throws Exception {
+        KylinConfig config = KylinConfig.getInstanceFromEnv();
+
+        // only deal with composite keys
+        if (columnNames.size() > 1 && !lookupTableCompositeKeyValues.containsKey(lookupTableName)) {
+            lookupTableCompositeKeyValues.put(lookupTableName, new HashSet<Array<String>>());
+        }
+
+        InputStream tableStream = null;
+        BufferedReader tableReader = null;
+        try {
+            Hashtable<String, Integer> zeroBasedInice = new Hashtable<String, Integer>();
+            for (String columnName : columnNames) {
+                ColumnDesc cDesc = MetadataManager.getInstance(config).getTableDesc(lookupTableName).findColumnByName(columnName);
+                zeroBasedInice.put(columnName, cDesc.getZeroBasedIndex());
+            }
+
+            String path = "/data/" + lookupTableName + ".csv";
+            tableStream = store.getResource(path);
+            tableReader = new BufferedReader(new InputStreamReader(tableStream));
+            tableReader.mark(0);
+            int rowCount = 0;
+            int curRowNum = 0;
+            String curRow;
+
+            while (tableReader.readLine() != null)
+                rowCount++;
+
+            HashSet<Integer> rows = new HashSet<Integer>();
+            distinctRowCount = (distinctRowCount < rowCount) ? distinctRowCount : rowCount;
+            while (rows.size() < distinctRowCount) {
+                rows.add(r.nextInt(rowCount));
+            }
+
+            // reopen the stream
+            tableStream.close();
+            tableReader.close();
+            tableStream = null;
+            tableReader = null;
+
+            tableStream = store.getResource(path);
+            tableReader = new BufferedReader(new InputStreamReader(tableStream));
+
+            while ((curRow = tableReader.readLine()) != null) {
+                if (rows.contains(curRowNum)) {
+                    String[] tokens = curRow.split(",");
+
+                    String[] comboKeys = null;
+                    int index = 0;
+                    if (columnNames.size() > 1)
+                        comboKeys = new String[columnNames.size()];
+
+                    for (String columnName : columnNames) {
+                        int zeroBasedIndex = zeroBasedInice.get(columnName);
+                        if (!feasibleValues.containsKey(lookupTableName + "/" + columnName))
+                            feasibleValues.put(lookupTableName + "/" + columnName, new ArrayList<String>());
+                        feasibleValues.get(lookupTableName + "/" + columnName).add(tokens[zeroBasedIndex]);
+
+                        if (columnNames.size() > 1) {
+                            comboKeys[index] = tokens[zeroBasedIndex];
+                            index++;
+                        }
+                    }
+
+                    if (columnNames.size() > 1) {
+                        Array<String> wrap = new Array<String>(comboKeys);
+                        if (lookupTableCompositeKeyValues.get(lookupTableName).contains(wrap)) {
+                            throw new Exception("The composite key already exist in the lookup table");
+                        }
+                        lookupTableCompositeKeyValues.get(lookupTableName).add(wrap);
+                    }
+                }
+                curRowNum++;
+            }
+
+            if (tableStream != null)
+                tableStream.close();
+            if (tableReader != null)
+                tableReader.close();
+
+        } catch (IOException e) {
+            e.printStackTrace();
+            System.exit(1);
+        }
+    }
+
+    // prepare the candidate values for each joined column
+    private void prepare() throws Exception {
+        // load config
+        loadConfig();
+
+        HashSet<String> factTableColumns = new HashSet<String>();
+
+        for (DimensionDesc dim : desc.getDimensions()) {
+            for (TblColRef col : dim.getColumnRefs()) {
+                if (col.getTable().equals(factTableName))
+                    factTableColumns.add(col.getName());
+            }
+
+            JoinDesc join = dim.getJoin();
+            if (join != null) {
+                String lookupTable = dim.getTable();
+                for (String column : join.getPrimaryKey()) {
+                    if (!lookupTableKeys.containsKey(lookupTable)) {
+                        lookupTableKeys.put(lookupTable, new LinkedList<String>());
+                    }
+
+                    if (!lookupTableKeys.get(lookupTable).contains(column))
+                        lookupTableKeys.get(lookupTable).add(column);
+                }
+            }
+        }
+
+        int distinctRowCount = (int) (this.rowCount / this.conflictRatio);
+        distinctRowCount = (distinctRowCount == 0) ? 1 : distinctRowCount;
+        // lookup tables
+        for (String lookupTable : lookupTableKeys.keySet()) {
+            this.loadLookupTableValues(lookupTable, lookupTableKeys.get(lookupTable), distinctRowCount);
+        }
+    }
+
+    private List<DimensionDesc> getSortedDimentsionDescs() {
+        List<DimensionDesc> dimensions = desc.getDimensions();
+        Collections.sort(dimensions, new Comparator<DimensionDesc>() {
+            @Override
+            public int compare(DimensionDesc o1, DimensionDesc o2) {
+                JoinDesc j1 = o2.getJoin();
+                JoinDesc j2 = o1.getJoin();
+                return Integer.valueOf(j1 != null ? j1.getPrimaryKey().length : 0).compareTo(j2 != null ? j2.getPrimaryKey().length : 0);
+            }
+        });
+        return dimensions;
+    }
+
+    private void execute(String joinType) throws Exception {
+
+        // main logic here , generate the data to a temp file
+        String tempFilePath = generate();
+
+        // Write to hbase
+        File tempFile = new File(tempFilePath);
+
+        InputStream in = new FileInputStream(tempFile);
+        String factTablePath = "/data/" + factTableName + ".csv";
+        store.deleteResource(factTablePath);
+        store.putResource(factTablePath, in, System.currentTimeMillis());
+        in.close();
+
+        // duplicate a copy of this fact table, with a naming convention with
+        // jointype added
+        // so that later test cases can select different data files
+        in = new FileInputStream(tempFile);
+        String factTablePathWithJoinType = "/data/" + factTableName + ".csv." + joinType.toLowerCase();
+        store.deleteResource(factTablePathWithJoinType);
+        store.putResource(factTablePathWithJoinType, in, System.currentTimeMillis());
+        in.close();
+
+        tempFile.delete();
+
+        System.out.println();
+        System.out.println("The new fact table has been written to $KYLIN_METADATA_URL" + factTablePath);
+        System.out.println();
+    }
+
+    /**
+     * Generate the fact table and put it into a temp file
+     *
+     * @return
+     * @throws Exception
+     */
+    private String generate() throws Exception {
+        // the columns on the fact table can be classified into three groups:
+        // 1. foreign keys
+        HashMap<String, String> factTableCol2LookupCol = new HashMap<String, String>();
+        // 2. metrics or directly used dimensions
+        HashSet<String> usedCols = new HashSet<String>();
+        // 3. others, not referenced anywhere
+
+        HashMap<String, String> lookupCol2factTableCol = new HashMap<String, String>();
+
+        // find fact table columns in fks
+        List<DimensionDesc> dimensions = getSortedDimentsionDescs();
+        for (DimensionDesc dim : dimensions) {
+            JoinDesc jDesc = dim.getJoin();
+            if (jDesc != null) {
+                String[] fks = jDesc.getForeignKey();
+                String[] pks = jDesc.getPrimaryKey();
+                int num = fks.length;
+                for (int i = 0; i < num; ++i) {
+                    String value = dim.getTable() + "/" + pks[i];
+
+                    lookupCol2factTableCol.put(value, fks[i]);
+
+                    if (factTableCol2LookupCol.containsKey(fks[i])) {
+                        if (!factTableCol2LookupCol.get(fks[i]).equals(value)) {
+                            System.out.println("Warning: Disambiguation on the mapping of column " + fks[i] + ", " + factTableCol2LookupCol.get(fks[i]) + "(chosen) or " + value);
+                            continue;
+                        }
+                    }
+                    factTableCol2LookupCol.put(fks[i], value);
+                }
+            }
+            //else, deal with it in next roung
+        }
+
+        // find fact table columns in direct dimension
+        // DO NOT merge this with the previous loop
+        for (DimensionDesc dim : dimensions) {
+            JoinDesc jDesc = dim.getJoin();
+            if (jDesc == null) {
+                // column on fact table used directly as a dimension
+                if (!factTableCol2LookupCol.containsKey(dim.getColumn()))
+                    usedCols.add(dim.getColumn());
+            }
+        }
+
+        // find fact table columns in measures
+        for (MeasureDesc mDesc : desc.getMeasures()) {
+            List<TblColRef> pcols = mDesc.getFunction().getParameter().getColRefs();
+            if (pcols != null) {
+                for (TblColRef col : pcols) {
+                    if (!factTableCol2LookupCol.containsKey(col.getName()))
+                        usedCols.add(col.getName());
+                }
+            }
+        }
+
+        return createTable(this.rowCount, factTableCol2LookupCol, lookupCol2factTableCol, usedCols);
+    }
+
+    private String normToTwoDigits(int v) {
+        if (v < 10)
+            return "0" + v;
+        else
+            return Integer.toString(v);
+    }
+
+    private String randomPick(ArrayList<String> candidates) {
+        int index = r.nextInt(candidates.size());
+        return candidates.get(index);
+    }
+
+    private String createRandomCell(ColumnDesc cDesc, ArrayList<String> range) throws Exception {
+        DataType type = cDesc.getType();
+        if (type.isStringFamily()) {
+            throw new Exception("Can't handle range values for string");
+
+        } else if (type.isIntegerFamily()) {
+            int low = Integer.parseInt(range.get(0));
+            int high = Integer.parseInt(range.get(1));
+            return Integer.toString(r.nextInt(high - low) + low);
+
+        } else if (type.isDouble()) {
+            double low = Double.parseDouble(range.get(0));
+            double high = Double.parseDouble(range.get(1));
+            return String.format("%.4f", r.nextDouble() * (high - low) + low);
+
+        } else if (type.isFloat()) {
+            float low = Float.parseFloat(range.get(0));
+            float high = Float.parseFloat(range.get(1));
+            return String.format("%.4f", r.nextFloat() * (high - low) + low);
+
+        } else if (type.isDecimal()) {
+            double low = Double.parseDouble(range.get(0));
+            double high = Double.parseDouble(range.get(1));
+            return String.format("%.4f", r.nextDouble() * (high - low) + low);
+
+        } else if (type.isDateTimeFamily()) {
+
+            SimpleDateFormat format = new SimpleDateFormat("yyyy-MM-dd");
+            Date start = format.parse(range.get(0));
+            Date end = format.parse(range.get(1));
+            long diff = end.getTime() - start.getTime();
+            Date temp = new Date(start.getTime() + (long) (diff * r.nextDouble()));
+            Calendar cal = Calendar.getInstance();
+            cal.setTime(temp);
+            // first day
+            cal.set(Calendar.DAY_OF_WEEK, cal.getFirstDayOfWeek());
+
+            return cal.get(Calendar.YEAR) + "-" + normToTwoDigits(cal.get(Calendar.MONTH) + 1) + "-" + normToTwoDigits(cal.get(Calendar.DAY_OF_MONTH));
+        } else {
+            System.out.println("The data type " + type + "is not recognized");
+            System.exit(1);
+        }
+        return null;
+    }
+
+    private String createRandomCell(ColumnDesc cDesc) {
+        String type = cDesc.getTypeName();
+        String s = type.toLowerCase();
+        if (s.equals("string") || s.equals("char") || s.equals("varchar")) {
+            StringBuilder sb = new StringBuilder();
+            for (int i = 0; i < 2; i++) {
+                sb.append((char) ('a' + r.nextInt(10)));// there are 10*10
+                // possible strings
+            }
+            return sb.toString();
+        } else if (s.equals("bigint") || s.equals("int") || s.equals("tinyint") || s.equals("smallint")) {
+            return Integer.toString(r.nextInt(128));
+        } else if (s.equals("double")) {
+            return String.format("%.4f", r.nextDouble() * 100);
+        } else if (s.equals("float")) {
+            return String.format("%.4f", r.nextFloat() * 100);
+        } else if (s.equals("decimal")) {
+            return String.format("%.4f", r.nextDouble() * 100);
+        } else if (s.equals("date")) {
+            long date20131231 = 61349312153265L;
+            long date20010101 = 60939158400000L;
+            long diff = date20131231 - date20010101;
+            Date temp = new Date(date20010101 + (long) (diff * r.nextDouble()));
+            Calendar cal = Calendar.getInstance();
+            cal.setTime(temp);
+            // first day
+            cal.set(Calendar.DAY_OF_WEEK, cal.getFirstDayOfWeek());
+
+            return cal.get(Calendar.YEAR) + "-" + normToTwoDigits(cal.get(Calendar.MONTH) + 1) + "-" + normToTwoDigits(cal.get(Calendar.DAY_OF_MONTH));
+        } else {
+            System.out.println("The data type " + type + "is not recognized");
+            System.exit(1);
+        }
+        return null;
+    }
+
+    private String createDefaultsCell(String type) {
+        String s = type.toLowerCase();
+        if (s.equals("string") || s.equals("char") || s.equals("varchar")) {
+            return "abcde";
+        } else if (s.equals("bigint") || s.equals("int") || s.equals("tinyint") || s.equals("smallint")) {
+            return "0";
+        } else if (s.equals("double")) {
+            return "0";
+        } else if (s.equals("float")) {
+            return "0";
+        } else if (s.equals("decimal")) {
+            return "0";
+        } else if (s.equals("date")) {
+            return "1970-01-01";
+        } else {
+            System.out.println("The data type " + type + "is not recognized");
+            System.exit(1);
+        }
+        return null;
+    }
+
+    private void printColumnMappings(HashMap<String, String> factTableCol2LookupCol, HashSet<String> usedCols, HashSet<String> defaultColumns) {
+
+        System.out.println("=======================================================================");
+        System.out.format("%-30s %s", "FACT_TABLE_COLUMN", "MAPPING");
+        System.out.println();
+        System.out.println();
+        for (Map.Entry<String, String> entry : factTableCol2LookupCol.entrySet()) {
+            System.out.format("%-30s %s", entry.getKey(), entry.getValue());
+            System.out.println();
+        }
+        for (String key : usedCols) {
+            System.out.format("%-30s %s", key, "Random Values");
+            System.out.println();
+        }
+        for (String key : defaultColumns) {
+            System.out.format("%-30s %s", key, "Default Values");
+            System.out.println();
+        }
+        System.out.println("=======================================================================");
+
+        System.out.println("Parameters:");
+        System.out.println();
+        System.out.println("CubeName:        " + cubeName);
+        System.out.println("RowCount:        " + rowCount);
+        System.out.println("ConflictRatio:   " + conflictRatio);
+        System.out.println("LinkableRatio:   " + linkableRatio);
+        System.out.println("Seed:            " + randomSeed);
+        System.out.println();
+        System.out.println("The number of actual unlinkable fact rows is: " + this.unlinkableRowCount);
+        System.out.println("You can vary the above parameters to generate different datasets.");
+        System.out.println();
+    }
+
+    // Any row in the column must finally appear in the flatten big table.
+    // for single-column joins the generated row is guaranteed to have a match
+    // in lookup table
+    // for composite keys we'll need an extra check
+    private boolean matchAllCompositeKeys(HashMap<String, String> lookupCol2FactTableCol, LinkedList<String> columnValues) {
+        KylinConfig config = KylinConfig.getInstanceFromEnv();
+
+        for (String lookupTable : lookupTableKeys.keySet()) {
+            if (lookupTableKeys.get(lookupTable).size() == 1)
+                continue;
+
+            String[] comboKey = new String[lookupTableKeys.get(lookupTable).size()];
+            int index = 0;
+            for (String column : lookupTableKeys.get(lookupTable)) {
+                String key = lookupTable + "/" + column;
+                String factTableCol = lookupCol2FactTableCol.get(key);
+                int cardinal = MetadataManager.getInstance(config).getTableDesc(factTableName).findColumnByName(factTableCol).getZeroBasedIndex();
+                comboKey[index] = columnValues.get(cardinal);
+
+                index++;
+            }
+            Array<String> wrap = new Array<String>(comboKey);
+            if (!lookupTableCompositeKeyValues.get(lookupTable).contains(wrap)) {
+                // System.out.println("Try " + wrap + " Failed, continue...");
+                return false;
+            }
+        }
+        return true;
+    }
+
+    private String createCell(ColumnDesc cDesc) throws Exception {
+        ColumnConfig cConfig = null;
+
+        if ((cConfig = genConf.getColumnConfigByName(cDesc.getName())) == null) {
+            // if the column is not configured, use random values
+            return (createRandomCell(cDesc));
+
+        } else {
+            // the column has a configuration
+            if (!cConfig.isAsRange() && !cConfig.isExclusive() && r.nextBoolean()) {
+                // if the column still allows random values
+                return (createRandomCell(cDesc));
+
+            } else {
+                // use specified values
+                ArrayList<String> valueSet = cConfig.getValueSet();
+                if (valueSet == null || valueSet.size() == 0)
+                    throw new Exception("Did you forget to specify value set for " + cDesc.getName());
+
+                if (!cConfig.isAsRange()) {
+                    return (randomPick(valueSet));
+                } else {
+                    if (valueSet.size() != 2)
+                        throw new Exception("Only two values can be set for range values, the column: " + cDesc.getName());
+
+                    return (createRandomCell(cDesc, valueSet));
+                }
+            }
+
+        }
+    }
+
+    private LinkedList<String> createRow(HashMap<String, String> factTableCol2LookupCol, HashSet<String> usedCols, HashSet<String> defaultColumns) throws Exception {
+        KylinConfig config = KylinConfig.getInstanceFromEnv();
+        LinkedList<String> columnValues = new LinkedList<String>();
+
+        for (ColumnDesc cDesc : MetadataManager.getInstance(config).getTableDesc(factTableName).getColumns()) {
+
+
+            String colName = cDesc.getName();
+
+            if (factTableCol2LookupCol.containsKey(colName)) {
+
+                // if the current column is a fk column in fact table
+                ArrayList<String> candidates = this.feasibleValues.get(factTableCol2LookupCol.get(colName));
+
+                columnValues.add(candidates.get(r.nextInt(candidates.size())));
+            } else if (usedCols.contains(colName)) {
+
+                // if the current column is a metric column in fact table
+                columnValues.add(createCell(cDesc));
+            } else {
+
+                // otherwise this column is not useful in OLAP
+                columnValues.add(createDefaultsCell(cDesc.getTypeName()));
+                defaultColumns.add(colName);
+            }
+        }
+
+        return columnValues;
+    }
+
+    private String createTable(int rowCount, HashMap<String, String> factTableCol2LookupCol, HashMap<String, String> lookupCol2FactTableCol, HashSet<String> usedCols) throws Exception {
+        try {
+            File tempFile = File.createTempFile("ftg", ".tmp");
+            BufferedWriter writer = new BufferedWriter(new FileWriter(tempFile));
+            HashSet<String> defaultColumns = new HashSet<String>();
+
+            for (int i = 0; i < rowCount; ) {
+
+                LinkedList<String> columnValues = createRow(factTableCol2LookupCol, usedCols, defaultColumns);
+
+                if (!matchAllCompositeKeys(lookupCol2FactTableCol, columnValues)) {
+                    if (unlinkableRowCount < unlinkableRowCountMax) {
+                        unlinkableRowCount++;
+                    } else {
+                        continue;
+                    }
+                }
+
+                StringBuffer sb = new StringBuffer();
+                for (String c : columnValues)
+                    sb.append(c + ",");
+                sb.deleteCharAt(sb.length() - 1);
+                writer.write(sb.toString());
+                writer.newLine();
+                i++;
+
+                // System.out.println("Just generated the " + i + "th record");
+            }
+            writer.flush();
+            writer.close();
+
+            printColumnMappings(factTableCol2LookupCol, usedCols, defaultColumns);
+
+            return tempFile.getAbsolutePath();
+
+        } catch (IOException e) {
+            e.printStackTrace();
+            System.exit(1);
+        }
+
+        return null;
+    }
+
+    /**
+     * Randomly create a fact table and put it to test_kylin_data table in hbase
+     *
+     * @param cubeName      name of the cube
+     * @param rowCount      expected row count generated
+     * @param linkableRatio the percentage of fact table rows that can be linked with all
+     *                      lookup table by INNER join
+     * @param randomSeed    random seed
+     */
+    public static void generate(String cubeName, String rowCount, String linkableRatio, String randomSeed, String joinType) throws Exception {
+
+        if (cubeName == null)
+            cubeName = "test_kylin_cube_with_slr_ready";
+        if (rowCount == null)
+            rowCount = "10000";
+        if (linkableRatio == null)
+            linkableRatio = "0.6";
+
+        //if (randomSeed == null)
+        // don't give it value
+
+        // String conflictRatio = "5";//this parameter do not allow configuring
+        // any more
+
+        FactTableGenerator generator = new FactTableGenerator();
+        long seed;
+        if (randomSeed != null) {
+            seed = Long.parseLong(randomSeed);
+        } else {
+            Random r = new Random();
+            seed = r.nextLong();
+        }
+
+        generator.init(cubeName, Integer.parseInt(rowCount), 5, Double.parseDouble(linkableRatio), seed);
+        generator.prepare();
+        generator.execute(joinType);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/cube/src/main/java/com/kylinolap/cube/dataGen/GenConfig.java
----------------------------------------------------------------------
diff --git a/cube/src/main/java/com/kylinolap/cube/dataGen/GenConfig.java b/cube/src/main/java/com/kylinolap/cube/dataGen/GenConfig.java
new file mode 100644
index 0000000..a877c63
--- /dev/null
+++ b/cube/src/main/java/com/kylinolap/cube/dataGen/GenConfig.java
@@ -0,0 +1,63 @@
+package com.kylinolap.cube.dataGen;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.util.ArrayList;
+import java.util.HashMap;
+
+import com.fasterxml.jackson.annotation.JsonAutoDetect;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.fasterxml.jackson.core.JsonParseException;
+import com.fasterxml.jackson.databind.JsonMappingException;
+import com.kylinolap.common.util.JsonUtil;
+
+/**
+ * Created by honma on 5/29/14.
+ */
+@JsonAutoDetect(fieldVisibility = JsonAutoDetect.Visibility.NONE, getterVisibility = JsonAutoDetect.Visibility.NONE, isGetterVisibility = JsonAutoDetect.Visibility.NONE, setterVisibility = JsonAutoDetect.Visibility.NONE)
+public class GenConfig {
+
+    @JsonProperty("columnConfigs")
+    private ArrayList<ColumnConfig> columnConfigs;
+
+    private HashMap<String, ColumnConfig> cache = new HashMap<String, ColumnConfig>();
+
+    public ArrayList<ColumnConfig> getColumnConfigs() {
+        return columnConfigs;
+    }
+
+    public void setColumnConfigs(ArrayList<ColumnConfig> columnConfigs) {
+        this.columnConfigs = columnConfigs;
+    }
+
+    public ColumnConfig getColumnConfigByName(String columnName) {
+        columnName = columnName.toLowerCase();
+
+        if (cache.containsKey(columnName))
+            return cache.get(columnName);
+
+        for (ColumnConfig cConfig : columnConfigs) {
+            if (cConfig.getColumnName().toLowerCase().equals(columnName)) {
+                cache.put(columnName, cConfig);
+                return cConfig;
+            }
+        }
+        cache.put(columnName, null);
+        return null;
+    }
+
+    public static GenConfig loadConfig(InputStream stream) {
+        try {
+            GenConfig config = JsonUtil.readValue(stream, GenConfig.class);
+            return config;
+        } catch (JsonMappingException e) {
+            e.printStackTrace();
+        } catch (JsonParseException e) {
+            e.printStackTrace();
+        } catch (IOException e) {
+            e.printStackTrace();
+        }
+
+        return null;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/cube/src/main/java/com/kylinolap/cube/estimation/CubeSizeEstimationCLI.java
----------------------------------------------------------------------
diff --git a/cube/src/main/java/com/kylinolap/cube/estimation/CubeSizeEstimationCLI.java b/cube/src/main/java/com/kylinolap/cube/estimation/CubeSizeEstimationCLI.java
new file mode 100644
index 0000000..494f1b0
--- /dev/null
+++ b/cube/src/main/java/com/kylinolap/cube/estimation/CubeSizeEstimationCLI.java
@@ -0,0 +1,163 @@
+package com.kylinolap.cube.estimation;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashSet;
+import java.util.LinkedList;
+import java.util.List;
+
+import com.kylinolap.common.KylinConfig;
+import com.kylinolap.cube.CubeInstance;
+import com.kylinolap.cube.CubeManager;
+import com.kylinolap.cube.cuboid.Cuboid;
+import com.kylinolap.cube.cuboid.CuboidScheduler;
+import com.kylinolap.metadata.model.cube.CubeDesc;
+import com.kylinolap.metadata.model.cube.DimensionDesc;
+import com.kylinolap.metadata.model.cube.HierarchyDesc;
+import com.kylinolap.metadata.model.cube.MeasureDesc;
+import com.kylinolap.metadata.model.cube.RowKeyColDesc;
+import com.kylinolap.metadata.model.cube.RowKeyDesc;
+import com.kylinolap.metadata.model.schema.DataType;
+
+/**
+ * Created by honma on 9/1/14.
+ */
+public class CubeSizeEstimationCLI {
+
+    public static class RowKeyColInfo {
+        public List<List<Integer>> hierachyColBitIndice;
+        public List<Integer> nonHierachyColBitIndice;
+    }
+
+    public static long estimatedCubeSize(String cubeName, long[] cardinality) {
+        KylinConfig config = KylinConfig.getInstanceFromEnv();
+        CubeManager cubeManager = CubeManager.getInstance(config);
+        CubeInstance cubeInstance = cubeManager.getCube(cubeName);
+        CubeDesc cubeDesc = cubeInstance.getDescriptor();
+
+        CuboidScheduler scheduler = new CuboidScheduler(cubeDesc);
+        long baseCuboid = Cuboid.getBaseCuboidId(cubeDesc);
+        LinkedList<Long> cuboidQueue = new LinkedList<Long>();
+        cuboidQueue.push(baseCuboid);
+
+        long totalSpace = 0;
+
+        while (!cuboidQueue.isEmpty()) {
+            long cuboidID = cuboidQueue.pop();
+            Collection<Long> spanningCuboid = scheduler.getSpanningCuboid(cuboidID);
+            for (Long sc : spanningCuboid) {
+                cuboidQueue.push(sc);
+            }
+
+            totalSpace += estimateCuboidSpace(cuboidID, cardinality, cubeDesc);
+        }
+        return totalSpace;
+    }
+
+    public static long estimateCuboidSpace(long cuboidID, long[] cardinality, CubeDesc cubeDesc) {
+
+        RowKeyColInfo rowKeyColInfo = extractRowKeyInfo(cubeDesc);
+        RowKeyDesc rowKeyDesc = cubeDesc.getRowkey();
+
+        long rowCount = 1;
+        int[] rowKeySpaces = estimateRowKeyColSpace(rowKeyDesc, cardinality);
+        int dimensionSpace = 0;
+        int measureSpace = getMeasureSpace(cubeDesc);
+
+        for (List<Integer> hlist : rowKeyColInfo.hierachyColBitIndice) {
+            // for hierachy columns, the cardinality of the most detailed column
+            // nominates.
+            int i;
+            for (i = 0; i < hlist.size() && rowKeyColExists(hlist.get(i), cuboidID); ++i) {
+                dimensionSpace += rowKeySpaces[hlist.get(i)];
+            }
+
+            if (i != 0)
+                rowCount *= cardinality[hlist.get(i - 1)];
+        }
+
+        for (int index : rowKeyColInfo.nonHierachyColBitIndice) {
+            if (rowKeyColExists(index, cuboidID)) {
+                rowCount *= cardinality[index];
+                dimensionSpace += rowKeySpaces[index];
+            }
+        }
+        return rowCount * (dimensionSpace + measureSpace);
+    }
+
+    private static int[] estimateRowKeyColSpace(RowKeyDesc rowKeyDesc, long[] cardinality) {
+        RowKeyColDesc[] rowKeyColDescs = rowKeyDesc.getRowKeyColumns();
+        int[] ret = new int[rowKeyColDescs.length];
+        for (int i = 0; i < rowKeyColDescs.length; ++i) {
+            RowKeyColDesc rowKeyColDesc = rowKeyColDescs[rowKeyColDescs.length - 1 - i];
+            if (rowKeyColDesc.getDictionary() == null) {
+                if (rowKeyColDesc.getLength() == 0)
+                    throw new IllegalStateException("The non-dictionary col " + rowKeyColDesc.getColumn() + " has length of 0");
+                ret[i] = rowKeyColDesc.getLength();
+            } else {
+                ret[i] = estimateDictionaryColSpace(cardinality[i]);
+            }
+        }
+        return ret;
+    }
+
+    // TODO what if it's date dictionary?
+    private static int estimateDictionaryColSpace(long cardinality) {
+        long mask = 1L;
+        int i;
+        for (i = Long.SIZE - 1; i >= 0; i--) {
+            if ((cardinality & (mask << i)) != 0) {
+                break;
+            }
+        }
+
+        if (i < 0)
+            throw new IllegalStateException("the cardinality is 0");
+
+        return ((i + 1) + 7) / 8;// the bytes required to save at most
+                                 // cardinality numbers
+    }
+
+    private static int getMeasureSpace(CubeDesc cubeDesc) {
+        int space = 0;
+        for (MeasureDesc measureDesc : cubeDesc.getMeasures()) {
+            DataType returnType = measureDesc.getFunction().getReturnDataType();
+            space += returnType.getSpaceEstimate();
+        }
+        return space;
+    }
+
+    private static boolean rowKeyColExists(int bitIndex, long cuboidID) {
+        long mask = 1L << bitIndex;
+        return (cuboidID & mask) != 0;
+    }
+
+    private static RowKeyColInfo extractRowKeyInfo(CubeDesc cubeDesc) {
+        RowKeyDesc rowKeyDesc = cubeDesc.getRowkey();
+        RowKeyColInfo info = new RowKeyColInfo();
+        info.hierachyColBitIndice = new ArrayList<List<Integer>>();
+        info.nonHierachyColBitIndice = new ArrayList<Integer>();
+        HashSet<Integer> heirachyIndexSet = new HashSet<Integer>();
+
+        for (DimensionDesc dim : cubeDesc.getDimensions()) {
+            if (dim.getHierarchy() != null) {
+                LinkedList<Integer> hlist = new LinkedList<Integer>();
+                for (HierarchyDesc hierarchyDesc : dim.getHierarchy()) {
+                    int index = rowKeyDesc.getColumnBitIndex(hierarchyDesc.getColumnRef());
+                    hlist.add(index);
+                    heirachyIndexSet.add(index);
+                }
+                info.hierachyColBitIndice.add(hlist);
+            }
+        }
+
+        for (int i = 0; i < rowKeyDesc.getRowKeyColumns().length; ++i) {
+            if (!heirachyIndexSet.contains(i)) {
+                info.nonHierachyColBitIndice.add(i);
+            }
+        }
+
+        return info;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/cube/src/main/java/com/kylinolap/cube/exception/CubeIntegrityException.java
----------------------------------------------------------------------
diff --git a/cube/src/main/java/com/kylinolap/cube/exception/CubeIntegrityException.java b/cube/src/main/java/com/kylinolap/cube/exception/CubeIntegrityException.java
new file mode 100644
index 0000000..a0ab338
--- /dev/null
+++ b/cube/src/main/java/com/kylinolap/cube/exception/CubeIntegrityException.java
@@ -0,0 +1,33 @@
+/*
+ * 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.cube.exception;
+
+/**
+ * @author xduo
+ * 
+ */
+public class CubeIntegrityException extends Exception {
+    /**
+     * @param string
+     */
+    public CubeIntegrityException(String string) {
+        super(string);
+    }
+
+    private static final long serialVersionUID = -7924187859607404390L;
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/cube/src/main/java/com/kylinolap/cube/invertedindex/BitMapContainer.java
----------------------------------------------------------------------
diff --git a/cube/src/main/java/com/kylinolap/cube/invertedindex/BitMapContainer.java b/cube/src/main/java/com/kylinolap/cube/invertedindex/BitMapContainer.java
new file mode 100644
index 0000000..f1af35e
--- /dev/null
+++ b/cube/src/main/java/com/kylinolap/cube/invertedindex/BitMapContainer.java
@@ -0,0 +1,190 @@
+/*
+ * 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.cube.invertedindex;
+
+import it.uniroma3.mat.extendedset.intset.ConciseSet;
+
+import java.nio.ByteBuffer;
+import java.nio.IntBuffer;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+
+import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
+
+import com.kylinolap.common.util.BytesUtil;
+import com.kylinolap.dict.Dictionary;
+
+/**
+ * @author yangli9
+ * 
+ */
+public class BitMapContainer implements ColumnValueContainer {
+
+    int valueLen;
+    int nValues;
+    int size;
+    ConciseSet[] sets;
+    boolean closedForChange;
+    
+    transient byte[] temp;
+
+    public BitMapContainer(TableRecordInfoDigest info, int col) {
+        this.valueLen = info.length(col);
+        this.size = 0;
+        this.nValues = info.getMaxID(col) + 1;
+        this.sets = null;
+        this.closedForChange = false;
+        
+        this.temp = new byte[valueLen];
+    }
+
+    @Override
+    public void append(ImmutableBytesWritable valueBytes) {
+        int value = BytesUtil.readUnsigned(valueBytes.get(), valueBytes.getOffset(), valueLen);
+        append(value);
+    }
+
+    public void append(int value) {
+        checkUpdateMode();
+        if (value == Dictionary.NULL_ID[valueLen]) {
+            value = nValues; // set[nValues] holds NULL
+        }
+        sets[value].add(size);
+        size++;
+    }
+    
+    @Override
+    public void getValueAt(int i, ImmutableBytesWritable valueBytes) {
+        int value = getValueIntAt(i);
+        BytesUtil.writeUnsigned(value, temp, 0, valueLen);
+        valueBytes.set(temp, 0, valueLen);
+    }
+    
+    public int getValueIntAt(int i) {
+        for (int v = 0; v < nValues; v++) {
+            if (sets[v].contains(i)) {
+                return v;
+            }
+        }
+        // if v is not in [0..nValues-1], then it must be nValue (NULL)
+        return Dictionary.NULL_ID[valueLen];
+    }
+
+    private void checkUpdateMode() {
+        if (isClosedForChange()) {
+            throw new IllegalStateException();
+        }
+        if (sets == null) {
+            sets = new ConciseSet[nValues + 1];
+            for (int i = 0; i <= nValues; i++) {
+                sets[i] = new ConciseSet();
+            }
+        }
+    }
+
+    private boolean isClosedForChange() {
+        return closedForChange;
+    }
+
+    @Override
+    public void closeForChange() {
+        closedForChange = true;
+    }
+
+    @Override
+    public int getSize() {
+        return size;
+    }
+
+    public List<ImmutableBytesWritable> toBytes() {
+        if (isClosedForChange() == false)
+            closeForChange();
+
+        List<ImmutableBytesWritable> r = new ArrayList<ImmutableBytesWritable>(nValues + 1);
+        for (int i = 0; i <= nValues; i++) {
+            r.add(setToBytes(sets[i]));
+        }
+        return r;
+    }
+
+    public void fromBytes(List<ImmutableBytesWritable> bytes) {
+        assert nValues + 1 == bytes.size();
+        sets = new ConciseSet[nValues + 1];
+        size = 0;
+        for (int i = 0; i <= nValues; i++) {
+            sets[i] = bytesToSet(bytes.get(i));
+            size += sets[i].size();
+        }
+        closedForChange = true;
+    }
+
+    private ImmutableBytesWritable setToBytes(ConciseSet set) {
+        byte[] array;
+        if (set.isEmpty()) // ConciseSet.toByteBuffer() throws exception when set is empty
+            array = BytesUtil.EMPTY_BYTE_ARRAY;
+        else
+            array = set.toByteBuffer().array();
+        return new ImmutableBytesWritable(array);
+    }
+
+    private ConciseSet bytesToSet(ImmutableBytesWritable bytes) {
+        if (bytes.get() == null || bytes.getLength() == 0) {
+            return new ConciseSet();
+        } else {
+            IntBuffer intBuffer = ByteBuffer.wrap(bytes.get(), bytes.getOffset(), bytes.getLength()).asIntBuffer();
+            int[] words = new int[intBuffer.capacity()];
+            intBuffer.get(words);
+            return new ConciseSet(words, false);
+        }
+    }
+
+    @Override
+    public int hashCode() {
+        final int prime = 31;
+        int result = 1;
+        result = prime * result + (closedForChange ? 1231 : 1237);
+        result = prime * result + nValues;
+        result = prime * result + Arrays.hashCode(sets);
+        result = prime * result + size;
+        result = prime * result + valueLen;
+        return result;
+    }
+
+    @Override
+    public boolean equals(Object obj) {
+        if (this == obj)
+            return true;
+        if (obj == null)
+            return false;
+        if (getClass() != obj.getClass())
+            return false;
+        BitMapContainer other = (BitMapContainer) obj;
+        if (closedForChange != other.closedForChange)
+            return false;
+        if (nValues != other.nValues)
+            return false;
+        if (!Arrays.equals(sets, other.sets))
+            return false;
+        if (size != other.size)
+            return false;
+        if (valueLen != other.valueLen)
+            return false;
+        return true;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/cube/src/main/java/com/kylinolap/cube/invertedindex/ColumnValueContainer.java
----------------------------------------------------------------------
diff --git a/cube/src/main/java/com/kylinolap/cube/invertedindex/ColumnValueContainer.java b/cube/src/main/java/com/kylinolap/cube/invertedindex/ColumnValueContainer.java
new file mode 100644
index 0000000..a2ac4c3
--- /dev/null
+++ b/cube/src/main/java/com/kylinolap/cube/invertedindex/ColumnValueContainer.java
@@ -0,0 +1,36 @@
+/*
+ * 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.cube.invertedindex;
+
+import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
+
+/**
+ * @author yangli9
+ * 
+ */
+public interface ColumnValueContainer {
+
+    void append(ImmutableBytesWritable valueBytes);
+
+    void closeForChange();
+
+    int getSize();
+
+    // works only after closeForChange()
+    void getValueAt(int i, ImmutableBytesWritable valueBytes);
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/cube/src/main/java/com/kylinolap/cube/invertedindex/CompressedValueContainer.java
----------------------------------------------------------------------
diff --git a/cube/src/main/java/com/kylinolap/cube/invertedindex/CompressedValueContainer.java b/cube/src/main/java/com/kylinolap/cube/invertedindex/CompressedValueContainer.java
new file mode 100644
index 0000000..6874e8f
--- /dev/null
+++ b/cube/src/main/java/com/kylinolap/cube/invertedindex/CompressedValueContainer.java
@@ -0,0 +1,132 @@
+/*
+ * 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.cube.invertedindex;
+
+import java.io.IOException;
+import java.util.Arrays;
+
+import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
+import org.apache.hadoop.hbase.util.Bytes;
+
+import com.kylinolap.common.util.BytesUtil;
+import com.ning.compress.lzf.LZFDecoder;
+import com.ning.compress.lzf.LZFEncoder;
+
+/**
+ * @author yangli9
+ * 
+ */
+public class CompressedValueContainer implements ColumnValueContainer {
+    int valueLen;
+    int cap;
+    int size;
+    byte[] uncompressed;
+    byte[] compressed;
+
+    public CompressedValueContainer(TableRecordInfoDigest info, int col, int cap) {
+        this.valueLen = info.length(col);
+        this.cap = cap;
+        this.size = 0;
+        this.uncompressed = null;
+        this.compressed = null;
+    }
+
+    @Override
+    public void append(ImmutableBytesWritable valueBytes) {
+        checkUpdateMode();
+        System.arraycopy(valueBytes.get(), valueBytes.getOffset(), uncompressed, valueLen * size, valueLen);
+        size++;
+    }
+
+    @Override
+    public void getValueAt(int i, ImmutableBytesWritable valueBytes) {
+        valueBytes.set(uncompressed, valueLen * i, valueLen);
+    }
+
+    private void checkUpdateMode() {
+        if (isClosedForChange()) {
+            throw new IllegalArgumentException();
+        }
+        if (uncompressed == null) {
+            uncompressed = new byte[valueLen * cap];
+        }
+    }
+
+    private boolean isClosedForChange() {
+        return compressed != null;
+    }
+
+    @Override
+    public void closeForChange() {
+        checkUpdateMode();
+        try {
+            compressed = LZFEncoder.encode(uncompressed, 0, valueLen * size);
+        } catch (Exception e) {
+            throw new RuntimeException("LZF encode failure", e);
+        }
+    }
+
+    @Override
+    public int getSize() {
+        return size;
+    }
+
+    public ImmutableBytesWritable toBytes() {
+        if (isClosedForChange() == false)
+            closeForChange();
+        return new ImmutableBytesWritable(compressed);
+    }
+
+    public void fromBytes(ImmutableBytesWritable bytes) {
+        try {
+            uncompressed = LZFDecoder.decode(bytes.get(), bytes.getOffset(), bytes.getLength());
+        } catch (IOException e) {
+            throw new RuntimeException("LZF decode failure", e);
+        }
+        size = cap = uncompressed.length / valueLen;
+        compressed = BytesUtil.EMPTY_BYTE_ARRAY; // mark closed
+    }
+
+    @Override
+    public int hashCode() {
+        final int prime = 31;
+        int result = 1;
+        result = prime * result + size;
+        result = prime * result + valueLen;
+        result = prime * result + Arrays.hashCode(uncompressed);
+        return result;
+    }
+
+    @Override
+    public boolean equals(Object obj) {
+        if (this == obj)
+            return true;
+        if (obj == null)
+            return false;
+        if (getClass() != obj.getClass())
+            return false;
+        CompressedValueContainer other = (CompressedValueContainer) obj;
+        if (size != other.size)
+            return false;
+        if (valueLen != other.valueLen)
+            return false;
+        if (!Bytes.equals(uncompressed, 0, size * valueLen, uncompressed, 0, size * valueLen))
+            return false;
+        return true;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/cube/src/main/java/com/kylinolap/cube/invertedindex/IIKeyValueCodec.java
----------------------------------------------------------------------
diff --git a/cube/src/main/java/com/kylinolap/cube/invertedindex/IIKeyValueCodec.java b/cube/src/main/java/com/kylinolap/cube/invertedindex/IIKeyValueCodec.java
new file mode 100644
index 0000000..7e77136
--- /dev/null
+++ b/cube/src/main/java/com/kylinolap/cube/invertedindex/IIKeyValueCodec.java
@@ -0,0 +1,243 @@
+/*
+ * 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.cube.invertedindex;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Iterator;
+import java.util.List;
+
+import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
+import org.apache.hadoop.hbase.util.Pair;
+
+import com.google.common.collect.Lists;
+import com.kylinolap.common.util.BytesUtil;
+
+/**
+ * @author yangli9
+ */
+public class IIKeyValueCodec {
+
+    public static final int SHARD_LEN = 2;
+    public static final int TIMEPART_LEN = 8;
+    public static final int COLNO_LEN = 2;
+
+    private TableRecordInfoDigest infoDigest;
+
+    public IIKeyValueCodec(TableRecordInfoDigest info) {
+        this.infoDigest = info;
+    }
+
+    public Collection<Pair<ImmutableBytesWritable, ImmutableBytesWritable>> encodeKeyValue(Slice slice) {
+        ArrayList<Pair<ImmutableBytesWritable, ImmutableBytesWritable>> result = Lists.newArrayList();
+        ColumnValueContainer[] containers = slice.containers;
+        for (int col = 0; col < containers.length; col++) {
+            if (containers[col] instanceof BitMapContainer) {
+                collectKeyValues(slice, col, (BitMapContainer) containers[col], result);
+            } else if (containers[col] instanceof CompressedValueContainer) {
+                collectKeyValues(slice, col, (CompressedValueContainer) containers[col], result);
+            } else {
+                throw new IllegalArgumentException("Unkown container class " + containers[col].getClass());
+            }
+        }
+        return result;
+    }
+
+    private void collectKeyValues(Slice slice, int col, CompressedValueContainer container, //
+            ArrayList<Pair<ImmutableBytesWritable, ImmutableBytesWritable>> result) {
+        ImmutableBytesWritable key = encodeKey(slice.getShard(), slice.getTimestamp(), col, -1);
+        ImmutableBytesWritable value = container.toBytes();
+        result.add(new Pair<ImmutableBytesWritable, ImmutableBytesWritable>(key, value));
+    }
+
+    private void collectKeyValues(Slice slice, int col, BitMapContainer container, //
+            ArrayList<Pair<ImmutableBytesWritable, ImmutableBytesWritable>> result) {
+        List<ImmutableBytesWritable> values = container.toBytes();
+        for (int v = 0; v < values.size(); v++) {
+            ImmutableBytesWritable key = encodeKey(slice.getShard(), slice.getTimestamp(), col, v);
+            result.add(new Pair<ImmutableBytesWritable, ImmutableBytesWritable>(key, values.get(v)));
+        }
+    }
+
+    ImmutableBytesWritable encodeKey(short shard, long timestamp, int col, int colValue) {
+        byte[] bytes = new byte[20];
+        int len = encodeKey(shard, timestamp, col, colValue, bytes, 0);
+        return new ImmutableBytesWritable(bytes, 0, len);
+    }
+
+    int encodeKey(short shard, long timestamp, int col, int colValue, byte[] buf, int offset) {
+        int i = offset;
+
+        BytesUtil.writeUnsigned(shard, buf, i, SHARD_LEN);
+        i += SHARD_LEN;
+        BytesUtil.writeLong(timestamp, buf, i, TIMEPART_LEN);
+        i += TIMEPART_LEN;
+
+        BytesUtil.writeUnsigned(col, buf, i, COLNO_LEN);
+        i += COLNO_LEN;
+
+        if (colValue >= 0) {
+            int colLen = infoDigest.length(col);
+            BytesUtil.writeUnsigned(colValue, buf, i, colLen);
+            i += colLen;
+        }
+
+        return i - offset;
+    }
+
+    public Iterable<Slice> decodeKeyValue(Iterable<Pair<ImmutableBytesWritable, ImmutableBytesWritable>> kvs) {
+        return new Decoder(infoDigest, kvs);
+    }
+
+    private static class Decoder implements Iterable<Slice> {
+
+        TableRecordInfoDigest info;
+        Iterator<Pair<ImmutableBytesWritable, ImmutableBytesWritable>> iterator;
+
+        Slice next = null;
+        short curShard = Short.MIN_VALUE;
+        long curSliceTimestamp = Long.MIN_VALUE;
+        int curCol = -1;
+        int curColValue = -1;
+        short lastShard = Short.MIN_VALUE;
+        long lastSliceTimestamp = Long.MIN_VALUE;
+        int lastCol = -1;
+        ColumnValueContainer[] containers = null;
+        List<ImmutableBytesWritable> bitMapValues = Lists.newArrayList();
+
+        Decoder(TableRecordInfoDigest info, Iterable<Pair<ImmutableBytesWritable, ImmutableBytesWritable>> kvs) {
+            this.info = info;
+            this.iterator = kvs.iterator();
+        }
+
+        private void goToNext() {
+            if (next != null) { // was not fetched
+                return;
+            }
+
+            // NOTE the input keys are ordered
+            while (next == null && iterator.hasNext()) {
+                Pair<ImmutableBytesWritable, ImmutableBytesWritable> kv = iterator.next();
+                ImmutableBytesWritable k = kv.getFirst();
+                ImmutableBytesWritable v = kv.getSecond();
+                decodeKey(k);
+
+                if (curShard != lastShard || curSliceTimestamp != lastSliceTimestamp) {
+                    makeNext();
+                }
+                consumeCurrent(v);
+            }
+            if (next == null) {
+                makeNext();
+            }
+        }
+
+        private void decodeKey(ImmutableBytesWritable k) {
+            byte[] buf = k.get();
+            int i = k.getOffset();
+
+            curShard = (short) BytesUtil.readUnsigned(buf, i, SHARD_LEN);
+            i += SHARD_LEN;
+            curSliceTimestamp = BytesUtil.readLong(buf, i, TIMEPART_LEN);
+            i += TIMEPART_LEN;
+
+            curCol = BytesUtil.readUnsigned(buf, i, COLNO_LEN);
+            i += COLNO_LEN;
+
+            if (i - k.getOffset() < k.getLength()) {
+                //bitmap
+                int colLen = info.length(curCol);
+                curColValue = BytesUtil.readUnsigned(buf, i, colLen);
+                i += colLen;
+            } else {
+                //value list
+                curColValue = -1;
+            }
+        }
+
+        private void consumeCurrent(ImmutableBytesWritable v) {
+            if (curCol != lastCol && bitMapValues.size() > 0) { // end of a bitmap container
+                addBitMapContainer(lastCol);
+            }
+            if (curColValue < 0) {
+                CompressedValueContainer c = new CompressedValueContainer(info, curCol, 0);
+                c.fromBytes(v);
+                addContainer(curCol, c);
+            } else {
+                assert curColValue == bitMapValues.size();
+                // make a copy, the value object from caller is typically reused through iteration
+                bitMapValues.add(new ImmutableBytesWritable(v));
+            }
+
+            lastShard = curShard;
+            lastSliceTimestamp = curSliceTimestamp;
+            lastCol = curCol;
+        }
+
+        private void makeNext() {
+            if (bitMapValues.isEmpty() == false) {
+                addBitMapContainer(lastCol);
+            }
+            if (containers != null) {
+                next = new Slice(info, lastShard, lastSliceTimestamp, containers);
+            }
+            lastSliceTimestamp = Long.MIN_VALUE;
+            lastCol = -1;
+            containers = null;
+            bitMapValues.clear();
+        }
+
+        private void addBitMapContainer(int col) {
+            BitMapContainer c = new BitMapContainer(info, col);
+            c.fromBytes(bitMapValues);
+            addContainer(col, c);
+            bitMapValues.clear();
+        }
+
+        private void addContainer(int col, ColumnValueContainer c) {
+            if (containers == null) {
+                containers = new ColumnValueContainer[info.getColumnCount()];
+            }
+            containers[col] = c;
+        }
+
+        @Override
+        public Iterator<Slice> iterator() {
+            return new Iterator<Slice>() {
+                @Override
+                public boolean hasNext() {
+                    goToNext();
+                    return next != null;
+                }
+
+                @Override
+                public Slice next() {
+                    Slice result = next;
+                    next = null;
+                    return result;
+                }
+
+                @Override
+                public void remove() {
+                    throw new UnsupportedOperationException();
+                }
+            };
+        }
+
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/cube/src/main/java/com/kylinolap/cube/invertedindex/InvertedIndexCLI.java
----------------------------------------------------------------------
diff --git a/cube/src/main/java/com/kylinolap/cube/invertedindex/InvertedIndexCLI.java b/cube/src/main/java/com/kylinolap/cube/invertedindex/InvertedIndexCLI.java
new file mode 100644
index 0000000..059be73
--- /dev/null
+++ b/cube/src/main/java/com/kylinolap/cube/invertedindex/InvertedIndexCLI.java
@@ -0,0 +1,101 @@
+/*
+ * 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.cube.invertedindex;
+
+import java.io.IOException;
+import java.util.Iterator;
+
+import org.apache.commons.io.IOUtils;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
+import org.apache.hadoop.hbase.util.Pair;
+import org.apache.hadoop.io.SequenceFile;
+import org.apache.hadoop.io.SequenceFile.Reader;
+
+import com.kylinolap.common.KylinConfig;
+import com.kylinolap.common.util.HadoopUtil;
+import com.kylinolap.cube.CubeInstance;
+import com.kylinolap.cube.CubeManager;
+
+/**
+ * @author yangli9
+ * 
+ */
+public class InvertedIndexCLI {
+
+    public static void main(String[] args) throws IOException {
+        Configuration hconf = HadoopUtil.getDefaultConfiguration();
+        CubeManager mgr = CubeManager.getInstance(KylinConfig.getInstanceFromEnv());
+
+        String cubeName = args[0];
+        CubeInstance cube = mgr.getCube(cubeName);
+
+        String path = args[1];
+        System.out.println("Reading from " + path + " ...");
+
+        TableRecordInfo info = new TableRecordInfo(cube.getFirstSegment());
+        IIKeyValueCodec codec = new IIKeyValueCodec(info);
+        int count = 0;
+        for (Slice slice : codec.decodeKeyValue(readSequenceKVs(hconf, path))) {
+            for (TableRecordBytes rec : slice) {
+                System.out.println((TableRecord)rec);
+                count++;
+            }
+        }
+        System.out.println("Total " + count + " records");
+    }
+
+    public static Iterable<Pair<ImmutableBytesWritable, ImmutableBytesWritable>> readSequenceKVs(Configuration hconf, String path) throws IOException {
+        final Reader reader = new Reader(hconf, SequenceFile.Reader.file(new Path(path)));
+        return new Iterable<Pair<ImmutableBytesWritable, ImmutableBytesWritable>>() {
+            @Override
+            public Iterator<Pair<ImmutableBytesWritable, ImmutableBytesWritable>> iterator() {
+                return new Iterator<Pair<ImmutableBytesWritable, ImmutableBytesWritable>>() {
+                    ImmutableBytesWritable k = new ImmutableBytesWritable();
+                    ImmutableBytesWritable v = new ImmutableBytesWritable();
+                    Pair<ImmutableBytesWritable, ImmutableBytesWritable> pair = new Pair<ImmutableBytesWritable, ImmutableBytesWritable>(k, v);
+
+                    @Override
+                    public boolean hasNext() {
+                        boolean hasNext = false;
+                        try {
+                            hasNext = reader.next(k, v);
+                        } catch (IOException e) {
+                            throw new RuntimeException(e);
+                        } finally {
+                            if (hasNext == false) {
+                                IOUtils.closeQuietly(reader);
+                            }
+                        }
+                        return hasNext;
+                    }
+
+                    @Override
+                    public Pair<ImmutableBytesWritable, ImmutableBytesWritable> next() {
+                        return pair;
+                    }
+
+                    @Override
+                    public void remove() {
+                        throw new UnsupportedOperationException();
+                    }
+                };
+            }
+        };
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/cube/src/main/java/com/kylinolap/cube/invertedindex/ShardingHash.java
----------------------------------------------------------------------
diff --git a/cube/src/main/java/com/kylinolap/cube/invertedindex/ShardingHash.java b/cube/src/main/java/com/kylinolap/cube/invertedindex/ShardingHash.java
new file mode 100644
index 0000000..3bd4603
--- /dev/null
+++ b/cube/src/main/java/com/kylinolap/cube/invertedindex/ShardingHash.java
@@ -0,0 +1,15 @@
+package com.kylinolap.cube.invertedindex;
+
+import com.google.common.hash.HashFunction;
+import com.google.common.hash.Hashing;
+
+public class ShardingHash {
+
+    static HashFunction hashFunc = Hashing.murmur3_128();
+    
+    public static long hashInt(int integer) {
+        return hashFunc.newHasher().putInt(integer).hash().asLong();
+    }
+    
+    
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/cube/src/main/java/com/kylinolap/cube/invertedindex/ShardingSliceBuilder.java
----------------------------------------------------------------------
diff --git a/cube/src/main/java/com/kylinolap/cube/invertedindex/ShardingSliceBuilder.java b/cube/src/main/java/com/kylinolap/cube/invertedindex/ShardingSliceBuilder.java
new file mode 100644
index 0000000..e9ffaf0
--- /dev/null
+++ b/cube/src/main/java/com/kylinolap/cube/invertedindex/ShardingSliceBuilder.java
@@ -0,0 +1,35 @@
+package com.kylinolap.cube.invertedindex;
+
+import java.util.List;
+
+import com.google.common.collect.Lists;
+
+public class ShardingSliceBuilder {
+
+    SliceBuilder[] builders;
+    
+    public ShardingSliceBuilder(TableRecordInfo info) {
+        int sharding = info.getDescriptor().getSharding();
+        builders = new SliceBuilder[sharding];
+        for (short i = 0; i < sharding; i++) {
+            builders[i] = new SliceBuilder(info, i);
+        }
+    }
+    
+    // NOTE: record must be appended in time order
+    public Slice append(TableRecord rec) {
+        short shard = rec.getShard();
+        return builders[shard].append(rec);
+    }
+    
+    public List<Slice> close() {
+        List<Slice> result = Lists.newArrayList();
+        for (SliceBuilder builder : builders) {
+            Slice slice = builder.close();
+            if (slice != null)
+                result.add(slice);
+        }
+        return result;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/cube/src/main/java/com/kylinolap/cube/invertedindex/Slice.java
----------------------------------------------------------------------
diff --git a/cube/src/main/java/com/kylinolap/cube/invertedindex/Slice.java b/cube/src/main/java/com/kylinolap/cube/invertedindex/Slice.java
new file mode 100644
index 0000000..250ece4
--- /dev/null
+++ b/cube/src/main/java/com/kylinolap/cube/invertedindex/Slice.java
@@ -0,0 +1,175 @@
+/*
+ * 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.cube.invertedindex;
+
+import java.util.Iterator;
+
+import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
+
+/**
+ * Within a partition (per timestampGranularity), records are further sliced
+ * (per sliceLength) to fit into HBASE cell.
+ *
+ * @author yangli9
+ */
+public class Slice implements Iterable<TableRecordBytes>, Comparable<Slice> {
+
+    TableRecordInfoDigest info;
+    int nColumns;
+
+    short shard;
+    long timestamp;
+    int nRecords;
+    ColumnValueContainer[] containers;
+
+    Slice(TableRecordInfoDigest info, short shard, long timestamp, ColumnValueContainer[] containers) {
+        this.info = info;
+        this.nColumns = info.getColumnCount();
+
+        this.shard = shard;
+        this.timestamp = timestamp;
+        this.nRecords = containers[0].getSize();
+        this.containers = containers;
+
+        assert nColumns == containers.length;
+        for (int i = 0; i < nColumns; i++) {
+            assert nRecords == containers[i].getSize();
+        }
+    }
+
+    public short getShard() {
+        return shard;
+    }
+
+    public long getTimestamp() {
+        return timestamp;
+    }
+
+    /**
+     * Standard iterator of Slice will return a iterator of TableRecordBytes,
+     * which cannot be printed/formated to readable text.
+     * By invoking this API client can avoid code like:
+     * <p/>
+     * for (TableRecordBytes rec : slice) {
+     * TableRecord realRecord = (TableRecord) rec.clone();
+     * }
+     * <p/>
+     * Note this API cannot be called simultaneously with iterator()
+     *
+     * @return
+     */
+    public Iterator<TableRecord> readableIterator() {
+        final Iterator<TableRecordBytes> innerIterator = iterator();
+
+        return new Iterator<TableRecord>() {
+
+
+            @Override
+            public boolean hasNext() {
+                return innerIterator.hasNext();
+            }
+
+            @Override
+            public TableRecord next() {
+                return (TableRecord) innerIterator.next();
+            }
+
+            @Override
+            public void remove() {
+                throw new UnsupportedOperationException();
+            }
+        };
+    }
+
+    @Override
+    public Iterator<TableRecordBytes> iterator() {
+        return new Iterator<TableRecordBytes>() {
+            int i = 0;
+            TableRecord rec = new TableRecord(info);
+            ImmutableBytesWritable temp = new ImmutableBytesWritable();
+
+            @Override
+            public boolean hasNext() {
+                return i < nRecords;
+            }
+
+            @Override
+            public TableRecordBytes next() {
+                for (int col = 0; col < nColumns; col++) {
+                    containers[col].getValueAt(i, temp);
+                    rec.setValueBytes(col, temp);
+                }
+                i++;
+                return rec;
+            }
+
+            @Override
+            public void remove() {
+                throw new UnsupportedOperationException();
+            }
+
+        };
+    }
+
+    /* (non-Javadoc)
+     * @see java.lang.Object#hashCode()
+     */
+    @Override
+    public int hashCode() {
+        final int prime = 31;
+        int result = 1;
+        result = prime * result + ((info == null) ? 0 : info.hashCode());
+        result = prime * result + shard;
+        result = prime * result + (int) (timestamp ^ (timestamp >>> 32));
+        return result;
+    }
+
+    /* (non-Javadoc)
+     * @see java.lang.Object#equals(java.lang.Object)
+     */
+    @Override
+    public boolean equals(Object obj) {
+        if (this == obj)
+            return true;
+        if (obj == null)
+            return false;
+        if (getClass() != obj.getClass())
+            return false;
+        Slice other = (Slice) obj;
+        if (info == null) {
+            if (other.info != null)
+                return false;
+        } else if (!info.equals(other.info))
+            return false;
+        if (shard != other.shard)
+            return false;
+        if (timestamp != other.timestamp)
+            return false;
+        return true;
+    }
+
+    @Override
+    public int compareTo(Slice o) {
+        int comp = this.shard - o.shard;
+        if (comp != 0)
+            return comp;
+
+        comp = (int) (this.timestamp - o.timestamp);
+        return comp;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/cube/src/main/java/com/kylinolap/cube/invertedindex/SliceBuilder.java
----------------------------------------------------------------------
diff --git a/cube/src/main/java/com/kylinolap/cube/invertedindex/SliceBuilder.java b/cube/src/main/java/com/kylinolap/cube/invertedindex/SliceBuilder.java
new file mode 100644
index 0000000..72523a8
--- /dev/null
+++ b/cube/src/main/java/com/kylinolap/cube/invertedindex/SliceBuilder.java
@@ -0,0 +1,122 @@
+/*
+ * 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.cube.invertedindex;
+
+import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
+
+
+/**
+ * @author yangli9
+ * 
+ */
+public class SliceBuilder {
+
+    TableRecordInfo info;
+    private int nColumns;
+    int nRecordsCap;
+
+    short shard;
+    long sliceTimestamp;
+    int nRecords;
+    private ColumnValueContainer[] containers;
+    
+    transient ImmutableBytesWritable temp = new ImmutableBytesWritable();
+
+    public SliceBuilder(TableRecordInfo info, short shard) {
+        this.info = info;
+        this.nColumns = info.getColumnCount();
+        this.nRecordsCap = Math.max(1, info.getDescriptor().getSliceSize());
+
+        this.shard = shard;
+        this.sliceTimestamp = Long.MIN_VALUE;
+        this.nRecords = 0;
+        this.containers = null;
+        
+        doneSlice(); // init containers
+    }
+
+    private Slice doneSlice() {
+        Slice r = null;
+        if (nRecords > 0) {
+            for (int i = 0; i < nColumns; i++) {
+                containers[i].closeForChange();
+            }
+            r = new Slice(info, shard, sliceTimestamp, containers);
+        }
+
+        // reset for next slice
+        nRecords = 0;
+        containers = new ColumnValueContainer[nColumns];
+        for (int i : info.getDescriptor().getBitmapColumns()) {
+            containers[i] = new BitMapContainer(info, i);
+        }
+        for (int i : info.getDescriptor().getValueColumns()) {
+            containers[i] = new CompressedValueContainer(info, i, nRecordsCap);
+        }
+        for (int i : info.getDescriptor().getMetricsColumns()) {
+            containers[i] = new CompressedValueContainer(info, i, nRecordsCap);
+        }
+
+        return r;
+
+    }
+
+    // NOTE: record must be appended in time order
+    public Slice append(TableRecord rec) {
+        if (rec.getShard() != shard)
+            throw new IllegalStateException();
+        
+        Slice doneSlice = null;
+        
+        if (isFull()) {
+            doneSlice = doneSlice();
+        }
+        
+        if (nRecords == 0) {
+            sliceTimestamp = increaseSliceTimestamp(rec.getTimestamp());
+        }
+
+        nRecords++;
+        for (int i = 0; i < nColumns; i++) {
+            rec.getValueBytes(i, temp);
+            containers[i].append(temp);
+        }
+
+        return doneSlice;
+    }
+
+    private long increaseSliceTimestamp(long timestamp) {
+        if (timestamp < sliceTimestamp)
+            throw new IllegalStateException();
+        
+        if (timestamp == sliceTimestamp)
+            return ++timestamp; // ensure slice timestamp increases
+        else
+            return timestamp;
+    }
+
+    public Slice close() {
+        Slice doneSlice = doneSlice();
+        this.sliceTimestamp = Long.MIN_VALUE;
+        this.nRecords = 0;
+        return doneSlice;
+    }
+
+    private boolean isFull() {
+        return nRecords >= nRecordsCap;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/cube/src/main/java/com/kylinolap/cube/invertedindex/TableRecord.java
----------------------------------------------------------------------
diff --git a/cube/src/main/java/com/kylinolap/cube/invertedindex/TableRecord.java b/cube/src/main/java/com/kylinolap/cube/invertedindex/TableRecord.java
new file mode 100644
index 0000000..53601e2
--- /dev/null
+++ b/cube/src/main/java/com/kylinolap/cube/invertedindex/TableRecord.java
@@ -0,0 +1,122 @@
+/*
+ * 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.cube.invertedindex;
+
+import com.kylinolap.dict.DateStrDictionary;
+import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.io.LongWritable;
+
+/**
+ * @author yangli9
+ */
+public class TableRecord extends TableRecordBytes {
+
+
+    public TableRecord(TableRecordInfoDigest info) {
+        super();
+
+        if (info instanceof TableRecordInfo) {
+        } else {
+            throw new IllegalStateException("Table Record must be initialized with a TableRecordInfo");
+        }
+
+        this.info = info;
+        this.buf = new byte[info.byteFormLen];
+        reset();
+    }
+
+    public TableRecord(TableRecord another) {
+        super();
+
+        this.info = another.info;
+        this.buf = Bytes.copy(another.buf);
+    }
+
+    @Override
+    public Object clone() {
+        return new TableRecord(this);
+    }
+
+
+    public long getTimestamp() {
+        String str = getValueString(info().getTimestampColumn());
+        return DateStrDictionary.stringToMillis(str);
+    }
+
+    public int length(int col) {
+        return info.length(col);
+    }
+
+    public void setValueString(int col, String value) {
+        if (info().isMetrics(col)) {
+            LongWritable v = info().codec(col).valueOf(value);
+            setValueMetrics(col, v);
+        } else {
+            int id = info().dict(col).getIdFromValue(value);
+            setValueID(col, id);
+        }
+    }
+
+    public String getValueString(int col) {
+        if (info().isMetrics(col))
+            return info().codec(col).toString(getValueMetrics(col));
+        else
+            return info().dict(col).getValueFromId(getValueID(col));
+    }
+
+    public void setValueBytes(int col, ImmutableBytesWritable bytes) {
+        System.arraycopy(bytes.get(), bytes.getOffset(), buf, info.offset(col), info.length(col));
+    }
+    
+    public void getValueBytes(int col, ImmutableBytesWritable bytes) {
+        bytes.set(buf, info.offset(col), info.length(col));
+    }
+
+    private void setValueMetrics(int col, LongWritable value) {
+        info().codec(col).write(value, buf, info.offset(col));
+    }
+
+    private LongWritable getValueMetrics(int col) {
+        return info().codec(col).read(buf, info.offset(col));
+    }
+
+    public short getShard() {
+        int timestampID = getValueID(info().getTimestampColumn());
+        return (short) (Math.abs(ShardingHash.hashInt(timestampID)) % info().getDescriptor().getSharding());
+    }
+
+    public TableRecordInfo info() {
+        return (TableRecordInfo) info;
+    }
+
+
+    @Override
+    public String toString() {
+        StringBuilder buf = new StringBuilder("[");
+        for (int col = 0; col < info.getColumnCount(); col++) {
+            if (col > 0)
+                buf.append(",");
+            buf.append(getValueString(col));
+        }
+        buf.append("]");
+        return buf.toString();
+    }
+
+
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/cube/src/main/java/com/kylinolap/cube/invertedindex/TableRecordBytes.java
----------------------------------------------------------------------
diff --git a/cube/src/main/java/com/kylinolap/cube/invertedindex/TableRecordBytes.java b/cube/src/main/java/com/kylinolap/cube/invertedindex/TableRecordBytes.java
new file mode 100644
index 0000000..a40362a
--- /dev/null
+++ b/cube/src/main/java/com/kylinolap/cube/invertedindex/TableRecordBytes.java
@@ -0,0 +1,84 @@
+package com.kylinolap.cube.invertedindex;
+
+import com.kylinolap.common.util.BytesUtil;
+import com.kylinolap.dict.Dictionary;
+import org.apache.hadoop.hbase.util.Bytes;
+
+import java.util.Arrays;
+
+/**
+ * Created by honma on 11/10/14.
+ */
+public class TableRecordBytes implements Cloneable {
+    TableRecordInfoDigest info;
+    byte[] buf; // consecutive column value IDs (encoded by dictionary)
+
+    public TableRecordBytes() {
+    }
+
+    public void setValueID(int col, int id) {
+        BytesUtil.writeUnsigned(id, buf, info.offset(col), info.length(col));
+    }
+
+    public int getValueID(int col) {
+        return BytesUtil.readUnsigned(buf, info.offset(col), info.length(col));
+    }
+
+
+    public byte[] getBytes() {
+        return buf;
+    }
+
+    public void setBytes(byte[] bytes, int offset, int length) {
+        assert buf.length == length;
+        System.arraycopy(bytes, offset, buf, 0, length);
+    }
+
+    public void reset() {
+        Arrays.fill(buf, Dictionary.NULL);
+    }
+
+    public TableRecordBytes(TableRecordInfoDigest info) {
+        this.info = info;
+        this.buf = new byte[info.byteFormLen];
+        reset();
+    }
+
+    public TableRecordBytes(TableRecordBytes another) {
+        this.info = another.info;
+        this.buf = Bytes.copy(another.buf);
+    }
+
+    @Override
+    public Object clone() {
+        return new TableRecordBytes(this);
+    }
+
+    @Override
+    public int hashCode() {
+        final int prime = 31;
+        int result = 1;
+        result = prime * result + Arrays.hashCode(buf);
+        result = prime * result + ((info == null) ? 0 : info.hashCode());
+        return result;
+    }
+
+    @Override
+    public boolean equals(Object obj) {
+        if (this == obj)
+            return true;
+        if (obj == null)
+            return false;
+        if (getClass() != obj.getClass())
+            return false;
+        TableRecord other = (TableRecord) obj;
+        if (!Arrays.equals(buf, other.buf))
+            return false;
+        if (info == null) {
+            if (other.info != null)
+                return false;
+        } else if (!info.equals(other.info))
+            return false;
+        return true;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/cube/src/main/java/com/kylinolap/cube/invertedindex/TableRecordFactory.java
----------------------------------------------------------------------
diff --git a/cube/src/main/java/com/kylinolap/cube/invertedindex/TableRecordFactory.java b/cube/src/main/java/com/kylinolap/cube/invertedindex/TableRecordFactory.java
new file mode 100644
index 0000000..6c6e02d
--- /dev/null
+++ b/cube/src/main/java/com/kylinolap/cube/invertedindex/TableRecordFactory.java
@@ -0,0 +1,8 @@
+package com.kylinolap.cube.invertedindex;
+
+/**
+ * Created by honma on 11/10/14.
+ */
+public interface TableRecordFactory {
+    public TableRecordBytes createTableRecord();
+}


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

Posted by lu...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/server/src/main/java/com/kylinolap/rest/model/SelectedColumnMeta.java
----------------------------------------------------------------------
diff --git a/server/src/main/java/com/kylinolap/rest/model/SelectedColumnMeta.java b/server/src/main/java/com/kylinolap/rest/model/SelectedColumnMeta.java
new file mode 100644
index 0000000..28c5260
--- /dev/null
+++ b/server/src/main/java/com/kylinolap/rest/model/SelectedColumnMeta.java
@@ -0,0 +1,140 @@
+/*
+ * 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.rest.model;
+
+/**
+ * Created by lukhan on 2/27/14.
+ */
+public class SelectedColumnMeta {
+    public SelectedColumnMeta(boolean isAutoIncrement, boolean isCaseSensitive, boolean isSearchable, boolean isCurrency, int isNullalbe, boolean isSigned, int displaySize, String label, String name, String schemaName, String catelogName, String tableName, int precision, int scale, int columnType, String columnTypeName, boolean isReadOnly, boolean isWritable, boolean isDefinitelyWritable) {
+        super();
+        this.isAutoIncrement = isAutoIncrement;
+        this.isCaseSensitive = isCaseSensitive;
+        this.isSearchable = isSearchable;
+        this.isCurrency = isCurrency;
+        this.isNullable = isNullalbe;
+        this.isSigned = isSigned;
+        this.displaySize = displaySize;
+        this.label = label;
+        this.name = name;
+        this.schemaName = schemaName;
+        this.catelogName = catelogName;
+        this.tableName = tableName;
+        this.precision = precision;
+        this.scale = scale;
+        this.columnType = columnType;
+        this.columnTypeName = columnTypeName;
+        this.isReadOnly = isReadOnly;
+        this.isWritable = isWritable;
+        this.isDefinitelyWritable = isDefinitelyWritable;
+    }
+
+    public boolean isAutoIncrement() {
+        return isAutoIncrement;
+    }
+
+    public boolean isCaseSensitive() {
+        return isCaseSensitive;
+    }
+
+    public boolean isSearchable() {
+        return isSearchable;
+    }
+
+    public boolean isCurrency() {
+        return isCurrency;
+    }
+
+    public int getIsNullable() {
+        return isNullable;
+    }
+
+    public boolean isSigned() {
+        return isSigned;
+    }
+
+    public int getDisplaySize() {
+        return displaySize;
+    }
+
+    public String getLabel() {
+        return label;
+    }
+
+    public String getName() {
+        return name;
+    }
+
+    public String getSchemaName() {
+        return schemaName;
+    }
+
+    public String getCatelogName() {
+        return catelogName;
+    }
+
+    public String getTableName() {
+        return tableName;
+    }
+
+    public int getPrecision() {
+        return precision;
+    }
+
+    public int getScale() {
+        return scale;
+    }
+
+    public int getColumnType() {
+        return columnType;
+    }
+
+    public String getColumnTypeName() {
+        return columnTypeName;
+    }
+
+    public boolean isReadOnly() {
+        return isReadOnly;
+    }
+
+    public boolean isWritable() {
+        return isWritable;
+    }
+
+    public boolean isDefinitelyWritable() {
+        return isDefinitelyWritable;
+    }
+
+    private final boolean isAutoIncrement;
+    private final boolean isCaseSensitive;
+    private final boolean isSearchable;
+    private final boolean isCurrency;
+    private final int isNullable;// 0:nonull, 1:nullable, 2: nullableunknown
+    private final boolean isSigned;
+    private final int displaySize;
+    private final String label;// AS keyword
+    private final String name;
+    private final String schemaName;
+    private final String catelogName;
+    private final String tableName;
+    private final int precision;
+    private final int scale;
+    private final int columnType;// as defined in java.sql.Types
+    private final String columnTypeName;
+    private final boolean isReadOnly;
+    private final boolean isWritable;
+    private final boolean isDefinitelyWritable;
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/server/src/main/java/com/kylinolap/rest/model/TableMeta.java
----------------------------------------------------------------------
diff --git a/server/src/main/java/com/kylinolap/rest/model/TableMeta.java b/server/src/main/java/com/kylinolap/rest/model/TableMeta.java
new file mode 100644
index 0000000..98580f4
--- /dev/null
+++ b/server/src/main/java/com/kylinolap/rest/model/TableMeta.java
@@ -0,0 +1,148 @@
+/*
+ * 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.rest.model;
+
+import java.io.Serializable;
+import java.util.ArrayList;
+import java.util.List;
+
+/**
+ * Created by lukhan on 2/27/14.
+ */
+public class TableMeta implements Serializable {
+
+    private static final long serialVersionUID = 1L;
+    private String TABLE_CAT;
+    private String TABLE_SCHEM;
+    private String TABLE_NAME;
+    private String TABLE_TYPE;
+    private String REMARKS;
+    private String TYPE_CAT;
+    private String TYPE_SCHEM;
+    private String TYPE_NAME;
+    private String SELF_REFERENCING_COL_NAME;
+    private String REF_GENERATION;
+    private List<ColumnMeta> columns = new ArrayList<ColumnMeta>();
+
+    public TableMeta() {
+    }
+
+    public TableMeta(String tABLE_CAT, String tABLE_SCHEM, String tABLE_NAME, String tABLE_TYPE, String rEMARKS, String tYPE_CAT, String tYPE_SCHEM, String tYPE_NAME, String sELF_REFERENCING_COL_NAME, String rEF_GENERATION) {
+        super();
+        TABLE_CAT = tABLE_CAT;
+        TABLE_SCHEM = tABLE_SCHEM;
+        TABLE_NAME = tABLE_NAME;
+        TABLE_TYPE = tABLE_TYPE;
+        REMARKS = rEMARKS;
+        TYPE_CAT = tYPE_CAT;
+        TYPE_SCHEM = tYPE_SCHEM;
+        TYPE_NAME = tYPE_NAME;
+        SELF_REFERENCING_COL_NAME = sELF_REFERENCING_COL_NAME;
+        REF_GENERATION = rEF_GENERATION;
+    }
+
+    public String getTABLE_CAT() {
+        return TABLE_CAT;
+    }
+
+    public void setTABLE_CAT(String tABLE_CAT) {
+        TABLE_CAT = tABLE_CAT;
+    }
+
+    public String getTABLE_SCHEM() {
+        return TABLE_SCHEM;
+    }
+
+    public void setTABLE_SCHEM(String tABLE_SCHEM) {
+        TABLE_SCHEM = tABLE_SCHEM;
+    }
+
+    public String getTABLE_NAME() {
+        return TABLE_NAME;
+    }
+
+    public void setTABLE_NAME(String tABLE_NAME) {
+        TABLE_NAME = tABLE_NAME;
+    }
+
+    public String getTABLE_TYPE() {
+        return TABLE_TYPE;
+    }
+
+    public void setTABLE_TYPE(String tABLE_TYPE) {
+        TABLE_TYPE = tABLE_TYPE;
+    }
+
+    public String getREMARKS() {
+        return REMARKS;
+    }
+
+    public void setREMARKS(String rEMARKS) {
+        REMARKS = rEMARKS;
+    }
+
+    public String getTYPE_CAT() {
+        return TYPE_CAT;
+    }
+
+    public void setTYPE_CAT(String tYPE_CAT) {
+        TYPE_CAT = tYPE_CAT;
+    }
+
+    public String getTYPE_SCHEM() {
+        return TYPE_SCHEM;
+    }
+
+    public void setTYPE_SCHEM(String tYPE_SCHEM) {
+        TYPE_SCHEM = tYPE_SCHEM;
+    }
+
+    public String getTYPE_NAME() {
+        return TYPE_NAME;
+    }
+
+    public void setTYPE_NAME(String tYPE_NAME) {
+        TYPE_NAME = tYPE_NAME;
+    }
+
+    public String getSELF_REFERENCING_COL_NAME() {
+        return SELF_REFERENCING_COL_NAME;
+    }
+
+    public void setSELF_REFERENCING_COL_NAME(String sELF_REFERENCING_COL_NAME) {
+        SELF_REFERENCING_COL_NAME = sELF_REFERENCING_COL_NAME;
+    }
+
+    public String getREF_GENERATION() {
+        return REF_GENERATION;
+    }
+
+    public void setREF_GENERATION(String rEF_GENERATION) {
+        REF_GENERATION = rEF_GENERATION;
+    }
+
+    public List<ColumnMeta> getColumns() {
+        return columns;
+    }
+
+    public void setColumns(List<ColumnMeta> columns) {
+        this.columns = columns;
+    }
+
+    public void addColumn(ColumnMeta column) {
+        this.columns.add(column);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/server/src/main/java/com/kylinolap/rest/request/AccessRequest.java
----------------------------------------------------------------------
diff --git a/server/src/main/java/com/kylinolap/rest/request/AccessRequest.java b/server/src/main/java/com/kylinolap/rest/request/AccessRequest.java
new file mode 100644
index 0000000..220aaad
--- /dev/null
+++ b/server/src/main/java/com/kylinolap/rest/request/AccessRequest.java
@@ -0,0 +1,65 @@
+/*
+ * 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.rest.request;
+
+/**
+ * @author xduo
+ * 
+ */
+public class AccessRequest {
+
+    private Long accessEntryId;
+    private String permission;
+    private String sid;
+    private boolean principal;
+
+    public AccessRequest() {
+    }
+
+    public Long getAccessEntryId() {
+        return accessEntryId;
+    }
+
+    public void setAccessEntryId(Long accessEntryId) {
+        this.accessEntryId = accessEntryId;
+    }
+
+    public String getPermission() {
+        return permission;
+    }
+
+    public void setPermission(String permission) {
+        this.permission = permission;
+    }
+
+    public String getSid() {
+        return sid;
+    }
+
+    public void setSid(String sid) {
+        this.sid = sid;
+    }
+
+    public boolean isPrincipal() {
+        return principal;
+    }
+
+    public void setPrincipal(boolean principal) {
+        this.principal = principal;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/server/src/main/java/com/kylinolap/rest/request/CardinalityRequest.java
----------------------------------------------------------------------
diff --git a/server/src/main/java/com/kylinolap/rest/request/CardinalityRequest.java b/server/src/main/java/com/kylinolap/rest/request/CardinalityRequest.java
new file mode 100644
index 0000000..06294f8
--- /dev/null
+++ b/server/src/main/java/com/kylinolap/rest/request/CardinalityRequest.java
@@ -0,0 +1,75 @@
+/*
+ * 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.rest.request;
+
+/**
+ * @author jianliu
+ * 
+ */
+public class CardinalityRequest {
+    private int delimiter;
+
+    private int format;
+
+    /**
+     * @return the delimiter
+     */
+    public String getDelimiter() {
+        switch (delimiter) {
+        case 0:
+            return null;
+        case 1:
+            return "177";
+        case 2:
+            return "t";
+        default:
+            return null;
+        }
+    }
+
+    /**
+     * @param delimiter
+     *            the delimiter to set
+     */
+    public void setDelimiter(int delimiter) {
+        this.delimiter = delimiter;
+    }
+
+    /**
+     * @return the format
+     */
+    public String getFormat() {
+        switch (format) {
+        case 0:
+            return null;
+        case 1:
+            return "text";
+        case 2:
+            return "sequence";
+        default:
+            return null;
+        }
+    }
+
+    /**
+     * @param format
+     *            the format to set
+     */
+    public void setFormat(int format) {
+        this.format = format;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/server/src/main/java/com/kylinolap/rest/request/CreateProjectRequest.java
----------------------------------------------------------------------
diff --git a/server/src/main/java/com/kylinolap/rest/request/CreateProjectRequest.java b/server/src/main/java/com/kylinolap/rest/request/CreateProjectRequest.java
new file mode 100644
index 0000000..49c7f4a
--- /dev/null
+++ b/server/src/main/java/com/kylinolap/rest/request/CreateProjectRequest.java
@@ -0,0 +1,29 @@
+package com.kylinolap.rest.request;
+
+/**
+ * Created by honma on 8/7/14.
+ */
+public class CreateProjectRequest {
+    private String name;
+    private String description;
+
+    public CreateProjectRequest() {
+    }
+
+    public String getName() {
+        return name;
+    }
+
+    public void setName(String name) {
+        this.name = name;
+    }
+
+    public String getDescription() {
+        return description;
+    }
+
+    public void setDescription(String description) {
+        this.description = description;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/server/src/main/java/com/kylinolap/rest/request/CubeRequest.java
----------------------------------------------------------------------
diff --git a/server/src/main/java/com/kylinolap/rest/request/CubeRequest.java b/server/src/main/java/com/kylinolap/rest/request/CubeRequest.java
new file mode 100644
index 0000000..51151b7
--- /dev/null
+++ b/server/src/main/java/com/kylinolap/rest/request/CubeRequest.java
@@ -0,0 +1,120 @@
+/*
+ * 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.rest.request;
+
+public class CubeRequest {
+
+    private String uuid;
+    private String cubeName;
+    private String cubeDescData;
+    private boolean successful;
+    private String message;
+    private String cubeDescName;
+    private String project;
+
+    public String getUuid() {
+        return uuid;
+    }
+
+    public void setUuid(String uuid) {
+        this.uuid = uuid;
+    }
+
+    /**
+     * @return the cubeDescName
+     */
+    public String getCubeDescName() {
+        return cubeDescName;
+    }
+
+    /**
+     * @param cubeDescName
+     *            the cubeDescName to set
+     */
+    public void setCubeDescName(String cubeDescName) {
+        this.cubeDescName = cubeDescName;
+    }
+
+    /**
+     * @return the message
+     */
+    public String getMessage() {
+        return message;
+    }
+
+    /**
+     * @param message
+     *            the message to set
+     */
+    public void setMessage(String message) {
+        this.message = message;
+    }
+
+    /**
+     * @return the status
+     */
+    public boolean getSuccessful() {
+        return successful;
+    }
+
+    /**
+     * @param status
+     *            the status to set
+     */
+    public void setSuccessful(boolean status) {
+        this.successful = status;
+    }
+
+    public CubeRequest() {
+    }
+
+    public CubeRequest(long id, String cubeName, String cubeDescData) {
+        this.cubeName = cubeName;
+        this.cubeDescData = cubeDescData;
+    }
+
+    public String getCubeDescData() {
+        return cubeDescData;
+    }
+
+    public void setCubeDescData(String cubeDescData) {
+        this.cubeDescData = cubeDescData;
+    }
+
+    /**
+     * @return the cubeDescName
+     */
+    public String getCubeName() {
+        return cubeName;
+    }
+
+    /**
+     * @param cubeName
+     *            the cubeDescName to set
+     */
+    public void setCubeName(String cubeName) {
+        this.cubeName = cubeName;
+    }
+
+    public String getProject() {
+        return project;
+    }
+
+    public void setProject(String project) {
+        this.project = project;
+    }
+
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/server/src/main/java/com/kylinolap/rest/request/JobBuildRequest.java
----------------------------------------------------------------------
diff --git a/server/src/main/java/com/kylinolap/rest/request/JobBuildRequest.java b/server/src/main/java/com/kylinolap/rest/request/JobBuildRequest.java
new file mode 100644
index 0000000..77636d3
--- /dev/null
+++ b/server/src/main/java/com/kylinolap/rest/request/JobBuildRequest.java
@@ -0,0 +1,55 @@
+/*
+ * 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.rest.request;
+
+/**
+ * @author xduo
+ * 
+ */
+public class JobBuildRequest {
+
+    private long startTime;
+
+    private long endTime;
+
+    private String buildType;
+
+    public long getStartTime() {
+        return startTime;
+    }
+
+    public void setStartTime(long startTime) {
+        this.startTime = startTime;
+    }
+
+    public long getEndTime() {
+        return endTime;
+    }
+
+    public void setEndTime(long endTime) {
+        this.endTime = endTime;
+    }
+
+    public String getBuildType() {
+        return buildType;
+    }
+
+    public void setBuildType(String buildType) {
+        this.buildType = buildType;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/server/src/main/java/com/kylinolap/rest/request/JobListRequest.java
----------------------------------------------------------------------
diff --git a/server/src/main/java/com/kylinolap/rest/request/JobListRequest.java b/server/src/main/java/com/kylinolap/rest/request/JobListRequest.java
new file mode 100644
index 0000000..bf82e97
--- /dev/null
+++ b/server/src/main/java/com/kylinolap/rest/request/JobListRequest.java
@@ -0,0 +1,76 @@
+/*
+ * 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.rest.request;
+
+import java.util.List;
+
+/**
+ * @author xduo
+ * 
+ */
+public class JobListRequest {
+
+    private List<Integer> status;
+    private String cubeName;
+    private String projectName;
+    private Integer offset;
+    private Integer limit;
+
+    public JobListRequest() {
+    }
+
+    public List<Integer> getStatus() {
+        return status;
+    }
+
+    public void setStatus(List<Integer> status) {
+        this.status = status;
+    }
+
+    public String getCubeName() {
+        return cubeName;
+    }
+
+    public void setCubeName(String cubeName) {
+        this.cubeName = cubeName;
+    }
+
+    public String getProjectName() {
+        return projectName;
+    }
+
+    public void setProjectName(String projectName) {
+        this.projectName = projectName;
+    }
+
+    public Integer getOffset() {
+        return offset;
+    }
+
+    public void setOffset(Integer offset) {
+        this.offset = offset;
+    }
+
+    public Integer getLimit() {
+        return limit;
+    }
+
+    public void setLimit(Integer limit) {
+        this.limit = limit;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/server/src/main/java/com/kylinolap/rest/request/MetaRequest.java
----------------------------------------------------------------------
diff --git a/server/src/main/java/com/kylinolap/rest/request/MetaRequest.java b/server/src/main/java/com/kylinolap/rest/request/MetaRequest.java
new file mode 100644
index 0000000..511bce5
--- /dev/null
+++ b/server/src/main/java/com/kylinolap/rest/request/MetaRequest.java
@@ -0,0 +1,24 @@
+package com.kylinolap.rest.request;
+
+/**
+ * Created by honma on 7/10/14.
+ */
+public class MetaRequest {
+
+    private String project;
+
+    public MetaRequest() {
+    }
+
+    public MetaRequest(String project) {
+        this.project = project;
+    }
+
+    public String getProject() {
+        return project;
+    }
+
+    public void setProject(String project) {
+        this.project = project;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/server/src/main/java/com/kylinolap/rest/request/MetricsRequest.java
----------------------------------------------------------------------
diff --git a/server/src/main/java/com/kylinolap/rest/request/MetricsRequest.java b/server/src/main/java/com/kylinolap/rest/request/MetricsRequest.java
new file mode 100644
index 0000000..e2f3d7c
--- /dev/null
+++ b/server/src/main/java/com/kylinolap/rest/request/MetricsRequest.java
@@ -0,0 +1,57 @@
+/*
+ * 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.rest.request;
+
+import java.util.Date;
+
+/**
+ * @author xduo
+ * 
+ */
+public class MetricsRequest {
+
+    private Date startTime;
+
+    private Date endTime;
+
+    private String account;
+
+    public Date getStartTime() {
+        return startTime;
+    }
+
+    public void setStartTime(Date startTime) {
+        this.startTime = startTime;
+    }
+
+    public Date getEndTime() {
+        return endTime;
+    }
+
+    public void setEndTime(Date endTime) {
+        this.endTime = endTime;
+    }
+
+    public String getAccount() {
+        return account;
+    }
+
+    public void setAccount(String account) {
+        this.account = account;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/server/src/main/java/com/kylinolap/rest/request/PrepareSqlRequest.java
----------------------------------------------------------------------
diff --git a/server/src/main/java/com/kylinolap/rest/request/PrepareSqlRequest.java b/server/src/main/java/com/kylinolap/rest/request/PrepareSqlRequest.java
new file mode 100644
index 0000000..ae80d52
--- /dev/null
+++ b/server/src/main/java/com/kylinolap/rest/request/PrepareSqlRequest.java
@@ -0,0 +1,62 @@
+/*
+ * 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.rest.request;
+
+/**
+ * @author xduo
+ * 
+ */
+public class PrepareSqlRequest extends SQLRequest {
+
+    public PrepareSqlRequest() {
+        super();
+    }
+
+    private StateParam[] params;
+
+    public StateParam[] getParams() {
+        return params;
+    }
+
+    public void setParams(StateParam[] params) {
+        this.params = params;
+    }
+
+    public static class StateParam {
+        private String className;
+        private String value;
+
+        public StateParam() {
+        }
+
+        public String getClassName() {
+            return className;
+        }
+
+        public void setClassName(String className) {
+            this.className = className;
+        }
+
+        public String getValue() {
+            return value;
+        }
+
+        public void setValue(String value) {
+            this.value = value;
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/server/src/main/java/com/kylinolap/rest/request/SQLRequest.java
----------------------------------------------------------------------
diff --git a/server/src/main/java/com/kylinolap/rest/request/SQLRequest.java b/server/src/main/java/com/kylinolap/rest/request/SQLRequest.java
new file mode 100644
index 0000000..f2b20eb
--- /dev/null
+++ b/server/src/main/java/com/kylinolap/rest/request/SQLRequest.java
@@ -0,0 +1,115 @@
+/*
+ * 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.rest.request;
+
+public class SQLRequest {
+
+    private String sql;
+    private String project;
+    private Integer offset = 0;
+    private Integer limit = 0;
+    private boolean acceptPartial = true;
+
+    public SQLRequest() {
+    }
+
+    public String getProject() {
+        return project;
+    }
+
+    public void setProject(String project) {
+        this.project = project;
+    }
+
+    public String getSql() {
+        return sql;
+    }
+
+    public void setSql(String sql) {
+        this.sql = sql;
+    }
+
+    public Integer getOffset() {
+        return offset;
+    }
+
+    public void setOffset(Integer offset) {
+        this.offset = offset;
+    }
+
+    public Integer getLimit() {
+        return limit;
+    }
+
+    public void setLimit(Integer limit) {
+        this.limit = limit;
+    }
+
+    public boolean isAcceptPartial() {
+        return acceptPartial;
+    }
+
+    public void setAcceptPartial(boolean acceptPartial) {
+        this.acceptPartial = acceptPartial;
+    }
+
+    @Override
+    public int hashCode() {
+        final int prime = 31;
+        int result = 1;
+        result = prime * result + (acceptPartial ? 1231 : 1237);
+        result = prime * result + ((offset == null) ? 0 : offset.hashCode());
+        result = prime * result + ((limit == null) ? 0 : limit.hashCode());
+        result = prime * result + ((project == null) ? 0 : project.hashCode());
+        result = prime * result + ((sql == null) ? 0 : sql.hashCode());
+        return result;
+    }
+
+    @Override
+    public boolean equals(Object obj) {
+        if (this == obj)
+            return true;
+        if (obj == null)
+            return false;
+        if (getClass() != obj.getClass())
+            return false;
+        SQLRequest other = (SQLRequest) obj;
+        if (acceptPartial != other.acceptPartial)
+            return false;
+        if (offset == null) {
+            if (other.offset != null)
+                return false;
+        } else if (!offset.equals(other.offset))
+            return false;
+        if (limit == null) {
+            if (other.limit != null)
+                return false;
+        } else if (!limit.equals(other.limit))
+            return false;
+        if (project == null) {
+            if (other.project != null)
+                return false;
+        } else if (!project.equals(other.project))
+            return false;
+        if (sql == null) {
+            if (other.sql != null)
+                return false;
+        } else if (!sql.equals(other.sql))
+            return false;
+        return true;
+    }
+
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/server/src/main/java/com/kylinolap/rest/request/SaveSqlRequest.java
----------------------------------------------------------------------
diff --git a/server/src/main/java/com/kylinolap/rest/request/SaveSqlRequest.java b/server/src/main/java/com/kylinolap/rest/request/SaveSqlRequest.java
new file mode 100644
index 0000000..c7fef63
--- /dev/null
+++ b/server/src/main/java/com/kylinolap/rest/request/SaveSqlRequest.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.rest.request;
+
+/**
+ * @author xduo
+ * 
+ */
+public class SaveSqlRequest {
+    private long id;
+    private String name;
+    private String sql;
+    private String project;
+    private String description;
+
+    public long getId() {
+        return id;
+    }
+
+    public void setId(long id) {
+        this.id = id;
+    }
+
+    public String getName() {
+        return name;
+    }
+
+    public void setName(String name) {
+        this.name = name;
+    }
+
+    public String getSql() {
+        return sql;
+    }
+
+    public void setSql(String sql) {
+        this.sql = sql;
+    }
+
+    public String getProject() {
+        return project;
+    }
+
+    public void setProject(String project) {
+        this.project = project;
+    }
+
+    public String getDescription() {
+        return description;
+    }
+
+    public void setDescription(String description) {
+        this.description = description;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/server/src/main/java/com/kylinolap/rest/request/UpdateConfigRequest.java
----------------------------------------------------------------------
diff --git a/server/src/main/java/com/kylinolap/rest/request/UpdateConfigRequest.java b/server/src/main/java/com/kylinolap/rest/request/UpdateConfigRequest.java
new file mode 100644
index 0000000..fc1c37d
--- /dev/null
+++ b/server/src/main/java/com/kylinolap/rest/request/UpdateConfigRequest.java
@@ -0,0 +1,44 @@
+/*
+ * 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.rest.request;
+
+/**
+ * @author xduo
+ * 
+ */
+public class UpdateConfigRequest {
+    private String key;
+
+    private String value;
+
+    public String getKey() {
+        return key;
+    }
+
+    public void setKey(String key) {
+        this.key = key;
+    }
+
+    public String getValue() {
+        return value;
+    }
+
+    public void setValue(String value) {
+        this.value = value;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/server/src/main/java/com/kylinolap/rest/request/UpdateProjectRequest.java
----------------------------------------------------------------------
diff --git a/server/src/main/java/com/kylinolap/rest/request/UpdateProjectRequest.java b/server/src/main/java/com/kylinolap/rest/request/UpdateProjectRequest.java
new file mode 100644
index 0000000..e9d551f
--- /dev/null
+++ b/server/src/main/java/com/kylinolap/rest/request/UpdateProjectRequest.java
@@ -0,0 +1,38 @@
+package com.kylinolap.rest.request;
+
+/**
+ * Created by honma on 8/7/14.
+ */
+public class UpdateProjectRequest {
+    private String formerProjectName;
+    private String newProjectName;
+    private String newDescription;
+
+    public UpdateProjectRequest() {
+    }
+
+    public String getFormerProjectName() {
+        return formerProjectName;
+    }
+
+    public void setFormerProjectName(String formerProjectName) {
+
+        this.formerProjectName = formerProjectName;
+    }
+
+    public String getNewDescription() {
+        return newDescription;
+    }
+
+    public void setNewDescription(String newDescription) {
+        this.newDescription = newDescription;
+    }
+
+    public String getNewProjectName() {
+        return newProjectName;
+    }
+
+    public void setNewProjectName(String newProjectName) {
+        this.newProjectName = newProjectName;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/server/src/main/java/com/kylinolap/rest/response/AccessEntryResponse.java
----------------------------------------------------------------------
diff --git a/server/src/main/java/com/kylinolap/rest/response/AccessEntryResponse.java b/server/src/main/java/com/kylinolap/rest/response/AccessEntryResponse.java
new file mode 100644
index 0000000..8c1790f
--- /dev/null
+++ b/server/src/main/java/com/kylinolap/rest/response/AccessEntryResponse.java
@@ -0,0 +1,68 @@
+/*
+ * 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.rest.response;
+
+import java.io.Serializable;
+
+import org.springframework.security.acls.model.Permission;
+import org.springframework.security.acls.model.Sid;
+import org.springframework.util.Assert;
+
+/**
+ * @author xduo
+ * 
+ */
+public class AccessEntryResponse {
+
+    private Permission permission;
+    private Serializable id;
+    private Sid sid;
+    private boolean granting;
+
+    public AccessEntryResponse() {
+    }
+
+    public AccessEntryResponse(Serializable id, Sid sid, Permission permission, boolean granting) {
+        Assert.notNull(sid, "Sid required");
+        Assert.notNull(permission, "Permission required");
+        this.id = id;
+        this.sid = sid;
+        this.permission = permission;
+        this.granting = granting;
+    }
+
+    public Permission getPermission() {
+        return permission;
+    }
+
+    public void setPermission(Permission permission) {
+        this.permission = permission;
+    }
+
+    public Serializable getId() {
+        return id;
+    }
+
+    public Sid getSid() {
+        return sid;
+    }
+
+    public boolean isGranting() {
+        return granting;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/server/src/main/java/com/kylinolap/rest/response/ErrorResponse.java
----------------------------------------------------------------------
diff --git a/server/src/main/java/com/kylinolap/rest/response/ErrorResponse.java b/server/src/main/java/com/kylinolap/rest/response/ErrorResponse.java
new file mode 100644
index 0000000..fce23c4
--- /dev/null
+++ b/server/src/main/java/com/kylinolap/rest/response/ErrorResponse.java
@@ -0,0 +1,36 @@
+/*
+ * 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.rest.response;
+
+/**
+ * @author xduo
+ * 
+ */
+public class ErrorResponse {
+
+    public String url;
+    public String exception;
+
+    /**
+     * @param exception
+     */
+    public ErrorResponse(String url, Exception exception) {
+        this.url = url;
+        this.exception = exception.getLocalizedMessage();
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/server/src/main/java/com/kylinolap/rest/response/GeneralResponse.java
----------------------------------------------------------------------
diff --git a/server/src/main/java/com/kylinolap/rest/response/GeneralResponse.java b/server/src/main/java/com/kylinolap/rest/response/GeneralResponse.java
new file mode 100644
index 0000000..d688db5
--- /dev/null
+++ b/server/src/main/java/com/kylinolap/rest/response/GeneralResponse.java
@@ -0,0 +1,29 @@
+/*
+ * 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.rest.response;
+
+import java.util.Properties;
+
+/**
+ * @author xduo
+ * 
+ */
+public class GeneralResponse extends Properties {
+
+    private static final long serialVersionUID = -6540990247122312493L;
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/server/src/main/java/com/kylinolap/rest/response/HBaseResponse.java
----------------------------------------------------------------------
diff --git a/server/src/main/java/com/kylinolap/rest/response/HBaseResponse.java b/server/src/main/java/com/kylinolap/rest/response/HBaseResponse.java
new file mode 100644
index 0000000..3636bbf
--- /dev/null
+++ b/server/src/main/java/com/kylinolap/rest/response/HBaseResponse.java
@@ -0,0 +1,73 @@
+/*
+ * 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.rest.response;
+
+import java.io.Serializable;
+
+public class HBaseResponse implements Serializable {
+    private static final long serialVersionUID = 7263557115683263492L;
+    private String tableName;
+    private long tableSize;
+    private int regionCount;
+
+    public HBaseResponse() {
+    }
+
+    /**
+     * @return The hbase table name.
+     */
+    public String getTableName() {
+        return tableName;
+    }
+
+    /**
+     * @param tableName
+     *            The hbase table name.
+     */
+    public void setTableName(String tableName) {
+        this.tableName = tableName;
+    }
+
+    /**
+     * @return the tableSize
+     */
+    public long getTableSize() {
+        return tableSize;
+    }
+
+    /**
+     * @param tableSize
+     *            the tableSize to set
+     */
+    public void setTableSize(long tableSize) {
+        this.tableSize = tableSize;
+    }
+
+    /**
+     * @return the regionCount
+     */
+    public int getRegionCount() {
+        return regionCount;
+    }
+
+    /**
+     * @param regionCount
+     *            the regionCount to set
+     */
+    public void setRegionCount(int regionCount) {
+        this.regionCount = regionCount;
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/server/src/main/java/com/kylinolap/rest/response/MetricsResponse.java
----------------------------------------------------------------------
diff --git a/server/src/main/java/com/kylinolap/rest/response/MetricsResponse.java b/server/src/main/java/com/kylinolap/rest/response/MetricsResponse.java
new file mode 100644
index 0000000..371f6fe
--- /dev/null
+++ b/server/src/main/java/com/kylinolap/rest/response/MetricsResponse.java
@@ -0,0 +1,53 @@
+/*
+ * 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.rest.response;
+
+import java.util.HashMap;
+
+/**
+ * @author xduo
+ * 
+ */
+public class MetricsResponse extends HashMap<String, Float> {
+
+    private static final long serialVersionUID = 1L;
+
+    public void increase(String key) {
+        increase(key, (float) 1);
+    }
+
+    public void increase(String key, Float increased) {
+        if (this.containsKey(key)) {
+            this.put(key, (this.get(key) + increased));
+        } else {
+            this.put(key, increased);
+        }
+    }
+
+    public void decrease(String key) {
+        decrease(key, (float) 1);
+    }
+
+    public void decrease(String key, Float decreased) {
+        if (this.containsKey(key)) {
+            this.put(key, (this.get(key) - decreased));
+        } else {
+            this.put(key, decreased);
+        }
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/server/src/main/java/com/kylinolap/rest/response/SQLResponse.java
----------------------------------------------------------------------
diff --git a/server/src/main/java/com/kylinolap/rest/response/SQLResponse.java b/server/src/main/java/com/kylinolap/rest/response/SQLResponse.java
new file mode 100644
index 0000000..9121bb6
--- /dev/null
+++ b/server/src/main/java/com/kylinolap/rest/response/SQLResponse.java
@@ -0,0 +1,141 @@
+/*
+ * 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.rest.response;
+
+import java.io.Serializable;
+import java.util.List;
+
+import com.kylinolap.rest.model.SelectedColumnMeta;
+
+public class SQLResponse implements Serializable {
+    private static final long serialVersionUID = 1L;
+
+    // private static final Logger logger =
+    // LoggerFactory.getLogger(SQLResponse.class);
+
+    // the data type for each column
+    private List<SelectedColumnMeta> columnMetas;
+
+    // the results rows, each row contains several columns
+    private List<List<String>> results;
+
+    private String cube;
+
+    // if not select query, only return affected row count
+    private int affectedRowCount;
+
+    // flag indicating whether an exception occurred
+    private boolean isException;
+
+    // if isException, the detailed exception message
+    private String exceptionMessage;
+
+    private long duration;
+
+    private boolean isPartial = false;
+
+    private long totalScanCount;
+
+    private boolean hitCache = false;
+
+    public SQLResponse() {
+    }
+
+    public SQLResponse(List<SelectedColumnMeta> columnMetas, List<List<String>> results, int affectedRowCount, boolean isException, String exceptionMessage) {
+        this.columnMetas = columnMetas;
+        this.results = results;
+        this.affectedRowCount = affectedRowCount;
+        this.isException = isException;
+        this.exceptionMessage = exceptionMessage;
+    }
+
+    public SQLResponse(List<SelectedColumnMeta> columnMetas, List<List<String>> results, String cube, int affectedRowCount, boolean isException, String exceptionMessage) {
+        this.columnMetas = columnMetas;
+        this.results = results;
+        this.cube = cube;
+        this.affectedRowCount = affectedRowCount;
+        this.isException = isException;
+        this.exceptionMessage = exceptionMessage;
+    }
+
+    public SQLResponse(List<SelectedColumnMeta> columnMetas, List<List<String>> results, String cube, int affectedRowCount, boolean isException, String exceptionMessage, boolean isPartial) {
+        this.columnMetas = columnMetas;
+        this.results = results;
+        this.cube = cube;
+        this.affectedRowCount = affectedRowCount;
+        this.isException = isException;
+        this.exceptionMessage = exceptionMessage;
+        this.isPartial = isPartial;
+    }
+
+    public List<SelectedColumnMeta> getColumnMetas() {
+        return columnMetas;
+    }
+
+    public List<List<String>> getResults() {
+        return results;
+    }
+
+    public void setResults(List<List<String>> results) {
+        this.results = results;
+    }
+
+    public String getCube() {
+        return cube;
+    }
+
+    public int getAffectedRowCount() {
+        return affectedRowCount;
+    }
+
+    public boolean getIsException() {
+        return isException;
+    }
+
+    public String getExceptionMessage() {
+        return exceptionMessage;
+    }
+
+    public long getDuration() {
+        return duration;
+    }
+
+    public void setDuration(long duration) {
+        this.duration = duration;
+    }
+
+    public boolean isPartial() {
+
+        return isPartial;
+    }
+
+    public long getTotalScanCount() {
+        return totalScanCount;
+    }
+
+    public void setTotalScanCount(long totalScanCount) {
+        this.totalScanCount = totalScanCount;
+    }
+
+    public boolean isHitCache() {
+        return hitCache;
+    }
+
+    public void setHitCache(boolean hitCache) {
+        this.hitCache = hitCache;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/server/src/main/java/com/kylinolap/rest/response/TableDescResponse.java
----------------------------------------------------------------------
diff --git a/server/src/main/java/com/kylinolap/rest/response/TableDescResponse.java b/server/src/main/java/com/kylinolap/rest/response/TableDescResponse.java
new file mode 100644
index 0000000..c97c30b
--- /dev/null
+++ b/server/src/main/java/com/kylinolap/rest/response/TableDescResponse.java
@@ -0,0 +1,77 @@
+/*
+ * 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.rest.response;
+
+import java.util.HashMap;
+import java.util.Map;
+
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.kylinolap.metadata.model.schema.TableDesc;
+
+/**
+ * A response class to wrap TableDesc
+ * 
+ * @author jianliu
+ * 
+ */
+public class TableDescResponse extends TableDesc {
+    @JsonProperty("exd")
+    Map<String, String> descExd = new HashMap<String, String>();
+    @JsonProperty("cardinality")
+    Map<String, Long> cardinality = new HashMap<String, Long>();
+
+    /**
+     * @return the cardinality
+     */
+    public Map<String, Long> getCardinality() {
+        return cardinality;
+    }
+
+    /**
+     * @param cardinality
+     *            the cardinality to set
+     */
+    public void setCardinality(Map<String, Long> cardinality) {
+        this.cardinality = cardinality;
+    }
+
+    /**
+     * @return the descExd
+     */
+    public Map<String, String> getDescExd() {
+        return descExd;
+    }
+
+    /**
+     * @param descExd
+     *            the descExd to set
+     */
+    public void setDescExd(Map<String, String> descExd) {
+        this.descExd = descExd;
+    }
+
+    /**
+     * @param table
+     */
+    public TableDescResponse(TableDesc table) {
+        this.setColumns(table.getColumns());
+        this.setDatabase(table.getDatabase());
+        this.setName(table.getName());
+        this.setUuid(table.getUuid());
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/server/src/main/java/com/kylinolap/rest/security/AclEntityFactory.java
----------------------------------------------------------------------
diff --git a/server/src/main/java/com/kylinolap/rest/security/AclEntityFactory.java b/server/src/main/java/com/kylinolap/rest/security/AclEntityFactory.java
new file mode 100644
index 0000000..cb51a0a
--- /dev/null
+++ b/server/src/main/java/com/kylinolap/rest/security/AclEntityFactory.java
@@ -0,0 +1,54 @@
+/*
+ * 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.rest.security;
+
+import com.kylinolap.common.persistence.RootPersistentEntity;
+import com.kylinolap.cube.CubeInstance;
+import com.kylinolap.cube.project.ProjectInstance;
+import com.kylinolap.job.JobInstance;
+
+/**
+ * @author xduo
+ * 
+ */
+public class AclEntityFactory {
+
+    public static RootPersistentEntity createAclEntity(String entityType, String uuid) {
+        if ("CubeInstance".equals(entityType)) {
+            CubeInstance cubeInstance = new CubeInstance();
+            cubeInstance.setUuid(uuid);
+
+            return cubeInstance;
+        }
+
+        if ("JobInstance".equals(entityType)) {
+            JobInstance jobInstance = new JobInstance();
+            jobInstance.setUuid(uuid);
+
+            return jobInstance;
+        }
+
+        if ("ProjectInstance".equals(entityType)) {
+            ProjectInstance projectInstance = new ProjectInstance();
+            projectInstance.setUuid(uuid);
+
+            return projectInstance;
+        }
+
+        throw new RuntimeException("Unsupported entity type!");
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/server/src/main/java/com/kylinolap/rest/security/AclPermission.java
----------------------------------------------------------------------
diff --git a/server/src/main/java/com/kylinolap/rest/security/AclPermission.java b/server/src/main/java/com/kylinolap/rest/security/AclPermission.java
new file mode 100644
index 0000000..587a6bb
--- /dev/null
+++ b/server/src/main/java/com/kylinolap/rest/security/AclPermission.java
@@ -0,0 +1,40 @@
+/*
+ * 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.rest.security;
+
+import org.springframework.security.acls.domain.BasePermission;
+import org.springframework.security.acls.model.Permission;
+
+/**
+ * @author xduo
+ * 
+ */
+public class AclPermission extends BasePermission {
+
+    private static final long serialVersionUID = 6121785617686149123L;
+
+    public static final Permission MANAGEMENT = new AclPermission(1 << 5, 'M'); // 32
+    public static final Permission OPERATION = new AclPermission(1 << 6, 'O'); // 64
+
+    protected AclPermission(int mask) {
+        super(mask);
+    }
+
+    protected AclPermission(int mask, char code) {
+        super(mask, code);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/server/src/main/java/com/kylinolap/rest/security/AclPermissionFactory.java
----------------------------------------------------------------------
diff --git a/server/src/main/java/com/kylinolap/rest/security/AclPermissionFactory.java b/server/src/main/java/com/kylinolap/rest/security/AclPermissionFactory.java
new file mode 100644
index 0000000..257845c
--- /dev/null
+++ b/server/src/main/java/com/kylinolap/rest/security/AclPermissionFactory.java
@@ -0,0 +1,75 @@
+/*
+ * 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.rest.security;
+
+import java.lang.reflect.Field;
+import java.util.ArrayList;
+import java.util.List;
+
+import org.springframework.security.acls.domain.DefaultPermissionFactory;
+import org.springframework.security.acls.model.Permission;
+
+/**
+ * @author xduo
+ * 
+ */
+public class AclPermissionFactory extends DefaultPermissionFactory {
+
+    public AclPermissionFactory() {
+        super();
+        registerPublicPermissions(AclPermission.class);
+    }
+
+    public static List<Permission> getPermissions() {
+        List<Permission> permissions = new ArrayList<Permission>();
+        Field[] fields = AclPermission.class.getFields();
+
+        for (Field field : fields) {
+            try {
+                Object fieldValue = field.get(null);
+
+                if (Permission.class.isAssignableFrom(fieldValue.getClass())) {
+                    // Found a Permission static field
+                    permissions.add((Permission) fieldValue);
+                }
+            } catch (Exception ignore) {
+            }
+        }
+
+        return permissions;
+    }
+
+    public static Permission getPermission(String perName) {
+        Field[] fields = AclPermission.class.getFields();
+
+        for (Field field : fields) {
+            try {
+                Object fieldValue = field.get(null);
+
+                if (Permission.class.isAssignableFrom(fieldValue.getClass())) {
+                    // Found a Permission static field
+                    if (perName.equals(field.getName())) {
+                        return (Permission) fieldValue;
+                    }
+                }
+            } catch (Exception ignore) {
+            }
+        }
+
+        return null;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/server/src/main/java/com/kylinolap/rest/security/AuthoritiesPopulator.java
----------------------------------------------------------------------
diff --git a/server/src/main/java/com/kylinolap/rest/security/AuthoritiesPopulator.java b/server/src/main/java/com/kylinolap/rest/security/AuthoritiesPopulator.java
new file mode 100644
index 0000000..4808e44
--- /dev/null
+++ b/server/src/main/java/com/kylinolap/rest/security/AuthoritiesPopulator.java
@@ -0,0 +1,75 @@
+/*
+ * 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.rest.security;
+
+import java.util.HashSet;
+import java.util.Set;
+
+import org.springframework.ldap.core.ContextSource;
+import org.springframework.security.core.GrantedAuthority;
+import org.springframework.security.core.authority.SimpleGrantedAuthority;
+import org.springframework.security.ldap.userdetails.DefaultLdapAuthoritiesPopulator;
+
+import com.kylinolap.rest.constant.Constant;
+
+/**
+ * @author xduo
+ * 
+ */
+public class AuthoritiesPopulator extends DefaultLdapAuthoritiesPopulator {
+
+    String adminRole;
+    SimpleGrantedAuthority adminRoleAsAuthority;
+    
+    SimpleGrantedAuthority adminAuthority = new SimpleGrantedAuthority(Constant.ROLE_ADMIN);
+    SimpleGrantedAuthority modelerAuthority = new SimpleGrantedAuthority(Constant.ROLE_MODELER);
+    SimpleGrantedAuthority analystAuthority = new SimpleGrantedAuthority(Constant.ROLE_ANALYST);
+    
+    Set<GrantedAuthority> defaultAuthorities = new HashSet<GrantedAuthority>();
+    
+    /**
+     * @param contextSource
+     * @param groupSearchBase
+     */
+    public AuthoritiesPopulator(ContextSource contextSource, String groupSearchBase, String adminRole, String defaultRole) {
+        super(contextSource, groupSearchBase);
+        this.adminRole = adminRole;
+        this.adminRoleAsAuthority = new SimpleGrantedAuthority(adminRole);
+        
+        if (defaultRole.contains(Constant.ROLE_MODELER))
+            this.defaultAuthorities.add(modelerAuthority);
+        if (defaultRole.contains(Constant.ROLE_ANALYST))
+            this.defaultAuthorities.add(analystAuthority);
+    }
+
+    @Override
+    public Set<GrantedAuthority> getGroupMembershipRoles(String userDn, String username) {
+        Set<GrantedAuthority> authorities = super.getGroupMembershipRoles(userDn, username);
+        
+        if (authorities.contains(adminRoleAsAuthority)) {
+            authorities.add(adminAuthority);
+            authorities.add(modelerAuthority);
+            authorities.add(analystAuthority);
+        }
+        
+        authorities.addAll(defaultAuthorities);
+        
+        return authorities;
+    }
+    
+    
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/server/src/main/java/com/kylinolap/rest/security/CrossDomainFilter.java
----------------------------------------------------------------------
diff --git a/server/src/main/java/com/kylinolap/rest/security/CrossDomainFilter.java b/server/src/main/java/com/kylinolap/rest/security/CrossDomainFilter.java
new file mode 100644
index 0000000..01d277b
--- /dev/null
+++ b/server/src/main/java/com/kylinolap/rest/security/CrossDomainFilter.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.rest.security;
+
+import java.io.IOException;
+
+import javax.servlet.Filter;
+import javax.servlet.FilterChain;
+import javax.servlet.FilterConfig;
+import javax.servlet.ServletException;
+import javax.servlet.ServletRequest;
+import javax.servlet.ServletResponse;
+import javax.servlet.http.HttpServletResponse;
+
+import com.kylinolap.common.KylinConfig;
+
+/**
+ * @author xduo
+ * 
+ */
+public class CrossDomainFilter implements Filter {
+
+    /*
+     * (non-Javadoc)
+     * 
+     * @see javax.servlet.Filter#init(javax.servlet.FilterConfig)
+     */
+    @Override
+    public void init(FilterConfig filterConfig) throws ServletException {
+    }
+
+    /*
+     * (non-Javadoc)
+     * 
+     * @see javax.servlet.Filter#doFilter(javax.servlet.ServletRequest,
+     * javax.servlet.ServletResponse, javax.servlet.FilterChain)
+     */
+    @Override
+    public void doFilter(ServletRequest request, ServletResponse response, FilterChain chain) throws IOException, ServletException {
+        if (Boolean.parseBoolean(KylinConfig.getInstanceFromEnv().getProperty("crossdomain.enable", "false"))) {
+            ((HttpServletResponse) response).addHeader("Access-Control-Allow-Origin", "*");
+            ((HttpServletResponse) response).addHeader("Access-Control-Allow-Methods", "GET, POST, PUT, DELETE, OPTIONS");
+            ((HttpServletResponse) response).addHeader("Access-Control-Allow-Headers", "Origin, No-Cache, X-Requested-With, If-Modified-Since, Pragma, Last-Modified, Cache-Control, Expires, Content-Type, X-E4M-With, Accept, Authorization");
+        }
+
+        chain.doFilter(request, response);
+    }
+
+    /*
+     * (non-Javadoc)
+     * 
+     * @see javax.servlet.Filter#destroy()
+     */
+    @Override
+    public void destroy() {
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/server/src/main/java/com/kylinolap/rest/security/LdapProvider.java
----------------------------------------------------------------------
diff --git a/server/src/main/java/com/kylinolap/rest/security/LdapProvider.java b/server/src/main/java/com/kylinolap/rest/security/LdapProvider.java
new file mode 100644
index 0000000..4faaa99
--- /dev/null
+++ b/server/src/main/java/com/kylinolap/rest/security/LdapProvider.java
@@ -0,0 +1,103 @@
+/*
+ * 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.rest.security;
+
+import java.security.MessageDigest;
+import java.security.NoSuchAlgorithmException;
+import java.util.Arrays;
+
+import net.sf.ehcache.Cache;
+import net.sf.ehcache.CacheManager;
+import net.sf.ehcache.Element;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.springframework.beans.factory.annotation.Autowired;
+import org.springframework.security.core.Authentication;
+import org.springframework.security.core.AuthenticationException;
+import org.springframework.security.core.context.SecurityContextHolder;
+import org.springframework.security.core.userdetails.User;
+import org.springframework.security.core.userdetails.UserDetails;
+import org.springframework.security.ldap.authentication.LdapAuthenticationProvider;
+import org.springframework.security.ldap.authentication.LdapAuthenticator;
+import org.springframework.security.ldap.userdetails.LdapAuthoritiesPopulator;
+
+import com.kylinolap.rest.service.UserService;
+
+/**
+ * @author xduo
+ * 
+ */
+public class LdapProvider extends LdapAuthenticationProvider {
+
+    private static final Logger logger = LoggerFactory.getLogger(LdapProvider.class);
+
+    @Autowired
+    UserService userService;
+
+    @Autowired
+    private CacheManager cacheManager;
+
+    MessageDigest md = null;
+
+    /**
+     * @param authenticator
+     * @param authoritiesPopulator
+     */
+    public LdapProvider(LdapAuthenticator authenticator, LdapAuthoritiesPopulator authoritiesPopulator) {
+        super(authenticator, authoritiesPopulator);
+
+        try {
+            md = MessageDigest.getInstance("MD5");
+        } catch (NoSuchAlgorithmException e) {
+            throw new RuntimeException("Failed to init Message Digest ", e);
+        }
+    }
+
+    @Override
+    public Authentication authenticate(Authentication authentication) throws AuthenticationException {
+        Authentication authed = null;
+        Cache userCache = cacheManager.getCache("UserCache");
+        md.reset();
+        byte[] hashKey = md.digest((authentication.getName() + authentication.getCredentials()).getBytes());
+        String userKey = Arrays.toString(hashKey);
+
+        Element authedUser = userCache.get(userKey);
+        if (null != authedUser) {
+            authed = (Authentication) authedUser.getObjectValue();
+            SecurityContextHolder.getContext().setAuthentication(authed);
+        } else {
+            try {
+                authed = super.authenticate(authentication);
+                userCache.put(new Element(userKey, authed));
+            } catch (AuthenticationException e) {
+                logger.error("Failed to auth user: " + authentication.getName(), e);
+                throw e;
+            }
+
+            UserDetails user = new User(authentication.getName(), "skippped-ldap", authed.getAuthorities());
+
+            if (!userService.userExists(authentication.getName())) {
+                userService.createUser(user);
+            } else {
+                userService.updateUser(user);
+            }
+        }
+
+        return authed;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/server/src/main/java/com/kylinolap/rest/security/PasswordPlaceholderConfigurer.java
----------------------------------------------------------------------
diff --git a/server/src/main/java/com/kylinolap/rest/security/PasswordPlaceholderConfigurer.java b/server/src/main/java/com/kylinolap/rest/security/PasswordPlaceholderConfigurer.java
new file mode 100644
index 0000000..beff51b
--- /dev/null
+++ b/server/src/main/java/com/kylinolap/rest/security/PasswordPlaceholderConfigurer.java
@@ -0,0 +1,82 @@
+/*
+ * 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.rest.security;
+
+import java.util.Properties;
+
+import javax.crypto.Cipher;
+import javax.crypto.spec.SecretKeySpec;
+
+import org.apache.commons.codec.binary.Base64;
+import org.springframework.beans.factory.config.PropertyPlaceholderConfigurer;
+import org.springframework.core.io.InputStreamResource;
+import org.springframework.core.io.Resource;
+import org.springframework.security.crypto.bcrypt.BCryptPasswordEncoder;
+
+import com.kylinolap.common.KylinConfig;
+
+/**
+ * @author xduo
+ * 
+ */
+public class PasswordPlaceholderConfigurer extends PropertyPlaceholderConfigurer {
+
+    private static byte[] key = { 0x74, 0x68, 0x69, 0x73, 0x49, 0x73, 0x41, 0x53, 0x65, 0x63, 0x72, 0x65, 0x74, 0x4b, 0x65, 0x79 };
+
+    public PasswordPlaceholderConfigurer() {
+        Resource[] resources = new Resource[1];
+        resources[0] = new InputStreamResource(KylinConfig.getKylinPropertiesAsInputSteam());
+        this.setLocations(resources);
+    }
+    
+    public static String encrypt(String strToEncrypt) {
+        try {
+            Cipher cipher = Cipher.getInstance("AES/ECB/PKCS5Padding");
+            final SecretKeySpec secretKey = new SecretKeySpec(key, "AES");
+            cipher.init(Cipher.ENCRYPT_MODE, secretKey);
+            final String encryptedString = Base64.encodeBase64String(cipher.doFinal(strToEncrypt.getBytes()));
+            return encryptedString;
+        } catch (Exception e) {
+            throw new RuntimeException(e.getMessage(), e);
+        }
+    }
+
+    public static String decrypt(String strToDecrypt) {
+        try {
+            Cipher cipher = Cipher.getInstance("AES/ECB/PKCS5PADDING");
+            final SecretKeySpec secretKey = new SecretKeySpec(key, "AES");
+            cipher.init(Cipher.DECRYPT_MODE, secretKey);
+            final String decryptedString = new String(cipher.doFinal(Base64.decodeBase64(strToDecrypt)));
+            return decryptedString;
+        } catch (Exception e) {
+            throw new RuntimeException(e.getMessage(), e);
+        }
+    }
+
+    protected String resolvePlaceholder(String placeholder, Properties props) {
+        if (placeholder.toLowerCase().contains("password")) {
+            return decrypt(props.getProperty(placeholder));
+        } else {
+            return props.getProperty(placeholder);
+        }
+    }
+
+    public static void main(String[] args) {
+        BCryptPasswordEncoder bCryptPasswordEncoder = new BCryptPasswordEncoder();
+        System.out.println(bCryptPasswordEncoder.encode("MODELER"));
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/server/src/main/java/com/kylinolap/rest/security/UnauthorisedEntryPoint.java
----------------------------------------------------------------------
diff --git a/server/src/main/java/com/kylinolap/rest/security/UnauthorisedEntryPoint.java b/server/src/main/java/com/kylinolap/rest/security/UnauthorisedEntryPoint.java
new file mode 100644
index 0000000..4c645fa
--- /dev/null
+++ b/server/src/main/java/com/kylinolap/rest/security/UnauthorisedEntryPoint.java
@@ -0,0 +1,26 @@
+package com.kylinolap.rest.security;
+
+import java.io.IOException;
+
+import javax.servlet.ServletException;
+import javax.servlet.http.HttpServletRequest;
+import javax.servlet.http.HttpServletResponse;
+
+import org.springframework.security.core.AuthenticationException;
+import org.springframework.security.web.AuthenticationEntryPoint;
+import org.springframework.stereotype.Component;
+
+/**
+ * Just return 401-unauthorized for every unauthorized request. The client side
+ * catches this and handles login.
+ * 
+ * @author xduo
+ */
+@Component(value = "unauthorisedEntryPoint")
+public class UnauthorisedEntryPoint implements AuthenticationEntryPoint {
+
+    public void commence(HttpServletRequest request, HttpServletResponse response, AuthenticationException exception) throws IOException, ServletException {
+        response.sendError(HttpServletResponse.SC_UNAUTHORIZED, "Unauthorized");
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/server/src/main/java/com/kylinolap/rest/security/UserManager.java
----------------------------------------------------------------------
diff --git a/server/src/main/java/com/kylinolap/rest/security/UserManager.java b/server/src/main/java/com/kylinolap/rest/security/UserManager.java
new file mode 100644
index 0000000..df33c43
--- /dev/null
+++ b/server/src/main/java/com/kylinolap/rest/security/UserManager.java
@@ -0,0 +1,11 @@
+package com.kylinolap.rest.security;
+
+import java.util.List;
+
+import org.springframework.security.provisioning.UserDetailsManager;
+
+public interface UserManager extends UserDetailsManager{
+
+    public List<String> getUserAuthorities();
+    
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/server/src/main/java/com/kylinolap/rest/service/AccessService.java
----------------------------------------------------------------------
diff --git a/server/src/main/java/com/kylinolap/rest/service/AccessService.java b/server/src/main/java/com/kylinolap/rest/service/AccessService.java
new file mode 100644
index 0000000..3bdb595
--- /dev/null
+++ b/server/src/main/java/com/kylinolap/rest/service/AccessService.java
@@ -0,0 +1,289 @@
+/*
+ * 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.rest.service;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+
+import org.springframework.beans.factory.annotation.Autowired;
+import org.springframework.security.access.prepost.PreAuthorize;
+import org.springframework.security.acls.domain.BasePermission;
+import org.springframework.security.acls.domain.GrantedAuthoritySid;
+import org.springframework.security.acls.domain.ObjectIdentityImpl;
+import org.springframework.security.acls.domain.PrincipalSid;
+import org.springframework.security.acls.model.AccessControlEntry;
+import org.springframework.security.acls.model.Acl;
+import org.springframework.security.acls.model.AlreadyExistsException;
+import org.springframework.security.acls.model.MutableAcl;
+import org.springframework.security.acls.model.NotFoundException;
+import org.springframework.security.acls.model.ObjectIdentity;
+import org.springframework.security.acls.model.Permission;
+import org.springframework.security.acls.model.Sid;
+import org.springframework.security.core.Authentication;
+import org.springframework.security.core.context.SecurityContextHolder;
+import org.springframework.stereotype.Component;
+import org.springframework.transaction.annotation.Transactional;
+import org.springframework.util.Assert;
+
+import com.kylinolap.common.persistence.AclEntity;
+import com.kylinolap.common.persistence.RootPersistentEntity;
+import com.kylinolap.rest.constant.Constant;
+import com.kylinolap.rest.exception.ForbiddenException;
+import com.kylinolap.rest.response.AccessEntryResponse;
+import com.kylinolap.rest.security.AclEntityFactory;
+
+/**
+ * @author xduo
+ * 
+ */
+@Component("accessService")
+public class AccessService {
+
+    @Autowired
+    private AclService aclService;
+
+    @Autowired
+    UserService userService;
+
+    // ~ Methods to manage acl life circle of domain objects ~
+
+    @Transactional
+    public Acl init(AclEntity ae, Permission initPermission) {
+        Acl acl = null;
+        ObjectIdentity objectIdentity = new ObjectIdentityImpl(ae.getClass(), ae.getId());
+
+        try {
+            // Create acl record for secured domain object.
+            acl = aclService.createAcl(objectIdentity);
+        } catch (AlreadyExistsException e) {
+            acl = (MutableAcl) aclService.readAclById(objectIdentity);
+        }
+
+        if (null != initPermission) {
+            Authentication auth = SecurityContextHolder.getContext().getAuthentication();
+            PrincipalSid sid = new PrincipalSid(auth);
+            acl = grant(ae, initPermission, sid);
+        }
+
+        return acl;
+    }
+
+    @Transactional
+    @PreAuthorize(Constant.ACCESS_HAS_ROLE_ADMIN + " or hasPermission(#ae, 'ADMINISTRATION')")
+    public Acl grant(AclEntity ae, Permission permission, Sid sid) {
+        Assert.notNull(ae, "Acl domain object required");
+        Assert.notNull(permission, "Acl permission required");
+        Assert.notNull(sid, "Sid required");
+
+        ObjectIdentity objectIdentity = new ObjectIdentityImpl(ae.getClass(), ae.getId());
+        MutableAcl acl = null;
+
+        try {
+            acl = (MutableAcl) aclService.readAclById(objectIdentity);
+        } catch (NotFoundException e) {
+            acl = (MutableAcl) init(ae, null);
+        }
+
+        int indexOfAce = -1;
+        for (int i = 0; i < acl.getEntries().size(); i++) {
+            AccessControlEntry ace = acl.getEntries().get(i);
+
+            if (ace.getSid().equals(sid)) {
+                indexOfAce = i;
+            }
+        }
+
+        if (indexOfAce != -1) {
+            secureOwner(acl, indexOfAce);
+            acl.updateAce(indexOfAce, permission);
+        } else {
+            acl.insertAce(acl.getEntries().size(), permission, sid, true);
+        }
+
+        acl = aclService.updateAcl(acl);
+
+        return acl;
+    }
+
+    @Transactional
+    @PreAuthorize(Constant.ACCESS_HAS_ROLE_ADMIN + " or hasPermission(#ae, 'ADMINISTRATION')")
+    public Acl update(AclEntity ae, Long accessEntryId, Permission newPermission) {
+        Assert.notNull(ae, "Acl domain object required");
+        Assert.notNull(accessEntryId, "Ace id required");
+        Assert.notNull(newPermission, "Acl permission required");
+
+        ObjectIdentity objectIdentity = new ObjectIdentityImpl(ae.getClass(), ae.getId());
+        MutableAcl acl = (MutableAcl) aclService.readAclById(objectIdentity);
+
+        int indexOfAce = -1;
+        for (int i = 0; i < acl.getEntries().size(); i++) {
+            AccessControlEntry ace = acl.getEntries().get(i);
+            if (ace.getId().equals(accessEntryId)) {
+                indexOfAce = i;
+                break;
+            }
+        }
+
+        if (indexOfAce != -1) {
+            secureOwner(acl, indexOfAce);
+
+            try {
+                acl.updateAce(indexOfAce, newPermission);
+                acl = aclService.updateAcl(acl);
+            } catch (NotFoundException e) {
+            }
+        }
+
+        return acl;
+    }
+
+    @Transactional
+    @PreAuthorize(Constant.ACCESS_HAS_ROLE_ADMIN + " or hasPermission(#ae, 'ADMINISTRATION')")
+    public Acl revoke(AclEntity ae, Long accessEntryId) {
+        Assert.notNull(ae, "Acl domain object required");
+        Assert.notNull(accessEntryId, "Ace id required");
+
+        ObjectIdentity objectIdentity = new ObjectIdentityImpl(ae.getClass(), ae.getId());
+        MutableAcl acl = (MutableAcl) aclService.readAclById(objectIdentity);
+        int indexOfAce = -1;
+
+        for (int i = 0; i < acl.getEntries().size(); i++) {
+            AccessControlEntry ace = acl.getEntries().get(i);
+            if (((Long) ace.getId()).equals(accessEntryId)) {
+                indexOfAce = i;
+                break;
+            }
+        }
+
+        if (indexOfAce != -1) {
+            secureOwner(acl, indexOfAce);
+
+            try {
+                acl.deleteAce(indexOfAce);
+                acl = aclService.updateAcl(acl);
+            } catch (NotFoundException e) {
+            }
+        }
+
+        return acl;
+    }
+
+    @Transactional
+    public void inherit(AclEntity ae, AclEntity parentAe) {
+        Assert.notNull(ae, "Acl domain object required");
+        Assert.notNull(parentAe, "Parent acl required");
+
+        ObjectIdentity objectIdentity = new ObjectIdentityImpl(ae.getClass(), ae.getId());
+        MutableAcl acl = null;
+        try {
+            acl = (MutableAcl) aclService.readAclById(objectIdentity);
+        } catch (NotFoundException e) {
+            acl = (MutableAcl) init(ae, null);
+        }
+
+        ObjectIdentity parentObjectIdentity = new ObjectIdentityImpl(parentAe.getClass(), parentAe.getId());
+        MutableAcl parentAcl = null;
+        try {
+            parentAcl = (MutableAcl) aclService.readAclById(parentObjectIdentity);
+        } catch (NotFoundException e) {
+            parentAcl = (MutableAcl) init(parentAe, null);
+        }
+
+        if (null == acl || null == parentAcl) {
+            return;
+        }
+
+        acl.setEntriesInheriting(true);
+        acl.setParent(parentAcl);
+        aclService.updateAcl(acl);
+    }
+
+    @Transactional
+    @PreAuthorize(Constant.ACCESS_HAS_ROLE_ADMIN + " or hasPermission(#ae, 'ADMINISTRATION')")
+    public void clean(AclEntity ae, boolean deleteChildren) {
+        Assert.notNull(ae, "Acl domain object required");
+
+        ObjectIdentity objectIdentity = new ObjectIdentityImpl(ae.getClass(), ae.getId());
+
+        try {
+            aclService.deleteAcl(objectIdentity, deleteChildren);
+        } catch (NotFoundException e) {
+        }
+    }
+
+    // ~ Methods to get acl info of domain objects ~
+
+    public RootPersistentEntity getAclEntity(String entityType, String uuid) {
+        if (null == uuid) {
+            return null;
+        }
+
+        return AclEntityFactory.createAclEntity(entityType, uuid);
+    }
+
+    public Acl getAcl(AclEntity ae) {
+        if (null == ae) {
+            return null;
+        }
+        ObjectIdentity objectIdentity = new ObjectIdentityImpl(ae.getClass(), ae.getId());
+        Acl acl = null;
+
+        try {
+            acl = (MutableAcl) aclService.readAclById(objectIdentity);
+        } catch (NotFoundException e) {
+        }
+
+        return acl;
+    }
+
+    public Sid getSid(String sid, boolean isPrincepal) {
+        if (isPrincepal) {
+            return new PrincipalSid(sid);
+        } else {
+            return new GrantedAuthoritySid(sid);
+        }
+    }
+
+    public List<AccessEntryResponse> generateAceResponses(Acl acl) {
+        if (null == acl) {
+            return Collections.emptyList();
+        }
+        List<AccessEntryResponse> accessControlEntities = new ArrayList<AccessEntryResponse>();
+
+        // Cause there is a circle reference in AccessControlEntry, it needs to
+        // set acl to null as a workaround.
+        for (AccessControlEntry ace : acl.getEntries()) {
+            accessControlEntities.add(new AccessEntryResponse(ace.getId(), ace.getSid(), ace.getPermission(), ace.isGranting()));
+        }
+
+        return accessControlEntities;
+    }
+
+    /**
+     * Protect admin permission granted to acl owner.
+     * 
+     * @param acl
+     * @param indexOfAce
+     */
+    private void secureOwner(MutableAcl acl, int indexOfAce) {
+        // Can't revoke admin permission from domain object owner
+        if (acl.getOwner().equals(acl.getEntries().get(indexOfAce).getSid()) && BasePermission.ADMINISTRATION.equals(acl.getEntries().get(indexOfAce).getPermission())) {
+            throw new ForbiddenException("Can't revoke admin permission of owner.");
+        }
+    }
+}


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

Posted by lu...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/job/src/main/java/com/kylinolap/job/hadoop/cube/MergeCuboidMapper.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/com/kylinolap/job/hadoop/cube/MergeCuboidMapper.java b/job/src/main/java/com/kylinolap/job/hadoop/cube/MergeCuboidMapper.java
new file mode 100644
index 0000000..627c397
--- /dev/null
+++ b/job/src/main/java/com/kylinolap/job/hadoop/cube/MergeCuboidMapper.java
@@ -0,0 +1,174 @@
+/*
+ * 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.job.hadoop.cube;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.mapreduce.Mapper;
+import org.apache.hadoop.mapreduce.lib.input.FileSplit;
+
+import com.kylinolap.common.KylinConfig;
+import com.kylinolap.common.util.BytesUtil;
+import com.kylinolap.cube.CubeInstance;
+import com.kylinolap.cube.CubeManager;
+import com.kylinolap.cube.CubeSegment;
+import com.kylinolap.cube.CubeSegmentStatusEnum;
+import com.kylinolap.cube.common.RowKeySplitter;
+import com.kylinolap.cube.common.SplittedBytes;
+import com.kylinolap.cube.cuboid.Cuboid;
+import com.kylinolap.cube.kv.RowConstants;
+import com.kylinolap.dict.Dictionary;
+import com.kylinolap.dict.DictionaryManager;
+import com.kylinolap.job.constant.BatchConstants;
+import com.kylinolap.job.hadoop.AbstractHadoopJob;
+import com.kylinolap.metadata.model.cube.CubeDesc;
+import com.kylinolap.metadata.model.cube.TblColRef;
+
+/**
+ * @author ysong1, honma
+ */
+public class MergeCuboidMapper extends Mapper<Text, Text, Text, Text> {
+
+    private KylinConfig config;
+    private String cubeName;
+    private String segmentName;
+    private CubeManager cubeManager;
+    private CubeInstance cube;
+    private CubeDesc cubeDesc;
+    private CubeSegment mergedCubeSegment;
+    private CubeSegment sourceCubeSegment;// Must be unique during a mapper's
+                                          // life cycle
+
+    private Text outputKey = new Text();
+
+    private byte[] newKeyBuf;
+    private RowKeySplitter rowKeySplitter;
+
+    private HashMap<TblColRef, Boolean> dictsNeedMerging = new HashMap<TblColRef, Boolean>();
+
+    private static final Pattern JOB_NAME_PATTERN = Pattern.compile("kylin-([0-9a-f]{8}-[0-9a-f]{4}-[0-9a-f]{4}-[0-9a-f]{4}-[0-9a-f]{12})");
+
+    private Boolean checkNeedMerging(TblColRef col) throws IOException {
+        Boolean ret = dictsNeedMerging.get(col);
+        if (ret != null)
+            return ret;
+        else {
+            ret = cubeDesc.getRowkey().isUseDictionary(col) && cubeDesc.getFactTable().equalsIgnoreCase((String) DictionaryManager.getInstance(config).decideSourceData(cubeDesc, col, null)[0]);
+            dictsNeedMerging.put(col, ret);
+            return ret;
+        }
+    }
+
+    private String extractJobIDFromPath(String path) {
+        Matcher matcher = JOB_NAME_PATTERN.matcher(path);
+        // check the first occurance
+        if (matcher.find()) {
+            return matcher.group(1);
+        } else {
+            throw new IllegalStateException("Can not extract job ID from file path : " + path);
+        }
+    }
+
+    private CubeSegment findSegmentWithUuid(String jobID, CubeInstance cubeInstance) {
+        for (CubeSegment segment : cubeInstance.getSegments()) {
+            if (segment.getUuid().equalsIgnoreCase(jobID)) {
+                return segment;
+            }
+        }
+
+        throw new IllegalStateException("No merging segment's last build job ID equals " + jobID);
+
+    }
+
+    @Override
+    protected void setup(Context context) throws IOException, InterruptedException {
+        cubeName = context.getConfiguration().get(BatchConstants.CFG_CUBE_NAME).toUpperCase();
+        segmentName = context.getConfiguration().get(BatchConstants.CFG_CUBE_SEGMENT_NAME).toUpperCase();
+
+        config = AbstractHadoopJob.loadKylinPropsAndMetadata(context.getConfiguration());
+
+        cubeManager = CubeManager.getInstance(config);
+        cube = cubeManager.getCube(cubeName);
+        cubeDesc = cube.getDescriptor();
+        mergedCubeSegment = cube.getSegment(segmentName, CubeSegmentStatusEnum.NEW);
+
+        // int colCount = cubeDesc.getRowkey().getRowKeyColumns().length;
+        newKeyBuf = new byte[256];// size will auto-grow
+
+        // decide which source segment
+        org.apache.hadoop.mapreduce.InputSplit inputSplit = context.getInputSplit();
+        String filePath = ((FileSplit) inputSplit).getPath().toString();
+        String jobID = extractJobIDFromPath(filePath);
+        sourceCubeSegment = findSegmentWithUuid(jobID, cube);
+
+        this.rowKeySplitter = new RowKeySplitter(sourceCubeSegment, 65, 255);
+    }
+
+    @Override
+    public void map(Text key, Text value, Context context) throws IOException, InterruptedException {
+        long cuboidID = rowKeySplitter.split(key.getBytes(), key.getBytes().length);
+        Cuboid cuboid = Cuboid.findById(cubeDesc, cuboidID);
+
+        SplittedBytes[] splittedByteses = rowKeySplitter.getSplitBuffers();
+        int bufOffset = 0;
+        BytesUtil.writeLong(cuboidID, newKeyBuf, bufOffset, RowConstants.ROWKEY_CUBOIDID_LEN);
+        bufOffset += RowConstants.ROWKEY_CUBOIDID_LEN;
+
+        for (int i = 0; i < cuboid.getColumns().size(); ++i) {
+            TblColRef col = cuboid.getColumns().get(i);
+
+            if (this.checkNeedMerging(col)) {
+                // if dictionary on fact table column, needs rewrite
+                DictionaryManager dictMgr = DictionaryManager.getInstance(config);
+                Dictionary<?> sourceDict = dictMgr.getDictionary(sourceCubeSegment.getDictResPath(col));
+                Dictionary<?> mergedDict = dictMgr.getDictionary(mergedCubeSegment.getDictResPath(col));
+
+                while (sourceDict.getSizeOfValue() > newKeyBuf.length - bufOffset || mergedDict.getSizeOfValue() > newKeyBuf.length - bufOffset) {
+                    byte[] oldBuf = newKeyBuf;
+                    newKeyBuf = new byte[2 * newKeyBuf.length];
+                    System.arraycopy(oldBuf, 0, newKeyBuf, 0, oldBuf.length);
+                }
+
+                int idInSourceDict = BytesUtil.readUnsigned(splittedByteses[i + 1].value, 0, splittedByteses[i + 1].length);
+                int size = sourceDict.getValueBytesFromId(idInSourceDict, newKeyBuf, bufOffset);
+                int idInMergedDict = mergedDict.getIdFromValueBytes(newKeyBuf, bufOffset, size);
+                BytesUtil.writeUnsigned(idInMergedDict, newKeyBuf, bufOffset, mergedDict.getSizeOfId());
+
+                bufOffset += mergedDict.getSizeOfId();
+            } else {
+                // keep as it is
+                while (splittedByteses[i + 1].length > newKeyBuf.length - bufOffset) {
+                    byte[] oldBuf = newKeyBuf;
+                    newKeyBuf = new byte[2 * newKeyBuf.length];
+                    System.arraycopy(oldBuf, 0, newKeyBuf, 0, oldBuf.length);
+                }
+
+                System.arraycopy(splittedByteses[i + 1].value, 0, newKeyBuf, bufOffset, splittedByteses[i + 1].length);
+                bufOffset += splittedByteses[i + 1].length;
+            }
+        }
+        byte[] newKey = Arrays.copyOf(newKeyBuf, bufOffset);
+        outputKey.set(newKey, 0, newKey.length);
+
+        context.write(outputKey, value);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/job/src/main/java/com/kylinolap/job/hadoop/cube/NDCuboidJob.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/com/kylinolap/job/hadoop/cube/NDCuboidJob.java b/job/src/main/java/com/kylinolap/job/hadoop/cube/NDCuboidJob.java
new file mode 100644
index 0000000..e06ef46
--- /dev/null
+++ b/job/src/main/java/com/kylinolap/job/hadoop/cube/NDCuboidJob.java
@@ -0,0 +1,36 @@
+/*
+ * 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.job.hadoop.cube;
+
+import org.apache.hadoop.util.ToolRunner;
+
+/**
+ * @author George Song (ysong1)
+ * 
+ */
+
+public class NDCuboidJob extends CuboidJob {
+
+    public NDCuboidJob() {
+        this.setMapperClass(NDCuboidMapper.class);
+    }
+
+    public static void main(String[] args) throws Exception {
+        CuboidJob job = new NDCuboidJob();
+        int exitCode = ToolRunner.run(job, args);
+        System.exit(exitCode);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/job/src/main/java/com/kylinolap/job/hadoop/cube/NDCuboidMapper.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/com/kylinolap/job/hadoop/cube/NDCuboidMapper.java b/job/src/main/java/com/kylinolap/job/hadoop/cube/NDCuboidMapper.java
new file mode 100644
index 0000000..b1f08b0
--- /dev/null
+++ b/job/src/main/java/com/kylinolap/job/hadoop/cube/NDCuboidMapper.java
@@ -0,0 +1,138 @@
+/*
+ * 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.job.hadoop.cube;
+
+import java.io.IOException;
+import java.util.Collection;
+
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.mapreduce.Mapper;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.kylinolap.common.KylinConfig;
+import com.kylinolap.cube.CubeInstance;
+import com.kylinolap.cube.CubeManager;
+import com.kylinolap.cube.CubeSegment;
+import com.kylinolap.cube.CubeSegmentStatusEnum;
+import com.kylinolap.cube.common.RowKeySplitter;
+import com.kylinolap.cube.common.SplittedBytes;
+import com.kylinolap.cube.cuboid.Cuboid;
+import com.kylinolap.cube.cuboid.CuboidScheduler;
+import com.kylinolap.job.constant.BatchConstants;
+import com.kylinolap.job.hadoop.AbstractHadoopJob;
+import com.kylinolap.metadata.model.cube.CubeDesc;
+
+/**
+ * @author George Song (ysong1)
+ * 
+ */
+public class NDCuboidMapper extends Mapper<Text, Text, Text, Text> {
+
+    private static final Logger logger = LoggerFactory.getLogger(NDCuboidMapper.class);
+
+    private Text outputKey = new Text();
+    private String cubeName;
+    private String segmentName;
+    private CubeDesc cubeDesc;
+    private CuboidScheduler cuboidScheduler;
+
+    private int handleCounter;
+    private int skipCounter;
+
+    private byte[] keyBuf = new byte[4096];
+    private RowKeySplitter rowKeySplitter;
+
+    @Override
+    protected void setup(Context context) throws IOException {
+
+        cubeName = context.getConfiguration().get(BatchConstants.CFG_CUBE_NAME).toUpperCase();
+        segmentName = context.getConfiguration().get(BatchConstants.CFG_CUBE_SEGMENT_NAME).toUpperCase();
+
+        KylinConfig config = AbstractHadoopJob.loadKylinPropsAndMetadata(context.getConfiguration());
+
+        CubeInstance cube = CubeManager.getInstance(config).getCube(cubeName);
+        CubeSegment cubeSegment = cube.getSegment(segmentName, CubeSegmentStatusEnum.NEW);
+        cubeDesc = cube.getDescriptor();
+
+        // initialize CubiodScheduler
+        cuboidScheduler = new CuboidScheduler(cubeDesc);
+
+        rowKeySplitter = new RowKeySplitter(cubeSegment, 65, 256);
+    }
+
+    private int buildKey(Cuboid parentCuboid, Cuboid childCuboid, SplittedBytes[] splitBuffers) {
+        int offset = 0;
+
+        // cuboid id
+        System.arraycopy(childCuboid.getBytes(), 0, keyBuf, offset, childCuboid.getBytes().length);
+        offset += childCuboid.getBytes().length;
+
+        // rowkey columns
+        long mask = Long.highestOneBit(parentCuboid.getId());
+        long parentCuboidId = parentCuboid.getId();
+        long childCuboidId = childCuboid.getId();
+        long parentCuboidIdActualLength = Long.SIZE - Long.numberOfLeadingZeros(parentCuboid.getId());
+        int index = 1; // skip cuboidId
+        for (int i = 0; i < parentCuboidIdActualLength; i++) {
+            if ((mask & parentCuboidId) > 0) {// if the this bit position equals
+                                              // 1
+                if ((mask & childCuboidId) > 0) {// if the child cuboid has this
+                                                 // column
+                    System.arraycopy(splitBuffers[index].value, 0, keyBuf, offset, splitBuffers[index].length);
+                    offset += splitBuffers[index].length;
+                }
+                index++;
+            }
+            mask = mask >> 1;
+        }
+
+        return offset;
+    }
+
+    @Override
+    public void map(Text key, Text value, Context context) throws IOException, InterruptedException {
+        long cuboidId = rowKeySplitter.split(key.getBytes(), key.getLength());
+        Cuboid parentCuboid = Cuboid.findById(cubeDesc, cuboidId);
+
+        Collection<Long> myChildren = cuboidScheduler.getSpanningCuboid(cuboidId);
+
+        // if still empty or null
+        if (myChildren == null || myChildren.size() == 0) {
+            context.getCounter(BatchConstants.MAPREDUCE_COUTNER_GROUP_NAME, "Skipped records").increment(1L);
+            skipCounter++;
+            if (skipCounter % BatchConstants.COUNTER_MAX == 0) {
+                logger.info("Skipped " + skipCounter + " records!");
+            }
+            return;
+        }
+
+        context.getCounter(BatchConstants.MAPREDUCE_COUTNER_GROUP_NAME, "Processed records").increment(1L);
+
+        handleCounter++;
+        if (handleCounter % BatchConstants.COUNTER_MAX == 0) {
+            logger.info("Handled " + handleCounter + " records!");
+        }
+
+        for (Long child : myChildren) {
+            Cuboid childCuboid = Cuboid.findById(cubeDesc, child);
+            int keyLength = buildKey(parentCuboid, childCuboid, rowKeySplitter.getSplitBuffers());
+            outputKey.set(keyBuf, 0, keyLength);
+            context.write(outputKey, value);
+        }
+
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/job/src/main/java/com/kylinolap/job/hadoop/cube/NewBaseCuboidMapper.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/com/kylinolap/job/hadoop/cube/NewBaseCuboidMapper.java b/job/src/main/java/com/kylinolap/job/hadoop/cube/NewBaseCuboidMapper.java
new file mode 100644
index 0000000..03ea2a0
--- /dev/null
+++ b/job/src/main/java/com/kylinolap/job/hadoop/cube/NewBaseCuboidMapper.java
@@ -0,0 +1,342 @@
+/*
+ * 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.job.hadoop.cube;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.LinkedList;
+import java.util.List;
+
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.Pair;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.mapreduce.Mapper;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.kylinolap.common.KylinConfig;
+import com.kylinolap.common.util.Array;
+import com.kylinolap.common.util.ByteArray;
+import com.kylinolap.cube.CubeInstance;
+import com.kylinolap.cube.CubeManager;
+import com.kylinolap.cube.CubeSegment;
+import com.kylinolap.cube.CubeSegmentStatusEnum;
+import com.kylinolap.cube.common.BytesSplitter;
+import com.kylinolap.cube.common.SplittedBytes;
+import com.kylinolap.cube.cuboid.Cuboid;
+import com.kylinolap.cube.kv.AbstractRowKeyEncoder;
+import com.kylinolap.cube.kv.RowConstants;
+import com.kylinolap.cube.measure.MeasureCodec;
+import com.kylinolap.dict.lookup.HiveTable;
+import com.kylinolap.dict.lookup.LookupBytesTable;
+import com.kylinolap.job.constant.BatchConstants;
+import com.kylinolap.job.hadoop.AbstractHadoopJob;
+import com.kylinolap.metadata.MetadataManager;
+import com.kylinolap.metadata.model.cube.CubeDesc;
+import com.kylinolap.metadata.model.cube.DimensionDesc;
+import com.kylinolap.metadata.model.cube.FunctionDesc;
+import com.kylinolap.metadata.model.cube.JoinDesc;
+import com.kylinolap.metadata.model.cube.MeasureDesc;
+import com.kylinolap.metadata.model.cube.ParameterDesc;
+import com.kylinolap.metadata.model.cube.TblColRef;
+import com.kylinolap.metadata.model.schema.TableDesc;
+
+/**
+ * @author George Song (ysong1),honma
+ */
+public class NewBaseCuboidMapper<KEYIN> extends Mapper<KEYIN, Text, Text, Text> {
+
+    private static final Logger logger = LoggerFactory.getLogger(NewBaseCuboidMapper.class);
+
+    private String cubeName;
+    private String segmentName;
+    private Cuboid baseCuboid;
+    private CubeInstance cube;
+    private CubeSegment cubeSegment;
+
+    private CubeDesc cubeDesc;
+    private MetadataManager metadataManager;
+    private TableDesc factTableDesc;
+
+    private boolean byteRowDelimiterInferred = false;
+    private byte byteRowDelimiter;
+
+    private int counter;
+    private Text outputKey = new Text();
+    private Text outputValue = new Text();
+    private Object[] measures;
+    private byte[][] keyBytesBuf;
+    private ByteBuffer valueBuf = ByteBuffer.allocate(RowConstants.ROWVALUE_BUFFER_SIZE);
+
+    private BytesSplitter bytesSplitter;
+    private AbstractRowKeyEncoder rowKeyEncoder;
+    private MeasureCodec measureCodec;
+
+    // deal with table join
+    private HashMap<String, LookupBytesTable> lookupTables;// name -> table
+    private LinkedList<TableJoin> tableJoins;
+    private LinkedList<Pair<Integer, Integer>> factTblColAsRowKey;// similar as
+    // TableJoin.dimTblColAsRowKey
+    private int[][] measureColumnIndice;
+    private byte[] nullValue;
+
+    private class TableJoin {
+        public LinkedList<Integer> fkIndice;// zero-based join columns on fact
+        // table
+        public String lookupTableName;
+        public String joinType;
+
+        // Pair.first -> zero-based column index in lookup table
+        // Pair.second -> zero based row key index
+        public LinkedList<Pair<Integer, Integer>> dimTblColAsRowKey;
+
+        private TableJoin(String joinType, LinkedList<Integer> fkIndice, String lookupTableName, LinkedList<Pair<Integer, Integer>> dimTblColAsRowKey) {
+            this.joinType = joinType;
+            this.fkIndice = fkIndice;
+            this.lookupTableName = lookupTableName;
+            this.dimTblColAsRowKey = dimTblColAsRowKey;
+        }
+    }
+
+    @Override
+    protected void setup(Context context) throws IOException {
+        cubeName = context.getConfiguration().get(BatchConstants.CFG_CUBE_NAME).toUpperCase();
+        segmentName = context.getConfiguration().get(BatchConstants.CFG_CUBE_SEGMENT_NAME);
+
+        KylinConfig config = AbstractHadoopJob.loadKylinPropsAndMetadata(context.getConfiguration());
+
+        metadataManager = MetadataManager.getInstance(config);
+        cube = CubeManager.getInstance(config).getCube(cubeName);
+        cubeSegment = cube.getSegment(segmentName, CubeSegmentStatusEnum.NEW);
+        cubeDesc = cube.getDescriptor();
+        factTableDesc = metadataManager.getTableDesc(cubeDesc.getFactTable());
+
+        long baseCuboidId = Cuboid.getBaseCuboidId(cubeDesc);
+        baseCuboid = Cuboid.findById(cubeDesc, baseCuboidId);
+
+        // intermediateTableDesc = new
+        // JoinedFlatTableDesc(cube.getDescriptor());
+
+        rowKeyEncoder = AbstractRowKeyEncoder.createInstance(cubeSegment, baseCuboid);
+
+        measureCodec = new MeasureCodec(cubeDesc.getMeasures());
+        measures = new Object[cubeDesc.getMeasures().size()];
+
+        int colCount = cubeDesc.getRowkey().getRowKeyColumns().length;
+        keyBytesBuf = new byte[colCount][];
+
+        bytesSplitter = new BytesSplitter(factTableDesc.getColumns().length, 4096);
+
+        nullValue = new byte[] { (byte) '\\', (byte) 'N' };// As in Hive, null
+        // value is
+        // represented by \N
+
+        prepareJoins();
+        prepareMetrics();
+    }
+
+    private void prepareJoins() throws IOException {
+        this.lookupTables = new HashMap<String, LookupBytesTable>();
+        this.tableJoins = new LinkedList<TableJoin>();
+        this.factTblColAsRowKey = new LinkedList<Pair<Integer, Integer>>();
+
+        for (DimensionDesc dim : cubeDesc.getDimensions()) {
+            JoinDesc join = dim.getJoin();
+            if (join != null) {
+                String joinType = join.getType().toUpperCase();
+                String lookupTableName = dim.getTable();
+
+                // load lookup tables
+                if (!lookupTables.containsKey(lookupTableName)) {
+                    HiveTable htable = new HiveTable(metadataManager, lookupTableName);
+                    LookupBytesTable btable = new LookupBytesTable(metadataManager.getTableDesc(lookupTableName), join.getPrimaryKey(), htable);
+                    lookupTables.put(lookupTableName, btable);
+                }
+
+                // create join infos
+                LinkedList<Integer> fkIndice = new LinkedList<Integer>();
+                for (TblColRef colRef : join.getForeignKeyColumns()) {
+                    fkIndice.add(colRef.getColumn().getZeroBasedIndex());
+                }
+                this.tableJoins.add(new TableJoin(joinType, fkIndice, lookupTableName, this.findColumnRowKeyRelationships(dim)));
+
+            } else {
+
+                this.factTblColAsRowKey.addAll(this.findColumnRowKeyRelationships(dim));
+            }
+        }
+
+        // put composite keys joins ahead of single key joins
+        Collections.sort(tableJoins, new Comparator<TableJoin>() {
+            @Override
+            public int compare(TableJoin o1, TableJoin o2) {
+                return Integer.valueOf(o2.fkIndice.size()).compareTo(Integer.valueOf(o1.fkIndice.size()));
+            }
+        });
+    }
+
+    private LinkedList<Pair<Integer, Integer>> findColumnRowKeyRelationships(DimensionDesc dim) {
+        LinkedList<Pair<Integer, Integer>> dimTblColAsRowKey = new LinkedList<Pair<Integer, Integer>>();
+        for (TblColRef colRef : dim.getColumnRefs()) {
+            int dimTableIndex = colRef.getColumn().getZeroBasedIndex();
+            int rowKeyIndex = cubeDesc.getRowkey().getRowKeyIndexByColumnName(colRef.getName());
+            dimTblColAsRowKey.add(new Pair<Integer, Integer>(dimTableIndex, rowKeyIndex));
+        }
+        return dimTblColAsRowKey;
+    }
+
+    private void prepareMetrics() {
+        List<MeasureDesc> measures = cubeDesc.getMeasures();
+        int measureSize = measures.size();
+        measureColumnIndice = new int[measureSize][];
+        for (int i = 0; i < measureSize; i++) {
+            FunctionDesc func = measures.get(i).getFunction();
+            List<TblColRef> colRefs = func.getParameter().getColRefs();
+            if (colRefs == null) {
+                measureColumnIndice[i] = null;
+            } else {
+                measureColumnIndice[i] = new int[colRefs.size()];
+                for (int j = 0; j < colRefs.size(); j++) {
+                    TblColRef c = colRefs.get(j);
+                    int factTblIdx = factTableDesc.findColumnByName(c.getName()).getZeroBasedIndex();
+                    measureColumnIndice[i][j] = factTblIdx;
+                }
+            }
+        }
+    }
+
+    private byte[] trimSplitBuffer(SplittedBytes splittedBytes) {
+        return Arrays.copyOf(splittedBytes.value, splittedBytes.length);
+    }
+
+    private byte[] buildKey(SplittedBytes[] splitBuffers) {
+
+        int filledDimension = 0;// debug
+
+        // join lookup tables, and fill into RowKey the columns in lookup table
+        for (TableJoin tableJoin : this.tableJoins) {
+            String dimTblName = tableJoin.lookupTableName;
+            LookupBytesTable dimTbl = this.lookupTables.get(dimTblName);
+            ByteArray[] rawKey = new ByteArray[tableJoin.fkIndice.size()];
+            for (int i = 0; i < tableJoin.fkIndice.size(); ++i) {
+                rawKey[i] = new ByteArray(trimSplitBuffer(splitBuffers[tableJoin.fkIndice.get(i)]));
+            }
+            Array<ByteArray> key = new Array<ByteArray>(rawKey);
+            ByteArray[] dimRow = dimTbl.getRow(key);
+            if (dimRow == null) {
+                if (tableJoin.joinType.equalsIgnoreCase("INNER")) {
+                    return null;
+                } else if (tableJoin.joinType.equalsIgnoreCase("LEFT")) {
+                    for (Pair<Integer, Integer> relation : tableJoin.dimTblColAsRowKey) {
+                        keyBytesBuf[relation.getSecond()] = nullValue;
+                        filledDimension++;
+                    }
+                }
+            } else {
+                for (Pair<Integer, Integer> relation : tableJoin.dimTblColAsRowKey) {
+                    keyBytesBuf[relation.getSecond()] = dimRow[relation.getFirst()].data;
+                    filledDimension++;
+                }
+            }
+        }
+
+        // fill into RowKey the columns in fact table
+        for (Pair<Integer, Integer> relation : this.factTblColAsRowKey) {
+            keyBytesBuf[relation.getSecond()] = trimSplitBuffer(splitBuffers[relation.getFirst()]);
+            filledDimension++;
+        }
+
+        assert filledDimension == keyBytesBuf.length;
+
+        // all the row key slots(keyBytesBuf) should be complete now
+        return rowKeyEncoder.encode(keyBytesBuf);
+    }
+
+    private void buildValue(SplittedBytes[] splitBuffers) {
+
+        for (int i = 0; i < measures.length; i++) {
+            byte[] valueBytes = getValueBytes(splitBuffers, i);
+            measures[i] = measureCodec.getSerializer(i).valueOf(valueBytes);
+        }
+
+        valueBuf.clear();
+        measureCodec.encode(measures, valueBuf);
+    }
+
+    private byte[] getValueBytes(SplittedBytes[] splitBuffers, int measureIdx) {
+        MeasureDesc desc = cubeDesc.getMeasures().get(measureIdx);
+        ParameterDesc paramDesc = desc.getFunction().getParameter();
+        int[] flatTableIdx = this.measureColumnIndice[measureIdx];
+
+        byte[] result = null;
+
+        // constant
+        if (flatTableIdx == null) {
+            result = Bytes.toBytes(paramDesc.getValue());
+        }
+        // column values
+        else {
+            for (int i = 0; i < flatTableIdx.length; i++) {
+                SplittedBytes split = splitBuffers[flatTableIdx[i]];
+                result = Arrays.copyOf(split.value, split.length);
+            }
+        }
+
+        if (desc.getFunction().isCount()) {
+            result = Bytes.toBytes("1");
+        }
+
+        return result;
+    }
+
+    @Override
+    public void map(KEYIN key, Text value, Context context) throws IOException, InterruptedException {
+        // combining the hive table flattening logic into base cuboid building.
+        // the input of this mapper is the fact table rows
+
+        counter++;
+        if (counter % BatchConstants.COUNTER_MAX == 0) {
+            logger.info("Handled " + counter + " records!");
+        }
+
+        if (!byteRowDelimiterInferred)
+            byteRowDelimiter = bytesSplitter.inferByteRowDelimiter(value.getBytes(), value.getLength(), factTableDesc.getColumns().length);
+
+        bytesSplitter.split(value.getBytes(), value.getLength(), byteRowDelimiter);
+
+        try {
+            byte[] rowKey = buildKey(bytesSplitter.getSplitBuffers());
+            if (rowKey == null)
+                return;// skip this fact table row
+
+            outputKey.set(rowKey, 0, rowKey.length);
+
+            buildValue(bytesSplitter.getSplitBuffers());
+            outputValue.set(valueBuf.array(), 0, valueBuf.position());
+
+            context.write(outputKey, outputValue);
+
+        } catch (Throwable t) {
+            logger.error("", t);
+            context.getCounter(BatchConstants.MAPREDUCE_COUTNER_GROUP_NAME, "Error records").increment(1L);
+            return;
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/job/src/main/java/com/kylinolap/job/hadoop/cube/RangeKeyDistributionJob.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/com/kylinolap/job/hadoop/cube/RangeKeyDistributionJob.java b/job/src/main/java/com/kylinolap/job/hadoop/cube/RangeKeyDistributionJob.java
new file mode 100644
index 0000000..8245266
--- /dev/null
+++ b/job/src/main/java/com/kylinolap/job/hadoop/cube/RangeKeyDistributionJob.java
@@ -0,0 +1,116 @@
+/*
+ * 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.job.hadoop.cube;
+
+import java.io.File;
+
+import org.apache.commons.cli.Options;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.mapreduce.Job;
+import org.apache.hadoop.mapreduce.lib.input.SequenceFileInputFormat;
+import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
+import org.apache.hadoop.mapreduce.lib.output.SequenceFileOutputFormat;
+import org.apache.hadoop.util.ToolRunner;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.kylinolap.common.KylinConfig;
+import com.kylinolap.cube.CubeInstance;
+import com.kylinolap.cube.CubeManager;
+import com.kylinolap.job.constant.BatchConstants;
+import com.kylinolap.job.hadoop.AbstractHadoopJob;
+import com.kylinolap.metadata.model.cube.CubeDesc.CubeCapacity;
+
+/**
+ * @author xjiang, ysong1
+ * 
+ */
+
+public class RangeKeyDistributionJob extends AbstractHadoopJob {
+    protected static final Logger log = LoggerFactory.getLogger(RangeKeyDistributionJob.class);
+
+    /*
+     * (non-Javadoc)
+     * 
+     * @see org.apache.hadoop.util.Tool#run(java.lang.String[])
+     */
+    @Override
+    public int run(String[] args) throws Exception {
+        Options options = new Options();
+
+        try {
+            options.addOption(OPTION_INPUT_PATH);
+            options.addOption(OPTION_OUTPUT_PATH);
+            options.addOption(OPTION_JOB_NAME);
+            options.addOption(OPTION_CUBE_NAME);
+
+            parseOptions(options, args);
+
+            // start job
+            String jobName = getOptionValue(OPTION_JOB_NAME);
+            job = Job.getInstance(getConf(), jobName);
+
+            File JarFile = new File(KylinConfig.getInstanceFromEnv().getKylinJobJarPath());
+            if (JarFile.exists()) {
+                job.setJar(KylinConfig.getInstanceFromEnv().getKylinJobJarPath());
+            } else {
+                job.setJarByClass(this.getClass());
+            }
+
+            addInputDirs(getOptionValue(OPTION_INPUT_PATH), job);
+
+            Path output = new Path(getOptionValue(OPTION_OUTPUT_PATH));
+            FileOutputFormat.setOutputPath(job, output);
+            // job.getConfiguration().set("dfs.block.size", "67108864");
+
+            // Mapper
+            job.setInputFormatClass(SequenceFileInputFormat.class);
+            job.setMapperClass(RangeKeyDistributionMapper.class);
+            job.setMapOutputKeyClass(Text.class);
+            job.setMapOutputValueClass(LongWritable.class);
+
+            // Reducer - only one
+            job.setReducerClass(RangeKeyDistributionReducer.class);
+            job.setOutputFormatClass(SequenceFileOutputFormat.class);
+            job.setOutputKeyClass(Text.class);
+            job.setOutputValueClass(LongWritable.class);
+            job.setNumReduceTasks(1);
+
+            this.deletePath(job.getConfiguration(), output);
+
+            String cubeName = getOptionValue(OPTION_CUBE_NAME).toUpperCase();
+            CubeManager cubeMgr = CubeManager.getInstance(KylinConfig.getInstanceFromEnv());
+            CubeInstance cube = cubeMgr.getCube(cubeName);
+            CubeCapacity cubeCapacity = cube.getDescriptor().getCapacity();
+            job.getConfiguration().set(BatchConstants.CUBE_CAPACITY, cubeCapacity.toString());
+
+            return waitForCompletion(job);
+        } catch (Exception e) {
+            printUsage(options);
+            log.error(e.getLocalizedMessage(), e);
+            return 2;
+        }
+    }
+
+    public static void main(String[] args) throws Exception {
+        int exitCode = ToolRunner.run(new RangeKeyDistributionJob(), args);
+        System.exit(exitCode);
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/job/src/main/java/com/kylinolap/job/hadoop/cube/RangeKeyDistributionMapper.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/com/kylinolap/job/hadoop/cube/RangeKeyDistributionMapper.java b/job/src/main/java/com/kylinolap/job/hadoop/cube/RangeKeyDistributionMapper.java
new file mode 100644
index 0000000..f02ae1a
--- /dev/null
+++ b/job/src/main/java/com/kylinolap/job/hadoop/cube/RangeKeyDistributionMapper.java
@@ -0,0 +1,64 @@
+/*
+ * 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.job.hadoop.cube;
+
+import java.io.IOException;
+
+import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.mapreduce.Mapper;
+
+/**
+ * @author ysong1
+ * 
+ */
+public class RangeKeyDistributionMapper extends Mapper<Text, Text, Text, LongWritable> {
+
+    private static final long ONE_MEGA_BYTES = 1L * 1024L * 1024L;
+
+    private LongWritable outputValue = new LongWritable(0);
+
+    private long bytesRead = 0;
+
+    private Text lastKey;
+
+    @Override
+    public void map(Text key, Text value, Context context) throws IOException, InterruptedException {
+        lastKey = key;
+
+        int bytesLength = key.getLength() + value.getLength();
+        bytesRead += bytesLength;
+
+        if (bytesRead >= ONE_MEGA_BYTES) {
+            outputValue.set(bytesRead);
+            context.write(key, outputValue);
+
+            // reset bytesRead
+            bytesRead = 0;
+        }
+
+    }
+
+    @Override
+    protected void cleanup(Context context) throws IOException, InterruptedException {
+        if (lastKey != null) {
+            outputValue.set(bytesRead);
+            context.write(lastKey, outputValue);
+        }
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/job/src/main/java/com/kylinolap/job/hadoop/cube/RangeKeyDistributionReducer.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/com/kylinolap/job/hadoop/cube/RangeKeyDistributionReducer.java b/job/src/main/java/com/kylinolap/job/hadoop/cube/RangeKeyDistributionReducer.java
new file mode 100644
index 0000000..dafea36
--- /dev/null
+++ b/job/src/main/java/com/kylinolap/job/hadoop/cube/RangeKeyDistributionReducer.java
@@ -0,0 +1,91 @@
+/*
+ * 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.job.hadoop.cube;
+
+import java.io.IOException;
+
+import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.mapreduce.Reducer;
+import org.apache.hadoop.util.StringUtils;
+
+import com.kylinolap.job.constant.BatchConstants;
+import com.kylinolap.metadata.model.cube.CubeDesc.CubeCapacity;
+
+/**
+ * @author ysong1
+ * 
+ */
+public class RangeKeyDistributionReducer extends Reducer<Text, LongWritable, Text, LongWritable> {
+
+    public static final long FIVE_GIGA_BYTES = 5L * 1024L * 1024L * 1024L;
+    public static final long TEN_GIGA_BYTES = 10L * 1024L * 1024L * 1024L;
+    public static final long TWENTY_GIGA_BYTES = 20L * 1024L * 1024L * 1024L;
+
+    private LongWritable outputValue = new LongWritable(0);
+
+    private long bytesRead = 0;
+    private Text lastKey;
+
+    private CubeCapacity cubeCapacity;
+    private long cut;
+
+    @Override
+    protected void setup(Context context) throws IOException {
+        cubeCapacity = CubeCapacity.valueOf(context.getConfiguration().get(BatchConstants.CUBE_CAPACITY));
+        switch (cubeCapacity) {
+        case SMALL:
+            cut = FIVE_GIGA_BYTES;
+            break;
+        case MEDIUM:
+            cut = TEN_GIGA_BYTES;
+            break;
+        case LARGE:
+            cut = TWENTY_GIGA_BYTES;
+            break;
+        }
+    }
+
+    @Override
+    public void reduce(Text key, Iterable<LongWritable> values, Context context) throws IOException, InterruptedException {
+        lastKey = key;
+        long length = 0;
+        for (LongWritable v : values) {
+            length += v.get();
+        }
+
+        bytesRead += length;
+
+        if (bytesRead >= cut) {
+            outputValue.set(bytesRead);
+            context.write(key, outputValue);
+            System.out.println(StringUtils.byteToHexString(key.getBytes()) + "\t" + outputValue.get());
+            // reset bytesRead
+            bytesRead = 0;
+        }
+
+    }
+
+    @Override
+    protected void cleanup(Context context) throws IOException, InterruptedException {
+        if (lastKey != null) {
+            outputValue.set(bytesRead);
+            context.write(lastKey, outputValue);
+            System.out.println(StringUtils.byteToHexString(lastKey.getBytes()) + "\t" + outputValue.get());
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/job/src/main/java/com/kylinolap/job/hadoop/cube/RowKeyDistributionCheckerJob.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/com/kylinolap/job/hadoop/cube/RowKeyDistributionCheckerJob.java b/job/src/main/java/com/kylinolap/job/hadoop/cube/RowKeyDistributionCheckerJob.java
new file mode 100644
index 0000000..718c188
--- /dev/null
+++ b/job/src/main/java/com/kylinolap/job/hadoop/cube/RowKeyDistributionCheckerJob.java
@@ -0,0 +1,97 @@
+/*
+ * 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.job.hadoop.cube;
+
+import org.apache.commons.cli.Option;
+import org.apache.commons.cli.OptionBuilder;
+import org.apache.commons.cli.Options;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.mapreduce.Job;
+import org.apache.hadoop.mapreduce.lib.input.SequenceFileInputFormat;
+import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
+import org.apache.hadoop.mapreduce.lib.output.SequenceFileOutputFormat;
+import org.apache.hadoop.util.ToolRunner;
+
+import com.kylinolap.job.hadoop.AbstractHadoopJob;
+
+/**
+ * @author ysong1
+ * 
+ */
+public class RowKeyDistributionCheckerJob extends AbstractHadoopJob {
+
+    @SuppressWarnings("static-access")
+    protected static final Option rowKeyStatsFilePath = OptionBuilder.withArgName("path").hasArg().isRequired(true).withDescription("rowKeyStatsFilePath").create("rowKeyStatsFilePath");
+
+    @Override
+    public int run(String[] args) throws Exception {
+        Options options = new Options();
+
+        try {
+            options.addOption(OPTION_INPUT_PATH);
+            options.addOption(OPTION_OUTPUT_PATH);
+            options.addOption(OPTION_JOB_NAME);
+            options.addOption(rowKeyStatsFilePath);
+
+            parseOptions(options, args);
+
+            String statsFilePath = getOptionValue(rowKeyStatsFilePath);
+
+            // start job
+            String jobName = getOptionValue(OPTION_JOB_NAME);
+            job = Job.getInstance(getConf(), jobName);
+
+            job.setJarByClass(this.getClass());
+
+            addInputDirs(getOptionValue(OPTION_INPUT_PATH), job);
+
+            Path output = new Path(getOptionValue(OPTION_OUTPUT_PATH));
+            FileOutputFormat.setOutputPath(job, output);
+
+            // Mapper
+            job.setInputFormatClass(SequenceFileInputFormat.class);
+            job.setMapperClass(RowKeyDistributionCheckerMapper.class);
+            job.setMapOutputKeyClass(Text.class);
+            job.setMapOutputValueClass(LongWritable.class);
+
+            // Reducer - only one
+            job.setReducerClass(RowKeyDistributionCheckerReducer.class);
+            job.setOutputFormatClass(SequenceFileOutputFormat.class);
+            job.setOutputKeyClass(Text.class);
+            job.setOutputValueClass(LongWritable.class);
+            job.setNumReduceTasks(1);
+
+            job.getConfiguration().set("rowKeyStatsFilePath", statsFilePath);
+
+            this.deletePath(job.getConfiguration(), output);
+
+            return waitForCompletion(job);
+        } catch (Exception e) {
+            printUsage(options);
+            log.error(e.getLocalizedMessage(), e);
+            return 2;
+        }
+    }
+
+    public static void main(String[] args) throws Exception {
+        int exitCode = ToolRunner.run(new RowKeyDistributionCheckerJob(), args);
+        System.exit(exitCode);
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/job/src/main/java/com/kylinolap/job/hadoop/cube/RowKeyDistributionCheckerMapper.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/com/kylinolap/job/hadoop/cube/RowKeyDistributionCheckerMapper.java b/job/src/main/java/com/kylinolap/job/hadoop/cube/RowKeyDistributionCheckerMapper.java
new file mode 100644
index 0000000..76e3f37
--- /dev/null
+++ b/job/src/main/java/com/kylinolap/job/hadoop/cube/RowKeyDistributionCheckerMapper.java
@@ -0,0 +1,107 @@
+/*
+ * 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.job.hadoop.cube;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.io.IOUtils;
+import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.io.SequenceFile;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.io.Writable;
+import org.apache.hadoop.mapreduce.Mapper;
+import org.apache.hadoop.util.ReflectionUtils;
+
+/**
+ * @author ysong1
+ * 
+ */
+public class RowKeyDistributionCheckerMapper extends Mapper<Text, Text, Text, LongWritable> {
+
+    String rowKeyStatsFilePath;
+    byte[][] splitKeys;
+    Map<Text, Long> resultMap;
+    List<Text> keyList;
+
+    @Override
+    protected void setup(Context context) throws IOException {
+        rowKeyStatsFilePath = context.getConfiguration().get("rowKeyStatsFilePath");
+        splitKeys = this.getSplits(context.getConfiguration(), new Path(rowKeyStatsFilePath));
+
+        resultMap = new HashMap<Text, Long>();
+        keyList = new ArrayList<Text>();
+        for (int i = 0; i < splitKeys.length; i++) {
+            Text key = new Text(splitKeys[i]);
+            resultMap.put(key, 0L);
+            keyList.add(new Text(splitKeys[i]));
+        }
+    }
+
+    @Override
+    public void map(Text key, Text value, Context context) throws IOException, InterruptedException {
+        for (Text t : keyList) {
+            if (key.compareTo(t) < 0) {
+                Long v = resultMap.get(t);
+                long length = key.getLength() + value.getLength();
+                v += length;
+                resultMap.put(t, v);
+                break;
+            }
+        }
+    }
+
+    @Override
+    protected void cleanup(Context context) throws IOException, InterruptedException {
+        LongWritable outputValue = new LongWritable();
+        for (Entry<Text, Long> kv : resultMap.entrySet()) {
+            outputValue.set(kv.getValue());
+            context.write(kv.getKey(), outputValue);
+        }
+    }
+
+    @SuppressWarnings("deprecation")
+    public byte[][] getSplits(Configuration conf, Path path) {
+        List<byte[]> rowkeyList = new ArrayList<byte[]>();
+        SequenceFile.Reader reader = null;
+        try {
+            reader = new SequenceFile.Reader(path.getFileSystem(conf), path, conf);
+            Writable key = (Writable) ReflectionUtils.newInstance(reader.getKeyClass(), conf);
+            Writable value = (Writable) ReflectionUtils.newInstance(reader.getValueClass(), conf);
+            while (reader.next(key, value)) {
+                byte[] tmp = ((Text) key).copyBytes();
+                if (rowkeyList.contains(tmp) == false) {
+                    rowkeyList.add(tmp);
+                }
+            }
+        } catch (Exception e) {
+            e.printStackTrace();
+        } finally {
+            IOUtils.closeStream(reader);
+        }
+
+        byte[][] retValue = rowkeyList.toArray(new byte[rowkeyList.size()][]);
+
+        return retValue;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/job/src/main/java/com/kylinolap/job/hadoop/cube/RowKeyDistributionCheckerReducer.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/com/kylinolap/job/hadoop/cube/RowKeyDistributionCheckerReducer.java b/job/src/main/java/com/kylinolap/job/hadoop/cube/RowKeyDistributionCheckerReducer.java
new file mode 100644
index 0000000..742f644
--- /dev/null
+++ b/job/src/main/java/com/kylinolap/job/hadoop/cube/RowKeyDistributionCheckerReducer.java
@@ -0,0 +1,44 @@
+/*
+ * 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.job.hadoop.cube;
+
+import java.io.IOException;
+
+import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.mapreduce.Reducer;
+
+/**
+ * @author ysong1
+ * 
+ */
+public class RowKeyDistributionCheckerReducer extends Reducer<Text, LongWritable, Text, LongWritable> {
+
+    LongWritable outputKey = new LongWritable(0L);
+
+    @Override
+    public void reduce(Text key, Iterable<LongWritable> values, Context context) throws IOException, InterruptedException {
+
+        long length = 0;
+        for (LongWritable v : values) {
+            length += v.get();
+        }
+
+        outputKey.set(length);
+        context.write(key, outputKey);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/job/src/main/java/com/kylinolap/job/hadoop/cube/StorageCleanupJob.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/com/kylinolap/job/hadoop/cube/StorageCleanupJob.java b/job/src/main/java/com/kylinolap/job/hadoop/cube/StorageCleanupJob.java
new file mode 100644
index 0000000..51d893c
--- /dev/null
+++ b/job/src/main/java/com/kylinolap/job/hadoop/cube/StorageCleanupJob.java
@@ -0,0 +1,217 @@
+/*
+ * 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.job.hadoop.cube;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.commons.cli.Option;
+import org.apache.commons.cli.OptionBuilder;
+import org.apache.commons.cli.Options;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.HBaseConfiguration;
+import org.apache.hadoop.hbase.HTableDescriptor;
+import org.apache.hadoop.hbase.MasterNotRunningException;
+import org.apache.hadoop.hbase.ZooKeeperConnectionException;
+import org.apache.hadoop.hbase.client.HBaseAdmin;
+import org.apache.hadoop.util.ToolRunner;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.kylinolap.common.KylinConfig;
+import com.kylinolap.cube.CubeInstance;
+import com.kylinolap.cube.CubeManager;
+import com.kylinolap.cube.CubeSegment;
+import com.kylinolap.job.JobDAO;
+import com.kylinolap.job.JobInstance;
+import com.kylinolap.job.constant.JobStatusEnum;
+import com.kylinolap.job.engine.JobEngineConfig;
+import com.kylinolap.job.hadoop.AbstractHadoopJob;
+
+/**
+ * @author ysong1
+ */
+public class StorageCleanupJob extends AbstractHadoopJob {
+
+    @SuppressWarnings("static-access")
+    private static final Option OPTION_DELETE = OptionBuilder.withArgName("delete").hasArg().isRequired(false).withDescription("Delete the unused storage").create("delete");
+
+    protected static final Logger log = LoggerFactory.getLogger(StorageCleanupJob.class);
+
+    boolean delete = false;
+
+    /*
+     * (non-Javadoc)
+     * 
+     * @see org.apache.hadoop.util.Tool#run(java.lang.String[])
+     */
+    @Override
+    public int run(String[] args) throws Exception {
+        Options options = new Options();
+        try {
+            options.addOption(OPTION_DELETE);
+            parseOptions(options, args);
+
+            delete = Boolean.parseBoolean(getOptionValue(OPTION_DELETE));
+
+            Configuration conf = HBaseConfiguration.create(getConf());
+
+            cleanUnusedHdfsFiles(conf);
+            cleanUnusedHBaseTables(conf);
+            cleanUnusedIntermediateHiveTable(conf);
+
+            return 0;
+        } catch (Exception e) {
+            e.printStackTrace(System.err);
+            log.error(e.getLocalizedMessage(), e);
+            return 2;
+        }
+    }
+
+    private boolean isJobInUse(JobInstance job) {
+        if (job.getStatus().equals(JobStatusEnum.NEW) || job.getStatus().equals(JobStatusEnum.PENDING) || job.getStatus().equals(JobStatusEnum.RUNNING) || job.getStatus().equals(JobStatusEnum.ERROR)) {
+            return true;
+        } else {
+            return false;
+        }
+    }
+
+
+    private void cleanUnusedHBaseTables(Configuration conf) throws MasterNotRunningException, ZooKeeperConnectionException, IOException {
+        CubeManager cubeMgr = CubeManager.getInstance(KylinConfig.getInstanceFromEnv());
+
+        // get all kylin hbase tables
+        HBaseAdmin hbaseAdmin = new HBaseAdmin(conf);
+        String tableNamePrefix = CubeManager.getHBaseStorageLocationPrefix();
+        HTableDescriptor[] tableDescriptors = hbaseAdmin.listTables(tableNamePrefix + ".*");
+        List<String> allTablesNeedToBeDropped = new ArrayList<String>();
+        for (HTableDescriptor desc : tableDescriptors) {
+            String host = desc.getValue(CubeManager.getHtableMetadataKey());
+            if (KylinConfig.getInstanceFromEnv().getMetadataUrlPrefix().equalsIgnoreCase(host)) {
+                //only take care htables that belongs to self
+                allTablesNeedToBeDropped.add(desc.getTableName().getNameAsString());
+            }
+        }
+
+        // remove every segment htable from drop list
+        for (CubeInstance cube : cubeMgr.listAllCubes()) {
+            for (CubeSegment seg : cube.getSegments()) {
+                String tablename = seg.getStorageLocationIdentifier();
+                allTablesNeedToBeDropped.remove(tablename);
+                log.info("Remove table " + tablename + " from drop list, as the table belongs to cube " + cube.getName() + " with status " + cube.getStatus());
+            }
+        }
+
+        if (delete == true) {
+            // drop tables
+            for (String htableName : allTablesNeedToBeDropped) {
+                log.info("Deleting HBase table " + htableName);
+                if (hbaseAdmin.tableExists(htableName)) {
+                    hbaseAdmin.disableTable(htableName);
+                    hbaseAdmin.deleteTable(htableName);
+                    log.info("Deleted HBase table " + htableName);
+                } else {
+                    log.info("HBase table" + htableName + " does not exist");
+                }
+            }
+        } else {
+            System.out.println("--------------- Tables To Be Dropped ---------------");
+            for (String htableName : allTablesNeedToBeDropped) {
+                System.out.println(htableName);
+            }
+            System.out.println("----------------------------------------------------");
+        }
+
+        hbaseAdmin.close();
+    }
+
+    private void cleanUnusedHdfsFiles(Configuration conf) throws IOException {
+        JobEngineConfig engineConfig = new JobEngineConfig(KylinConfig.getInstanceFromEnv());
+        CubeManager cubeMgr = CubeManager.getInstance(KylinConfig.getInstanceFromEnv());
+
+        FileSystem fs = FileSystem.get(conf);
+        List<String> allHdfsPathsNeedToBeDeleted = new ArrayList<String>();
+        // GlobFilter filter = new
+        // GlobFilter(KylinConfig.getInstanceFromEnv().getHdfsWorkingDirectory()
+        // + "/kylin-.*");
+        FileStatus[] fStatus = fs.listStatus(new Path(KylinConfig.getInstanceFromEnv().getHdfsWorkingDirectory()));
+        for (FileStatus status : fStatus) {
+            String path = status.getPath().getName();
+            // System.out.println(path);
+            if (path.startsWith(JobInstance.JOB_WORKING_DIR_PREFIX)) {
+                String kylinJobPath = engineConfig.getHdfsWorkingDirectory() + "/" + path;
+                allHdfsPathsNeedToBeDeleted.add(kylinJobPath);
+            }
+        }
+
+        List<JobInstance> allJobs = JobDAO.getInstance(KylinConfig.getInstanceFromEnv()).listAllJobs();
+        for (JobInstance jobInstance : allJobs) {
+            // only remove FINISHED and DISCARDED job intermediate files
+            if (isJobInUse(jobInstance) == true) {
+                String path = JobInstance.getJobWorkingDir(jobInstance, engineConfig);
+                allHdfsPathsNeedToBeDeleted.remove(path);
+                log.info("Remove " + path + " from deletion list, as the path belongs to job " + jobInstance.getUuid() + " with status " + jobInstance.getStatus());
+            }
+        }
+
+        // remove every segment working dir from deletion list
+        for (CubeInstance cube : cubeMgr.listAllCubes()) {
+            for (CubeSegment seg : cube.getSegments()) {
+                String jobUuid = seg.getLastBuildJobID();
+                if (jobUuid != null && jobUuid.equals("") == false) {
+                    String path = JobInstance.getJobWorkingDir(jobUuid, engineConfig.getHdfsWorkingDirectory());
+                    allHdfsPathsNeedToBeDeleted.remove(path);
+                    log.info("Remove " + path + " from deletion list, as the path belongs to segment " + seg + " of cube " + cube.getName());
+                }
+            }
+        }
+
+        if (delete == true) {
+            // remove files
+            for (String hdfsPath : allHdfsPathsNeedToBeDeleted) {
+                log.info("Deleting hdfs path " + hdfsPath);
+                Path p = new Path(hdfsPath);
+                if (fs.exists(p) == true) {
+                    fs.delete(p, true);
+                    log.info("Deleted hdfs path " + hdfsPath);
+                } else {
+                    log.info("Hdfs path " + hdfsPath + "does not exist");
+                }
+            }
+        } else {
+            System.out.println("--------------- HDFS Path To Be Deleted ---------------");
+            for (String hdfsPath : allHdfsPathsNeedToBeDeleted) {
+                System.out.println(hdfsPath);
+            }
+            System.out.println("-------------------------------------------------------");
+        }
+
+    }
+
+    private void cleanUnusedIntermediateHiveTable(Configuration conf) throws IOException {
+
+    }
+
+    public static void main(String[] args) throws Exception {
+        int exitCode = ToolRunner.run(new StorageCleanupJob(), args);
+        System.exit(exitCode);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/job/src/main/java/com/kylinolap/job/hadoop/dict/CreateDictionaryJob.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/com/kylinolap/job/hadoop/dict/CreateDictionaryJob.java b/job/src/main/java/com/kylinolap/job/hadoop/dict/CreateDictionaryJob.java
new file mode 100644
index 0000000..0ee1811
--- /dev/null
+++ b/job/src/main/java/com/kylinolap/job/hadoop/dict/CreateDictionaryJob.java
@@ -0,0 +1,67 @@
+/*
+ * 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.job.hadoop.dict;
+
+import org.apache.commons.cli.Options;
+import org.apache.hadoop.util.ToolRunner;
+
+import com.kylinolap.common.KylinConfig;
+import com.kylinolap.cube.cli.DictionaryGeneratorCLI;
+import com.kylinolap.job.hadoop.AbstractHadoopJob;
+
+/**
+ * @author ysong1
+ * 
+ */
+
+public class CreateDictionaryJob extends AbstractHadoopJob {
+
+    private int returnCode = 0;
+
+    @Override
+    public int run(String[] args) throws Exception {
+        Options options = new Options();
+
+        try {
+            options.addOption(OPTION_CUBE_NAME);
+            options.addOption(OPTION_SEGMENT_NAME);
+            options.addOption(OPTION_INPUT_PATH);
+            parseOptions(options, args);
+
+            String cubeName = getOptionValue(OPTION_CUBE_NAME);
+            String segmentName = getOptionValue(OPTION_SEGMENT_NAME);
+            String factColumnsInputPath = getOptionValue(OPTION_INPUT_PATH);
+
+            KylinConfig config = KylinConfig.getInstanceFromEnv();
+
+            DictionaryGeneratorCLI.processSegment(config, cubeName, segmentName, factColumnsInputPath);
+        } catch (Exception e) {
+            printUsage(options);
+            e.printStackTrace(System.err);
+            log.error(e.getLocalizedMessage(), e);
+            returnCode = 2;
+        }
+
+        return returnCode;
+    }
+
+    public static void main(String[] args) throws Exception {
+        int exitCode = ToolRunner.run(new CreateDictionaryJob(), args);
+        System.exit(exitCode);
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/job/src/main/java/com/kylinolap/job/hadoop/dict/CreateInvertedIndexDictionaryJob.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/com/kylinolap/job/hadoop/dict/CreateInvertedIndexDictionaryJob.java b/job/src/main/java/com/kylinolap/job/hadoop/dict/CreateInvertedIndexDictionaryJob.java
new file mode 100644
index 0000000..d75a4a9
--- /dev/null
+++ b/job/src/main/java/com/kylinolap/job/hadoop/dict/CreateInvertedIndexDictionaryJob.java
@@ -0,0 +1,66 @@
+/*
+ * 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.job.hadoop.dict;
+
+import org.apache.commons.cli.Options;
+import org.apache.hadoop.util.ToolRunner;
+
+import com.kylinolap.common.KylinConfig;
+import com.kylinolap.cube.CubeInstance;
+import com.kylinolap.cube.CubeManager;
+import com.kylinolap.job.hadoop.AbstractHadoopJob;
+
+/**
+ * @author ysong1
+ * 
+ */
+public class CreateInvertedIndexDictionaryJob extends AbstractHadoopJob {
+
+    @Override
+    public int run(String[] args) throws Exception {
+        Options options = new Options();
+
+        try {
+            options.addOption(OPTION_CUBE_NAME);
+            options.addOption(OPTION_INPUT_PATH);
+            parseOptions(options, args);
+
+            String cubeName = getOptionValue(OPTION_CUBE_NAME);
+            String factColumnsInputPath = getOptionValue(OPTION_INPUT_PATH);
+            KylinConfig config = KylinConfig.getInstanceFromEnv();
+
+            CubeManager mgr = CubeManager.getInstance(config);
+            CubeInstance cube = mgr.getCube(cubeName);
+            if (cube == null || cube.isInvertedIndex() == false)
+                throw new IllegalArgumentException("No Inverted Index Cube found by name " + cubeName);
+
+            mgr.buildInvertedIndexDictionary(cube.getFirstSegment(), factColumnsInputPath);
+            return 0;
+        } catch (Exception e) {
+            printUsage(options);
+            e.printStackTrace(System.err);
+            log.error(e.getLocalizedMessage(), e);
+            return 2;
+        }
+    }
+
+    public static void main(String[] args) throws Exception {
+        int exitCode = ToolRunner.run(new CreateInvertedIndexDictionaryJob(), args);
+        System.exit(exitCode);
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/job/src/main/java/com/kylinolap/job/hadoop/hbase/BulkLoadJob.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/com/kylinolap/job/hadoop/hbase/BulkLoadJob.java b/job/src/main/java/com/kylinolap/job/hadoop/hbase/BulkLoadJob.java
new file mode 100644
index 0000000..1037c70
--- /dev/null
+++ b/job/src/main/java/com/kylinolap/job/hadoop/hbase/BulkLoadJob.java
@@ -0,0 +1,95 @@
+/*
+ * 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.job.hadoop.hbase;
+
+import org.apache.commons.cli.Options;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.hbase.HBaseConfiguration;
+import org.apache.hadoop.hbase.mapreduce.LoadIncrementalHFiles;
+import org.apache.hadoop.util.ToolRunner;
+
+import com.kylinolap.common.KylinConfig;
+import com.kylinolap.cube.CubeInstance;
+import com.kylinolap.cube.CubeManager;
+import com.kylinolap.job.hadoop.AbstractHadoopJob;
+import com.kylinolap.metadata.model.cube.CubeDesc;
+import com.kylinolap.metadata.model.cube.HBaseColumnFamilyDesc;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * @author ysong1
+ * 
+ */
+public class BulkLoadJob extends AbstractHadoopJob {
+
+    protected static final Logger log = LoggerFactory.getLogger(BulkLoadJob.class);
+
+    @Override
+    public int run(String[] args) throws Exception {
+        Options options = new Options();
+
+        try {
+            options.addOption(OPTION_INPUT_PATH);
+            options.addOption(OPTION_HTABLE_NAME);
+            options.addOption(OPTION_CUBE_NAME);
+            parseOptions(options, args);
+
+            String tableName = getOptionValue(OPTION_HTABLE_NAME).toUpperCase();
+            // e.g
+            // /tmp/kylin-3f150b00-3332-41ca-9d3d-652f67f044d7/test_kylin_cube_with_slr_ready_2_segments/hfile/
+            // end with "/"
+            String input = getOptionValue(OPTION_INPUT_PATH);
+
+            Configuration conf = HBaseConfiguration.create(getConf());
+            FileSystem fs = FileSystem.get(conf);
+
+            String cubeName = getOptionValue(OPTION_CUBE_NAME).toUpperCase();
+            KylinConfig config = KylinConfig.getInstanceFromEnv();
+            CubeManager cubeMgr = CubeManager.getInstance(config);
+            CubeInstance cube = cubeMgr.getCube(cubeName);
+            CubeDesc cubeDesc = cube.getDescriptor();
+            FsPermission permission = new FsPermission((short) 0777);
+            for (HBaseColumnFamilyDesc cf : cubeDesc.getHBaseMapping().getColumnFamily()) {
+                String cfName = cf.getName();
+                fs.setPermission(new Path(input + cfName), permission);
+            }
+
+            String[] newArgs = new String[2];
+            newArgs[0] = input;
+            newArgs[1] = tableName;
+
+            log.debug("Start to run LoadIncrementalHFiles");
+            int ret = ToolRunner.run(new LoadIncrementalHFiles(conf), newArgs);
+            log.debug("End to run LoadIncrementalHFiles");
+            return ret;
+        } catch (Exception e) {
+            printUsage(options);
+            e.printStackTrace(System.err);
+            log.error(e.getLocalizedMessage(), e);
+            return 2;
+        }
+    }
+
+    public static void main(String[] args) throws Exception {
+        int exitCode = ToolRunner.run(new BulkLoadJob(), args);
+        System.exit(exitCode);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/job/src/main/java/com/kylinolap/job/hadoop/hbase/CreateHTableJob.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/com/kylinolap/job/hadoop/hbase/CreateHTableJob.java b/job/src/main/java/com/kylinolap/job/hadoop/hbase/CreateHTableJob.java
new file mode 100644
index 0000000..4cb20cb
--- /dev/null
+++ b/job/src/main/java/com/kylinolap/job/hadoop/hbase/CreateHTableJob.java
@@ -0,0 +1,186 @@
+/*
+ * 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.job.hadoop.hbase;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.commons.cli.Options;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.HBaseConfiguration;
+import org.apache.hadoop.hbase.HColumnDescriptor;
+import org.apache.hadoop.hbase.HTableDescriptor;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.HBaseAdmin;
+import org.apache.hadoop.hbase.io.compress.Compression.Algorithm;
+import org.apache.hadoop.hbase.io.encoding.DataBlockEncoding;
+import org.apache.hadoop.hbase.regionserver.ConstantSizeRegionSplitPolicy;
+import org.apache.hadoop.hbase.security.User;
+import org.apache.hadoop.io.IOUtils;
+import org.apache.hadoop.io.SequenceFile;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.io.Writable;
+import org.apache.hadoop.util.ReflectionUtils;
+import org.apache.hadoop.util.ToolRunner;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.kylinolap.common.KylinConfig;
+import com.kylinolap.common.util.HadoopUtil;
+import com.kylinolap.cube.CubeInstance;
+import com.kylinolap.cube.CubeManager;
+import com.kylinolap.job.hadoop.AbstractHadoopJob;
+import com.kylinolap.job.tools.DeployCoprocessorCLI;
+import com.kylinolap.job.tools.LZOSupportnessChecker;
+import com.kylinolap.metadata.model.cube.CubeDesc;
+import com.kylinolap.metadata.model.cube.HBaseColumnFamilyDesc;
+
+/**
+ * @author George Song (ysong1)
+ */
+
+public class CreateHTableJob extends AbstractHadoopJob {
+
+    protected static final Logger log = LoggerFactory.getLogger(CreateHTableJob.class);
+
+    @Override
+    public int run(String[] args) throws Exception {
+        Options options = new Options();
+
+        options.addOption(OPTION_CUBE_NAME);
+        options.addOption(OPTION_PARTITION_FILE_PATH);
+        options.addOption(OPTION_HTABLE_NAME);
+        parseOptions(options, args);
+
+        Path partitionFilePath = new Path(getOptionValue(OPTION_PARTITION_FILE_PATH));
+
+        String cubeName = getOptionValue(OPTION_CUBE_NAME).toUpperCase();
+        KylinConfig config = KylinConfig.getInstanceFromEnv();
+        CubeManager cubeMgr = CubeManager.getInstance(config);
+        CubeInstance cube = cubeMgr.getCube(cubeName);
+        CubeDesc cubeDesc = cube.getDescriptor();
+
+        String tableName = getOptionValue(OPTION_HTABLE_NAME).toUpperCase();
+        HTableDescriptor tableDesc = new HTableDescriptor(TableName.valueOf(tableName));
+        // https://hbase.apache.org/apidocs/org/apache/hadoop/hbase/regionserver/ConstantSizeRegionSplitPolicy.html
+        tableDesc.setValue(HTableDescriptor.SPLIT_POLICY, ConstantSizeRegionSplitPolicy.class.getName());
+        tableDesc.setValue(CubeManager.getHtableMetadataKey(),config.getMetadataUrlPrefix());
+
+        Configuration conf = HBaseConfiguration.create(getConf());
+        HBaseAdmin admin = new HBaseAdmin(conf);
+
+        try {
+            if (User.isHBaseSecurityEnabled(conf)) {
+                // add coprocessor for bulk load
+                tableDesc.addCoprocessor("org.apache.hadoop.hbase.security.access.SecureBulkLoadEndpoint");
+            }
+
+            for (HBaseColumnFamilyDesc cfDesc : cubeDesc.getHBaseMapping().getColumnFamily()) {
+                HColumnDescriptor cf = new HColumnDescriptor(cfDesc.getName());
+                cf.setMaxVersions(1);
+
+                if (LZOSupportnessChecker.getSupportness()) {
+                    log.info("hbase will use lzo to compress data");
+                    cf.setCompressionType(Algorithm.LZO);
+                } else {
+                    log.info("hbase will not use lzo to compress data");
+                }
+
+                cf.setDataBlockEncoding(DataBlockEncoding.FAST_DIFF);
+                cf.setInMemory(false);
+                cf.setBlocksize(4 * 1024 * 1024); // set to 4MB
+                tableDesc.addFamily(cf);
+            }
+
+            byte[][] splitKeys = getSplits(conf, partitionFilePath);
+
+            if (admin.tableExists(tableName)) {
+                // admin.disableTable(tableName);
+                // admin.deleteTable(tableName);
+                throw new RuntimeException("HBase table " + tableName + " exists!");
+            }
+
+            try {
+                initHTableCoprocessor(tableDesc);
+                log.info("hbase table " + tableName + " deployed with coprocessor.");
+
+            } catch (Exception ex) {
+                log.error("Error deploying coprocessor on " + tableName, ex);
+                log.error("Will try creating the table without coprocessor.");
+            }
+
+            admin.createTable(tableDesc, splitKeys);
+            log.info("create hbase table " + tableName + " done.");
+
+            return 0;
+        } catch (Exception e) {
+            printUsage(options);
+            e.printStackTrace(System.err);
+            log.error(e.getLocalizedMessage(), e);
+            return 2;
+        } finally {
+            admin.close();
+        }
+    }
+
+    private void initHTableCoprocessor(HTableDescriptor desc) throws IOException {
+        KylinConfig kylinConfig = KylinConfig.getInstanceFromEnv();
+        Configuration hconf = HadoopUtil.getDefaultConfiguration();
+        FileSystem fileSystem = FileSystem.get(hconf);
+
+        String localCoprocessorJar = kylinConfig.getCoprocessorLocalJar();
+        Path hdfsCoprocessorJar = DeployCoprocessorCLI.uploadCoprocessorJar(localCoprocessorJar, fileSystem, null);
+
+        DeployCoprocessorCLI.setCoprocessorOnHTable(desc, hdfsCoprocessorJar);
+    }
+
+    @SuppressWarnings("deprecation")
+    public byte[][] getSplits(Configuration conf, Path path) throws Exception {
+        List<byte[]> rowkeyList = new ArrayList<byte[]>();
+        SequenceFile.Reader reader = null;
+        try {
+            reader = new SequenceFile.Reader(path.getFileSystem(conf), path, conf);
+            Writable key = (Writable) ReflectionUtils.newInstance(reader.getKeyClass(), conf);
+            Writable value = (Writable) ReflectionUtils.newInstance(reader.getValueClass(), conf);
+            while (reader.next(key, value)) {
+                byte[] tmp = ((Text) key).copyBytes();
+                if (rowkeyList.contains(tmp) == false) {
+                    rowkeyList.add(tmp);
+                }
+            }
+        } catch (Exception e) {
+            e.printStackTrace();
+            throw e;
+        } finally {
+            IOUtils.closeStream(reader);
+        }
+
+        byte[][] retValue = rowkeyList.toArray(new byte[rowkeyList.size()][]);
+        if (retValue.length == 0) {
+            throw new IllegalStateException("Split number is 0, no records in cube??");
+        }
+
+        return retValue;
+    }
+
+    public static void main(String[] args) throws Exception {
+        int exitCode = ToolRunner.run(new CreateHTableJob(), args);
+        System.exit(exitCode);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/job/src/main/java/com/kylinolap/job/hadoop/hive/JoinedFlatTableDesc.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/com/kylinolap/job/hadoop/hive/JoinedFlatTableDesc.java b/job/src/main/java/com/kylinolap/job/hadoop/hive/JoinedFlatTableDesc.java
new file mode 100644
index 0000000..2fdce86
--- /dev/null
+++ b/job/src/main/java/com/kylinolap/job/hadoop/hive/JoinedFlatTableDesc.java
@@ -0,0 +1,171 @@
+/*
+ * 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.job.hadoop.hive;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import com.kylinolap.cube.CubeSegment;
+import com.kylinolap.cube.cuboid.Cuboid;
+import com.kylinolap.metadata.model.cube.CubeDesc;
+import com.kylinolap.metadata.model.cube.FunctionDesc;
+import com.kylinolap.metadata.model.cube.MeasureDesc;
+import com.kylinolap.metadata.model.cube.TblColRef;
+
+/**
+ * @author George Song (ysong1)
+ */
+public class JoinedFlatTableDesc {
+
+    private String tableName;
+    private final CubeDesc cubeDesc;
+    private final CubeSegment cubeSegment;
+
+    private int[] rowKeyColumnIndexes; // the column index on flat table
+    private int[][] measureColumnIndexes; // [i] is the i.th measure related
+                                          // column index on flat table
+
+    public JoinedFlatTableDesc(CubeDesc cubeDesc, CubeSegment cubeSegment) {
+        this.cubeDesc = cubeDesc;
+        this.cubeSegment = cubeSegment;
+        parseCubeDesc();
+    }
+
+    /**
+     * @return the cubeSegment
+     */
+    public CubeSegment getCubeSegment() {
+        return cubeSegment;
+    }
+
+    private List<IntermediateColumnDesc> columnList = new ArrayList<IntermediateColumnDesc>();
+
+    public List<IntermediateColumnDesc> getColumnList() {
+        return columnList;
+    }
+
+    // check what columns from hive tables are required, and index them
+    private void parseCubeDesc() {
+        int rowkeyColCount = cubeDesc.getRowkey().getRowKeyColumns().length;
+        long baseCuboidId = Cuboid.getBaseCuboidId(cubeDesc);
+        Cuboid baseCuboid = Cuboid.findById(cubeDesc, baseCuboidId);
+
+        if (cubeSegment == null) {
+            this.tableName = "kylin_intermediate_" + cubeDesc.getName();
+        } else {
+            this.tableName = "kylin_intermediate_" + cubeDesc.getName() + "_" + cubeSegment.getName();
+        }
+
+        Map<String, Integer> dimensionIndexMap = new HashMap<String, Integer>();
+        int columnIndex = 0;
+        for (TblColRef col : cubeDesc.listDimensionColumnsExcludingDerived()) {
+            dimensionIndexMap.put(col.getName(), columnIndex);
+            columnList.add(new IntermediateColumnDesc(String.valueOf(columnIndex), col.getName(), col.getDatatype(), col.getTable()));
+            columnIndex++;
+        }
+
+        // build index
+        List<TblColRef> cuboidColumns = baseCuboid.getColumns();
+        rowKeyColumnIndexes = new int[rowkeyColCount];
+        for (int i = 0; i < rowkeyColCount; i++) {
+            String colName = cuboidColumns.get(i).getName();
+            Integer dimIdx = dimensionIndexMap.get(colName);
+            if (dimIdx == null) {
+                throw new RuntimeException("Can't find column " + colName);
+            }
+            rowKeyColumnIndexes[i] = dimIdx;
+        }
+
+        List<MeasureDesc> measures = cubeDesc.getMeasures();
+        int measureSize = measures.size();
+        measureColumnIndexes = new int[measureSize][];
+        for (int i = 0; i < measureSize; i++) {
+            FunctionDesc func = measures.get(i).getFunction();
+            List<TblColRef> colRefs = func.getParameter().getColRefs();
+            if (colRefs == null) {
+                measureColumnIndexes[i] = null;
+            } else {
+                measureColumnIndexes[i] = new int[colRefs.size()];
+                for (int j = 0; j < colRefs.size(); j++) {
+                    TblColRef c = colRefs.get(j);
+                    measureColumnIndexes[i][j] = contains(columnList, c);
+                    if (measureColumnIndexes[i][j] < 0) {
+                        measureColumnIndexes[i][j] = columnIndex;
+                        columnList.add(new IntermediateColumnDesc(String.valueOf(columnIndex), c.getName(), c.getDatatype(), c.getTable()));
+                        columnIndex++;
+                    }
+                }
+            }
+        }
+    }
+
+    private int contains(List<IntermediateColumnDesc> columnList, TblColRef c) {
+        for (int i = 0; i < columnList.size(); i++) {
+            IntermediateColumnDesc col = columnList.get(i);
+            if (col.getColumnName().equals(c.getName()) && col.getTableName().equals(c.getTable()))
+                return i;
+        }
+        return -1;
+    }
+
+    public CubeDesc getCubeDesc() {
+        return cubeDesc;
+    }
+
+    public String getTableName(String jobUUID) {
+        return tableName + "_" + jobUUID.replace("-", "_");
+    }
+
+    public int[] getRowKeyColumnIndexes() {
+        return rowKeyColumnIndexes;
+    }
+
+    public int[][] getMeasureColumnIndexes() {
+        return measureColumnIndexes;
+    }
+
+    public static class IntermediateColumnDesc {
+        private String id;
+        private String columnName;
+        private String dataType;
+        private String tableName;
+
+        public IntermediateColumnDesc(String id, String columnName, String dataType, String tableName) {
+            this.id = id;
+            this.columnName = columnName;
+            this.dataType = dataType;
+            this.tableName = tableName;
+        }
+
+        public String getId() {
+            return id;
+        }
+
+        public String getColumnName() {
+            return columnName;
+        }
+
+        public String getDataType() {
+            return dataType;
+        }
+
+        public String getTableName() {
+            return tableName;
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/job/src/main/java/com/kylinolap/job/hadoop/hive/SqlHiveDataTypeMapping.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/com/kylinolap/job/hadoop/hive/SqlHiveDataTypeMapping.java b/job/src/main/java/com/kylinolap/job/hadoop/hive/SqlHiveDataTypeMapping.java
new file mode 100644
index 0000000..294ffae
--- /dev/null
+++ b/job/src/main/java/com/kylinolap/job/hadoop/hive/SqlHiveDataTypeMapping.java
@@ -0,0 +1,43 @@
+/*
+ * 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.job.hadoop.hive;
+
+import java.util.HashMap;
+import java.util.Map;
+
+/**
+ * @author George Song (ysong1)
+ * 
+ */
+public class SqlHiveDataTypeMapping {
+
+    private static final Map<String, String> sqlToHiveDataTypeMapping = new HashMap<String, String>();
+
+    static {
+        sqlToHiveDataTypeMapping.put("short", "smallint");
+        sqlToHiveDataTypeMapping.put("long", "bigint");
+        sqlToHiveDataTypeMapping.put("byte", "tinyint");
+        sqlToHiveDataTypeMapping.put("datetime", "date");
+    }
+
+    public static String getHiveDataType(String javaDataType) {
+        String hiveDataType = sqlToHiveDataTypeMapping.get(javaDataType.toLowerCase());
+        if (hiveDataType == null) {
+            hiveDataType = javaDataType;
+        }
+        return hiveDataType.toLowerCase();
+    }
+}


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

Posted by lu...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/cube/src/main/java/com/kylinolap/cube/invertedindex/TableRecordInfo.java
----------------------------------------------------------------------
diff --git a/cube/src/main/java/com/kylinolap/cube/invertedindex/TableRecordInfo.java b/cube/src/main/java/com/kylinolap/cube/invertedindex/TableRecordInfo.java
new file mode 100644
index 0000000..b831328
--- /dev/null
+++ b/cube/src/main/java/com/kylinolap/cube/invertedindex/TableRecordInfo.java
@@ -0,0 +1,170 @@
+/*
+ * 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.cube.invertedindex;
+
+import java.io.IOException;
+
+import org.apache.hadoop.io.LongWritable;
+
+import com.kylinolap.cube.CubeSegment;
+import com.kylinolap.cube.measure.fixedlen.FixedLenMeasureCodec;
+import com.kylinolap.dict.Dictionary;
+import com.kylinolap.dict.DictionaryManager;
+import com.kylinolap.metadata.model.cube.TblColRef;
+import com.kylinolap.metadata.model.invertedindex.InvertedIndexDesc;
+import com.kylinolap.metadata.model.schema.ColumnDesc;
+import com.kylinolap.metadata.model.schema.TableDesc;
+
+/**
+ * @author yangli9
+ *         <p/>
+ *         TableRecordInfo stores application-aware knowledges,
+ *         while TableRecordInfoDigest only stores byte level knowleges
+ */
+public class TableRecordInfo extends TableRecordInfoDigest {
+
+    final CubeSegment seg;
+    final InvertedIndexDesc desc;
+    final TableDesc tableDesc;
+
+    final String[] colNames;
+    final Dictionary<?>[] dictionaries;
+    final FixedLenMeasureCodec<?>[] measureSerializers;
+
+
+    public TableRecordInfo(CubeSegment cubeSeg) throws IOException {
+
+        seg = cubeSeg;
+        desc = seg.getCubeInstance().getInvertedIndexDesc();
+        tableDesc = desc.getFactTableDesc();
+
+        nColumns = tableDesc.getColumnCount();
+        colNames = new String[nColumns];
+        dictionaries = new Dictionary<?>[nColumns];
+        measureSerializers = new FixedLenMeasureCodec<?>[nColumns];
+
+        DictionaryManager dictMgr = DictionaryManager.getInstance(desc.getConfig());
+        for (ColumnDesc col : tableDesc.getColumns()) {
+            int i = col.getZeroBasedIndex();
+            colNames[i] = col.getName();
+            if (desc.isMetricsCol(i)) {
+                measureSerializers[i] = FixedLenMeasureCodec.get(col.getType());
+            } else {
+                String dictPath = seg.getDictResPath(new TblColRef(col));
+                dictionaries[i] = dictMgr.getDictionary(dictPath);
+            }
+        }
+
+        //isMetric
+        isMetric = new boolean[nColumns];
+        for (int i = 0; i < nColumns; ++i) {
+            isMetric[i] = desc.isMetricsCol(i);
+        }
+
+        //lengths
+        lengths = new int[nColumns];
+        for (int i = 0; i < nColumns; ++i) {
+            lengths[i] = isMetrics(i) ? measureSerializers[i].getLength() : dictionaries[i].getSizeOfId();
+        }
+
+        //dict max id
+        dictMaxIds = new int[nColumns];
+        for (int i = 0; i < nColumns; ++i) {
+            if (!isMetrics(i))
+                dictMaxIds[i] = dictionaries[i].getMaxId();
+        }
+
+        //offsets
+        int pos = 0;
+        offsets = new int[nColumns];
+        for (int i = 0; i < nColumns; i++) {
+            offsets[i] = pos;
+            pos += length(i);
+        }
+
+        byteFormLen = pos;
+    }
+
+    @Override
+    public TableRecordBytes createTableRecord() {
+        return new TableRecord(this);
+    }
+
+    public InvertedIndexDesc getDescriptor() {
+        return desc;
+    }
+
+    public ColumnDesc[] getColumns() {
+        return tableDesc.getColumns();
+    }
+
+
+    // dimensions go with dictionary
+    @SuppressWarnings("unchecked")
+    public Dictionary<String> dict(int col) {
+        // yes, all dictionaries are string based
+        return (Dictionary<String>) dictionaries[col];
+    }
+
+    // metrics go with fixed-len codec
+    @SuppressWarnings("unchecked")
+    public FixedLenMeasureCodec<LongWritable> codec(int col) {
+        // yes, all metrics are long currently
+        return (FixedLenMeasureCodec<LongWritable>) measureSerializers[col];
+    }
+
+
+    public int getTimestampColumn() {
+        return desc.getTimestampColumn();
+    }
+
+    /*
+     * (non-Javadoc)
+     * 
+     * @see java.lang.Object#hashCode()
+     */
+    @Override
+    public int hashCode() {
+        final int prime = 31;
+        int result = 1;
+        result = prime * result + ((seg == null) ? 0 : seg.hashCode());
+        return result;
+    }
+
+    /*
+     * (non-Javadoc)
+     * 
+     * @see java.lang.Object#equals(java.lang.Object)
+     */
+    @Override
+    public boolean equals(Object obj) {
+        if (this == obj)
+            return true;
+        if (obj == null)
+            return false;
+        if (getClass() != obj.getClass())
+            return false;
+        TableRecordInfo other = (TableRecordInfo) obj;
+        if (seg == null) {
+            if (other.seg != null)
+                return false;
+        } else if (!seg.equals(other.seg))
+            return false;
+        return true;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/cube/src/main/java/com/kylinolap/cube/invertedindex/TableRecordInfoDigest.java
----------------------------------------------------------------------
diff --git a/cube/src/main/java/com/kylinolap/cube/invertedindex/TableRecordInfoDigest.java b/cube/src/main/java/com/kylinolap/cube/invertedindex/TableRecordInfoDigest.java
new file mode 100644
index 0000000..2833183
--- /dev/null
+++ b/cube/src/main/java/com/kylinolap/cube/invertedindex/TableRecordInfoDigest.java
@@ -0,0 +1,89 @@
+package com.kylinolap.cube.invertedindex;
+
+
+import com.kylinolap.common.util.BytesSerializer;
+import com.kylinolap.common.util.BytesUtil;
+
+import java.nio.ByteBuffer;
+
+/**
+ * Created by honma on 11/10/14.
+ */
+public class TableRecordInfoDigest implements TableRecordFactory {
+
+    protected int nColumns;
+    protected int byteFormLen;
+
+    protected int[] offsets;//column offset in byte form row
+    protected int[] dictMaxIds;//max id for each of the dict
+    protected int[] lengths;//length of each encoded dict
+    protected boolean[] isMetric;//whether it's metric or dict
+
+    public boolean isMetrics(int col) {
+        return isMetric[col];
+    }
+
+    public int getColumnCount() {
+        return nColumns;
+    }
+
+    public int offset(int col) {
+        return offsets[col];
+    }
+
+    public int length(int col) {
+        return lengths[col];
+    }
+
+    public int getMaxID(int col) {
+        return dictMaxIds[col];
+    }
+
+    @Override
+    public TableRecordBytes createTableRecord() {
+        return new TableRecordBytes(this);
+    }
+
+    public static byte[] serialize(TableRecordInfoDigest o) {
+        ByteBuffer buf = ByteBuffer.allocate(Serializer.SERIALIZE_BUFFER_SIZE);
+        serializer.serialize(o, buf);
+        byte[] result = new byte[buf.position()];
+        System.arraycopy(buf.array(), 0, result, 0, buf.position());
+        return result;
+    }
+
+    public static TableRecordInfoDigest deserialize(byte[] bytes) {
+        return serializer.deserialize(ByteBuffer.wrap(bytes));
+    }
+
+    public static TableRecordInfoDigest deserialize(ByteBuffer buffer) {
+        return serializer.deserialize(buffer);
+    }
+
+    private static final Serializer serializer = new Serializer();
+
+    private static class Serializer implements BytesSerializer<TableRecordInfoDigest> {
+
+        @Override
+        public void serialize(TableRecordInfoDigest value, ByteBuffer out) {
+            BytesUtil.writeVInt(value.nColumns, out);
+            BytesUtil.writeVInt(value.byteFormLen, out);
+            BytesUtil.writeIntArray(value.offsets, out);
+            BytesUtil.writeIntArray(value.dictMaxIds, out);
+            BytesUtil.writeIntArray(value.lengths, out);
+        }
+
+        @Override
+        public TableRecordInfoDigest deserialize(ByteBuffer in) {
+            TableRecordInfoDigest result = new TableRecordInfoDigest();
+            result.nColumns = BytesUtil.readVInt(in);
+            result.byteFormLen = BytesUtil.readVInt(in);
+            result.offsets = BytesUtil.readIntArray(in);
+            result.dictMaxIds = BytesUtil.readIntArray(in);
+            result.lengths = BytesUtil.readIntArray(in);
+            return result;
+        }
+
+
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/cube/src/main/java/com/kylinolap/cube/kv/AbstractRowKeyEncoder.java
----------------------------------------------------------------------
diff --git a/cube/src/main/java/com/kylinolap/cube/kv/AbstractRowKeyEncoder.java b/cube/src/main/java/com/kylinolap/cube/kv/AbstractRowKeyEncoder.java
new file mode 100644
index 0000000..e2a367c
--- /dev/null
+++ b/cube/src/main/java/com/kylinolap/cube/kv/AbstractRowKeyEncoder.java
@@ -0,0 +1,75 @@
+/*
+ * 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.cube.kv;
+
+import java.util.HashMap;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.kylinolap.cube.CubeSegment;
+import com.kylinolap.cube.cuboid.Cuboid;
+import com.kylinolap.dict.Dictionary;
+import com.kylinolap.metadata.model.cube.TblColRef;
+
+/**
+ * 
+ * @author xjiang
+ * 
+ */
+public abstract class AbstractRowKeyEncoder {
+
+    public static final byte DEFAULT_BLANK_BYTE = Dictionary.NULL;
+
+    protected static final Logger logger = LoggerFactory.getLogger(AbstractRowKeyEncoder.class);
+
+    private static final Map<String, Map<Long, AbstractRowKeyEncoder>> ENCODER_CACHE = new ConcurrentHashMap<String, Map<Long, AbstractRowKeyEncoder>>();
+
+    public static AbstractRowKeyEncoder createInstance(CubeSegment cubeSeg, Cuboid cuboid) {
+
+        // The storage location identifier is unique for every segment
+        Map<Long, AbstractRowKeyEncoder> cubeCache = ENCODER_CACHE.get(cubeSeg.getStorageLocationIdentifier());
+
+        if (cubeCache == null) {
+            cubeCache = new HashMap<Long, AbstractRowKeyEncoder>();
+            ENCODER_CACHE.put(cuboid.getCube().getName(), cubeCache);
+        }
+
+        AbstractRowKeyEncoder encoder = cubeCache.get(cuboid.getId());
+        if (encoder == null) {
+            encoder = new RowKeyEncoder(cubeSeg, cuboid);
+            cubeCache.put(cuboid.getId(), encoder);
+        }
+        return encoder;
+    }
+
+    protected final Cuboid cuboid;
+    protected byte blankByte = DEFAULT_BLANK_BYTE;
+
+    protected AbstractRowKeyEncoder(Cuboid cuboid) {
+        this.cuboid = cuboid;
+    }
+
+    public void setBlankByte(byte blankByte) {
+        this.blankByte = blankByte;
+    }
+
+    abstract public byte[] encode(Map<TblColRef, String> valueMap);
+
+    abstract public byte[] encode(byte[][] values);
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/cube/src/main/java/com/kylinolap/cube/kv/FuzzyKeyEncoder.java
----------------------------------------------------------------------
diff --git a/cube/src/main/java/com/kylinolap/cube/kv/FuzzyKeyEncoder.java b/cube/src/main/java/com/kylinolap/cube/kv/FuzzyKeyEncoder.java
new file mode 100644
index 0000000..172374e
--- /dev/null
+++ b/cube/src/main/java/com/kylinolap/cube/kv/FuzzyKeyEncoder.java
@@ -0,0 +1,40 @@
+/*
+ * 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.cube.kv;
+
+import java.util.Arrays;
+
+import com.kylinolap.cube.CubeSegment;
+import com.kylinolap.cube.cuboid.Cuboid;
+
+/**
+ * 
+ * @author xjiang
+ * 
+ */
+public class FuzzyKeyEncoder extends RowKeyEncoder {
+
+    public FuzzyKeyEncoder(CubeSegment seg, Cuboid cuboid) {
+        super(seg, cuboid);
+    }
+
+    @Override
+    protected byte[] defaultValue(int length) {
+        byte[] keyBytes = new byte[length];
+        Arrays.fill(keyBytes, RowConstants.FUZZY_MASK_ZERO);
+        return keyBytes;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/cube/src/main/java/com/kylinolap/cube/kv/FuzzyMaskEncoder.java
----------------------------------------------------------------------
diff --git a/cube/src/main/java/com/kylinolap/cube/kv/FuzzyMaskEncoder.java b/cube/src/main/java/com/kylinolap/cube/kv/FuzzyMaskEncoder.java
new file mode 100644
index 0000000..e42ba1e
--- /dev/null
+++ b/cube/src/main/java/com/kylinolap/cube/kv/FuzzyMaskEncoder.java
@@ -0,0 +1,52 @@
+/*
+ * 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.cube.kv;
+
+import java.util.Arrays;
+
+import com.kylinolap.cube.CubeSegment;
+import com.kylinolap.cube.cuboid.Cuboid;
+import com.kylinolap.metadata.model.cube.TblColRef;
+
+/**
+ * 
+ * @author xjiang
+ * 
+ */
+public class FuzzyMaskEncoder extends RowKeyEncoder {
+
+    public FuzzyMaskEncoder(CubeSegment seg, Cuboid cuboid) {
+        super(seg, cuboid);
+    }
+
+    @Override
+    protected int fillHeader(byte[] bytes, byte[][] values) {
+        // always fuzzy match cuboid ID to lock on the selected cuboid
+        int cuboidStart = this.headerLength - RowConstants.ROWKEY_CUBOIDID_LEN;
+        Arrays.fill(bytes, 0, cuboidStart, RowConstants.FUZZY_MASK_ONE);
+        Arrays.fill(bytes, cuboidStart, this.headerLength, RowConstants.FUZZY_MASK_ZERO);
+        return this.headerLength;
+    }
+
+    @Override
+    protected void fillColumnValue(TblColRef column, int columnLen, byte[] value, int valueLen, byte[] outputValue, int outputValueOffset) {
+        if (value == null) {
+            Arrays.fill(outputValue, outputValueOffset, outputValueOffset + columnLen, RowConstants.FUZZY_MASK_ONE);
+        } else {
+            Arrays.fill(outputValue, outputValueOffset, outputValueOffset + columnLen, RowConstants.FUZZY_MASK_ZERO);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/cube/src/main/java/com/kylinolap/cube/kv/RowConstants.java
----------------------------------------------------------------------
diff --git a/cube/src/main/java/com/kylinolap/cube/kv/RowConstants.java b/cube/src/main/java/com/kylinolap/cube/kv/RowConstants.java
new file mode 100644
index 0000000..fc196fe
--- /dev/null
+++ b/cube/src/main/java/com/kylinolap/cube/kv/RowConstants.java
@@ -0,0 +1,48 @@
+/*
+ * 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.cube.kv;
+
+/**
+ * 
+ * @author xjiang
+ * 
+ */
+public class RowConstants {
+
+    // row key fixed length place holder
+    public static final byte ROWKEY_PLACE_HOLDER_BYTE = 9;
+    // row key lower bound
+    public static final byte ROWKEY_LOWER_BYTE = 0;
+    // row key upper bound
+    public static final byte ROWKEY_UPPER_BYTE = (byte) 0xff;
+    // row key cuboid id length
+    public static final int ROWKEY_CUBOIDID_LEN = 8;
+
+    // fuzzy mask
+    public static final byte FUZZY_MASK_ZERO = 0;
+    public static final byte FUZZY_MASK_ONE = 1;
+
+    // row value delimiter
+    public static final byte ROWVALUE_DELIMITER_BYTE = 7;
+    public static final String ROWVALUE_DELIMITER_STRING = String.valueOf((char) 7);
+    public static final byte[] ROWVALUE_DELIMITER_BYTES = { 7 };
+
+    public static final int ROWVALUE_BUFFER_SIZE = 1024 * 1024; // 1 MB
+
+    // marker class
+    public static final byte[][] BYTE_ARR_MARKER = new byte[0][];
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/cube/src/main/java/com/kylinolap/cube/kv/RowKeyColumnIO.java
----------------------------------------------------------------------
diff --git a/cube/src/main/java/com/kylinolap/cube/kv/RowKeyColumnIO.java b/cube/src/main/java/com/kylinolap/cube/kv/RowKeyColumnIO.java
new file mode 100644
index 0000000..d064769
--- /dev/null
+++ b/cube/src/main/java/com/kylinolap/cube/kv/RowKeyColumnIO.java
@@ -0,0 +1,165 @@
+/*
+ * 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.cube.kv;
+
+import java.util.Arrays;
+
+import org.apache.hadoop.hbase.util.Bytes;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.kylinolap.common.util.BytesUtil;
+import com.kylinolap.cube.CubeManager;
+import com.kylinolap.cube.CubeSegment;
+import com.kylinolap.dict.Dictionary;
+import com.kylinolap.metadata.model.cube.RowKeyDesc;
+import com.kylinolap.metadata.model.cube.TblColRef;
+
+/**
+ * Read/Write column values from/into bytes
+ * 
+ * @author yangli9
+ */
+@SuppressWarnings("unchecked")
+public class RowKeyColumnIO {
+
+    private static final Logger logger = LoggerFactory.getLogger(RowKeyColumnIO.class);
+
+    private CubeSegment seg;
+    private RowKeyDesc rowkeyDesc;
+    private boolean forceNoDict = Boolean.getBoolean("forceNoDict");
+
+    public RowKeyColumnIO(CubeSegment cubeSeg) {
+        this.seg = cubeSeg;
+        this.rowkeyDesc = seg.getCubeDesc().getRowkey();
+    }
+
+    public CubeSegment getCubeSegment() {
+        return seg;
+    }
+
+    public int getColumnLength(TblColRef col) {
+        Dictionary<String> dict = getDictionary(col);
+        if (dict == null) {
+            return rowkeyDesc.getColumnLength(col);
+        } else {
+            return dict.getSizeOfId();
+        }
+    }
+
+    public void writeColumn(TblColRef column, byte[] value, int valueLen, byte dft, byte[] output, int outputOffset) {
+        writeColumn(column, value, valueLen, 0, dft, output, outputOffset);
+    }
+
+    public void writeColumn(TblColRef column, byte[] value, int valueLen, int roundingFlag, byte dft, byte[] output, int outputOffset) {
+
+        Dictionary<String> dict = getDictionary(column);
+        int columnLen = getColumnLength(column);
+
+        // non-dict value
+        if (dict == null) {
+            byte[] valueBytes = padFixLen(columnLen, value);
+            System.arraycopy(valueBytes, 0, output, outputOffset, columnLen);
+            return;
+        }
+
+        // dict value
+        try {
+            int id = dict.getIdFromValueBytes(value, 0, valueLen, roundingFlag);
+            BytesUtil.writeUnsigned(id, output, outputOffset, dict.getSizeOfId());
+        } catch (IllegalArgumentException ex) {
+            for (int i = outputOffset; i < outputOffset + columnLen; i++)
+                output[i] = dft;
+            logger.error("Can't translate value " + Bytes.toString(value, 0, valueLen) + " to dictionary ID, roundingFlag " + roundingFlag + ". Using default value " + String.format("\\x%02X", dft));
+        }
+    }
+
+    private byte[] padFixLen(int length, byte[] valueBytes) {
+        int valLen = valueBytes.length;
+        if (valLen == length) {
+            return valueBytes;
+        } else if (valLen < length) {
+            byte[] newValueBytes = new byte[length];
+            System.arraycopy(valueBytes, 0, newValueBytes, 0, valLen);
+            Arrays.fill(newValueBytes, valLen, length, RowConstants.ROWKEY_PLACE_HOLDER_BYTE);
+            return newValueBytes;
+        } else {
+            return Arrays.copyOf(valueBytes, length);
+        }
+    }
+
+    public String readColumnString(TblColRef col, byte[] bytes, int bytesLen) {
+        Dictionary<String> dict = getDictionary(col);
+        if (dict == null) {
+            bytes = Bytes.head(bytes, bytesLen);
+            if (isNull(bytes)) {
+                return null;
+            }
+            bytes = removeFixLenPad(bytes, 0);
+            return Bytes.toString(bytes);
+        } else {
+            int id = BytesUtil.readUnsigned(bytes, 0, bytesLen);
+            try {
+                String value = dict.getValueFromId(id);
+                return value;
+            } catch (IllegalArgumentException e) {
+                logger.error("Can't get dictionary value for column " + col.getName() + " (id = " + id + ")");
+                return "";
+            }
+        }
+    }
+
+    private boolean isNull(byte[] bytes) {
+        // all 0xFF is NULL
+        if (bytes.length == 0)
+            return false;
+        for (int i = 0; i < bytes.length; i++) {
+            if (bytes[i] != AbstractRowKeyEncoder.DEFAULT_BLANK_BYTE)
+                return false;
+        }
+        return true;
+    }
+
+    private byte[] removeFixLenPad(byte[] bytes, int offset) {
+        int padCount = 0;
+        for (int i = offset; i < bytes.length; i++) {
+            byte vb = bytes[i];
+            if (vb == RowConstants.ROWKEY_PLACE_HOLDER_BYTE) {
+                padCount++;
+            }
+        }
+
+        int size = bytes.length - offset - padCount;
+        byte[] stripBytes = new byte[size];
+        int index = 0;
+        for (int i = offset; i < bytes.length; i++) {
+            byte vb = bytes[i];
+            if (vb != RowConstants.ROWKEY_PLACE_HOLDER_BYTE) {
+                stripBytes[index++] = vb;
+            }
+        }
+        return stripBytes;
+    }
+
+    public Dictionary<String> getDictionary(TblColRef col) {
+        if (forceNoDict)
+            return null;
+
+        return (Dictionary<String>) CubeManager.getInstance(seg.getCubeInstance().getConfig()).getDictionary(seg, col);
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/cube/src/main/java/com/kylinolap/cube/kv/RowKeyColumnOrder.java
----------------------------------------------------------------------
diff --git a/cube/src/main/java/com/kylinolap/cube/kv/RowKeyColumnOrder.java b/cube/src/main/java/com/kylinolap/cube/kv/RowKeyColumnOrder.java
new file mode 100644
index 0000000..db598c8
--- /dev/null
+++ b/cube/src/main/java/com/kylinolap/cube/kv/RowKeyColumnOrder.java
@@ -0,0 +1,106 @@
+/*
+ * 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.cube.kv;
+
+import java.util.Collection;
+import java.util.Comparator;
+
+import com.kylinolap.metadata.model.schema.DataType;
+
+/**
+ * @author yangli9
+ */
+abstract public class RowKeyColumnOrder implements Comparator<String> {
+
+    public static final NumberOrder NUMBER_ORDER = new NumberOrder();
+    public static final StringOrder STRING_ORDER = new StringOrder();
+
+    public static RowKeyColumnOrder getInstance(DataType type) {
+        if (type.isNumberFamily())
+            return NUMBER_ORDER;
+        else
+            return STRING_ORDER;
+    }
+
+    public String max(Collection<String> values) {
+        String max = null;
+        for (String v : values) {
+            if (max == null || compare(max, v) < 0)
+                max = v;
+        }
+        return max;
+    }
+
+    public String min(Collection<String> values) {
+        String min = null;
+        for (String v : values) {
+            if (min == null || compare(min, v) > 0)
+                min = v;
+        }
+        return min;
+    }
+
+    public String min(String v1, String v2) {
+        if (v1 == null)
+            return v2;
+        else if (v2 == null)
+            return v1;
+        else
+            return compare(v1, v2) <= 0 ? v1 : v2;
+    }
+
+    public String max(String v1, String v2) {
+        if (v1 == null)
+            return v2;
+        else if (v2 == null)
+            return v1;
+        else
+            return compare(v1, v2) >= 0 ? v1 : v2;
+    }
+
+    @Override
+    public int compare(String o1, String o2) {
+        // consider null
+        if (o1 == o2)
+            return 0;
+        if (o1 == null)
+            return -1;
+        if (o2 == null)
+            return 1;
+
+        return compareNonNull(o1, o2);
+    }
+
+    abstract int compareNonNull(String o1, String o2);
+
+    private static class StringOrder extends RowKeyColumnOrder {
+        @Override
+        public int compareNonNull(String o1, String o2) {
+            return o1.compareTo(o2);
+        }
+    }
+
+    private static class NumberOrder extends RowKeyColumnOrder {
+        @Override
+        public int compareNonNull(String o1, String o2) {
+            double d1 = Double.parseDouble(o1);
+            double d2 = Double.parseDouble(o2);
+            return Double.compare(d1, d2);
+        }
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/cube/src/main/java/com/kylinolap/cube/kv/RowKeyDecoder.java
----------------------------------------------------------------------
diff --git a/cube/src/main/java/com/kylinolap/cube/kv/RowKeyDecoder.java b/cube/src/main/java/com/kylinolap/cube/kv/RowKeyDecoder.java
new file mode 100644
index 0000000..527f546
--- /dev/null
+++ b/cube/src/main/java/com/kylinolap/cube/kv/RowKeyDecoder.java
@@ -0,0 +1,137 @@
+/*
+ * 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.cube.kv;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+
+import com.kylinolap.cube.CubeSegment;
+import com.kylinolap.cube.common.RowKeySplitter;
+import com.kylinolap.cube.common.SplittedBytes;
+import com.kylinolap.cube.cuboid.Cuboid;
+import com.kylinolap.metadata.model.cube.CubeDesc;
+import com.kylinolap.metadata.model.cube.TblColRef;
+
+/**
+ * 
+ * @author xjiang
+ * 
+ */
+public class RowKeyDecoder {
+
+    private final CubeDesc cubeDesc;
+    private final RowKeyColumnIO colIO;
+    private final RowKeySplitter rowKeySplitter;
+
+    private Cuboid cuboid;
+    private List<String> names;
+    private List<String> values;
+
+    public RowKeyDecoder(CubeSegment cubeSegment) {
+        this.cubeDesc = cubeSegment.getCubeDesc();
+        this.rowKeySplitter = new RowKeySplitter(cubeSegment, 65, 255);
+        this.colIO = new RowKeyColumnIO(cubeSegment);
+        this.values = new ArrayList<String>();
+    }
+
+    public long decode(byte[] bytes) throws IOException {
+        this.values.clear();
+
+        long cuboidId = rowKeySplitter.split(bytes, bytes.length);
+        initCuboid(cuboidId);
+
+        SplittedBytes[] splits = rowKeySplitter.getSplitBuffers();
+
+        int offset = 1; // skip cuboid id part
+
+        for (int i = 0; i < this.cuboid.getColumns().size(); i++) {
+            TblColRef col = this.cuboid.getColumns().get(i);
+            collectValue(col, splits[offset].value, splits[offset].length);
+            offset++;
+        }
+
+        return cuboidId;
+    }
+
+    private void initCuboid(long cuboidID) {
+        if (this.cuboid != null && this.cuboid.getId() == cuboidID) {
+            return;
+        }
+        this.cuboid = Cuboid.findById(cubeDesc, cuboidID);
+    }
+
+    private void collectValue(TblColRef col, byte[] valueBytes, int length) throws IOException {
+        String strValue = colIO.readColumnString(col, valueBytes, length);
+        values.add(strValue);
+    }
+
+    public RowKeySplitter getRowKeySplitter() {
+        return rowKeySplitter;
+    }
+
+    public void setCuboid(Cuboid cuboid) {
+        this.cuboid = cuboid;
+        this.names = null;
+    }
+
+    public List<String> getNames(Map<TblColRef, String> aliasMap) {
+        if (names == null) {
+            names = buildNameList(aliasMap);
+        }
+        return names;
+    }
+
+    private List<String> buildNameList(Map<TblColRef, String> aliasMap) {
+        List<TblColRef> columnList = getColumns();
+        List<String> result = new ArrayList<String>(columnList.size());
+        for (TblColRef col : columnList)
+            result.add(findName(col, aliasMap));
+        return result;
+    }
+
+    private String findName(TblColRef column, Map<TblColRef, String> aliasMap) {
+        String name = null;
+        if (aliasMap != null) {
+            name = aliasMap.get(column);
+        }
+        if (name == null) {
+            name = column.getName();
+        }
+        return name;
+    }
+
+    public List<TblColRef> getColumns() {
+        return cuboid.getColumns();
+    }
+
+    public List<String> getValues() {
+        return values;
+    }
+
+    @Override
+    public String toString() {
+        StringBuilder buf = new StringBuilder();
+        buf.append(cuboid.getId());
+        for (Object value : values) {
+            buf.append(",");
+            buf.append(value);
+        }
+        return buf.toString();
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/cube/src/main/java/com/kylinolap/cube/kv/RowKeyEncoder.java
----------------------------------------------------------------------
diff --git a/cube/src/main/java/com/kylinolap/cube/kv/RowKeyEncoder.java b/cube/src/main/java/com/kylinolap/cube/kv/RowKeyEncoder.java
new file mode 100644
index 0000000..6485d9f
--- /dev/null
+++ b/cube/src/main/java/com/kylinolap/cube/kv/RowKeyEncoder.java
@@ -0,0 +1,140 @@
+/*
+ * 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.cube.kv;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.hadoop.hbase.util.Bytes;
+
+import com.kylinolap.cube.CubeSegment;
+import com.kylinolap.cube.cuboid.Cuboid;
+import com.kylinolap.metadata.model.cube.TblColRef;
+
+/**
+ * @author George Song (ysong1)
+ */
+public class RowKeyEncoder extends AbstractRowKeyEncoder {
+
+    private int bytesLength;
+    protected int headerLength;
+    private RowKeyColumnIO colIO;
+
+    protected RowKeyEncoder(CubeSegment cubeSeg, Cuboid cuboid) {
+        super(cuboid);
+        colIO = new RowKeyColumnIO(cubeSeg);
+        bytesLength = headerLength = RowConstants.ROWKEY_CUBOIDID_LEN; // header
+        for (TblColRef column : cuboid.getColumns()) {
+            bytesLength += colIO.getColumnLength(column);
+        }
+    }
+
+    public RowKeyColumnIO getColumnIO() {
+        return colIO;
+    }
+
+    public int getColumnOffset(TblColRef col) {
+        int offset = RowConstants.ROWKEY_CUBOIDID_LEN;
+
+        for (TblColRef dimCol : cuboid.getColumns()) {
+            if (col.equals(dimCol))
+                return offset;
+            offset += colIO.getColumnLength(dimCol);
+        }
+
+        throw new IllegalArgumentException("Column " + col + " not found on cuboid " + cuboid);
+    }
+
+    public int getColumnLength(TblColRef col) {
+        return colIO.getColumnLength(col);
+    }
+
+    public int getRowKeyLength() {
+        return bytesLength;
+    }
+
+    public int getHeaderLength() {
+        return headerLength;
+    }
+
+    @Override
+    public byte[] encode(Map<TblColRef, String> valueMap) {
+        List<byte[]> valueList = new ArrayList<byte[]>();
+        for (TblColRef bdCol : cuboid.getColumns()) {
+            String value = valueMap.get(bdCol);
+            valueList.add(valueStringToBytes(value));
+        }
+        byte[][] values = valueList.toArray(RowConstants.BYTE_ARR_MARKER);
+        return encode(values);
+    }
+
+    public byte[] valueStringToBytes(String value) {
+        if (value == null)
+            return null;
+        else
+            return Bytes.toBytes(value);
+    }
+
+    @Override
+    public byte[] encode(byte[][] values) {
+        byte[] bytes = new byte[this.bytesLength];
+        int offset = fillHeader(bytes, values);
+
+        for (int i = 0; i < cuboid.getColumns().size(); i++) {
+            TblColRef column = cuboid.getColumns().get(i);
+            int colLength = colIO.getColumnLength(column);
+            byte[] value = values[i];
+            if (value == null) {
+                fillColumnValue(column, colLength, null, 0, bytes, offset);
+            } else {
+                fillColumnValue(column, colLength, value, value.length, bytes, offset);
+            }
+            offset += colLength;
+
+        }
+        return bytes;
+    }
+
+    protected int fillHeader(byte[] bytes, byte[][] values) {
+        int offset = 0;
+        System.arraycopy(cuboid.getBytes(), 0, bytes, offset, RowConstants.ROWKEY_CUBOIDID_LEN);
+        offset += RowConstants.ROWKEY_CUBOIDID_LEN;
+        if (this.headerLength != offset) {
+            throw new IllegalStateException("Expected header length is " + headerLength + ". But the offset is " + offset);
+        }
+        return offset;
+    }
+
+    protected void fillColumnValue(TblColRef column, int columnLen, byte[] value, int valueLen, byte[] outputValue, int outputValueOffset) {
+        // special null value case
+        if (value == null) {
+            byte[] valueBytes = defaultValue(columnLen);
+            System.arraycopy(valueBytes, 0, outputValue, outputValueOffset, columnLen);
+            return;
+        }
+
+        colIO.writeColumn(column, value, valueLen, this.blankByte, outputValue, outputValueOffset);
+    }
+
+    protected byte[] defaultValue(int length) {
+        byte[] values = new byte[length];
+        Arrays.fill(values, this.blankByte);
+        return values;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/cube/src/main/java/com/kylinolap/cube/kv/RowValueDecoder.java
----------------------------------------------------------------------
diff --git a/cube/src/main/java/com/kylinolap/cube/kv/RowValueDecoder.java b/cube/src/main/java/com/kylinolap/cube/kv/RowValueDecoder.java
new file mode 100644
index 0000000..c36bc81
--- /dev/null
+++ b/cube/src/main/java/com/kylinolap/cube/kv/RowValueDecoder.java
@@ -0,0 +1,136 @@
+/*
+ * 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.cube.kv;
+
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.BitSet;
+import java.util.Collection;
+import java.util.List;
+
+import org.apache.hadoop.io.DoubleWritable;
+import org.apache.hadoop.io.FloatWritable;
+import org.apache.hadoop.io.IntWritable;
+import org.apache.hadoop.io.LongWritable;
+
+import com.kylinolap.cube.measure.MeasureCodec;
+import com.kylinolap.metadata.model.cube.FunctionDesc;
+import com.kylinolap.metadata.model.cube.HBaseColumnDesc;
+import com.kylinolap.metadata.model.cube.MeasureDesc;
+
+/**
+ * 
+ * @author xjiang
+ * 
+ */
+public class RowValueDecoder implements Cloneable {
+
+    private final HBaseColumnDesc hbaseColumn;
+    private final MeasureCodec codec;
+    private final BitSet projectionIndex;
+    private final MeasureDesc[] measures;
+    private final List<String> names;
+    private Object[] values;
+
+    public RowValueDecoder(RowValueDecoder rowValueDecoder) {
+        this.hbaseColumn = rowValueDecoder.getHBaseColumn();
+        this.projectionIndex = rowValueDecoder.getProjectionIndex();
+        this.names = new ArrayList<String>();
+        this.measures = hbaseColumn.getMeasures();
+        for (MeasureDesc measure : measures) {
+            this.names.add(measure.getFunction().getRewriteFieldName());
+        }
+        this.codec = new MeasureCodec(measures);
+        this.values = new Object[measures.length];
+    }
+
+    public RowValueDecoder(HBaseColumnDesc hbaseColumn) {
+        this.hbaseColumn = hbaseColumn;
+        this.projectionIndex = new BitSet();
+        this.names = new ArrayList<String>();
+        this.measures = hbaseColumn.getMeasures();
+        for (MeasureDesc measure : measures) {
+            this.names.add(measure.getFunction().getRewriteFieldName());
+        }
+        this.codec = new MeasureCodec(measures);
+        this.values = new Object[measures.length];
+    }
+
+    public void decode(byte[] bytes) {
+        codec.decode(ByteBuffer.wrap(bytes), values);
+        convertToJavaObjects(values, values);
+    }
+
+    private void convertToJavaObjects(Object[] mapredObjs, Object[] results) {
+        for (int i = 0; i < mapredObjs.length; i++) {
+            Object o = mapredObjs[i];
+
+            if (o instanceof LongWritable)
+                o = Long.valueOf(((LongWritable) o).get());
+            else if (o instanceof IntWritable)
+                o = Integer.valueOf(((IntWritable) o).get());
+            else if (o instanceof DoubleWritable)
+                o = Double.valueOf(((DoubleWritable) o).get());
+            else if (o instanceof FloatWritable)
+                o = Float.valueOf(((FloatWritable) o).get());
+
+            results[i] = o;
+        }
+    }
+
+    public void setIndex(int bitIndex) {
+        projectionIndex.set(bitIndex);
+    }
+
+    public HBaseColumnDesc getHBaseColumn() {
+        return hbaseColumn;
+    }
+
+    public BitSet getProjectionIndex() {
+        return projectionIndex;
+    }
+
+    public Object[] getValues() {
+        return values;
+    }
+
+    public List<String> getNames() {
+        return names;
+    }
+
+    public MeasureDesc[] getMeasures() {
+        return measures;
+    }
+
+    public boolean hasMemHungryCountDistinct() {
+        for (int i = projectionIndex.nextSetBit(0); i >= 0; i = projectionIndex.nextSetBit(i + 1)) {
+            FunctionDesc func = measures[i].getFunction();
+            if (func.isCountDistinct() && !func.isHolisticCountDistinct()) {
+                return true;
+            }
+        }
+        return false;
+    }
+
+    public static boolean hasMemHungryCountDistinct(Collection<RowValueDecoder> rowValueDecoders) {
+        for (RowValueDecoder decoder : rowValueDecoders) {
+            if (decoder.hasMemHungryCountDistinct())
+                return true;
+        }
+        return false;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/cube/src/main/java/com/kylinolap/cube/measure/BigDecimalMaxAggregator.java
----------------------------------------------------------------------
diff --git a/cube/src/main/java/com/kylinolap/cube/measure/BigDecimalMaxAggregator.java b/cube/src/main/java/com/kylinolap/cube/measure/BigDecimalMaxAggregator.java
new file mode 100644
index 0000000..4316ac0
--- /dev/null
+++ b/cube/src/main/java/com/kylinolap/cube/measure/BigDecimalMaxAggregator.java
@@ -0,0 +1,51 @@
+/*
+ * 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.cube.measure;
+
+import java.math.BigDecimal;
+
+/**
+ * @author yangli9
+ * 
+ */
+public class BigDecimalMaxAggregator extends MeasureAggregator<BigDecimal> {
+
+    BigDecimal max = null;
+
+    @Override
+    public void reset() {
+        max = null;
+    }
+
+    @Override
+    public void aggregate(BigDecimal value) {
+        if (max == null)
+            max = value;
+        else if (max.compareTo(value) < 0)
+            max = value;
+    }
+
+    @Override
+    public BigDecimal getState() {
+        return max;
+    }
+
+    @Override
+    public int getMemBytes() {
+        return guessBigDecimalMemBytes();
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/cube/src/main/java/com/kylinolap/cube/measure/BigDecimalMinAggregator.java
----------------------------------------------------------------------
diff --git a/cube/src/main/java/com/kylinolap/cube/measure/BigDecimalMinAggregator.java b/cube/src/main/java/com/kylinolap/cube/measure/BigDecimalMinAggregator.java
new file mode 100644
index 0000000..52dbcd7
--- /dev/null
+++ b/cube/src/main/java/com/kylinolap/cube/measure/BigDecimalMinAggregator.java
@@ -0,0 +1,52 @@
+/*
+ * 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.cube.measure;
+
+import java.math.BigDecimal;
+
+/**
+ * @author yangli9
+ * 
+ */
+public class BigDecimalMinAggregator extends MeasureAggregator<BigDecimal> {
+
+    BigDecimal max = null;
+
+    @Override
+    public void reset() {
+        max = null;
+    }
+
+    @Override
+    public void aggregate(BigDecimal value) {
+        if (max == null)
+            max = value;
+        else if (max.compareTo(value) > 0)
+            max = value;
+    }
+
+    @Override
+    public BigDecimal getState() {
+        return max;
+    }
+
+    @Override
+    public int getMemBytes() {
+        return guessBigDecimalMemBytes();
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/cube/src/main/java/com/kylinolap/cube/measure/BigDecimalSerializer.java
----------------------------------------------------------------------
diff --git a/cube/src/main/java/com/kylinolap/cube/measure/BigDecimalSerializer.java b/cube/src/main/java/com/kylinolap/cube/measure/BigDecimalSerializer.java
new file mode 100644
index 0000000..5d893c4
--- /dev/null
+++ b/cube/src/main/java/com/kylinolap/cube/measure/BigDecimalSerializer.java
@@ -0,0 +1,61 @@
+/*
+ * 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.cube.measure;
+
+import java.math.BigDecimal;
+import java.math.BigInteger;
+import java.nio.ByteBuffer;
+
+import org.apache.hadoop.hbase.util.Bytes;
+
+import com.kylinolap.common.util.BytesUtil;
+
+/**
+ * @author yangli9
+ * 
+ */
+public class BigDecimalSerializer extends MeasureSerializer<BigDecimal> {
+
+    @Override
+    public void serialize(BigDecimal value, ByteBuffer out) {
+        byte[] bytes = value.unscaledValue().toByteArray();
+
+        BytesUtil.writeVInt(value.scale(), out);
+        BytesUtil.writeVInt(bytes.length, out);
+        out.put(bytes);
+    }
+
+    @Override
+    public BigDecimal deserialize(ByteBuffer in) {
+        int scale = BytesUtil.readVInt(in);
+        int n = BytesUtil.readVInt(in);
+
+        byte[] bytes = new byte[n];
+        in.get(bytes);
+
+        return new BigDecimal(new BigInteger(bytes), scale);
+    }
+
+    @Override
+    public BigDecimal valueOf(byte[] value) {
+        if (value == null)
+            return new BigDecimal(0);
+        else
+            return new BigDecimal(Bytes.toString(value));
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/cube/src/main/java/com/kylinolap/cube/measure/BigDecimalSumAggregator.java
----------------------------------------------------------------------
diff --git a/cube/src/main/java/com/kylinolap/cube/measure/BigDecimalSumAggregator.java b/cube/src/main/java/com/kylinolap/cube/measure/BigDecimalSumAggregator.java
new file mode 100644
index 0000000..20bd093
--- /dev/null
+++ b/cube/src/main/java/com/kylinolap/cube/measure/BigDecimalSumAggregator.java
@@ -0,0 +1,48 @@
+/*
+ * 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.cube.measure;
+
+import java.math.BigDecimal;
+
+/**
+ * @author yangli9
+ * 
+ */
+public class BigDecimalSumAggregator extends MeasureAggregator<BigDecimal> {
+
+    BigDecimal sum = new BigDecimal(0);
+
+    @Override
+    public void reset() {
+        sum = new BigDecimal(0);
+    }
+
+    @Override
+    public void aggregate(BigDecimal value) {
+        sum = sum.add(value);
+    }
+
+    @Override
+    public BigDecimal getState() {
+        return sum;
+    }
+
+    @Override
+    public int getMemBytes() {
+        return guessBigDecimalMemBytes();
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/cube/src/main/java/com/kylinolap/cube/measure/DoubleMaxAggregator.java
----------------------------------------------------------------------
diff --git a/cube/src/main/java/com/kylinolap/cube/measure/DoubleMaxAggregator.java b/cube/src/main/java/com/kylinolap/cube/measure/DoubleMaxAggregator.java
new file mode 100644
index 0000000..8ed20c8
--- /dev/null
+++ b/cube/src/main/java/com/kylinolap/cube/measure/DoubleMaxAggregator.java
@@ -0,0 +1,52 @@
+/*
+ * 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.cube.measure;
+
+import org.apache.hadoop.io.DoubleWritable;
+
+/**
+ * @author yangli9
+ * 
+ */
+public class DoubleMaxAggregator extends MeasureAggregator<DoubleWritable> {
+
+    DoubleWritable max = null;
+
+    @Override
+    public void reset() {
+        max = null;
+    }
+
+    @Override
+    public void aggregate(DoubleWritable value) {
+        if (max == null)
+            max = new DoubleWritable(value.get());
+        else if (max.get() < value.get())
+            max.set(value.get());
+    }
+
+    @Override
+    public DoubleWritable getState() {
+        return max;
+    }
+
+    @Override
+    public int getMemBytes() {
+        return guessDoubleMemBytes();
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/cube/src/main/java/com/kylinolap/cube/measure/DoubleMinAggregator.java
----------------------------------------------------------------------
diff --git a/cube/src/main/java/com/kylinolap/cube/measure/DoubleMinAggregator.java b/cube/src/main/java/com/kylinolap/cube/measure/DoubleMinAggregator.java
new file mode 100644
index 0000000..e03e386
--- /dev/null
+++ b/cube/src/main/java/com/kylinolap/cube/measure/DoubleMinAggregator.java
@@ -0,0 +1,52 @@
+/*
+ * 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.cube.measure;
+
+import org.apache.hadoop.io.DoubleWritable;
+
+/**
+ * @author yangli9
+ * 
+ */
+public class DoubleMinAggregator extends MeasureAggregator<DoubleWritable> {
+
+    DoubleWritable min = null;
+
+    @Override
+    public void reset() {
+        min = null;
+    }
+
+    @Override
+    public void aggregate(DoubleWritable value) {
+        if (min == null)
+            min = new DoubleWritable(value.get());
+        else if (min.get() > value.get())
+            min.set(value.get());
+    }
+
+    @Override
+    public DoubleWritable getState() {
+        return min;
+    }
+
+    @Override
+    public int getMemBytes() {
+        return guessDoubleMemBytes();
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/cube/src/main/java/com/kylinolap/cube/measure/DoubleSerializer.java
----------------------------------------------------------------------
diff --git a/cube/src/main/java/com/kylinolap/cube/measure/DoubleSerializer.java b/cube/src/main/java/com/kylinolap/cube/measure/DoubleSerializer.java
new file mode 100644
index 0000000..54e8c1f
--- /dev/null
+++ b/cube/src/main/java/com/kylinolap/cube/measure/DoubleSerializer.java
@@ -0,0 +1,53 @@
+/*
+ * 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.cube.measure;
+
+import java.nio.ByteBuffer;
+
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.io.DoubleWritable;
+
+/**
+ * @author yangli9
+ * 
+ */
+public class DoubleSerializer extends MeasureSerializer<DoubleWritable> {
+
+    // avoid mass object creation
+    DoubleWritable current = new DoubleWritable();
+
+    @Override
+    public void serialize(DoubleWritable value, ByteBuffer out) {
+        out.putDouble(value.get());
+    }
+
+    @Override
+    public DoubleWritable deserialize(ByteBuffer in) {
+        current.set(in.getDouble());
+        return current;
+    }
+
+    @Override
+    public DoubleWritable valueOf(byte[] value) {
+        if (value == null)
+            current.set(0d);
+        else
+            current.set(Double.parseDouble(Bytes.toString(value)));
+        return current;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/cube/src/main/java/com/kylinolap/cube/measure/DoubleSumAggregator.java
----------------------------------------------------------------------
diff --git a/cube/src/main/java/com/kylinolap/cube/measure/DoubleSumAggregator.java b/cube/src/main/java/com/kylinolap/cube/measure/DoubleSumAggregator.java
new file mode 100644
index 0000000..88b90c7
--- /dev/null
+++ b/cube/src/main/java/com/kylinolap/cube/measure/DoubleSumAggregator.java
@@ -0,0 +1,49 @@
+/*
+ * 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.cube.measure;
+
+import org.apache.hadoop.io.DoubleWritable;
+
+/**
+ * @author yangli9
+ * 
+ */
+public class DoubleSumAggregator extends MeasureAggregator<DoubleWritable> {
+
+    DoubleWritable sum = new DoubleWritable();
+
+    @Override
+    public void reset() {
+        sum.set(0.0);
+    }
+
+    @Override
+    public void aggregate(DoubleWritable value) {
+        sum.set(sum.get() + value.get());
+    }
+
+    @Override
+    public DoubleWritable getState() {
+        return sum;
+    }
+
+    @Override
+    public int getMemBytes() {
+        return guessDoubleMemBytes();
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/cube/src/main/java/com/kylinolap/cube/measure/HLLCAggregator.java
----------------------------------------------------------------------
diff --git a/cube/src/main/java/com/kylinolap/cube/measure/HLLCAggregator.java b/cube/src/main/java/com/kylinolap/cube/measure/HLLCAggregator.java
new file mode 100644
index 0000000..cb070d8
--- /dev/null
+++ b/cube/src/main/java/com/kylinolap/cube/measure/HLLCAggregator.java
@@ -0,0 +1,56 @@
+/*
+
+ * 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.cube.measure;
+
+import com.kylinolap.common.hll.HyperLogLogPlusCounter;
+
+/**
+ * @author yangli9
+ * 
+ */
+public class HLLCAggregator extends MeasureAggregator<HyperLogLogPlusCounter> {
+
+    HyperLogLogPlusCounter sum = null;
+
+    @Override
+    public void reset() {
+        sum = null;
+    }
+
+    @Override
+    public void aggregate(HyperLogLogPlusCounter value) {
+        if (sum == null)
+            sum = new HyperLogLogPlusCounter(value);
+        else
+            sum.merge(value);
+    }
+
+    @Override
+    public HyperLogLogPlusCounter getState() {
+        return sum;
+    }
+
+    @Override
+    public int getMemBytes() {
+        if (sum == null)
+            return Integer.MIN_VALUE;
+        else
+            return 4 + sum.getMemBytes();
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/cube/src/main/java/com/kylinolap/cube/measure/HLLCSerializer.java
----------------------------------------------------------------------
diff --git a/cube/src/main/java/com/kylinolap/cube/measure/HLLCSerializer.java b/cube/src/main/java/com/kylinolap/cube/measure/HLLCSerializer.java
new file mode 100644
index 0000000..ef3818b
--- /dev/null
+++ b/cube/src/main/java/com/kylinolap/cube/measure/HLLCSerializer.java
@@ -0,0 +1,65 @@
+/*
+ * 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.cube.measure;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+
+import com.kylinolap.common.hll.HyperLogLogPlusCounter;
+
+/**
+ * @author yangli9
+ * 
+ */
+public class HLLCSerializer extends MeasureSerializer<HyperLogLogPlusCounter> {
+
+    HyperLogLogPlusCounter current;
+
+    HLLCSerializer(int p) {
+        current = new HyperLogLogPlusCounter(p);
+    }
+
+    @Override
+    public void serialize(HyperLogLogPlusCounter value, ByteBuffer out) {
+        try {
+            value.writeRegisters(out);
+        } catch (IOException e) {
+            throw new RuntimeException(e);
+        }
+    }
+
+    @Override
+    public HyperLogLogPlusCounter deserialize(ByteBuffer in) {
+        try {
+            current.readRegisters(in);
+        } catch (IOException e) {
+            throw new RuntimeException(e);
+        }
+        return current;
+    }
+
+    @Override
+    public HyperLogLogPlusCounter valueOf(byte[] value) {
+        current.clear();
+        if (value == null)
+            current.add("__nUlL__");
+        else
+            current.add(value);
+        return current;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/cube/src/main/java/com/kylinolap/cube/measure/LDCAggregator.java
----------------------------------------------------------------------
diff --git a/cube/src/main/java/com/kylinolap/cube/measure/LDCAggregator.java b/cube/src/main/java/com/kylinolap/cube/measure/LDCAggregator.java
new file mode 100644
index 0000000..94f5f19
--- /dev/null
+++ b/cube/src/main/java/com/kylinolap/cube/measure/LDCAggregator.java
@@ -0,0 +1,62 @@
+/*
+ * 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.cube.measure;
+
+import org.apache.hadoop.io.LongWritable;
+
+/**
+ * Long Distinct Count
+ * 
+ * @author xjiang
+ * 
+ */
+public class LDCAggregator extends MeasureAggregator<LongWritable> {
+
+    private static LongWritable ZERO = new LongWritable(0);
+
+    private HLLCAggregator hllAgg = null;
+    private LongWritable state = new LongWritable(0);
+
+    @SuppressWarnings("rawtypes")
+    public void setDependentAggregator(MeasureAggregator agg) {
+        this.hllAgg = (HLLCAggregator) agg;
+    }
+
+    @Override
+    public void reset() {
+    }
+
+    @Override
+    public void aggregate(LongWritable value) {
+    }
+
+    @Override
+    public LongWritable getState() {
+        if (hllAgg == null) {
+            return ZERO;
+        } else {
+            state.set(hllAgg.getState().getCountEstimate());
+            return state;
+        }
+    }
+
+    @Override
+    public int getMemBytes() {
+        return guessLongMemBytes();
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/cube/src/main/java/com/kylinolap/cube/measure/LongMaxAggregator.java
----------------------------------------------------------------------
diff --git a/cube/src/main/java/com/kylinolap/cube/measure/LongMaxAggregator.java b/cube/src/main/java/com/kylinolap/cube/measure/LongMaxAggregator.java
new file mode 100644
index 0000000..b0b27ca
--- /dev/null
+++ b/cube/src/main/java/com/kylinolap/cube/measure/LongMaxAggregator.java
@@ -0,0 +1,52 @@
+/*
+ * 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.cube.measure;
+
+import org.apache.hadoop.io.LongWritable;
+
+/**
+ * @author yangli9
+ * 
+ */
+public class LongMaxAggregator extends MeasureAggregator<LongWritable> {
+
+    LongWritable max = null;
+
+    @Override
+    public void reset() {
+        max = null;
+    }
+
+    @Override
+    public void aggregate(LongWritable value) {
+        if (max == null)
+            max = new LongWritable(value.get());
+        else if (max.get() < value.get())
+            max.set(value.get());
+    }
+
+    @Override
+    public LongWritable getState() {
+        return max;
+    }
+
+    @Override
+    public int getMemBytes() {
+        return guessLongMemBytes();
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/cube/src/main/java/com/kylinolap/cube/measure/LongMinAggregator.java
----------------------------------------------------------------------
diff --git a/cube/src/main/java/com/kylinolap/cube/measure/LongMinAggregator.java b/cube/src/main/java/com/kylinolap/cube/measure/LongMinAggregator.java
new file mode 100644
index 0000000..3d3bc22
--- /dev/null
+++ b/cube/src/main/java/com/kylinolap/cube/measure/LongMinAggregator.java
@@ -0,0 +1,52 @@
+/*
+ * 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.cube.measure;
+
+import org.apache.hadoop.io.LongWritable;
+
+/**
+ * @author yangli9
+ * 
+ */
+public class LongMinAggregator extends MeasureAggregator<LongWritable> {
+
+    LongWritable min = null;
+
+    @Override
+    public void reset() {
+        min = null;
+    }
+
+    @Override
+    public void aggregate(LongWritable value) {
+        if (min == null)
+            min = new LongWritable(value.get());
+        else if (min.get() > value.get())
+            min.set(value.get());
+    }
+
+    @Override
+    public LongWritable getState() {
+        return min;
+    }
+
+    @Override
+    public int getMemBytes() {
+        return guessLongMemBytes();
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/cube/src/main/java/com/kylinolap/cube/measure/LongSerializer.java
----------------------------------------------------------------------
diff --git a/cube/src/main/java/com/kylinolap/cube/measure/LongSerializer.java b/cube/src/main/java/com/kylinolap/cube/measure/LongSerializer.java
new file mode 100644
index 0000000..40ff496
--- /dev/null
+++ b/cube/src/main/java/com/kylinolap/cube/measure/LongSerializer.java
@@ -0,0 +1,55 @@
+/*
+ * 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.cube.measure;
+
+import java.nio.ByteBuffer;
+
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.io.LongWritable;
+
+import com.kylinolap.common.util.BytesUtil;
+
+/**
+ * @author yangli9
+ * 
+ */
+public class LongSerializer extends MeasureSerializer<LongWritable> {
+
+    // avoid mass object creation
+    LongWritable current = new LongWritable();
+
+    @Override
+    public void serialize(LongWritable value, ByteBuffer out) {
+        BytesUtil.writeVLong(value.get(), out);
+    }
+
+    @Override
+    public LongWritable deserialize(ByteBuffer in) {
+        current.set(BytesUtil.readVLong(in));
+        return current;
+    }
+
+    @Override
+    public LongWritable valueOf(byte[] value) {
+        if (value == null)
+            current.set(0L);
+        else
+            current.set(Long.parseLong(Bytes.toString(value)));
+        return current;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/cube/src/main/java/com/kylinolap/cube/measure/LongSumAggregator.java
----------------------------------------------------------------------
diff --git a/cube/src/main/java/com/kylinolap/cube/measure/LongSumAggregator.java b/cube/src/main/java/com/kylinolap/cube/measure/LongSumAggregator.java
new file mode 100644
index 0000000..a62b456
--- /dev/null
+++ b/cube/src/main/java/com/kylinolap/cube/measure/LongSumAggregator.java
@@ -0,0 +1,49 @@
+/*
+ * 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.cube.measure;
+
+import org.apache.hadoop.io.LongWritable;
+
+/**
+ * @author yangli9
+ * 
+ */
+public class LongSumAggregator extends MeasureAggregator<LongWritable> {
+
+    LongWritable sum = new LongWritable();
+
+    @Override
+    public void reset() {
+        sum.set(0);
+    }
+
+    @Override
+    public void aggregate(LongWritable value) {
+        sum.set(sum.get() + value.get());
+    }
+
+    @Override
+    public LongWritable getState() {
+        return sum;
+    }
+
+    @Override
+    public int getMemBytes() {
+        return guessLongMemBytes();
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/cube/src/main/java/com/kylinolap/cube/measure/MeasureAggregator.java
----------------------------------------------------------------------
diff --git a/cube/src/main/java/com/kylinolap/cube/measure/MeasureAggregator.java b/cube/src/main/java/com/kylinolap/cube/measure/MeasureAggregator.java
new file mode 100644
index 0000000..726cfca
--- /dev/null
+++ b/cube/src/main/java/com/kylinolap/cube/measure/MeasureAggregator.java
@@ -0,0 +1,100 @@
+/*
+ * 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.cube.measure;
+
+import com.kylinolap.metadata.model.cube.FunctionDesc;
+import com.kylinolap.metadata.model.schema.DataType;
+
+/**
+ * @author yangli9
+ * 
+ */
+abstract public class MeasureAggregator<V> {
+
+    public static MeasureAggregator<?> create(String funcName, String returnType) {
+        if (FunctionDesc.FUNC_SUM.equalsIgnoreCase(funcName) || FunctionDesc.FUNC_COUNT.equalsIgnoreCase(funcName)) {
+            if (isInteger(returnType))
+                return new LongSumAggregator();
+            else if (isBigDecimal(returnType))
+                return new BigDecimalSumAggregator();
+            else if (isDouble(returnType))
+                return new DoubleSumAggregator();
+        } else if (FunctionDesc.FUNC_COUNT_DISTINCT.equalsIgnoreCase(funcName)) {
+            if (DataType.getInstance(returnType).isHLLC())
+                return new HLLCAggregator();
+            else
+                return new LDCAggregator();
+        } else if (FunctionDesc.FUNC_MAX.equalsIgnoreCase(funcName)) {
+            if (isInteger(returnType))
+                return new LongMaxAggregator();
+            else if (isBigDecimal(returnType))
+                return new BigDecimalMaxAggregator();
+            else if (isDouble(returnType))
+                return new DoubleMaxAggregator();
+        } else if (FunctionDesc.FUNC_MIN.equalsIgnoreCase(funcName)) {
+            if (isInteger(returnType))
+                return new LongMinAggregator();
+            else if (isBigDecimal(returnType))
+                return new BigDecimalMinAggregator();
+            else if (isDouble(returnType))
+                return new DoubleMinAggregator();
+        }
+        throw new IllegalArgumentException("No aggregator for func '" + funcName + "' and return type '" + returnType + "'");
+    }
+
+    public static boolean isBigDecimal(String type) {
+        return "decimal".equalsIgnoreCase(type);
+    }
+
+    public static boolean isDouble(String type) {
+        return "double".equalsIgnoreCase(type) || "float".equalsIgnoreCase(type) || "real".equalsIgnoreCase(type);
+    }
+
+    public static boolean isInteger(String type) {
+        return "long".equalsIgnoreCase(type) || "bigint".equalsIgnoreCase(type) || "int".equalsIgnoreCase(type) || "integer".equalsIgnoreCase(type);
+    }
+
+    public static int guessBigDecimalMemBytes() {
+        return 4 // ref
+        + 20; // guess of BigDecimal
+    }
+
+    public static int guessDoubleMemBytes() {
+        return 4 // ref
+        + 8;
+    }
+
+    public static int guessLongMemBytes() {
+        return 4 // ref
+        + 8;
+    }
+
+    // ============================================================================
+
+    @SuppressWarnings("rawtypes")
+    public void setDependentAggregator(MeasureAggregator agg) {
+    }
+
+    abstract public void reset();
+
+    abstract public void aggregate(V value);
+
+    abstract public V getState();
+
+    // get an estimate of memory consumption
+    abstract public int getMemBytes();
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/cube/src/main/java/com/kylinolap/cube/measure/MeasureAggregators.java
----------------------------------------------------------------------
diff --git a/cube/src/main/java/com/kylinolap/cube/measure/MeasureAggregators.java b/cube/src/main/java/com/kylinolap/cube/measure/MeasureAggregators.java
new file mode 100644
index 0000000..aa463b8
--- /dev/null
+++ b/cube/src/main/java/com/kylinolap/cube/measure/MeasureAggregators.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.cube.measure;
+
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.Map;
+
+import com.kylinolap.metadata.model.cube.FunctionDesc;
+import com.kylinolap.metadata.model.cube.MeasureDesc;
+
+/**
+ * @author yangli9
+ * 
+ */
+@SuppressWarnings({ "rawtypes", "unchecked" })
+public class MeasureAggregators {
+
+    private MeasureDesc[] descs;
+    private MeasureAggregator[] aggs;
+
+    public MeasureAggregators(Collection<MeasureDesc> measureDescs) {
+        this((MeasureDesc[]) measureDescs.toArray(new MeasureDesc[measureDescs.size()]));
+    }
+
+    public MeasureAggregators(MeasureDesc... measureDescs) {
+        descs = measureDescs;
+        aggs = new MeasureAggregator[descs.length];
+
+        Map<String, Integer> measureIndexMap = new HashMap<String, Integer>();
+        for (int i = 0; i < descs.length; i++) {
+            FunctionDesc func = descs[i].getFunction();
+            aggs[i] = MeasureAggregator.create(func.getExpression(), func.getReturnType());
+            measureIndexMap.put(descs[i].getName(), i);
+        }
+        // fill back dependent aggregator
+        for (int i = 0; i < descs.length; i++) {
+            String depMsrRef = descs[i].getDependentMeasureRef();
+            if (depMsrRef != null) {
+                int index = measureIndexMap.get(depMsrRef);
+                aggs[i].setDependentAggregator(aggs[index]);
+            }
+        }
+    }
+
+    public void reset() {
+        for (int i = 0; i < aggs.length; i++) {
+            aggs[i].reset();
+        }
+    }
+
+    public void aggregate(Object[] values) {
+        assert values.length == descs.length;
+
+        for (int i = 0; i < descs.length; i++) {
+            aggs[i].aggregate(values[i]);
+        }
+    }
+
+    public void collectStates(Object[] states) {
+        for (int i = 0; i < descs.length; i++) {
+            states[i] = aggs[i].getState();
+        }
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/cube/src/main/java/com/kylinolap/cube/measure/MeasureCodec.java
----------------------------------------------------------------------
diff --git a/cube/src/main/java/com/kylinolap/cube/measure/MeasureCodec.java b/cube/src/main/java/com/kylinolap/cube/measure/MeasureCodec.java
new file mode 100644
index 0000000..4a511e7
--- /dev/null
+++ b/cube/src/main/java/com/kylinolap/cube/measure/MeasureCodec.java
@@ -0,0 +1,82 @@
+/*
+ * 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.cube.measure;
+
+import java.nio.ByteBuffer;
+import java.util.Collection;
+
+import org.apache.hadoop.io.Text;
+
+import com.kylinolap.metadata.model.cube.MeasureDesc;
+
+/**
+ * @author yangli9
+ * 
+ */
+@SuppressWarnings({ "rawtypes", "unchecked" })
+public class MeasureCodec {
+
+    int nMeasures;
+    MeasureSerializer[] serializers;
+
+    public MeasureCodec(Collection<MeasureDesc> measureDescs) {
+        this((MeasureDesc[]) measureDescs.toArray(new MeasureDesc[measureDescs.size()]));
+    }
+
+    public MeasureCodec(MeasureDesc... measureDescs) {
+        String[] dataTypes = new String[measureDescs.length];
+        for (int i = 0; i < dataTypes.length; i++) {
+            dataTypes[i] = measureDescs[i].getFunction().getReturnType();
+        }
+        init(dataTypes);
+    }
+
+    public MeasureCodec(String... dataTypes) {
+        init(dataTypes);
+    }
+
+    private void init(String[] dataTypes) {
+        nMeasures = dataTypes.length;
+        serializers = new MeasureSerializer[nMeasures];
+
+        for (int i = 0; i < nMeasures; i++) {
+            serializers[i] = MeasureSerializer.create(dataTypes[i]);
+        }
+    }
+
+    public MeasureSerializer getSerializer(int idx) {
+        return serializers[idx];
+    }
+
+    public void decode(Text bytes, Object[] result) {
+        decode(ByteBuffer.wrap(bytes.getBytes(), 0, bytes.getLength()), result);
+    }
+
+    public void decode(ByteBuffer buf, Object[] result) {
+        assert result.length == nMeasures;
+        for (int i = 0; i < nMeasures; i++) {
+            result[i] = serializers[i].deserialize(buf);
+        }
+    }
+
+    public void encode(Object[] values, ByteBuffer out) {
+        assert values.length == nMeasures;
+        for (int i = 0; i < nMeasures; i++) {
+            serializers[i].serialize(values[i], out);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/cube/src/main/java/com/kylinolap/cube/measure/MeasureSerializer.java
----------------------------------------------------------------------
diff --git a/cube/src/main/java/com/kylinolap/cube/measure/MeasureSerializer.java b/cube/src/main/java/com/kylinolap/cube/measure/MeasureSerializer.java
new file mode 100644
index 0000000..9b2ece7
--- /dev/null
+++ b/cube/src/main/java/com/kylinolap/cube/measure/MeasureSerializer.java
@@ -0,0 +1,66 @@
+/*
+ * 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.cube.measure;
+
+import java.util.HashMap;
+
+import com.kylinolap.common.util.BytesSerializer;
+import com.kylinolap.metadata.model.schema.DataType;
+
+/**
+ * @author yangli9
+ * 
+ */
+abstract public class MeasureSerializer<T> implements BytesSerializer<T> {
+
+    final static HashMap<String, Class<?>> implementations = new HashMap<String, Class<?>>();
+    static {
+        implementations.put("decimal", BigDecimalSerializer.class);
+        implementations.put("double", DoubleSerializer.class);
+        implementations.put("float", DoubleSerializer.class);
+        implementations.put("bigint", LongSerializer.class);
+        implementations.put("long", LongSerializer.class);
+        implementations.put("integer", LongSerializer.class);
+        implementations.put("int", LongSerializer.class);
+    }
+
+    public static MeasureSerializer<?> create(String dataType) {
+        DataType type = DataType.getInstance(dataType);
+        if (type.isHLLC()) {
+            return new HLLCSerializer(type.getPrecision());
+        }
+
+        Class<?> clz = implementations.get(type.getName());
+        if (clz == null)
+            throw new RuntimeException("No MeasureSerializer for type " + dataType);
+
+        try {
+            return (MeasureSerializer<?>) clz.newInstance();
+        } catch (Exception e) {
+            throw new RuntimeException(e); // never happen
+        }
+    }
+
+    abstract public T valueOf(byte[] value);
+
+    public String toString(T value) {
+        if (value == null)
+            return "NULL";
+        else
+            return value.toString();
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/cube/src/main/java/com/kylinolap/cube/measure/fixedlen/FixedLenMeasureCodec.java
----------------------------------------------------------------------
diff --git a/cube/src/main/java/com/kylinolap/cube/measure/fixedlen/FixedLenMeasureCodec.java b/cube/src/main/java/com/kylinolap/cube/measure/fixedlen/FixedLenMeasureCodec.java
new file mode 100644
index 0000000..e8a7a08
--- /dev/null
+++ b/cube/src/main/java/com/kylinolap/cube/measure/fixedlen/FixedLenMeasureCodec.java
@@ -0,0 +1,21 @@
+package com.kylinolap.cube.measure.fixedlen;
+
+import com.kylinolap.metadata.model.schema.DataType;
+
+abstract public class FixedLenMeasureCodec<T> {
+    
+    public static FixedLenMeasureCodec<?> get(DataType type) {
+        return new FixedPointLongCodec(type.getScale());
+    }
+
+    abstract public int getLength();
+
+    abstract public T valueOf(String value);
+
+    abstract public String toString(T value);
+
+    abstract public T read(byte[] buf, int offset);
+
+    abstract public void write(T v, byte[] buf, int offset);
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/cube/src/main/java/com/kylinolap/cube/measure/fixedlen/FixedPointLongCodec.java
----------------------------------------------------------------------
diff --git a/cube/src/main/java/com/kylinolap/cube/measure/fixedlen/FixedPointLongCodec.java b/cube/src/main/java/com/kylinolap/cube/measure/fixedlen/FixedPointLongCodec.java
new file mode 100644
index 0000000..84ba9fb
--- /dev/null
+++ b/cube/src/main/java/com/kylinolap/cube/measure/fixedlen/FixedPointLongCodec.java
@@ -0,0 +1,54 @@
+package com.kylinolap.cube.measure.fixedlen;
+
+import org.apache.hadoop.io.LongWritable;
+
+import com.kylinolap.common.util.BytesUtil;
+
+public class FixedPointLongCodec extends FixedLenMeasureCodec<LongWritable> {
+    
+    private static final int SIZE = 8;
+    // number of digits after decimal point
+    int scale;
+    double scalePower;
+    // avoid mass object creation
+    LongWritable current = new LongWritable();
+
+    public FixedPointLongCodec(int scale) {
+        scale = Math.max(0, scale);
+        this.scale = scale;
+        this.scalePower = Math.pow(10, scale);
+    }
+
+    @Override
+    public int getLength() {
+        return SIZE;
+    }
+
+    @Override
+    public LongWritable valueOf(String value) {
+        if (value == null)
+            current.set(0L);
+        else
+            current.set((long) (Double.parseDouble(value) * scalePower));
+        return current;
+    }
+
+    @Override
+    public String toString(LongWritable value) {
+        if (scale == 0)
+            return value.toString();
+        else
+            return "" + (value.get() / scalePower);
+    }
+
+    @Override
+    public LongWritable read(byte[] buf, int offset) {
+        current.set(BytesUtil.readLong(buf, offset, SIZE));
+        return current;
+    }
+
+    @Override
+    public void write(LongWritable v, byte[] buf, int offset) {
+        BytesUtil.writeLong(v.get(), buf, offset, SIZE);
+    }
+}


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

Posted by lu...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/jdbc/src/main/java/com/kylinolap/jdbc/Driver.java
----------------------------------------------------------------------
diff --git a/jdbc/src/main/java/com/kylinolap/jdbc/Driver.java b/jdbc/src/main/java/com/kylinolap/jdbc/Driver.java
new file mode 100644
index 0000000..0f5e380
--- /dev/null
+++ b/jdbc/src/main/java/com/kylinolap/jdbc/Driver.java
@@ -0,0 +1,137 @@
+/*
+ * 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.jdbc;
+
+import java.sql.SQLException;
+
+import net.hydromatic.avatica.AvaticaConnection;
+import net.hydromatic.avatica.AvaticaStatement;
+import net.hydromatic.avatica.DriverVersion;
+import net.hydromatic.avatica.Handler;
+import net.hydromatic.avatica.HandlerImpl;
+import net.hydromatic.avatica.UnregisteredDriver;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.kylinolap.jdbc.stub.ConnectionException;
+import com.kylinolap.jdbc.stub.RemoteClient;
+
+/**
+ * <p>
+ * Kylin JDBC Driver based on optiq avatica and kylin restful api.<br>
+ * Supported versions:
+ * </p>
+ * <ul>
+ * <li>jdbc 4.0</li>
+ * <li>jdbc 4.1</li>
+ * </ul>
+ * 
+ * <p>
+ * Supported Statements:
+ * </p>
+ * <ul>
+ * <li>{@link KylinStatementImpl}</li>
+ * <li>{@link KylinPrepareStatementImpl}</li>
+ * </ul>
+ * 
+ * <p>
+ * Supported properties:
+ * <ul>
+ * <li>user: username</li>
+ * <li>password: password</li>
+ * <li>ssl: true/false</li>
+ * </ul>
+ * </p>
+ * 
+ * <p>
+ * Driver init code sample:<br>
+ * 
+ * <pre>
+ * Driver driver = (Driver) Class.forName(&quot;com.kylinolap.kylin.jdbc.Driver&quot;).newInstance();
+ * Properties info = new Properties();
+ * info.put(&quot;user&quot;, &quot;user&quot;);
+ * info.put(&quot;password&quot;, &quot;password&quot;);
+ * info.put(&quot;ssl&quot;, true);
+ * Connection conn = driver.connect(&quot;jdbc:kylin://{domain}/{project}&quot;, info);
+ * </pre>
+ * 
+ * </p>
+ * 
+ * @author xduo
+ * 
+ */
+public class Driver extends UnregisteredDriver {
+    private static final Logger logger = LoggerFactory.getLogger(Driver.class);
+
+    public static final String CONNECT_STRING_PREFIX = "jdbc:kylin:";
+
+    @Override
+    protected DriverVersion createDriverVersion() {
+        return DriverVersion.load(Driver.class, "com-kylinolap-kylin-jdbc.properties", "Kylin JDBC Driver", "unknown version", "Kylin", "unknown version");
+    }
+
+    @Override
+    protected String getFactoryClassName(JdbcVersion jdbcVersion) {
+        switch (jdbcVersion) {
+        case JDBC_30:
+            throw new UnsupportedOperationException();
+        case JDBC_40:
+            return "com.kylinolap.kylin.jdbc.KylinJdbc40Factory";
+        case JDBC_41:
+        default:
+            return "com.kylinolap.kylin.jdbc.KylinJdbc41Factory";
+        }
+    }
+
+    @Override
+    protected Handler createHandler() {
+        return new HandlerImpl() {
+            @Override
+            public void onConnectionInit(AvaticaConnection connection_) throws SQLException {
+                KylinConnectionImpl kylinConn = (KylinConnectionImpl) connection_;
+                RemoteClient runner = ((KylinJdbc41Factory) factory).newRemoteClient(kylinConn);
+                try {
+                    runner.connect();
+                    kylinConn.setMetaProject(runner.getMetadata(kylinConn.getProject()));
+                    logger.debug("Connection inited.");
+                } catch (ConnectionException e) {
+                    logger.error(e.getLocalizedMessage(), e);
+                    throw new SQLException(e.getLocalizedMessage());
+                }
+            }
+
+            public void onConnectionClose(AvaticaConnection connection) {
+                logger.debug("Connection closed.");
+            }
+
+            public void onStatementExecute(AvaticaStatement statement, ResultSink resultSink) {
+                logger.debug("statement executed.");
+            }
+
+            public void onStatementClose(AvaticaStatement statement) {
+                logger.debug("statement closed.");
+            }
+        };
+    }
+
+    @Override
+    protected String getConnectStringPrefix() {
+        return CONNECT_STRING_PREFIX;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/jdbc/src/main/java/com/kylinolap/jdbc/KylinConnectionImpl.java
----------------------------------------------------------------------
diff --git a/jdbc/src/main/java/com/kylinolap/jdbc/KylinConnectionImpl.java b/jdbc/src/main/java/com/kylinolap/jdbc/KylinConnectionImpl.java
new file mode 100644
index 0000000..2b412ed
--- /dev/null
+++ b/jdbc/src/main/java/com/kylinolap/jdbc/KylinConnectionImpl.java
@@ -0,0 +1,157 @@
+/*
+ * 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.jdbc;
+
+import java.sql.PreparedStatement;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Properties;
+
+import javax.xml.bind.DatatypeConverter;
+
+import net.hydromatic.avatica.AvaticaConnection;
+import net.hydromatic.avatica.AvaticaFactory;
+import net.hydromatic.avatica.AvaticaPreparedStatement;
+import net.hydromatic.avatica.AvaticaStatement;
+import net.hydromatic.avatica.Meta;
+import net.hydromatic.avatica.UnregisteredDriver;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.kylinolap.jdbc.KylinMetaImpl.MetaProject;
+import com.kylinolap.jdbc.KylinPrepare.PrepareResult;
+
+/**
+ * Kylin connection implementation
+ * 
+ * @author xduo
+ * 
+ */
+public abstract class KylinConnectionImpl extends AvaticaConnection {
+    private static final Logger logger = LoggerFactory.getLogger(KylinConnectionImpl.class);
+
+    private final String baseUrl;
+    private final String project;
+    private MetaProject metaProject;
+    public final List<AvaticaStatement> statements;
+    static final Trojan TROJAN = createTrojan();
+
+    protected KylinConnectionImpl(UnregisteredDriver driver, AvaticaFactory factory, String url, Properties info) {
+        super(driver, factory, url, info);
+
+        String odbcUrl = url;
+        odbcUrl = odbcUrl.replace(Driver.CONNECT_STRING_PREFIX + "//", "");
+        String[] temps = odbcUrl.split("/");
+
+        assert temps.length == 2;
+
+        this.baseUrl = temps[0];
+        this.project = temps[1];
+
+        logger.debug("Kylin base url " + this.baseUrl + ", project name " + this.project);
+
+        statements = new ArrayList<AvaticaStatement>();
+    }
+
+    @Override
+    protected Meta createMeta() {
+        return new KylinMetaImpl(this, (KylinJdbc41Factory) factory);
+    }
+
+    @Override
+    public AvaticaStatement createStatement(int resultSetType, int resultSetConcurrency, int resultSetHoldability) throws SQLException {
+        AvaticaStatement statement = super.createStatement(resultSetType, resultSetConcurrency, resultSetHoldability);
+        statements.add(statement);
+
+        return statement;
+    }
+
+    @Override
+    public PreparedStatement prepareStatement(String sql) throws SQLException {
+        PrepareResult pr = new KylinPrepareImpl().prepare(sql);
+        AvaticaPreparedStatement statement = ((KylinJdbc41Factory) factory).newPreparedStatement(this, pr, ResultSet.TYPE_FORWARD_ONLY, ResultSet.CONCUR_READ_ONLY, this.getHoldability());
+        statements.add(statement);
+
+        return statement;
+    }
+
+    // ~ kylin specified implements
+
+    public String getBasicAuthHeader() {
+        String username = this.info.getProperty("user");
+        String password = this.info.getProperty("password");
+
+        return DatatypeConverter.printBase64Binary((username + ":" + password).getBytes());
+    }
+
+    public String getConnectUrl() {
+        boolean isSsl = Boolean.parseBoolean((this.info.getProperty("ssl", "false")));
+        return (isSsl ? "https://" : "http://") + this.baseUrl + ":" + (isSsl ? 443 : 80) + "/kylin/api/user/authentication";
+    }
+
+    public String getMetaProjectUrl(String project) {
+        assert project != null;
+        boolean isSsl = Boolean.parseBoolean((this.info.getProperty("ssl", "false")));
+        return (isSsl ? "https://" : "http://") + this.baseUrl + ":" + (isSsl ? 443 : 80) + "/kylin/api/tables_and_columns?project=" + project;
+    }
+
+    public String getQueryUrl() {
+        boolean isSsl = Boolean.parseBoolean((this.info.getProperty("ssl", "false")));
+        return (isSsl ? "https://" : "http://") + this.baseUrl + ":" + (isSsl ? 443 : 80) + "/kylin/api/query";
+    }
+
+    public String getProject() {
+        return this.project;
+    }
+
+    public Meta getMeta() {
+        return this.meta;
+    }
+
+    public AvaticaFactory getFactory() {
+        return this.factory;
+    }
+
+    public UnregisteredDriver getDriver() {
+        return this.driver;
+    }
+
+    public MetaProject getMetaProject() {
+        return metaProject;
+    }
+
+    public void setMetaProject(MetaProject metaProject) {
+        this.metaProject = metaProject;
+    }
+
+    @Override
+    public void close() throws SQLException {
+        super.close();
+
+        this.metaProject = null;
+        this.statements.clear();
+    }
+
+    @Override
+    public String toString() {
+        return "KylinConnectionImpl [baseUrl=" + baseUrl + ", project=" + project + ", metaProject=" + metaProject + "]";
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/jdbc/src/main/java/com/kylinolap/jdbc/KylinEnumerator.java
----------------------------------------------------------------------
diff --git a/jdbc/src/main/java/com/kylinolap/jdbc/KylinEnumerator.java b/jdbc/src/main/java/com/kylinolap/jdbc/KylinEnumerator.java
new file mode 100644
index 0000000..8ef6f66
--- /dev/null
+++ b/jdbc/src/main/java/com/kylinolap/jdbc/KylinEnumerator.java
@@ -0,0 +1,85 @@
+/*
+ * 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.jdbc;
+
+import java.util.Collection;
+import java.util.Iterator;
+
+import net.hydromatic.linq4j.Enumerator;
+
+/**
+ * Query results enumerator
+ * 
+ * @author xduo
+ * 
+ */
+public class KylinEnumerator<E> implements Enumerator<E> {
+
+    /**
+     * current row
+     */
+    private E current;
+
+    /**
+     * data collection
+     */
+    private Collection<E> dataCollection;
+
+    /**
+     * result iterator
+     */
+    private Iterator<E> cursor;
+
+    public KylinEnumerator(Collection<E> dataCollection) {
+        this.dataCollection = dataCollection;
+        this.cursor = this.dataCollection.iterator();
+
+        if (null == this.cursor) {
+            throw new RuntimeException("Cursor can't be null");
+        }
+    }
+
+    @Override
+    public E current() {
+        return current;
+    }
+
+    @Override
+    public boolean moveNext() {
+        if (!cursor.hasNext()) {
+            this.reset();
+
+            return false;
+        }
+
+        current = cursor.next();
+
+        return true;
+    }
+
+    @Override
+    public void reset() {
+        this.cursor = this.dataCollection.iterator();
+    }
+
+    @Override
+    public void close() {
+        this.cursor = null;
+        this.dataCollection = null;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/jdbc/src/main/java/com/kylinolap/jdbc/KylinJdbc40Factory.java
----------------------------------------------------------------------
diff --git a/jdbc/src/main/java/com/kylinolap/jdbc/KylinJdbc40Factory.java b/jdbc/src/main/java/com/kylinolap/jdbc/KylinJdbc40Factory.java
new file mode 100644
index 0000000..83fcca3
--- /dev/null
+++ b/jdbc/src/main/java/com/kylinolap/jdbc/KylinJdbc40Factory.java
@@ -0,0 +1,28 @@
+/*
+ * 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.jdbc;
+
+/**
+ * @author xduo
+ * 
+ */
+public class KylinJdbc40Factory extends KylinJdbc41Factory {
+
+    public KylinJdbc40Factory() {
+        super(4, 0);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/jdbc/src/main/java/com/kylinolap/jdbc/KylinJdbc41Factory.java
----------------------------------------------------------------------
diff --git a/jdbc/src/main/java/com/kylinolap/jdbc/KylinJdbc41Factory.java b/jdbc/src/main/java/com/kylinolap/jdbc/KylinJdbc41Factory.java
new file mode 100644
index 0000000..10a8e02
--- /dev/null
+++ b/jdbc/src/main/java/com/kylinolap/jdbc/KylinJdbc41Factory.java
@@ -0,0 +1,125 @@
+/*
+ * 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.jdbc;
+
+import java.sql.ResultSetMetaData;
+import java.sql.SQLException;
+import java.util.List;
+import java.util.Properties;
+import java.util.TimeZone;
+
+import net.hydromatic.avatica.AvaticaConnection;
+import net.hydromatic.avatica.AvaticaDatabaseMetaData;
+import net.hydromatic.avatica.AvaticaFactory;
+import net.hydromatic.avatica.AvaticaPrepareResult;
+import net.hydromatic.avatica.AvaticaPreparedStatement;
+import net.hydromatic.avatica.AvaticaResultSet;
+import net.hydromatic.avatica.AvaticaResultSetMetaData;
+import net.hydromatic.avatica.AvaticaStatement;
+import net.hydromatic.avatica.ColumnMetaData;
+import net.hydromatic.avatica.UnregisteredDriver;
+
+import com.kylinolap.jdbc.stub.KylinClient;
+import com.kylinolap.jdbc.stub.RemoteClient;
+
+/**
+ * Kylin JDBC factory.
+ * 
+ * @author xduo
+ * 
+ */
+public class KylinJdbc41Factory implements AvaticaFactory {
+    private final int major;
+    private final int minor;
+
+    /** Creates a JDBC factory. */
+    public KylinJdbc41Factory() {
+        this(4, 1);
+    }
+
+    /** Creates a JDBC factory with given major/minor version number. */
+    protected KylinJdbc41Factory(int major, int minor) {
+        this.major = major;
+        this.minor = minor;
+    }
+
+    public int getJdbcMajorVersion() {
+        return major;
+    }
+
+    public int getJdbcMinorVersion() {
+        return minor;
+    }
+
+    public AvaticaConnection newConnection(UnregisteredDriver driver, AvaticaFactory factory, String url, Properties info) {
+        return new KylinJdbc41Connection(driver, factory, url, info);
+    }
+
+    public AvaticaDatabaseMetaData newDatabaseMetaData(AvaticaConnection connection) {
+        return new AvaticaJdbc41DatabaseMetaData(connection);
+    }
+
+    public AvaticaStatement newStatement(AvaticaConnection connection, int resultSetType, int resultSetConcurrency, int resultSetHoldability) {
+        return new KylinJdbc41Statement(connection, resultSetType, resultSetConcurrency, resultSetHoldability);
+    }
+
+    public AvaticaPreparedStatement newPreparedStatement(AvaticaConnection connection, AvaticaPrepareResult prepareResult, int resultSetType, int resultSetConcurrency, int resultSetHoldability) throws SQLException {
+        return new KylinJdbc41PreparedStatement(connection, prepareResult, resultSetType, resultSetConcurrency, resultSetHoldability);
+    }
+
+    public AvaticaResultSet newResultSet(AvaticaStatement statement, AvaticaPrepareResult prepareResult, TimeZone timeZone) {
+        final ResultSetMetaData metaData = newResultSetMetaData(statement, prepareResult.getColumnList());
+        return new KylinResultSet(statement, prepareResult, metaData, timeZone);
+    }
+
+    public AvaticaResultSetMetaData newResultSetMetaData(AvaticaStatement statement, List<ColumnMetaData> columnMetaDataList) {
+        return new AvaticaResultSetMetaData(statement, null, columnMetaDataList);
+    }
+
+    // ~ kylin sepcified
+    public RemoteClient newRemoteClient(KylinConnectionImpl connection) {
+        return new KylinClient(connection);
+    }
+
+    /** Implementation of Connection for JDBC 4.1. */
+    private static class KylinJdbc41Connection extends KylinConnectionImpl {
+        KylinJdbc41Connection(UnregisteredDriver driver, AvaticaFactory factory, String url, Properties info) {
+            super(driver, (KylinJdbc41Factory) factory, url, info);
+        }
+    }
+
+    /** Implementation of Statement for JDBC 4.1. */
+    public static class KylinJdbc41Statement extends KylinStatementImpl {
+        public KylinJdbc41Statement(AvaticaConnection connection, int resultSetType, int resultSetConcurrency, int resultSetHoldability) {
+            super(connection, resultSetType, resultSetConcurrency, resultSetHoldability);
+        }
+    }
+
+    /** Implementation of PreparedStatement for JDBC 4.1. */
+    public static class KylinJdbc41PreparedStatement extends KylinPrepareStatementImpl {
+        KylinJdbc41PreparedStatement(AvaticaConnection connection, AvaticaPrepareResult sql, int resultSetType, int resultSetConcurrency, int resultSetHoldability) throws SQLException {
+            super(connection, sql, resultSetType, resultSetConcurrency, resultSetHoldability);
+        }
+    }
+
+    /** Implementation of DatabaseMetaData for JDBC 4.1. */
+    private static class AvaticaJdbc41DatabaseMetaData extends AvaticaDatabaseMetaData {
+        AvaticaJdbc41DatabaseMetaData(AvaticaConnection connection) {
+            super(connection);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/jdbc/src/main/java/com/kylinolap/jdbc/KylinMetaImpl.java
----------------------------------------------------------------------
diff --git a/jdbc/src/main/java/com/kylinolap/jdbc/KylinMetaImpl.java b/jdbc/src/main/java/com/kylinolap/jdbc/KylinMetaImpl.java
new file mode 100644
index 0000000..8be47b0
--- /dev/null
+++ b/jdbc/src/main/java/com/kylinolap/jdbc/KylinMetaImpl.java
@@ -0,0 +1,825 @@
+/*
+ * 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.jdbc;
+
+import java.lang.reflect.Field;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.sql.Types;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
+import java.util.regex.Pattern;
+
+import net.hydromatic.avatica.AvaticaPrepareResult;
+import net.hydromatic.avatica.AvaticaResultSet;
+import net.hydromatic.avatica.AvaticaStatement;
+import net.hydromatic.avatica.ColumnMetaData;
+import net.hydromatic.avatica.ColumnMetaData.Rep;
+import net.hydromatic.avatica.Cursor;
+import net.hydromatic.avatica.Meta;
+import net.hydromatic.linq4j.Enumerator;
+import net.hydromatic.optiq.runtime.EnumeratorCursor;
+
+import org.eigenbase.sql.SqlJdbcFunctionCall;
+import org.eigenbase.sql.parser.SqlParser;
+import org.eigenbase.util.Util;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.kylinolap.jdbc.stub.DataSet;
+import com.kylinolap.jdbc.stub.KylinColumnMetaData;
+import com.kylinolap.jdbc.stub.RemoteClient;
+import com.kylinolap.jdbc.util.SQLTypeMap;
+
+/**
+ * Implementation of avatica interface
+ *
+ * @author xduo
+ */
+public class KylinMetaImpl implements Meta {
+
+    private static final Logger logger = LoggerFactory.getLogger(KylinMetaImpl.class);
+
+    private final KylinConnectionImpl conn;
+
+    private final KylinJdbc41Factory factory;
+
+    /**
+     * @param conn
+     */
+    public KylinMetaImpl(KylinConnectionImpl conn, KylinJdbc41Factory factory) {
+        super();
+        this.conn = conn;
+        this.factory = factory;
+    }
+
+    private ResultSet mockEmptyResultSet() {
+        AvaticaResultSet resultSet = null;
+        try {
+            List<ColumnMetaData> columnMetas = new ArrayList<ColumnMetaData>();
+            List<Object[]> data = new ArrayList<Object[]>();
+            resultSet = this.conn.getFactory().newResultSet(this.conn.createStatement(), new KylinPrepare.PrepareResult(null, null, new KylinEnumerator<Object[]>(data), ColumnMetaData.struct(columnMetas)), this.conn.getTimeZone());
+            KylinConnectionImpl.TROJAN.execute(resultSet);
+        } catch (SQLException e) {
+            logger.error(e.getLocalizedMessage(), e);
+        }
+
+        return resultSet;
+    }
+
+    public String getSqlKeywords() {
+        return SqlParser.create("").getMetadata().getJdbcKeywords();
+    }
+
+    public String getNumericFunctions() {
+        return SqlJdbcFunctionCall.getNumericFunctions();
+    }
+
+    public String getStringFunctions() {
+        return SqlJdbcFunctionCall.getStringFunctions();
+    }
+
+    public String getSystemFunctions() {
+        return SqlJdbcFunctionCall.getSystemFunctions();
+    }
+
+    public String getTimeDateFunctions() {
+        return SqlJdbcFunctionCall.getTimeDateFunctions();
+    }
+
+    public ResultSet getTables(String catalog, Pat schemaPattern, Pat tableNamePattern, List<String> typeList) {
+        logger.debug("Get tables with conn " + conn);
+        MetaProject metaProject = conn.getMetaProject();
+
+        if (null != metaProject) {
+            final DataSet<MetaTable> tables = metaProject.getMetaTables(catalog, schemaPattern, tableNamePattern);
+            final NamedFieldGetter<MetaTable> tableGetter = new NamedFieldGetter<MetaTable>(MetaTable.class, tables.getMeta(), "TABLE_CAT", "TABLE_SCHEM", "TABLE_NAME", "TABLE_TYPE", "REMARKS", "TYPE_CAT", "TYPE_SCHEM", "TYPE_NAME", "SELF_REFERENCING_COL_NAME", "REF_GENERATION");
+
+            AvaticaResultSet resultSet = null;
+            try {
+                resultSet = this.conn.getFactory().newResultSet(this.conn.createStatement(), new KylinPrepare.PrepareResult(null, null, null, tableGetter.structType) {
+                    @Override
+                    public Cursor createCursor() {
+                        return tableGetter.cursor(tables.getEnumerator());
+                    }
+                }, this.conn.getTimeZone());
+                KylinConnectionImpl.TROJAN.execute(resultSet);
+            } catch (SQLException e) {
+                logger.error(e.getLocalizedMessage(), e);
+            }
+
+            return resultSet;
+        } else {
+            return mockEmptyResultSet();
+        }
+    }
+
+    public ResultSet getColumns(String catalog, Pat schemaPattern, Pat tableNamePattern, Pat columnNamePattern) {
+        logger.debug("Get columns with conn " + conn);
+        MetaProject metaProject = conn.getMetaProject();
+
+        if (null != metaProject) {
+            final DataSet<MetaColumn> columns = metaProject.getMetaColumns(catalog, schemaPattern, tableNamePattern, columnNamePattern);
+            final NamedFieldGetter<MetaColumn> columnGetter = new NamedFieldGetter<MetaColumn>(MetaColumn.class, columns.getMeta(), "TABLE_CAT", "TABLE_SCHEM", "TABLE_NAME", "COLUMN_NAME", "DATA_TYPE", "TYPE_NAME", "COLUMN_SIZE", "BUFFER_LENGTH", "DECIMAL_DIGITS", "NUM_PREC_RADIX", "NULLABLE", "REMARKS", "COLUMN_DEF", "SQL_DATA_TYPE", "SQL_DATETIME_SUB", "CHAR_OCTET_LENGTH", "ORDINAL_POSITION", "IS_NULLABLE", "SCOPE_CATALOG", "SCOPE_TABLE", "SOURCE_DATA_TYPE", "IS_AUTOINCREMENT", "IS_GENERATEDCOLUMN");
+
+            AvaticaResultSet resultSet = null;
+            try {
+                resultSet = this.conn.getFactory().newResultSet(this.conn.createStatement(), new KylinPrepare.PrepareResult(null, null, null, columnGetter.structType) {
+                    @Override
+                    public Cursor createCursor() {
+                        return columnGetter.cursor(columns.getEnumerator());
+                    }
+                }, this.conn.getTimeZone());
+
+                KylinConnectionImpl.TROJAN.execute(resultSet);
+            } catch (SQLException e) {
+                logger.error(e.getLocalizedMessage(), e);
+            }
+
+            return resultSet;
+        } else {
+            return mockEmptyResultSet();
+        }
+    }
+
+    public ResultSet getSchemas(String catalog, Pat schemaPattern) {
+        logger.debug("Get schemas with conn " + conn);
+        MetaProject metaProject = conn.getMetaProject();
+
+        if (null != metaProject) {
+            final DataSet<MetaSchema> schemas = metaProject.getMetaSchemas(catalog, schemaPattern);
+            final NamedFieldGetter<MetaSchema> schemaGetter = new NamedFieldGetter<MetaSchema>(MetaSchema.class, schemas.getMeta(), "TABLE_SCHEM", "TABLE_CATALOG");
+
+            AvaticaResultSet resultSet = null;
+            try {
+                resultSet = this.conn.getFactory().newResultSet(this.conn.createStatement(), new KylinPrepare.PrepareResult(null, null, null, schemaGetter.structType) {
+                    @Override
+                    public Cursor createCursor() {
+                        return schemaGetter.cursor(schemas.getEnumerator());
+                    }
+                }, this.conn.getTimeZone());
+
+                KylinConnectionImpl.TROJAN.execute(resultSet);
+            } catch (SQLException e) {
+                logger.error(e.getLocalizedMessage(), e);
+            }
+
+            return resultSet;
+        } else {
+            return mockEmptyResultSet();
+        }
+    }
+
+    public ResultSet getCatalogs() {
+        MetaProject metaProject = conn.getMetaProject();
+
+        if (null != metaProject) {
+            final DataSet<MetaCatalog> catalogs = metaProject.getMetaCatalogs();
+            final NamedFieldGetter<MetaCatalog> catalogGetter = new NamedFieldGetter<MetaCatalog>(MetaCatalog.class, catalogs.getMeta(), "TABLE_CATALOG");
+
+            AvaticaResultSet resultSet = null;
+            try {
+                resultSet = this.conn.getFactory().newResultSet(this.conn.createStatement(), new KylinPrepare.PrepareResult(null, null, null, catalogGetter.structType) {
+                    @Override
+                    public Cursor createCursor() {
+                        return catalogGetter.cursor(catalogs.getEnumerator());
+                    }
+                }, this.conn.getTimeZone());
+
+                KylinConnectionImpl.TROJAN.execute(resultSet);
+            } catch (SQLException e) {
+                logger.error(e.getLocalizedMessage(), e);
+            }
+
+            return resultSet;
+        } else {
+            return mockEmptyResultSet();
+        }
+    }
+
+    public ResultSet getTableTypes() {
+        List<ColumnMetaData> tableTypeMeta = new ArrayList<ColumnMetaData>();
+        tableTypeMeta.add(ColumnMetaData.dummy(ColumnMetaData.scalar(Types.VARCHAR, "varchar", Rep.STRING), false));
+        List<Object[]> data = new ArrayList<Object[]>();
+        Object[] row = new Object[1];
+        row[0] = "TABLE";
+        data.add(row);
+
+        AvaticaResultSet resultSet = null;
+        try {
+            resultSet = this.conn.getFactory().newResultSet(this.conn.createStatement(), new KylinPrepare.PrepareResult(null, null, new KylinEnumerator<Object[]>(data), ColumnMetaData.struct(tableTypeMeta)), this.conn.getTimeZone());
+            KylinConnectionImpl.TROJAN.execute(resultSet);
+        } catch (SQLException e) {
+            logger.error(e.getLocalizedMessage(), e);
+        }
+
+        return resultSet;
+    }
+
+    public ResultSet getProcedures(String catalog, Pat schemaPattern, Pat procedureNamePattern) {
+        return mockEmptyResultSet();
+    }
+
+    public ResultSet getProcedureColumns(String catalog, Pat schemaPattern, Pat procedureNamePattern, Pat columnNamePattern) {
+        return mockEmptyResultSet();
+    }
+
+    public ResultSet getColumnPrivileges(String catalog, String schema, String table, Pat columnNamePattern) {
+        return mockEmptyResultSet();
+    }
+
+    public ResultSet getTablePrivileges(String catalog, Pat schemaPattern, Pat tableNamePattern) {
+        return mockEmptyResultSet();
+    }
+
+    public ResultSet getBestRowIdentifier(String catalog, String schema, String table, int scope, boolean nullable) {
+        return mockEmptyResultSet();
+    }
+
+    public ResultSet getVersionColumns(String catalog, String schema, String table) {
+        return mockEmptyResultSet();
+    }
+
+    public ResultSet getPrimaryKeys(String catalog, String schema, String table) {
+        return mockEmptyResultSet();
+    }
+
+    public ResultSet getImportedKeys(String catalog, String schema, String table) {
+        return mockEmptyResultSet();
+    }
+
+    public ResultSet getExportedKeys(String catalog, String schema, String table) {
+        return mockEmptyResultSet();
+    }
+
+    public ResultSet getCrossReference(String parentCatalog, String parentSchema, String parentTable, String foreignCatalog, String foreignSchema, String foreignTable) {
+        return mockEmptyResultSet();
+    }
+
+    public ResultSet getTypeInfo() {
+        return mockEmptyResultSet();
+    }
+
+    public ResultSet getIndexInfo(String catalog, String schema, String table, boolean unique, boolean approximate) {
+        return mockEmptyResultSet();
+    }
+
+    public ResultSet getUDTs(String catalog, Pat schemaPattern, Pat typeNamePattern, int[] types) {
+        return mockEmptyResultSet();
+    }
+
+    public ResultSet getSuperTypes(String catalog, Pat schemaPattern, Pat typeNamePattern) {
+        return mockEmptyResultSet();
+    }
+
+    public ResultSet getSuperTables(String catalog, Pat schemaPattern, Pat tableNamePattern) {
+        return mockEmptyResultSet();
+    }
+
+    public ResultSet getAttributes(String catalog, Pat schemaPattern, Pat typeNamePattern, Pat attributeNamePattern) {
+        return mockEmptyResultSet();
+    }
+
+    public ResultSet getClientInfoProperties() {
+        return mockEmptyResultSet();
+    }
+
+    public ResultSet getFunctions(String catalog, Pat schemaPattern, Pat functionNamePattern) {
+        return mockEmptyResultSet();
+    }
+
+    public ResultSet getFunctionColumns(String catalog, Pat schemaPattern, Pat functionNamePattern, Pat columnNamePattern) {
+        return mockEmptyResultSet();
+    }
+
+    public ResultSet getPseudoColumns(String catalog, Pat schemaPattern, Pat tableNamePattern, Pat columnNamePattern) {
+        return mockEmptyResultSet();
+    }
+
+    public Cursor createCursor(AvaticaResultSet resultSet) {
+
+        if (!(resultSet instanceof KylinResultSet))
+            throw new IllegalStateException("resultSet is not KylinResultSet");
+
+        KylinPrepare.PrepareResult result = ((KylinResultSet) resultSet).getPrepareResult();
+
+        return result.createCursor();
+    }
+
+    /* 
+     * Client could request metadata after prepare
+     * 
+     * (non-Javadoc)
+     * @see net.hydromatic.avatica.Meta#prepare(net.hydromatic.avatica.AvaticaStatement, java.lang.String)
+     */
+    public AvaticaPrepareResult prepare(AvaticaStatement statement, String sql) {
+        RemoteClient client = factory.newRemoteClient(conn);
+        DataSet<Object[]> result = null;
+
+        try {
+            result = (DataSet<Object[]>) client.query(statement, sql);
+        } catch (Exception e) {
+            logger.error(e.getLocalizedMessage(), e);
+            throw new RuntimeException("Failed to query kylin server with exception " + e.getLocalizedMessage());
+        }
+
+        return new KylinPrepare.PrepareResult(sql, null, (Enumerator<Object[]>) result.getEnumerator(), ColumnMetaData.struct(result.getMeta()));
+    }
+
+    /**
+     * Tree node used by project tree-like structure
+     *
+     * @author xduo
+     */
+    interface Node {
+        /**
+         * Get the node name
+         *
+         * @return
+         */
+        public String getName();
+
+        /**
+         * Get direct children of the node.
+         *
+         * @return
+         */
+        public List<? extends Node> getChildren();
+
+        /**
+         * Search the subtree of the node with patterns. One pattern, one level.
+         *
+         * @param patterns
+         * @return
+         */
+        public List<? extends Node> searchByPatterns(Pat... patterns);
+    }
+
+    /**
+     * Abstract of the tree-like structure
+     *
+     * @author xduo
+     */
+    public static abstract class AbstractNode implements Node {
+
+        public List<? extends Node> searchByPatterns(Pat... patterns) {
+            if (patterns.length == 1) {
+                return findChildren(patterns[0]);
+            } else {
+                List<Node> children = new ArrayList<Node>();
+
+                for (Node child : this.findChildren(patterns[0])) {
+                    children.addAll(child.searchByPatterns(Arrays.copyOfRange(patterns, 1, patterns.length)));
+                }
+
+                return children;
+            }
+        }
+
+        private List<? extends Node> findChildren(Pat pattern) {
+            if (null == pattern.s || pattern.s.equals("%")) {
+                return this.getChildren();
+            }
+
+            List<Node> list = new ArrayList<Node>();
+
+            for (Node c : this.getChildren()) {
+                if (likeToRegex(pattern).matcher(c.getName()).matches()) {
+                    list.add(c);
+                }
+            }
+
+            return list;
+        }
+
+        ;
+
+        /**
+         * Converts a LIKE-style pattern (where '%' represents a wild-card,
+         * escaped using '\') to a Java regex.
+         */
+        private Pattern likeToRegex(Pat pattern) {
+            StringBuilder buf = new StringBuilder("^");
+            char[] charArray = pattern.s.toCharArray();
+            int slash = -2;
+            for (int i = 0; i < charArray.length; i++) {
+                char c = charArray[i];
+                if (slash == i - 1) {
+                    buf.append('[').append(c).append(']');
+                } else {
+                    switch (c) {
+                    case '\\':
+                        slash = i;
+                        break;
+                    case '%':
+                        buf.append(".*");
+                        break;
+                    case '[':
+                        buf.append("\\[");
+                        break;
+                    case ']':
+                        buf.append("\\]");
+                        break;
+                    default:
+                        buf.append('[').append(c).append(']');
+                    }
+                }
+            }
+            buf.append("$");
+
+            return Pattern.compile(buf.toString());
+        }
+    }
+
+    public static class MetaProject extends AbstractNode {
+        public final String project;
+        public final List<MetaCatalog> catalogs;
+
+        public MetaProject(String project, List<MetaCatalog> catalogs) {
+            super();
+            this.project = project;
+            this.catalogs = catalogs;
+        }
+
+        public DataSet<MetaCatalog> getMetaCatalogs() {
+            return new DataSet<MetaCatalog>(MetaCatalog.meta, new KylinEnumerator<MetaCatalog>(catalogs));
+        }
+
+        /**
+         * facade method to search schemas in current project.
+         *
+         * @param catalog
+         * @param schemaPattern
+         * @return
+         */
+        @SuppressWarnings("unchecked")
+        public DataSet<MetaSchema> getMetaSchemas(String catalog, Pat schemaPattern) {
+            List<? extends Node> metaSchemas = this.searchByPatterns(Pat.of(catalog), schemaPattern);
+
+            return new DataSet<MetaSchema>(MetaSchema.meta, new KylinEnumerator<MetaSchema>((Collection<MetaSchema>) metaSchemas));
+        }
+
+        /**
+         * facade method to search tables in current project
+         *
+         * @param catalog
+         * @param schemaPattern
+         * @param tableNamePattern
+         * @return
+         */
+        @SuppressWarnings("unchecked")
+        public DataSet<MetaTable> getMetaTables(String catalog, Pat schemaPattern, Pat tableNamePattern) {
+            logger.debug("getMetaTables with catalog:" + catalog + ", schema:" + schemaPattern.s + ", table:" + tableNamePattern.s);
+            List<? extends Node> tables = this.searchByPatterns(Pat.of(catalog), schemaPattern, tableNamePattern);
+
+            return new DataSet<MetaTable>(MetaTable.meta, new KylinEnumerator<MetaTable>((Collection<MetaTable>) tables));
+        }
+
+        /**
+         * facade method to search columns in current project
+         *
+         * @param catalog
+         * @param schemaPattern
+         * @param tableNamePattern
+         * @param columnNamePattern
+         * @return
+         */
+        @SuppressWarnings("unchecked")
+        public DataSet<MetaColumn> getMetaColumns(String catalog, Pat schemaPattern, Pat tableNamePattern, Pat columnNamePattern) {
+            logger.debug("getMetaColumns with catalog:" + catalog + ", schema:" + schemaPattern.s + ", table:" + tableNamePattern.s + ", column:" + columnNamePattern.s);
+            List<? extends Node> columns = this.searchByPatterns(Pat.of(catalog), schemaPattern, tableNamePattern, columnNamePattern);
+
+            return new DataSet<MetaColumn>(MetaColumn.meta, new KylinEnumerator<MetaColumn>((Collection<MetaColumn>) columns));
+        }
+
+        @Override
+        public String getName() {
+            return project;
+        }
+
+        @Override
+        public List<? extends Node> getChildren() {
+            return this.catalogs;
+        }
+    }
+
+    /**
+     * Metadata describing a catalog.
+     */
+    public static class MetaCatalog extends AbstractNode {
+        public static final List<ColumnMetaData> meta = new ArrayList<ColumnMetaData>();
+        public final String tableCatalog;
+        public final List<MetaSchema> schemas;
+
+        static {
+            meta.add(KylinColumnMetaData.dummy(0, "TABLE_CAT", "TABLE_CAT", ColumnMetaData.scalar(Types.VARCHAR, "varchar", Rep.STRING), true));
+        }
+
+        public MetaCatalog(String tableCatalog, List<MetaSchema> schemas) {
+            this.tableCatalog = tableCatalog;
+            this.schemas = schemas;
+        }
+
+        public String getName() {
+            return tableCatalog;
+        }
+
+        @Override
+        public int hashCode() {
+            final int prime = 31;
+            int result = 1;
+            result = prime * result + ((tableCatalog == null) ? 0 : tableCatalog.hashCode());
+            return result;
+        }
+
+        @Override
+        public boolean equals(Object obj) {
+            if (this == obj)
+                return true;
+            if (obj == null)
+                return false;
+            if (getClass() != obj.getClass())
+                return false;
+            MetaCatalog other = (MetaCatalog) obj;
+            if (tableCatalog == null) {
+                if (other.tableCatalog != null)
+                    return false;
+            } else if (!tableCatalog.equals(other.tableCatalog))
+                return false;
+            return true;
+        }
+
+        @Override
+        public List<? extends Node> getChildren() {
+            return schemas;
+        }
+
+    }
+
+    /**
+     * Metadata describing a schema.
+     */
+    public static class MetaSchema extends AbstractNode {
+        public static final List<ColumnMetaData> meta = new ArrayList<ColumnMetaData>();
+        public final String tableCatalog;
+        public final String tableSchem;
+        public final List<MetaTable> tables;
+
+        static {
+            for (ColumnMetaData cmd : SQLTypeMap.schemaMetaTypeMapping.values()) {
+                meta.add(cmd);
+            }
+        }
+
+        public MetaSchema(String tableCatalog, String tableSchem, List<MetaTable> tables) {
+            this.tableCatalog = tableCatalog;
+            this.tableSchem = tableSchem;
+            this.tables = tables;
+        }
+
+        public String getName() {
+            return tableSchem;
+        }
+
+        @Override
+        public int hashCode() {
+            final int prime = 31;
+            int result = 1;
+            result = prime * result + ((tableCatalog == null) ? 0 : tableCatalog.hashCode());
+            result = prime * result + ((tableSchem == null) ? 0 : tableSchem.hashCode());
+            return result;
+        }
+
+        @Override
+        public boolean equals(Object obj) {
+            if (this == obj)
+                return true;
+            if (obj == null)
+                return false;
+            if (getClass() != obj.getClass())
+                return false;
+            MetaSchema other = (MetaSchema) obj;
+            if (tableCatalog == null) {
+                if (other.tableCatalog != null)
+                    return false;
+            } else if (!tableCatalog.equals(other.tableCatalog))
+                return false;
+            if (tableSchem == null) {
+                if (other.tableSchem != null)
+                    return false;
+            } else if (!tableSchem.equals(other.tableSchem))
+                return false;
+            return true;
+        }
+
+        @Override
+        public List<MetaTable> getChildren() {
+            return this.tables;
+        }
+    }
+
+    /**
+     * Metadata describing a table type.
+     */
+    public static class MetaTableType {
+        public final String tableType;
+
+        public MetaTableType(String tableType) {
+            this.tableType = tableType;
+        }
+    }
+
+    /**
+     * Metadata describing a table.
+     */
+    public static class MetaTable extends AbstractNode {
+        public static final List<ColumnMetaData> meta = new ArrayList<ColumnMetaData>();
+        public final String tableCat;
+        public final String tableSchem;
+        public final String tableName;
+        public final String tableType;
+        public final String remarks;
+        public final String typeCat;
+        public final String typeSchem;
+        public final String typeName;
+        public final String selfReferencingColName;
+        public final String refGeneration;
+        public final List<MetaColumn> columns;
+
+        static {
+            for (ColumnMetaData cmd : SQLTypeMap.tableMetaTypeMapping.values()) {
+                meta.add(cmd);
+            }
+        }
+
+        public MetaTable(String tableCat, String tableSchem, String tableName, String tableType, String remarks, String typeCat, String typeSchem, String typeName, String selfReferencingColName, String refGeneration, List<MetaColumn> columns) {
+            this.tableCat = tableCat;
+            this.tableSchem = tableSchem;
+            this.tableName = tableName;
+            this.tableType = tableType;
+            this.remarks = remarks;
+            this.typeCat = typeCat;
+            this.typeSchem = typeSchem;
+            this.typeName = typeName;
+            this.selfReferencingColName = selfReferencingColName;
+            this.refGeneration = refGeneration;
+            this.columns = columns;
+        }
+
+        public String getName() {
+            return tableName;
+        }
+
+        @Override
+        public List<? extends Node> getChildren() {
+            return this.columns;
+        }
+    }
+
+    /**
+     * Metadata describing a column.
+     */
+    public static class MetaColumn implements Node {
+        public static final List<ColumnMetaData> meta = new ArrayList<ColumnMetaData>();
+        public final String tableCat;
+        public final String tableSchem;
+        public final String tableName;
+        public final String columnName;
+        public final int dataType;
+        public final String typeName;
+        public final int columnSize;
+        public final int bufferLength;
+        public final int decimalDigits;
+        public final int numPrecRadix;
+        public final int nullable;
+        public final String remarks;
+        public final String columnDef;
+        public final int sqlDataType;
+        public final int sqlDatetimeSub;
+        public final int charOctetLength;
+        public final int ordinalPosition;
+        public final String isNullable;
+        public final String scopeCatalog;
+        public final String scopeTable;
+        public final int sourceDataType;
+        public final String isAutoincrement;
+        public final String isGeneratedcolumn;
+
+        static {
+            for (ColumnMetaData cmd : SQLTypeMap.columnMetaTypeMapping.values()) {
+                meta.add(cmd);
+            }
+        }
+
+        public MetaColumn(String tableCat, String tableSchem, String tableName, String columnName, int dataType, String typeName, int columnSize, int bufferLength, int decimalDigits, int numPrecRadix, int nullable, String remarks, String columnDef, int sqlDataType, int sqlDatetimeSub, int charOctetLength, int ordinalPosition, String isNullable, String scopeCatalog, String scopeTable, int sourceDataType, String isAutoincrement, String isGeneratedcolumn) {
+            super();
+            this.tableCat = tableCat;
+            this.tableSchem = tableSchem;
+            this.tableName = tableName;
+            this.columnName = columnName;
+            this.dataType = dataType;
+            this.typeName = typeName;
+            this.columnSize = columnSize;
+            this.bufferLength = bufferLength;
+            this.decimalDigits = decimalDigits;
+            this.numPrecRadix = numPrecRadix;
+            this.nullable = nullable;
+            this.remarks = remarks;
+            this.columnDef = columnDef;
+            this.sqlDataType = sqlDataType;
+            this.sqlDatetimeSub = sqlDatetimeSub;
+            this.charOctetLength = charOctetLength;
+            this.ordinalPosition = ordinalPosition;
+            this.isNullable = isNullable;
+            this.scopeCatalog = scopeCatalog;
+            this.scopeTable = scopeTable;
+            this.sourceDataType = sourceDataType;
+            this.isAutoincrement = isAutoincrement;
+            this.isGeneratedcolumn = isGeneratedcolumn;
+        }
+
+        public String getName() {
+            return columnName;
+        }
+
+        @Override
+        public List<? extends Node> getChildren() {
+            return Collections.emptyList();
+        }
+
+        @Override
+        public List<? extends Node> searchByPatterns(Pat... patterns) {
+            return Collections.emptyList();
+        }
+    }
+
+    /**
+     * Accesses fields by name.
+     */
+    private static class NamedFieldGetter<T> {
+        private final List<Field> fields = new ArrayList<Field>();
+        private final ColumnMetaData.StructType structType;
+
+        public NamedFieldGetter(Class<T> clazz, List<ColumnMetaData> columns, String... names) {
+            init(clazz, names, fields);
+            structType = ColumnMetaData.struct(columns);
+        }
+
+        private void init(Class<T> clazz, String[] names, List<Field> fields) {
+            for (String name : names) {
+                final String fieldName = Util.toCamelCase(name);
+                final Field field;
+                try {
+                    field = clazz.getField(fieldName);
+                } catch (NoSuchFieldException e) {
+                    throw new RuntimeException(e);
+                }
+                fields.add(field);
+            }
+        }
+
+        Object get(Object o, int columnIndex) {
+            try {
+                return fields.get(columnIndex).get(o);
+            } catch (IllegalArgumentException e) {
+                throw new RuntimeException(e);
+            } catch (IllegalAccessException e) {
+                throw new RuntimeException(e);
+            }
+        }
+
+        public Cursor cursor(Enumerator<T> enumerator) {
+            // noinspection unchecked
+            return new EnumeratorCursor<T>(enumerator) {
+                protected Getter createGetter(final int ordinal) {
+                    return new Getter() {
+                        public Object getObject() {
+                            return get(current(), ordinal);
+                        }
+
+                        public boolean wasNull() {
+                            return getObject() == null;
+                        }
+                    };
+                }
+            };
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/jdbc/src/main/java/com/kylinolap/jdbc/KylinPrepare.java
----------------------------------------------------------------------
diff --git a/jdbc/src/main/java/com/kylinolap/jdbc/KylinPrepare.java b/jdbc/src/main/java/com/kylinolap/jdbc/KylinPrepare.java
new file mode 100644
index 0000000..a19b1b1
--- /dev/null
+++ b/jdbc/src/main/java/com/kylinolap/jdbc/KylinPrepare.java
@@ -0,0 +1,96 @@
+/*
+ * 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.jdbc;
+
+import java.util.List;
+
+import net.hydromatic.avatica.AvaticaParameter;
+import net.hydromatic.avatica.AvaticaPrepareResult;
+import net.hydromatic.avatica.ColumnMetaData;
+import net.hydromatic.avatica.Cursor;
+import net.hydromatic.linq4j.Enumerator;
+import net.hydromatic.optiq.runtime.EnumeratorCursor;
+
+/**
+ * Interface of kylin prepare statement implementation
+ * 
+ * @author xduo
+ * 
+ */
+public interface KylinPrepare {
+
+    PrepareResult prepare(String sql);
+
+    /**
+     * The result of preparing a query. It gives the Avatica driver framework
+     * the information it needs to create a prepared statement, or to execute a
+     * statement directly, without an explicit prepare step.
+     */
+    public static class PrepareResult implements AvaticaPrepareResult {
+        public final String sql; // for debug
+        public final ColumnMetaData.StructType structType;
+        public final Enumerator<Object[]> enumerator;
+        public final List<AvaticaParameter> parameterList;
+
+        public PrepareResult(String sql, List<AvaticaParameter> parameterList, Enumerator<Object[]> enumerator, ColumnMetaData.StructType structType) {
+            super();
+            this.sql = sql;
+            this.parameterList = parameterList;
+            this.enumerator = enumerator;
+            this.structType = structType;
+        }
+
+        public Cursor createCursor() {
+            return new EnumeratorCursor<Object[]>(enumerator) {
+                @Override
+                protected Getter createGetter(int ordinal) {
+                    return new ArrayEnumeratorGetter(ordinal);
+                }
+
+                /**
+                 * Row field accessor via index
+                 */
+                class ArrayEnumeratorGetter extends AbstractGetter {
+                    protected final int field;
+
+                    public ArrayEnumeratorGetter(int field) {
+                        this.field = field;
+                    }
+
+                    public Object getObject() {
+                        Object o = current()[field];
+                        wasNull[0] = o == null;
+                        return o;
+                    }
+                }
+            };
+        }
+
+        public List<ColumnMetaData> getColumnList() {
+            return structType.columns;
+        }
+
+        public List<AvaticaParameter> getParameterList() {
+            return parameterList;
+        }
+
+        public String getSql() {
+            return sql;
+        }
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/jdbc/src/main/java/com/kylinolap/jdbc/KylinPrepareImpl.java
----------------------------------------------------------------------
diff --git a/jdbc/src/main/java/com/kylinolap/jdbc/KylinPrepareImpl.java b/jdbc/src/main/java/com/kylinolap/jdbc/KylinPrepareImpl.java
new file mode 100644
index 0000000..3e1d5ce
--- /dev/null
+++ b/jdbc/src/main/java/com/kylinolap/jdbc/KylinPrepareImpl.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.jdbc;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import net.hydromatic.avatica.AvaticaParameter;
+import net.hydromatic.avatica.ColumnMetaData;
+
+/**
+ * @author xduo
+ * 
+ */
+public class KylinPrepareImpl implements KylinPrepare {
+
+    @Override
+    public PrepareResult prepare(String sql) {
+        List<AvaticaParameter> aps = new ArrayList<AvaticaParameter>();
+
+        int startIndex = 0;
+        while (sql.indexOf("?", startIndex) >= 0) {
+            AvaticaParameter ap = new AvaticaParameter(false, 0, 0, 0, null, null, null);
+            aps.add(ap);
+            startIndex = sql.indexOf("?", startIndex) + 1;
+        }
+
+        return new KylinPrepare.PrepareResult(sql, aps, null, ColumnMetaData.struct(new ArrayList<ColumnMetaData>()));
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/jdbc/src/main/java/com/kylinolap/jdbc/KylinPrepareStatementImpl.java
----------------------------------------------------------------------
diff --git a/jdbc/src/main/java/com/kylinolap/jdbc/KylinPrepareStatementImpl.java b/jdbc/src/main/java/com/kylinolap/jdbc/KylinPrepareStatementImpl.java
new file mode 100644
index 0000000..648c806
--- /dev/null
+++ b/jdbc/src/main/java/com/kylinolap/jdbc/KylinPrepareStatementImpl.java
@@ -0,0 +1,168 @@
+/*
+ * 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.jdbc;
+
+import java.io.InputStream;
+import java.io.Reader;
+import java.sql.NClob;
+import java.sql.ResultSet;
+import java.sql.RowId;
+import java.sql.SQLException;
+import java.sql.SQLXML;
+import java.util.Collections;
+import java.util.List;
+
+import net.hydromatic.avatica.AvaticaConnection;
+import net.hydromatic.avatica.AvaticaPrepareResult;
+import net.hydromatic.avatica.AvaticaPreparedStatement;
+import net.hydromatic.avatica.AvaticaResultSet;
+
+/**
+ * Kylin prepare statement. <br>
+ * Supported operations:
+ * <ul>
+ * <li>setString</li>
+ * <li>setInt</li>
+ * <li>setShort</li>
+ * <li>setLong</li>
+ * <li>setFloat</li>
+ * <li>setDouble</li>
+ * <li>setBoolean</li>
+ * <li>setByte</li>
+ * <li>setDate</li>
+ * <li>setTime</li>
+ * <li>setTimestamp</li>
+ * </ul>
+ * 
+ * @author xduo
+ * 
+ */
+public abstract class KylinPrepareStatementImpl extends AvaticaPreparedStatement {
+
+    /**
+     * Before real query,
+     */
+    protected AvaticaPrepareResult prequeryResult;
+
+    protected KylinPrepareStatementImpl(AvaticaConnection connection, AvaticaPrepareResult prepareResult, int resultSetType, int resultSetConcurrency, int resultSetHoldability) throws SQLException {
+        super(connection, prepareResult, resultSetType, resultSetConcurrency, resultSetHoldability);
+
+        this.prequeryResult = prepareResult;
+    }
+
+    @Override
+    public ResultSet executeQuery() throws SQLException {
+        AvaticaPrepareResult queriedResult = ((KylinConnectionImpl) this.connection).getMeta().prepare(this, this.prequeryResult.getSql());
+
+        return executeQueryInternal(queriedResult);
+    }
+
+    @Override
+    protected void close_() {
+        if (!closed) {
+            closed = true;
+            final KylinConnectionImpl connection_ = (KylinConnectionImpl) connection;
+            connection_.statements.remove(this);
+            if (openResultSet != null) {
+                AvaticaResultSet c = openResultSet;
+                openResultSet = null;
+                c.close();
+            }
+            // If onStatementClose throws, this method will throw an
+            // exception (later
+            // converted to SQLException), but this statement still gets
+            // closed.
+            connection_.getDriver().handler.onStatementClose(this);
+        }
+    }
+
+    public List<Object> getParameterValues() {
+        return (List<Object>) Collections.unmodifiableList(super.getParameterValues());
+    }
+
+    public void setRowId(int parameterIndex, RowId x) throws SQLException {
+        getParameter(parameterIndex).setRowId(x);
+    }
+
+    public void setNString(int parameterIndex, String value) throws SQLException {
+        getParameter(parameterIndex).setNString(value);
+    }
+
+    public void setNCharacterStream(int parameterIndex, Reader value, long length) throws SQLException {
+        getParameter(parameterIndex).setNCharacterStream(value, length);
+    }
+
+    public void setNClob(int parameterIndex, NClob value) throws SQLException {
+        getParameter(parameterIndex).setNClob(value);
+    }
+
+    public void setClob(int parameterIndex, Reader reader, long length) throws SQLException {
+        getParameter(parameterIndex).setClob(reader, length);
+    }
+
+    public void setBlob(int parameterIndex, InputStream inputStream, long length) throws SQLException {
+        getParameter(parameterIndex).setBlob(inputStream, length);
+    }
+
+    public void setNClob(int parameterIndex, Reader reader, long length) throws SQLException {
+        getParameter(parameterIndex).setNClob(reader, length);
+    }
+
+    public void setSQLXML(int parameterIndex, SQLXML xmlObject) throws SQLException {
+        getParameter(parameterIndex).setSQLXML(xmlObject);
+    }
+
+    public void setAsciiStream(int parameterIndex, InputStream x, long length) throws SQLException {
+        getParameter(parameterIndex).setAsciiStream(x, length);
+    }
+
+    public void setBinaryStream(int parameterIndex, InputStream x, long length) throws SQLException {
+        getParameter(parameterIndex).setBinaryStream(x, length);
+    }
+
+    public void setCharacterStream(int parameterIndex, Reader reader, long length) throws SQLException {
+        getParameter(parameterIndex).setCharacterStream(reader, length);
+    }
+
+    public void setAsciiStream(int parameterIndex, InputStream x) throws SQLException {
+        getParameter(parameterIndex).setAsciiStream(x);
+    }
+
+    public void setBinaryStream(int parameterIndex, InputStream x) throws SQLException {
+        getParameter(parameterIndex).setBinaryStream(x);
+    }
+
+    public void setCharacterStream(int parameterIndex, Reader reader) throws SQLException {
+        getParameter(parameterIndex).setCharacterStream(reader);
+    }
+
+    public void setNCharacterStream(int parameterIndex, Reader value) throws SQLException {
+        getParameter(parameterIndex).setNCharacterStream(value);
+    }
+
+    public void setClob(int parameterIndex, Reader reader) throws SQLException {
+        getParameter(parameterIndex).setClob(reader);
+    }
+
+    public void setBlob(int parameterIndex, InputStream inputStream) throws SQLException {
+        getParameter(parameterIndex).setBlob(inputStream);
+    }
+
+    public void setNClob(int parameterIndex, Reader reader) throws SQLException {
+        getParameter(parameterIndex).setNClob(reader);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/jdbc/src/main/java/com/kylinolap/jdbc/KylinResultSet.java
----------------------------------------------------------------------
diff --git a/jdbc/src/main/java/com/kylinolap/jdbc/KylinResultSet.java b/jdbc/src/main/java/com/kylinolap/jdbc/KylinResultSet.java
new file mode 100644
index 0000000..c73cc96
--- /dev/null
+++ b/jdbc/src/main/java/com/kylinolap/jdbc/KylinResultSet.java
@@ -0,0 +1,43 @@
+/*
+ * 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.jdbc;
+
+import java.sql.ResultSetMetaData;
+import java.util.TimeZone;
+
+import net.hydromatic.avatica.AvaticaPrepareResult;
+import net.hydromatic.avatica.AvaticaResultSet;
+import net.hydromatic.avatica.AvaticaStatement;
+
+import com.kylinolap.jdbc.KylinPrepare.PrepareResult;
+
+/**
+ * Kylin query result set
+ * 
+ * @author xduo
+ * 
+ */
+public class KylinResultSet extends AvaticaResultSet {
+
+    public KylinResultSet(AvaticaStatement statement, AvaticaPrepareResult prepareResult, ResultSetMetaData resultSetMetaData, TimeZone timeZone) {
+        super(statement, prepareResult, resultSetMetaData, timeZone);
+    }
+
+    public KylinPrepare.PrepareResult getPrepareResult() {
+        return (PrepareResult) prepareResult;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/jdbc/src/main/java/com/kylinolap/jdbc/KylinStatementImpl.java
----------------------------------------------------------------------
diff --git a/jdbc/src/main/java/com/kylinolap/jdbc/KylinStatementImpl.java b/jdbc/src/main/java/com/kylinolap/jdbc/KylinStatementImpl.java
new file mode 100644
index 0000000..1dcef83
--- /dev/null
+++ b/jdbc/src/main/java/com/kylinolap/jdbc/KylinStatementImpl.java
@@ -0,0 +1,52 @@
+/*
+ * Copyright 2013-2014 eBay Softwarimport net.hydromatic.avatica.AvaticaConnection;
+import net.hydromatic.avatica.AvaticaResultSet;
+import net.hydromatic.avatica.AvaticaStatement;
+ense.
+ * 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.jdbc;
+
+import net.hydromatic.avatica.AvaticaConnection;
+import net.hydromatic.avatica.AvaticaResultSet;
+import net.hydromatic.avatica.AvaticaStatement;
+
+/**
+ * Kylin statement implementation
+ * 
+ * @author xduo
+ * 
+ */
+public abstract class KylinStatementImpl extends AvaticaStatement {
+
+    protected KylinStatementImpl(AvaticaConnection connection, int resultSetType, int resultSetConcurrency, int resultSetHoldability) {
+        super(connection, resultSetType, resultSetConcurrency, resultSetHoldability);
+    }
+
+    @Override
+    protected void close_() {
+        if (!closed) {
+            closed = true;
+            final KylinConnectionImpl connection_ = (KylinConnectionImpl) connection;
+            connection_.statements.remove(this);
+            if (openResultSet != null) {
+                AvaticaResultSet c = openResultSet;
+                openResultSet = null;
+                c.close();
+            }
+            // If onStatementClose throws, this method will throw an exception
+            // (later
+            // converted to SQLException), but this statement still gets closed.
+            connection_.getDriver().handler.onStatementClose(this);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/jdbc/src/main/java/com/kylinolap/jdbc/stub/ConnectionException.java
----------------------------------------------------------------------
diff --git a/jdbc/src/main/java/com/kylinolap/jdbc/stub/ConnectionException.java b/jdbc/src/main/java/com/kylinolap/jdbc/stub/ConnectionException.java
new file mode 100644
index 0000000..bce8280
--- /dev/null
+++ b/jdbc/src/main/java/com/kylinolap/jdbc/stub/ConnectionException.java
@@ -0,0 +1,53 @@
+/*
+ * 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.jdbc.stub;
+
+/**
+ * @author xduo
+ * 
+ */
+public class ConnectionException extends Exception {
+
+    private static final long serialVersionUID = 1L;
+
+    public ConnectionException() {
+        super();
+    }
+
+    /**
+     * @param message
+     * @param cause
+     */
+    public ConnectionException(String message, Throwable cause) {
+        super(message, cause);
+    }
+
+    /**
+     * @param message
+     */
+    public ConnectionException(String message) {
+        super(message);
+    }
+
+    /**
+     * @param cause
+     */
+    public ConnectionException(Throwable cause) {
+        super(cause);
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/jdbc/src/main/java/com/kylinolap/jdbc/stub/DataSet.java
----------------------------------------------------------------------
diff --git a/jdbc/src/main/java/com/kylinolap/jdbc/stub/DataSet.java b/jdbc/src/main/java/com/kylinolap/jdbc/stub/DataSet.java
new file mode 100644
index 0000000..f717234
--- /dev/null
+++ b/jdbc/src/main/java/com/kylinolap/jdbc/stub/DataSet.java
@@ -0,0 +1,53 @@
+/*
+ * 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.jdbc.stub;
+
+import java.util.List;
+
+import net.hydromatic.avatica.ColumnMetaData;
+import net.hydromatic.linq4j.Enumerator;
+
+/**
+ * Data set wrapper.
+ * 
+ * @author xduo
+ * 
+ */
+public class DataSet<E> {
+
+    private final List<ColumnMetaData> meta;
+
+    private final Enumerator<E> enumerator;
+
+    /**
+     * @param meta
+     * @param enumerator
+     */
+    public DataSet(List<ColumnMetaData> meta, Enumerator<E> enumerator) {
+        this.meta = meta;
+        this.enumerator = enumerator;
+    }
+
+    public List<ColumnMetaData> getMeta() {
+        return meta;
+    }
+
+    public Enumerator<E> getEnumerator() {
+        return enumerator;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/jdbc/src/main/java/com/kylinolap/jdbc/stub/KylinClient.java
----------------------------------------------------------------------
diff --git a/jdbc/src/main/java/com/kylinolap/jdbc/stub/KylinClient.java b/jdbc/src/main/java/com/kylinolap/jdbc/stub/KylinClient.java
new file mode 100644
index 0000000..0e8fbcf
--- /dev/null
+++ b/jdbc/src/main/java/com/kylinolap/jdbc/stub/KylinClient.java
@@ -0,0 +1,376 @@
+/*
+ * 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.jdbc.stub;
+
+import java.io.IOException;
+import java.sql.SQLException;
+import java.util.ArrayList;
+import java.util.List;
+
+import net.hydromatic.avatica.AvaticaStatement;
+import net.hydromatic.avatica.ColumnMetaData;
+import net.hydromatic.avatica.ColumnMetaData.Rep;
+import net.hydromatic.avatica.ColumnMetaData.ScalarType;
+
+import org.apache.commons.httpclient.HttpClient;
+import org.apache.commons.httpclient.HttpException;
+import org.apache.commons.httpclient.HttpMethodBase;
+import org.apache.commons.httpclient.methods.GetMethod;
+import org.apache.commons.httpclient.methods.PostMethod;
+import org.apache.commons.httpclient.methods.StringRequestEntity;
+import org.apache.commons.httpclient.protocol.Protocol;
+import org.apache.commons.httpclient.protocol.ProtocolSocketFactory;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.fasterxml.jackson.core.JsonProcessingException;
+import com.fasterxml.jackson.core.type.TypeReference;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.google.common.collect.HashMultimap;
+import com.kylinolap.jdbc.KylinConnectionImpl;
+import com.kylinolap.jdbc.KylinEnumerator;
+import com.kylinolap.jdbc.KylinJdbc41Factory.KylinJdbc41PreparedStatement;
+import com.kylinolap.jdbc.KylinMetaImpl.MetaCatalog;
+import com.kylinolap.jdbc.KylinMetaImpl.MetaColumn;
+import com.kylinolap.jdbc.KylinMetaImpl.MetaProject;
+import com.kylinolap.jdbc.KylinMetaImpl.MetaSchema;
+import com.kylinolap.jdbc.KylinMetaImpl.MetaTable;
+import com.kylinolap.jdbc.stub.TableMetaStub.ColumnMetaStub;
+import com.kylinolap.jdbc.util.DefaultSslProtocolSocketFactory;
+import com.kylinolap.jdbc.util.SQLTypeMap;
+
+/**
+ * @author xduo
+ * 
+ */
+public class KylinClient implements RemoteClient {
+    private static final Logger logger = LoggerFactory.getLogger(KylinClient.class);
+
+    private final KylinConnectionImpl conn;
+
+    public KylinClient(KylinConnectionImpl conn) {
+        this.conn = conn;
+    }
+
+    @Override
+    public void connect() throws ConnectionException {
+        PostMethod post = new PostMethod(conn.getConnectUrl());
+        HttpClient httpClient = new HttpClient();
+
+        if (conn.getConnectUrl().toLowerCase().startsWith("https://")) {
+            registerSsl();
+        }
+        addPostHeaders(post);
+
+        try {
+            StringRequestEntity requestEntity = new StringRequestEntity("{}", "application/json", "UTF-8");
+            post.setRequestEntity(requestEntity);
+            httpClient.executeMethod(post);
+
+            if (post.getStatusCode() != 200 && post.getStatusCode() != 201) {
+                logger.error("Authentication Failed with error code " + post.getStatusCode() + " and message:\n" + post.getResponseBodyAsString());
+
+                throw new ConnectionException("Authentication Failed.");
+            }
+        } catch (HttpException e) {
+            logger.error(e.getLocalizedMessage(), e);
+            throw new ConnectionException(e.getLocalizedMessage());
+        } catch (IOException e) {
+            logger.error(e.getLocalizedMessage(), e);
+            throw new ConnectionException(e.getLocalizedMessage());
+        }
+    }
+
+    @Override
+    public MetaProject getMetadata(String project) throws ConnectionException {
+        GetMethod get = new GetMethod(conn.getMetaProjectUrl(project));
+        HttpClient httpClient = new HttpClient();
+
+        if (conn.getConnectUrl().toLowerCase().startsWith("https://")) {
+            registerSsl();
+        }
+        addPostHeaders(get);
+
+        List<TableMetaStub> tableMetaStubs = null;
+        try {
+            httpClient.executeMethod(get);
+
+            if (get.getStatusCode() != 200 && get.getStatusCode() != 201) {
+                logger.error("Authentication Failed with error code " + get.getStatusCode() + " and message:\n" + get.getResponseBodyAsString());
+
+                throw new ConnectionException("Authentication Failed.");
+            }
+
+            tableMetaStubs = new ObjectMapper().readValue(get.getResponseBodyAsString(), new TypeReference<List<TableMetaStub>>() {
+            });
+
+            List<MetaTable> tables = new ArrayList<MetaTable>();
+            HashMultimap<String, MetaTable> schemasMap = HashMultimap.create();
+
+            for (TableMetaStub tableMetaStub : tableMetaStubs) {
+                List<MetaColumn> columns = new ArrayList<MetaColumn>();
+
+                for (ColumnMetaStub columnMetaStub : tableMetaStub.getColumns()) {
+                    MetaColumn column = createNewColumn(columnMetaStub);
+                    columns.add(column);
+                }
+
+                MetaTable table = createNewTable(tableMetaStub, columns);
+                tables.add(table);
+                schemasMap.put(tableMetaStub.getTABLE_CAT() + "#" + tableMetaStub.getTABLE_SCHEM(), table);
+            }
+
+            HashMultimap<String, MetaSchema> catalogMap = HashMultimap.create();
+            List<MetaSchema> schemas = new ArrayList<MetaSchema>();
+            for (String key : schemasMap.keySet()) {
+                String cat = key.split("#")[0];
+                String schema = key.split("#")[1];
+                MetaSchema metaSchema = new MetaSchema(cat, schema, new ArrayList<MetaTable>(schemasMap.get(key)));
+                schemas.add(metaSchema);
+                catalogMap.put(cat, metaSchema);
+            }
+
+            List<MetaCatalog> catalogs = new ArrayList<MetaCatalog>();
+            for (String key : catalogMap.keySet()) {
+                MetaCatalog metaCatalog = new MetaCatalog(key, new ArrayList<MetaSchema>(catalogMap.get(key)));
+                catalogs.add(metaCatalog);
+            }
+
+            return new MetaProject(project, catalogs);
+        } catch (HttpException e) {
+            logger.error(e.getLocalizedMessage(), e);
+            throw new ConnectionException(e.getLocalizedMessage());
+        } catch (IOException e) {
+            logger.error(e.getLocalizedMessage(), e);
+            throw new ConnectionException(e.getLocalizedMessage());
+        }
+    }
+
+    private MetaTable createNewTable(TableMetaStub tableMetaStub, List<MetaColumn> columns) {
+        MetaTable table = new MetaTable(tableMetaStub.getTABLE_CAT(), tableMetaStub.getTABLE_SCHEM(), tableMetaStub.getTABLE_NAME(), tableMetaStub.getTABLE_TYPE(), tableMetaStub.getREMARKS(), tableMetaStub.getTYPE_CAT(), tableMetaStub.getTYPE_SCHEM(), tableMetaStub.getTYPE_NAME(), tableMetaStub.getSELF_REFERENCING_COL_NAME(), tableMetaStub.getREF_GENERATION(), columns);
+        return table;
+    }
+
+    private MetaColumn createNewColumn(ColumnMetaStub columnMetaStub) {
+        MetaColumn column = new MetaColumn(columnMetaStub.getTABLE_CAT(), columnMetaStub.getTABLE_SCHEM(), columnMetaStub.getTABLE_NAME(), columnMetaStub.getCOLUMN_NAME(), columnMetaStub.getDATA_TYPE(), columnMetaStub.getTYPE_NAME(), columnMetaStub.getCOLUMN_SIZE(), columnMetaStub.getBUFFER_LENGTH(), columnMetaStub.getDECIMAL_DIGITS(), columnMetaStub.getNUM_PREC_RADIX(), columnMetaStub.getNULLABLE(), columnMetaStub.getREMARKS(), columnMetaStub.getCOLUMN_DEF(), columnMetaStub.getSQL_DATA_TYPE(), columnMetaStub.getSQL_DATETIME_SUB(), columnMetaStub.getCHAR_OCTET_LENGTH(), columnMetaStub.getORDINAL_POSITION(), columnMetaStub.getIS_NULLABLE(), columnMetaStub.getSCOPE_CATLOG(), columnMetaStub.getSCOPE_TABLE(), columnMetaStub.getSOURCE_DATA_TYPE(), columnMetaStub.getIS_AUTOINCREMENT(), columnMetaStub.getSCOPE_SCHEMA());
+        return column;
+    }
+
+    @Override
+    public DataSet<Object[]> query(AvaticaStatement statement, String sql) throws SQLException {
+        SQLResponseStub queryRes = null;
+
+        List<StateParam> params = null;
+        if (statement instanceof KylinJdbc41PreparedStatement) {
+            params = genPrestateStates(statement);
+        }
+
+        queryRes = runKylinQuery(sql, params);
+
+        List<ColumnMetaData> metas = genColumnMeta(queryRes);
+        List<Object[]> data = genResultData(queryRes, metas);
+
+        return new DataSet<Object[]>(metas, new KylinEnumerator<Object[]>(data));
+    }
+
+    /**
+     * @param queryRes
+     * @param metas
+     * @return
+     */
+    private List<Object[]> genResultData(SQLResponseStub queryRes, List<ColumnMetaData> metas) {
+        List<Object[]> data = new ArrayList<Object[]>();
+        for (String[] result : queryRes.getResults()) {
+            Object[] row = new Object[result.length];
+
+            for (int i = 0; i < result.length; i++) {
+                ColumnMetaData meta = metas.get(i);
+                row[i] = SQLTypeMap.wrapObject(result[i], meta.type.type);
+            }
+
+            data.add(row);
+        }
+        return data;
+    }
+
+    /**
+     * @param statement
+     * @param params
+     */
+    private List<StateParam> genPrestateStates(AvaticaStatement statement) {
+        List<StateParam> params = new ArrayList<StateParam>();
+        List<Object> values = ((KylinJdbc41PreparedStatement) statement).getParameterValues();
+
+        for (int i = 0; i < values.size(); i++) {
+            Object value = values.get(i);
+            params.add(new StateParam(value.getClass().getCanonicalName(), String.valueOf(value)));
+        }
+
+        return params;
+    }
+
+    /**
+     * @param queryRes
+     * @return
+     */
+    private List<ColumnMetaData> genColumnMeta(SQLResponseStub queryRes) {
+        List<ColumnMetaData> metas = new ArrayList<ColumnMetaData>();
+        for (int i = 0; i < queryRes.getColumnMetas().size(); i++) {
+            SQLResponseStub.ColumnMetaStub scm = queryRes.getColumnMetas().get(i);
+            ScalarType type = ColumnMetaData.scalar(scm.getColumnType(), scm.getColumnTypeName(), Rep.of(SQLTypeMap.convert(scm.getColumnType())));
+
+            ColumnMetaData meta = new ColumnMetaData(i, scm.isAutoIncrement(), scm.isCaseSensitive(), scm.isSearchable(), scm.isCurrency(), scm.getIsNullable(), scm.isSigned(), scm.getDisplaySize(), scm.getLabel(), scm.getName(), scm.getSchemaName(), scm.getPrecision(), scm.getScale(), scm.getTableName(), scm.getSchemaName(), type, scm.isReadOnly(), scm.isWritable(), scm.isWritable(), null);
+
+            metas.add(meta);
+        }
+
+        return metas;
+    }
+
+    /**
+     * @param sql
+     * @return
+     * @throws IOException
+     */
+    private SQLResponseStub runKylinQuery(String sql, List<StateParam> params) throws SQLException {
+        String url = conn.getQueryUrl();
+        String project = conn.getProject();
+        QueryRequest request = null;
+
+        if (null != params) {
+            request = new PreQueryRequest();
+            ((PreQueryRequest) request).setParams(params);
+            url += "/prestate";
+        } else {
+            request = new QueryRequest();
+        }
+        request.setSql(sql);
+        request.setProject(project);
+
+        PostMethod post = new PostMethod(url);
+        addPostHeaders(post);
+        HttpClient httpClient = new HttpClient();
+        if (conn.getQueryUrl().toLowerCase().startsWith("https://")) {
+            registerSsl();
+        }
+
+        String postBody = null;
+        ObjectMapper mapper = new ObjectMapper();
+        try {
+            postBody = mapper.writeValueAsString(request);
+            logger.debug("Post body:\n " + postBody);
+        } catch (JsonProcessingException e) {
+            logger.error(e.getLocalizedMessage(), e);
+        }
+        String response = null;
+        SQLResponseStub queryRes = null;
+
+        try {
+            StringRequestEntity requestEntity = new StringRequestEntity(postBody, "application/json", "UTF-8");
+            post.setRequestEntity(requestEntity);
+
+            httpClient.executeMethod(post);
+            response = post.getResponseBodyAsString();
+
+            if (post.getStatusCode() != 200 && post.getStatusCode() != 201) {
+                logger.error("Failed to query", response);
+                throw new SQLException(response);
+            }
+
+            queryRes = new ObjectMapper().readValue(response, SQLResponseStub.class);
+
+        } catch (HttpException e) {
+            logger.error(e.getLocalizedMessage(), e);
+            throw new SQLException(e.getLocalizedMessage());
+        } catch (IOException e) {
+            logger.error(e.getLocalizedMessage(), e);
+            throw new SQLException(e.getLocalizedMessage());
+        }
+
+        return queryRes;
+    }
+
+    private void addPostHeaders(HttpMethodBase method) {
+        method.addRequestHeader("Accept", "application/json, text/plain, */*");
+        method.addRequestHeader("Content-Type", "application/json");
+        method.addRequestHeader("Authorization", "Basic " + conn.getBasicAuthHeader());
+    }
+
+    private void registerSsl() {
+        Protocol.registerProtocol("https", new Protocol("https", (ProtocolSocketFactory) new DefaultSslProtocolSocketFactory(), 443));
+    }
+
+    public class QueryRequest {
+        private String sql;
+        private String project;
+
+        public String getSql() {
+            return sql;
+        }
+
+        public void setSql(String sql) {
+            this.sql = sql;
+        }
+
+        public String getProject() {
+            return project;
+        }
+
+        public void setProject(String project) {
+            this.project = project;
+        }
+    }
+
+    public class PreQueryRequest extends QueryRequest {
+        private List<StateParam> params;
+
+        public List<StateParam> getParams() {
+            return params;
+        }
+
+        public void setParams(List<StateParam> params) {
+            this.params = params;
+        }
+    }
+
+    public class StateParam {
+        private String className;
+        private String value;
+
+        public StateParam(String className, String value) {
+            super();
+            this.className = className;
+            this.value = value;
+        }
+
+        public String getClassName() {
+            return className;
+        }
+
+        public void setClazz(String className) {
+            this.className = className;
+        }
+
+        public String getValue() {
+            return value;
+        }
+
+        public void setValue(String value) {
+            this.value = value;
+        }
+    }
+}


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

Posted by lu...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/examples/test_case_data/localmeta/table_snapshot/TEST_SELLER_TYPE_DIM.csv/b43dd3f1-9a34-4721-8abc-df90962e94d8.snapshot
----------------------------------------------------------------------
diff --git a/examples/test_case_data/localmeta/table_snapshot/TEST_SELLER_TYPE_DIM.csv/b43dd3f1-9a34-4721-8abc-df90962e94d8.snapshot b/examples/test_case_data/localmeta/table_snapshot/TEST_SELLER_TYPE_DIM.csv/b43dd3f1-9a34-4721-8abc-df90962e94d8.snapshot
new file mode 100644
index 0000000..d4bc695
Binary files /dev/null and b/examples/test_case_data/localmeta/table_snapshot/TEST_SELLER_TYPE_DIM.csv/b43dd3f1-9a34-4721-8abc-df90962e94d8.snapshot differ

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/examples/test_case_data/localmeta/table_snapshot/TEST_SELLER_TYPE_DIM.csv/d1b6a60a-14d4-4653-92f7-c96d01191c75.snapshot
----------------------------------------------------------------------
diff --git a/examples/test_case_data/localmeta/table_snapshot/TEST_SELLER_TYPE_DIM.csv/d1b6a60a-14d4-4653-92f7-c96d01191c75.snapshot b/examples/test_case_data/localmeta/table_snapshot/TEST_SELLER_TYPE_DIM.csv/d1b6a60a-14d4-4653-92f7-c96d01191c75.snapshot
new file mode 100644
index 0000000..07d7965
Binary files /dev/null and b/examples/test_case_data/localmeta/table_snapshot/TEST_SELLER_TYPE_DIM.csv/d1b6a60a-14d4-4653-92f7-c96d01191c75.snapshot differ

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/examples/test_case_data/localmeta/table_snapshot/TEST_SELLER_TYPE_DIM.csv/f818054b-1a76-4cbd-bdd8-7871a240360c.snapshot
----------------------------------------------------------------------
diff --git a/examples/test_case_data/localmeta/table_snapshot/TEST_SELLER_TYPE_DIM.csv/f818054b-1a76-4cbd-bdd8-7871a240360c.snapshot b/examples/test_case_data/localmeta/table_snapshot/TEST_SELLER_TYPE_DIM.csv/f818054b-1a76-4cbd-bdd8-7871a240360c.snapshot
new file mode 100644
index 0000000..140962f
Binary files /dev/null and b/examples/test_case_data/localmeta/table_snapshot/TEST_SELLER_TYPE_DIM.csv/f818054b-1a76-4cbd-bdd8-7871a240360c.snapshot differ

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/examples/test_case_data/localmeta/table_snapshot/TEST_SITES.csv/1271e1f2-dd65-4249-a1f2-aca02a651158.snapshot
----------------------------------------------------------------------
diff --git a/examples/test_case_data/localmeta/table_snapshot/TEST_SITES.csv/1271e1f2-dd65-4249-a1f2-aca02a651158.snapshot b/examples/test_case_data/localmeta/table_snapshot/TEST_SITES.csv/1271e1f2-dd65-4249-a1f2-aca02a651158.snapshot
new file mode 100644
index 0000000..8b405f4
Binary files /dev/null and b/examples/test_case_data/localmeta/table_snapshot/TEST_SITES.csv/1271e1f2-dd65-4249-a1f2-aca02a651158.snapshot differ

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/examples/test_case_data/localmeta/table_snapshot/TEST_SITES.csv/1888015f-f059-4ccc-8324-0cdd70b46673.snapshot
----------------------------------------------------------------------
diff --git a/examples/test_case_data/localmeta/table_snapshot/TEST_SITES.csv/1888015f-f059-4ccc-8324-0cdd70b46673.snapshot b/examples/test_case_data/localmeta/table_snapshot/TEST_SITES.csv/1888015f-f059-4ccc-8324-0cdd70b46673.snapshot
new file mode 100644
index 0000000..2fbd6c5
Binary files /dev/null and b/examples/test_case_data/localmeta/table_snapshot/TEST_SITES.csv/1888015f-f059-4ccc-8324-0cdd70b46673.snapshot differ

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/examples/test_case_data/localmeta/table_snapshot/TEST_SITES.csv/28130338-fcf4-429e-91b0-cd8dfd397280.snapshot
----------------------------------------------------------------------
diff --git a/examples/test_case_data/localmeta/table_snapshot/TEST_SITES.csv/28130338-fcf4-429e-91b0-cd8dfd397280.snapshot b/examples/test_case_data/localmeta/table_snapshot/TEST_SITES.csv/28130338-fcf4-429e-91b0-cd8dfd397280.snapshot
new file mode 100644
index 0000000..ef5c7fd
Binary files /dev/null and b/examples/test_case_data/localmeta/table_snapshot/TEST_SITES.csv/28130338-fcf4-429e-91b0-cd8dfd397280.snapshot differ

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/examples/test_case_data/localmeta/table_snapshot/TEST_SITES.csv/7d70b82e-43f6-4999-b012-0e91a9bb8408.snapshot
----------------------------------------------------------------------
diff --git a/examples/test_case_data/localmeta/table_snapshot/TEST_SITES.csv/7d70b82e-43f6-4999-b012-0e91a9bb8408.snapshot b/examples/test_case_data/localmeta/table_snapshot/TEST_SITES.csv/7d70b82e-43f6-4999-b012-0e91a9bb8408.snapshot
new file mode 100644
index 0000000..ce75b97
Binary files /dev/null and b/examples/test_case_data/localmeta/table_snapshot/TEST_SITES.csv/7d70b82e-43f6-4999-b012-0e91a9bb8408.snapshot differ

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/examples/test_case_data/localmeta/table_snapshot/TEST_SITES.csv/d95e0b5b-a399-4f08-8f40-b0fa53b565e0.snapshot
----------------------------------------------------------------------
diff --git a/examples/test_case_data/localmeta/table_snapshot/TEST_SITES.csv/d95e0b5b-a399-4f08-8f40-b0fa53b565e0.snapshot b/examples/test_case_data/localmeta/table_snapshot/TEST_SITES.csv/d95e0b5b-a399-4f08-8f40-b0fa53b565e0.snapshot
new file mode 100644
index 0000000..19fe6a5
Binary files /dev/null and b/examples/test_case_data/localmeta/table_snapshot/TEST_SITES.csv/d95e0b5b-a399-4f08-8f40-b0fa53b565e0.snapshot differ

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/examples/test_case_data/sandbox/capacity-scheduler.xml
----------------------------------------------------------------------
diff --git a/examples/test_case_data/sandbox/capacity-scheduler.xml b/examples/test_case_data/sandbox/capacity-scheduler.xml
new file mode 100644
index 0000000..cddca6c
--- /dev/null
+++ b/examples/test_case_data/sandbox/capacity-scheduler.xml
@@ -0,0 +1,111 @@
+<!--
+  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. See accompanying LICENSE file.
+-->
+<configuration>
+
+  <property>
+    <name>yarn.scheduler.capacity.maximum-applications</name>
+    <value>10000</value>
+    <description>
+      Maximum number of applications that can be pending and running.
+    </description>
+  </property>
+
+  <property>
+    <name>yarn.scheduler.capacity.maximum-am-resource-percent</name>
+    <value>0.1</value>
+    <description>
+      Maximum percent of resources in the cluster which can be used to run 
+      application masters i.e. controls number of concurrent running
+      applications.
+    </description>
+  </property>
+
+  <property>
+    <name>yarn.scheduler.capacity.resource-calculator</name>
+    <value>org.apache.hadoop.yarn.util.resource.DefaultResourceCalculator</value>
+    <description>
+      The ResourceCalculator implementation to be used to compare 
+      Resources in the scheduler.
+      The default i.e. DefaultResourceCalculator only uses Memory while
+      DominantResourceCalculator uses dominant-resource to compare 
+      multi-dimensional resources such as Memory, CPU etc.
+    </description>
+  </property>
+
+  <property>
+    <name>yarn.scheduler.capacity.root.queues</name>
+    <value>default</value>
+    <description>
+      The queues at the this level (root is the root queue).
+    </description>
+  </property>
+
+  <property>
+    <name>yarn.scheduler.capacity.root.default.capacity</name>
+    <value>100</value>
+    <description>Default queue target capacity.</description>
+  </property>
+
+  <property>
+    <name>yarn.scheduler.capacity.root.default.user-limit-factor</name>
+    <value>1</value>
+    <description>
+      Default queue user limit a percentage from 0.0 to 1.0.
+    </description>
+  </property>
+
+  <property>
+    <name>yarn.scheduler.capacity.root.default.maximum-capacity</name>
+    <value>100</value>
+    <description>
+      The maximum capacity of the default queue. 
+    </description>
+  </property>
+
+  <property>
+    <name>yarn.scheduler.capacity.root.default.state</name>
+    <value>RUNNING</value>
+    <description>
+      The state of the default queue. State can be one of RUNNING or STOPPED.
+    </description>
+  </property>
+
+  <property>
+    <name>yarn.scheduler.capacity.root.default.acl_submit_applications</name>
+    <value>*</value>
+    <description>
+      The ACL of who can submit jobs to the default queue.
+    </description>
+  </property>
+
+  <property>
+    <name>yarn.scheduler.capacity.root.default.acl_administer_queue</name>
+    <value>*</value>
+    <description>
+      The ACL of who can administer jobs on the default queue.
+    </description>
+  </property>
+
+  <property>
+    <name>yarn.scheduler.capacity.node-locality-delay</name>
+    <value>-1</value>
+    <description>
+      Number of missed scheduling opportunities after which the CapacityScheduler 
+      attempts to schedule rack-local containers. 
+      Typically this should be set to number of racks in the cluster, this 
+      feature is disabled by default, set to -1.
+    </description>
+  </property>
+
+</configuration>

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/examples/test_case_data/sandbox/core-site.xml
----------------------------------------------------------------------
diff --git a/examples/test_case_data/sandbox/core-site.xml b/examples/test_case_data/sandbox/core-site.xml
new file mode 100644
index 0000000..c697835
--- /dev/null
+++ b/examples/test_case_data/sandbox/core-site.xml
@@ -0,0 +1,93 @@
+<!--Mon Feb 10 22:50:51 2014-->
+  <configuration>
+    <property>
+    <name>hadoop.proxyuser.hcat.groups</name>
+    <value>*</value>
+  </property>
+    <property>
+    <name>hadoop.proxyuser.hcat.hosts</name>
+    <value>*</value>
+  </property>
+    <property>
+    <name>fs.trash.interval</name>
+    <value>360</value>
+  </property>
+    <property>
+    <name>io.compression.codecs</name>
+    <value>org.apache.hadoop.io.compress.GzipCodec,org.apache.hadoop.io.compress.DefaultCodec</value>
+  </property>
+    <property>
+    <name>hadoop.proxyuser.hive.groups</name>
+    <value>*</value>
+  </property>
+    <property>
+    <name>hadoop.proxyuser.hue.hosts</name>
+    <value>*</value>
+  </property>
+    <property>
+    <name>io.file.buffer.size</name>
+    <value>131072</value>
+  </property>
+    <property>
+    <name>ipc.client.connection.maxidletime</name>
+    <value>30000</value>
+  </property>
+    <property>
+    <name>mapreduce.jobtracker.webinterface.trusted</name>
+    <value>false</value>
+  </property>
+    <property>
+    <name>hadoop.security.authorization</name>
+    <value>false</value>
+  </property>
+    <property>
+    <name>hadoop.proxyuser.oozie.groups</name>
+    <value>*</value>
+  </property>
+    <property>
+    <name>hadoop.security.auth_to_local</name>
+    <value>
+        RULE:[2:$1@$0]([rn]m@.*)s/.*/yarn/
+        RULE:[2:$1@$0](jhs@.*)s/.*/mapred/
+        RULE:[2:$1@$0]([nd]n@.*)s/.*/hdfs/
+        RULE:[2:$1@$0](hm@.*)s/.*/hbase/
+        RULE:[2:$1@$0](rs@.*)s/.*/hbase/
+        DEFAULT</value>
+  </property>
+    <property>
+    <name>hadoop.security.authentication</name>
+    <value>simple</value>
+  </property>
+    <property>
+    <name>ipc.client.idlethreshold</name>
+    <value>8000</value>
+  </property>
+    <property>
+    <name>fs.checkpoint.size</name>
+    <value>0.5</value>
+  </property>
+    <property>
+    <name>fs.defaultFS</name>
+    <value>hdfs://sandbox.hortonworks.com:8020</value>
+  </property>
+    <property>
+    <name>hadoop.proxyuser.oozie.hosts</name>
+    <value>*</value>
+  </property>
+    <property>
+    <name>hadoop.proxyuser.hive.hosts</name>
+    <value>*</value>
+  </property>
+    <property>
+    <name>io.serializations</name>
+    <value>org.apache.hadoop.io.serializer.WritableSerialization</value>
+  </property>
+    <property>
+    <name>hadoop.proxyuser.hue.groups</name>
+    <value>*</value>
+  </property>
+    <property>
+    <name>ipc.client.connect.max.retries</name>
+    <value>50</value>
+  </property>
+  </configuration>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/examples/test_case_data/sandbox/hadoop-policy.xml
----------------------------------------------------------------------
diff --git a/examples/test_case_data/sandbox/hadoop-policy.xml b/examples/test_case_data/sandbox/hadoop-policy.xml
new file mode 100644
index 0000000..1bcd78a
--- /dev/null
+++ b/examples/test_case_data/sandbox/hadoop-policy.xml
@@ -0,0 +1,219 @@
+<?xml version="1.0"?>
+<?xml-stylesheet type="text/xsl" href="configuration.xsl"?>
+<!--
+
+ Copyright 2011 The Apache Software Foundation
+ 
+ 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.
+
+-->
+
+<!-- Put site-specific property overrides in this file. -->
+
+<configuration>
+  <property>
+    <name>security.client.protocol.acl</name>
+    <value>*</value>
+    <description>ACL for ClientProtocol, which is used by user code
+    via the DistributedFileSystem.
+    The ACL is a comma-separated list of user and group names. The user and
+    group list is separated by a blank. For e.g. "alice,bob users,wheel".
+    A special value of "*" means all users are allowed.</description>
+  </property>
+
+  <property>
+    <name>security.client.datanode.protocol.acl</name>
+    <value>*</value>
+    <description>ACL for ClientDatanodeProtocol, the client-to-datanode protocol
+    for block recovery.
+    The ACL is a comma-separated list of user and group names. The user and
+    group list is separated by a blank. For e.g. "alice,bob users,wheel".
+    A special value of "*" means all users are allowed.</description>
+  </property>
+
+  <property>
+    <name>security.datanode.protocol.acl</name>
+    <value>*</value>
+    <description>ACL for DatanodeProtocol, which is used by datanodes to
+    communicate with the namenode.
+    The ACL is a comma-separated list of user and group names. The user and
+    group list is separated by a blank. For e.g. "alice,bob users,wheel".
+    A special value of "*" means all users are allowed.</description>
+  </property>
+
+  <property>
+    <name>security.inter.datanode.protocol.acl</name>
+    <value>*</value>
+    <description>ACL for InterDatanodeProtocol, the inter-datanode protocol
+    for updating generation timestamp.
+    The ACL is a comma-separated list of user and group names. The user and
+    group list is separated by a blank. For e.g. "alice,bob users,wheel".
+    A special value of "*" means all users are allowed.</description>
+  </property>
+
+  <property>
+    <name>security.namenode.protocol.acl</name>
+    <value>*</value>
+    <description>ACL for NamenodeProtocol, the protocol used by the secondary
+    namenode to communicate with the namenode.
+    The ACL is a comma-separated list of user and group names. The user and
+    group list is separated by a blank. For e.g. "alice,bob users,wheel".
+    A special value of "*" means all users are allowed.</description>
+  </property>
+
+ <property>
+    <name>security.admin.operations.protocol.acl</name>
+    <value>*</value>
+    <description>ACL for AdminOperationsProtocol. Used for admin commands.
+    The ACL is a comma-separated list of user and group names. The user and
+    group list is separated by a blank. For e.g. "alice,bob users,wheel".
+    A special value of "*" means all users are allowed.</description>
+  </property>
+
+  <property>
+    <name>security.refresh.usertogroups.mappings.protocol.acl</name>
+    <value>*</value>
+    <description>ACL for RefreshUserMappingsProtocol. Used to refresh
+    users mappings. The ACL is a comma-separated list of user and
+    group names. The user and group list is separated by a blank. For
+    e.g. "alice,bob users,wheel".  A special value of "*" means all
+    users are allowed.</description>
+  </property>
+
+  <property>
+    <name>security.refresh.policy.protocol.acl</name>
+    <value>*</value>
+    <description>ACL for RefreshAuthorizationPolicyProtocol, used by the
+    dfsadmin and mradmin commands to refresh the security policy in-effect.
+    The ACL is a comma-separated list of user and group names. The user and
+    group list is separated by a blank. For e.g. "alice,bob users,wheel".
+    A special value of "*" means all users are allowed.</description>
+  </property>
+
+  <property>
+    <name>security.ha.service.protocol.acl</name>
+    <value>*</value>
+    <description>ACL for HAService protocol used by HAAdmin to manage the
+      active and stand-by states of namenode.</description>
+  </property>
+
+  <property>
+    <name>security.zkfc.protocol.acl</name>
+    <value>*</value>
+    <description>ACL for access to the ZK Failover Controller
+    </description>
+  </property>
+
+  <property>
+    <name>security.qjournal.service.protocol.acl</name>
+    <value>*</value>
+    <description>ACL for QJournalProtocol, used by the NN to communicate with
+    JNs when using the QuorumJournalManager for edit logs.</description>
+  </property>
+
+  <property>
+    <name>security.mrhs.client.protocol.acl</name>
+    <value>*</value>
+    <description>ACL for HSClientProtocol, used by job clients to
+    communciate with the MR History Server job status etc. 
+    The ACL is a comma-separated list of user and group names. The user and
+    group list is separated by a blank. For e.g. "alice,bob users,wheel".
+    A special value of "*" means all users are allowed.</description>
+  </property>
+
+  <!-- YARN Protocols -->
+
+  <property>
+    <name>security.resourcetracker.protocol.acl</name>
+    <value>*</value>
+    <description>ACL for ResourceTrackerProtocol, used by the
+    ResourceManager and NodeManager to communicate with each other.
+    The ACL is a comma-separated list of user and group names. The user and
+    group list is separated by a blank. For e.g. "alice,bob users,wheel".
+    A special value of "*" means all users are allowed.</description>
+  </property>
+
+  <property>
+    <name>security.resourcemanager-administration.protocol.acl</name>
+    <value>*</value>
+    <description>ACL for ResourceManagerAdministrationProtocol, for admin commands. 
+    The ACL is a comma-separated list of user and group names. The user and
+    group list is separated by a blank. For e.g. "alice,bob users,wheel".
+    A special value of "*" means all users are allowed.</description>
+  </property>
+
+  <property>
+    <name>security.applicationclient.protocol.acl</name>
+    <value>*</value>
+    <description>ACL for ApplicationClientProtocol, used by the ResourceManager 
+    and applications submission clients to communicate with each other.
+    The ACL is a comma-separated list of user and group names. The user and
+    group list is separated by a blank. For e.g. "alice,bob users,wheel".
+    A special value of "*" means all users are allowed.</description>
+  </property>
+
+  <property>
+    <name>security.applicationmaster.protocol.acl</name>
+    <value>*</value>
+    <description>ACL for ApplicationMasterProtocol, used by the ResourceManager 
+    and ApplicationMasters to communicate with each other.
+    The ACL is a comma-separated list of user and group names. The user and
+    group list is separated by a blank. For e.g. "alice,bob users,wheel".
+    A special value of "*" means all users are allowed.</description>
+  </property>
+
+  <property>
+    <name>security.containermanagement.protocol.acl</name>
+    <value>*</value>
+    <description>ACL for ContainerManagementProtocol protocol, used by the NodeManager 
+    and ApplicationMasters to communicate with each other.
+    The ACL is a comma-separated list of user and group names. The user and
+    group list is separated by a blank. For e.g. "alice,bob users,wheel".
+    A special value of "*" means all users are allowed.</description>
+  </property>
+
+  <property>
+    <name>security.resourcelocalizer.protocol.acl</name>
+    <value>*</value>
+    <description>ACL for ResourceLocalizer protocol, used by the NodeManager 
+    and ResourceLocalizer to communicate with each other.
+    The ACL is a comma-separated list of user and group names. The user and
+    group list is separated by a blank. For e.g. "alice,bob users,wheel".
+    A special value of "*" means all users are allowed.</description>
+  </property>
+
+  <property>
+    <name>security.job.task.protocol.acl</name>
+    <value>*</value>
+    <description>ACL for TaskUmbilicalProtocol, used by the map and reduce
+    tasks to communicate with the parent tasktracker.
+    The ACL is a comma-separated list of user and group names. The user and
+    group list is separated by a blank. For e.g. "alice,bob users,wheel".
+    A special value of "*" means all users are allowed.</description>
+  </property>
+
+  <property>
+    <name>security.job.client.protocol.acl</name>
+    <value>*</value>
+    <description>ACL for MRClientProtocol, used by job clients to
+    communciate with the MR ApplicationMaster to query job status etc. 
+    The ACL is a comma-separated list of user and group names. The user and
+    group list is separated by a blank. For e.g. "alice,bob users,wheel".
+    A special value of "*" means all users are allowed.</description>
+  </property>
+
+</configuration>

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/examples/test_case_data/sandbox/hbase-site.xml
----------------------------------------------------------------------
diff --git a/examples/test_case_data/sandbox/hbase-site.xml b/examples/test_case_data/sandbox/hbase-site.xml
new file mode 100644
index 0000000..aafb16e
--- /dev/null
+++ b/examples/test_case_data/sandbox/hbase-site.xml
@@ -0,0 +1,179 @@
+<configuration>
+
+	<property>
+		<name>hbase.master.info.bindAddress</name>
+		<value>0.0.0.0</value>
+	</property>
+
+	<property>
+		<name>hbase.hstore.flush.retries.number</name>
+		<value>120</value>
+	</property>
+
+	<property>
+		<name>hbase.client.keyvalue.maxsize</name>
+		<value>10485760</value>
+	</property>
+
+	<property>
+		<name>hbase.regionserver.keytab.file</name>
+		<value>/etc/security/keytabs/hbase.service.keytab</value>
+	</property>
+
+	<property>
+		<name>hbase.hstore.compactionThreshold</name>
+		<value>3</value>
+	</property>
+
+	<property>
+		<name>hbase.zookeeper.property.clientPort</name>
+		<value>2181</value>
+	</property>
+
+	<property>
+		<name>hbase.regionserver.handler.count</name>
+		<value>60</value>
+	</property>
+
+	<property>
+		<name>hbase.regionserver.global.memstore.lowerLimit</name>
+		<value>0.38</value>
+	</property>
+
+	<property>
+		<name>hbase.master.kerberos.principal</name>
+		<value>hbase/_HOST@EXAMPLE.COM</value>
+	</property>
+
+	<property>
+		<name>hbase.hregion.memstore.block.multiplier</name>
+		<value>2</value>
+	</property>
+
+	<property>
+		<name>hbase.hregion.memstore.flush.size</name>
+		<value>134217728</value>
+	</property>
+
+	<property>
+		<name>hbase.superuser</name>
+		<value>hbase</value>
+	</property>
+
+	<property>
+		<name>hbase.rootdir</name>
+		<value>hdfs://sandbox.hortonworks.com:8020/apps/hbase/data
+		</value>
+	</property>
+
+	<property>
+		<name>hbase.regionserver.kerberos.principal</name>
+		<value>hbase/_HOST@EXAMPLE.COM</value>
+	</property>
+
+	<property>
+		<name>hbase.regionserver.global.memstore.upperLimit</name>
+		<value>0.4</value>
+	</property>
+
+	<property>
+		<name>zookeeper.session.timeout</name>
+		<value>30000</value>
+	</property>
+
+	<property>
+		<name>hbase.client.scanner.caching</name>
+		<value>100</value>
+	</property>
+
+	<property>
+		<name>hbase.tmp.dir</name>
+		<value>/tmp/hbase</value>
+	</property>
+
+	<property>
+		<name>hfile.block.cache.size</name>
+		<value>0.40</value>
+	</property>
+
+	<property>
+		<name>hbase.hregion.max.filesize</name>
+		<value>10737418240</value>
+	</property>
+
+	<property>
+		<name>hbase.security.authentication</name>
+		<value>simple</value>
+	</property>
+
+	<property>
+		<name>hbase.defaults.for.version.skip</name>
+		<value>true</value>
+	</property>
+
+	<property>
+		<name>hbase.master.info.port</name>
+		<value>60010</value>
+	</property>
+
+	<property>
+		<name>hbase.zookeeper.quorum</name>
+		<value>sandbox.hortonworks.com</value>
+	</property>
+
+	<property>
+		<name>hbase.regionserver.info.port</name>
+		<value>60030</value>
+	</property>
+
+	<property>
+		<name>zookeeper.znode.parent</name>
+		<value>/hbase-unsecure</value>
+	</property>
+
+	<property>
+		<name>hbase.hstore.blockingStoreFiles</name>
+		<value>10</value>
+	</property>
+
+	<property>
+		<name>hbase.hregion.majorcompaction</name>
+		<value>86400000</value>
+	</property>
+
+	<property>
+		<name>hbase.security.authorization</name>
+		<value>false</value>
+	</property>
+
+	<property>
+		<name>hbase.master.keytab.file</name>
+		<value>/etc/security/keytabs/hbase.service.keytab</value>
+	</property>
+
+	<property>
+		<name>hbase.local.dir</name>
+		<value>${hbase.tmp.dir}/local</value>
+	</property>
+
+	<property>
+		<name>hbase.cluster.distributed</name>
+		<value>true</value>
+	</property>
+
+	<property>
+		<name>hbase.hregion.memstore.mslab.enabled</name>
+		<value>true</value>
+	</property>
+
+	<property>
+		<name>dfs.domain.socket.path</name>
+		<value>/var/lib/hadoop-hdfs/dn_socket</value>
+	</property>
+
+	<property>
+		<name>hbase.zookeeper.useMulti</name>
+		<value>true</value>
+	</property>
+
+</configuration>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/examples/test_case_data/sandbox/hdfs-site.xml
----------------------------------------------------------------------
diff --git a/examples/test_case_data/sandbox/hdfs-site.xml b/examples/test_case_data/sandbox/hdfs-site.xml
new file mode 100644
index 0000000..6150416
--- /dev/null
+++ b/examples/test_case_data/sandbox/hdfs-site.xml
@@ -0,0 +1,175 @@
+<!--Mon Feb 10 22:50:51 2014-->
+  <configuration>
+    <property>
+    <name>dfs.namenode.stale.datanode.interval</name>
+    <value>30000</value>
+  </property>
+    <property>
+    <name>dfs.https.port</name>
+    <value>50470</value>
+  </property>
+    <property>
+    <name>dfs.datanode.address</name>
+    <value>0.0.0.0:50010</value>
+  </property>
+    <property>
+    <name>dfs.cluster.administrators</name>
+    <value> hdfs</value>
+  </property>
+    <property>
+    <name>dfs.namenode.checkpoint.dir</name>
+    <value>/hadoop/hdfs/namesecondary</value>
+  </property>
+    <property>
+    <name>dfs.client.block.write.replace-datanode-on-failure.policy</name>
+    <value>NEVER</value>
+  </property>
+    <property>
+    <name>dfs.client.read.shortcircuit</name>
+    <value>true</value>
+  </property>
+    <property>
+    <name>dfs.blockreport.initialDelay</name>
+    <value>120</value>
+  </property>
+    <property>
+    <name>dfs.namenode.avoid.read.stale.datanode</name>
+    <value>true</value>
+  </property>
+    <property>
+    <name>dfs.namenode.checkpoint.period</name>
+    <value>21600</value>
+  </property>
+    <property>
+    <name>dfs.namenode.secondary.http-address</name>
+    <value>sandbox.hortonworks.com:50090</value>
+  </property>
+    <property>
+    <name>dfs.support.append</name>
+    <value>true</value>
+  </property>
+    <property>
+    <name>dfs.heartbeat.interval</name>
+    <value>3</value>
+  </property>
+    <property>
+    <name>dfs.datanode.http.address</name>
+    <value>0.0.0.0:50075</value>
+  </property>
+    <property>
+    <name>dfs.replication</name>
+    <value>3</value>
+  </property>
+    <property>
+    <name>dfs.datanode.du.reserved</name>
+    <value>1073741824</value>
+  </property>
+    <property>
+    <name>dfs.permissions.superusergroup</name>
+    <value>hdfs</value>
+  </property>
+    <property>
+    <name>dfs.datanode.balance.bandwidthPerSec</name>
+    <value>6250000</value>
+  </property>
+    <property>
+    <name>dfs.blocksize</name>
+    <value>134217728</value>
+  </property>
+    <property>
+    <name>dfs.namenode.name.dir</name>
+    <value>/hadoop/hdfs/namenode</value>
+  </property>
+    <property>
+    <name>dfs.namenode.checkpoint.edits.dir</name>
+    <value>${dfs.namenode.checkpoint.dir}</value>
+  </property>
+    <property>
+    <name>dfs.namenode.accesstime.precision</name>
+    <value>0</value>
+  </property>
+    <property>
+    <name>dfs.datanode.ipc.address</name>
+    <value>0.0.0.0:8010</value>
+  </property>
+    <property>
+    <name>dfs.datanode.data.dir.perm</name>
+    <value>750</value>
+  </property>
+    <property>
+    <name>dfs.webhdfs.enabled</name>
+    <value>true</value>
+  </property>
+    <property>
+    <name>dfs.namenode.handler.count</name>
+    <value>5</value>
+  </property>
+    <property>
+    <name>dfs.client.read.shortcircuit.streams.cache.size</name>
+    <value>4096</value>
+  </property>
+    <property>
+    <name>dfs.replication.max</name>
+    <value>50</value>
+  </property>
+    <property>
+    <name>dfs.namenode.write.stale.datanode.ratio</name>
+    <value>1.0f</value>
+  </property>
+    <property>
+    <name>dfs.datanode.max.transfer.threads</name>
+    <value>1024</value>
+  </property>
+    <property>
+    <name>dfs.namenode.https-address</name>
+    <value>sandbox.hortonworks.com:50470</value>
+  </property>
+  <property>
+    <name>dfs.permissions</name>
+    <value>false</value>
+  </property>
+    <property>
+    <name>dfs.domain.socket.path</name>
+    <value>/var/lib/hadoop-hdfs/dn_socket</value>
+  </property>
+    <property>
+    <name>dfs.namenode.avoid.write.stale.datanode</name>
+    <value>true</value>
+  </property>
+    <property>
+    <name>dfs.block.access.token.enable</name>
+    <value>true</value>
+  </property>
+    <property>
+    <name>dfs.hosts.exclude</name>
+    <value>/etc/hadoop/conf/dfs.exclude</value>
+  </property>
+    <property>
+    <name>dfs.journalnode.http-address</name>
+    <value>0.0.0.0:8480</value>
+  </property>
+    <property>
+    <name>fs.permissions.umask-mode</name>
+    <value>022</value>
+  </property>
+    <property>
+    <name>dfs.datanode.data.dir</name>
+    <value>/hadoop/hdfs/data</value>
+  </property>
+    <property>
+    <name>dfs.journalnode.edits.dir</name>
+    <value>/grid/0/hdfs/journal</value>
+  </property>
+    <property>
+    <name>dfs.namenode.safemode.threshold-pct</name>
+    <value>1.0f</value>
+  </property>
+    <property>
+    <name>dfs.namenode.http-address</name>
+    <value>sandbox.hortonworks.com:50070</value>
+  </property>
+    <property>
+    <name>dfs.datanode.failed.volumes.tolerated</name>
+    <value>0</value>
+  </property>
+  </configuration>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/examples/test_case_data/sandbox/httpfs-site.xml
----------------------------------------------------------------------
diff --git a/examples/test_case_data/sandbox/httpfs-site.xml b/examples/test_case_data/sandbox/httpfs-site.xml
new file mode 100644
index 0000000..f27544f
--- /dev/null
+++ b/examples/test_case_data/sandbox/httpfs-site.xml
@@ -0,0 +1,17 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+  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.
+-->
+<configuration>
+
+</configuration>

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/examples/test_case_data/sandbox/kylin.properties
----------------------------------------------------------------------
diff --git a/examples/test_case_data/sandbox/kylin.properties b/examples/test_case_data/sandbox/kylin.properties
new file mode 100644
index 0000000..0675b21
--- /dev/null
+++ b/examples/test_case_data/sandbox/kylin.properties
@@ -0,0 +1,75 @@
+## Config for Kylin Engine ##
+
+# List of web servers in use, this enables one web server instance to sync up with other servers.
+kylin.rest.servers=sandbox.hortonworks.com:7070
+
+# The metadata store in hbase
+kylin.metadata.url=kylin_metadata_qa@hbase:sandbox.hortonworks.com:2181:/hbase-unsecure
+
+# The storage for final cube file in hbase
+kylin.storage.url=hbase:sandbox.hortonworks.com:2181:/hbase-unsecure
+
+# Path to the local(relative to job engine) coprocessor jar, job engine will upload this file to hbase
+kylin.coprocessor.local.jar=/tmp/kylin/kylin-storage-latest-coprocessor.jar
+
+# Temp folder in hdfs
+kylin.hdfs.working.dir=/tmp
+
+# Path to the local(relative to job engine) job jar, job engine will use this jar
+kylin.job.jar=/tmp/kylin/kylin-job-latest-job.jar
+
+kylin.job.mapreduce.default.reduce.input.mb=500
+
+# If true, job engine will not assume that hadoop CLI reside on the same server as it self
+# you will have to specify kylin.job.remote.cli.hostname, kylin.job.remote.cli.username and kylin.job.remote.cli.password
+kylin.job.run.as.remote.cmd=true
+
+# Only necessary when kylin.job.run.as.remote.cmd=true
+kylin.job.remote.cli.hostname=sandbox.hortonworks.com
+
+# Only necessary when kylin.job.run.as.remote.cmd=true
+kylin.job.remote.cli.username=root
+
+# Only necessary when kylin.job.run.as.remote.cmd=true
+kylin.job.remote.cli.password=hadoop
+
+# Used by test cases to prepare synthetic data for sample cube
+kylin.job.remote.cli.working.dir=/tmp/kylin
+
+# Use hive to flatten star schema tables
+kylin.job.hive.flatten=true
+
+# Max count of concurrent jobs running
+kylin.job.concurrent.max.limit=10
+
+# The url to check hadoop job status
+kylin.job.yarn.app.rest.check.status.url=http://sandbox:8088/ws/v1/cluster/apps/${job_id}?anonymous=true
+
+# Time interval to check hadoop job status
+kylin.job.yarn.app.rest.check.interval.seconds=10
+
+
+## Config for Restful APP ##
+# database connection settings:
+ldap.server=
+ldap.username=
+ldap.password=
+ldap.user.searchBase=
+ldap.user.searchPattern=
+ldap.user.groupSearchBase=
+ldap.service.searchBase=OU=
+ldap.service.searchPattern=
+ldap.service.groupSearchBase=
+acl.adminRole=
+acl.defaultRole=
+ganglia.group=
+ganglia.port=8664
+
+## Config for mail service
+
+# If true, will send email notification;
+mail.enabled=true
+mail.host=atom.corp.ebay.com
+mail.username=_kylin_ldap
+mail.password=
+mail.sender=DL-eBay-Kylin@corp.ebay.com
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/examples/test_case_data/sandbox/kylin_job_conf.lzo_disabled.xml
----------------------------------------------------------------------
diff --git a/examples/test_case_data/sandbox/kylin_job_conf.lzo_disabled.xml b/examples/test_case_data/sandbox/kylin_job_conf.lzo_disabled.xml
new file mode 100644
index 0000000..f62ce55
--- /dev/null
+++ b/examples/test_case_data/sandbox/kylin_job_conf.lzo_disabled.xml
@@ -0,0 +1,60 @@
+<?xml version="1.0"?>
+<configuration>
+
+    <property>
+        <name>mapreduce.job.split.metainfo.maxsize</name>
+        <value>-1</value>
+        <description>The maximum permissible size of the split metainfo file.
+            The JobTracker won't attempt to read split metainfo files bigger than
+            the configured value. No limits if set to -1.
+        </description>
+    </property>
+
+    <!-- uncomment the following 5 properties to enable lzo compressing
+
+	<property>
+		<name>mapred.compress.map.output</name>
+		<value>true</value>
+		<description>Compress map outputs</description>
+	</property>
+
+	<property>
+		<name>mapred.map.output.compression.codec</name>
+		<value>com.hadoop.compression.lzo.LzoCodec</value>
+		<description>The compression codec to use for map outputs
+		</description>
+	</property>
+
+	<property>
+		<name>mapred.output.compress</name>
+		<value>true</value>
+		<description>Compress the output of a MapReduce job</description>
+	</property>
+
+	<property>
+		<name>mapred.output.compression.codec</name>
+		<value>com.hadoop.compression.lzo.LzoCodec</value>
+		<description>The compression codec to use for job outputs
+		</description>
+	</property>
+
+	<property>
+		<name>mapred.output.compression.type</name>
+		<value>BLOCK</value>
+		<description>The compression type to use for job outputs</description>
+	</property>
+
+	!-->
+
+    <property>
+        <name>mapreduce.job.max.split.locations</name>
+        <value>2000</value>
+        <description>No description</description>
+    </property>
+
+    <property>
+        <name>dfs.replication</name>
+        <value>2</value>
+        <description>Block replication</description>
+    </property>
+</configuration>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/examples/test_case_data/sandbox/kylin_job_conf.lzo_enabled.xml
----------------------------------------------------------------------
diff --git a/examples/test_case_data/sandbox/kylin_job_conf.lzo_enabled.xml b/examples/test_case_data/sandbox/kylin_job_conf.lzo_enabled.xml
new file mode 100644
index 0000000..f62ce55
--- /dev/null
+++ b/examples/test_case_data/sandbox/kylin_job_conf.lzo_enabled.xml
@@ -0,0 +1,60 @@
+<?xml version="1.0"?>
+<configuration>
+
+    <property>
+        <name>mapreduce.job.split.metainfo.maxsize</name>
+        <value>-1</value>
+        <description>The maximum permissible size of the split metainfo file.
+            The JobTracker won't attempt to read split metainfo files bigger than
+            the configured value. No limits if set to -1.
+        </description>
+    </property>
+
+    <!-- uncomment the following 5 properties to enable lzo compressing
+
+	<property>
+		<name>mapred.compress.map.output</name>
+		<value>true</value>
+		<description>Compress map outputs</description>
+	</property>
+
+	<property>
+		<name>mapred.map.output.compression.codec</name>
+		<value>com.hadoop.compression.lzo.LzoCodec</value>
+		<description>The compression codec to use for map outputs
+		</description>
+	</property>
+
+	<property>
+		<name>mapred.output.compress</name>
+		<value>true</value>
+		<description>Compress the output of a MapReduce job</description>
+	</property>
+
+	<property>
+		<name>mapred.output.compression.codec</name>
+		<value>com.hadoop.compression.lzo.LzoCodec</value>
+		<description>The compression codec to use for job outputs
+		</description>
+	</property>
+
+	<property>
+		<name>mapred.output.compression.type</name>
+		<value>BLOCK</value>
+		<description>The compression type to use for job outputs</description>
+	</property>
+
+	!-->
+
+    <property>
+        <name>mapreduce.job.max.split.locations</name>
+        <value>2000</value>
+        <description>No description</description>
+    </property>
+
+    <property>
+        <name>dfs.replication</name>
+        <value>2</value>
+        <description>Block replication</description>
+    </property>
+</configuration>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/examples/test_case_data/sandbox/kylin_job_conf.xml
----------------------------------------------------------------------
diff --git a/examples/test_case_data/sandbox/kylin_job_conf.xml b/examples/test_case_data/sandbox/kylin_job_conf.xml
new file mode 100644
index 0000000..f62ce55
--- /dev/null
+++ b/examples/test_case_data/sandbox/kylin_job_conf.xml
@@ -0,0 +1,60 @@
+<?xml version="1.0"?>
+<configuration>
+
+    <property>
+        <name>mapreduce.job.split.metainfo.maxsize</name>
+        <value>-1</value>
+        <description>The maximum permissible size of the split metainfo file.
+            The JobTracker won't attempt to read split metainfo files bigger than
+            the configured value. No limits if set to -1.
+        </description>
+    </property>
+
+    <!-- uncomment the following 5 properties to enable lzo compressing
+
+	<property>
+		<name>mapred.compress.map.output</name>
+		<value>true</value>
+		<description>Compress map outputs</description>
+	</property>
+
+	<property>
+		<name>mapred.map.output.compression.codec</name>
+		<value>com.hadoop.compression.lzo.LzoCodec</value>
+		<description>The compression codec to use for map outputs
+		</description>
+	</property>
+
+	<property>
+		<name>mapred.output.compress</name>
+		<value>true</value>
+		<description>Compress the output of a MapReduce job</description>
+	</property>
+
+	<property>
+		<name>mapred.output.compression.codec</name>
+		<value>com.hadoop.compression.lzo.LzoCodec</value>
+		<description>The compression codec to use for job outputs
+		</description>
+	</property>
+
+	<property>
+		<name>mapred.output.compression.type</name>
+		<value>BLOCK</value>
+		<description>The compression type to use for job outputs</description>
+	</property>
+
+	!-->
+
+    <property>
+        <name>mapreduce.job.max.split.locations</name>
+        <value>2000</value>
+        <description>No description</description>
+    </property>
+
+    <property>
+        <name>dfs.replication</name>
+        <value>2</value>
+        <description>Block replication</description>
+    </property>
+</configuration>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/examples/test_case_data/sandbox/mapred-site.xml
----------------------------------------------------------------------
diff --git a/examples/test_case_data/sandbox/mapred-site.xml b/examples/test_case_data/sandbox/mapred-site.xml
new file mode 100644
index 0000000..355f64b
--- /dev/null
+++ b/examples/test_case_data/sandbox/mapred-site.xml
@@ -0,0 +1,155 @@
+<!--Mon Feb 10 22:50:51 2014-->
+  <configuration>
+    <property>
+    <name>mapreduce.jobhistory.webapp.address</name>
+    <value>sandbox.hortonworks.com:19888</value>
+  </property>
+    <property>
+    <name>mapreduce.shuffle.port</name>
+    <value>13562</value>
+  </property>
+    <property>
+    <name>mapreduce.cluster.administrators</name>
+    <value> hadoop</value>
+  </property>
+    <property>
+    <name>mapreduce.task.timeout</name>
+    <value>300000</value>
+  </property>
+    <property>
+    <name>mapreduce.map.sort.spill.percent</name>
+    <value>0.7</value>
+  </property>
+    <property>
+    <name>mapreduce.admin.user.env</name>
+    <value>LD_LIBRARY_PATH=/usr/lib/hadoop/lib/native:/usr/lib/hadoop/lib/native/`$JAVA_HOME/bin/java -d32 -version &amp;&gt; /dev/null;if [ $? -eq 0 ]; then echo Linux-i386-32; else echo Linux-amd64-64;fi`</value>
+  </property>
+    <property>
+    <name>mapreduce.reduce.shuffle.input.buffer.percent</name>
+    <value>0.7</value>
+  </property>
+    <property>
+    <name>mapreduce.admin.reduce.child.java.opts</name>
+    <value>-Djava.net.preferIPv4Stack=true -Dhadoop.metrics.log.level=WARN</value>
+  </property>
+    <property>
+    <name>mapreduce.reduce.input.buffer.percent</name>
+    <value>0.0</value>
+  </property>
+    <property>
+    <name>mapreduce.reduce.shuffle.parallelcopies</name>
+    <value>30</value>
+  </property>
+    <property>
+    <name>mapreduce.output.fileoutputformat.compress.type</name>
+    <value>BLOCK</value>
+  </property>
+    <property>
+    <name>mapreduce.reduce.memory.mb</name>
+    <value>1024</value>
+  </property>
+    <property>
+    <name>mapreduce.task.io.sort.mb</name>
+    <value>200</value>
+  </property>
+    <property>
+    <name>mapreduce.job.reduce.slowstart.completedmaps</name>
+    <value>0.05</value>
+  </property>
+    <property>
+    <name>yarn.app.mapreduce.am.log.level</name>
+    <value>INFO</value>
+  </property>
+    <property>
+    <name>mapreduce.map.java.opts</name>
+    <value>-Xmx512m</value>
+  </property>
+    <property>
+    <name>mapreduce.reduce.log.level</name>
+    <value>INFO</value>
+  </property>
+    <property>
+    <name>mapreduce.map.log.level</name>
+    <value>INFO</value>
+  </property>
+    <property>
+    <name>mapreduce.am.max-attempts</name>
+    <value>2</value>
+  </property>
+    <property>
+    <name>yarn.app.mapreduce.am.staging-dir</name>
+    <value>/user</value>
+  </property>
+    <property>
+    <name>mapreduce.task.io.sort.factor</name>
+    <value>100</value>
+  </property>
+    <property>
+    <name>mapreduce.framework.name</name>
+    <value>yarn</value>
+  </property>
+    <property>
+    <name>mapreduce.reduce.shuffle.merge.percent</name>
+    <value>0.66</value>
+  </property>
+    <property>
+    <name>mapreduce.reduce.speculative</name>
+    <value>false</value>
+  </property>
+  <property>
+    <name>mapreduce.application.classpath</name>
+    <value>/tmp/kylin/*,/usr/lib/hbase/lib/*</value>
+  </property>
+  <property>
+    <name>mapreduce.map.output.compress</name>
+    <value>false</value>
+  </property>
+    <property>
+    <name>mapreduce.map.speculative</name>
+    <value>false</value>
+  </property>
+    <property>
+    <name>mapreduce.map.memory.mb</name>
+    <value>1024</value>
+  </property>
+    <property>
+    <name>yarn.app.mapreduce.am.admin-command-opts</name>
+    <value>-Djava.net.preferIPv4Stack=true -Dhadoop.metrics.log.level=WARN</value>
+  </property>
+    <property>
+    <name>mapreduce.jobhistory.address</name>
+    <value>sandbox.hortonworks.com:10020</value>
+  </property>
+    <property>
+    <name>mapreduce.jobhistory.done-dir</name>
+    <value>/mr-history/done</value>
+  </property>
+    <property>
+    <name>mapreduce.output.fileoutputformat.compress</name>
+    <value>false</value>
+  </property>
+    <property>
+    <name>yarn.app.mapreduce.am.command-opts</name>
+    <value>-Xmx312m</value>
+  </property>
+    <property>
+    <name>mapreduce.reduce.java.opts</name>
+    <value>-Xmx512m</value>
+  </property>
+    <property>
+    <name>yarn.app.mapreduce.am.resource.mb</name>
+    <value>250</value>
+  </property>
+    <property>
+    <name>mapreduce.admin.map.child.java.opts</name>
+    <value>-Djava.net.preferIPv4Stack=true -Dhadoop.metrics.log.level=WARN</value>
+  </property>
+    <property>
+    <name>mapreduce.jobhistory.intermediate-done-dir</name>
+    <value>/mr-history/tmp</value>
+  </property>
+  <property>
+	<name>mapreduce.app-submission.cross-platform</name>
+	<value>true</value>
+  </property>
+  </configuration>

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/examples/test_case_data/sandbox/yarn-site.xml
----------------------------------------------------------------------
diff --git a/examples/test_case_data/sandbox/yarn-site.xml b/examples/test_case_data/sandbox/yarn-site.xml
new file mode 100644
index 0000000..41fb09d
--- /dev/null
+++ b/examples/test_case_data/sandbox/yarn-site.xml
@@ -0,0 +1,147 @@
+<!--Sun Jan 26 18:32:31 2014-->
+  <configuration>
+    <property>
+    <name>yarn.nodemanager.aux-services</name>
+    <value>mapreduce_shuffle</value>
+  </property>
+    <property>
+    <name>yarn.nodemanager.aux-services.mapreduce_shuffle.class</name>
+    <value>org.apache.hadoop.mapred.ShuffleHandler</value>
+  </property>
+    <property>
+    <name>yarn.log-aggregation.retain-seconds</name>
+    <value>2592000</value>
+  </property>
+    <property>
+    <name>yarn.nodemanager.address</name>
+    <value>0.0.0.0:45454</value>
+  </property>
+    <property>
+    <name>yarn.nodemanager.log-aggregation.compression-type</name>
+    <value>gz</value>
+  </property>
+    <property>
+    <name>yarn.nodemanager.resource.memory-mb</name>
+    <value>8000</value>
+  </property>
+    <property>
+    <name>yarn.nodemanager.container-executor.class</name>
+    <value>org.apache.hadoop.yarn.server.nodemanager.DefaultContainerExecutor</value>
+  </property>
+    <property>
+    <name>yarn.nodemanager.linux-container-executor.group</name>
+    <value>hadoop</value>
+  </property>
+    <property>
+    <name>yarn.resourcemanager.resource-tracker.address</name>
+    <value>sandbox.hortonworks.com:8025</value>
+  </property>
+  <property>
+    <name>yarn.application.classpath</name>
+    <value>/etc/hadoop/conf,/usr/lib/hadoop/*,/usr/lib/hadoop/lib/*,/usr/lib/hadoop-hdfs/*,/usr/lib/hadoop-hdfs/lib/*,/usr/lib/hadoop-yarn/*,/usr/lib/hadoop-yarn/lib/*,/usr/lib/hadoop-mapreduce/*,/usr/lib/hadoop-mapreduce/lib/*</value>
+  </property>
+    <property>
+    <name>yarn.nodemanager.log.retain-second</name>
+    <value>604800</value>
+  </property>
+    <property>
+    <name>yarn.resourcemanager.hostname</name>
+    <value>sandbox.hortonworks.com</value>
+  </property>
+    <property>
+    <name>yarn.nodemanager.admin-env</name>
+    <value>MALLOC_ARENA_MAX=$MALLOC_ARENA_MAX</value>
+  </property>
+    <property>
+    <name>yarn.scheduler.minimum-allocation-mb</name>
+    <value>64</value>
+  </property>
+    <property>
+    <name>yarn.nodemanager.health-checker.script.timeout-ms</name>
+    <value>60000</value>
+  </property>
+    <property>
+    <name>yarn.nodemanager.health-checker.interval-ms</name>
+    <value>135000</value>
+  </property>
+    <property>
+    <name>yarn.nodemanager.container-monitor.interval-ms</name>
+    <value>3000</value>
+  </property>
+    <property>
+    <name>yarn.nodemanager.remote-app-log-dir-suffix</name>
+    <value>logs</value>
+  </property>
+    <property>
+    <name>yarn.resourcemanager.scheduler.class</name>
+    <value>org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacityScheduler</value>
+  </property>
+    <property>
+    <name>yarn.admin.acl</name>
+    <value>*</value>
+  </property>
+    <property>
+    <name>yarn.nodemanager.disk-health-checker.min-healthy-disks</name>
+    <value>0.25</value>
+  </property>
+    <property>
+    <name>yarn.nodemanager.remote-app-log-dir</name>
+    <value>/app-logs</value>
+  </property>
+    <property>
+    <name>yarn.nodemanager.local-dirs</name>
+    <value>/hadoop/yarn/local</value>
+  </property>
+    <property>
+    <name>yarn.scheduler.maximum-allocation-mb</name>
+    <value>4096</value>
+  </property>
+    <property>
+    <name>yarn.nodemanager.log-dirs</name>
+    <value>/hadoop/yarn/log</value>
+  </property>
+    <property>
+    <name>yarn.resourcemanager.admin.address</name>
+    <value>sandbox.hortonworks.com:8141</value>
+  </property>
+    <property>
+    <name>yarn.log.server.url</name>
+    <value>http://sandbox.hortonworks.com:19888/jobhistory/logs</value>
+  </property>
+    <property>
+    <name>yarn.acl.enable</name>
+    <value>true</value>
+  </property>
+    <property>
+    <name>yarn.nodemanager.vmem-pmem-ratio</name>
+    <value>10</value>
+  </property>
+    <property>
+    <name>yarn.log-aggregation-enable</name>
+    <value>true</value>
+  </property>
+    <property>
+    <name>yarn.resourcemanager.address</name>
+    <value>sandbox.hortonworks.com:8050</value>
+  </property>
+    <property>
+    <name>yarn.resourcemanager.scheduler.address</name>
+    <value>sandbox.hortonworks.com:8030</value>
+  </property>
+    <property>
+    <name>yarn.resourcemanager.webapp.address</name>
+    <value>sandbox.hortonworks.com:8088</value>
+  </property>
+    <property>
+    <name>yarn.nodemanager.vmem-check-enabled</name>
+    <value>false</value>
+  </property>
+    <property>
+    <name>yarn.resourcemanager.am.max-attempts</name>
+    <value>2</value>
+  </property>
+    <property>
+    <name>yarn.nodemanager.delete.debug-delay-sec</name>
+    <value>0</value>
+  </property>
+  </configuration>

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/jdbc/.settings/org.eclipse.core.resources.prefs
----------------------------------------------------------------------
diff --git a/jdbc/.settings/org.eclipse.core.resources.prefs b/jdbc/.settings/org.eclipse.core.resources.prefs
new file mode 100644
index 0000000..365bbd6
--- /dev/null
+++ b/jdbc/.settings/org.eclipse.core.resources.prefs
@@ -0,0 +1,5 @@
+eclipse.preferences.version=1
+encoding//src/main/java=UTF-8
+encoding//src/main/resources=UTF-8
+encoding//src/test/java=UTF-8
+encoding/<project>=UTF-8

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/jdbc/.settings/org.eclipse.jdt.core.prefs
----------------------------------------------------------------------
diff --git a/jdbc/.settings/org.eclipse.jdt.core.prefs b/jdbc/.settings/org.eclipse.jdt.core.prefs
new file mode 100644
index 0000000..a903301
--- /dev/null
+++ b/jdbc/.settings/org.eclipse.jdt.core.prefs
@@ -0,0 +1,379 @@
+eclipse.preferences.version=1
+org.eclipse.jdt.core.compiler.annotation.inheritNullAnnotations=disabled
+org.eclipse.jdt.core.compiler.annotation.missingNonNullByDefaultAnnotation=ignore
+org.eclipse.jdt.core.compiler.annotation.nonnull=org.eclipse.jdt.annotation.NonNull
+org.eclipse.jdt.core.compiler.annotation.nonnullbydefault=org.eclipse.jdt.annotation.NonNullByDefault
+org.eclipse.jdt.core.compiler.annotation.nullable=org.eclipse.jdt.annotation.Nullable
+org.eclipse.jdt.core.compiler.annotation.nullanalysis=disabled
+org.eclipse.jdt.core.compiler.codegen.inlineJsrBytecode=enabled
+org.eclipse.jdt.core.compiler.codegen.targetPlatform=1.7
+org.eclipse.jdt.core.compiler.codegen.unusedLocal=preserve
+org.eclipse.jdt.core.compiler.compliance=1.7
+org.eclipse.jdt.core.compiler.debug.lineNumber=generate
+org.eclipse.jdt.core.compiler.debug.localVariable=generate
+org.eclipse.jdt.core.compiler.debug.sourceFile=generate
+org.eclipse.jdt.core.compiler.problem.annotationSuperInterface=warning
+org.eclipse.jdt.core.compiler.problem.assertIdentifier=error
+org.eclipse.jdt.core.compiler.problem.autoboxing=ignore
+org.eclipse.jdt.core.compiler.problem.comparingIdentical=warning
+org.eclipse.jdt.core.compiler.problem.deadCode=warning
+org.eclipse.jdt.core.compiler.problem.deprecation=warning
+org.eclipse.jdt.core.compiler.problem.deprecationInDeprecatedCode=disabled
+org.eclipse.jdt.core.compiler.problem.deprecationWhenOverridingDeprecatedMethod=disabled
+org.eclipse.jdt.core.compiler.problem.discouragedReference=warning
+org.eclipse.jdt.core.compiler.problem.emptyStatement=ignore
+org.eclipse.jdt.core.compiler.problem.enumIdentifier=error
+org.eclipse.jdt.core.compiler.problem.explicitlyClosedAutoCloseable=ignore
+org.eclipse.jdt.core.compiler.problem.fallthroughCase=ignore
+org.eclipse.jdt.core.compiler.problem.fatalOptionalError=disabled
+org.eclipse.jdt.core.compiler.problem.fieldHiding=ignore
+org.eclipse.jdt.core.compiler.problem.finalParameterBound=warning
+org.eclipse.jdt.core.compiler.problem.finallyBlockNotCompletingNormally=warning
+org.eclipse.jdt.core.compiler.problem.forbiddenReference=warning
+org.eclipse.jdt.core.compiler.problem.hiddenCatchBlock=warning
+org.eclipse.jdt.core.compiler.problem.includeNullInfoFromAsserts=disabled
+org.eclipse.jdt.core.compiler.problem.incompatibleNonInheritedInterfaceMethod=warning
+org.eclipse.jdt.core.compiler.problem.incompleteEnumSwitch=warning
+org.eclipse.jdt.core.compiler.problem.indirectStaticAccess=ignore
+org.eclipse.jdt.core.compiler.problem.localVariableHiding=ignore
+org.eclipse.jdt.core.compiler.problem.methodWithConstructorName=warning
+org.eclipse.jdt.core.compiler.problem.missingDefaultCase=ignore
+org.eclipse.jdt.core.compiler.problem.missingDeprecatedAnnotation=ignore
+org.eclipse.jdt.core.compiler.problem.missingEnumCaseDespiteDefault=disabled
+org.eclipse.jdt.core.compiler.problem.missingHashCodeMethod=ignore
+org.eclipse.jdt.core.compiler.problem.missingOverrideAnnotation=ignore
+org.eclipse.jdt.core.compiler.problem.missingOverrideAnnotationForInterfaceMethodImplementation=enabled
+org.eclipse.jdt.core.compiler.problem.missingSerialVersion=warning
+org.eclipse.jdt.core.compiler.problem.missingSynchronizedOnInheritedMethod=ignore
+org.eclipse.jdt.core.compiler.problem.noEffectAssignment=warning
+org.eclipse.jdt.core.compiler.problem.noImplicitStringConversion=warning
+org.eclipse.jdt.core.compiler.problem.nonExternalizedStringLiteral=ignore
+org.eclipse.jdt.core.compiler.problem.nonnullParameterAnnotationDropped=warning
+org.eclipse.jdt.core.compiler.problem.nullAnnotationInferenceConflict=error
+org.eclipse.jdt.core.compiler.problem.nullReference=warning
+org.eclipse.jdt.core.compiler.problem.nullSpecViolation=error
+org.eclipse.jdt.core.compiler.problem.nullUncheckedConversion=warning
+org.eclipse.jdt.core.compiler.problem.overridingPackageDefaultMethod=warning
+org.eclipse.jdt.core.compiler.problem.parameterAssignment=ignore
+org.eclipse.jdt.core.compiler.problem.possibleAccidentalBooleanAssignment=ignore
+org.eclipse.jdt.core.compiler.problem.potentialNullReference=ignore
+org.eclipse.jdt.core.compiler.problem.potentiallyUnclosedCloseable=ignore
+org.eclipse.jdt.core.compiler.problem.rawTypeReference=warning
+org.eclipse.jdt.core.compiler.problem.redundantNullAnnotation=warning
+org.eclipse.jdt.core.compiler.problem.redundantNullCheck=ignore
+org.eclipse.jdt.core.compiler.problem.redundantSpecificationOfTypeArguments=ignore
+org.eclipse.jdt.core.compiler.problem.redundantSuperinterface=ignore
+org.eclipse.jdt.core.compiler.problem.reportMethodCanBePotentiallyStatic=ignore
+org.eclipse.jdt.core.compiler.problem.reportMethodCanBeStatic=ignore
+org.eclipse.jdt.core.compiler.problem.specialParameterHidingField=disabled
+org.eclipse.jdt.core.compiler.problem.staticAccessReceiver=warning
+org.eclipse.jdt.core.compiler.problem.suppressOptionalErrors=disabled
+org.eclipse.jdt.core.compiler.problem.suppressWarnings=enabled
+org.eclipse.jdt.core.compiler.problem.syntacticNullAnalysisForFields=disabled
+org.eclipse.jdt.core.compiler.problem.syntheticAccessEmulation=ignore
+org.eclipse.jdt.core.compiler.problem.typeParameterHiding=warning
+org.eclipse.jdt.core.compiler.problem.unavoidableGenericTypeProblems=enabled
+org.eclipse.jdt.core.compiler.problem.uncheckedTypeOperation=warning
+org.eclipse.jdt.core.compiler.problem.unclosedCloseable=warning
+org.eclipse.jdt.core.compiler.problem.undocumentedEmptyBlock=ignore
+org.eclipse.jdt.core.compiler.problem.unhandledWarningToken=warning
+org.eclipse.jdt.core.compiler.problem.unnecessaryElse=ignore
+org.eclipse.jdt.core.compiler.problem.unnecessaryTypeCheck=ignore
+org.eclipse.jdt.core.compiler.problem.unqualifiedFieldAccess=ignore
+org.eclipse.jdt.core.compiler.problem.unusedDeclaredThrownException=ignore
+org.eclipse.jdt.core.compiler.problem.unusedDeclaredThrownExceptionExemptExceptionAndThrowable=enabled
+org.eclipse.jdt.core.compiler.problem.unusedDeclaredThrownExceptionIncludeDocCommentReference=enabled
+org.eclipse.jdt.core.compiler.problem.unusedDeclaredThrownExceptionWhenOverriding=disabled
+org.eclipse.jdt.core.compiler.problem.unusedImport=warning
+org.eclipse.jdt.core.compiler.problem.unusedLabel=warning
+org.eclipse.jdt.core.compiler.problem.unusedLocal=warning
+org.eclipse.jdt.core.compiler.problem.unusedObjectAllocation=ignore
+org.eclipse.jdt.core.compiler.problem.unusedParameter=ignore
+org.eclipse.jdt.core.compiler.problem.unusedParameterIncludeDocCommentReference=enabled
+org.eclipse.jdt.core.compiler.problem.unusedParameterWhenImplementingAbstract=disabled
+org.eclipse.jdt.core.compiler.problem.unusedParameterWhenOverridingConcrete=disabled
+org.eclipse.jdt.core.compiler.problem.unusedPrivateMember=warning
+org.eclipse.jdt.core.compiler.problem.unusedTypeParameter=ignore
+org.eclipse.jdt.core.compiler.problem.unusedWarningToken=warning
+org.eclipse.jdt.core.compiler.problem.varargsArgumentNeedCast=warning
+org.eclipse.jdt.core.compiler.source=1.7
+org.eclipse.jdt.core.formatter.align_type_members_on_columns=false
+org.eclipse.jdt.core.formatter.alignment_for_arguments_in_allocation_expression=16
+org.eclipse.jdt.core.formatter.alignment_for_arguments_in_annotation=0
+org.eclipse.jdt.core.formatter.alignment_for_arguments_in_enum_constant=16
+org.eclipse.jdt.core.formatter.alignment_for_arguments_in_explicit_constructor_call=16
+org.eclipse.jdt.core.formatter.alignment_for_arguments_in_method_invocation=16
+org.eclipse.jdt.core.formatter.alignment_for_arguments_in_qualified_allocation_expression=16
+org.eclipse.jdt.core.formatter.alignment_for_assignment=0
+org.eclipse.jdt.core.formatter.alignment_for_binary_expression=16
+org.eclipse.jdt.core.formatter.alignment_for_compact_if=16
+org.eclipse.jdt.core.formatter.alignment_for_conditional_expression=80
+org.eclipse.jdt.core.formatter.alignment_for_enum_constants=0
+org.eclipse.jdt.core.formatter.alignment_for_expressions_in_array_initializer=16
+org.eclipse.jdt.core.formatter.alignment_for_method_declaration=0
+org.eclipse.jdt.core.formatter.alignment_for_multiple_fields=16
+org.eclipse.jdt.core.formatter.alignment_for_parameters_in_constructor_declaration=16
+org.eclipse.jdt.core.formatter.alignment_for_parameters_in_method_declaration=16
+org.eclipse.jdt.core.formatter.alignment_for_resources_in_try=80
+org.eclipse.jdt.core.formatter.alignment_for_selector_in_method_invocation=16
+org.eclipse.jdt.core.formatter.alignment_for_superclass_in_type_declaration=16
+org.eclipse.jdt.core.formatter.alignment_for_superinterfaces_in_enum_declaration=16
+org.eclipse.jdt.core.formatter.alignment_for_superinterfaces_in_type_declaration=16
+org.eclipse.jdt.core.formatter.alignment_for_throws_clause_in_constructor_declaration=16
+org.eclipse.jdt.core.formatter.alignment_for_throws_clause_in_method_declaration=16
+org.eclipse.jdt.core.formatter.alignment_for_union_type_in_multicatch=16
+org.eclipse.jdt.core.formatter.blank_lines_after_imports=1
+org.eclipse.jdt.core.formatter.blank_lines_after_package=1
+org.eclipse.jdt.core.formatter.blank_lines_before_field=0
+org.eclipse.jdt.core.formatter.blank_lines_before_first_class_body_declaration=0
+org.eclipse.jdt.core.formatter.blank_lines_before_imports=1
+org.eclipse.jdt.core.formatter.blank_lines_before_member_type=1
+org.eclipse.jdt.core.formatter.blank_lines_before_method=1
+org.eclipse.jdt.core.formatter.blank_lines_before_new_chunk=1
+org.eclipse.jdt.core.formatter.blank_lines_before_package=0
+org.eclipse.jdt.core.formatter.blank_lines_between_import_groups=1
+org.eclipse.jdt.core.formatter.blank_lines_between_type_declarations=1
+org.eclipse.jdt.core.formatter.brace_position_for_annotation_type_declaration=end_of_line
+org.eclipse.jdt.core.formatter.brace_position_for_anonymous_type_declaration=end_of_line
+org.eclipse.jdt.core.formatter.brace_position_for_array_initializer=end_of_line
+org.eclipse.jdt.core.formatter.brace_position_for_block=end_of_line
+org.eclipse.jdt.core.formatter.brace_position_for_block_in_case=end_of_line
+org.eclipse.jdt.core.formatter.brace_position_for_constructor_declaration=end_of_line
+org.eclipse.jdt.core.formatter.brace_position_for_enum_constant=end_of_line
+org.eclipse.jdt.core.formatter.brace_position_for_enum_declaration=end_of_line
+org.eclipse.jdt.core.formatter.brace_position_for_method_declaration=end_of_line
+org.eclipse.jdt.core.formatter.brace_position_for_switch=end_of_line
+org.eclipse.jdt.core.formatter.brace_position_for_type_declaration=end_of_line
+org.eclipse.jdt.core.formatter.comment.clear_blank_lines_in_block_comment=false
+org.eclipse.jdt.core.formatter.comment.clear_blank_lines_in_javadoc_comment=false
+org.eclipse.jdt.core.formatter.comment.format_block_comments=false
+org.eclipse.jdt.core.formatter.comment.format_header=false
+org.eclipse.jdt.core.formatter.comment.format_html=true
+org.eclipse.jdt.core.formatter.comment.format_javadoc_comments=false
+org.eclipse.jdt.core.formatter.comment.format_line_comments=false
+org.eclipse.jdt.core.formatter.comment.format_source_code=true
+org.eclipse.jdt.core.formatter.comment.indent_parameter_description=true
+org.eclipse.jdt.core.formatter.comment.indent_root_tags=true
+org.eclipse.jdt.core.formatter.comment.insert_new_line_before_root_tags=insert
+org.eclipse.jdt.core.formatter.comment.insert_new_line_for_parameter=insert
+org.eclipse.jdt.core.formatter.comment.line_length=80
+org.eclipse.jdt.core.formatter.comment.new_lines_at_block_boundaries=true
+org.eclipse.jdt.core.formatter.comment.new_lines_at_javadoc_boundaries=true
+org.eclipse.jdt.core.formatter.comment.preserve_white_space_between_code_and_line_comments=false
+org.eclipse.jdt.core.formatter.compact_else_if=true
+org.eclipse.jdt.core.formatter.continuation_indentation=2
+org.eclipse.jdt.core.formatter.continuation_indentation_for_array_initializer=2
+org.eclipse.jdt.core.formatter.disabling_tag=@formatter\:off
+org.eclipse.jdt.core.formatter.enabling_tag=@formatter\:on
+org.eclipse.jdt.core.formatter.format_guardian_clause_on_one_line=false
+org.eclipse.jdt.core.formatter.format_line_comment_starting_on_first_column=true
+org.eclipse.jdt.core.formatter.indent_body_declarations_compare_to_annotation_declaration_header=true
+org.eclipse.jdt.core.formatter.indent_body_declarations_compare_to_enum_constant_header=true
+org.eclipse.jdt.core.formatter.indent_body_declarations_compare_to_enum_declaration_header=true
+org.eclipse.jdt.core.formatter.indent_body_declarations_compare_to_type_header=true
+org.eclipse.jdt.core.formatter.indent_breaks_compare_to_cases=true
+org.eclipse.jdt.core.formatter.indent_empty_lines=false
+org.eclipse.jdt.core.formatter.indent_statements_compare_to_block=true
+org.eclipse.jdt.core.formatter.indent_statements_compare_to_body=true
+org.eclipse.jdt.core.formatter.indent_switchstatements_compare_to_cases=true
+org.eclipse.jdt.core.formatter.indent_switchstatements_compare_to_switch=false
+org.eclipse.jdt.core.formatter.indentation.size=4
+org.eclipse.jdt.core.formatter.insert_new_line_after_annotation_on_field=insert
+org.eclipse.jdt.core.formatter.insert_new_line_after_annotation_on_local_variable=insert
+org.eclipse.jdt.core.formatter.insert_new_line_after_annotation_on_method=insert
+org.eclipse.jdt.core.formatter.insert_new_line_after_annotation_on_package=insert
+org.eclipse.jdt.core.formatter.insert_new_line_after_annotation_on_parameter=do not insert
+org.eclipse.jdt.core.formatter.insert_new_line_after_annotation_on_type=insert
+org.eclipse.jdt.core.formatter.insert_new_line_after_label=do not insert
+org.eclipse.jdt.core.formatter.insert_new_line_after_opening_brace_in_array_initializer=do not insert
+org.eclipse.jdt.core.formatter.insert_new_line_at_end_of_file_if_missing=do not insert
+org.eclipse.jdt.core.formatter.insert_new_line_before_catch_in_try_statement=do not insert
+org.eclipse.jdt.core.formatter.insert_new_line_before_closing_brace_in_array_initializer=do not insert
+org.eclipse.jdt.core.formatter.insert_new_line_before_else_in_if_statement=do not insert
+org.eclipse.jdt.core.formatter.insert_new_line_before_finally_in_try_statement=do not insert
+org.eclipse.jdt.core.formatter.insert_new_line_before_while_in_do_statement=do not insert
+org.eclipse.jdt.core.formatter.insert_new_line_in_empty_annotation_declaration=insert
+org.eclipse.jdt.core.formatter.insert_new_line_in_empty_anonymous_type_declaration=insert
+org.eclipse.jdt.core.formatter.insert_new_line_in_empty_block=insert
+org.eclipse.jdt.core.formatter.insert_new_line_in_empty_enum_constant=insert
+org.eclipse.jdt.core.formatter.insert_new_line_in_empty_enum_declaration=insert
+org.eclipse.jdt.core.formatter.insert_new_line_in_empty_method_body=insert
+org.eclipse.jdt.core.formatter.insert_new_line_in_empty_type_declaration=insert
+org.eclipse.jdt.core.formatter.insert_space_after_and_in_type_parameter=insert
+org.eclipse.jdt.core.formatter.insert_space_after_assignment_operator=insert
+org.eclipse.jdt.core.formatter.insert_space_after_at_in_annotation=do not insert
+org.eclipse.jdt.core.formatter.insert_space_after_at_in_annotation_type_declaration=do not insert
+org.eclipse.jdt.core.formatter.insert_space_after_binary_operator=insert
+org.eclipse.jdt.core.formatter.insert_space_after_closing_angle_bracket_in_type_arguments=insert
+org.eclipse.jdt.core.formatter.insert_space_after_closing_angle_bracket_in_type_parameters=insert
+org.eclipse.jdt.core.formatter.insert_space_after_closing_brace_in_block=insert
+org.eclipse.jdt.core.formatter.insert_space_after_closing_paren_in_cast=insert
+org.eclipse.jdt.core.formatter.insert_space_after_colon_in_assert=insert
+org.eclipse.jdt.core.formatter.insert_space_after_colon_in_case=insert
+org.eclipse.jdt.core.formatter.insert_space_after_colon_in_conditional=insert
+org.eclipse.jdt.core.formatter.insert_space_after_colon_in_for=insert
+org.eclipse.jdt.core.formatter.insert_space_after_colon_in_labeled_statement=insert
+org.eclipse.jdt.core.formatter.insert_space_after_comma_in_allocation_expression=insert
+org.eclipse.jdt.core.formatter.insert_space_after_comma_in_annotation=insert
+org.eclipse.jdt.core.formatter.insert_space_after_comma_in_array_initializer=insert
+org.eclipse.jdt.core.formatter.insert_space_after_comma_in_constructor_declaration_parameters=insert
+org.eclipse.jdt.core.formatter.insert_space_after_comma_in_constructor_declaration_throws=insert
+org.eclipse.jdt.core.formatter.insert_space_after_comma_in_enum_constant_arguments=insert
+org.eclipse.jdt.core.formatter.insert_space_after_comma_in_enum_declarations=insert
+org.eclipse.jdt.core.formatter.insert_space_after_comma_in_explicitconstructorcall_arguments=insert
+org.eclipse.jdt.core.formatter.insert_space_after_comma_in_for_increments=insert
+org.eclipse.jdt.core.formatter.insert_space_after_comma_in_for_inits=insert
+org.eclipse.jdt.core.formatter.insert_space_after_comma_in_method_declaration_parameters=insert
+org.eclipse.jdt.core.formatter.insert_space_after_comma_in_method_declaration_throws=insert
+org.eclipse.jdt.core.formatter.insert_space_after_comma_in_method_invocation_arguments=insert
+org.eclipse.jdt.core.formatter.insert_space_after_comma_in_multiple_field_declarations=insert
+org.eclipse.jdt.core.formatter.insert_space_after_comma_in_multiple_local_declarations=insert
+org.eclipse.jdt.core.formatter.insert_space_after_comma_in_parameterized_type_reference=insert
+org.eclipse.jdt.core.formatter.insert_space_after_comma_in_superinterfaces=insert
+org.eclipse.jdt.core.formatter.insert_space_after_comma_in_type_arguments=insert
+org.eclipse.jdt.core.formatter.insert_space_after_comma_in_type_parameters=insert
+org.eclipse.jdt.core.formatter.insert_space_after_ellipsis=insert
+org.eclipse.jdt.core.formatter.insert_space_after_opening_angle_bracket_in_parameterized_type_reference=do not insert
+org.eclipse.jdt.core.formatter.insert_space_after_opening_angle_bracket_in_type_arguments=do not insert
+org.eclipse.jdt.core.formatter.insert_space_after_opening_angle_bracket_in_type_parameters=do not insert
+org.eclipse.jdt.core.formatter.insert_space_after_opening_brace_in_array_initializer=insert
+org.eclipse.jdt.core.formatter.insert_space_after_opening_bracket_in_array_allocation_expression=do not insert
+org.eclipse.jdt.core.formatter.insert_space_after_opening_bracket_in_array_reference=do not insert
+org.eclipse.jdt.core.formatter.insert_space_after_opening_paren_in_annotation=do not insert
+org.eclipse.jdt.core.formatter.insert_space_after_opening_paren_in_cast=do not insert
+org.eclipse.jdt.core.formatter.insert_space_after_opening_paren_in_catch=do not insert
+org.eclipse.jdt.core.formatter.insert_space_after_opening_paren_in_constructor_declaration=do not insert
+org.eclipse.jdt.core.formatter.insert_space_after_opening_paren_in_enum_constant=do not insert
+org.eclipse.jdt.core.formatter.insert_space_after_opening_paren_in_for=do not insert
+org.eclipse.jdt.core.formatter.insert_space_after_opening_paren_in_if=do not insert
+org.eclipse.jdt.core.formatter.insert_space_after_opening_paren_in_method_declaration=do not insert
+org.eclipse.jdt.core.formatter.insert_space_after_opening_paren_in_method_invocation=do not insert
+org.eclipse.jdt.core.formatter.insert_space_after_opening_paren_in_parenthesized_expression=do not insert
+org.eclipse.jdt.core.formatter.insert_space_after_opening_paren_in_switch=do not insert
+org.eclipse.jdt.core.formatter.insert_space_after_opening_paren_in_synchronized=do not insert
+org.eclipse.jdt.core.formatter.insert_space_after_opening_paren_in_try=do not insert
+org.eclipse.jdt.core.formatter.insert_space_after_opening_paren_in_while=do not insert
+org.eclipse.jdt.core.formatter.insert_space_after_postfix_operator=do not insert
+org.eclipse.jdt.core.formatter.insert_space_after_prefix_operator=do not insert
+org.eclipse.jdt.core.formatter.insert_space_after_question_in_conditional=insert
+org.eclipse.jdt.core.formatter.insert_space_after_question_in_wildcard=do not insert
+org.eclipse.jdt.core.formatter.insert_space_after_semicolon_in_for=insert
+org.eclipse.jdt.core.formatter.insert_space_after_semicolon_in_try_resources=insert
+org.eclipse.jdt.core.formatter.insert_space_after_unary_operator=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_and_in_type_parameter=insert
+org.eclipse.jdt.core.formatter.insert_space_before_assignment_operator=insert
+org.eclipse.jdt.core.formatter.insert_space_before_at_in_annotation_type_declaration=insert
+org.eclipse.jdt.core.formatter.insert_space_before_binary_operator=insert
+org.eclipse.jdt.core.formatter.insert_space_before_closing_angle_bracket_in_parameterized_type_reference=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_closing_angle_bracket_in_type_arguments=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_closing_angle_bracket_in_type_parameters=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_closing_brace_in_array_initializer=insert
+org.eclipse.jdt.core.formatter.insert_space_before_closing_bracket_in_array_allocation_expression=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_closing_bracket_in_array_reference=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_closing_paren_in_annotation=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_closing_paren_in_cast=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_closing_paren_in_catch=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_closing_paren_in_constructor_declaration=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_closing_paren_in_enum_constant=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_closing_paren_in_for=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_closing_paren_in_if=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_closing_paren_in_method_declaration=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_closing_paren_in_method_invocation=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_closing_paren_in_parenthesized_expression=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_closing_paren_in_switch=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_closing_paren_in_synchronized=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_closing_paren_in_try=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_closing_paren_in_while=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_colon_in_assert=insert
+org.eclipse.jdt.core.formatter.insert_space_before_colon_in_case=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_colon_in_conditional=insert
+org.eclipse.jdt.core.formatter.insert_space_before_colon_in_default=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_colon_in_for=insert
+org.eclipse.jdt.core.formatter.insert_space_before_colon_in_labeled_statement=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_comma_in_allocation_expression=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_comma_in_annotation=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_comma_in_array_initializer=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_comma_in_constructor_declaration_parameters=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_comma_in_constructor_declaration_throws=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_comma_in_enum_constant_arguments=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_comma_in_enum_declarations=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_comma_in_explicitconstructorcall_arguments=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_comma_in_for_increments=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_comma_in_for_inits=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_comma_in_method_declaration_parameters=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_comma_in_method_declaration_throws=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_comma_in_method_invocation_arguments=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_comma_in_multiple_field_declarations=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_comma_in_multiple_local_declarations=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_comma_in_parameterized_type_reference=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_comma_in_superinterfaces=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_comma_in_type_arguments=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_comma_in_type_parameters=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_ellipsis=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_opening_angle_bracket_in_parameterized_type_reference=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_opening_angle_bracket_in_type_arguments=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_opening_angle_bracket_in_type_parameters=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_opening_brace_in_annotation_type_declaration=insert
+org.eclipse.jdt.core.formatter.insert_space_before_opening_brace_in_anonymous_type_declaration=insert
+org.eclipse.jdt.core.formatter.insert_space_before_opening_brace_in_array_initializer=insert
+org.eclipse.jdt.core.formatter.insert_space_before_opening_brace_in_block=insert
+org.eclipse.jdt.core.formatter.insert_space_before_opening_brace_in_constructor_declaration=insert
+org.eclipse.jdt.core.formatter.insert_space_before_opening_brace_in_enum_constant=insert
+org.eclipse.jdt.core.formatter.insert_space_before_opening_brace_in_enum_declaration=insert
+org.eclipse.jdt.core.formatter.insert_space_before_opening_brace_in_method_declaration=insert
+org.eclipse.jdt.core.formatter.insert_space_before_opening_brace_in_switch=insert
+org.eclipse.jdt.core.formatter.insert_space_before_opening_brace_in_type_declaration=insert
+org.eclipse.jdt.core.formatter.insert_space_before_opening_bracket_in_array_allocation_expression=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_opening_bracket_in_array_reference=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_opening_bracket_in_array_type_reference=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_opening_paren_in_annotation=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_opening_paren_in_annotation_type_member_declaration=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_opening_paren_in_catch=insert
+org.eclipse.jdt.core.formatter.insert_space_before_opening_paren_in_constructor_declaration=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_opening_paren_in_enum_constant=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_opening_paren_in_for=insert
+org.eclipse.jdt.core.formatter.insert_space_before_opening_paren_in_if=insert
+org.eclipse.jdt.core.formatter.insert_space_before_opening_paren_in_method_declaration=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_opening_paren_in_method_invocation=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_opening_paren_in_parenthesized_expression=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_opening_paren_in_switch=insert
+org.eclipse.jdt.core.formatter.insert_space_before_opening_paren_in_synchronized=insert
+org.eclipse.jdt.core.formatter.insert_space_before_opening_paren_in_try=insert
+org.eclipse.jdt.core.formatter.insert_space_before_opening_paren_in_while=insert
+org.eclipse.jdt.core.formatter.insert_space_before_parenthesized_expression_in_return=insert
+org.eclipse.jdt.core.formatter.insert_space_before_parenthesized_expression_in_throw=insert
+org.eclipse.jdt.core.formatter.insert_space_before_postfix_operator=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_prefix_operator=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_question_in_conditional=insert
+org.eclipse.jdt.core.formatter.insert_space_before_question_in_wildcard=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_semicolon=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_semicolon_in_for=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_semicolon_in_try_resources=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_unary_operator=do not insert
+org.eclipse.jdt.core.formatter.insert_space_between_brackets_in_array_type_reference=do not insert
+org.eclipse.jdt.core.formatter.insert_space_between_empty_braces_in_array_initializer=do not insert
+org.eclipse.jdt.core.formatter.insert_space_between_empty_brackets_in_array_allocation_expression=do not insert
+org.eclipse.jdt.core.formatter.insert_space_between_empty_parens_in_annotation_type_member_declaration=do not insert
+org.eclipse.jdt.core.formatter.insert_space_between_empty_parens_in_constructor_declaration=do not insert
+org.eclipse.jdt.core.formatter.insert_space_between_empty_parens_in_enum_constant=do not insert
+org.eclipse.jdt.core.formatter.insert_space_between_empty_parens_in_method_declaration=do not insert
+org.eclipse.jdt.core.formatter.insert_space_between_empty_parens_in_method_invocation=do not insert
+org.eclipse.jdt.core.formatter.join_lines_in_comments=true
+org.eclipse.jdt.core.formatter.join_wrapped_lines=true
+org.eclipse.jdt.core.formatter.keep_else_statement_on_same_line=false
+org.eclipse.jdt.core.formatter.keep_empty_array_initializer_on_one_line=false
+org.eclipse.jdt.core.formatter.keep_imple_if_on_one_line=false
+org.eclipse.jdt.core.formatter.keep_then_statement_on_same_line=false
+org.eclipse.jdt.core.formatter.lineSplit=999
+org.eclipse.jdt.core.formatter.never_indent_block_comments_on_first_column=false
+org.eclipse.jdt.core.formatter.never_indent_line_comments_on_first_column=false
+org.eclipse.jdt.core.formatter.number_of_blank_lines_at_beginning_of_method_body=0
+org.eclipse.jdt.core.formatter.number_of_empty_lines_to_preserve=1
+org.eclipse.jdt.core.formatter.put_empty_statement_on_new_line=true
+org.eclipse.jdt.core.formatter.tabulation.char=space
+org.eclipse.jdt.core.formatter.tabulation.size=4
+org.eclipse.jdt.core.formatter.use_on_off_tags=false
+org.eclipse.jdt.core.formatter.use_tabs_only_for_leading_indentations=false
+org.eclipse.jdt.core.formatter.wrap_before_binary_operator=true
+org.eclipse.jdt.core.formatter.wrap_before_or_operator_multicatch=true
+org.eclipse.jdt.core.formatter.wrap_outer_expressions_when_nested=true

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/jdbc/.settings/org.eclipse.jdt.ui.prefs
----------------------------------------------------------------------
diff --git a/jdbc/.settings/org.eclipse.jdt.ui.prefs b/jdbc/.settings/org.eclipse.jdt.ui.prefs
new file mode 100644
index 0000000..dece0e6
--- /dev/null
+++ b/jdbc/.settings/org.eclipse.jdt.ui.prefs
@@ -0,0 +1,7 @@
+eclipse.preferences.version=1
+formatter_profile=_Space Indent & Long Lines
+formatter_settings_version=12
+org.eclipse.jdt.ui.ignorelowercasenames=true
+org.eclipse.jdt.ui.importorder=java;javax;org;com;
+org.eclipse.jdt.ui.ondemandthreshold=99
+org.eclipse.jdt.ui.staticondemandthreshold=1

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/jdbc/pom.xml
----------------------------------------------------------------------
diff --git a/jdbc/pom.xml b/jdbc/pom.xml
new file mode 100644
index 0000000..e69cac0
--- /dev/null
+++ b/jdbc/pom.xml
@@ -0,0 +1,80 @@
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+	<modelVersion>4.0.0</modelVersion>
+	<artifactId>kylin-jdbc</artifactId>
+	<packaging>jar</packaging>
+
+	<!-- More project information. -->
+	<name>Kylin:Jdbc</name>
+	<description>kylin jdbc on optiq avatica</description>
+	<url>https://github.scm.corp.ebay.com/BIPlatform/Kylin</url>
+
+	<parent>
+		<groupId>com.kylinolap</groupId>
+		<artifactId>kylin</artifactId>
+		<version>0.6.3-SNAPSHOT</version>
+	</parent>
+
+	<properties>
+		<project.build.sourceEncoding>UTF-8</project.build.sourceEncoding>
+	</properties>
+
+	<dependencies>
+		<dependency>
+			<groupId>org.apache.calcite</groupId>
+			<artifactId>calcite-core</artifactId>
+		</dependency>
+		<dependency>
+			<groupId>org.apache.calcite</groupId>
+			<artifactId>calcite-avatica</artifactId>
+		</dependency>
+		<dependency>
+			<groupId>commons-httpclient</groupId>
+			<artifactId>commons-httpclient</artifactId>
+		</dependency>
+		<dependency>
+			<groupId>log4j</groupId>
+			<artifactId>log4j</artifactId>
+		</dependency>
+		<dependency>
+			<groupId>org.slf4j</groupId>
+			<artifactId>jcl-over-slf4j</artifactId>
+		</dependency>
+		<dependency>
+			<groupId>org.slf4j</groupId>
+			<artifactId>slf4j-api</artifactId>
+		</dependency>
+		<dependency>
+			<groupId>org.slf4j</groupId>
+			<artifactId>slf4j-log4j12</artifactId>
+		</dependency>
+		<dependency>
+			<groupId>junit</groupId>
+			<artifactId>junit</artifactId>
+			<scope>test</scope>
+		</dependency>
+	</dependencies>
+
+	<build>
+		<plugins>
+			<plugin>
+				<artifactId>maven-assembly-plugin</artifactId>
+				<configuration>
+					<descriptorRefs>
+						<descriptorRef>jar-with-dependencies</descriptorRef>
+					</descriptorRefs>
+					<appendAssemblyId>false</appendAssemblyId>
+				</configuration>
+				<executions>
+					<execution>
+						<id>make-assembly</id>
+						<phase>package</phase>
+						<goals>
+							<goal>single</goal>
+						</goals>
+					</execution>
+				</executions>
+			</plugin>
+		</plugins>
+	</build>
+
+</project>


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

Posted by lu...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/atopcalcite/src/main/java/net/hydromatic/optiq/runtime/SqlFunctions.java
----------------------------------------------------------------------
diff --git a/atopcalcite/src/main/java/net/hydromatic/optiq/runtime/SqlFunctions.java b/atopcalcite/src/main/java/net/hydromatic/optiq/runtime/SqlFunctions.java
new file mode 100644
index 0000000..f9ba797
--- /dev/null
+++ b/atopcalcite/src/main/java/net/hydromatic/optiq/runtime/SqlFunctions.java
@@ -0,0 +1,1705 @@
+/*
+ * OVERRIDE POINTS:
+ * - divide(BigDecimal,BigDecimal), was `b0.divide(b1)`, now `b0.divide(b1, MathContext.DECIMAL64);` 
+ */
+
+/*
+ * 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 net.hydromatic.optiq.runtime;
+
+import net.hydromatic.avatica.ByteString;
+
+import net.hydromatic.linq4j.Enumerable;
+import net.hydromatic.linq4j.Linq4j;
+import net.hydromatic.linq4j.expressions.Primitive;
+import net.hydromatic.linq4j.function.Deterministic;
+import net.hydromatic.linq4j.function.Function1;
+import net.hydromatic.linq4j.function.NonDeterministic;
+
+import net.hydromatic.optiq.DataContext;
+
+import org.eigenbase.util14.DateTimeUtil;
+
+import java.math.*;
+import java.sql.SQLException;
+import java.sql.Timestamp;
+import java.text.DecimalFormat;
+import java.util.*;
+import java.util.regex.Pattern;
+
+/**
+ * Helper methods to implement SQL functions in generated code.
+ *
+ * <p>Not present: and, or, not (builtin operators are better, because they
+ * use lazy evaluation. Implementations do not check for null values; the
+ * calling code must do that.</p>
+ *
+ * <p>Many of the functions do not check for null values. This is intentional.
+ * If null arguments are possible, the code-generation framework checks for
+ * nulls before calling the functions.</p>
+ */
+@SuppressWarnings({"unused", "rawtypes", "unchecked"})
+@Deterministic
+public class SqlFunctions {
+  private static final DecimalFormat DOUBLE_FORMAT =
+      new DecimalFormat("0.0E0");
+
+  /** The julian date of the epoch, 1970-01-01. */
+  public static final int EPOCH_JULIAN = 2440588;
+
+  private static final TimeZone LOCAL_TZ = TimeZone.getDefault();
+
+  private static final Function1<List<Object>, Enumerable<Object>>
+  LIST_AS_ENUMERABLE =
+      new Function1<List<Object>, Enumerable<Object>>() {
+        public Enumerable<Object> apply(List<Object> list) {
+          return Linq4j.asEnumerable(list);
+        }
+      };
+
+  private SqlFunctions() {
+  }
+
+  /** SQL SUBSTRING(string FROM ... FOR ...) function. */
+  public static String substring(String s, int from, int for_) {
+    return s.substring(from - 1, Math.min(from - 1 + for_, s.length()));
+  }
+
+  /** SQL SUBSTRING(string FROM ...) function. */
+  public static String substring(String s, int from) {
+    return s.substring(from - 1);
+  }
+
+  /** SQL UPPER(string) function. */
+  public static String upper(String s) {
+    return s.toUpperCase();
+  }
+
+  /** SQL LOWER(string) function. */
+  public static String lower(String s) {
+    return s.toLowerCase();
+  }
+
+  /** SQL INITCAP(string) function. */
+  public static String initcap(String s) {
+    // Assumes Alpha as [A-Za-z0-9]
+    // white space is treated as everything else.
+    final int len = s.length();
+    boolean start = true;
+    final StringBuilder newS = new StringBuilder();
+
+    for (int i = 0; i < len; i++) {
+      char curCh = s.charAt(i);
+      final int c = (int) curCh;
+      if (start) {  // curCh is whitespace or first character of word.
+        if (c > 47 && c < 58) { // 0-9
+          start = false;
+        } else if (c > 64 && c < 91) {  // A-Z
+          start = false;
+        } else if (c > 96 && c < 123) {  // a-z
+          start = false;
+          curCh = (char) (c - 32); // Uppercase this character
+        }
+        // else {} whitespace
+      } else {  // Inside of a word or white space after end of word.
+        if (c > 47 && c < 58) { // 0-9
+          // noop
+        } else if (c > 64 && c < 91) {  // A-Z
+          curCh = (char) (c + 32); // Lowercase this character
+        } else if (c > 96 && c < 123) {  // a-z
+          // noop
+        } else { // whitespace
+          start = true;
+        }
+      }
+      newS.append(curCh);
+    } // for each character in s
+    return newS.toString();
+  }
+
+  /** SQL CHARACTER_LENGTH(string) function. */
+  public static int charLength(String s) {
+    return s.length();
+  }
+
+  /** SQL {@code string || string} operator. */
+  public static String concat(String s0, String s1) {
+    return s0 + s1;
+  }
+
+  /** SQL {@code binary || binary} operator. */
+  public static ByteString concat(ByteString s0, ByteString s1) {
+    return s0.concat(s1);
+  }
+
+  /** SQL {@code RTRIM} function applied to string. */
+  public static String rtrim(String s) {
+    return trim_(s, false, true, ' ');
+  }
+
+  /** SQL {@code LTRIM} function. */
+  public static String ltrim(String s) {
+    return trim_(s, true, false, ' ');
+  }
+
+  /** SQL {@code TRIM(... seek FROM s)} function. */
+  public static String trim(boolean leading, boolean trailing, String seek,
+      String s) {
+    return trim_(s, leading, trailing, seek.charAt(0));
+  }
+
+  /** SQL {@code TRIM} function. */
+  private static String trim_(String s, boolean left, boolean right, char c) {
+    int j = s.length();
+    if (right) {
+      for (;;) {
+        if (j == 0) {
+          return "";
+        }
+        if (s.charAt(j - 1) != c) {
+          break;
+        }
+        --j;
+      }
+    }
+    int i = 0;
+    if (left) {
+      for (;;) {
+        if (i == j) {
+          return "";
+        }
+        if (s.charAt(i) != c) {
+          break;
+        }
+        ++i;
+      }
+    }
+    return s.substring(i, j);
+  }
+
+  /** SQL {@code TRIM} function applied to binary string. */
+  public static ByteString trim(ByteString s) {
+    return trim_(s, true, true);
+  }
+
+  /** Helper for CAST. */
+  public static ByteString rtrim(ByteString s) {
+    return trim_(s, false, true);
+  }
+
+  /** SQL {@code TRIM} function applied to binary string. */
+  private static ByteString trim_(ByteString s, boolean left, boolean right) {
+    int j = s.length();
+    if (right) {
+      for (;;) {
+        if (j == 0) {
+          return ByteString.EMPTY;
+        }
+        if (s.byteAt(j - 1) != 0) {
+          break;
+        }
+        --j;
+      }
+    }
+    int i = 0;
+    if (left) {
+      for (;;) {
+        if (i == j) {
+          return ByteString.EMPTY;
+        }
+        if (s.byteAt(i) != 0) {
+          break;
+        }
+        ++i;
+      }
+    }
+    return s.substring(i, j);
+  }
+
+  /** SQL {@code OVERLAY} function. */
+  public static String overlay(String s, String r, int start) {
+    if (s == null || r == null) {
+      return null;
+    }
+    return s.substring(0, start - 1)
+        + r
+        + s.substring(start - 1 + r.length());
+  }
+
+  /** SQL {@code OVERLAY} function. */
+  public static String overlay(String s, String r, int start, int length) {
+    if (s == null || r == null) {
+      return null;
+    }
+    return s.substring(0, start - 1)
+        + r
+        + s.substring(start - 1 + length);
+  }
+
+  /** SQL {@code OVERLAY} function applied to binary strings. */
+  public static ByteString overlay(ByteString s, ByteString r, int start) {
+    if (s == null || r == null) {
+      return null;
+    }
+    return s.substring(0, start - 1)
+           .concat(r)
+           .concat(s.substring(start - 1 + r.length()));
+  }
+
+  /** SQL {@code OVERLAY} function applied to binary strings. */
+  public static ByteString overlay(ByteString s, ByteString r, int start,
+      int length) {
+    if (s == null || r == null) {
+      return null;
+    }
+    return s.substring(0, start - 1)
+           .concat(r)
+           .concat(s.substring(start - 1 + length));
+  }
+
+  /** SQL {@code LIKE} function. */
+  public static boolean like(String s, String pattern) {
+    final String regex = Like.sqlToRegexLike(pattern, null);
+    return Pattern.matches(regex, s);
+  }
+
+  /** SQL {@code LIKE} function with escape. */
+  public static boolean like(String s, String pattern, String escape) {
+    final String regex = Like.sqlToRegexLike(pattern, escape);
+    return Pattern.matches(regex, s);
+  }
+
+  /** SQL {@code SIMILAR} function. */
+  public static boolean similar(String s, String pattern) {
+    final String regex = Like.sqlToRegexSimilar(pattern, null);
+    return Pattern.matches(regex, s);
+  }
+
+  /** SQL {@code SIMILAR} function with escape. */
+  public static boolean similar(String s, String pattern, String escape) {
+    final String regex = Like.sqlToRegexSimilar(pattern, escape);
+    return Pattern.matches(regex, s);
+  }
+
+  // =
+
+  /** SQL = operator applied to Object values (including String; neither
+   * side may be null). */
+  public static boolean eq(Object b0, Object b1) {
+    return b0.equals(b1);
+  }
+
+  /** SQL = operator applied to BigDecimal values (neither may be null). */
+  public static boolean eq(BigDecimal b0, BigDecimal b1) {
+    return b0.stripTrailingZeros().equals(b1.stripTrailingZeros());
+  }
+
+  // <>
+
+  /** SQL &lt;&gt; operator applied to Object values (including String;
+   * neither side may be null). */
+  public static boolean ne(Object b0, Object b1) {
+    return !b0.equals(b1);
+  }
+
+  /** SQL &lt;&gt; operator applied to BigDecimal values. */
+  public static boolean ne(BigDecimal b0, BigDecimal b1) {
+    return b0.compareTo(b1) != 0;
+  }
+
+  // <
+
+  /** SQL &lt; operator applied to boolean values. */
+  public static boolean lt(boolean b0, boolean b1) {
+    return compare(b0, b1) < 0;
+  }
+
+  /** SQL &lt; operator applied to String values. */
+  public static boolean lt(String b0, String b1) {
+    return b0.compareTo(b1) < 0;
+  }
+
+  /** SQL &lt; operator applied to ByteString values. */
+  public static boolean lt(ByteString b0, ByteString b1) {
+    return b0.compareTo(b1) < 0;
+  }
+
+  /** SQL &lt; operator applied to BigDecimal values. */
+  public static boolean lt(BigDecimal b0, BigDecimal b1) {
+    return b0.compareTo(b1) < 0;
+  }
+
+  // <=
+
+  /** SQL &le; operator applied to boolean values. */
+  public static boolean le(boolean b0, boolean b1) {
+    return compare(b0, b1) <= 0;
+  }
+
+  /** SQL &le; operator applied to String values. */
+  public static boolean le(String b0, String b1) {
+    return b0.compareTo(b1) <= 0;
+  }
+
+  /** SQL &le; operator applied to ByteString values. */
+  public static boolean le(ByteString b0, ByteString b1) {
+    return b0.compareTo(b1) <= 0;
+  }
+
+  /** SQL &le; operator applied to BigDecimal values. */
+  public static boolean le(BigDecimal b0, BigDecimal b1) {
+    return b0.compareTo(b1) <= 0;
+  }
+
+  // >
+
+  /** SQL &gt; operator applied to boolean values. */
+  public static boolean gt(boolean b0, boolean b1) {
+    return compare(b0, b1) > 0;
+  }
+
+  /** SQL &gt; operator applied to String values. */
+  public static boolean gt(String b0, String b1) {
+    return b0.compareTo(b1) > 0;
+  }
+
+  /** SQL &gt; operator applied to ByteString values. */
+  public static boolean gt(ByteString b0, ByteString b1) {
+    return b0.compareTo(b1) > 0;
+  }
+
+  /** SQL &gt; operator applied to BigDecimal values. */
+  public static boolean gt(BigDecimal b0, BigDecimal b1) {
+    return b0.compareTo(b1) > 0;
+  }
+
+  // >=
+
+  /** SQL &ge; operator applied to boolean values. */
+  public static boolean ge(boolean b0, boolean b1) {
+    return compare(b0, b1) >= 0;
+  }
+
+  /** SQL &ge; operator applied to String values. */
+  public static boolean ge(String b0, String b1) {
+    return b0.compareTo(b1) >= 0;
+  }
+
+  /** SQL &ge; operator applied to ByteString values. */
+  public static boolean ge(ByteString b0, ByteString b1) {
+    return b0.compareTo(b1) >= 0;
+  }
+
+  /** SQL &ge; operator applied to BigDecimal values. */
+  public static boolean ge(BigDecimal b0, BigDecimal b1) {
+    return b0.compareTo(b1) >= 0;
+  }
+
+  // +
+
+  /** SQL <code>+</code> operator applied to int values. */
+  public static int plus(int b0, int b1) {
+    return b0 + b1;
+  }
+
+  /** SQL <code>+</code> operator applied to int values; left side may be
+   * null. */
+  public static Integer plus(Integer b0, int b1) {
+    return b0 == null ? null : (b0 + b1);
+  }
+
+  /** SQL <code>+</code> operator applied to int values; right side may be
+   * null. */
+  public static Integer plus(int b0, Integer b1) {
+    return b1 == null ? null : (b0 + b1);
+  }
+
+  /** SQL <code>+</code> operator applied to nullable int values. */
+  public static Integer plus(Integer b0, Integer b1) {
+    return (b0 == null || b1 == null) ? null : (b0 + b1);
+  }
+
+  /** SQL <code>+</code> operator applied to nullable long and int values. */
+  public static Long plus(Long b0, Integer b1) {
+    return (b0 == null || b1 == null)
+        ? null
+        : (b0.longValue() + b1.longValue());
+  }
+
+  /** SQL <code>+</code> operator applied to nullable int and long values. */
+  public static Long plus(Integer b0, Long b1) {
+    return (b0 == null || b1 == null)
+        ? null
+        : (b0.longValue() + b1.longValue());
+  }
+
+  /** SQL <code>+</code> operator applied to BigDecimal values. */
+  public static BigDecimal plus(BigDecimal b0, BigDecimal b1) {
+    return (b0 == null || b1 == null) ? null : b0.add(b1);
+  }
+
+  // -
+
+  /** SQL <code>-</code> operator applied to int values. */
+  public static int minus(int b0, int b1) {
+    return b0 - b1;
+  }
+
+  /** SQL <code>-</code> operator applied to int values; left side may be
+   * null. */
+  public static Integer minus(Integer b0, int b1) {
+    return b0 == null ? null : (b0 - b1);
+  }
+
+  /** SQL <code>-</code> operator applied to int values; right side may be
+   * null. */
+  public static Integer minus(int b0, Integer b1) {
+    return b1 == null ? null : (b0 - b1);
+  }
+
+  /** SQL <code>-</code> operator applied to nullable int values. */
+  public static Integer minus(Integer b0, Integer b1) {
+    return (b0 == null || b1 == null) ? null : (b0 - b1);
+  }
+
+  /** SQL <code>-</code> operator applied to nullable long and int values. */
+  public static Long minus(Long b0, Integer b1) {
+    return (b0 == null || b1 == null)
+        ? null
+        : (b0.longValue() - b1.longValue());
+  }
+
+  /** SQL <code>-</code> operator applied to nullable int and long values. */
+  public static Long minus(Integer b0, Long b1) {
+    return (b0 == null || b1 == null)
+        ? null
+        : (b0.longValue() - b1.longValue());
+  }
+
+  /** SQL <code>-</code> operator applied to BigDecimal values. */
+  public static BigDecimal minus(BigDecimal b0, BigDecimal b1) {
+    return (b0 == null || b1 == null) ? null : b0.subtract(b1);
+  }
+
+  // /
+
+  /** SQL <code>/</code> operator applied to int values. */
+  public static int divide(int b0, int b1) {
+    return b0 / b1;
+  }
+
+  /** SQL <code>/</code> operator applied to int values; left side may be
+   * null. */
+  public static Integer divide(Integer b0, int b1) {
+    return b0 == null ? null : (b0 / b1);
+  }
+
+  /** SQL <code>/</code> operator applied to int values; right side may be
+   * null. */
+  public static Integer divide(int b0, Integer b1) {
+    return b1 == null ? null : (b0 / b1);
+  }
+
+  /** SQL <code>/</code> operator applied to nullable int values. */
+  public static Integer divide(Integer b0, Integer b1) {
+    return (b0 == null || b1 == null) ? null : (b0 / b1);
+  }
+
+  /** SQL <code>/</code> operator applied to nullable long and int values. */
+  public static Long divide(Long b0, Integer b1) {
+    return (b0 == null || b1 == null)
+        ? null
+        : (b0.longValue() / b1.longValue());
+  }
+
+  /** SQL <code>/</code> operator applied to nullable int and long values. */
+  public static Long divide(Integer b0, Long b1) {
+    return (b0 == null || b1 == null)
+        ? null
+        : (b0.longValue() / b1.longValue());
+  }
+
+  /** SQL <code>/</code> operator applied to BigDecimal values. */
+  public static BigDecimal divide(BigDecimal b0, BigDecimal b1) {
+    // OVERRIDE POINT
+    return (b0 == null || b1 == null) ? null : b0.divide(b1, MathContext.DECIMAL64);
+  }
+
+  // *
+
+  /** SQL <code>*</code> operator applied to int values. */
+  public static int multiply(int b0, int b1) {
+    return b0 * b1;
+  }
+
+  /** SQL <code>*</code> operator applied to int values; left side may be
+   * null. */
+  public static Integer multiply(Integer b0, int b1) {
+    return b0 == null ? null : (b0 * b1);
+  }
+
+  /** SQL <code>*</code> operator applied to int values; right side may be
+   * null. */
+  public static Integer multiply(int b0, Integer b1) {
+    return b1 == null ? null : (b0 * b1);
+  }
+
+  /** SQL <code>*</code> operator applied to nullable int values. */
+  public static Integer multiply(Integer b0, Integer b1) {
+    return (b0 == null || b1 == null) ? null : (b0 * b1);
+  }
+
+  /** SQL <code>*</code> operator applied to nullable long and int values. */
+  public static Long multiply(Long b0, Integer b1) {
+    return (b0 == null || b1 == null)
+        ? null
+        : (b0.longValue() * b1.longValue());
+  }
+
+  /** SQL <code>*</code> operator applied to nullable int and long values. */
+  public static Long multiply(Integer b0, Long b1) {
+    return (b0 == null || b1 == null)
+        ? null
+        : (b0.longValue() * b1.longValue());
+  }
+
+  /** SQL <code>*</code> operator applied to BigDecimal values. */
+  public static BigDecimal multiply(BigDecimal b0, BigDecimal b1) {
+    return (b0 == null || b1 == null) ? null : b0.multiply(b1);
+  }
+
+  // EXP
+
+  /** SQL <code>EXP</code> operator applied to double values. */
+  public static double exp(double b0) {
+    return Math.exp(b0);
+  }
+
+  public static double exp(long b0) {
+    return Math.exp(b0);
+  }
+
+  // POWER
+
+  /** SQL <code>POWER</code> operator applied to double values. */
+  public static double power(double b0, double b1) {
+    return Math.pow(b0, b1);
+  }
+
+  public static double power(long b0, long b1) {
+    return Math.pow(b0, b1);
+  }
+
+  public static double power(long b0, BigDecimal b1) {
+    return Math.pow(b0, b1.doubleValue());
+  }
+
+  // LN
+
+  /** SQL {@code LN(number)} function applied to double values. */
+  public static double ln(double d) {
+    return Math.log(d);
+  }
+
+  /** SQL {@code LN(number)} function applied to long values. */
+  public static double ln(long b0) {
+    return Math.log(b0);
+  }
+
+  /** SQL {@code LN(number)} function applied to BigDecimal values. */
+  public static double ln(BigDecimal d) {
+    return Math.log(d.doubleValue());
+  }
+
+  // LOG10
+
+  /** SQL <code>LOG10(numeric)</code> operator applied to double values. */
+  public static double log10(double b0) {
+    return Math.log10(b0);
+  }
+
+  /** SQL {@code LOG10(number)} function applied to long values. */
+  public static double log10(long b0) {
+    return Math.log10(b0);
+  }
+
+  /** SQL {@code LOG10(number)} function applied to BigDecimal values. */
+  public static double log10(BigDecimal d) {
+    return Math.log10(d.doubleValue());
+  }
+
+  // MOD
+
+  /** SQL <code>MOD</code> operator applied to byte values. */
+  public static byte mod(byte b0, byte b1) {
+    return (byte) (b0 % b1);
+  }
+
+  /** SQL <code>MOD</code> operator applied to short values. */
+  public static short mod(short b0, short b1) {
+    return (short) (b0 % b1);
+  }
+
+  /** SQL <code>MOD</code> operator applied to int values. */
+  public static int mod(int b0, int b1) {
+    return b0 % b1;
+  }
+
+  /** SQL <code>MOD</code> operator applied to long values. */
+  public static long mod(long b0, long b1) {
+    return b0 % b1;
+  }
+
+  // temporary
+  public static BigDecimal mod(BigDecimal b0, int b1) {
+    return mod(b0, BigDecimal.valueOf(b1));
+  }
+
+  // temporary
+  public static int mod(int b0, BigDecimal b1) {
+    return mod(b0, b1.intValue());
+  }
+
+  public static BigDecimal mod(BigDecimal b0, BigDecimal b1) {
+    final BigDecimal[] bigDecimals = b0.divideAndRemainder(b1);
+    return bigDecimals[1];
+  }
+
+  // ABS
+
+  /** SQL <code>ABS</code> operator applied to byte values. */
+  public static byte abs(byte b0) {
+    return (byte) Math.abs(b0);
+  }
+
+  /** SQL <code>ABS</code> operator applied to short values. */
+  public static short abs(short b0) {
+    return (short) Math.abs(b0);
+  }
+
+  /** SQL <code>ABS</code> operator applied to int values. */
+  public static int abs(int b0) {
+    return Math.abs(b0);
+  }
+
+  /** SQL <code>ABS</code> operator applied to long values. */
+  public static long abs(long b0) {
+    return Math.abs(b0);
+  }
+
+  /** SQL <code>ABS</code> operator applied to float values. */
+  public static float abs(float b0) {
+    return Math.abs(b0);
+  }
+
+  /** SQL <code>ABS</code> operator applied to double values. */
+  public static double abs(double b0) {
+    return Math.abs(b0);
+  }
+
+  /** SQL <code>ABS</code> operator applied to BigDecimal values. */
+  public static BigDecimal abs(BigDecimal b0) {
+    return b0.abs();
+  }
+
+  // Helpers
+
+  /** Helper for implementing MIN. Somewhat similar to LEAST operator. */
+  public static <T extends Comparable<T>> T lesser(T b0, T b1) {
+    return b0 == null || b0.compareTo(b1) > 0 ? b1 : b0;
+  }
+
+  /** LEAST operator. */
+  public static <T extends Comparable<T>> T least(T b0, T b1) {
+    return b0 == null || b1 != null && b0.compareTo(b1) > 0 ? b1 : b0;
+  }
+
+  public static boolean greater(boolean b0, boolean b1) {
+    return b0 || b1;
+  }
+
+  public static boolean lesser(boolean b0, boolean b1) {
+    return b0 && b1;
+  }
+
+  public static byte greater(byte b0, byte b1) {
+    return b0 > b1 ? b0 : b1;
+  }
+
+  public static byte lesser(byte b0, byte b1) {
+    return b0 > b1 ? b1 : b0;
+  }
+
+  public static char greater(char b0, char b1) {
+    return b0 > b1 ? b0 : b1;
+  }
+
+  public static char lesser(char b0, char b1) {
+    return b0 > b1 ? b1 : b0;
+  }
+
+  public static short greater(short b0, short b1) {
+    return b0 > b1 ? b0 : b1;
+  }
+
+  public static short lesser(short b0, short b1) {
+    return b0 > b1 ? b1 : b0;
+  }
+
+  public static int greater(int b0, int b1) {
+    return b0 > b1 ? b0 : b1;
+  }
+
+  public static int lesser(int b0, int b1) {
+    return b0 > b1 ? b1 : b0;
+  }
+
+  public static long greater(long b0, long b1) {
+    return b0 > b1 ? b0 : b1;
+  }
+
+  public static long lesser(long b0, long b1) {
+    return b0 > b1 ? b1 : b0;
+  }
+
+  public static float greater(float b0, float b1) {
+    return b0 > b1 ? b0 : b1;
+  }
+
+  public static float lesser(float b0, float b1) {
+    return b0 > b1 ? b1 : b0;
+  }
+
+  public static double greater(double b0, double b1) {
+    return b0 > b1 ? b0 : b1;
+  }
+
+  public static double lesser(double b0, double b1) {
+    return b0 > b1 ? b1 : b0;
+  }
+
+  /** Helper for implementing MAX. Somewhat similar to GREATEST operator. */
+  public static <T extends Comparable<T>> T greater(T b0, T b1) {
+    return b0 == null || b0.compareTo(b1) < 0 ? b1 : b0;
+  }
+
+  /** GREATEST operator. */
+  public static <T extends Comparable<T>> T greatest(T b0, T b1) {
+    return b0 == null || b1 != null && b0.compareTo(b1) < 0 ? b1 : b0;
+  }
+
+  /** Boolean comparison. */
+  public static int compare(boolean x, boolean y) {
+    return x == y ? 0 : x ? 1 : -1;
+  }
+
+  /** CAST(FLOAT AS VARCHAR). */
+  public static String toString(float x) {
+    if (x == 0) {
+      return "0E0";
+    }
+    BigDecimal bigDecimal =
+        new BigDecimal(x, MathContext.DECIMAL32).stripTrailingZeros();
+    final String s = bigDecimal.toString();
+    return s.replaceAll("0*E", "E").replace("E+", "E");
+  }
+
+  /** CAST(DOUBLE AS VARCHAR). */
+  public static String toString(double x) {
+    if (x == 0) {
+      return "0E0";
+    }
+    BigDecimal bigDecimal =
+        new BigDecimal(x, MathContext.DECIMAL64).stripTrailingZeros();
+    final String s = bigDecimal.toString();
+    return s.replaceAll("0*E", "E").replace("E+", "E");
+  }
+
+  /** CAST(DECIMAL AS VARCHAR). */
+  public static String toString(BigDecimal x) {
+    final String s = x.toString();
+    if (s.startsWith("0")) {
+      // we want ".1" not "0.1"
+      return s.substring(1);
+    } else if (s.startsWith("-0")) {
+      // we want "-.1" not "-0.1"
+      return "-" + s.substring(2);
+    } else {
+      return s;
+    }
+  }
+
+  /** CAST(BOOLEAN AS VARCHAR). */
+  public static String toString(boolean x) {
+    // Boolean.toString returns lower case -- no good.
+    return x ? "TRUE" : "FALSE";
+  }
+
+  @NonDeterministic
+  private static Object cannotConvert(Object o, Class toType) {
+    throw new RuntimeException("Cannot convert " + o + " to " + toType);
+  }
+
+  /** CAST(VARCHAR AS BOOLEAN). */
+  public static boolean toBoolean(String s) {
+    s = trim_(s, true, true, ' ');
+    if (s.equalsIgnoreCase("TRUE")) {
+      return true;
+    } else if (s.equalsIgnoreCase("FALSE")) {
+      return false;
+    } else {
+      throw new RuntimeException("Invalid character for cast");
+    }
+  }
+
+  public static boolean toBoolean(Number number) {
+    return !number.equals(0);
+  }
+
+  public static boolean toBoolean(Object o) {
+    return o instanceof Boolean ? (Boolean) o
+        : o instanceof Number ? toBoolean((Number) o)
+        : o instanceof String ? toBoolean((String) o)
+        : (Boolean) cannotConvert(o, boolean.class);
+  }
+
+  // Don't need parseByte etc. - Byte.parseByte is sufficient.
+
+  public static byte toByte(Object o) {
+    return o instanceof Byte ? (Byte) o
+        : o instanceof Number ? toByte((Number) o)
+        : Byte.parseByte(o.toString());
+  }
+
+  public static byte toByte(Number number) {
+    return number.byteValue();
+  }
+
+  public static char toChar(String s) {
+    return s.charAt(0);
+  }
+
+  public static Character toCharBoxed(String s) {
+    return s.charAt(0);
+  }
+
+  public static short toShort(String s) {
+    return Short.parseShort(s.trim());
+  }
+
+  public static short toShort(Number number) {
+    return number.shortValue();
+  }
+
+  public static short toShort(Object o) {
+    return o instanceof Short ? (Short) o
+        : o instanceof Number ? toShort((Number) o)
+        : o instanceof String ? toShort((String) o)
+        : (Short) cannotConvert(o, short.class);
+  }
+
+  public static int toInt(java.util.Date v) {
+    return toInt(v, LOCAL_TZ);
+  }
+
+  public static int toInt(java.util.Date v, TimeZone timeZone) {
+    return (int) (toLong(v, timeZone)  / DateTimeUtil.MILLIS_PER_DAY);
+  }
+
+  public static Integer toIntOptional(java.util.Date v) {
+    return v == null ? null : toInt(v);
+  }
+
+  public static Integer toIntOptional(java.util.Date v, TimeZone timeZone) {
+    return v == null
+        ? null
+        : toInt(v, timeZone);
+  }
+
+  public static long toLong(Date v) {
+    return toLong(v, LOCAL_TZ);
+  }
+
+  public static int toInt(java.sql.Time v) {
+    return (int) (toLong(v) % DateTimeUtil.MILLIS_PER_DAY);
+  }
+
+  public static Integer toIntOptional(java.sql.Time v) {
+    return v == null ? null : toInt(v);
+  }
+
+  public static int toInt(String s) {
+    return Integer.parseInt(s.trim());
+  }
+
+  public static int toInt(Number number) {
+    return number.intValue();
+  }
+
+  public static int toInt(Object o) {
+    return o instanceof Integer ? (Integer) o
+        : o instanceof Number ? toInt((Number) o)
+        : o instanceof String ? toInt((String) o)
+        : (Integer) cannotConvert(o, int.class);
+  }
+
+  public static long toLong(Timestamp v) {
+    return toLong(v, LOCAL_TZ);
+  }
+
+  // mainly intended for java.sql.Timestamp but works for other dates also
+  public static long toLong(java.util.Date v, TimeZone timeZone) {
+    final long time = v.getTime();
+    return time + timeZone.getOffset(time);
+  }
+
+  // mainly intended for java.sql.Timestamp but works for other dates also
+  public static Long toLongOptional(java.util.Date v) {
+    return v == null ? null : toLong(v, LOCAL_TZ);
+  }
+
+  public static Long toLongOptional(Timestamp v, TimeZone timeZone) {
+    if (v == null) {
+      return null;
+    }
+    return toLong(v, LOCAL_TZ);
+  }
+
+  public static long toLong(String s) {
+    if (s.startsWith("199") && s.contains(":")) {
+      return Timestamp.valueOf(s).getTime();
+    }
+    return Long.parseLong(s.trim());
+  }
+
+  public static long toLong(Number number) {
+    return number.longValue();
+  }
+
+  public static long toLong(Object o) {
+    return o instanceof Long ? (Long) o
+        : o instanceof Number ? toLong((Number) o)
+        : o instanceof String ? toLong((String) o)
+        : (Long) cannotConvert(o, long.class);
+  }
+
+  public static float toFloat(String s) {
+    return Float.parseFloat(s.trim());
+  }
+
+  public static float toFloat(Number number) {
+    return number.floatValue();
+  }
+
+  public static float toFloat(Object o) {
+    return o instanceof Float ? (Float) o
+        : o instanceof Number ? toFloat((Number) o)
+            : o instanceof String ? toFloat((String) o)
+                : (Float) cannotConvert(o, float.class);
+  }
+
+  public static double toDouble(String s) {
+    return Double.parseDouble(s.trim());
+  }
+
+  public static double toDouble(Number number) {
+    return number.doubleValue();
+  }
+
+  public static double toDouble(Object o) {
+    return o instanceof Double ? (Double) o
+        : o instanceof Number ? toDouble((Number) o)
+        : o instanceof String ? toDouble((String) o)
+        : (Double) cannotConvert(o, double.class);
+  }
+
+  public static BigDecimal toBigDecimal(String s) {
+    return new BigDecimal(s.trim());
+  }
+
+  public static BigDecimal toBigDecimal(Number number) {
+    // There are some values of "long" that cannot be represented as "double".
+    // Not so "int". If it isn't a long, go straight to double.
+    return number instanceof BigDecimal ? (BigDecimal) number
+        : number instanceof BigInteger ? new BigDecimal((BigInteger) number)
+        : number instanceof Long ? new BigDecimal(number.longValue())
+        : new BigDecimal(number.doubleValue());
+  }
+
+  public static BigDecimal toBigDecimal(Object o) {
+    return o instanceof Number ? toBigDecimal((Number) o)
+        : toBigDecimal(o.toString());
+  }
+
+  // Don't need shortValueOf etc. - Short.valueOf is sufficient.
+
+  /** Helper for CAST(... AS VARCHAR(maxLength)). */
+  public static String truncate(String s, int maxLength) {
+    return s == null ? null
+        : s.length() > maxLength ? s.substring(0, maxLength)
+        : s;
+  }
+
+  /** Helper for CAST(... AS VARBINARY(maxLength)). */
+  public static ByteString truncate(ByteString s, int maxLength) {
+    return s == null ? null
+        : s.length() > maxLength ? s.substring(0, maxLength)
+        : s;
+  }
+
+  /** SQL {@code POSITION(seek IN string)} function. */
+  public static int position(String seek, String s) {
+    return s.indexOf(seek) + 1;
+  }
+
+  /** SQL {@code POSITION(seek IN string)} function. */
+  public static int position(ByteString seek, ByteString s) {
+    return s.indexOf(seek) + 1;
+  }
+
+  /** Cheap, unsafe, long power. power(2, 3) returns 8. */
+  public static long powerX(long a, long b) {
+    long x = 1;
+    while (b > 0) {
+      x *= a;
+      --b;
+    }
+    return x;
+  }
+
+  /** Helper for rounding. Truncate(12345, 1000) returns 12000. */
+  public static long round(long v, long x) {
+    return truncate(v + x / 2, x);
+  }
+
+  /** Helper for rounding. Truncate(12345, 1000) returns 12000. */
+  public static long truncate(long v, long x) {
+    long remainder = v % x;
+    if (remainder < 0) {
+      remainder += x;
+    }
+    return v - remainder;
+  }
+
+  /** Helper for rounding. Truncate(12345, 1000) returns 12000. */
+  public static int round(int v, int x) {
+    return truncate(v + x / 2, x);
+  }
+
+  /** Helper for rounding. Truncate(12345, 1000) returns 12000. */
+  public static int truncate(int v, int x) {
+    int remainder = v % x;
+    if (remainder < 0) {
+      remainder += x;
+    }
+    return v - remainder;
+  }
+
+  /** Helper for CAST({timestamp} AS VARCHAR(n)). */
+  public static String unixTimestampToString(long timestamp) {
+    final StringBuilder buf = new StringBuilder(17);
+    int date = (int) (timestamp / DateTimeUtil.MILLIS_PER_DAY);
+    int time = (int) (timestamp % DateTimeUtil.MILLIS_PER_DAY);
+    if (time < 0) {
+      --date;
+      time += DateTimeUtil.MILLIS_PER_DAY;
+    }
+    unixDateToString(buf, date);
+    buf.append(' ');
+    unixTimeToString(buf, time);
+    return buf.toString();
+  }
+
+  /** Helper for CAST({timestamp} AS VARCHAR(n)). */
+  public static String unixTimeToString(int time) {
+    final StringBuilder buf = new StringBuilder(8);
+    unixTimeToString(buf, time);
+    return buf.toString();
+  }
+
+  private static void unixTimeToString(StringBuilder buf, int time) {
+    int h = time / 3600000;
+    int time2 = time % 3600000;
+    int m = time2 / 60000;
+    int time3 = time2 % 60000;
+    int s = time3 / 1000;
+    int ms = time3 % 1000;
+    int2(buf, h);
+    buf.append(':');
+    int2(buf, m);
+    buf.append(':');
+    int2(buf, s);
+  }
+
+  /** SQL {@code CURRENT_TIMESTAMP} function. */
+  @NonDeterministic
+  public static long currentTimestamp(DataContext root) {
+    // Cast required for JDK 1.6.
+    return (Long) DataContext.Variable.CURRENT_TIMESTAMP.get(root);
+  }
+
+  /** SQL {@code CURRENT_TIME} function. */
+  @NonDeterministic
+  public static int currentTime(DataContext root) {
+    int time = (int) (currentTimestamp(root) % DateTimeUtil.MILLIS_PER_DAY);
+    if (time < 0) {
+      time += DateTimeUtil.MILLIS_PER_DAY;
+    }
+    return time;
+  }
+
+  /** SQL {@code CURRENT_DATE} function. */
+  @NonDeterministic
+  public static int currentDate(DataContext root) {
+    final long timestamp = currentTimestamp(root);
+    int date = (int) (timestamp / DateTimeUtil.MILLIS_PER_DAY);
+    final int time = (int) (timestamp % DateTimeUtil.MILLIS_PER_DAY);
+    if (time < 0) {
+      --date;
+    }
+    return date;
+  }
+
+  /** SQL {@code LOCAL_TIMESTAMP} function. */
+  @NonDeterministic
+  public static long localTimestamp(DataContext root) {
+    // Cast required for JDK 1.6.
+    return (Long) DataContext.Variable.LOCAL_TIMESTAMP.get(root);
+  }
+
+  /** SQL {@code LOCAL_TIME} function. */
+  @NonDeterministic
+  public static int localTime(DataContext root) {
+    return (int) (localTimestamp(root) % DateTimeUtil.MILLIS_PER_DAY);
+  }
+
+  private static void int2(StringBuilder buf, int i) {
+    buf.append((char) ('0' + (i / 10) % 10));
+    buf.append((char) ('0' + i % 10));
+  }
+
+  private static void int4(StringBuilder buf, int i) {
+    buf.append((char) ('0' + (i / 1000) % 10));
+    buf.append((char) ('0' + (i / 100) % 10));
+    buf.append((char) ('0' + (i / 10) % 10));
+    buf.append((char) ('0' + i % 10));
+  }
+
+  public static int dateStringToUnixDate(String s) {
+    int hyphen1 = s.indexOf('-');
+    int y;
+    int m;
+    int d;
+    if (hyphen1 < 0) {
+      y = Integer.parseInt(s.trim());
+      m = 1;
+      d = 1;
+    } else {
+      y = Integer.parseInt(s.substring(0, hyphen1).trim());
+      final int hyphen2 = s.indexOf('-', hyphen1 + 1);
+      if (hyphen2 < 0) {
+        m = Integer.parseInt(s.substring(hyphen1 + 1).trim());
+        d = 1;
+      } else {
+        m = Integer.parseInt(s.substring(hyphen1 + 1, hyphen2).trim());
+        d = Integer.parseInt(s.substring(hyphen2 + 1).trim());
+      }
+    }
+    return ymdToUnixDate(y, m, d);
+  }
+
+  public static int timeStringToUnixDate(String v) {
+    return timeStringToUnixDate(v, 0);
+  }
+
+  public static int timeStringToUnixDate(String v, int start) {
+    final int colon1 = v.indexOf(':', start);
+    int hour;
+    int minute;
+    int second;
+    int milli;
+    if (colon1 < 0) {
+      hour = Integer.parseInt(v.trim());
+      minute = 1;
+      second = 1;
+      milli = 0;
+    } else {
+      hour = Integer.parseInt(v.substring(start, colon1).trim());
+      final int colon2 = v.indexOf(':', colon1 + 1);
+      if (colon2 < 0) {
+        minute = Integer.parseInt(v.substring(colon1 + 1).trim());
+        second = 1;
+        milli = 0;
+      } else {
+        minute = Integer.parseInt(v.substring(colon1 + 1, colon2).trim());
+        int dot = v.indexOf('.', colon2);
+        if (dot < 0) {
+          second = Integer.parseInt(v.substring(colon2 + 1).trim());
+          milli = 0;
+        } else {
+          second = Integer.parseInt(v.substring(colon2 + 1, dot).trim());
+          milli = Integer.parseInt(v.substring(dot + 1).trim());
+        }
+      }
+    }
+    return hour * (int) DateTimeUtil.MILLIS_PER_HOUR
+        + minute * (int) DateTimeUtil.MILLIS_PER_MINUTE
+        + second * (int) DateTimeUtil.MILLIS_PER_SECOND
+        + milli;
+  }
+
+  public static long timestampStringToUnixDate(String s) {
+    final long d;
+    final long t;
+    s = s.trim();
+    int space = s.indexOf(' ');
+    if (space >= 0) {
+      d = dateStringToUnixDate(s.substring(0, space));
+      t = timeStringToUnixDate(s, space + 1);
+    } else {
+      d = dateStringToUnixDate(s);
+      t = 0;
+    }
+    return d * DateTimeUtil.MILLIS_PER_DAY + t;
+  }
+
+  /** Helper for CAST({date} AS VARCHAR(n)). */
+  public static String unixDateToString(int date) {
+    final StringBuilder buf = new StringBuilder(10);
+    unixDateToString(buf, date);
+    return buf.toString();
+  }
+
+  private static void unixDateToString(StringBuilder buf, int date) {
+    julianToString(buf, date + EPOCH_JULIAN);
+  }
+
+  private static void julianToString(StringBuilder buf, int julian) {
+    // this shifts the epoch back to astronomical year -4800 instead of the
+    // start of the Christian era in year AD 1 of the proleptic Gregorian
+    // calendar.
+    int j = julian + 32044;
+    int g = j / 146097;
+    int dg = j % 146097;
+    int c = (dg / 36524 + 1) * 3 / 4;
+    int dc = dg - c * 36524;
+    int b = dc / 1461;
+    int db = dc % 1461;
+    int a = (db / 365 + 1) * 3 / 4;
+    int da = db - a * 365;
+
+    // integer number of full years elapsed since March 1, 4801 BC
+    int y = g * 400 + c * 100 + b * 4 + a;
+    // integer number of full months elapsed since the last March 1
+    int m = (da * 5 + 308) / 153 - 2;
+    // number of days elapsed since day 1 of the month
+    int d = da - (m + 4) * 153 / 5 + 122;
+    int year = y - 4800 + (m + 2) / 12;
+    int month = (m + 2) % 12 + 1;
+    int day = d + 1;
+    int4(buf, year);
+    buf.append('-');
+    int2(buf, month);
+    buf.append('-');
+    int2(buf, day);
+  }
+
+  public static long unixDateExtract(TimeUnitRange range, long date) {
+    return julianExtract(range, (int) date + EPOCH_JULIAN);
+  }
+
+  private static int julianExtract(TimeUnitRange range, int julian) {
+    // this shifts the epoch back to astronomical year -4800 instead of the
+    // start of the Christian era in year AD 1 of the proleptic Gregorian
+    // calendar.
+    int j = julian + 32044;
+    int g = j / 146097;
+    int dg = j % 146097;
+    int c = (dg / 36524 + 1) * 3 / 4;
+    int dc = dg - c * 36524;
+    int b = dc / 1461;
+    int db = dc % 1461;
+    int a = (db / 365 + 1) * 3 / 4;
+    int da = db - a * 365;
+
+    // integer number of full years elapsed since March 1, 4801 BC
+    int y = g * 400 + c * 100 + b * 4 + a;
+    // integer number of full months elapsed since the last March 1
+    int m = (da * 5 + 308) / 153 - 2;
+    // number of days elapsed since day 1 of the month
+    int d = da - (m + 4) * 153 / 5 + 122;
+    int year = y - 4800 + (m + 2) / 12;
+    int month = (m + 2) % 12 + 1;
+    int day = d + 1;
+    switch (range) {
+    case YEAR:
+      return year;
+    case MONTH:
+      return month;
+    case DAY:
+      return day;
+    default:
+      throw new AssertionError(range);
+    }
+  }
+
+  public static int ymdToUnixDate(int year, int month, int day) {
+    final int julian = ymdToJulian(year, month, day);
+    return julian - EPOCH_JULIAN;
+  }
+
+  public static int ymdToJulian(int year, int month, int day) {
+    int a = (14 - month) / 12;
+    int y = year + 4800 - a;
+    int m = month + 12 * a - 3;
+    int j = day + (153 * m + 2) / 5
+        + 365 * y
+        + y / 4
+        - y / 100
+        + y / 400
+        - 32045;
+    if (j < 2299161) {
+      j = day + (153 * m + 2) / 5 + 365 * y + y / 4 - 32083;
+    }
+    return j;
+  }
+
+  public static String intervalYearMonthToString(int v, TimeUnitRange range) {
+    final StringBuilder buf = new StringBuilder();
+    if (v >= 0) {
+      buf.append('+');
+    } else {
+      buf.append('-');
+      v = -v;
+    }
+    final int y;
+    final int m;
+    switch (range) {
+    case YEAR:
+      v = roundUp(v, 12);
+      y = v / 12;
+      buf.append(y);
+      break;
+    case YEAR_TO_MONTH:
+      y = v / 12;
+      buf.append(y);
+      buf.append('-');
+      m = v % 12;
+      number(buf, m, 2);
+      break;
+    case MONTH:
+      m = v;
+      buf.append(m);
+      break;
+    default:
+      throw new AssertionError(range);
+    }
+    return buf.toString();
+  }
+
+  private static StringBuilder number(StringBuilder buf, int v, int n) {
+    for (int k = digitCount(v); k < n; k++) {
+      buf.append('0');
+    }
+    return buf.append(v);
+  }
+
+  public static int digitCount(int v) {
+    for (int n = 1;; n++) {
+      v /= 10;
+      if (v == 0) {
+        return n;
+      }
+    }
+  }
+
+  public static String intervalDayTimeToString(long v, TimeUnitRange range,
+      int scale) {
+    final StringBuilder buf = new StringBuilder();
+    if (v >= 0) {
+      buf.append('+');
+    } else {
+      buf.append('-');
+      v = -v;
+    }
+    final long ms;
+    final long s;
+    final long m;
+    final long h;
+    final long d;
+    switch (range) {
+    case DAY_TO_SECOND:
+      v = roundUp(v, powerX(10, 3 - scale));
+      ms = v % 1000;
+      v /= 1000;
+      s = v % 60;
+      v /= 60;
+      m = v % 60;
+      v /= 60;
+      h = v % 24;
+      v /= 24;
+      d = v;
+      buf.append((int) d);
+      buf.append(' ');
+      number(buf, (int) h, 2);
+      buf.append(':');
+      number(buf, (int) m, 2);
+      buf.append(':');
+      number(buf, (int) s, 2);
+      fraction(buf, scale, ms);
+      break;
+    case DAY_TO_MINUTE:
+      v = roundUp(v, 1000 * 60);
+      v /= 1000;
+      v /= 60;
+      m = v % 60;
+      v /= 60;
+      h = v % 24;
+      v /= 24;
+      d = v;
+      buf.append((int) d);
+      buf.append(' ');
+      number(buf, (int) h, 2);
+      buf.append(':');
+      number(buf, (int) m, 2);
+      break;
+    case DAY_TO_HOUR:
+      v = roundUp(v, 1000 * 60 * 60);
+      v /= 1000;
+      v /= 60;
+      v /= 60;
+      h = v % 24;
+      v /= 24;
+      d = v;
+      buf.append((int) d);
+      buf.append(' ');
+      number(buf, (int) h, 2);
+      break;
+    case DAY:
+      v = roundUp(v, 1000 * 60 * 60 * 24);
+      d = v / (1000 * 60 * 60 * 24);
+      buf.append((int) d);
+      break;
+    case HOUR:
+      v = roundUp(v, 1000 * 60 * 60);
+      v /= 1000;
+      v /= 60;
+      v /= 60;
+      h = v;
+      buf.append((int) h);
+      break;
+    case HOUR_TO_MINUTE:
+      v = roundUp(v, 1000 * 60);
+      v /= 1000;
+      v /= 60;
+      m = v % 60;
+      v /= 60;
+      h = v;
+      buf.append((int) h);
+      buf.append(':');
+      number(buf, (int) m, 2);
+      break;
+    case HOUR_TO_SECOND:
+      v = roundUp(v, powerX(10, 3 - scale));
+      ms = v % 1000;
+      v /= 1000;
+      s = v % 60;
+      v /= 60;
+      m = v % 60;
+      v /= 60;
+      h = v;
+      buf.append((int) h);
+      buf.append(':');
+      number(buf, (int) m, 2);
+      buf.append(':');
+      number(buf, (int) s, 2);
+      fraction(buf, scale, ms);
+      break;
+    case MINUTE_TO_SECOND:
+      v = roundUp(v, powerX(10, 3 - scale));
+      ms = v % 1000;
+      v /= 1000;
+      s = v % 60;
+      v /= 60;
+      m = v;
+      buf.append((int) m);
+      buf.append(':');
+      number(buf, (int) s, 2);
+      fraction(buf, scale, ms);
+      break;
+    case MINUTE:
+      v = roundUp(v, 1000 * 60);
+      v /= 1000;
+      v /= 60;
+      m = v;
+      buf.append((int) m);
+      break;
+    case SECOND:
+      v = roundUp(v, powerX(10, 3 - scale));
+      ms = v % 1000;
+      v /= 1000;
+      s = v;
+      buf.append((int) s);
+      fraction(buf, scale, ms);
+      break;
+    default:
+      throw new AssertionError(range);
+    }
+    return buf.toString();
+  }
+
+  /**
+   * Rounds a dividend to the nearest divisor.
+   * For example roundUp(31, 10) yields 30; roundUp(37, 10) yields 40.
+   * @param dividend Number to be divided
+   * @param divisor Number to divide by
+   * @return Rounded dividend
+   */
+  private static long roundUp(long dividend, long divisor) {
+    long remainder = dividend % divisor;
+    dividend -= remainder;
+    if (remainder * 2 > divisor) {
+      dividend += divisor;
+    }
+    return dividend;
+  }
+
+  private static int roundUp(int dividend, int divisor) {
+    int remainder = dividend % divisor;
+    dividend -= remainder;
+    if (remainder * 2 > divisor) {
+      dividend += divisor;
+    }
+    return dividend;
+  }
+
+  private static void fraction(StringBuilder buf, int scale, long ms) {
+    if (scale > 0) {
+      buf.append('.');
+      long v1 = scale == 3 ? ms
+          : scale == 2 ? ms / 10
+          : scale == 1 ? ms / 100
+            : 0;
+      number(buf, (int) v1, scale);
+    }
+  }
+
+  /** Helper for "array element reference". Caller has already ensured that
+   * array and index are not null. Index is 1-based, per SQL. */
+  public static Object arrayItem(List list, int item) {
+    if (item < 1 || item > list.size()) {
+      return null;
+    }
+    return list.get(item - 1);
+  }
+
+  /** Helper for "map element reference". Caller has already ensured that
+   * array and index are not null. Index is 1-based, per SQL. */
+  public static Object mapItem(Map map, Object item) {
+    return map.get(item);
+  }
+
+  /** Implements the {@code [ ... ]} operator on an object whose type is not
+   * known until runtime.
+   */
+  public static Object item(Object object, Object index) {
+    if (object instanceof Map) {
+      return ((Map) object).get(index);
+    }
+    if (object instanceof List && index instanceof Number) {
+      List list = (List) object;
+      return list.get(((Number) index).intValue());
+    }
+    return null;
+  }
+
+  /** NULL &rarr; FALSE, FALSE &rarr; FALSE, TRUE &rarr; TRUE. */
+  public static boolean isTrue(Boolean b) {
+    return b != null && b;
+  }
+
+  /** NULL &rarr; TRUE, FALSE &rarr; FALSE, TRUE &rarr; TRUE. */
+  public static boolean isNotFalse(Boolean b) {
+    return b == null || b;
+  }
+
+  /** NULL &rarr; NULL, FALSE &rarr; TRUE, TRUE &rarr; FALSE. */
+  public static Boolean not(Boolean b) {
+    return (b == null) ? null : !b;
+  }
+
+  /** Converts a JDBC array to a list. */
+  public static List arrayToList(final java.sql.Array a) {
+    if (a == null) {
+      return null;
+    }
+    try {
+      return Primitive.asList(a.getArray());
+    } catch (SQLException e) {
+      throw new RuntimeException(e);
+    }
+  }
+
+  /** Support the SLICE function. */
+  public static List slice(List list) {
+    return list;
+  }
+
+  /** Support the ELEMENT function. */
+  public static Object element(List list) {
+    switch (list.size()) {
+    case 0:
+      return null;
+    case 1:
+      return list.get(0);
+    default:
+      throw new RuntimeException("more than one value");
+    }
+  }
+
+  /** Returns a lambda that converts a list to an enumerable. */
+  public static <E> Function1<List<E>, Enumerable<E>> listToEnumerable() {
+    //noinspection unchecked
+    return (Function1<List<E>, Enumerable<E>>) (Function1) LIST_AS_ENUMERABLE;
+  }
+
+  /** A range of time units. The first is more significant than the
+   * other (e.g. year-to-day) or the same as the other
+   * (e.g. month). */
+  public enum TimeUnitRange {
+    YEAR,
+    YEAR_TO_MONTH,
+    MONTH,
+    DAY,
+    DAY_TO_HOUR,
+    DAY_TO_MINUTE,
+    DAY_TO_SECOND,
+    HOUR,
+    HOUR_TO_MINUTE,
+    HOUR_TO_SECOND,
+    MINUTE,
+    MINUTE_TO_SECOND,
+    SECOND;
+
+    /** Whether this is in the YEAR-TO-MONTH family of intervals. */
+    public boolean monthly() {
+      return ordinal() <= MONTH.ordinal();
+    }
+  }
+}
+
+// End SqlFunctions.java


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

Posted by lu...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/query/src/main/java/com/kylinolap/query/enumerator/CubeEnumerator.java
----------------------------------------------------------------------
diff --git a/query/src/main/java/com/kylinolap/query/enumerator/CubeEnumerator.java b/query/src/main/java/com/kylinolap/query/enumerator/CubeEnumerator.java
new file mode 100644
index 0000000..5768e60
--- /dev/null
+++ b/query/src/main/java/com/kylinolap/query/enumerator/CubeEnumerator.java
@@ -0,0 +1,223 @@
+/*
+ * 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.query.enumerator;
+
+import java.util.Collection;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Properties;
+
+import net.hydromatic.linq4j.Enumerator;
+import net.hydromatic.optiq.DataContext;
+import net.hydromatic.optiq.jdbc.OptiqConnection;
+
+import org.eigenbase.reltype.RelDataTypeField;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.kylinolap.metadata.model.cube.DimensionDesc;
+import com.kylinolap.metadata.model.cube.FunctionDesc;
+import com.kylinolap.metadata.model.cube.MeasureDesc;
+import com.kylinolap.metadata.model.cube.TblColRef;
+import com.kylinolap.query.relnode.OLAPContext;
+import com.kylinolap.storage.IStorageEngine;
+import com.kylinolap.storage.StorageEngineFactory;
+import com.kylinolap.storage.filter.CompareTupleFilter;
+import com.kylinolap.storage.filter.TupleFilter;
+import com.kylinolap.storage.tuple.ITuple;
+import com.kylinolap.storage.tuple.ITupleIterator;
+
+/**
+ * @author xjiang
+ */
+public class CubeEnumerator implements Enumerator<Object[]> {
+
+    private final static Logger logger = LoggerFactory.getLogger(CubeEnumerator.class);
+
+    private final OLAPContext olapContext;
+    private final DataContext optiqContext;
+    private final Object[] current;
+    private ITupleIterator cursor;
+    private int[] fieldIndexes;
+
+    public CubeEnumerator(OLAPContext olapContext, DataContext optiqContext) {
+        this.olapContext = olapContext;
+        this.optiqContext = optiqContext;
+        this.current = new Object[olapContext.olapRowType.getFieldCount()];
+        this.cursor = null;
+        this.fieldIndexes = null;
+    }
+
+    @Override
+    public Object[] current() {
+        return current;
+    }
+
+    @Override
+    public boolean moveNext() {
+        if (cursor == null) {
+            cursor = queryStorage();
+        }
+
+        if (!cursor.hasNext()) {
+            return false;
+        }
+
+        ITuple tuple = cursor.next();
+        if (tuple == null) {
+            return false;
+        }
+        convertCurrentRow(tuple);
+        return true;
+    }
+
+    @Override
+    public void reset() {
+        close();
+        cursor = queryStorage();
+    }
+
+    @Override
+    public void close() {
+        if (cursor != null) {
+            cursor.close();
+        }
+    }
+
+    private Object[] convertCurrentRow(ITuple tuple) {
+
+        // build field index map
+        if (this.fieldIndexes == null) {
+            List<String> fields = tuple.getAllFields();
+            int size = fields.size();
+            this.fieldIndexes = new int[size];
+            for (int i = 0; i < size; i++) {
+                String field = fields.get(i);
+                RelDataTypeField relField = olapContext.olapRowType.getField(field, true);
+                if (relField != null) {
+                    fieldIndexes[i] = relField.getIndex();
+                } else {
+                    fieldIndexes[i] = -1;
+                }
+            }
+        }
+
+        // set field value
+        Object[] values = tuple.getAllValues();
+        for (int i = 0, n = values.length; i < n; i++) {
+            Object value = values[i];
+            int index = fieldIndexes[i];
+            if (index >= 0) {
+                current[index] = value;
+            }
+        }
+
+        return current;
+    }
+
+    private ITupleIterator queryStorage() {
+        logger.debug("query storage...");
+
+        // set connection properties
+        setConnectionProperties();
+
+        // bind dynamic variables
+        bindVariable(olapContext.filter);
+
+        // build dimension & metrics
+        Collection<TblColRef> dimensions = new HashSet<TblColRef>();
+        Collection<FunctionDesc> metrics = new HashSet<FunctionDesc>();
+        buildDimensionsAndMetrics(dimensions, metrics);
+
+        // query storage engine
+        IStorageEngine storageEngine = StorageEngineFactory.getStorageEngine(olapContext.cubeInstance);
+        ITupleIterator iterator = storageEngine.search(dimensions, olapContext.filter, olapContext.groupByColumns, metrics, olapContext.storageContext);
+        if (logger.isDebugEnabled()) {
+            logger.debug("return TupleIterator...");
+        }
+
+        this.fieldIndexes = null;
+        return iterator;
+    }
+
+    private void buildDimensionsAndMetrics(Collection<TblColRef> dimensions, Collection<FunctionDesc> metrics) {
+
+        for (FunctionDesc func : olapContext.aggregations) {
+            if (!func.isAppliedOnDimension()) {
+                metrics.add(func);
+            }
+        }
+
+        if (olapContext.isSimpleQuery()) {
+            // In order to prevent coprocessor from doing the real aggregating,
+            // All dimensions are injected
+            for (DimensionDesc dim : olapContext.cubeDesc.getDimensions()) {
+                for (TblColRef col : dim.getColumnRefs()) {
+                    dimensions.add(col);
+                }
+            }
+            // select sth from fact table
+            for (MeasureDesc measure : olapContext.cubeDesc.getMeasures()) {
+                FunctionDesc func = measure.getFunction();
+                if (func.isSum()) {
+                    // the rewritten name for sum(metric) is metric itself
+                    metrics.add(func);
+                }
+            }
+            olapContext.storageContext.markAvoidAggregation();
+        } else {
+            for (TblColRef column : olapContext.allColumns) {
+                // skip measure columns
+                if (olapContext.metricsColumns.contains(column)) {
+                    continue;
+                }
+                dimensions.add(column);
+            }
+        }
+    }
+
+    private void bindVariable(TupleFilter filter) {
+        if (filter == null) {
+            return;
+        }
+
+        for (TupleFilter childFilter : filter.getChildren()) {
+            bindVariable(childFilter);
+        }
+
+        if (filter instanceof CompareTupleFilter && optiqContext != null) {
+            CompareTupleFilter compFilter = (CompareTupleFilter) filter;
+            for (Map.Entry<String, String> entry : compFilter.getVariables().entrySet()) {
+                String variable = entry.getKey();
+                Object value = optiqContext.get(variable);
+                if (value != null) {
+                    compFilter.bindVariable(variable, value.toString());
+                }
+
+            }
+        }
+    }
+
+    private void setConnectionProperties() {
+        OptiqConnection conn = (OptiqConnection) optiqContext.getQueryProvider();
+        Properties connProps = conn.getProperties();
+
+        String propThreshold = connProps.getProperty(OLAPQuery.PROP_SCAN_THRESHOLD);
+        int threshold = Integer.valueOf(propThreshold);
+        olapContext.storageContext.setThreshold(threshold);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/query/src/main/java/com/kylinolap/query/enumerator/HiveEnumerator.java
----------------------------------------------------------------------
diff --git a/query/src/main/java/com/kylinolap/query/enumerator/HiveEnumerator.java b/query/src/main/java/com/kylinolap/query/enumerator/HiveEnumerator.java
new file mode 100644
index 0000000..a5914a8
--- /dev/null
+++ b/query/src/main/java/com/kylinolap/query/enumerator/HiveEnumerator.java
@@ -0,0 +1,130 @@
+/*
+ * 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.query.enumerator;
+
+import java.sql.Connection;
+import java.sql.DriverManager;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.sql.Statement;
+
+import net.hydromatic.linq4j.Enumerator;
+
+import org.eigenbase.reltype.RelDataTypeField;
+
+import com.kylinolap.query.relnode.OLAPContext;
+
+/**
+ * Hive Query Result Enumerator
+ * 
+ * @author xjiang
+ * 
+ */
+public class HiveEnumerator implements Enumerator<Object[]> {
+
+    private final OLAPContext olapContext;
+    private final Object[] current;
+    private ResultSet rs;
+    private Connection conn;
+
+    public HiveEnumerator(OLAPContext olapContext) {
+        this.olapContext = olapContext;
+        this.current = new Object[olapContext.olapRowType.getFieldCount()];
+    }
+
+    @Override
+    public Object[] current() {
+        return current;
+    }
+
+    @Override
+    public boolean moveNext() {
+        if (rs == null) {
+            rs = executeQuery();
+        }
+        return populateResult();
+    }
+
+    private ResultSet executeQuery() {
+        String url = olapContext.olapSchema.getStarSchemaUrl();
+        String user = olapContext.olapSchema.getStarSchemaUser();
+        String pwd = olapContext.olapSchema.getStarSchemaPassword();
+        String sql = olapContext.sql;
+        Statement stmt = null;
+        try {
+            conn = DriverManager.getConnection(url, user, pwd);
+            stmt = conn.createStatement();
+            return stmt.executeQuery(sql);
+        } catch (SQLException e) {
+            throw new IllegalStateException(url + " can't execute query " + sql, e);
+        } finally {
+            if (stmt != null) {
+                try {
+                    stmt.close();
+                } catch (SQLException ex) {
+                    ex.printStackTrace();
+                }
+            }
+            stmt = null;
+            if (conn != null) {
+                try {
+                    conn.close();
+                } catch (SQLException ex) {
+                    ex.printStackTrace();
+                }
+                conn = null;
+            }
+        }
+    }
+
+    private boolean populateResult() {
+        try {
+            boolean hasNext = rs.next();
+            if (hasNext) {
+                for (RelDataTypeField relField : olapContext.olapRowType.getFieldList()) {
+                    Object value = rs.getObject(relField.getName().toLowerCase());
+                    current[relField.getIndex()] = value;
+                }
+            }
+            return hasNext;
+        } catch (SQLException e) {
+            throw new IllegalStateException("Can't populate result!", e);
+        }
+    }
+
+    @Override
+    public void reset() {
+        close();
+        rs = executeQuery();
+    }
+
+    @Override
+    public void close() {
+        try {
+            if (rs != null) {
+                rs.close();
+                rs = null;
+            }
+            if (conn != null) {
+                conn.close();
+                conn = null;
+            }
+        } catch (SQLException e) {
+            throw new IllegalStateException("Can't close ResultSet!", e);
+        }
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/query/src/main/java/com/kylinolap/query/enumerator/LookupTableEnumerator.java
----------------------------------------------------------------------
diff --git a/query/src/main/java/com/kylinolap/query/enumerator/LookupTableEnumerator.java b/query/src/main/java/com/kylinolap/query/enumerator/LookupTableEnumerator.java
new file mode 100644
index 0000000..e0079ea
--- /dev/null
+++ b/query/src/main/java/com/kylinolap/query/enumerator/LookupTableEnumerator.java
@@ -0,0 +1,96 @@
+/*
+ * 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.query.enumerator;
+
+import java.util.Collection;
+import java.util.Iterator;
+import java.util.List;
+
+import net.hydromatic.linq4j.Enumerator;
+
+import com.kylinolap.cube.CubeInstance;
+import com.kylinolap.cube.CubeManager;
+import com.kylinolap.dict.lookup.LookupStringTable;
+import com.kylinolap.metadata.model.cube.DimensionDesc;
+import com.kylinolap.metadata.model.schema.ColumnDesc;
+import com.kylinolap.query.relnode.OLAPContext;
+import com.kylinolap.query.schema.OLAPTable;
+import com.kylinolap.storage.tuple.Tuple;
+
+/**
+ * @author yangli9
+ * 
+ */
+public class LookupTableEnumerator implements Enumerator<Object[]> {
+
+    private final Collection<String[]> allRows;
+    private final List<ColumnDesc> colDescs;
+    private final Object[] current;
+    private Iterator<String[]> iterator;
+
+    public LookupTableEnumerator(OLAPContext olapContext) {
+
+        String lookupTableName = olapContext.firstTableScan.getCubeTable();
+        DimensionDesc dim = olapContext.cubeDesc.findDimensionByTable(lookupTableName);
+        if (dim == null)
+            throw new IllegalStateException("No dimension with derived columns found for lookup table " + lookupTableName + ", cube desc " + olapContext.cubeDesc);
+
+        CubeInstance cube = olapContext.cubeInstance;
+        CubeManager cubeMgr = CubeManager.getInstance(cube.getConfig());
+        LookupStringTable table = cubeMgr.getLookupTable(cube.getLatestReadySegment(), dim);
+        this.allRows = table.getAllRows();
+
+        OLAPTable olapTable = (OLAPTable) olapContext.firstTableScan.getOlapTable();
+        this.colDescs = olapTable.getExposedColumns();
+        this.current = new Object[colDescs.size()];
+
+        reset();
+    }
+
+    @Override
+    public boolean moveNext() {
+        boolean hasNext = iterator.hasNext();
+        if (hasNext) {
+            String[] row = iterator.next();
+            for (int i = 0, n = colDescs.size(); i < n; i++) {
+                ColumnDesc colDesc = colDescs.get(i);
+                int colIdx = colDesc.getZeroBasedIndex();
+                if (colIdx >= 0) {
+                    current[i] = Tuple.convertOptiqCellValue(row[colIdx], colDesc.getType().getName());
+                } else {
+                    current[i] = null; // fake column
+                }
+            }
+        }
+        return hasNext;
+    }
+
+    @Override
+    public Object[] current() {
+        return current;
+    }
+
+    @Override
+    public void reset() {
+        this.iterator = allRows.iterator();
+    }
+
+    @Override
+    public void close() {
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/query/src/main/java/com/kylinolap/query/enumerator/OLAPQuery.java
----------------------------------------------------------------------
diff --git a/query/src/main/java/com/kylinolap/query/enumerator/OLAPQuery.java b/query/src/main/java/com/kylinolap/query/enumerator/OLAPQuery.java
new file mode 100644
index 0000000..78a5cf1
--- /dev/null
+++ b/query/src/main/java/com/kylinolap/query/enumerator/OLAPQuery.java
@@ -0,0 +1,65 @@
+/*
+ * 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.query.enumerator;
+
+import net.hydromatic.linq4j.AbstractEnumerable;
+import net.hydromatic.linq4j.Enumerable;
+import net.hydromatic.linq4j.Enumerator;
+import net.hydromatic.optiq.DataContext;
+
+import com.kylinolap.query.relnode.OLAPContext;
+
+/**
+ * 
+ * @author xjiang
+ * 
+ */
+public class OLAPQuery extends AbstractEnumerable<Object[]> implements Enumerable<Object[]> {
+
+    public static final String PROP_SCAN_THRESHOLD = "scan_threshold";
+
+    public enum EnumeratorTypeEnum {
+        CUBE, LOOKUP_TABLE, HIVE
+    }
+
+    private final DataContext optiqContext;
+    private final EnumeratorTypeEnum type;
+    private final int contextId;
+
+    public OLAPQuery(DataContext optiqContext, EnumeratorTypeEnum type, int ctxId) {
+        this.optiqContext = optiqContext;
+        this.type = type;
+        this.contextId = ctxId;
+    }
+
+    public OLAPQuery(EnumeratorTypeEnum type, int ctxSeq) {
+        this(null, type, ctxSeq);
+    }
+
+    public Enumerator<Object[]> enumerator() {
+        OLAPContext olapContext = OLAPContext.getThreadLocalContextById(contextId);
+        switch (type) {
+        case CUBE:
+            return new CubeEnumerator(olapContext, optiqContext);
+        case LOOKUP_TABLE:
+            return new LookupTableEnumerator(olapContext);
+        case HIVE:
+            return new HiveEnumerator(olapContext);
+        default:
+            throw new IllegalArgumentException("Wrong type " + type + "!");
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/query/src/main/java/com/kylinolap/query/optrule/OLAPAggregateRule.java
----------------------------------------------------------------------
diff --git a/query/src/main/java/com/kylinolap/query/optrule/OLAPAggregateRule.java b/query/src/main/java/com/kylinolap/query/optrule/OLAPAggregateRule.java
new file mode 100644
index 0000000..c1c5603
--- /dev/null
+++ b/query/src/main/java/com/kylinolap/query/optrule/OLAPAggregateRule.java
@@ -0,0 +1,52 @@
+/*
+ * 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.query.optrule;
+
+import org.eigenbase.rel.AggregateRel;
+import org.eigenbase.rel.InvalidRelException;
+import org.eigenbase.rel.RelNode;
+import org.eigenbase.rel.convert.ConverterRule;
+import org.eigenbase.relopt.Convention;
+import org.eigenbase.relopt.RelTraitSet;
+
+import com.kylinolap.query.relnode.OLAPAggregateRel;
+import com.kylinolap.query.relnode.OLAPRel;
+
+/**
+ * 
+ * @author xjiang
+ * 
+ */
+public class OLAPAggregateRule extends ConverterRule {
+
+    public static final ConverterRule INSTANCE = new OLAPAggregateRule();
+
+    public OLAPAggregateRule() {
+        super(AggregateRel.class, Convention.NONE, OLAPRel.CONVENTION, "OLAPAggregateRule");
+    }
+
+    @Override
+    public RelNode convert(RelNode rel) {
+        AggregateRel agg = (AggregateRel) rel;
+        RelTraitSet traitSet = agg.getTraitSet().replace(OLAPRel.CONVENTION);
+        try {
+            return new OLAPAggregateRel(agg.getCluster(), traitSet, convert(agg.getChild(), traitSet), agg.getGroupSet(), agg.getAggCallList());
+        } catch (InvalidRelException e) {
+            throw new IllegalStateException("Can't create OLAPAggregateRel!", e);
+        }
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/query/src/main/java/com/kylinolap/query/optrule/OLAPFilterRule.java
----------------------------------------------------------------------
diff --git a/query/src/main/java/com/kylinolap/query/optrule/OLAPFilterRule.java b/query/src/main/java/com/kylinolap/query/optrule/OLAPFilterRule.java
new file mode 100644
index 0000000..4c35616
--- /dev/null
+++ b/query/src/main/java/com/kylinolap/query/optrule/OLAPFilterRule.java
@@ -0,0 +1,49 @@
+/*
+ * 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.query.optrule;
+
+import org.eigenbase.rel.FilterRel;
+import org.eigenbase.relopt.RelOptRule;
+import org.eigenbase.relopt.RelOptRuleCall;
+import org.eigenbase.relopt.RelTraitSet;
+
+import com.kylinolap.query.relnode.OLAPFilterRel;
+import com.kylinolap.query.relnode.OLAPRel;
+
+/**
+ * 
+ * @author xjiang
+ * 
+ */
+
+public class OLAPFilterRule extends RelOptRule {
+
+    public static final RelOptRule INSTANCE = new OLAPFilterRule();
+
+    public OLAPFilterRule() {
+        super(operand(FilterRel.class, any()));
+    }
+
+    @Override
+    public void onMatch(RelOptRuleCall call) {
+        FilterRel filter = call.rel(0);
+
+        RelTraitSet traitSet = filter.getTraitSet().replace(OLAPRel.CONVENTION);
+        OLAPFilterRel olapFilter = new OLAPFilterRel(filter.getCluster(), traitSet, convert(filter.getChild(), traitSet), filter.getCondition());
+        call.transformTo(olapFilter);
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/query/src/main/java/com/kylinolap/query/optrule/OLAPJoinRule.java
----------------------------------------------------------------------
diff --git a/query/src/main/java/com/kylinolap/query/optrule/OLAPJoinRule.java b/query/src/main/java/com/kylinolap/query/optrule/OLAPJoinRule.java
new file mode 100644
index 0000000..f3ac3df
--- /dev/null
+++ b/query/src/main/java/com/kylinolap/query/optrule/OLAPJoinRule.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.query.optrule;
+
+import org.eigenbase.rel.InvalidRelException;
+import org.eigenbase.rel.JoinInfo;
+import org.eigenbase.rel.JoinRel;
+import org.eigenbase.rel.JoinRelType;
+import org.eigenbase.rel.RelNode;
+import org.eigenbase.rel.convert.ConverterRule;
+import org.eigenbase.relopt.Convention;
+import org.eigenbase.relopt.RelOptCluster;
+import org.eigenbase.relopt.RelTraitSet;
+
+import com.kylinolap.query.relnode.OLAPFilterRel;
+import com.kylinolap.query.relnode.OLAPJoinRel;
+import com.kylinolap.query.relnode.OLAPRel;
+
+/**
+ * 
+ * @author xjiang
+ * 
+ */
+public class OLAPJoinRule extends ConverterRule {
+
+    public static final ConverterRule INSTANCE = new OLAPJoinRule();
+
+    public OLAPJoinRule() {
+        super(JoinRel.class, Convention.NONE, OLAPRel.CONVENTION, "OLAPJoinRule");
+    }
+
+    @Override
+    public RelNode convert(RelNode rel) {
+        JoinRel join = (JoinRel) rel;
+        RelNode left = join.getInput(0);
+        RelNode right = join.getInput(1);
+
+        RelTraitSet traitSet = join.getTraitSet().replace(OLAPRel.CONVENTION);
+        left = convert(left, traitSet);
+        right = convert(right, traitSet);
+
+        final JoinInfo info = JoinInfo.of(left, right, join.getCondition());
+        if (!info.isEqui() && join.getJoinType() != JoinRelType.INNER) {
+            // EnumerableJoinRel only supports equi-join. We can put a filter on top
+            // if it is an inner join.
+            return null;
+        }
+
+        RelOptCluster cluster = join.getCluster();
+        RelNode newRel;
+        try {
+            newRel = new OLAPJoinRel(cluster, traitSet, left, right, //
+                    info.getEquiCondition(left, right, cluster.getRexBuilder()), //
+                    info.leftKeys, info.rightKeys, join.getJoinType(), join.getVariablesStopped());
+        } catch (InvalidRelException e) {
+            // Semantic error not possible. Must be a bug. Convert to
+            // internal error.
+            throw new AssertionError(e);
+            // LOGGER.fine(e.toString());
+            // return null;
+        }
+        if (!info.isEqui()) {
+            newRel = new OLAPFilterRel(cluster, newRel.getTraitSet(), newRel, info.getRemaining(cluster.getRexBuilder()));
+        }
+        return newRel;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/query/src/main/java/com/kylinolap/query/optrule/OLAPLimitRule.java
----------------------------------------------------------------------
diff --git a/query/src/main/java/com/kylinolap/query/optrule/OLAPLimitRule.java b/query/src/main/java/com/kylinolap/query/optrule/OLAPLimitRule.java
new file mode 100644
index 0000000..af14759
--- /dev/null
+++ b/query/src/main/java/com/kylinolap/query/optrule/OLAPLimitRule.java
@@ -0,0 +1,56 @@
+/*
+ * 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.query.optrule;
+
+import org.eigenbase.rel.RelNode;
+import org.eigenbase.rel.SortRel;
+import org.eigenbase.relopt.RelOptRule;
+import org.eigenbase.relopt.RelOptRuleCall;
+import org.eigenbase.relopt.RelTraitSet;
+
+import com.kylinolap.query.relnode.OLAPLimitRel;
+import com.kylinolap.query.relnode.OLAPRel;
+
+/**
+ * 
+ * @author xjiang
+ * 
+ */
+public class OLAPLimitRule extends RelOptRule {
+
+    public static final RelOptRule INSTANCE = new OLAPLimitRule();
+
+    public OLAPLimitRule() {
+        super(operand(SortRel.class, any()), "OLAPLimitRule");
+    }
+
+    @Override
+    public void onMatch(RelOptRuleCall call) {
+        final SortRel sort = call.rel(0);
+        if (sort.offset == null && sort.fetch == null) {
+            return;
+        }
+        final RelTraitSet traitSet = sort.getTraitSet().replace(OLAPRel.CONVENTION);
+        RelNode input = sort.getChild();
+        if (!sort.getCollation().getFieldCollations().isEmpty()) {
+            // Create a sort with the same sort key, but no offset or fetch.
+            input = sort.copy(sort.getTraitSet(), input, sort.getCollation(), null, null);
+        }
+        RelNode x = convert(input, input.getTraitSet().replace(OLAPRel.CONVENTION));
+        call.transformTo(new OLAPLimitRel(sort.getCluster(), traitSet, x, sort.offset, sort.fetch));
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/query/src/main/java/com/kylinolap/query/optrule/OLAPProjectRule.java
----------------------------------------------------------------------
diff --git a/query/src/main/java/com/kylinolap/query/optrule/OLAPProjectRule.java b/query/src/main/java/com/kylinolap/query/optrule/OLAPProjectRule.java
new file mode 100644
index 0000000..e174366
--- /dev/null
+++ b/query/src/main/java/com/kylinolap/query/optrule/OLAPProjectRule.java
@@ -0,0 +1,48 @@
+/*
+ * 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.query.optrule;
+
+import org.eigenbase.rel.ProjectRel;
+import org.eigenbase.relopt.RelOptRule;
+import org.eigenbase.relopt.RelOptRuleCall;
+import org.eigenbase.relopt.RelTraitSet;
+
+import com.kylinolap.query.relnode.OLAPProjectRel;
+import com.kylinolap.query.relnode.OLAPRel;
+
+/**
+ * 
+ * @author xjiang
+ * 
+ */
+public class OLAPProjectRule extends RelOptRule {
+
+    public static final RelOptRule INSTANCE = new OLAPProjectRule();
+
+    public OLAPProjectRule() {
+        super(operand(ProjectRel.class, any()));
+    }
+
+    @Override
+    public void onMatch(RelOptRuleCall call) {
+        ProjectRel project = call.rel(0);
+
+        RelTraitSet traitSet = project.getTraitSet().replace(OLAPRel.CONVENTION);
+        OLAPProjectRel olapProj = new OLAPProjectRel(project.getCluster(), traitSet, convert(project.getChild(), project.getTraitSet().replace(OLAPRel.CONVENTION)), project.getProjects(), project.getRowType(), project.getFlags());
+        call.transformTo(olapProj);
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/query/src/main/java/com/kylinolap/query/optrule/OLAPSortRule.java
----------------------------------------------------------------------
diff --git a/query/src/main/java/com/kylinolap/query/optrule/OLAPSortRule.java b/query/src/main/java/com/kylinolap/query/optrule/OLAPSortRule.java
new file mode 100644
index 0000000..cccfe47
--- /dev/null
+++ b/query/src/main/java/com/kylinolap/query/optrule/OLAPSortRule.java
@@ -0,0 +1,51 @@
+/*
+ * 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.query.optrule;
+
+import org.eigenbase.rel.RelNode;
+import org.eigenbase.rel.SortRel;
+import org.eigenbase.rel.convert.ConverterRule;
+import org.eigenbase.relopt.Convention;
+import org.eigenbase.relopt.RelTraitSet;
+
+import com.kylinolap.query.relnode.OLAPRel;
+import com.kylinolap.query.relnode.OLAPSortRel;
+
+/**
+ * @author xjiang
+ * 
+ */
+public class OLAPSortRule extends ConverterRule {
+
+    public static final OLAPSortRule INSTANCE = new OLAPSortRule();
+
+    public OLAPSortRule() {
+        super(SortRel.class, Convention.NONE, OLAPRel.CONVENTION, "OLAPSortRule");
+    }
+
+    @Override
+    public RelNode convert(RelNode rel) {
+        final SortRel sort = (SortRel) rel;
+        if (sort.offset != null || sort.fetch != null) {
+            return null;
+        }
+        final RelTraitSet traitSet = sort.getTraitSet().replace(OLAPRel.CONVENTION);
+        final RelNode input = sort.getChild();
+        return new OLAPSortRel(rel.getCluster(), traitSet, convert(input, input.getTraitSet().replace(OLAPRel.CONVENTION)), sort.getCollation(), sort.offset, sort.fetch);
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/query/src/main/java/com/kylinolap/query/optrule/OLAPToEnumerableConverterRule.java
----------------------------------------------------------------------
diff --git a/query/src/main/java/com/kylinolap/query/optrule/OLAPToEnumerableConverterRule.java b/query/src/main/java/com/kylinolap/query/optrule/OLAPToEnumerableConverterRule.java
new file mode 100644
index 0000000..ec11756
--- /dev/null
+++ b/query/src/main/java/com/kylinolap/query/optrule/OLAPToEnumerableConverterRule.java
@@ -0,0 +1,46 @@
+/*
+ * 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.query.optrule;
+
+import net.hydromatic.optiq.rules.java.EnumerableConvention;
+
+import org.eigenbase.rel.RelNode;
+import org.eigenbase.rel.convert.ConverterRule;
+import org.eigenbase.relopt.RelTraitSet;
+
+import com.kylinolap.query.relnode.OLAPRel;
+import com.kylinolap.query.relnode.OLAPToEnumerableConverter;
+
+/**
+ * 
+ * @author xjiang
+ * 
+ */
+public class OLAPToEnumerableConverterRule extends ConverterRule {
+
+    public static final ConverterRule INSTANCE = new OLAPToEnumerableConverterRule();
+
+    public OLAPToEnumerableConverterRule() {
+        super(RelNode.class, OLAPRel.CONVENTION, EnumerableConvention.INSTANCE, "OLAPToEnumerableConverterRule");
+    }
+
+    @Override
+    public RelNode convert(RelNode rel) {
+        RelTraitSet newTraitSet = rel.getTraitSet().replace(getOutConvention());
+        return new OLAPToEnumerableConverter(rel.getCluster(), newTraitSet, rel);
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/query/src/main/java/com/kylinolap/query/relnode/ColumnRowType.java
----------------------------------------------------------------------
diff --git a/query/src/main/java/com/kylinolap/query/relnode/ColumnRowType.java b/query/src/main/java/com/kylinolap/query/relnode/ColumnRowType.java
new file mode 100644
index 0000000..a0dd3cf
--- /dev/null
+++ b/query/src/main/java/com/kylinolap/query/relnode/ColumnRowType.java
@@ -0,0 +1,84 @@
+/*
+ * 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.query.relnode;
+
+import java.util.Collections;
+import java.util.List;
+import java.util.Set;
+
+import com.kylinolap.metadata.model.cube.TblColRef;
+
+/**
+ * 
+ * @author xjiang
+ * 
+ */
+public class ColumnRowType {
+
+    private List<TblColRef> columns;
+    // for calculated column, like (CASE LSTG_FORMAT_NAME WHEN 'Auction' THEN
+    // '111' ELSE '222' END)
+    // source columns are the contributing physical columns, here the
+    // LSTG_FORMAT_NAME
+    private List<Set<TblColRef>> sourceColumns;
+
+    public ColumnRowType(List<TblColRef> columns) {
+        this(columns, null);
+    }
+
+    public ColumnRowType(List<TblColRef> columns, List<Set<TblColRef>> sourceColumns) {
+        this.columns = columns;
+        this.sourceColumns = sourceColumns;
+    }
+
+    public TblColRef getColumnByIndex(int index) {
+        return columns.get(index);
+    }
+
+    public int getIndexByName(String columnName) {
+        for (int i = 0; i < columns.size(); i++) {
+            if (columns.get(i).getName().equals(columnName)) {
+                return i;
+            }
+        }
+        return -1;
+    }
+
+    public Set<TblColRef> getSourceColumnsByIndex(int i) {
+        Set<TblColRef> result = null;
+        if (sourceColumns != null) {
+            result = sourceColumns.get(i);
+        }
+        if (result == null || result.isEmpty()) {
+            result = Collections.singleton(getColumnByIndex(i));
+        }
+        return result;
+    }
+
+    public List<TblColRef> getAllColumns() {
+        return columns;
+    }
+
+    public int size() {
+        return columns.size();
+    }
+
+    @Override
+    public String toString() {
+        return "ColumnRowType [" + columns + "]";
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/query/src/main/java/com/kylinolap/query/relnode/OLAPAggregateRel.java
----------------------------------------------------------------------
diff --git a/query/src/main/java/com/kylinolap/query/relnode/OLAPAggregateRel.java b/query/src/main/java/com/kylinolap/query/relnode/OLAPAggregateRel.java
new file mode 100644
index 0000000..33a403d
--- /dev/null
+++ b/query/src/main/java/com/kylinolap/query/relnode/OLAPAggregateRel.java
@@ -0,0 +1,370 @@
+/*
+ * 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.query.relnode;
+
+import java.util.ArrayList;
+import java.util.BitSet;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import net.hydromatic.optiq.AggregateFunction;
+import net.hydromatic.optiq.FunctionParameter;
+import net.hydromatic.optiq.impl.AggregateFunctionImpl;
+import net.hydromatic.optiq.rules.java.EnumerableConvention;
+import net.hydromatic.optiq.rules.java.EnumerableRel;
+import net.hydromatic.optiq.rules.java.EnumerableRelImplementor;
+import net.hydromatic.optiq.rules.java.JavaRules.EnumerableAggregateRel;
+
+import org.eigenbase.rel.AggregateCall;
+import org.eigenbase.rel.AggregateRelBase;
+import org.eigenbase.rel.Aggregation;
+import org.eigenbase.rel.InvalidRelException;
+import org.eigenbase.rel.RelNode;
+import org.eigenbase.relopt.RelOptCluster;
+import org.eigenbase.relopt.RelOptCost;
+import org.eigenbase.relopt.RelOptPlanner;
+import org.eigenbase.relopt.RelTrait;
+import org.eigenbase.relopt.RelTraitSet;
+import org.eigenbase.reltype.RelDataType;
+import org.eigenbase.reltype.RelDataTypeFactory;
+import org.eigenbase.reltype.RelDataTypeField;
+import org.eigenbase.sql.SqlAggFunction;
+import org.eigenbase.sql.SqlIdentifier;
+import org.eigenbase.sql.fun.SqlSumEmptyIsZeroAggFunction;
+import org.eigenbase.sql.parser.SqlParserPos;
+import org.eigenbase.sql.type.InferTypes;
+import org.eigenbase.sql.type.OperandTypes;
+import org.eigenbase.sql.type.ReturnTypes;
+import org.eigenbase.sql.type.SqlTypeFamily;
+import org.eigenbase.sql.validate.SqlUserDefinedAggFunction;
+import org.eigenbase.util.Util;
+
+import com.google.common.base.Preconditions;
+import com.kylinolap.metadata.model.cube.FunctionDesc;
+import com.kylinolap.metadata.model.cube.ParameterDesc;
+import com.kylinolap.metadata.model.cube.TblColRef;
+import com.kylinolap.metadata.model.schema.ColumnDesc;
+import com.kylinolap.metadata.model.schema.TableDesc;
+import com.kylinolap.query.sqlfunc.HLLDistinctCountAggFunc;
+
+/**
+ * @author xjiang
+ */
+public class OLAPAggregateRel extends AggregateRelBase implements OLAPRel, EnumerableRel {
+
+    private final static Map<String, String> AGGR_FUNC_MAP = new HashMap<String, String>();
+
+    static {
+        AGGR_FUNC_MAP.put("SUM", "SUM");
+        AGGR_FUNC_MAP.put("$SUM0", "SUM");
+        AGGR_FUNC_MAP.put("COUNT", "COUNT");
+        AGGR_FUNC_MAP.put("COUNT_DISTINCT", "COUNT_DISTINCT");
+        AGGR_FUNC_MAP.put("HLL_COUNT", "COUNT_DISTINCT");
+        AGGR_FUNC_MAP.put("MAX", "MAX");
+        AGGR_FUNC_MAP.put("MIN", "MIN");
+    }
+
+    private static String getFuncName(AggregateCall aggCall) {
+        String aggName = aggCall.getAggregation().getName();
+        if (aggCall.isDistinct()) {
+            aggName = aggName + "_DISTINCT";
+        }
+        String funcName = AGGR_FUNC_MAP.get(aggName);
+        if (funcName == null) {
+            throw new IllegalStateException("Don't suppoprt aggregation " + aggName);
+        }
+        return funcName;
+    }
+
+    private OLAPContext context;
+    private ColumnRowType columnRowType;
+    private boolean afterAggregate;
+    private List<AggregateCall> rewriteAggCalls;
+    private List<TblColRef> groups;
+    private List<FunctionDesc> aggregations;
+
+    public OLAPAggregateRel(RelOptCluster cluster, RelTraitSet traits, RelNode child, BitSet groupSet, List<AggregateCall> aggCalls) throws InvalidRelException {
+        super(cluster, traits, child, groupSet, aggCalls);
+        Preconditions.checkArgument(getConvention() == OLAPRel.CONVENTION);
+        this.afterAggregate = false;
+        this.rewriteAggCalls = aggCalls;
+        this.rowType = getRowType();
+    }
+
+    @Override
+    public AggregateRelBase copy(RelTraitSet traitSet, RelNode input, BitSet groupSet, List<AggregateCall> aggCalls) {
+        try {
+            return new OLAPAggregateRel(getCluster(), traitSet, input, groupSet, aggCalls);
+        } catch (InvalidRelException e) {
+            throw new IllegalStateException("Can't create OLAPAggregateRel!", e);
+        }
+    }
+
+    @Override
+    public RelOptCost computeSelfCost(RelOptPlanner planner) {
+        double factor = .5;
+        for (AggregateCall aggCall : aggCalls) {
+            if ("$SUM0".equals(aggCall.getAggregation().getName())) {
+                factor = .2;
+            }
+        }
+        return super.computeSelfCost(planner).multiplyBy(factor);
+    }
+
+    @Override
+    public void implementOLAP(OLAPImplementor implementor) {
+
+        implementor.visitChild(getChild(), this);
+
+        this.context = implementor.getContext();
+        this.columnRowType = buildColumnRowType();
+        this.afterAggregate = this.context.afterAggregate;
+
+        // only translate the first aggregation
+        if (!this.afterAggregate) {
+            translateGroupBy();
+            fillbackOptimizedColumn();
+            translateAggregation();
+            this.context.afterAggregate = true;
+        } else {
+            for (AggregateCall aggCall : aggCalls) {
+                // check if supported by kylin
+                if (aggCall.isDistinct()) {
+                    throw new IllegalStateException("Distinct count is only allowed in innermost sub-query.");
+                }
+            }
+        }
+    }
+
+    private ColumnRowType buildColumnRowType() {
+        buildGroups();
+        buildAggregations();
+
+        ColumnRowType inputColumnRowType = ((OLAPRel) getChild()).getColumnRowType();
+        List<TblColRef> columns = new ArrayList<TblColRef>(this.rowType.getFieldCount());
+        columns.addAll(this.groups);
+
+        for (int i = 0; i < this.aggregations.size(); i++) {
+            FunctionDesc aggFunc = this.aggregations.get(i);
+            TblColRef aggCol = null;
+            if (aggFunc.needRewrite()) {
+                aggCol = buildRewriteColumn(aggFunc);
+            } else {
+                AggregateCall aggCall = this.rewriteAggCalls.get(i);
+                if (!aggCall.getArgList().isEmpty()) {
+                    int index = aggCall.getArgList().get(0);
+                    aggCol = inputColumnRowType.getColumnByIndex(index);
+                }
+            }
+            columns.add(aggCol);
+        }
+        return new ColumnRowType(columns);
+    }
+
+    private TblColRef buildRewriteColumn(FunctionDesc aggFunc) {
+        TblColRef colRef = null;
+        if (aggFunc.needRewrite()) {
+            ColumnDesc column = new ColumnDesc();
+            column.setName(aggFunc.getRewriteFieldName());
+            TableDesc table = this.context.firstTableScan.getOlapTable().getSourceTable();
+            column.setTable(table);
+            colRef = new TblColRef(column);
+        }
+        return colRef;
+    }
+
+    private void buildGroups() {
+        ColumnRowType inputColumnRowType = ((OLAPRel) getChild()).getColumnRowType();
+        this.groups = new ArrayList<TblColRef>();
+        for (int i = getGroupSet().nextSetBit(0); i >= 0; i = getGroupSet().nextSetBit(i + 1)) {
+            Set<TblColRef> columns = inputColumnRowType.getSourceColumnsByIndex(i);
+            this.groups.addAll(columns);
+        }
+    }
+
+    private void buildAggregations() {
+        ColumnRowType inputColumnRowType = ((OLAPRel) getChild()).getColumnRowType();
+        this.aggregations = new ArrayList<FunctionDesc>();
+        for (AggregateCall aggCall : this.rewriteAggCalls) {
+            ParameterDesc parameter = null;
+            if (!aggCall.getArgList().isEmpty()) {
+                int index = aggCall.getArgList().get(0);
+                TblColRef column = inputColumnRowType.getColumnByIndex(index);
+                if (!column.isInnerColumn()) {
+                    parameter = new ParameterDesc();
+                    parameter.setValue(column.getName());
+                    parameter.setType("column");
+                }
+            }
+            FunctionDesc aggFunc = new FunctionDesc();
+            String funcName = getFuncName(aggCall);
+            aggFunc.setExpression(funcName);
+            aggFunc.setParameter(parameter);
+            this.aggregations.add(aggFunc);
+        }
+    }
+
+    private void translateGroupBy() {
+        context.groupByColumns.addAll(this.groups);
+    }
+
+    private void translateAggregation() {
+        ColumnRowType inputColumnRowType = ((OLAPRel) getChild()).getColumnRowType();
+        for (int i = 0; i < this.aggregations.size(); i++) {
+            FunctionDesc aggFunc = this.aggregations.get(i);
+            context.aggregations.add(aggFunc);
+            if (aggFunc.needRewrite()) {
+                String rewriteFieldName = aggFunc.getRewriteFieldName();
+                context.rewriteFields.put(rewriteFieldName, null);
+
+                TblColRef column = buildRewriteColumn(aggFunc);
+                this.context.metricsColumns.add(column);
+            }
+            AggregateCall aggCall = this.rewriteAggCalls.get(i);
+            if (!aggCall.getArgList().isEmpty()) {
+                int index = aggCall.getArgList().get(0);
+                TblColRef column = inputColumnRowType.getColumnByIndex(index);
+                if (!column.isInnerColumn()) {
+                    this.context.metricsColumns.add(column);
+                }
+            }
+        }
+    }
+
+    private void fillbackOptimizedColumn() {
+        // some aggcall will be optimized out in sub-query (e.g. tableau
+        // generated sql)
+        // we need to fill them back
+        RelDataType inputAggRow = getChild().getRowType();
+        RelDataType outputAggRow = getRowType();
+        if (inputAggRow.getFieldCount() != outputAggRow.getFieldCount()) {
+            for (RelDataTypeField inputField : inputAggRow.getFieldList()) {
+                String inputFieldName = inputField.getName();
+                if (outputAggRow.getField(inputFieldName, true) == null) {
+                    TblColRef column = this.columnRowType.getColumnByIndex(inputField.getIndex());
+                    this.context.metricsColumns.add(column);
+                }
+            }
+        }
+    }
+
+    @Override
+    public void implementRewrite(RewriteImplementor implementor) {
+        implementor.visitChild(this, getChild());
+
+        // only rewrite the first aggregation
+        if (!this.afterAggregate && RewriteImplementor.needRewrite(this.context)) {
+            // rewrite the aggCalls
+            this.rewriteAggCalls = new ArrayList<AggregateCall>(aggCalls.size());
+            for (int i = 0; i < this.aggCalls.size(); i++) {
+                AggregateCall aggCall = this.aggCalls.get(i);
+                FunctionDesc cubeFunc = this.context.aggregations.get(i);
+                if (cubeFunc.needRewrite()) {
+                    aggCall = rewriteAggregateCall(aggCall, cubeFunc);
+                }
+                this.rewriteAggCalls.add(aggCall);
+            }
+        }
+
+        // rebuild rowType & columnRowType
+        this.rowType = this.deriveRowType();
+        this.columnRowType = this.buildColumnRowType();
+
+    }
+
+    private AggregateCall rewriteAggregateCall(AggregateCall aggCall, FunctionDesc func) {
+
+        // rebuild parameters
+        List<Integer> newArgList = new ArrayList<Integer>(1);
+        String fieldName = func.getRewriteFieldName();
+        RelDataTypeField field = getChild().getRowType().getField(fieldName, true);
+        newArgList.add(field.getIndex());
+
+        // rebuild function
+        RelDataType fieldType = aggCall.getType();
+        Aggregation newAgg = aggCall.getAggregation();
+        if (func.isCountDistinct()) {
+            newAgg = createHyperLogLogAggFunction(fieldType);
+        } else if (func.isCount()) {
+            newAgg = new SqlSumEmptyIsZeroAggFunction(fieldType);
+        }
+
+        // rebuild aggregate call
+        AggregateCall newAggCall = new AggregateCall(newAgg, false, newArgList, fieldType, newAgg.getName());
+
+        // To make sure specified type matches the inferReturnType, or otherwise
+        // there will be assertion failure in optiq
+        // The problem is BIGINT != BIGINT NOT NULL
+        // Details see https://github.scm.corp.ebay.com/Kylin/Kylin/issues/323
+        SqlAggFunction aggFunction = (SqlAggFunction) newAggCall.getAggregation();
+        AggCallBinding callBinding = newAggCall.createBinding(this);
+        RelDataType inferReturnType = aggFunction.inferReturnType(callBinding);
+
+        return new AggregateCall(newAgg, false, newArgList, inferReturnType, newAgg.getName());
+    }
+
+    private Aggregation createHyperLogLogAggFunction(RelDataType returnType) {
+        RelDataTypeFactory typeFactory = getCluster().getTypeFactory();
+        SqlIdentifier sqlIdentifier = new SqlIdentifier("HLL_COUNT", new SqlParserPos(1, 1));
+        AggregateFunction aggFunction = AggregateFunctionImpl.create(HLLDistinctCountAggFunc.class);
+        List<RelDataType> argTypes = new ArrayList<RelDataType>();
+        List<SqlTypeFamily> typeFamilies = new ArrayList<SqlTypeFamily>();
+        for (FunctionParameter o : aggFunction.getParameters()) {
+            final RelDataType type = o.getType(typeFactory);
+            argTypes.add(type);
+            typeFamilies.add(Util.first(type.getSqlTypeName().getFamily(), SqlTypeFamily.ANY));
+        }
+        return new SqlUserDefinedAggFunction(sqlIdentifier, ReturnTypes.explicit(returnType), InferTypes.explicit(argTypes), OperandTypes.family(typeFamilies), aggFunction);
+    }
+
+    @Override
+    public Result implement(EnumerableRelImplementor implementor, Prefer pref) {
+
+        EnumerableAggregateRel enumAggRel;
+        try {
+            enumAggRel = new EnumerableAggregateRel(getCluster(), getCluster().traitSetOf(EnumerableConvention.INSTANCE), getChild(), this.groupSet, rewriteAggCalls);
+        } catch (InvalidRelException e) {
+            throw new IllegalStateException("Can't create EnumerableAggregateRel!", e);
+        }
+
+        return enumAggRel.implement(implementor, pref);
+    }
+
+    @Override
+    public OLAPContext getContext() {
+        return context;
+    }
+
+    @Override
+    public ColumnRowType getColumnRowType() {
+        return columnRowType;
+    }
+
+    @Override
+    public boolean hasSubQuery() {
+        OLAPRel olapChild = (OLAPRel) getChild();
+        return olapChild.hasSubQuery();
+    }
+
+    @Override
+    public RelTraitSet replaceTraitSet(RelTrait trait) {
+        RelTraitSet oldTraitSet = this.traitSet;
+        this.traitSet = this.traitSet.replace(trait);
+        return oldTraitSet;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/query/src/main/java/com/kylinolap/query/relnode/OLAPContext.java
----------------------------------------------------------------------
diff --git a/query/src/main/java/com/kylinolap/query/relnode/OLAPContext.java b/query/src/main/java/com/kylinolap/query/relnode/OLAPContext.java
new file mode 100644
index 0000000..7de1817
--- /dev/null
+++ b/query/src/main/java/com/kylinolap/query/relnode/OLAPContext.java
@@ -0,0 +1,122 @@
+/*
+ * 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.query.relnode;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+
+import org.eigenbase.reltype.RelDataType;
+
+import com.kylinolap.cube.CubeInstance;
+import com.kylinolap.metadata.model.cube.CubeDesc;
+import com.kylinolap.metadata.model.cube.FunctionDesc;
+import com.kylinolap.metadata.model.cube.JoinDesc;
+import com.kylinolap.metadata.model.cube.TblColRef;
+import com.kylinolap.query.schema.OLAPSchema;
+import com.kylinolap.storage.StorageContext;
+import com.kylinolap.storage.filter.TupleFilter;
+
+/**
+ * @author xjiang
+ * 
+ */
+public class OLAPContext {
+
+    public static final String PRM_ACCEPT_PARTIAL_RESULT = "AcceptPartialResult";
+
+    private static final ThreadLocal<Map<String, String>> _localPrarameters = new ThreadLocal<Map<String, String>>();
+
+    private static final ThreadLocal<Map<Integer, OLAPContext>> _localContexts = new ThreadLocal<Map<Integer, OLAPContext>>();
+
+    public static void setParameters(Map<String, String> parameters) {
+        _localPrarameters.set(parameters);
+    }
+
+    public static void clearParameter() {
+        _localPrarameters.remove();
+    }
+
+    public static void registerContext(OLAPContext ctx) {
+        if (_localContexts.get() == null) {
+            Map<Integer, OLAPContext> contextMap = new HashMap<Integer, OLAPContext>();
+            _localContexts.set(contextMap);
+        }
+        _localContexts.get().put(ctx.id, ctx);
+    }
+
+    public static Collection<OLAPContext> getThreadLocalContexts() {
+        Map<Integer, OLAPContext> map = _localContexts.get();
+        return map == null ? null : map.values();
+    }
+
+    public static OLAPContext getThreadLocalContextById(int id) {
+        return _localContexts.get().get(id);
+    }
+
+    public static void clearThreadLocalContexts() {
+        _localContexts.remove();
+    }
+
+    public OLAPContext(int seq) {
+        this.id = seq;
+        this.storageContext = new StorageContext();
+        Map<String, String> parameters = _localPrarameters.get();
+        if (parameters != null) {
+            String acceptPartialResult = parameters.get(PRM_ACCEPT_PARTIAL_RESULT);
+            if (acceptPartialResult != null) {
+                this.storageContext.setAcceptPartialResult(Boolean.parseBoolean(acceptPartialResult));
+            }
+        }
+    }
+
+    public final int id;
+    public final StorageContext storageContext;
+
+    // query info
+    public OLAPSchema olapSchema = null;
+    public OLAPTableScan firstTableScan = null; // to be fact table scan except
+                                                // "select * from lookupTable"
+    public RelDataType olapRowType = null;
+    public boolean afterAggregate = false;
+    public boolean afterJoin = false;
+    public boolean hasJoin = false;
+
+    // cube metadata
+    public CubeInstance cubeInstance;
+    public CubeDesc cubeDesc;
+    public Collection<TblColRef> allColumns = new HashSet<TblColRef>();
+    public Collection<TblColRef> metricsColumns = new HashSet<TblColRef>();
+    public Collection<TblColRef> groupByColumns = new ArrayList<TblColRef>();
+    public List<FunctionDesc> aggregations = new ArrayList<FunctionDesc>();
+    public List<JoinDesc> joins = new LinkedList<JoinDesc>();
+    public TupleFilter filter;
+
+    // rewrite info
+    public Map<String, RelDataType> rewriteFields = new HashMap<String, RelDataType>();
+
+    // hive query
+    public String sql = "";
+
+    public boolean isSimpleQuery() {
+        return (joins.size() == 0) && (groupByColumns.size() == 0) && (aggregations.size() == 0);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/query/src/main/java/com/kylinolap/query/relnode/OLAPFilterRel.java
----------------------------------------------------------------------
diff --git a/query/src/main/java/com/kylinolap/query/relnode/OLAPFilterRel.java b/query/src/main/java/com/kylinolap/query/relnode/OLAPFilterRel.java
new file mode 100644
index 0000000..923a540
--- /dev/null
+++ b/query/src/main/java/com/kylinolap/query/relnode/OLAPFilterRel.java
@@ -0,0 +1,326 @@
+/*
+ * 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.query.relnode;
+
+import java.util.Calendar;
+import java.util.GregorianCalendar;
+import java.util.LinkedList;
+import java.util.List;
+
+import net.hydromatic.optiq.rules.java.EnumerableConvention;
+import net.hydromatic.optiq.rules.java.EnumerableRel;
+import net.hydromatic.optiq.rules.java.EnumerableRelImplementor;
+import net.hydromatic.optiq.rules.java.JavaRules.EnumerableCalcRel;
+import net.hydromatic.optiq.runtime.SqlFunctions;
+
+import org.eigenbase.rel.FilterRelBase;
+import org.eigenbase.rel.RelCollation;
+import org.eigenbase.rel.RelNode;
+import org.eigenbase.relopt.RelOptCluster;
+import org.eigenbase.relopt.RelOptCost;
+import org.eigenbase.relopt.RelOptPlanner;
+import org.eigenbase.relopt.RelTrait;
+import org.eigenbase.relopt.RelTraitSet;
+import org.eigenbase.reltype.RelDataType;
+import org.eigenbase.rex.RexBuilder;
+import org.eigenbase.rex.RexCall;
+import org.eigenbase.rex.RexDynamicParam;
+import org.eigenbase.rex.RexInputRef;
+import org.eigenbase.rex.RexLiteral;
+import org.eigenbase.rex.RexLocalRef;
+import org.eigenbase.rex.RexNode;
+import org.eigenbase.rex.RexProgram;
+import org.eigenbase.rex.RexProgramBuilder;
+import org.eigenbase.rex.RexVisitorImpl;
+import org.eigenbase.sql.SqlKind;
+import org.eigenbase.sql.SqlOperator;
+import org.eigenbase.util.NlsString;
+
+import com.google.common.base.Preconditions;
+import com.google.common.collect.ImmutableList;
+import com.kylinolap.metadata.model.cube.TblColRef;
+import com.kylinolap.storage.filter.CaseTupleFilter;
+import com.kylinolap.storage.filter.ColumnTupleFilter;
+import com.kylinolap.storage.filter.CompareTupleFilter;
+import com.kylinolap.storage.filter.ConstantTupleFilter;
+import com.kylinolap.storage.filter.DynamicTupleFilter;
+import com.kylinolap.storage.filter.ExtractTupleFilter;
+import com.kylinolap.storage.filter.LogicalTupleFilter;
+import com.kylinolap.storage.filter.TupleFilter;
+import com.kylinolap.storage.filter.TupleFilter.FilterOperatorEnum;
+
+/**
+ * @author xjiang
+ */
+public class OLAPFilterRel extends FilterRelBase implements OLAPRel, EnumerableRel {
+
+    private static class TupleFilterVisitor extends RexVisitorImpl<TupleFilter> {
+
+        private final ColumnRowType inputRowType;
+        private final OLAPContext context;
+
+        public TupleFilterVisitor(ColumnRowType inputRowType, OLAPContext context) {
+            super(true);
+            this.inputRowType = inputRowType;
+            this.context = context;
+        }
+
+        @Override
+        public TupleFilter visitCall(RexCall call) {
+            TupleFilter filter = null;
+            SqlOperator op = call.getOperator();
+            switch (op.getKind()) {
+            case AND:
+                filter = new LogicalTupleFilter(FilterOperatorEnum.AND);
+                break;
+            case OR:
+                filter = new LogicalTupleFilter(FilterOperatorEnum.OR);
+                break;
+            case NOT:
+                filter = new LogicalTupleFilter(FilterOperatorEnum.NOT);
+                break;
+            case EQUALS:
+                filter = new CompareTupleFilter(FilterOperatorEnum.EQ);
+                break;
+            case GREATER_THAN:
+                filter = new CompareTupleFilter(FilterOperatorEnum.GT);
+                break;
+            case LESS_THAN:
+                filter = new CompareTupleFilter(FilterOperatorEnum.LT);
+                break;
+            case GREATER_THAN_OR_EQUAL:
+                filter = new CompareTupleFilter(FilterOperatorEnum.GTE);
+                break;
+            case LESS_THAN_OR_EQUAL:
+                filter = new CompareTupleFilter(FilterOperatorEnum.LTE);
+                break;
+            case NOT_EQUALS:
+                filter = new CompareTupleFilter(FilterOperatorEnum.NEQ);
+                break;
+            case IS_NULL:
+                filter = new CompareTupleFilter(FilterOperatorEnum.ISNULL);
+                break;
+            case IS_NOT_NULL:
+                filter = new CompareTupleFilter(FilterOperatorEnum.ISNOTNULL);
+                break;
+            case CAST:
+            case REINTERPRET:
+                // NOTE: use child directly
+                break;
+            case CASE:
+                filter = new CaseTupleFilter();
+                break;
+            case OTHER:
+                if (op.getName().equalsIgnoreCase("extract_date")) {
+                    filter = new ExtractTupleFilter(FilterOperatorEnum.EXTRACT);
+                } else {
+                    throw new UnsupportedOperationException(op.getName());
+                }
+                break;
+            default:
+                throw new UnsupportedOperationException(op.getName());
+            }
+
+            for (RexNode operand : call.operands) {
+                TupleFilter childFilter = operand.accept(this);
+                if (filter == null) {
+                    filter = childFilter;
+                } else {
+                    filter.addChild(childFilter);
+                }
+            }
+
+            if (op.getKind() == SqlKind.OR) {
+                CompareTupleFilter inFilter = mergeToInClause(filter);
+                if (inFilter != null) {
+                    filter = inFilter;
+                }
+            }
+            return filter;
+        }
+
+        private CompareTupleFilter mergeToInClause(TupleFilter filter) {
+            List<? extends TupleFilter> children = filter.getChildren();
+            TblColRef inColumn = null;
+            List<String> inValues = new LinkedList<String>();
+            for (TupleFilter child : children) {
+                if (child.getOperator() == FilterOperatorEnum.EQ) {
+                    CompareTupleFilter compFilter = (CompareTupleFilter) child;
+                    TblColRef column = compFilter.getColumn();
+                    if (inColumn == null) {
+                        inColumn = column;
+                    }
+
+                    if (column == null || !column.equals(inColumn)) {
+                        return null;
+                    }
+                    inValues.addAll(compFilter.getValues());
+                } else {
+                    return null;
+                }
+            }
+
+            children.clear();
+
+            CompareTupleFilter inFilter = new CompareTupleFilter(FilterOperatorEnum.IN);
+            inFilter.addChild(new ColumnTupleFilter(inColumn));
+            inFilter.addChild(new ConstantTupleFilter(inValues));
+            return inFilter;
+        }
+
+        @Override
+        public TupleFilter visitLocalRef(RexLocalRef localRef) {
+            throw new UnsupportedOperationException("local ref:" + localRef);
+        }
+
+        @Override
+        public TupleFilter visitInputRef(RexInputRef inputRef) {
+            TblColRef column = inputRowType.getColumnByIndex(inputRef.getIndex());
+            context.allColumns.add(column);
+            ColumnTupleFilter filter = new ColumnTupleFilter(column);
+            return filter;
+        }
+
+        private String normToTwoDigits(int i) {
+            if (i < 10)
+                return "0" + i;
+            else
+                return "" + i;
+        }
+
+        @Override
+        public TupleFilter visitLiteral(RexLiteral literal) {
+            String strValue = null;
+            Object literalValue = literal.getValue();
+            if (literalValue instanceof NlsString) {
+                strValue = ((NlsString) literalValue).getValue();
+            } else if (literalValue instanceof GregorianCalendar) {
+                GregorianCalendar g = (GregorianCalendar) literalValue;
+                strValue = "" + g.get(Calendar.YEAR) + "-" + normToTwoDigits(g.get(Calendar.MONTH) + 1) + "-" + normToTwoDigits(g.get(Calendar.DAY_OF_MONTH));
+            } else if (literalValue instanceof SqlFunctions.TimeUnitRange) {
+                // Extract(x from y) in where clause
+                strValue = ((SqlFunctions.TimeUnitRange) literalValue).name();
+            } else if (literalValue == null) {
+                strValue = null;
+            } else {
+                strValue = literalValue.toString();
+            }
+            TupleFilter filter = new ConstantTupleFilter(strValue);
+            return filter;
+        }
+
+        @Override
+        public TupleFilter visitDynamicParam(RexDynamicParam dynamicParam) {
+            String name = dynamicParam.getName();
+            TupleFilter filter = new DynamicTupleFilter(name);
+            return filter;
+        }
+    }
+
+    private ColumnRowType columnRowType;
+    private OLAPContext context;
+
+    public OLAPFilterRel(RelOptCluster cluster, RelTraitSet traits, RelNode child, RexNode condition) {
+        super(cluster, traits, child, condition);
+        Preconditions.checkArgument(getConvention() == OLAPRel.CONVENTION);
+        Preconditions.checkArgument(getConvention() == child.getConvention());
+        this.rowType = getRowType();
+    }
+
+    @Override
+    public RelOptCost computeSelfCost(RelOptPlanner planner) {
+        return super.computeSelfCost(planner).multiplyBy(.05);
+    }
+
+    @Override
+    public FilterRelBase copy(RelTraitSet traitSet, RelNode input, RexNode condition) {
+        return new OLAPFilterRel(getCluster(), traitSet, input, condition);
+    }
+
+    @Override
+    public void implementOLAP(OLAPImplementor implementor) {
+        implementor.visitChild(getChild(), this);
+
+        this.columnRowType = buildColumnRowType();
+        this.context = implementor.getContext();
+
+        // only translate where clause and don't translate having clause
+        if (!context.afterAggregate) {
+            translateFilter(context);
+        }
+    }
+
+    private ColumnRowType buildColumnRowType() {
+        OLAPRel olapChild = (OLAPRel) getChild();
+        ColumnRowType inputColumnRowType = olapChild.getColumnRowType();
+        return inputColumnRowType;
+    }
+
+    private void translateFilter(OLAPContext context) {
+        if (this.condition == null) {
+            return;
+        }
+
+        TupleFilterVisitor visitor = new TupleFilterVisitor(this.columnRowType, context);
+        context.filter = this.condition.accept(visitor);
+    }
+
+    @Override
+    public Result implement(EnumerableRelImplementor implementor, Prefer pref) {
+        // keep it for having clause
+        RexBuilder rexBuilder = getCluster().getRexBuilder();
+        RelDataType inputRowType = getChild().getRowType();
+        RexProgramBuilder programBuilder = new RexProgramBuilder(inputRowType, rexBuilder);
+        programBuilder.addIdentity();
+        programBuilder.addCondition(this.condition);
+        RexProgram program = programBuilder.getProgram();
+
+        EnumerableCalcRel enumCalcRel = new EnumerableCalcRel(getCluster(), getCluster().traitSetOf(EnumerableConvention.INSTANCE), getChild(), this.rowType, program, ImmutableList.<RelCollation> of());
+
+        return enumCalcRel.implement(implementor, pref);
+    }
+
+    @Override
+    public void implementRewrite(RewriteImplementor implementor) {
+        implementor.visitChild(this, getChild());
+
+        this.rowType = this.deriveRowType();
+        this.columnRowType = buildColumnRowType();
+    }
+
+    @Override
+    public OLAPContext getContext() {
+        return context;
+    }
+
+    @Override
+    public ColumnRowType getColumnRowType() {
+        return columnRowType;
+    }
+
+    @Override
+    public boolean hasSubQuery() {
+        OLAPRel olapChild = (OLAPRel) getChild();
+        return olapChild.hasSubQuery();
+    }
+
+    @Override
+    public RelTraitSet replaceTraitSet(RelTrait trait) {
+        RelTraitSet oldTraitSet = this.traitSet;
+        this.traitSet = this.traitSet.replace(trait);
+        return oldTraitSet;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/query/src/main/java/com/kylinolap/query/relnode/OLAPJoinRel.java
----------------------------------------------------------------------
diff --git a/query/src/main/java/com/kylinolap/query/relnode/OLAPJoinRel.java b/query/src/main/java/com/kylinolap/query/relnode/OLAPJoinRel.java
new file mode 100644
index 0000000..83e7993
--- /dev/null
+++ b/query/src/main/java/com/kylinolap/query/relnode/OLAPJoinRel.java
@@ -0,0 +1,294 @@
+/*
+ * 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.query.relnode;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import net.hydromatic.linq4j.expressions.Blocks;
+import net.hydromatic.linq4j.expressions.Expressions;
+import net.hydromatic.optiq.rules.java.EnumerableRelImplementor;
+import net.hydromatic.optiq.rules.java.JavaRules.EnumerableJoinRel;
+import net.hydromatic.optiq.rules.java.PhysType;
+import net.hydromatic.optiq.rules.java.PhysTypeImpl;
+
+import org.eigenbase.rel.InvalidRelException;
+import org.eigenbase.rel.JoinInfo;
+import org.eigenbase.rel.JoinRelType;
+import org.eigenbase.rel.RelNode;
+import org.eigenbase.relopt.RelOptCluster;
+import org.eigenbase.relopt.RelOptCost;
+import org.eigenbase.relopt.RelOptPlanner;
+import org.eigenbase.relopt.RelOptTable;
+import org.eigenbase.relopt.RelTrait;
+import org.eigenbase.relopt.RelTraitSet;
+import org.eigenbase.reltype.RelDataType;
+import org.eigenbase.reltype.RelDataTypeFactory.FieldInfoBuilder;
+import org.eigenbase.reltype.RelDataTypeField;
+import org.eigenbase.reltype.RelDataTypeFieldImpl;
+import org.eigenbase.rex.RexCall;
+import org.eigenbase.rex.RexInputRef;
+import org.eigenbase.rex.RexNode;
+import org.eigenbase.sql.SqlKind;
+import org.eigenbase.util.ImmutableIntList;
+
+import com.google.common.base.Preconditions;
+import com.kylinolap.metadata.model.cube.JoinDesc;
+import com.kylinolap.metadata.model.cube.TblColRef;
+import com.kylinolap.query.schema.OLAPTable;
+
+/**
+ * @author xjiang
+ */
+public class OLAPJoinRel extends EnumerableJoinRel implements OLAPRel {
+
+    private final static String[] COLUMN_ARRAY_MARKER = new String[0];
+
+    private OLAPContext context;
+    private ColumnRowType columnRowType;
+    private boolean isTopJoin;
+    private boolean hasSubQuery;
+
+    public OLAPJoinRel(RelOptCluster cluster, RelTraitSet traits, RelNode left, RelNode right, //
+            RexNode condition, ImmutableIntList leftKeys, ImmutableIntList rightKeys, //
+            JoinRelType joinType, Set<String> variablesStopped) throws InvalidRelException {
+        super(cluster, traits, left, right, condition, leftKeys, rightKeys, joinType, variablesStopped);
+        Preconditions.checkArgument(getConvention() == OLAPRel.CONVENTION);
+        this.rowType = getRowType();
+        this.isTopJoin = false;
+    }
+
+    @Override
+    public EnumerableJoinRel copy(RelTraitSet traitSet, RexNode conditionExpr, RelNode left, RelNode right, //
+            JoinRelType joinType, boolean semiJoinDone) {
+
+        final JoinInfo joinInfo = JoinInfo.of(left, right, condition);
+        assert joinInfo.isEqui();
+        try {
+            return new OLAPJoinRel(getCluster(), traitSet, left, right, condition, joinInfo.leftKeys, joinInfo.rightKeys, joinType, variablesStopped);
+        } catch (InvalidRelException e) {
+            // Semantic error not possible. Must be a bug. Convert to
+            // internal error.
+            throw new AssertionError(e);
+        }
+    }
+
+    @Override
+    public RelOptCost computeSelfCost(RelOptPlanner planner) {
+        return super.computeSelfCost(planner).multiplyBy(.05);
+    }
+
+    @Override
+    public double getRows() {
+        return super.getRows() * 0.1;
+    }
+
+    @Override
+    public void implementOLAP(OLAPImplementor implementor) {
+
+        // create context for root join
+        if (!(implementor.getParentNode() instanceof OLAPJoinRel)) {
+            implementor.allocateContext();
+        }
+        this.context = implementor.getContext();
+        this.isTopJoin = !this.context.hasJoin;
+        this.context.hasJoin = true;
+
+        // as we keep the first table as fact table, we need to visit from left
+        // to right
+        implementor.visitChild(this.left, this);
+        if (this.context != implementor.getContext() || ((OLAPRel) this.left).hasSubQuery()) {
+            this.hasSubQuery = true;
+            implementor.freeContext();
+        }
+        implementor.visitChild(this.right, this);
+        if (this.context != implementor.getContext() || ((OLAPRel) this.right).hasSubQuery()) {
+            this.hasSubQuery = true;
+            implementor.freeContext();
+        }
+
+        this.columnRowType = buildColumnRowType();
+        if (isTopJoin) {
+            this.context.afterJoin = true;
+        }
+
+        if (!this.hasSubQuery) {
+            this.context.allColumns.clear();
+            this.context.olapRowType = getRowType();
+            buildAliasMap();
+
+            // build JoinDesc
+            RexCall condition = (RexCall) this.getCondition();
+            JoinDesc join = buildJoin(condition);
+
+            JoinRelType joinRelType = this.getJoinType();
+            String joinType = joinRelType == JoinRelType.INNER ? "INNER" : joinRelType == JoinRelType.LEFT ? "LEFT" : null;
+            join.setType(joinType);
+
+            this.context.joins.add(join);
+        }
+    }
+
+    private ColumnRowType buildColumnRowType() {
+        List<TblColRef> columns = new ArrayList<TblColRef>();
+
+        OLAPRel olapLeft = (OLAPRel) this.left;
+        ColumnRowType leftColumnRowType = olapLeft.getColumnRowType();
+        columns.addAll(leftColumnRowType.getAllColumns());
+
+        OLAPRel olapRight = (OLAPRel) this.right;
+        ColumnRowType rightColumnRowType = olapRight.getColumnRowType();
+        columns.addAll(rightColumnRowType.getAllColumns());
+
+        if (columns.size() != this.rowType.getFieldCount()) {
+            throw new IllegalStateException("RowType=" + this.rowType.getFieldCount() + ", ColumnRowType=" + columns.size());
+        }
+        return new ColumnRowType(columns);
+    }
+
+    private JoinDesc buildJoin(RexCall condition) {
+        Map<TblColRef, TblColRef> joinColumns = new HashMap<TblColRef, TblColRef>();
+        translateJoinColumn(condition, joinColumns);
+
+        List<String> pks = new ArrayList<String>();
+        List<TblColRef> pkCols = new ArrayList<TblColRef>();
+        List<String> fks = new ArrayList<String>();
+        List<TblColRef> fkCols = new ArrayList<TblColRef>();
+        String factTable = context.firstTableScan.getCubeTable();
+        for (Map.Entry<TblColRef, TblColRef> columnPair : joinColumns.entrySet()) {
+            TblColRef fromCol = columnPair.getKey();
+            TblColRef toCol = columnPair.getValue();
+            if (factTable.equalsIgnoreCase(fromCol.getTable())) {
+                fks.add(fromCol.getName());
+                fkCols.add(fromCol);
+                pks.add(toCol.getName());
+                pkCols.add(toCol);
+            } else {
+                fks.add(toCol.getName());
+                fkCols.add(toCol);
+                pks.add(fromCol.getName());
+                pkCols.add(fromCol);
+            }
+        }
+
+        JoinDesc join = new JoinDesc();
+        join.setForeignKey(fks.toArray(COLUMN_ARRAY_MARKER));
+        join.setForeignKeyColumns(fkCols.toArray(new TblColRef[fkCols.size()]));
+        join.setPrimaryKey(pks.toArray(COLUMN_ARRAY_MARKER));
+        join.setPrimaryKeyColumns(pkCols.toArray(new TblColRef[pkCols.size()]));
+        return join;
+    }
+
+    private void translateJoinColumn(RexCall condition, Map<TblColRef, TblColRef> joinColumns) {
+        SqlKind kind = condition.getOperator().getKind();
+        if (kind == SqlKind.AND) {
+            for (RexNode operand : condition.getOperands()) {
+                RexCall subCond = (RexCall) operand;
+                translateJoinColumn(subCond, joinColumns);
+            }
+        } else if (kind == SqlKind.EQUALS) {
+            List<RexNode> operands = condition.getOperands();
+            RexInputRef op0 = (RexInputRef) operands.get(0);
+            TblColRef col0 = columnRowType.getColumnByIndex(op0.getIndex());
+            RexInputRef op1 = (RexInputRef) operands.get(1);
+            TblColRef col1 = columnRowType.getColumnByIndex(op1.getIndex());
+            joinColumns.put(col0, col1);
+        }
+    }
+
+    private void buildAliasMap() {
+        int size = this.rowType.getFieldList().size();
+
+        for (int i = 0; i < size; i++) {
+            RelDataTypeField field = this.rowType.getFieldList().get(i);
+            TblColRef column = this.columnRowType.getColumnByIndex(i);
+            context.storageContext.addAlias(column, field.getName());
+        }
+    }
+
+    @Override
+    public Result implement(EnumerableRelImplementor implementor, Prefer pref) {
+        Result result = null;
+        if (this.hasSubQuery) {
+            result = super.implement(implementor, pref);
+        } else {
+            PhysType physType = PhysTypeImpl.of(implementor.getTypeFactory(), getRowType(), pref.preferArray());
+
+            RelOptTable factTable = context.firstTableScan.getTable();
+            result = implementor.result(physType, Blocks.toBlock(Expressions.call(factTable.getExpression(OLAPTable.class), "executeCubeQuery", implementor.getRootExpression(), Expressions.constant(context.id))));
+        }
+
+        return result;
+    }
+
+    @Override
+    public ColumnRowType getColumnRowType() {
+        return columnRowType;
+    }
+
+    @Override
+    public void implementRewrite(RewriteImplementor implementor) {
+        implementor.visitChild(this, this.left);
+        implementor.visitChild(this, this.right);
+
+        this.rowType = this.deriveRowType();
+
+        if (this.isTopJoin && RewriteImplementor.needRewrite(this.context)) {
+            // find missed rewrite fields
+            int paramIndex = this.rowType.getFieldList().size();
+            List<RelDataTypeField> newFieldList = new LinkedList<RelDataTypeField>();
+            for (Map.Entry<String, RelDataType> rewriteField : this.context.rewriteFields.entrySet()) {
+                String fieldName = rewriteField.getKey();
+                if (this.rowType.getField(fieldName, true) == null) {
+                    RelDataType fieldType = rewriteField.getValue();
+                    RelDataTypeField newField = new RelDataTypeFieldImpl(fieldName, paramIndex++, fieldType);
+                    newFieldList.add(newField);
+                }
+            }
+
+            // rebuild row type
+            FieldInfoBuilder fieldInfo = getCluster().getTypeFactory().builder();
+            fieldInfo.addAll(this.rowType.getFieldList());
+            fieldInfo.addAll(newFieldList);
+            this.rowType = getCluster().getTypeFactory().createStructType(fieldInfo);
+            this.context.olapRowType = this.rowType;
+
+            // rebuild columns
+            this.columnRowType = this.buildColumnRowType();
+        }
+    }
+
+    @Override
+    public OLAPContext getContext() {
+        return context;
+    }
+
+    @Override
+    public boolean hasSubQuery() {
+        return this.hasSubQuery;
+    }
+
+    @Override
+    public RelTraitSet replaceTraitSet(RelTrait trait) {
+        RelTraitSet oldTraitSet = this.traitSet;
+        this.traitSet = this.traitSet.replace(trait);
+        return oldTraitSet;
+    }
+}


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

Posted by lu...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/job/src/main/java/com/kylinolap/job/JobManager.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/com/kylinolap/job/JobManager.java b/job/src/main/java/com/kylinolap/job/JobManager.java
new file mode 100644
index 0000000..91d3d43
--- /dev/null
+++ b/job/src/main/java/com/kylinolap/job/JobManager.java
@@ -0,0 +1,314 @@
+/*
+ * 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.job;
+
+import java.io.IOException;
+import java.net.UnknownHostException;
+import java.text.SimpleDateFormat;
+import java.util.ArrayList;
+import java.util.Date;
+import java.util.List;
+import java.util.TimeZone;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.kylinolap.common.KylinConfig;
+import com.kylinolap.cube.CubeBuildTypeEnum;
+import com.kylinolap.cube.CubeInstance;
+import com.kylinolap.cube.CubeManager;
+import com.kylinolap.cube.CubeSegment;
+import com.kylinolap.cube.CubeSegmentStatusEnum;
+import com.kylinolap.cube.exception.CubeIntegrityException;
+import com.kylinolap.cube.project.ProjectInstance;
+import com.kylinolap.cube.project.ProjectManager;
+import com.kylinolap.job.JobInstance.JobStep;
+import com.kylinolap.job.constant.JobConstants;
+import com.kylinolap.job.constant.JobStatusEnum;
+import com.kylinolap.job.constant.JobStepStatusEnum;
+import com.kylinolap.job.engine.JobEngine;
+import com.kylinolap.job.engine.JobEngineConfig;
+import com.kylinolap.job.exception.InvalidJobInstanceException;
+import com.kylinolap.job.exception.JobException;
+import com.kylinolap.job.hadoop.hive.JoinedFlatTableDesc;
+import com.kylinolap.metadata.model.cube.CubeDesc;
+
+/**
+ * @author xjiang, ysong1
+ * 
+ */
+public class JobManager {
+
+    private static Logger log = LoggerFactory.getLogger(JobManager.class);
+
+    private final KylinConfig config;
+    private final JobEngineConfig engineConfig;
+    private final JobEngine jobEngine;
+    private final JobDAO jobDAO;
+
+    public JobManager(String engineID, JobEngineConfig engineCfg) throws JobException, UnknownHostException {
+        this.engineConfig = engineCfg;
+        this.config = engineConfig.getConfig();
+        this.jobDAO = JobDAO.getInstance(config);
+
+        // InetAddress ia = InetAddress.getLocalHost();
+        // this.jobEngine =
+        // Constant.getInstanceFromEnv(ia.getCanonicalHostName(), cfg);
+        this.jobEngine = JobEngine.getInstance(engineID, engineCfg);
+    }
+
+    public JobInstance createJob(String cubeName, String segmentName, String segmentId, CubeBuildTypeEnum jobType, String submitter) throws IOException {
+        // build job instance
+        JobInstance jobInstance = buildJobInstance(cubeName, segmentName, segmentId, jobType, submitter);
+
+        // create job steps based on job type
+        JobInstanceBuilder stepBuilder = new JobInstanceBuilder(this.engineConfig);
+        jobInstance.addSteps(stepBuilder.buildSteps(jobInstance));
+
+        return jobInstance;
+    }
+
+    private JobInstance buildJobInstance(String cubeName, String segmentName, String segmentId, CubeBuildTypeEnum jobType, String submitter) {
+        SimpleDateFormat format = new SimpleDateFormat("z yyyy-MM-dd HH:mm:ss");
+        format.setTimeZone(TimeZone.getTimeZone(this.engineConfig.getTimeZone()));
+
+        JobInstance jobInstance = new JobInstance();
+        jobInstance.setUuid(segmentId);
+        jobInstance.setType(jobType);
+        jobInstance.setRelatedCube(cubeName);
+        jobInstance.setRelatedSegment(segmentName);
+        jobInstance.setName(cubeName + " - " + segmentName + " - " + jobType.toString() + " - " + format.format(new Date(System.currentTimeMillis())));
+        jobInstance.setSubmitter(submitter);
+        return jobInstance;
+    }
+
+    public String submitJob(JobInstance job) throws IOException, InvalidJobInstanceException {
+        if (hasDuplication(job) == false) {
+            // submitted job status should always be PENDING
+            // job.setStatus(JobStatusEnum.PENDING);
+            jobDAO.updateJobInstance(job);
+            return job.getUuid();
+        } else {
+            throw new InvalidJobInstanceException("Job " + job.getName() + " is duplicated!");
+        }
+    }
+
+    public void resumeJob(String uuid) throws IOException, JobException {
+        JobInstance jobInstance = jobDAO.getJob(uuid);
+        log.info("Resuming job " + uuid);
+        if (jobInstance.getStatus() != JobStatusEnum.ERROR) {
+            throw new RuntimeException("Can't resume job with status " + jobInstance.getStatus().toString());
+        }
+
+        for (JobStep jobStep : jobInstance.getSteps()) {
+            if (jobStep.getStatus() == JobStepStatusEnum.ERROR) {
+                jobStep.setStatus(JobStepStatusEnum.PENDING);
+                // jobStep.setCmdOutput("");
+                jobStep.clearInfo();
+                jobDAO.saveJobOutput(jobStep, "");
+            }
+        }
+        jobDAO.updateJobInstance(jobInstance);
+    }
+
+    private boolean hasDuplication(JobInstance newJob) throws IOException {
+        List<JobInstance> allJobs = listJobs(null, null);
+        for (JobInstance job : allJobs) {
+            if (job.getRelatedCube().equals(newJob.getRelatedCube()) && job.getRelatedSegment().equals(newJob.getRelatedSegment()) && job.getType().equals(newJob.getType()) && job.getStatus().equals(newJob.getStatus())) {
+                return true;
+            }
+        }
+        return false;
+    }
+
+    public void discardJob(String uuid) throws IOException, CubeIntegrityException, JobException {
+        // check job status
+        JobInstance jobInstance = jobDAO.getJob(uuid);
+        CubeInstance cube = CubeManager.getInstance(config).getCube(jobInstance.getRelatedCube());
+
+        switch (jobInstance.getStatus()) {
+        case RUNNING:
+            try {
+                killRunningJob(jobInstance);
+            } finally {
+                CubeManager.getInstance(config).updateSegmentOnJobDiscard(cube, jobInstance.getRelatedSegment());
+            }
+            break;
+        case PENDING:
+            try {
+                killRunningJob(jobInstance);
+            } finally {
+                CubeManager.getInstance(config).updateSegmentOnJobDiscard(cube, jobInstance.getRelatedSegment());
+            }
+            break;
+        case ERROR:
+            try {
+                for (JobStep jobStep : jobInstance.getSteps()) {
+                    if (jobStep.getStatus() != JobStepStatusEnum.FINISHED) {
+                        jobStep.setStatus(JobStepStatusEnum.DISCARDED);
+                    }
+                }
+                jobDAO.updateJobInstance(jobInstance);
+            } finally {
+                CubeManager.getInstance(config).updateSegmentOnJobDiscard(cube, jobInstance.getRelatedSegment());
+            }
+            break;
+        default:
+            throw new IllegalStateException("Invalid status to discard : " + jobInstance.getStatus());
+        }
+    }
+
+    /**
+     * @param uuid
+     * @param jobInstance
+     * @throws IOException
+     * @throws JobException
+     */
+    private void killRunningJob(JobInstance jobInstance) throws IOException, JobException {
+        // find the running step
+        JobStep runningStep = jobInstance.getRunningStep();
+        if (runningStep == null) {
+            throw new IllegalStateException("There is no running step in job " + jobInstance.getUuid());
+        }
+
+        // update job to DISCARDED
+        runningStep.setStatus(JobStepStatusEnum.DISCARDED);
+        runningStep.setExecEndTime(System.currentTimeMillis());
+        jobDAO.updateJobInstance(jobInstance);
+
+        // cancel job in engine
+        this.jobEngine.interruptJob(jobInstance, runningStep);
+    }
+
+    public List<JobInstance> listJobs(String cubeName, String projectName) throws IOException {
+        List<JobInstance> jobs = jobDAO.listAllJobs(cubeName);
+
+        if (null == projectName || null == ProjectManager.getInstance(config).getProject(projectName)) {
+            return jobs;
+        } else {
+            List<JobInstance> filtedJobs = new ArrayList<JobInstance>();
+            ProjectInstance project = ProjectManager.getInstance(config).getProject(projectName);
+            for (JobInstance job : jobs) {
+                if (project.getCubes().contains(job.getRelatedCube().toUpperCase())) {
+                    filtedJobs.add(job);
+                }
+            }
+
+            return filtedJobs;
+        }
+    }
+
+    public JobInstance getJob(String uuid) throws IOException {
+        return jobDAO.getJob(uuid);
+    }
+
+    public String getJobStepOutput(String jobUuid, int stepSequenceId) throws IOException {
+        JobStepOutput output = jobDAO.getJobOutput(jobUuid, stepSequenceId);
+        if (null == output) {
+            return "";
+        } else {
+            return output.getOutput();
+        }
+    }
+
+    public void deleteJob(String uuid) throws IOException {
+        jobDAO.deleteJob(uuid);
+    }
+
+    public void deleteAllJobs() throws IOException {
+        List<JobInstance> allJobs = listJobs(null, null);
+        for (JobInstance job : allJobs) {
+            jobDAO.deleteJob(job);
+        }
+    }
+
+    public String previewFlatHiveQL(String cubeName, String segmentName) {
+        CubeInstance cube = CubeManager.getInstance(config).getCube(cubeName);
+        CubeDesc cubeDesc = cube.getDescriptor();
+        CubeSegment cubeSegment = cube.getSegment(segmentName, CubeSegmentStatusEnum.READY);
+        JoinedFlatTableDesc flatTableDesc = new JoinedFlatTableDesc(cubeDesc, cubeSegment);
+        return JoinedFlatTable.generateSelectDataStatement(flatTableDesc);
+    }
+
+    public void startJobEngine() throws Exception {
+        startJobEngine(JobConstants.DEFAULT_SCHEDULER_INTERVAL_SECONDS);
+    }
+
+    public void startJobEngine(int daemonJobIntervalInSeconds) throws Exception {
+        jobDAO.updateRunningJobToError();
+        jobEngine.start(daemonJobIntervalInSeconds);
+    }
+
+    public void stopJobEngine() throws JobException {
+        jobEngine.stop();
+    }
+
+    // Job engine metrics related methods
+
+    public int getNumberOfJobStepsExecuted() {
+        return jobEngine.getNumberOfJobStepsExecuted();
+    }
+
+    public String getPrimaryEngineID() throws Exception {
+        return jobEngine.getPrimaryEngineID();
+    }
+
+    public double getMinJobStepDuration() {
+        return jobEngine.getMinJobStepDuration();
+    }
+
+    public double getMaxJobStepDuration() {
+        return jobEngine.getMaxJobStepDuration();
+    }
+
+    /**
+     * @param percentile
+     *            (eg. 95 percentile)
+     * @return the percentile value
+     */
+    public double getPercentileJobStepDuration(double percentile) {
+        return jobEngine.getPercentileJobStepDuration(percentile);
+    }
+
+    /**
+     * @return
+     */
+    public Integer getScheduledJobsSzie() {
+        return jobEngine.getScheduledJobsSzie();
+    }
+
+    /**
+     * @return
+     */
+    public int getEngineThreadPoolSize() {
+        return jobEngine.getEngineThreadPoolSize();
+    }
+
+    /**
+     * @return
+     */
+    public int getNumberOfIdleSlots() {
+        return jobEngine.getNumberOfIdleSlots();
+    }
+
+    /**
+     * @return
+     */
+    public int getNumberOfJobStepsRunning() {
+        return jobEngine.getNumberOfJobStepsRunning();
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/job/src/main/java/com/kylinolap/job/JobStepOutput.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/com/kylinolap/job/JobStepOutput.java b/job/src/main/java/com/kylinolap/job/JobStepOutput.java
new file mode 100644
index 0000000..cfa3e0c
--- /dev/null
+++ b/job/src/main/java/com/kylinolap/job/JobStepOutput.java
@@ -0,0 +1,71 @@
+/*
+ * 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.job;
+
+import com.fasterxml.jackson.annotation.JsonAutoDetect;
+import com.fasterxml.jackson.annotation.JsonAutoDetect.Visibility;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.kylinolap.common.persistence.RootPersistentEntity;
+
+/**
+ * @author ysong1
+ * 
+ */
+
+@JsonAutoDetect(fieldVisibility = Visibility.NONE, getterVisibility = Visibility.NONE, isGetterVisibility = Visibility.NONE, setterVisibility = Visibility.NONE)
+public class JobStepOutput extends RootPersistentEntity {
+
+    public static String nameOfOutput(String jobUuid, int stepSequenceId) {
+        return jobUuid + "." + stepSequenceId;
+    }
+
+    @JsonProperty("name")
+    private String name; // job uuid + "." +step sequence id
+    @JsonProperty("output")
+    private String output = "";
+
+    /**
+     * @return the name
+     */
+    public String getName() {
+        return name;
+    }
+
+    /**
+     * @return the output
+     */
+    public String getOutput() {
+        return output;
+    }
+
+    /**
+     * @param output
+     *            the output to set
+     */
+    public void setOutput(String output) {
+        this.output = output;
+    }
+
+    /**
+     * @param name
+     *            the name to set
+     */
+    public void setName(String name) {
+        this.name = name;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/job/src/main/java/com/kylinolap/job/JoinedFlatTable.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/com/kylinolap/job/JoinedFlatTable.java b/job/src/main/java/com/kylinolap/job/JoinedFlatTable.java
new file mode 100644
index 0000000..c5f93b4
--- /dev/null
+++ b/job/src/main/java/com/kylinolap/job/JoinedFlatTable.java
@@ -0,0 +1,220 @@
+/*
+ * 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.job;
+
+import java.io.File;
+import java.io.IOException;
+import java.text.SimpleDateFormat;
+import java.util.Date;
+import java.util.HashSet;
+import java.util.Set;
+import java.util.TimeZone;
+
+import javax.xml.parsers.DocumentBuilder;
+import javax.xml.parsers.DocumentBuilderFactory;
+import javax.xml.parsers.ParserConfigurationException;
+
+import org.w3c.dom.Document;
+import org.w3c.dom.NodeList;
+import org.xml.sax.SAXException;
+
+import com.kylinolap.common.util.StringUtil;
+import com.kylinolap.cube.CubeSegment;
+import com.kylinolap.job.engine.JobEngineConfig;
+import com.kylinolap.job.hadoop.hive.JoinedFlatTableDesc;
+import com.kylinolap.job.hadoop.hive.JoinedFlatTableDesc.IntermediateColumnDesc;
+import com.kylinolap.job.hadoop.hive.SqlHiveDataTypeMapping;
+import com.kylinolap.metadata.model.cube.CubeDesc;
+import com.kylinolap.metadata.model.cube.DimensionDesc;
+import com.kylinolap.metadata.model.cube.JoinDesc;
+import com.kylinolap.metadata.model.cube.TblColRef;
+
+/**
+ * @author George Song (ysong1)
+ * 
+ */
+public class JoinedFlatTable {
+
+    public static String getTableDir(JoinedFlatTableDesc intermediateTableDesc, String storageDfsDir, String jobUUID) {
+        return storageDfsDir + "/" + intermediateTableDesc.getTableName(jobUUID);
+    }
+
+    public static String generateCreateTableStatement(JoinedFlatTableDesc intermediateTableDesc, String storageDfsDir, String jobUUID) {
+        StringBuilder ddl = new StringBuilder();
+
+        ddl.append("CREATE EXTERNAL TABLE IF NOT EXISTS " + intermediateTableDesc.getTableName(jobUUID) + "\n");
+
+        ddl.append("(" + "\n");
+        for (int i = 0; i < intermediateTableDesc.getColumnList().size(); i++) {
+            IntermediateColumnDesc col = intermediateTableDesc.getColumnList().get(i);
+            if (i > 0) {
+                ddl.append(",");
+            }
+            ddl.append(col.getColumnName() + " " + SqlHiveDataTypeMapping.getHiveDataType(col.getDataType()) + "\n");
+        }
+        ddl.append(")" + "\n");
+
+        ddl.append("ROW FORMAT DELIMITED FIELDS TERMINATED BY '\\177'" + "\n");
+        ddl.append("STORED AS SEQUENCEFILE" + "\n");
+        ddl.append("LOCATION '" + storageDfsDir + "/" + intermediateTableDesc.getTableName(jobUUID) + "'" + ";");
+        // ddl.append("TBLPROPERTIES ('serialization.null.format'='\\\\N')" +
+        // ";\n");
+        return ddl.toString();
+    }
+
+    public static String generateDropTableStatement(JoinedFlatTableDesc intermediateTableDesc, String jobUUID) {
+        StringBuilder ddl = new StringBuilder();
+        ddl.append("DROP TABLE IF EXISTS " + intermediateTableDesc.getTableName(jobUUID) + ";");
+        return ddl.toString();
+    }
+
+    public static String generateInsertDataStatement(JoinedFlatTableDesc intermediateTableDesc, String jobUUID, JobEngineConfig engineConfig) throws IOException {
+        StringBuilder sql = new StringBuilder();
+
+        File hadoopPropertiesFile = new File(engineConfig.getHadoopJobConfFilePath(intermediateTableDesc.getCubeDesc().getCapacity()));
+
+        if (hadoopPropertiesFile.exists()) {
+            DocumentBuilderFactory factory = DocumentBuilderFactory.newInstance();
+            DocumentBuilder builder;
+            Document doc;
+            try {
+                builder = factory.newDocumentBuilder();
+                doc = builder.parse(hadoopPropertiesFile);
+                NodeList nl = doc.getElementsByTagName("property");
+                for (int i = 0; i < nl.getLength(); i++) {
+                    String name = doc.getElementsByTagName("name").item(i).getFirstChild().getNodeValue();
+                    String value = doc.getElementsByTagName("value").item(i).getFirstChild().getNodeValue();
+                    if (name.equals("tmpjars") == false) {
+                        sql.append("SET " + name + "=" + value + ";\n");
+                    }
+                }
+
+            } catch (ParserConfigurationException e) {
+                throw new IOException(e);
+            } catch (SAXException e) {
+                throw new IOException(e);
+            }
+        }
+
+        // hard coded below mr parameters to enable map-side join
+        sql.append("SET hive.exec.compress.output=true;" + "\n");
+        sql.append("SET hive.auto.convert.join.noconditionaltask = true;" + "\n");
+        sql.append("SET hive.auto.convert.join.noconditionaltask.size = 300000000;" + "\n");
+        sql.append("INSERT OVERWRITE TABLE " + intermediateTableDesc.getTableName(jobUUID) + "\n");
+
+        sql.append(generateSelectDataStatement(intermediateTableDesc));
+        sql.append(";");
+        return sql.toString();
+    }
+
+    public static String generateSelectDataStatement(JoinedFlatTableDesc intermediateTableDesc) {
+        StringBuilder sql = new StringBuilder();
+        sql.append("SELECT" + "\n");
+        for (int i = 0; i < intermediateTableDesc.getColumnList().size(); i++) {
+            IntermediateColumnDesc col = intermediateTableDesc.getColumnList().get(i);
+            if (i > 0) {
+                sql.append(",");
+            }
+            sql.append(col.getTableName() + "." + col.getColumnName() + "\n");
+        }
+        appendJoinStatement(intermediateTableDesc, sql);
+        appendWhereStatement(intermediateTableDesc, sql);
+        return sql.toString();
+    }
+
+    private static void appendJoinStatement(JoinedFlatTableDesc intermediateTableDesc, StringBuilder sql) {
+        Set<String> dimTableCache = new HashSet<String>();
+
+        CubeDesc cubeDesc = intermediateTableDesc.getCubeDesc();
+        String factTableName = cubeDesc.getFactTable();
+        sql.append("FROM " + factTableName + "\n");
+
+        for (DimensionDesc dim : cubeDesc.getDimensions()) {
+            JoinDesc join = dim.getJoin();
+            if (join != null && join.getType().equals("") == false) {
+                String joinType = join.getType().toUpperCase();
+                String dimTableName = dim.getTable();
+                if (!dimTableCache.contains(dimTableName)) {
+                    TblColRef[] pk = join.getPrimaryKeyColumns();
+                    TblColRef[] fk = join.getForeignKeyColumns();
+                    if (pk.length != fk.length) {
+                        throw new RuntimeException("Invalid join condition of dimension " + dim.getName());
+                    }
+                    sql.append(joinType + " JOIN " + dimTableName + "\n");
+                    sql.append("ON ");
+                    for (int i = 0; i < pk.length; i++) {
+                        if (i > 0) {
+                            sql.append(" AND ");
+                        }
+                        sql.append(factTableName + "." + fk[i].getName() + " = " + dimTableName + "." + pk[i].getName());
+                    }
+                    sql.append("\n");
+
+                    dimTableCache.add(dimTableName);
+                }
+            }
+        }
+    }
+
+    private static void appendWhereStatement(JoinedFlatTableDesc intermediateTableDesc, StringBuilder sql) {
+        boolean hasCondition = false;
+        StringBuilder whereBuilder = new StringBuilder();
+        whereBuilder.append("WHERE");
+
+        CubeDesc cubeDesc = intermediateTableDesc.getCubeDesc();
+
+        if (cubeDesc.getFilterCondition() != null && cubeDesc.getFilterCondition().equals("") == false) {
+            whereBuilder.append(" (").append(cubeDesc.getFilterCondition()).append(") ");
+            hasCondition = true;
+        }
+
+        CubeSegment cubeSegment = intermediateTableDesc.getCubeSegment();
+
+        if (null != cubeSegment) {
+            long dateStart = cubeSegment.getDateRangeStart();
+            long dateEnd = cubeSegment.getDateRangeEnd();
+
+            if (cubeSegment.getCubeInstance().needMergeImmediatelyAfterBuild(cubeSegment)) {
+                dateStart = cubeSegment.getCubeInstance().getDateRange()[1];
+            }
+            if (!(dateStart == 0 && dateEnd == 0)) {
+                String partitionColumnName = cubeDesc.getCubePartitionDesc().getPartitionDateColumn();
+
+                whereBuilder.append(hasCondition ? " AND (" : " (");
+                if (dateStart > 0) {
+                    whereBuilder.append(partitionColumnName + " >= '" + formatDateTimeInWhereClause(dateStart) + "' ");
+                    whereBuilder.append("AND ");
+                }
+                whereBuilder.append(partitionColumnName + " < '" + formatDateTimeInWhereClause(dateEnd) + "'");
+                whereBuilder.append(")\n");
+                hasCondition = true;
+            }
+        }
+
+        if (hasCondition) {
+            sql.append(whereBuilder.toString());
+        }
+    }
+
+    private static String formatDateTimeInWhereClause(long datetime) {
+        SimpleDateFormat f = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss");
+        f.setTimeZone(TimeZone.getTimeZone("GMT"));
+        Date date = new Date(datetime);
+        String str = f.format(date);
+        // note "2014-10-01" >= "2014-10-01 00:00:00" is FALSE
+        return StringUtil.dropSuffix(str, " 00:00:00");
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/job/src/main/java/com/kylinolap/job/cmd/ICommandOutput.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/com/kylinolap/job/cmd/ICommandOutput.java b/job/src/main/java/com/kylinolap/job/cmd/ICommandOutput.java
new file mode 100644
index 0000000..ca8cc08
--- /dev/null
+++ b/job/src/main/java/com/kylinolap/job/cmd/ICommandOutput.java
@@ -0,0 +1,41 @@
+/*
+ * 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.job.cmd;
+
+import com.kylinolap.job.constant.JobStepStatusEnum;
+
+/**
+ * @author xjiang
+ * 
+ */
+public interface ICommandOutput {
+
+    public void setStatus(JobStepStatusEnum status);
+
+    public JobStepStatusEnum getStatus();
+
+    public void appendOutput(String message);
+
+    public String getOutput();
+
+    public void setExitCode(int exitCode);
+
+    public int getExitCode();
+
+    public void reset();
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/job/src/main/java/com/kylinolap/job/cmd/IJobCommand.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/com/kylinolap/job/cmd/IJobCommand.java b/job/src/main/java/com/kylinolap/job/cmd/IJobCommand.java
new file mode 100644
index 0000000..eceada9
--- /dev/null
+++ b/job/src/main/java/com/kylinolap/job/cmd/IJobCommand.java
@@ -0,0 +1,30 @@
+/*
+ * 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.job.cmd;
+
+import com.kylinolap.job.exception.JobException;
+
+/**
+ * @author xjiang
+ * 
+ */
+public interface IJobCommand {
+
+    public ICommandOutput execute() throws JobException;
+
+    public void cancel() throws JobException;
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/job/src/main/java/com/kylinolap/job/cmd/JavaHadoopCmd.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/com/kylinolap/job/cmd/JavaHadoopCmd.java b/job/src/main/java/com/kylinolap/job/cmd/JavaHadoopCmd.java
new file mode 100644
index 0000000..12b4d2c
--- /dev/null
+++ b/job/src/main/java/com/kylinolap/job/cmd/JavaHadoopCmd.java
@@ -0,0 +1,73 @@
+/*
+ * 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.job.cmd;
+
+import org.apache.hadoop.util.ToolRunner;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.kylinolap.job.constant.JobStepStatusEnum;
+import com.kylinolap.job.engine.JobEngineConfig;
+import com.kylinolap.job.exception.JobException;
+import com.kylinolap.job.hadoop.AbstractHadoopJob;
+
+/**
+ * @author xduo
+ * 
+ */
+public class JavaHadoopCmd implements IJobCommand {
+    protected static final Logger log = LoggerFactory.getLogger(JavaHadoopCmd.class);
+
+    private final String executeCommand;
+    private final ICommandOutput output;
+    private final AbstractHadoopJob job;
+
+    public JavaHadoopCmd(String executeCommand, String jobInstanceID, int jobStepID, JobEngineConfig engineConfig, AbstractHadoopJob job, boolean isAsync) {
+        super();
+        this.executeCommand = executeCommand;
+        this.job = job;
+        this.output = new JavaHadoopCmdOutput(jobInstanceID, jobStepID, engineConfig, job, isAsync);
+    }
+
+    @Override
+    public ICommandOutput execute() throws JobException {
+        output.appendOutput("Start to execute command: \n" + this.executeCommand);
+        String[] args = executeCommand.trim().split("\\s+");
+
+        try {
+            output.setStatus(JobStepStatusEnum.RUNNING);
+            int exitCode = ToolRunner.run(job, args);
+            output.setExitCode(exitCode);
+        } catch (Exception e) {
+            output.appendOutput(e.getLocalizedMessage());
+            output.setExitCode(-1);
+        }
+
+        output.appendOutput("Command execute return code " + output.getExitCode());
+
+        if (output.getExitCode() != 0) {
+            output.setStatus(JobStepStatusEnum.ERROR);
+        }
+
+        return output;
+    }
+
+    @Override
+    public void cancel() throws JobException {
+        job.kill();
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/job/src/main/java/com/kylinolap/job/cmd/JavaHadoopCmdOutput.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/com/kylinolap/job/cmd/JavaHadoopCmdOutput.java b/job/src/main/java/com/kylinolap/job/cmd/JavaHadoopCmdOutput.java
new file mode 100644
index 0000000..20dd9dd
--- /dev/null
+++ b/job/src/main/java/com/kylinolap/job/cmd/JavaHadoopCmdOutput.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.job.cmd;
+
+import java.util.Map;
+
+import org.apache.hadoop.mapreduce.TaskCounter;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.kylinolap.common.KylinConfig;
+import com.kylinolap.job.JobDAO;
+import com.kylinolap.job.JobInstance;
+import com.kylinolap.job.JobInstance.JobStep;
+import com.kylinolap.job.constant.JobStepStatusEnum;
+import com.kylinolap.job.engine.JobEngineConfig;
+import com.kylinolap.job.hadoop.AbstractHadoopJob;
+import com.kylinolap.job.tools.HadoopStatusChecker;
+
+/**
+ * @author xduo
+ * 
+ */
+public class JavaHadoopCmdOutput implements ICommandOutput {
+
+    protected static final Logger log = LoggerFactory.getLogger(JavaHadoopCmdOutput.class);
+
+    protected StringBuilder output;
+    protected int exitCode;
+    protected JobStepStatusEnum status;
+    private final KylinConfig config;
+    private final String jobInstanceID;
+    private final int jobStepID;
+    private final String yarnUrl;
+    private final AbstractHadoopJob job;
+    private String mrJobID = null;
+    private String trackUrl = null;
+    private boolean isAsync;
+
+    public JavaHadoopCmdOutput(String jobInstanceID, int jobStepID, JobEngineConfig engineConfig, AbstractHadoopJob job, boolean isAsync) {
+        super();
+        this.config = engineConfig.getConfig();
+        this.yarnUrl = engineConfig.getYarnStatusServiceUrl();
+        this.jobInstanceID = jobInstanceID;
+        this.jobStepID = jobStepID;
+        this.job = job;
+        this.isAsync = isAsync;
+
+        init();
+    }
+
+    @Override
+    public void setStatus(JobStepStatusEnum status) {
+        this.status = status;
+    }
+
+    @Override
+    public JobStepStatusEnum getStatus() {
+        if (this.isAsync) {
+            if (this.status == JobStepStatusEnum.ERROR) {
+                return status;
+            }
+
+            if (null == this.mrJobID || null == this.trackUrl) {
+                updateHadoopJobInfo();
+            }
+
+            status = new HadoopStatusChecker(this.yarnUrl, this.mrJobID, output).checkStatus();
+
+            if (this.status.isComplete()) {
+                updateJobCounter();
+            }
+        } else {
+            status = (this.exitCode == 0) ? JobStepStatusEnum.FINISHED : JobStepStatusEnum.ERROR;
+        }
+
+        return status;
+    }
+
+    @Override
+    public void appendOutput(String message) {
+        log.debug(message);
+        output.append(message).append("\n");
+    }
+
+    @Override
+    public String getOutput() {
+        return output.toString();
+    }
+
+    @Override
+    public void setExitCode(int exitCode) {
+        this.exitCode = exitCode;
+    }
+
+    @Override
+    public int getExitCode() {
+        return exitCode;
+    }
+
+    @Override
+    public void reset() {
+        init();
+    }
+
+    private void init() {
+        output = new StringBuilder();
+        exitCode = -1;
+        status = JobStepStatusEnum.NEW;
+    }
+
+    /**
+     * @param jobStatus
+     */
+    private void updateHadoopJobInfo() {
+        try {
+            Map<String, String> jobInfo = job.getInfo();
+
+            JobDAO jobDAO = JobDAO.getInstance(config);
+            JobInstance jobInstance = jobDAO.getJob(jobInstanceID);
+            JobStep jobStep = jobInstance.getSteps().get(jobStepID);
+            boolean hasChange = false;
+
+            if (null == this.mrJobID && jobInfo.containsKey(JobInstance.MR_JOB_ID)) {
+                this.mrJobID = jobInfo.get(JobInstance.MR_JOB_ID);
+                jobStep.putInfo(JobInstance.MR_JOB_ID, this.mrJobID);
+                output.append("Get job id " + this.mrJobID).append("\n");
+                hasChange = true;
+            }
+
+            if (null == this.trackUrl && jobInfo.containsKey(JobInstance.YARN_APP_URL)) {
+                this.trackUrl = jobInfo.get(JobInstance.YARN_APP_URL);
+                jobStep.putInfo(JobInstance.YARN_APP_URL, this.trackUrl);
+                output.append("Get job track url " + this.trackUrl).append("\n");
+                hasChange = true;
+            }
+            if (hasChange) {
+                jobDAO.updateJobInstance(jobInstance);
+            }
+        } catch (Exception e) {
+            log.error(e.getLocalizedMessage(), e);
+            output.append(e.getLocalizedMessage());
+        }
+    }
+
+    private void updateJobCounter() {
+        try {
+            this.output.append(job.getCounters().toString()).append("\n");
+            log.debug(job.getCounters().toString());
+
+            JobDAO jobDAO = JobDAO.getInstance(config);
+            JobInstance jobInstance = jobDAO.getJob(jobInstanceID);
+            JobStep jobStep = jobInstance.getSteps().get(jobStepID);
+
+            long mapInputRecords = job.getCounters().findCounter(TaskCounter.MAP_INPUT_RECORDS).getValue();
+            jobStep.putInfo(JobInstance.SOURCE_RECORDS_COUNT, String.valueOf(mapInputRecords));
+            long hdfsBytesWritten = job.getCounters().findCounter("FileSystemCounters", "HDFS_BYTES_WRITTEN").getValue();
+            jobStep.putInfo(JobInstance.HDFS_BYTES_WRITTEN, String.valueOf(hdfsBytesWritten));
+
+            jobDAO.updateJobInstance(jobInstance);
+        } catch (Exception e) {
+            log.error(e.getLocalizedMessage(), e);
+            output.append(e.getLocalizedMessage());
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/job/src/main/java/com/kylinolap/job/cmd/JobCommandFactory.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/com/kylinolap/job/cmd/JobCommandFactory.java b/job/src/main/java/com/kylinolap/job/cmd/JobCommandFactory.java
new file mode 100644
index 0000000..39bda56
--- /dev/null
+++ b/job/src/main/java/com/kylinolap/job/cmd/JobCommandFactory.java
@@ -0,0 +1,97 @@
+/*
+ * 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.job.cmd;
+
+import com.kylinolap.job.JobInstance;
+import com.kylinolap.job.JobInstance.JobStep;
+import com.kylinolap.job.constant.JobStepCmdTypeEnum;
+import com.kylinolap.job.engine.JobEngineConfig;
+import com.kylinolap.job.hadoop.AbstractHadoopJob;
+import com.kylinolap.job.hadoop.cube.BaseCuboidMapper;
+import com.kylinolap.job.hadoop.cube.CubeHFileJob;
+import com.kylinolap.job.hadoop.cube.CuboidJob;
+import com.kylinolap.job.hadoop.cube.FactDistinctColumnsJob;
+import com.kylinolap.job.hadoop.cube.MergeCuboidJob;
+import com.kylinolap.job.hadoop.cube.NDCuboidMapper;
+import com.kylinolap.job.hadoop.cube.RangeKeyDistributionJob;
+import com.kylinolap.job.hadoop.dict.CreateDictionaryJob;
+import com.kylinolap.job.hadoop.hbase.BulkLoadJob;
+import com.kylinolap.job.hadoop.hbase.CreateHTableJob;
+
+/**
+ * @author xjiang
+ * 
+ */
+public class JobCommandFactory {
+
+    public static IJobCommand getJobCommand(String command, JobInstance jobInstance, int jobStepID, JobEngineConfig engineConfig) {
+        String instanceID = jobInstance.getUuid();
+
+        boolean isRemote = engineConfig.isRunAsRemoteCommand();
+        String hostname = isRemote ? engineConfig.getRemoteHadoopCliHostname() : null;
+        String username = isRemote ? engineConfig.getRemoteHadoopCliUsername() : null;
+        String password = isRemote ? engineConfig.getRemoteHadoopCliPassword() : null;
+        JobStep jobStep = jobInstance.getSteps().get(jobStepID);
+        boolean isAsync = jobStep.isRunAsync();
+        JobStepCmdTypeEnum type = jobStep.getCmdType();
+
+        switch (type) {
+        case SHELL_CMD_HADOOP:
+            return new ShellHadoopCmd(command, hostname, username, password, isAsync, instanceID, jobStepID, engineConfig);
+        case JAVA_CMD_HADOOP_FACTDISTINCT:
+            FactDistinctColumnsJob factDistinctJob = new FactDistinctColumnsJob();
+            factDistinctJob.setAsync(isAsync);
+            return new JavaHadoopCmd(command, instanceID, jobStepID, engineConfig, factDistinctJob, isAsync);
+        case JAVA_CMD_HADOOP_BASECUBOID:
+            CuboidJob baseCuboidJob = new CuboidJob();
+            baseCuboidJob.setAsync(isAsync);
+            baseCuboidJob.setMapperClass(BaseCuboidMapper.class);
+            return new JavaHadoopCmd(command, instanceID, jobStepID, engineConfig, baseCuboidJob, isAsync);
+        case JAVA_CMD_HADOOP_NDCUBOID:
+            CuboidJob ndCuboidJob = new CuboidJob();
+            ndCuboidJob.setAsync(isAsync);
+            ndCuboidJob.setMapperClass(NDCuboidMapper.class);
+            return new JavaHadoopCmd(command, instanceID, jobStepID, engineConfig, ndCuboidJob, isAsync);
+        case JAVA_CMD_HADOOP_RANGEKEYDISTRIBUTION:
+            AbstractHadoopJob rangeKeyDistributionJob = new RangeKeyDistributionJob();
+            rangeKeyDistributionJob.setAsync(isAsync);
+            return new JavaHadoopCmd(command, instanceID, jobStepID, engineConfig, rangeKeyDistributionJob, isAsync);
+        case JAVA_CMD_HADOOP_CONVERTHFILE:
+            CubeHFileJob cubeHFileJob = new CubeHFileJob();
+            cubeHFileJob.setAsync(isAsync);
+            return new JavaHadoopCmd(command, instanceID, jobStepID, engineConfig, cubeHFileJob, isAsync);
+        case JAVA_CMD_HADOOP_MERGECUBOID:
+            MergeCuboidJob mergeCuboidJob = new MergeCuboidJob();
+            mergeCuboidJob.setAsync(isAsync);
+            return new JavaHadoopCmd(command, instanceID, jobStepID, engineConfig, mergeCuboidJob, isAsync);
+        case JAVA_CMD_HADOOP_NO_MR_DICTIONARY:
+            CreateDictionaryJob createDictionaryJob = new CreateDictionaryJob();
+            createDictionaryJob.setAsync(isAsync);
+            return new JavaHadoopCmd(command, instanceID, jobStepID, engineConfig, createDictionaryJob, isAsync);
+        case JAVA_CMD_HADDOP_NO_MR_CREATEHTABLE:
+            CreateHTableJob createHTableJob = new CreateHTableJob();
+            createHTableJob.setAsync(isAsync);
+            return new JavaHadoopCmd(command, instanceID, jobStepID, engineConfig, createHTableJob, isAsync);
+        case JAVA_CMD_HADOOP_NO_MR_BULKLOAD:
+            BulkLoadJob bulkLoadJob = new BulkLoadJob();
+            bulkLoadJob.setAsync(isAsync);
+            return new JavaHadoopCmd(command, instanceID, jobStepID, engineConfig, bulkLoadJob, isAsync);
+        default:
+            return new ShellCmd(command, hostname, username, password, isAsync);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/job/src/main/java/com/kylinolap/job/cmd/ShellCmd.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/com/kylinolap/job/cmd/ShellCmd.java b/job/src/main/java/com/kylinolap/job/cmd/ShellCmd.java
new file mode 100644
index 0000000..6ecb85c
--- /dev/null
+++ b/job/src/main/java/com/kylinolap/job/cmd/ShellCmd.java
@@ -0,0 +1,236 @@
+/*
+ * 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.job.cmd;
+
+import java.io.BufferedReader;
+import java.io.File;
+import java.io.IOException;
+import java.io.InputStreamReader;
+import java.io.PipedInputStream;
+import java.io.PipedOutputStream;
+import java.util.concurrent.Callable;
+import java.util.concurrent.CancellationException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.FutureTask;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.jcraft.jsch.Channel;
+import com.jcraft.jsch.ChannelExec;
+import com.jcraft.jsch.JSch;
+import com.jcraft.jsch.Session;
+import com.kylinolap.job.constant.JobStepStatusEnum;
+import com.kylinolap.job.exception.JobException;
+
+/**
+ * @author xjiang
+ * 
+ * FIXME should reuse common.util.SSHClient
+ */
+public class ShellCmd implements IJobCommand {
+
+    private static Logger log = LoggerFactory.getLogger(ShellCmd.class);
+
+    private final String executeCommand;
+    private final ICommandOutput output;
+    private final String remoteHost;
+    private final String remoteUser;
+    private final String remotePassword;
+    private final String identityPath;
+    private final boolean isAsync;
+
+    private FutureTask<Integer> future;
+
+    protected ShellCmd(String executeCmd, ICommandOutput out, String host, String user, String password, boolean async) {
+        this.executeCommand = executeCmd;
+        this.output = out;
+        this.remoteHost = host;
+        this.remoteUser = user;
+        if (password != null && new File(password).exists()) {
+            this.identityPath = new File(password).getAbsolutePath();
+            this.remotePassword = null;
+        } else {
+            this.remotePassword = password;
+            this.identityPath = null;
+        }
+        this.isAsync = async;
+    }
+
+    public ShellCmd(String executeCmd, String host, String user, String password, boolean async) {
+        this(executeCmd, new ShellCmdOutput(), host, user, password, async);
+    }
+
+    @Override
+    public ICommandOutput execute() throws JobException {
+
+        final ExecutorService executor = Executors.newSingleThreadExecutor();
+        future = new FutureTask<Integer>(new Callable<Integer>() {
+            public Integer call() throws JobException {
+                executor.shutdown();
+                return executeCommand(executeCommand);
+            }
+        });
+        executor.execute(future);
+
+        int exitCode = -1;
+        if (!isAsync) {
+            try {
+                exitCode = future.get();
+                log.info("finish executing");
+            } catch (CancellationException e) {
+                log.debug("Command is cancelled");
+                exitCode = -2;
+            } catch (Exception e) {
+                throw new JobException("Error when exectute job " + executeCommand, e);
+            } finally {
+                if (exitCode == 0) {
+                    output.setStatus(JobStepStatusEnum.FINISHED);
+                } else if (exitCode == -2) {
+                    output.setStatus(JobStepStatusEnum.DISCARDED);
+                } else {
+                    output.setStatus(JobStepStatusEnum.ERROR);
+                }
+                output.setExitCode(exitCode);
+            }
+        }
+        return output;
+    }
+
+    protected int executeCommand(String command) throws JobException {
+        output.reset();
+        if (remoteHost != null) {
+            log.debug("Executing remote cmd: " + command);
+            return remoteExec(command);
+        } else {
+            log.debug("Executing local cmd: " + command);
+            return localExec(command);
+        }
+    }
+
+    private int localExec(String command) throws JobException {
+        output.setStatus(JobStepStatusEnum.RUNNING);
+        String[] cmd = new String[3];
+        cmd[0] = "/bin/bash";
+        cmd[1] = "-c";
+        cmd[2] = command;
+
+        BufferedReader reader = null;
+        int exitCode = -1;
+        try {
+            ProcessBuilder builder = new ProcessBuilder(cmd);
+            builder.redirectErrorStream(true);
+            Process proc = builder.start();
+
+            reader = new BufferedReader(new InputStreamReader(proc.getInputStream()));
+            String line = null;
+            while ((line = reader.readLine()) != null) {
+                output.appendOutput(line);
+            }
+
+            exitCode = proc.waitFor();
+        } catch (Exception e) {
+            throw new JobException(e);
+        } finally {
+            if (reader != null) {
+                try {
+                    reader.close();
+                } catch (IOException e) {
+                    throw new JobException(e);
+                }
+            }
+        }
+        return exitCode;
+    }
+
+    private int remoteExec(String command) throws JobException {
+        output.setStatus(JobStepStatusEnum.RUNNING);
+        Session session = null;
+        Channel channel = null;
+        int exitCode = -1;
+        try {
+            JSch jsch = new JSch();
+            if (identityPath != null) {
+                jsch.addIdentity(identityPath);
+            }
+
+            session = jsch.getSession(remoteUser, remoteHost, 22);
+            if (remotePassword != null) {
+                session.setPassword(remotePassword);
+            }
+            session.setConfig("StrictHostKeyChecking", "no");
+            session.connect();
+
+            channel = session.openChannel("exec");
+            ((ChannelExec) channel).setCommand(command);
+            channel.setInputStream(null);
+            PipedInputStream in = new PipedInputStream(64 * 1024);
+            PipedOutputStream out = new PipedOutputStream(in);
+            channel.setOutputStream(out);
+            ((ChannelExec) channel).setErrStream(out); // redirect error to out
+            channel.connect();
+
+            byte[] tmp = new byte[1024];
+            while (true) {
+                while (in.available() > 0) {
+                    int i = in.read(tmp, 0, 1024);
+                    if (i < 0)
+                        break;
+                    output.appendOutput(new String(tmp, 0, i));
+                }
+                if (channel.isClosed()) {
+                    if (in.available() > 0) {
+                        continue;
+                    }
+                    exitCode = channel.getExitStatus();
+                    break;
+                }
+                try {
+                    Thread.sleep(1000);
+                } catch (Exception ee) {
+                    throw ee;
+                }
+            }
+        } catch (Exception e) {
+            throw new JobException(e);
+        } finally {
+            if (channel != null) {
+                channel.disconnect();
+            }
+            if (session != null) {
+                session.disconnect();
+            }
+        }
+        return exitCode;
+    }
+
+    @Override
+    public void cancel() throws JobException {
+        future.cancel(true);
+    }
+
+    public static void main(String[] args) throws JobException {
+        ShellCmdOutput output = new ShellCmdOutput();
+        ShellCmd shellCmd = new ShellCmd(args[0], output, args[1], args[2], args[3], false);
+        shellCmd.execute();
+        
+        System.out.println("============================================================================");
+        System.out.println(output.getExitCode());
+        System.out.println(output.getOutput());
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/job/src/main/java/com/kylinolap/job/cmd/ShellCmdOutput.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/com/kylinolap/job/cmd/ShellCmdOutput.java b/job/src/main/java/com/kylinolap/job/cmd/ShellCmdOutput.java
new file mode 100644
index 0000000..7197283
--- /dev/null
+++ b/job/src/main/java/com/kylinolap/job/cmd/ShellCmdOutput.java
@@ -0,0 +1,82 @@
+/*
+ * 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.job.cmd;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.kylinolap.job.constant.JobStepStatusEnum;
+
+/**
+ * @author xjiang
+ * 
+ */
+public class ShellCmdOutput implements ICommandOutput {
+
+    protected static final Logger log = LoggerFactory.getLogger(ShellCmdOutput.class);
+
+    protected StringBuilder output;
+    protected int exitCode;
+    protected JobStepStatusEnum status;
+
+    public ShellCmdOutput() {
+        init();
+    }
+
+    private void init() {
+        output = new StringBuilder();
+        exitCode = -1;
+        status = JobStepStatusEnum.NEW;
+    }
+
+    @Override
+    public JobStepStatusEnum getStatus() {
+        return status;
+    }
+
+    @Override
+    public void setStatus(JobStepStatusEnum s) {
+        this.status = s;
+    }
+
+    @Override
+    public String getOutput() {
+        return output.toString();
+    }
+
+    @Override
+    public void appendOutput(String message) {
+        output.append(message).append(System.getProperty("line.separator"));
+        log.debug(message);
+    }
+
+    @Override
+    public int getExitCode() {
+        return exitCode;
+    }
+
+    @Override
+    public void setExitCode(int code) {
+        exitCode = code;
+    }
+
+    @Override
+    public void reset() {
+        init();
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/job/src/main/java/com/kylinolap/job/cmd/ShellHadoopCmd.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/com/kylinolap/job/cmd/ShellHadoopCmd.java b/job/src/main/java/com/kylinolap/job/cmd/ShellHadoopCmd.java
new file mode 100644
index 0000000..f0b7a4e
--- /dev/null
+++ b/job/src/main/java/com/kylinolap/job/cmd/ShellHadoopCmd.java
@@ -0,0 +1,60 @@
+/*
+ * 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.job.cmd;
+
+import java.io.IOException;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.kylinolap.job.JobDAO;
+import com.kylinolap.job.JobInstance;
+import com.kylinolap.job.engine.JobEngineConfig;
+import com.kylinolap.job.exception.JobException;
+
+/**
+ * @author xjiang
+ * 
+ */
+public class ShellHadoopCmd extends ShellCmd {
+    private static Logger log = LoggerFactory.getLogger(ShellHadoopCmd.class);
+
+    private final String jobInstanceID;
+    private final int jobStepID;
+    private final JobEngineConfig engineConfig;
+
+    public ShellHadoopCmd(String executeCmd, String host, String user, String password, boolean async, String instanceID, int stepID, JobEngineConfig engineConfig) {
+        super(executeCmd, new ShellHadoopCmdOutput(instanceID, stepID, engineConfig), host, user, password, async);
+        this.jobInstanceID = instanceID;
+        this.jobStepID = stepID;
+        this.engineConfig = engineConfig;
+    }
+
+    @Override
+    public void cancel() throws JobException {
+        JobDAO jobDAO = JobDAO.getInstance(engineConfig.getConfig());
+        JobInstance jobInstance = null;
+        try {
+            jobInstance = jobDAO.getJob(jobInstanceID);
+            String mrJobId = jobInstance.getSteps().get(jobStepID).getInfo(JobInstance.MR_JOB_ID);
+            log.debug("kill MR job " + mrJobId);
+            executeCommand("hadoop job -kill " + mrJobId);
+        } catch (IOException e) {
+            throw new JobException(e);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/job/src/main/java/com/kylinolap/job/cmd/ShellHadoopCmdOutput.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/com/kylinolap/job/cmd/ShellHadoopCmdOutput.java b/job/src/main/java/com/kylinolap/job/cmd/ShellHadoopCmdOutput.java
new file mode 100644
index 0000000..6672862
--- /dev/null
+++ b/job/src/main/java/com/kylinolap/job/cmd/ShellHadoopCmdOutput.java
@@ -0,0 +1,161 @@
+/*
+ * 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.job.cmd;
+
+import java.io.IOException;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+
+import com.kylinolap.common.KylinConfig;
+import com.kylinolap.job.JobDAO;
+import com.kylinolap.job.JobInstance;
+import com.kylinolap.job.JobInstance.JobStep;
+import com.kylinolap.job.constant.JobStepStatusEnum;
+import com.kylinolap.job.engine.JobEngineConfig;
+import com.kylinolap.job.tools.HadoopStatusChecker;
+
+/**
+ * @author xjiang
+ * 
+ */
+public class ShellHadoopCmdOutput extends ShellCmdOutput {
+
+    // mr job
+    private static final Pattern PATTERN_APP_ID = Pattern.compile("Submitted application (.*?) to ResourceManager");
+    private static final Pattern PATTERN_APP_URL = Pattern.compile("The url to track the job: (.*)");
+    private static final Pattern PATTERN_JOB_ID = Pattern.compile("Running job: (.*)");
+    private static final Pattern PATTERN_HDFS_BYTES_WRITTEN = Pattern.compile("HDFS: Number of bytes written=(\\d+)");
+    private static final Pattern PATTERN_SOURCE_RECORDS_COUNT = Pattern.compile("Map input records=(\\d+)");
+    private static final Pattern PATTERN_SOURCE_RECORDS_SIZE = Pattern.compile("HDFS Read: (\\d+) HDFS Write");
+
+    // hive
+    private static final Pattern PATTERN_HIVE_APP_ID_URL = Pattern.compile("Starting Job = (.*?), Tracking URL = (.*)");
+    private static final Pattern PATTERN_HIVE_BYTES_WRITTEN = Pattern.compile("HDFS Read: (\\d+) HDFS Write: (\\d+) SUCCESS");
+
+    private final KylinConfig config;
+    private final String jobInstanceID;
+    private final int jobStepID;
+    private final String yarnUrl;
+
+    private String mrJobID;
+
+    public ShellHadoopCmdOutput(String jobInstanceID, int jobStepID, JobEngineConfig engineConfig) {
+        super();
+        this.config = engineConfig.getConfig();
+        this.yarnUrl = engineConfig.getYarnStatusServiceUrl();
+        this.jobInstanceID = jobInstanceID;
+        this.jobStepID = jobStepID;
+        this.mrJobID = null;
+    }
+
+    @Override
+    public JobStepStatusEnum getStatus() {
+        if (status.isComplete()) {
+            return status;
+        }
+
+        if (null == mrJobID || mrJobID.trim().length() == 0) {
+            return JobStepStatusEnum.WAITING;
+        }
+
+        HadoopStatusChecker hadoopJobChecker = new HadoopStatusChecker(this.yarnUrl, this.mrJobID, output);
+        status = hadoopJobChecker.checkStatus();
+
+        return status;
+    }
+
+    @Override
+    public void appendOutput(String message) {
+        super.appendOutput(message);
+        try {
+            updateJobStepInfo(message);
+        } catch (IOException e) {
+            log.error("Failed to append output!\n" + message, e);
+            status = JobStepStatusEnum.ERROR;
+        }
+
+    }
+
+    private void updateJobStepInfo(final String message) throws IOException {
+
+        JobDAO jobDAO = JobDAO.getInstance(config);
+        JobInstance jobInstance = jobDAO.getJob(jobInstanceID);
+        JobStep jobStep = jobInstance.getSteps().get(jobStepID);
+
+        Matcher matcher = PATTERN_APP_ID.matcher(message);
+        if (matcher.find()) {
+            String appId = matcher.group(1);
+            jobStep.putInfo(JobInstance.YARN_APP_ID, appId);
+            jobDAO.updateJobInstance(jobInstance);
+        }
+
+        matcher = PATTERN_APP_URL.matcher(message);
+        if (matcher.find()) {
+            String appTrackingUrl = matcher.group(1);
+            jobStep.putInfo(JobInstance.YARN_APP_URL, appTrackingUrl);
+            jobDAO.updateJobInstance(jobInstance);
+        }
+
+        matcher = PATTERN_JOB_ID.matcher(message);
+        if (matcher.find()) {
+            String mrJobID = matcher.group(1);
+            jobStep.putInfo(JobInstance.MR_JOB_ID, mrJobID);
+            jobDAO.updateJobInstance(jobInstance);
+            this.mrJobID = mrJobID;
+            log.debug("Get hadoop job id " + mrJobID);
+        }
+
+        matcher = PATTERN_HDFS_BYTES_WRITTEN.matcher(message);
+        if (matcher.find()) {
+            String hdfsWritten = matcher.group(1);
+            jobStep.putInfo(JobInstance.HDFS_BYTES_WRITTEN, hdfsWritten);
+            jobDAO.updateJobInstance(jobInstance);
+        }
+
+        matcher = PATTERN_SOURCE_RECORDS_COUNT.matcher(message);
+        if (matcher.find()) {
+            String sourceCount = matcher.group(1);
+            jobStep.putInfo(JobInstance.SOURCE_RECORDS_COUNT, sourceCount);
+            jobDAO.updateJobInstance(jobInstance);
+        }
+
+        matcher = PATTERN_SOURCE_RECORDS_SIZE.matcher(message);
+        if (matcher.find()) {
+            String sourceSize = matcher.group(1);
+            jobStep.putInfo(JobInstance.SOURCE_RECORDS_SIZE, sourceSize);
+            jobDAO.updateJobInstance(jobInstance);
+        }
+
+        // hive
+        matcher = PATTERN_HIVE_APP_ID_URL.matcher(message);
+        if (matcher.find()) {
+            String jobId = matcher.group(1);
+            String trackingUrl = matcher.group(2);
+            jobStep.putInfo(JobInstance.MR_JOB_ID, jobId);
+            jobStep.putInfo(JobInstance.YARN_APP_URL, trackingUrl);
+            jobDAO.updateJobInstance(jobInstance);
+        }
+
+        matcher = PATTERN_HIVE_BYTES_WRITTEN.matcher(message);
+        if (matcher.find()) {
+            // String hdfsRead = matcher.group(1);
+            String hdfsWritten = matcher.group(2);
+            jobStep.putInfo(JobInstance.HDFS_BYTES_WRITTEN, hdfsWritten);
+            jobDAO.updateJobInstance(jobInstance);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/job/src/main/java/com/kylinolap/job/constant/BatchConstants.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/com/kylinolap/job/constant/BatchConstants.java b/job/src/main/java/com/kylinolap/job/constant/BatchConstants.java
new file mode 100644
index 0000000..b1dab02
--- /dev/null
+++ b/job/src/main/java/com/kylinolap/job/constant/BatchConstants.java
@@ -0,0 +1,46 @@
+/*
+ * 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.job.constant;
+
+/**
+ * @author George Song (ysong1)
+ * 
+ */
+public interface BatchConstants {
+
+    public static final char INTERMEDIATE_TABLE_ROW_DELIMITER = 127;
+
+    public static final String CFG_CUBE_NAME = "cube.name";
+    public static final String CFG_CUBE_SEGMENT_NAME = "cube.segment.name";
+
+    public static final String INPUT_DELIM = "input.delim";
+
+    public static final String OUTPUT_PATH = "output.path";
+
+    public static final String TABLE_NAME = "table.name";
+    public static final String TABLE_COLUMNS = "table.columns";
+
+    public static final String CFG_CUBE_INTERMEDIATE_TABLE_ROW_DELIMITER = "cube.intermediate.table.row.delimiter";
+
+    public static final String MAPREDUCE_COUTNER_GROUP_NAME = "Cube Builder";
+
+    public static final String MAPPER_SAMPLE_NUMBER = "mapper.sample.number";
+    public static final String REGION_NUMBER = "region.number";
+    public static final String CUBE_CAPACITY = "cube.capacity";
+
+    public static final int COUNTER_MAX = 100000;
+    public static final int ERROR_RECORD_THRESHOLD = 10;
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/job/src/main/java/com/kylinolap/job/constant/JobConstants.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/com/kylinolap/job/constant/JobConstants.java b/job/src/main/java/com/kylinolap/job/constant/JobConstants.java
new file mode 100644
index 0000000..21d099b
--- /dev/null
+++ b/job/src/main/java/com/kylinolap/job/constant/JobConstants.java
@@ -0,0 +1,57 @@
+/*
+ * 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.job.constant;
+
+/**
+ * @author ysong1, xduo
+ * 
+ */
+public class JobConstants {
+
+    public static final String GLOBAL_LISTENER_NAME = "ChainListener";
+
+    public static final int DEFAULT_SCHEDULER_INTERVAL_SECONDS = 60;
+
+    public static final String CUBE_JOB_GROUP_NAME = "cube_job_group";
+    public static final String DAEMON_JOB_GROUP_NAME = "daemon_job_group";
+
+    public static final String STEP_NAME_BUILD_DICTIONARY = "Build Dimension Dictionary";
+    public static final String STEP_NAME_CREATE_FLAT_HIVE_TABLE = "Create Intermediate Flat Hive Table";
+    public static final String STEP_NAME_FACT_DISTINCT_COLUMNS = "Extract Fact Table Distinct Columns";
+    public static final String STEP_NAME_BUILD_BASE_CUBOID = "Build Base Cuboid Data";
+    public static final String STEP_NAME_BUILD_N_D_CUBOID = "Build N-Dimension Cuboid Data";
+    public static final String STEP_NAME_GET_CUBOID_KEY_DISTRIBUTION = "Calculate HTable Region Splits";
+    public static final String STEP_NAME_CREATE_HBASE_TABLE = "Create HTable";
+    public static final String STEP_NAME_CONVERT_CUBOID_TO_HFILE = "Convert Cuboid Data to HFile";
+    public static final String STEP_NAME_BULK_LOAD_HFILE = "Load HFile to HBase Table";
+    public static final String STEP_NAME_MERGE_CUBOID = "Merge Cuboid Data";
+
+    public static final String PROP_ENGINE_CONTEXT = "jobengineConfig";
+    public static final String PROP_JOB_FLOW = "jobFlow";
+    public static final String PROP_JOBINSTANCE_UUID = "jobInstanceUuid";
+    public static final String PROP_JOBSTEP_SEQ_ID = "jobStepSequenceID";
+    public static final String PROP_COMMAND = "command";
+    // public static final String PROP_STORAGE_LOCATION =
+    // "storageLocationIdentifier";
+    public static final String PROP_JOB_ASYNC = "jobAsync";
+    public static final String PROP_JOB_CMD_EXECUTOR = "jobCmdExecutor";
+    public static final String PROP_JOB_CMD_OUTPUT = "jobCmdOutput";
+    public static final String PROP_JOB_KILLED = "jobKilled";
+    public static final String PROP_JOB_RUNTIME_FLOWS = "jobFlows";
+
+    public static final String NOTIFY_EMAIL_TEMPLATE = "<div><b>Build Result of Job ${job_name}</b><pre><ul>" + "<li>Build Result: <b>${result}</b></li>" + "<li>Job Engine: ${job_engine}</li>" + "<li>Cube Name: ${cube_name}</li>" + "<li>Start Time: ${start_time}</li>" + "<li>Duration: ${duration}</li>" + "<li>MR Waiting: ${mr_waiting}</li>" + "<li>Last Update Time: ${last_update_time}</li>" + "<li>Submitter: ${submitter}</li>" + "<li>Error Log: ${error_log}</li>" + "</ul></pre><div/>";
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/job/src/main/java/com/kylinolap/job/constant/JobStatusEnum.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/com/kylinolap/job/constant/JobStatusEnum.java b/job/src/main/java/com/kylinolap/job/constant/JobStatusEnum.java
new file mode 100644
index 0000000..bf67d0a
--- /dev/null
+++ b/job/src/main/java/com/kylinolap/job/constant/JobStatusEnum.java
@@ -0,0 +1,46 @@
+/*
+ * 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.job.constant;
+
+public enum JobStatusEnum {
+
+    NEW(0), PENDING(1), RUNNING(2), FINISHED(4), ERROR(8), DISCARDED(16);
+
+    private final int code;
+
+    private JobStatusEnum(int statusCode) {
+        this.code = statusCode;
+    }
+
+    public static JobStatusEnum getByCode(int statusCode) {
+        for (JobStatusEnum status : values()) {
+            if (status.getCode() == statusCode) {
+                return status;
+            }
+        }
+
+        return null;
+    }
+
+    public int getCode() {
+        return this.code;
+    }
+
+    public boolean isComplete() {
+        return code == JobStatusEnum.FINISHED.getCode() || code == JobStatusEnum.ERROR.getCode() || code == JobStatusEnum.DISCARDED.getCode();
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/job/src/main/java/com/kylinolap/job/constant/JobStepCmdTypeEnum.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/com/kylinolap/job/constant/JobStepCmdTypeEnum.java b/job/src/main/java/com/kylinolap/job/constant/JobStepCmdTypeEnum.java
new file mode 100644
index 0000000..fde17d0
--- /dev/null
+++ b/job/src/main/java/com/kylinolap/job/constant/JobStepCmdTypeEnum.java
@@ -0,0 +1,25 @@
+/*
+ * 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.job.constant;
+
+/**
+ * @author xduo, ysong1
+ * 
+ */
+public enum JobStepCmdTypeEnum {
+    SHELL_CMD, SHELL_CMD_HADOOP, JAVA_CMD_HADOOP_FACTDISTINCT, JAVA_CMD_HADOOP_BASECUBOID, JAVA_CMD_HADOOP_NDCUBOID, JAVA_CMD_HADOOP_RANGEKEYDISTRIBUTION, JAVA_CMD_HADOOP_CONVERTHFILE, JAVA_CMD_HADOOP_MERGECUBOID, JAVA_CMD_HADOOP_NO_MR_DICTIONARY, JAVA_CMD_HADDOP_NO_MR_CREATEHTABLE, JAVA_CMD_HADOOP_NO_MR_BULKLOAD
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/job/src/main/java/com/kylinolap/job/constant/JobStepStatusEnum.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/com/kylinolap/job/constant/JobStepStatusEnum.java b/job/src/main/java/com/kylinolap/job/constant/JobStepStatusEnum.java
new file mode 100644
index 0000000..debfb3c
--- /dev/null
+++ b/job/src/main/java/com/kylinolap/job/constant/JobStepStatusEnum.java
@@ -0,0 +1,48 @@
+/*
+ * 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.job.constant;
+
+public enum JobStepStatusEnum {
+    NEW(0), PENDING(1), RUNNING(2), FINISHED(4), ERROR(8), DISCARDED(16), WAITING(32);
+
+    private final int code;
+
+    private JobStepStatusEnum(int statusCode) {
+        this.code = statusCode;
+    }
+
+    public static JobStepStatusEnum getByCode(int statusCode) {
+        for (JobStepStatusEnum status : values()) {
+            if (status.getCode() == statusCode) {
+                return status;
+            }
+        }
+
+        return null;
+    }
+
+    public int getCode() {
+        return this.code;
+    }
+
+    public boolean isComplete() {
+        return code == JobStepStatusEnum.FINISHED.getCode() || code == JobStepStatusEnum.ERROR.getCode() || code == JobStepStatusEnum.DISCARDED.getCode();
+    }
+
+    public boolean isRunable() {
+        return code == JobStepStatusEnum.PENDING.getCode() || code == JobStepStatusEnum.ERROR.getCode();
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/job/src/main/java/com/kylinolap/job/constant/SchedulerTypeEnum.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/com/kylinolap/job/constant/SchedulerTypeEnum.java b/job/src/main/java/com/kylinolap/job/constant/SchedulerTypeEnum.java
new file mode 100644
index 0000000..8078023
--- /dev/null
+++ b/job/src/main/java/com/kylinolap/job/constant/SchedulerTypeEnum.java
@@ -0,0 +1,25 @@
+/*
+ * 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.job.constant;
+
+/**
+ * @author xduo
+ * 
+ */
+public enum SchedulerTypeEnum {
+    QUATZ;
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/job/src/main/java/com/kylinolap/job/deployment/DeploymentUtilityChecker.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/com/kylinolap/job/deployment/DeploymentUtilityChecker.java b/job/src/main/java/com/kylinolap/job/deployment/DeploymentUtilityChecker.java
new file mode 100644
index 0000000..00333bc
--- /dev/null
+++ b/job/src/main/java/com/kylinolap/job/deployment/DeploymentUtilityChecker.java
@@ -0,0 +1,21 @@
+package com.kylinolap.job.deployment;
+
+/**
+ * Created by honma on 9/29/14.
+ */
+public enum DeploymentUtilityChecker {
+    HIVE_CHECKER {
+        @Override
+        boolean isOkay() {
+            return true;
+        }
+    },
+    HBASE_CHECKER {
+        @Override
+        boolean isOkay() {
+            return true;
+        }
+    };
+
+    abstract boolean isOkay();
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/job/src/main/java/com/kylinolap/job/deployment/HbaseConfigPrinter.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/com/kylinolap/job/deployment/HbaseConfigPrinter.java b/job/src/main/java/com/kylinolap/job/deployment/HbaseConfigPrinter.java
new file mode 100644
index 0000000..ddf6251
--- /dev/null
+++ b/job/src/main/java/com/kylinolap/job/deployment/HbaseConfigPrinter.java
@@ -0,0 +1,131 @@
+package com.kylinolap.job.deployment;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.Map;
+
+import org.apache.commons.io.FileUtils;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.HBaseConfiguration;
+import org.apache.hadoop.hbase.HConstants;
+
+import com.kylinolap.job.tools.LZOSupportnessChecker;
+
+/**
+ * Created by honma on 9/30/14.
+ * <p/>
+ * This class is assumed to be run by
+ * "hbase org.apache.hadoop.util.RunJar kylin-job-0.5.7-SNAPSHOT-job.jar com.kylinolap.job.deployment.HadoopConfigPrinter "
+ * in the shell, so that hbase and hadoop related environment variables will be
+ * visible to this class.
+ */
+public class HbaseConfigPrinter {
+    public static void main(String[] args) throws IOException {
+        if (args.length != 1) {
+            System.out.println("Usage: hbase org.apache.hadoop.util.RunJar kylin-job-0.5.7-SNAPSHOT-job.jar com.kylinolap.job.deployment.HadoopConfigPrinter targetFile");
+            System.exit(1);
+        }
+
+        printConfigs(args[0]);
+    }
+
+    private static void printConfigs(String targetFile) throws IOException {
+
+        File output = new File(targetFile);
+        if (output.exists() && output.isDirectory()) {
+            throw new IllegalStateException("The output file: " + targetFile + " is a directory");
+        }
+
+        StringBuilder sb = new StringBuilder();
+
+        sb.append("export KYLIN_LZO_SUPPORTED=" + ConfigLoader.LZO_INFO_LOADER.loadValue() + "\n");
+        sb.append("export KYLIN_LD_LIBRARY_PATH=" + ConfigLoader.LD_LIBRARY_PATH_LOADER.loadValue() + "\n");
+        sb.append("export KYLIN_HBASE_CLASSPATH=" + ConfigLoader.HBASE_CLASSPATH_LOADER.loadValue() + "\n");
+        sb.append("export KYLIN_HBASE_CONF_PATH=" + ConfigLoader.HBASE_CONF_FOLDER_LOADER.loadValue() + "\n");
+        sb.append("export KYLIN_ZOOKEEPER_QUORUM=" + ConfigLoader.ZOOKEEP_QUORUM_LOADER.loadValue() + "\n");
+        sb.append("export KYLIN_ZOOKEEPER_CLIENT_PORT=" + ConfigLoader.ZOOKEEPER_CLIENT_PORT_LOADER.loadValue() + "\n");
+        sb.append("export KYLIN_ZOOKEEPER_ZNODE_PARENT=" + ConfigLoader.ZOOKEEPER_ZNODE_PARENT_LOADER.loadValue() + "\n");
+
+        FileUtils.writeStringToFile(output, sb.toString());
+    }
+
+    @SuppressWarnings("unused")
+    private static void printAllEnv() {
+        for (Map.Entry<String, String> entry : System.getenv().entrySet()) {
+            System.out.println("Key: " + entry.getKey());
+            System.out.println("Value: " + entry.getValue());
+            System.out.println();
+        }
+    }
+
+    enum ConfigLoader {
+
+        LZO_INFO_LOADER {
+            @Override
+            public String loadValue() {
+                return LZOSupportnessChecker.getSupportness() ? "true" : "false";
+            }
+        },
+
+        ZOOKEEP_QUORUM_LOADER {
+            @Override
+            public String loadValue() {
+                Configuration conf = HBaseConfiguration.create();
+                return conf.get(HConstants.ZOOKEEPER_QUORUM);
+            }
+        },
+
+        ZOOKEEPER_ZNODE_PARENT_LOADER {
+            @Override
+            public String loadValue() {
+                Configuration conf = HBaseConfiguration.create();
+                return conf.get(HConstants.ZOOKEEPER_ZNODE_PARENT);
+            }
+        },
+
+        ZOOKEEPER_CLIENT_PORT_LOADER {
+            @Override
+            public String loadValue() {
+                Configuration conf = HBaseConfiguration.create();
+                return conf.get(HConstants.ZOOKEEPER_CLIENT_PORT);
+
+            }
+        },
+
+        LD_LIBRARY_PATH_LOADER {
+            @Override
+            public String loadValue() {
+                return System.getenv("LD_LIBRARY_PATH");
+            }
+        },
+
+        HBASE_CLASSPATH_LOADER {
+            @Override
+            public String loadValue() {
+                return System.getenv("CLASSPATH");
+            }
+        },
+
+        HBASE_CONF_FOLDER_LOADER {
+            @Override
+            public String loadValue() {
+                String output = HBASE_CLASSPATH_LOADER.loadValue();
+                String[] paths = output.split(":");
+                StringBuilder sb = new StringBuilder();
+
+                for (String path : paths) {
+                    path = path.trim();
+                    File f = new File(path);
+                    if (StringUtils.containsIgnoreCase(path, "conf") && f.exists() && f.isDirectory() && f.getName().equalsIgnoreCase("conf")) {
+                        sb.append(":" + path);
+                    }
+                }
+                return sb.toString();
+            }
+        };
+
+        public abstract String loadValue();
+    }
+
+}


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

Posted by lu...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/cube/src/test/java/com/kylinolap/cube/kv/RowKeyEncoderTest.java
----------------------------------------------------------------------
diff --git a/cube/src/test/java/com/kylinolap/cube/kv/RowKeyEncoderTest.java b/cube/src/test/java/com/kylinolap/cube/kv/RowKeyEncoderTest.java
new file mode 100644
index 0000000..a89e04f
--- /dev/null
+++ b/cube/src/test/java/com/kylinolap/cube/kv/RowKeyEncoderTest.java
@@ -0,0 +1,143 @@
+/*
+ * 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.cube.kv;
+
+import static org.junit.Assert.*;
+
+import java.util.Arrays;
+
+import org.apache.hadoop.hbase.util.Bytes;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+import com.kylinolap.common.util.LocalFileMetadataTestCase;
+import com.kylinolap.cube.CubeInstance;
+import com.kylinolap.cube.CubeManager;
+import com.kylinolap.cube.cuboid.Cuboid;
+import com.kylinolap.metadata.MetadataManager;
+import com.kylinolap.metadata.model.cube.CubeDesc;
+
+/**
+ * @author George Song (ysong1)
+ * 
+ */
+public class RowKeyEncoderTest extends LocalFileMetadataTestCase {
+
+    @Before
+    public void setUp() throws Exception {
+        this.createTestMetadata();
+        MetadataManager.removeInstance(this.getTestConfig());
+    }
+
+    @After
+    public void after() throws Exception {
+        this.cleanupTestMetadata();
+    }
+
+    @Test
+    public void testEncodeWithoutSlr() throws Exception {
+        CubeInstance cube = CubeManager.getInstance(this.getTestConfig()).getCube("TEST_KYLIN_CUBE_WITHOUT_SLR_READY");
+        // CubeSegment seg = cube.getTheOnlySegment();
+        CubeDesc cubeDesc = cube.getDescriptor();
+        // String data =
+        // "2013-08-18Abbigliamento e accessoriDonna: AccessoriSciarpFoulard e ScialliAuctionItalyRegular";
+        byte[][] data = new byte[8][];
+        data[0] = Bytes.toBytes("2012-12-15");
+        data[1] = Bytes.toBytes("11848");
+        data[2] = Bytes.toBytes("Health & Beauty");
+        data[3] = Bytes.toBytes("Fragrances");
+        data[4] = Bytes.toBytes("Women");
+        data[5] = Bytes.toBytes("FP-GTC");
+        data[6] = Bytes.toBytes("0");
+        data[7] = Bytes.toBytes("15");
+
+        long baseCuboidId = Cuboid.getBaseCuboidId(cubeDesc);
+        Cuboid baseCuboid = Cuboid.findById(cubeDesc, baseCuboidId);
+        AbstractRowKeyEncoder rowKeyEncoder = AbstractRowKeyEncoder.createInstance(cube.getFirstSegment(), baseCuboid);
+
+        byte[] encodedKey = rowKeyEncoder.encode(data);
+        assertEquals(30, encodedKey.length);
+        byte[] cuboidId = Arrays.copyOfRange(encodedKey, 0, 8);
+        byte[] rest = Arrays.copyOfRange(encodedKey, 8, encodedKey.length);
+        assertEquals(255, Bytes.toLong(cuboidId));
+        assertArrayEquals(new byte[] { 11, 55, -13, 13, 22, 34, 121, 70, 80, 45, 71, 84, 67, 9, 9, 9, 9, 9, 9, 0, 10, 5 }, rest);
+    }
+
+    @Test
+    public void testEncodeWithSlr() throws Exception {
+        CubeInstance cube = CubeManager.getInstance(this.getTestConfig()).getCube("TEST_KYLIN_CUBE_WITH_SLR_READY");
+        // CubeSegment seg = cube.getTheOnlySegment();
+        CubeDesc cubeDesc = cube.getDescriptor();
+        // String data =
+        // "1234567892013-08-18Abbigliamento e accessoriDonna: AccessoriSciarpFoulard e ScialliAuctionItalyRegular";
+        byte[][] data = new byte[9][];
+        data[0] = Bytes.toBytes("123456789");
+        data[1] = Bytes.toBytes("2012-12-15");
+        data[2] = Bytes.toBytes("11848");
+        data[3] = Bytes.toBytes("Health & Beauty");
+        data[4] = Bytes.toBytes("Fragrances");
+        data[5] = Bytes.toBytes("Women");
+        data[6] = Bytes.toBytes("FP-GTC");
+        data[7] = Bytes.toBytes("0");
+        data[8] = Bytes.toBytes("15");
+
+        long baseCuboidId = Cuboid.getBaseCuboidId(cubeDesc);
+        Cuboid baseCuboid = Cuboid.findById(cubeDesc, baseCuboidId);
+        AbstractRowKeyEncoder rowKeyEncoder = AbstractRowKeyEncoder.createInstance(cube.getFirstSegment(), baseCuboid);
+
+        byte[] encodedKey = rowKeyEncoder.encode(data);
+        assertEquals(48, encodedKey.length);
+        byte[] sellerId = Arrays.copyOfRange(encodedKey, 8, 26);
+        byte[] cuboidId = Arrays.copyOfRange(encodedKey, 0, 8);
+        byte[] rest = Arrays.copyOfRange(encodedKey, 26, encodedKey.length);
+        assertTrue(Bytes.toString(sellerId).startsWith("123456789"));
+        assertEquals(511, Bytes.toLong(cuboidId));
+        assertArrayEquals(new byte[] { 11, 55, -13, 13, 22, 34, 121, 70, 80, 45, 71, 84, 67, 9, 9, 9, 9, 9, 9, 0, 10, 5 }, rest);
+    }
+
+    @Test
+    public void testEncodeWithSlr2() throws Exception {
+        CubeInstance cube = CubeManager.getInstance(this.getTestConfig()).getCube("TEST_KYLIN_CUBE_WITH_SLR_READY");
+        // CubeSegment seg = cube.getTheOnlySegment();
+        CubeDesc cubeDesc = cube.getDescriptor();
+        // String data =
+        // "1234567892013-08-18Abbigliamento e accessoriDonna: AccessoriSciarpFoulard e ScialliAuctionItalyRegular";
+        byte[][] data = new byte[9][];
+        data[0] = Bytes.toBytes("123456789");
+        data[1] = null;
+        data[2] = null;
+        data[3] = null;
+        data[4] = null;
+        data[5] = null;
+        data[6] = null;
+        data[7] = null;
+        data[8] = null;
+
+        long baseCuboidId = Cuboid.getBaseCuboidId(cubeDesc);
+        Cuboid baseCuboid = Cuboid.findById(cubeDesc, baseCuboidId);
+        AbstractRowKeyEncoder rowKeyEncoder = AbstractRowKeyEncoder.createInstance(cube.getFirstSegment(), baseCuboid);
+
+        byte[] encodedKey = rowKeyEncoder.encode(data);
+        assertEquals(48, encodedKey.length);
+        byte[] sellerId = Arrays.copyOfRange(encodedKey, 8, 26);
+        byte[] cuboidId = Arrays.copyOfRange(encodedKey, 0, 8);
+        byte[] rest = Arrays.copyOfRange(encodedKey, 26, encodedKey.length);
+        assertTrue(Bytes.toString(sellerId).startsWith("123456789"));
+        assertEquals(511, Bytes.toLong(cuboidId));
+        assertArrayEquals(new byte[] { -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1 }, rest);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/cube/src/test/java/com/kylinolap/cube/kv/RowValueDecoderTest.java
----------------------------------------------------------------------
diff --git a/cube/src/test/java/com/kylinolap/cube/kv/RowValueDecoderTest.java b/cube/src/test/java/com/kylinolap/cube/kv/RowValueDecoderTest.java
new file mode 100644
index 0000000..6595adc
--- /dev/null
+++ b/cube/src/test/java/com/kylinolap/cube/kv/RowValueDecoderTest.java
@@ -0,0 +1,88 @@
+/*
+ * 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.cube.kv;
+
+import static org.junit.Assert.*;
+
+import java.math.BigDecimal;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import java.util.List;
+
+import org.apache.hadoop.io.LongWritable;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+import com.kylinolap.common.util.LocalFileMetadataTestCase;
+import com.kylinolap.cube.CubeManager;
+import com.kylinolap.cube.measure.MeasureCodec;
+import com.kylinolap.metadata.MetadataManager;
+import com.kylinolap.metadata.model.cube.CubeDesc;
+import com.kylinolap.metadata.model.cube.FunctionDesc;
+import com.kylinolap.metadata.model.cube.HBaseColumnDesc;
+import com.kylinolap.metadata.model.cube.MeasureDesc;
+
+/**
+ * @author George Song (ysong1)
+ * 
+ */
+public class RowValueDecoderTest extends LocalFileMetadataTestCase {
+
+    @Before
+    public void setUp() throws Exception {
+        this.createTestMetadata();
+        MetadataManager.removeInstance(this.getTestConfig());
+    }
+
+    @After
+    public void after() throws Exception {
+        this.cleanupTestMetadata();
+    }
+
+    @Test
+    public void testDecode() throws Exception {
+        CubeDesc cubeDesc = CubeManager.getInstance(this.getTestConfig()).getCube("test_kylin_cube_with_slr_ready").getDescriptor();
+        HBaseColumnDesc hbaseCol = cubeDesc.getHBaseMapping().getColumnFamily()[0].getColumns()[0];
+
+        MeasureCodec codec = new MeasureCodec(hbaseCol.getMeasures());
+        BigDecimal sum = new BigDecimal("333.1234567");
+        BigDecimal min = new BigDecimal("333.1111111");
+        BigDecimal max = new BigDecimal("333.1999999");
+        LongWritable count = new LongWritable(2);
+        ByteBuffer buf = ByteBuffer.allocate(RowConstants.ROWVALUE_BUFFER_SIZE);
+        codec.encode(new Object[] { sum, min, max, count }, buf);
+
+        buf.flip();
+        byte[] valueBytes = new byte[buf.limit()];
+        System.arraycopy(buf.array(), 0, valueBytes, 0, buf.limit());
+
+        RowValueDecoder rowValueDecoder = new RowValueDecoder(hbaseCol);
+        for (MeasureDesc measure : cubeDesc.getMeasures()) {
+            FunctionDesc aggrFunc = measure.getFunction();
+            int index = hbaseCol.findMeasureIndex(aggrFunc);
+            rowValueDecoder.setIndex(index);
+        }
+
+        rowValueDecoder.decode(valueBytes);
+        List<String> measureNames = rowValueDecoder.getNames();
+        Object[] measureValues = rowValueDecoder.getValues();
+
+        assertEquals("[PRICE, MIN_PRICE_, MAX_PRICE_, COUNT__]", measureNames.toString());
+        assertEquals("[333.1234567, 333.1111111, 333.1999999, 2]", Arrays.toString(measureValues));
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/cube/src/test/java/com/kylinolap/cube/measure/MeasureCodecTest.java
----------------------------------------------------------------------
diff --git a/cube/src/test/java/com/kylinolap/cube/measure/MeasureCodecTest.java b/cube/src/test/java/com/kylinolap/cube/measure/MeasureCodecTest.java
new file mode 100644
index 0000000..e3e78c9
--- /dev/null
+++ b/cube/src/test/java/com/kylinolap/cube/measure/MeasureCodecTest.java
@@ -0,0 +1,75 @@
+/*
+ * 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.cube.measure;
+
+import static org.junit.Assert.*;
+
+import java.math.BigDecimal;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+
+import org.apache.hadoop.io.DoubleWritable;
+import org.apache.hadoop.io.LongWritable;
+import org.junit.Test;
+
+import com.kylinolap.common.hll.HyperLogLogPlusCounter;
+import com.kylinolap.cube.kv.RowConstants;
+import com.kylinolap.metadata.model.cube.FunctionDesc;
+import com.kylinolap.metadata.model.cube.MeasureDesc;
+
+/**
+ * @author yangli9
+ * 
+ */
+public class MeasureCodecTest {
+
+    @Test
+    public void basicTest() {
+        MeasureDesc descs[] = new MeasureDesc[] { measure("double"), measure("long"), measure("decimal"), measure("HLLC16"), measure("HLLC16") };
+        MeasureCodec codec = new MeasureCodec(descs);
+
+        DoubleWritable d = new DoubleWritable(1.0);
+        LongWritable l = new LongWritable(2);
+        BigDecimal b = new BigDecimal("333.1234567");
+        HyperLogLogPlusCounter hllc = new HyperLogLogPlusCounter(16);
+        hllc.add("1234567");
+        hllc.add("abcdefg");
+        HyperLogLogPlusCounter hllc2 = new HyperLogLogPlusCounter(16);
+        hllc.add("1234567");
+        hllc.add("abcdefg");
+        Object values[] = new Object[] { d, l, b, hllc, hllc2 };
+
+        ByteBuffer buf = ByteBuffer.allocate(RowConstants.ROWVALUE_BUFFER_SIZE);
+
+        codec.encode(values, buf);
+        buf.flip();
+        System.out.println("size: " + buf.limit());
+
+        Object copy[] = new Object[values.length];
+        codec.decode(buf, copy);
+
+        assertTrue(Arrays.equals(values, copy));
+    }
+
+    private MeasureDesc measure(String returnType) {
+        MeasureDesc desc = new MeasureDesc();
+        FunctionDesc func = new FunctionDesc();
+        func.setReturnType(returnType);
+        desc.setFunction(func);
+        return desc;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/cube/src/test/java/com/kylinolap/cube/project/ProjectManagerTest.java
----------------------------------------------------------------------
diff --git a/cube/src/test/java/com/kylinolap/cube/project/ProjectManagerTest.java b/cube/src/test/java/com/kylinolap/cube/project/ProjectManagerTest.java
new file mode 100644
index 0000000..532c2dc
--- /dev/null
+++ b/cube/src/test/java/com/kylinolap/cube/project/ProjectManagerTest.java
@@ -0,0 +1,165 @@
+/*
+ * 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.cube.project;
+
+import static org.junit.Assert.*;
+
+import java.io.IOException;
+
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+import com.kylinolap.common.persistence.ResourceStore;
+import com.kylinolap.common.util.JsonUtil;
+import com.kylinolap.common.util.LocalFileMetadataTestCase;
+import com.kylinolap.cube.CubeInstance;
+import com.kylinolap.cube.CubeManager;
+import com.kylinolap.metadata.MetadataManager;
+import com.kylinolap.metadata.model.cube.CubeDesc;
+
+/**
+ * @author xduo
+ * 
+ */
+public class ProjectManagerTest extends LocalFileMetadataTestCase {
+
+    @Before
+    public void setUp() throws Exception {
+        this.createTestMetadata();
+        MetadataManager.removeInstance(this.getTestConfig());
+        CubeManager.removeInstance(this.getTestConfig());
+        ProjectManager.removeInstance(this.getTestConfig());
+    }
+
+    @After
+    public void after() throws Exception {
+        this.cleanupTestMetadata();
+    }
+
+    @Test(expected = IllegalStateException.class)
+    public void testDropNonemptyProject1() throws IOException {
+        ProjectManager.getInstance(this.getTestConfig()).dropProject("DEFAULT");
+    }
+
+    @Test(expected = IllegalStateException.class)
+    public void testDropNonemptyProject2() throws IOException {
+        ProjectManager.getInstance(this.getTestConfig()).dropProject("DEFAULT???");
+    }
+
+    @Test
+    public void testNewProject() throws Exception {
+        int originalProjectCount = ProjectManager.getInstance(this.getTestConfig()).listAllProjects().size();
+        int originalCubeCount = CubeManager.getInstance(this.getTestConfig()).listAllCubes().size();
+        int originalCubeCountInDefault = ProjectManager.getInstance(this.getTestConfig()).listAllCubes("default").size();
+        ResourceStore store = getStore();
+
+        // clean legacy in case last run failed
+        store.deleteResource("/cube/cube_in_alien_project.json");
+
+        MetadataManager metaMgr = getMetadataManager();
+        CubeDesc desc = metaMgr.getCubeDesc("test_kylin_cube_with_slr_desc");
+        CubeInstance createdCube = CubeManager.getInstance(this.getTestConfig()).createCube("cube_in_alien_project", "alien", desc, null);
+        assertTrue(createdCube == CubeManager.getInstance(this.getTestConfig()).getCube("cube_in_alien_project"));
+        assertTrue(ProjectManager.getInstance(getTestConfig()).listAllCubes("alien").contains(createdCube));
+
+        System.out.println(JsonUtil.writeValueAsIndentString(createdCube));
+
+        assertTrue(ProjectManager.getInstance(this.getTestConfig()).listAllProjects().size() == originalProjectCount + 1);
+        assertTrue(ProjectManager.getInstance(this.getTestConfig()).listAllCubes("ALIEN").get(0).getName().equalsIgnoreCase("CUBE_IN_ALIEN_PROJECT"));
+        assertTrue(CubeManager.getInstance(this.getTestConfig()).listAllCubes().size() == originalCubeCount + 1);
+
+        ProjectManager.getInstance(this.getTestConfig()).updateCubeToProject("cube_in_alien_project", "default", null);
+        assertTrue(ProjectManager.getInstance(this.getTestConfig()).listAllCubes("ALIEN").size() == 0);
+        assertTrue(ProjectManager.getInstance(this.getTestConfig()).listAllCubes("default").size() == originalCubeCountInDefault + 1);
+        assertTrue(ProjectManager.getInstance(getTestConfig()).listAllCubes("default").contains(createdCube));
+
+        ProjectManager.getInstance(this.getTestConfig()).updateCubeToProject("cube_in_alien_project", "alien", null);
+        assertTrue(ProjectManager.getInstance(this.getTestConfig()).listAllCubes("ALIEN").size() == 1);
+        assertTrue(ProjectManager.getInstance(this.getTestConfig()).listAllCubes("default").size() == originalCubeCountInDefault);
+        assertTrue(ProjectManager.getInstance(getTestConfig()).listAllCubes("alien").contains(createdCube));
+
+        assertTrue(ProjectManager.getInstance(this.getTestConfig()).isCubeInProject("alien", createdCube));
+
+        CubeInstance droppedCube = CubeManager.getInstance(this.getTestConfig()).dropCube("cube_in_alien_project", true);
+
+        assertTrue(createdCube == droppedCube);
+        assertNull(CubeManager.getInstance(this.getTestConfig()).getCube("cube_in_alien_project"));
+        assertTrue(ProjectManager.getInstance(this.getTestConfig()).listAllProjects().size() == originalProjectCount + 1);
+        assertTrue(CubeManager.getInstance(this.getTestConfig()).listAllCubes().size() == originalCubeCount);
+
+        ProjectManager.getInstance(this.getTestConfig()).dropProject("alien");
+        assertTrue(ProjectManager.getInstance(this.getTestConfig()).listAllProjects().size() == originalProjectCount);
+    }
+
+    @Test
+    public void testExistingProject() throws Exception {
+        int originalProjectCount = ProjectManager.getInstance(this.getTestConfig()).listAllProjects().size();
+        int originalCubeCount = CubeManager.getInstance(this.getTestConfig()).listAllCubes().size();
+        ResourceStore store = getStore();
+
+        // clean legacy in case last run failed
+        store.deleteResource("/cube/new_cube_in_default.json");
+
+        MetadataManager metaMgr = getMetadataManager();
+        CubeDesc desc = metaMgr.getCubeDesc("test_kylin_cube_with_slr_desc");
+        CubeInstance createdCube = CubeManager.getInstance(this.getTestConfig()).createCube("new_cube_in_default", ProjectInstance.DEFAULT_PROJECT_NAME, desc, null);
+        assertTrue(createdCube == CubeManager.getInstance(this.getTestConfig()).getCube("new_cube_in_default"));
+
+        System.out.println(JsonUtil.writeValueAsIndentString(createdCube));
+
+        assertTrue(ProjectManager.getInstance(this.getTestConfig()).listAllProjects().size() == originalProjectCount);
+        assertTrue(CubeManager.getInstance(this.getTestConfig()).listAllCubes().size() == originalCubeCount + 1);
+
+        CubeInstance droppedCube = CubeManager.getInstance(this.getTestConfig()).dropCube("new_cube_in_default", true);
+
+        assertTrue(createdCube == droppedCube);
+        assertNull(CubeManager.getInstance(this.getTestConfig()).getCube("new_cube_in_default"));
+        assertTrue(ProjectManager.getInstance(this.getTestConfig()).listAllProjects().size() == originalProjectCount);
+        assertTrue(CubeManager.getInstance(this.getTestConfig()).listAllCubes().size() == originalCubeCount);
+    }
+
+    @Test
+    public void testProjectsDrop() throws IOException {
+        CubeInstance cube = CubeManager.getInstance(getTestConfig()).getCube("test_kylin_cube_with_slr_empty");
+        assertTrue(ProjectManager.getInstance(this.getTestConfig()).getCubesByTable("default", "test_kylin_fact").contains(cube));
+        assertTrue(ProjectManager.getInstance(this.getTestConfig()).listAllCubes("default").contains(cube));
+
+        CubeManager.getInstance(getTestConfig()).dropCube(cube.getName(), true);
+
+        assertTrue(!ProjectManager.getInstance(this.getTestConfig()).getCubesByTable("default", "test_kylin_fact").contains(cube));
+        assertTrue(!ProjectManager.getInstance(this.getTestConfig()).listAllCubes("default").contains(cube));
+    }
+
+    @Test
+    public void testProjectsLoadAfterProjectChange() throws IOException {
+        CubeInstance cube = CubeManager.getInstance(getTestConfig()).getCube("test_kylin_cube_with_slr_empty");
+        assertTrue(ProjectManager.getInstance(this.getTestConfig()).getCubesByTable("default", "test_kylin_fact").contains(cube));
+
+        ProjectManager.getInstance(getTestConfig()).removeCubeFromProjects(cube.getName());
+
+        assertTrue(!ProjectManager.getInstance(this.getTestConfig()).getCubesByTable("default", "test_kylin_fact").contains(cube));
+
+        ProjectManager.getInstance(getTestConfig()).updateCubeToProject(cube.getName(), "default", "tester");
+
+        assertTrue(ProjectManager.getInstance(this.getTestConfig()).getCubesByTable("default", "test_kylin_fact").contains(cube));
+    }
+
+    private MetadataManager getMetadataManager() {
+        return MetadataManager.getInstance(getTestConfig());
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/cube/src/test/resources/placeholder.txt
----------------------------------------------------------------------
diff --git a/cube/src/test/resources/placeholder.txt b/cube/src/test/resources/placeholder.txt
new file mode 100644
index 0000000..e69de29

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/deploy.sh
----------------------------------------------------------------------
diff --git a/deploy.sh b/deploy.sh
new file mode 100755
index 0000000..4dd8355
--- /dev/null
+++ b/deploy.sh
@@ -0,0 +1,206 @@
+#!/usr/bin/env bash
+
+set -o pipefail  # trace ERR through pipes
+set -o errtrace  # trace ERR through 'time command' and other functions
+
+function error() {
+SCRIPT="$0"           # script name
+LASTLINE="$1"         # line of error occurrence
+LASTERR="$2"          # error code
+echo "ERROR exit from ${SCRIPT} : line ${LASTLINE} with exit code ${LASTERR}"
+exit 1
+}
+
+trap 'error ${LINENO} ${?}' ERR
+
+echo ""
+echo "Welcome to use Kylin-Deploy script"
+echo "This script will help you:"
+echo "1. Check environment"
+echo "2. Build Kylin artifacts"
+echo "3. Prepare test cube related data"
+echo "4. Lauch a web service to build cube and query with (at http://localhost:7070)"
+echo "Please make sure you are running this script on a hadoop CLI machine, and you have enough permissions."
+echo "Also, We assume you have installed: JAVA, TOMCAT, NPM and MAVEN."
+echo "[Warning] The installation may break existing tomcat applications on this CLI"
+echo ""
+
+
+[[ "$SILENT" ]] || ( read -p "Are you sure you want to proceed?(press Y or y to confirm) " -n 1 -r
+echo    # (optional) move to a new line
+if [[ ! $REPLY =~ ^[Yy]$ ]]
+then
+    echo "Not going to proceed, quit without finishing! You can rerun the script to have another try."
+    exit 1
+fi )
+
+echo "Checking JAVA status..."
+
+if [ -z "$JAVA_HOME" ]
+then
+    echo "Please set JAVA_HOME so that Kylin-Deploy can proceed"
+    exit 1
+else
+    echo "JAVA_HOME is set to $JAVA_HOME"
+fi
+
+if [ -d "$JAVA_HOME" ]
+then
+    echo "$JAVA_HOME exists"
+else
+    echo " $JAVA_HOME does not exist or is not a directory."
+    exit 1
+fi
+
+echo "Checking tomcat status..."
+
+if [ -z "$CATALINA_HOME" ]
+then
+    echo "Please set CATALINA_HOME so that Kylin-Deploy knows where to start tomcat"
+    exit 1
+else
+    echo "CATALINA_HOME is set to $CATALINA_HOME"
+fi
+
+if [ -d "$CATALINA_HOME" ]
+then
+    echo "$CATALINA_HOME exists"
+else
+    echo " $CATALINA_HOME does not exist or is not a directory."
+    exit 1
+fi
+
+echo "Checking maven..."
+
+if [ -z "$(command -v mvn)" ]
+then
+    echo "Please install maven first so that Kylin-Deploy can proceed"
+    exit 1
+else
+    echo "maven check passed"
+fi
+
+echo "Checking npm..."
+
+if [ -z "$(command -v npm)" ]
+then
+    echo "Please install npm first so that Kylin-Deploy can proceed"
+    exit 1
+else
+    echo "npm check passed"
+fi
+
+KYLIN_HOME="$( cd "$( dirname "${BASH_SOURCE[0]}" )"  && pwd )"
+echo "Kylin home folder path is $KYLIN_HOME"
+cd $KYLIN_HOME
+
+
+echo "Building and packaging..."
+source ./package.sh
+
+echo "retrieving classpath..."
+cd $KYLIN_HOME/job/target
+JOB_JAR_NAME="kylin-job-latest.jar"
+#generate config variables
+hbase org.apache.hadoop.util.RunJar $JOB_JAR_NAME com.kylinolap.job.deployment.HbaseConfigPrinter /tmp/kylin_retrieve.sh
+#load config variables
+source /tmp/kylin_retrieve.sh
+
+cd $KYLIN_HOME
+mkdir -p /etc/kylin
+
+HOSTNAME=`hostname`
+CLI_HOSTNAME_DEFAULT="kylin.job.remote.cli.hostname=sandbox.hortonworks.com"
+CLI_PASSWORD_DEFAULT="kylin.job.remote.cli.password=hadoop"
+METADATA_URL_DEFAULT="kylin.metadata.url=kylin_metadata_qa@hbase:sandbox.hortonworks.com:2181:/hbase-unsecure"
+STORAGE_URL_DEFAULT="kylin.storage.url=hbase:sandbox.hortonworks.com:2181:/hbase-unsecure"
+CHECK_URL_DEFAULT="kylin.job.yarn.app.rest.check.status.url=http://sandbox"
+
+
+NEW_CLI_HOSTNAME_PREFIX="kylin.job.remote.cli.hostname="
+NEW_CLI_PASSWORD_PREFIX="kylin.job.remote.cli.password="
+NEW_METADATA_URL_PREFIX="kylin.metadata.url=kylin_metadata_qa@hbase:"
+NEW_STORAGE_URL_PREFIX="kylin.storage.url=hbase:"
+NEW_CHECK_URL_PREFIX="kylin.job.yarn.app.rest.check.status.url=http://"
+
+KYLIN_ZOOKEEPER_URL=${KYLIN_ZOOKEEPER_QUORUM}:${KYLIN_ZOOKEEPER_CLIENT_PORT}:${KYLIN_ZOOKEEPER_ZNODE_PARENT}
+
+echo "Kylin install script requires root password for ${HOSTNAME}"
+echo "(The default root password for hortonworks VM is hadoop, and for cloudera VM is cloudera)"
+
+[[ "$SILENT" ]] || read -s -p "Enter Password for root: " ROOTPASS
+
+#deploy kylin.properties to /etc/kylin
+cat examples/test_case_data/sandbox/kylin.properties | \
+    sed -e "s,${CHECK_URL_DEFAULT},${NEW_CHECK_URL_PREFIX}${HOSTNAME}," | \
+    sed -e "s,${CLI_HOSTNAME_DEFAULT},${NEW_CLI_HOSTNAME_PREFIX}${HOSTNAME}," | \
+    sed -e "s,${CLI_PASSWORD_DEFAULT},${NEW_CLI_PASSWORD_PREFIX}${ROOTPASS}," | \
+    sed -e "s,${METADATA_URL_DEFAULT},${NEW_METADATA_URL_PREFIX}${KYLIN_ZOOKEEPER_URL}," | \
+    sed -e "s,${STORAGE_URL_DEFAULT},${NEW_STORAGE_URL_PREFIX}${KYLIN_ZOOKEEPER_URL}," >  /etc/kylin/kylin.properties
+
+
+echo "a copy of kylin config is generated at /etc/kylin/kylin.properties:"
+echo "==================================================================="
+cat /etc/kylin/kylin.properties
+echo ""
+echo "==================================================================="
+echo ""
+
+[[ "$SILENT" ]] || ( read -p "please ensure the CLI address/username/password is correct, and press y to proceed: " -n 1 -r
+echo    # (optional) move to a new line
+if [[ ! $REPLY =~ ^[Yy]$ ]]
+then
+    echo "Not going to proceed, quit without finishing! You can rerun the script to have another try."
+    exit 1
+fi )
+
+# 1. generate synthetic fact table(test_kylin_fact) data and dump it into hive
+# 2. create empty cubes on these data, ready to be built
+cd $KYLIN_HOME
+mvn test -Dtest=com.kylinolap.job.SampleCubeSetupTest -DfailIfNoTests=false
+
+sudo -i "${CATALINA_HOME}/bin/shutdown.sh" || true # avoid trapping
+cd $KYLIN_HOME/server/target
+WAR_NAME="kylin.war"
+rm -rf $CATALINA_HOME/webapps/kylin
+rm -f $CATALINA_HOME/webapps/$WAR_NAME
+cp $KYLIN_HOME/server/target/$WAR_NAME $CATALINA_HOME/webapps/
+cd $CATALINA_HOME/webapps;
+chmod 644 $WAR_NAME;
+echo "REST service deployed"
+
+rm -rf /var/www/html/kylin
+mkdir -p /var/www/html/kylin
+cd $KYLIN_HOME/
+tar -xf webapp/dist/Web.tar -C /var/www/html/kylin
+echo "Web deployed"
+
+cd $KYLIN_HOME/
+#deploy setenv.sh
+rm -rf $CATALINA_HOME/bin/setenv.sh
+echo JAVA_OPTS=\"-Djava.library.path=${KYLIN_LD_LIBRARY_PATH}\" >> ${CATALINA_HOME}/bin/setenv.sh
+echo CATALINA_OPTS=\"-Dorg.apache.tomcat.util.buf.UDecoder.ALLOW_ENCODED_SLASH=true -Dorg.apache.catalina.connector.CoyoteAdapter.ALLOW_BACKSLASH=true -Dspring.profiles.active=sandbox \" >> ${CATALINA_HOME}/bin/setenv.sh
+echo CLASSPATH=\"${CATALINA_HOME}/lib/*:${KYLIN_HBASE_CLASSPATH}:/etc/kylin\" >> ${CATALINA_HOME}/bin/setenv.sh
+echo "setenv.sh created"
+
+#deploy server.xml
+rm -rf ${CATALINA_HOME}/conf/server.xml
+cp deploy/server.xml ${CATALINA_HOME}/conf/server.xml
+echo "server.xml copied"
+
+#deploy web.xml
+rm -rf ${CATALINA_HOME}/conf/web.xml
+cp deploy/web.xml ${CATALINA_HOME}/conf/web.xml
+echo "web.xml copied"
+
+echo "Tomcat ready"
+
+# redeploy coprocessor
+#hbase org.apache.hadoop.util.RunJar /usr/lib/kylin/kylin-job-latest.jar com.kylinolap.job.tools.DeployCoprocessorCLI /usr/lib/kylin/kylin-coprocessor-latest.jar
+
+
+sudo -i "${CATALINA_HOME}/bin/startup.sh"
+
+
+echo "Kylin-Deploy Success!"
+echo "Please visit http://<your_sandbox_ip>:7070 to play with the cubes! (Useranme: ADMIN, Password: KYLIN)"

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/deploy/install.sh
----------------------------------------------------------------------
diff --git a/deploy/install.sh b/deploy/install.sh
new file mode 100755
index 0000000..a8c7108
--- /dev/null
+++ b/deploy/install.sh
@@ -0,0 +1,15 @@
+#!/bin/sh
+
+
+cd ~
+wget http://apache.cs.uu.nl/dist/tomcat/tomcat-7/v7.0.56/bin/apache-tomcat-7.0.56.tar.gz
+tar -xzvf apache-tomcat-7.0.56.tar.gz
+export CATALINA_HOME=/root/apache-tomcat-7.0.56
+
+wget http://apache.proserve.nl/maven/maven-3/3.2.3/binaries/apache-maven-3.2.3-bin.tar.gz
+tar -xzvf apache-maven-3.2.3-bin.tar.gz
+ln -s /root/apache-maven-3.2.3/bin/mvn /usr/bin/mvn
+
+wget http://nodejs.org/dist/v0.10.32/node-v0.10.32-linux-x64.tar.gz
+tar -xzvf node-v0.10.32-linux-x64.tar.gz
+export PATH=/root/node-v0.10.32-linux-x64/bin:$PATH
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/deploy/server.xml
----------------------------------------------------------------------
diff --git a/deploy/server.xml b/deploy/server.xml
new file mode 100644
index 0000000..4bd26d0
--- /dev/null
+++ b/deploy/server.xml
@@ -0,0 +1,156 @@
+<?xml version='1.0' encoding='utf-8'?>
+<!--
+  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.
+-->
+<!-- Note:  A "Server" is not itself a "Container", so you may not
+     define subcomponents such as "Valves" at this level.
+     Documentation at /docs/config/server.html
+ -->
+<Server port="9005" shutdown="SHUTDOWN">
+    <!-- Security listener. Documentation at /docs/config/listeners.html
+    <Listener className="org.apache.catalina.security.SecurityListener" />
+    -->
+    <!--APR library loader. Documentation at /docs/apr.html -->
+    <Listener className="org.apache.catalina.core.AprLifecycleListener" SSLEngine="on" />
+    <!--Initialize Jasper prior to webapps are loaded. Documentation at /docs/jasper-howto.html -->
+    <Listener className="org.apache.catalina.core.JasperListener" />
+    <!-- Prevent memory leaks due to use of particular java/javax APIs-->
+    <Listener className="org.apache.catalina.core.JreMemoryLeakPreventionListener" />
+    <Listener className="org.apache.catalina.mbeans.GlobalResourcesLifecycleListener" />
+    <Listener className="org.apache.catalina.core.ThreadLocalLeakPreventionListener" />
+
+    <!-- Global JNDI resources
+         Documentation at /docs/jndi-resources-howto.html
+    -->
+    <GlobalNamingResources>
+        <!-- Editable user database that can also be used by
+             UserDatabaseRealm to authenticate users
+        -->
+        <Resource name="UserDatabase" auth="Container"
+                  type="org.apache.catalina.UserDatabase"
+                  description="User database that can be updated and saved"
+                  factory="org.apache.catalina.users.MemoryUserDatabaseFactory"
+                  pathname="conf/tomcat-users.xml" />
+    </GlobalNamingResources>
+
+    <!-- A "Service" is a collection of one or more "Connectors" that share
+         a single "Container" Note:  A "Service" is not itself a "Container",
+         so you may not define subcomponents such as "Valves" at this level.
+         Documentation at /docs/config/service.html
+     -->
+    <Service name="Catalina">
+
+        <!--The connectors can use a shared executor, you can define one or more named thread pools-->
+        <!--
+        <Executor name="tomcatThreadPool" namePrefix="catalina-exec-"
+            maxThreads="150" minSpareThreads="4"/>
+        -->
+
+
+        <!-- A "Connector" represents an endpoint by which requests are received
+             and responses are returned. Documentation at :
+             Java HTTP Connector: /docs/config/http.html (blocking & non-blocking)
+             Java AJP  Connector: /docs/config/ajp.html
+             APR (HTTP/AJP) Connector: /docs/apr.html
+             Define a non-SSL HTTP/1.1 Connector on port 8080
+        -->
+        <Connector port="7070" protocol="HTTP/1.1"
+                   connectionTimeout="20000"
+                   redirectPort="9443"
+                   compression="on"
+                   compressionMinSize="2048"
+                   noCompressionUserAgents="gozilla,traviata"
+                   compressableMimeType="text/html,text/xml,text/javascript,application/javascript,application/json,text/css,text/plain"
+                />
+        <!-- A "Connector" using the shared thread pool-->
+        <!--
+        <Connector executor="tomcatThreadPool"
+                   port="8080" protocol="HTTP/1.1"
+                   connectionTimeout="20000"
+                   redirectPort="8443" />
+        -->
+        <!-- Define a SSL HTTP/1.1 Connector on port 8443
+             This connector uses the BIO implementation that requires the JSSE
+             style configuration. When using the APR/native implementation, the
+             OpenSSL style configuration is required as described in the APR/native
+             documentation -->
+        <!--
+        <Connector port="8443" protocol="org.apache.coyote.http11.Http11Protocol"
+                   maxThreads="150" SSLEnabled="true" scheme="https" secure="true"
+                   clientAuth="false" sslProtocol="TLS" />
+        -->
+
+        <!-- Define an AJP 1.3 Connector on port 8009 -->
+        <Connector port="9009" protocol="AJP/1.3" redirectPort="9443" />
+
+
+        <!-- An Engine represents the entry point (within Catalina) that processes
+             every request.  The Engine implementation for Tomcat stand alone
+             analyzes the HTTP headers included with the request, and passes them
+             on to the appropriate Host (virtual host).
+             Documentation at /docs/config/engine.html -->
+
+        <!-- You should set jvmRoute to support load-balancing via AJP ie :
+        <Engine name="Catalina" defaultHost="localhost" jvmRoute="jvm1">
+        -->
+        <Engine name="Catalina" defaultHost="localhost">
+
+            <!--For clustering, please take a look at documentation at:
+                /docs/cluster-howto.html  (simple how to)
+                /docs/config/cluster.html (reference documentation) -->
+            <!--
+            <Cluster className="org.apache.catalina.ha.tcp.SimpleTcpCluster"/>
+            -->
+
+            <!-- Use the LockOutRealm to prevent attempts to guess user passwords
+                 via a brute-force attack -->
+            <Realm className="org.apache.catalina.realm.LockOutRealm">
+                <!-- This Realm uses the UserDatabase configured in the global JNDI
+                     resources under the key "UserDatabase".  Any edits
+                     that are performed against this UserDatabase are immediately
+                     available for use by the Realm.  -->
+                <Realm className="org.apache.catalina.realm.UserDatabaseRealm"
+                       resourceName="UserDatabase"/>
+            </Realm>
+
+            <Host name="localhost"  appBase="webapps"
+                  unpackWARs="true" autoDeploy="true">
+
+                <!-- SingleSignOn valve, share authentication between web applications
+                     Documentation at: /docs/config/valve.html -->
+                <!--
+                <Valve className="org.apache.catalina.authenticator.SingleSignOn" />
+                -->
+
+                <!-- Access log processes all example.
+                     Documentation at: /docs/config/valve.html
+                     Note: The pattern used is equivalent to using pattern="common" -->
+                <Valve className="org.apache.catalina.valves.AccessLogValve" directory="logs"
+                       prefix="localhost_access_log." suffix=".txt"
+                       pattern="%h %l %u %t &quot;%r&quot; %s %b" />
+                <Context path="" docBase="/var/www/html/kylin">
+                    <!-- Default set of monitored resources -->
+                    <WatchedResource>WEB-INF/web.xml</WatchedResource>
+                </Context>
+                <Context path="/ROOT" docBase="ROOT">
+                    <!-- Default set of monitored resources -->
+                    <WatchedResource>WEB-INF/web.xml</WatchedResource>
+                </Context>
+
+            </Host>
+        </Engine>
+    </Service>
+</Server>


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

Posted by lu...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/job/src/main/java/com/kylinolap/job/tools/DeployCoprocessorCLI.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/com/kylinolap/job/tools/DeployCoprocessorCLI.java b/job/src/main/java/com/kylinolap/job/tools/DeployCoprocessorCLI.java
new file mode 100644
index 0000000..5721d67
--- /dev/null
+++ b/job/src/main/java/com/kylinolap/job/tools/DeployCoprocessorCLI.java
@@ -0,0 +1,270 @@
+/*
+ * 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.job.tools;
+
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.regex.Matcher;
+
+import org.apache.commons.io.IOUtils;
+import org.apache.commons.lang.StringUtils;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.HTableDescriptor;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.TableNotFoundException;
+import org.apache.hadoop.hbase.client.HBaseAdmin;
+import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.kylinolap.common.KylinConfig;
+import com.kylinolap.common.util.HadoopUtil;
+import com.kylinolap.cube.CubeInstance;
+import com.kylinolap.cube.CubeManager;
+import com.kylinolap.cube.CubeSegment;
+import com.kylinolap.cube.CubeSegmentStatusEnum;
+
+/**
+ * @author yangli9
+ */
+public class DeployCoprocessorCLI {
+
+    private static final Logger logger = LoggerFactory.getLogger(DeployCoprocessorCLI.class);
+
+    public static final String AGGR_COPROCESSOR_CLS_NAME = "com.kylinolap.storage.hbase.observer.AggregateRegionObserver";
+
+    public static void main(String[] args) throws IOException {
+        KylinConfig kylinConfig = KylinConfig.getInstanceFromEnv();
+        Configuration hconf = HadoopUtil.newHBaseConfiguration(kylinConfig.getStorageUrl());
+        FileSystem fileSystem = FileSystem.get(hconf);
+        HBaseAdmin hbaseAdmin = new HBaseAdmin(hconf);
+
+        String localCoprocessorJar = new File(args[0]).getAbsolutePath();
+        logger.info("Identify coprocessor jar " + localCoprocessorJar);
+
+        List<String> tableNames = getHTableNames(kylinConfig);
+        logger.info("Identify tables " + tableNames);
+
+        Set<String> oldJarPaths = getCoprocessorJarPaths(hbaseAdmin, tableNames);
+        logger.info("Old coprocessor jar: " + oldJarPaths);
+
+        Path hdfsCoprocessorJar = uploadCoprocessorJar(localCoprocessorJar, fileSystem, oldJarPaths);
+        logger.info("New coprocessor jar: " + hdfsCoprocessorJar);
+
+        List<String> processedTables = resetCoprocessorOnHTables(hbaseAdmin, hdfsCoprocessorJar, tableNames);
+
+        // Don't remove old jars, missing coprocessor jar will fail hbase
+        // removeOldJars(oldJarPaths, fileSystem);
+
+        hbaseAdmin.close();
+
+        logger.info("Processed " + processedTables);
+        logger.info("Active coprocessor jar: " + hdfsCoprocessorJar);
+    }
+
+    public static void setCoprocessorOnHTable(HTableDescriptor desc, Path hdfsCoprocessorJar) throws IOException {
+        logger.info("Set coprocessor on " + desc.getNameAsString());
+        desc.addCoprocessor(AGGR_COPROCESSOR_CLS_NAME, hdfsCoprocessorJar, 1001, null);
+    }
+
+    public static void resetCoprocessor(String tableName, HBaseAdmin hbaseAdmin, Path hdfsCoprocessorJar) throws IOException {
+        logger.info("Disable " + tableName);
+        hbaseAdmin.disableTable(tableName);
+
+        logger.info("Unset coprocessor on " + tableName);
+        HTableDescriptor desc = hbaseAdmin.getTableDescriptor(TableName.valueOf(tableName));
+        while (desc.hasCoprocessor(AGGR_COPROCESSOR_CLS_NAME)) {
+            desc.removeCoprocessor(AGGR_COPROCESSOR_CLS_NAME);
+        }
+
+        setCoprocessorOnHTable(desc, hdfsCoprocessorJar);
+        hbaseAdmin.modifyTable(tableName, desc);
+
+        logger.info("Enable " + tableName);
+        hbaseAdmin.enableTable(tableName);
+    }
+
+    private static List<String> resetCoprocessorOnHTables(HBaseAdmin hbaseAdmin, Path hdfsCoprocessorJar, List<String> tableNames) throws IOException {
+        List<String> processed = new ArrayList<String>();
+
+        for (String tableName : tableNames) {
+            try {
+                resetCoprocessor(tableName, hbaseAdmin, hdfsCoprocessorJar);
+                processed.add(tableName);
+            } catch (IOException ex) {
+                logger.error("Error processing " + tableName, ex);
+            }
+        }
+        return processed;
+    }
+
+    public static Path getNewestCoprocessorJar(KylinConfig config, FileSystem fileSystem) throws IOException {
+        Path coprocessorDir = getCoprocessorHDFSDir(fileSystem, config);
+        FileStatus newestJar = null;
+        for (FileStatus fileStatus : fileSystem.listStatus(coprocessorDir)) {
+            if (fileStatus.getPath().toString().endsWith(".jar")) {
+                if (newestJar == null) {
+                    newestJar = fileStatus;
+                } else {
+                    if (newestJar.getModificationTime() < fileStatus.getModificationTime())
+                        newestJar = fileStatus;
+                }
+            }
+        }
+        if (newestJar == null)
+            return null;
+
+        Path path = newestJar.getPath().makeQualified(fileSystem.getUri(), null);
+        logger.info("The newest coprocessor is " + path.toString());
+        return path;
+    }
+
+    public static Path uploadCoprocessorJar(String localCoprocessorJar, FileSystem fileSystem, Set<String> oldJarPaths) throws IOException {
+        Path uploadPath = null;
+        File localCoprocessorFile = new File(localCoprocessorJar);
+
+        // check existing jars
+        if (oldJarPaths == null) {
+            oldJarPaths = new HashSet<String>();
+        }
+        Path coprocessorDir = getCoprocessorHDFSDir(fileSystem, KylinConfig.getInstanceFromEnv());
+        for (FileStatus fileStatus : fileSystem.listStatus(coprocessorDir)) {
+            if (fileStatus.getLen() == localCoprocessorJar.length() && fileStatus.getModificationTime() == localCoprocessorFile.lastModified()) {
+                uploadPath = fileStatus.getPath();
+                break;
+            }
+            String filename = fileStatus.getPath().toString();
+            if (filename.endsWith(".jar")) {
+                oldJarPaths.add(filename);
+            }
+        }
+
+        // upload if not existing
+        if (uploadPath == null) {
+            // figure out a unique new jar file name
+            Set<String> oldJarNames = new HashSet<String>();
+            for (String path : oldJarPaths) {
+                oldJarNames.add(new Path(path).getName());
+            }
+            String baseName = getBaseFileName(localCoprocessorJar);
+            String newName = null;
+            int i = 0;
+            while (newName == null) {
+                newName = baseName + "-" + (i++) + ".jar";
+                if (oldJarNames.contains(newName))
+                    newName = null;
+            }
+
+            // upload
+            uploadPath = new Path(coprocessorDir, newName);
+            FileInputStream in = null;
+            FSDataOutputStream out = null;
+            try {
+                in = new FileInputStream(localCoprocessorFile);
+                out = fileSystem.create(uploadPath);
+                IOUtils.copy(in, out);
+            } finally {
+                IOUtils.closeQuietly(in);
+                IOUtils.closeQuietly(out);
+            }
+
+            fileSystem.setTimes(uploadPath, localCoprocessorFile.lastModified(), System.currentTimeMillis());
+
+        }
+
+        uploadPath = uploadPath.makeQualified(fileSystem.getUri(), null);
+        return uploadPath;
+    }
+
+    private static String getBaseFileName(String localCoprocessorJar) {
+        File localJar = new File(localCoprocessorJar);
+        String baseName = localJar.getName();
+        if (baseName.endsWith(".jar"))
+            baseName = baseName.substring(0, baseName.length() - ".jar".length());
+        return baseName;
+    }
+
+    private static Path getCoprocessorHDFSDir(FileSystem fileSystem, KylinConfig config) throws IOException {
+        String hdfsWorkingDirectory = config.getHdfsWorkingDirectory();
+        Path coprocessorDir = new Path(hdfsWorkingDirectory, "coprocessor");
+        fileSystem.mkdirs(coprocessorDir);
+        return coprocessorDir;
+    }
+
+    private static Set<String> getCoprocessorJarPaths(HBaseAdmin hbaseAdmin, List<String> tableNames) throws IOException {
+        HashSet<String> result = new HashSet<String>();
+
+        for (String tableName : tableNames) {
+            HTableDescriptor tableDescriptor = null;
+            try {
+                tableDescriptor = hbaseAdmin.getTableDescriptor(TableName.valueOf(tableName));
+            } catch (TableNotFoundException e) {
+                logger.warn("Table not found " + tableName, e);
+                continue;
+            }
+
+            Matcher keyMatcher;
+            Matcher valueMatcher;
+            for (Map.Entry<ImmutableBytesWritable, ImmutableBytesWritable> e : tableDescriptor.getValues().entrySet()) {
+                keyMatcher = HConstants.CP_HTD_ATTR_KEY_PATTERN.matcher(Bytes.toString(e.getKey().get()));
+                if (!keyMatcher.matches()) {
+                    continue;
+                }
+                valueMatcher = HConstants.CP_HTD_ATTR_VALUE_PATTERN.matcher(Bytes.toString(e.getValue().get()));
+                if (!valueMatcher.matches()) {
+                    continue;
+                }
+
+                String jarPath = valueMatcher.group(1).trim();
+                String clsName = valueMatcher.group(2).trim();
+
+                if (AGGR_COPROCESSOR_CLS_NAME.equals(clsName)) {
+                    result.add(jarPath);
+                }
+            }
+        }
+
+        return result;
+    }
+
+    private static List<String> getHTableNames(KylinConfig config) {
+        CubeManager cubeMgr = CubeManager.getInstance(config);
+
+        ArrayList<String> result = new ArrayList<String>();
+        for (CubeInstance cube : cubeMgr.listAllCubes()) {
+            for (CubeSegment seg : cube.getSegments(CubeSegmentStatusEnum.READY)) {
+                String tableName = seg.getStorageLocationIdentifier();
+                if (StringUtils.isBlank(tableName) == false)
+                    result.add(tableName);
+            }
+        }
+
+        return result;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/job/src/main/java/com/kylinolap/job/tools/HadoopStatusChecker.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/com/kylinolap/job/tools/HadoopStatusChecker.java b/job/src/main/java/com/kylinolap/job/tools/HadoopStatusChecker.java
new file mode 100644
index 0000000..c11698a
--- /dev/null
+++ b/job/src/main/java/com/kylinolap/job/tools/HadoopStatusChecker.java
@@ -0,0 +1,194 @@
+/*
+ * 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.job.tools;
+
+import java.io.IOException;
+import java.text.SimpleDateFormat;
+import java.util.Date;
+
+import org.apache.commons.httpclient.Header;
+import org.apache.commons.httpclient.HttpClient;
+import org.apache.commons.httpclient.HttpMethod;
+import org.apache.commons.httpclient.methods.GetMethod;
+import org.apache.commons.httpclient.protocol.Protocol;
+import org.apache.commons.httpclient.protocol.ProtocolSocketFactory;
+import org.apache.hadoop.mapreduce.JobStatus;
+import org.apache.hadoop.yarn.api.records.FinalApplicationStatus;
+import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppState;
+import org.codehaus.jackson.JsonNode;
+import org.codehaus.jackson.map.ObjectMapper;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.kylinolap.job.constant.JobStepStatusEnum;
+
+/**
+ * @author xduo
+ * 
+ */
+public class HadoopStatusChecker {
+
+    protected static final Logger log = LoggerFactory.getLogger(HadoopStatusChecker.class);
+
+    private final String yarnUrl;
+    private final String mrJobID;
+    private final StringBuilder output;
+
+    public HadoopStatusChecker(String yarnUrl, String mrJobID, StringBuilder output) {
+        super();
+        this.yarnUrl = yarnUrl;
+        this.mrJobID = mrJobID;
+        this.output = output;
+    }
+
+    public JobStepStatusEnum checkStatus() {
+        if (null == mrJobID) {
+            this.output.append("Skip status check with empty job id..\n");
+            return JobStepStatusEnum.WAITING;
+        }
+
+        String applicationId = mrJobID.replace("job", "application");
+        String url = yarnUrl.replace("${job_id}", applicationId);
+        JobStepStatusEnum status = null;
+        String checkResponse = null;
+        try {
+            checkResponse = getHttpResponse(url);
+            JsonNode root = new ObjectMapper().readTree(checkResponse);
+            RMAppState state = RMAppState.valueOf(root.findValue("state").getTextValue());
+            FinalApplicationStatus finalStatus = FinalApplicationStatus.valueOf(root.findValue("finalStatus").getTextValue());
+
+            log.debug("State of Hadoop job: " + mrJobID + ":" + state + "-" + finalStatus);
+            output.append(new SimpleDateFormat("yyyy-MM-dd HH:mm:ss.S").format(new Date()) + " - State of Hadoop job: " + mrJobID + ":" + state + " - " + finalStatus + "\n");
+
+            switch (finalStatus) {
+            case SUCCEEDED:
+                status = JobStepStatusEnum.FINISHED;
+                break;
+            case FAILED:
+                status = JobStepStatusEnum.ERROR;
+                break;
+            case KILLED:
+                status = JobStepStatusEnum.ERROR;
+                break;
+            case UNDEFINED:
+                switch (state) {
+                case NEW:
+                case NEW_SAVING:
+                case SUBMITTED:
+                case ACCEPTED:
+                    status = JobStepStatusEnum.WAITING;
+                    break;
+                case RUNNING:
+                    status = JobStepStatusEnum.RUNNING;
+                    break;
+                case FINAL_SAVING:
+                case FINISHING:
+                case FINISHED:
+                case FAILED:
+                case KILLING:
+                case KILLED:
+                }
+                break;
+            }
+
+        } catch (Exception e) {
+            output.append("Failed to get status from response with url + " + url + "\n");
+            output.append("Exception: " + e.getLocalizedMessage() + "\n");
+            log.error("Failed to get status from response with url + " + url + "!\n" + checkResponse, e);
+            status = JobStepStatusEnum.ERROR;
+        }
+
+        return status;
+    }
+
+    private String getHttpResponse(String url) throws IOException {
+        HttpClient client = new HttpClient();
+
+        String response = null;
+        while (response == null) { // follow redirects via 'refresh'
+            if (url.startsWith("https://")) {
+                registerEasyHttps();
+            }
+            if (url.contains("anonymous=true") == false) {
+                url += url.contains("?") ? "&" : "?";
+                url += "anonymous=true";
+            }
+
+            HttpMethod get = new GetMethod(url);
+            client.executeMethod(get);
+
+            String redirect = null;
+            Header h = get.getResponseHeader("Refresh");
+            if (h != null) {
+                String s = h.getValue();
+                int cut = s.indexOf("url=");
+                if (cut >= 0) {
+                    redirect = s.substring(cut + 4);
+                }
+            }
+
+            if (redirect == null) {
+                response = get.getResponseBodyAsString();
+                output.append("Job " + mrJobID + " get status check result.\n");
+                log.debug("Job " + mrJobID + " get status check result.\n");
+            } else {
+                url = redirect;
+                output.append("Job " + mrJobID + " check redirect url " + url + ".\n");
+                log.debug("Job " + mrJobID + " check redirect url " + url + ".\n");
+            }
+
+            get.releaseConnection();
+        }
+
+        return response;
+    }
+
+    private static Protocol EASY_HTTPS = null;
+
+    private static void registerEasyHttps() {
+        // by pass all https issue
+        if (EASY_HTTPS == null) {
+            EASY_HTTPS = new Protocol("https", (ProtocolSocketFactory) new DefaultSslProtocolSocketFactory(), 443);
+            Protocol.registerProtocol("https", EASY_HTTPS);
+        }
+    }
+
+    public JobStepStatusEnum calculateStatus(JobStatus jobStatus) {
+        JobStepStatusEnum status;
+        switch (jobStatus.getState()) {
+        case RUNNING:
+            status = JobStepStatusEnum.RUNNING;
+            break;
+        case SUCCEEDED:
+            status = JobStepStatusEnum.FINISHED;
+            break;
+        case FAILED:
+            status = JobStepStatusEnum.ERROR;
+            break;
+        case PREP:
+            status = JobStepStatusEnum.WAITING;
+            break;
+        case KILLED:
+            status = JobStepStatusEnum.ERROR;
+            break;
+        default:
+            status = JobStepStatusEnum.ERROR;
+        }
+
+        return status;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/job/src/main/java/com/kylinolap/job/tools/HtableAlterMetadataCLI.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/com/kylinolap/job/tools/HtableAlterMetadataCLI.java b/job/src/main/java/com/kylinolap/job/tools/HtableAlterMetadataCLI.java
new file mode 100644
index 0000000..66f9702
--- /dev/null
+++ b/job/src/main/java/com/kylinolap/job/tools/HtableAlterMetadataCLI.java
@@ -0,0 +1,72 @@
+package com.kylinolap.job.tools;
+
+import com.kylinolap.job.hadoop.AbstractHadoopJob;
+import org.apache.commons.cli.Option;
+import org.apache.commons.cli.OptionBuilder;
+import org.apache.commons.cli.Options;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.HBaseConfiguration;
+import org.apache.hadoop.hbase.HTableDescriptor;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.HBaseAdmin;
+import org.apache.hadoop.util.ToolRunner;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+
+/**
+ * Created by honma on 11/11/14.
+ */
+@SuppressWarnings("static-access")
+public class HtableAlterMetadataCLI extends AbstractHadoopJob {
+
+    private static final Option OPTION_METADATA_KEY = OptionBuilder.withArgName("key").hasArg().isRequired(true).withDescription("The metadata key").create("key");
+    private static final Option OPTION_METADATA_VALUE = OptionBuilder.withArgName("value").hasArg().isRequired(true).withDescription("The metadata value").create("value");
+
+    protected static final Logger log = LoggerFactory.getLogger(HtableAlterMetadataCLI.class);
+
+    String tableName;
+    String metadataKey;
+    String metadataValue;
+
+    @Override
+    public int run(String[] args) throws Exception {
+        Options options = new Options();
+        try {
+            options.addOption(OPTION_HTABLE_NAME);
+            options.addOption(OPTION_METADATA_KEY);
+            options.addOption(OPTION_METADATA_VALUE);
+
+            parseOptions(options, args);
+            tableName = getOptionValue(OPTION_HTABLE_NAME);
+            metadataKey = getOptionValue(OPTION_METADATA_KEY);
+            metadataValue = getOptionValue(OPTION_METADATA_VALUE);
+
+            alter();
+
+            return 0;
+        } catch (Exception e) {
+            e.printStackTrace(System.err);
+            log.error(e.getLocalizedMessage(), e);
+            return 2;
+        }
+    }
+
+    private void alter() throws IOException {
+        Configuration conf = HBaseConfiguration.create();
+        HBaseAdmin hbaseAdmin = new HBaseAdmin(conf);
+        HTableDescriptor table = hbaseAdmin.getTableDescriptor(TableName.valueOf(tableName));
+
+        hbaseAdmin.disableTable(table.getTableName());
+        table.setValue(metadataKey, metadataValue);
+        hbaseAdmin.modifyTable(table.getTableName(), table);
+        hbaseAdmin.enableTable(table.getTableName());
+        hbaseAdmin.close();
+    }
+
+    public static void main(String[] args) throws Exception {
+        int exitCode = ToolRunner.run(new HtableAlterMetadataCLI(), args);
+        System.exit(exitCode);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/job/src/main/java/com/kylinolap/job/tools/LZOSupportnessChecker.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/com/kylinolap/job/tools/LZOSupportnessChecker.java b/job/src/main/java/com/kylinolap/job/tools/LZOSupportnessChecker.java
new file mode 100644
index 0000000..62dc594
--- /dev/null
+++ b/job/src/main/java/com/kylinolap/job/tools/LZOSupportnessChecker.java
@@ -0,0 +1,21 @@
+package com.kylinolap.job.tools;
+
+import org.apache.hadoop.hbase.util.CompressionTest;
+
+import java.io.File;
+
+/**
+ * Created by honma on 10/21/14.
+ */
+public class LZOSupportnessChecker {
+    public static boolean getSupportness()
+    {
+        try {
+            File temp = File.createTempFile("test", ".tmp");
+            CompressionTest.main(new String[] { "file://" + temp.toString(), "lzo" });
+        } catch (Exception e) {
+            return false;
+        }
+        return true;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/job/src/main/java/com/kylinolap/job/tools/OptionsHelper.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/com/kylinolap/job/tools/OptionsHelper.java b/job/src/main/java/com/kylinolap/job/tools/OptionsHelper.java
new file mode 100644
index 0000000..ea0b7e3
--- /dev/null
+++ b/job/src/main/java/com/kylinolap/job/tools/OptionsHelper.java
@@ -0,0 +1,78 @@
+/*
+ * 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.job.tools;
+
+import java.io.File;
+
+import org.apache.commons.cli.CommandLine;
+import org.apache.commons.cli.CommandLineParser;
+import org.apache.commons.cli.GnuParser;
+import org.apache.commons.cli.HelpFormatter;
+import org.apache.commons.cli.Option;
+import org.apache.commons.cli.Options;
+import org.apache.commons.cli.ParseException;
+
+/**
+ * @author George Song (ysong1)
+ * 
+ */
+public class OptionsHelper {
+    private CommandLine commandLine;
+
+    public void parseOptions(Options options, String[] args) throws ParseException {
+        CommandLineParser parser = new GnuParser();
+        commandLine = parser.parse(options, args);
+    }
+
+    public Option[] getOptions() {
+        return commandLine.getOptions();
+    }
+
+    public String getOptionsAsString() {
+        StringBuilder buf = new StringBuilder();
+        for (Option option : commandLine.getOptions()) {
+            buf.append(" ");
+            buf.append(option.getOpt());
+            if (option.hasArg()) {
+                buf.append("=");
+                buf.append(option.getValue());
+            }
+        }
+        return buf.toString();
+    }
+
+    public String getOptionValue(Option option) {
+        return commandLine.getOptionValue(option.getOpt());
+    }
+
+    public boolean hasOption(Option option) {
+        return commandLine.hasOption(option.getOpt());
+    }
+
+    public void printUsage(String programName, Options options) {
+        HelpFormatter formatter = new HelpFormatter();
+        formatter.printHelp(programName, options);
+    }
+
+    public static String convertToFileURL(String path) {
+        if (File.separatorChar != '/') {
+            path = path.replace(File.separatorChar, '/');
+        }
+
+        return path;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/job/src/test/java/com/kylinolap/job/BuildCubeWithEngineTest.java
----------------------------------------------------------------------
diff --git a/job/src/test/java/com/kylinolap/job/BuildCubeWithEngineTest.java b/job/src/test/java/com/kylinolap/job/BuildCubeWithEngineTest.java
new file mode 100644
index 0000000..dc56ad8
--- /dev/null
+++ b/job/src/test/java/com/kylinolap/job/BuildCubeWithEngineTest.java
@@ -0,0 +1,252 @@
+/*
+ * 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.job;
+
+import static org.junit.Assert.*;
+
+import java.io.File;
+import java.io.IOException;
+import java.text.SimpleDateFormat;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.TimeZone;
+
+import org.junit.After;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.quartz.SchedulerException;
+
+import com.google.common.collect.Lists;
+import com.kylinolap.common.KylinConfig;
+import com.kylinolap.common.util.ClasspathUtil;
+import com.kylinolap.common.util.HBaseMetadataTestCase;
+import com.kylinolap.common.util.JsonUtil;
+import com.kylinolap.cube.CubeBuildTypeEnum;
+import com.kylinolap.cube.CubeInstance;
+import com.kylinolap.cube.CubeManager;
+import com.kylinolap.cube.CubeSegment;
+import com.kylinolap.cube.exception.CubeIntegrityException;
+import com.kylinolap.job.constant.JobStatusEnum;
+import com.kylinolap.job.engine.JobEngineConfig;
+import com.kylinolap.job.exception.InvalidJobInstanceException;
+
+/**
+ * @author ysong1
+ */
+public class BuildCubeWithEngineTest extends HBaseMetadataTestCase {
+
+    protected JobManager jobManager;
+    protected JobEngineConfig engineConfig;
+
+    @BeforeClass
+    public static void beforeClass() throws Exception {
+        ClasspathUtil.addClasspath(new File(SANDBOX_TEST_DATA).getAbsolutePath());
+    }
+
+    @Before
+    public void before() throws Exception {
+        this.createTestMetadata();
+
+        DeployUtil.initCliWorkDir();
+        DeployUtil.deployMetadata();
+        DeployUtil.overrideJobJarLocations();
+        DeployUtil.overrideJobConf(SANDBOX_TEST_DATA);
+
+        engineConfig = new JobEngineConfig(KylinConfig.getInstanceFromEnv());
+        jobManager = new JobManager("Build_Test_Cube_Engine", engineConfig);
+        jobManager.deleteAllJobs();
+    }
+
+    @After
+    public void after() throws IOException {
+        // jobManager.deleteAllJobs();
+        this.cleanupTestMetadata();
+    }
+
+    @Test
+    public void testCubes() throws Exception {
+
+        // start job schedule engine
+        jobManager.startJobEngine(10);
+
+//        testSimpleLeftJoinCube();
+        
+        // keep this order.
+        testLeftJoinCube();
+        testInnerJoinCube();
+
+        jobManager.stopJobEngine();
+    }
+
+    /**
+     * For cube test_kylin_cube_with_slr_empty, we will create 2 segments For
+     * cube test_kylin_cube_without_slr_empty, since it doesn't support
+     * incremental build, we will create only 1 segment (full build)
+     *
+     * @throws Exception
+     */
+    private void testInnerJoinCube() throws Exception {
+        DeployUtil.prepareTestData("inner", "test_kylin_cube_with_slr_empty");
+
+        SimpleDateFormat f = new SimpleDateFormat("yyyy-MM-dd");
+        f.setTimeZone(TimeZone.getTimeZone("GMT"));
+        long dateStart;
+        long dateEnd;
+
+        ArrayList<String> jobs = new ArrayList<String>();
+
+        // this cube's start date is 0, end date is 20501112000000
+        dateStart = 0;
+        dateEnd = f.parse("2013-01-01").getTime();
+        jobs.addAll(this.submitJob("test_kylin_cube_with_slr_empty", dateStart, dateEnd, CubeBuildTypeEnum.BUILD));
+
+        // this cube doesn't support incremental build, always do full build
+        jobs.addAll(this.submitJob("test_kylin_cube_without_slr_empty", 0, 0, CubeBuildTypeEnum.BUILD));
+
+        waitCubeBuilt(jobs);
+
+        // then submit a incremental job, start date is 20130101000000, end date
+        // is 20220101000000
+        jobs.clear();
+        dateStart = f.parse("2013-01-01").getTime();
+        dateEnd = f.parse("2022-01-01").getTime();
+        jobs.addAll(this.submitJob("test_kylin_cube_with_slr_empty", dateStart, dateEnd, CubeBuildTypeEnum.BUILD));
+        waitCubeBuilt(jobs);
+    }
+
+    /**
+     * For cube test_kylin_cube_without_slr_left_join_empty, it is using
+     * update_insert, we will create 2 segments, and then merge these 2 segments
+     * into a larger segment For cube test_kylin_cube_with_slr_left_join_empty,
+     * we will create only 1 segment
+     *
+     * @throws Exception
+     */
+    private void testLeftJoinCube() throws Exception {
+        DeployUtil.prepareTestData("left", "test_kylin_cube_with_slr_left_join_empty");
+
+        SimpleDateFormat f = new SimpleDateFormat("yyyy-MM-dd");
+        f.setTimeZone(TimeZone.getTimeZone("GMT"));
+        long dateStart;
+        long dateEnd;
+
+        ArrayList<String> jobs = new ArrayList<String>();
+
+        // this cube's start date is 0, end date is 20501112000000
+        CubeManager cubeMgr = CubeManager.getInstance(KylinConfig.getInstanceFromEnv());
+        dateStart = cubeMgr.getCube("test_kylin_cube_with_slr_left_join_empty").getDescriptor().getCubePartitionDesc().getPartitionDateStart();
+        dateEnd = f.parse("2050-11-12").getTime();
+        jobs.addAll(this.submitJob("test_kylin_cube_with_slr_left_join_empty", dateStart, dateEnd, CubeBuildTypeEnum.BUILD));
+
+        // this cube's start date is 0, end date is 20120601000000
+        dateStart = cubeMgr.getCube("test_kylin_cube_without_slr_left_join_empty").getDescriptor().getCubePartitionDesc().getPartitionDateStart();
+        dateEnd = f.parse("2012-06-01").getTime();
+        jobs.addAll(this.submitJob("test_kylin_cube_without_slr_left_join_empty", dateStart, dateEnd, CubeBuildTypeEnum.BUILD));
+
+        waitCubeBuilt(jobs);
+
+        jobs.clear();
+        // then submit a update_insert job, start date is 20120101000000, end
+        // date is 20220101000000
+        dateStart = f.parse("2012-03-01").getTime();
+        dateEnd = f.parse("2022-01-01").getTime();
+        jobs.addAll(this.submitJob("test_kylin_cube_without_slr_left_join_empty", dateStart, dateEnd, CubeBuildTypeEnum.BUILD));
+
+        waitCubeBuilt(jobs);
+
+        jobs.clear();
+
+        // final submit a merge job, start date is 0, end date is 20220101000000
+        //dateEnd = f.parse("2022-01-01").getTime();
+        //jobs.addAll(this.submitJob("test_kylin_cube_without_slr_left_join_empty", 0, dateEnd, CubeBuildTypeEnum.MERGE));
+        //waitCubeBuilt(jobs);
+    }
+
+    @SuppressWarnings("unused")
+    private void testSimpleLeftJoinCube() throws Exception {
+        DeployUtil.prepareTestData("left", "test_kylin_cube_with_slr_left_join_empty");
+        
+        SimpleDateFormat f = new SimpleDateFormat("yyyy-MM-dd");
+        f.setTimeZone(TimeZone.getTimeZone("GMT"));
+        long dateStart;
+        long dateEnd;
+        
+        ArrayList<String> jobs = new ArrayList<String>();
+        
+        // this cube's start date is 0, end date is 20501112000000
+        CubeManager cubeMgr = CubeManager.getInstance(KylinConfig.getInstanceFromEnv());
+        dateStart = cubeMgr.getCube("test_kylin_cube_with_slr_left_join_empty").getDescriptor().getCubePartitionDesc().getPartitionDateStart();
+        dateEnd = f.parse("2050-11-12").getTime();
+        jobs.addAll(this.submitJob("test_kylin_cube_with_slr_left_join_empty", dateStart, dateEnd, CubeBuildTypeEnum.BUILD));
+        
+        // this cube's start date is 0, end date is 20501112000000
+        dateStart = cubeMgr.getCube("test_kylin_cube_without_slr_left_join_empty").getDescriptor().getCubePartitionDesc().getPartitionDateStart();
+        dateEnd = f.parse("2050-11-12").getTime();
+        jobs.addAll(this.submitJob("test_kylin_cube_without_slr_left_join_empty", dateStart, dateEnd, CubeBuildTypeEnum.BUILD));
+        
+        waitCubeBuilt(jobs);
+    }
+    
+    protected void waitCubeBuilt(List<String> jobs) throws Exception {
+
+        boolean allFinished = false;
+        while (!allFinished) {
+            // sleep for 1 minutes
+            Thread.sleep(60 * 1000L);
+
+            allFinished = true;
+            for (String job : jobs) {
+                JobInstance savedJob = jobManager.getJob(job);
+                JobStatusEnum jobStatus = savedJob.getStatus();
+                if (jobStatus.getCode() <= JobStatusEnum.RUNNING.getCode()) {
+                    allFinished = false;
+                    break;
+                }
+            }
+        }
+
+        for (String job : jobs)
+            assertEquals("Job fail - " + job, JobStatusEnum.FINISHED, jobManager.getJob(job).getStatus());
+    }
+
+    protected List<String> submitJob(String cubename, long startDate, long endDate, CubeBuildTypeEnum jobType) throws SchedulerException, IOException, InvalidJobInstanceException, CubeIntegrityException {
+
+        CubeManager cubeMgr = CubeManager.getInstance(KylinConfig.getInstanceFromEnv());
+        CubeInstance cube = cubeMgr.getCube(cubename);
+        CubeManager.getInstance(this.getTestConfig()).loadCubeCache(cube);
+
+        System.out.println(JsonUtil.writeValueAsIndentString(cube));
+        List<CubeSegment> newSegments = cubeMgr.allocateSegments(cube, jobType, startDate, endDate);
+        System.out.println(JsonUtil.writeValueAsIndentString(cube));
+
+        List<String> jobUuids = Lists.newArrayList();
+        List<JobInstance> jobs = Lists.newArrayList();
+        for (CubeSegment seg : newSegments) {
+            String uuid = seg.getUuid();
+            jobUuids.add(uuid);
+            jobs.add(jobManager.createJob(cubename, seg.getName(), uuid, jobType,"KylinTest"));
+            seg.setLastBuildJobID(uuid);
+        }
+        cubeMgr.updateCube(cube);
+        for (JobInstance job: jobs) {
+            // submit job to store
+            jobManager.submitJob(job);
+        }
+        return jobUuids;
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/job/src/test/java/com/kylinolap/job/DeployUtil.java
----------------------------------------------------------------------
diff --git a/job/src/test/java/com/kylinolap/job/DeployUtil.java b/job/src/test/java/com/kylinolap/job/DeployUtil.java
new file mode 100644
index 0000000..80774b5
--- /dev/null
+++ b/job/src/test/java/com/kylinolap/job/DeployUtil.java
@@ -0,0 +1,216 @@
+package com.kylinolap.job;
+
+import java.io.File;
+import java.io.FileOutputStream;
+import java.io.FileReader;
+import java.io.IOException;
+import java.io.InputStream;
+
+import org.apache.commons.io.IOUtils;
+import org.apache.hadoop.hbase.util.Pair;
+import org.apache.maven.model.Model;
+import org.apache.maven.model.io.xpp3.MavenXpp3Reader;
+import org.codehaus.plexus.util.FileUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.kylinolap.common.KylinConfig;
+import com.kylinolap.common.persistence.ResourceTool;
+import com.kylinolap.common.util.AbstractKylinTestCase;
+import com.kylinolap.common.util.CliCommandExecutor;
+import com.kylinolap.cube.CubeInstance;
+import com.kylinolap.cube.CubeManager;
+import com.kylinolap.cube.dataGen.FactTableGenerator;
+import com.kylinolap.job.engine.JobEngineConfig;
+import com.kylinolap.job.hadoop.hive.SqlHiveDataTypeMapping;
+import com.kylinolap.job.tools.LZOSupportnessChecker;
+import com.kylinolap.metadata.MetadataManager;
+import com.kylinolap.metadata.model.schema.ColumnDesc;
+import com.kylinolap.metadata.model.schema.TableDesc;
+
+public class DeployUtil {
+    @SuppressWarnings("unused")
+    private static final Logger logger = LoggerFactory.getLogger(DeployUtil.class);
+
+    public static void initCliWorkDir() throws IOException {
+        execCliCommand("rm -rf " + getHadoopCliWorkingDir());
+        execCliCommand("mkdir -p " + config().getKylinJobLogDir());
+    }
+
+    public static void deployMetadata() throws IOException {
+        // install metadata to hbase
+        ResourceTool.reset(config());
+        ResourceTool.copy(KylinConfig.createInstanceFromUri(AbstractKylinTestCase.LOCALMETA_TEST_DATA), config());
+
+        // update cube desc signature.
+        for (CubeInstance cube : CubeManager.getInstance(config()).listAllCubes()) {
+            cube.getDescriptor().setSignature(cube.getDescriptor().calculateSignature());
+            CubeManager.getInstance(config()).updateCube(cube);
+        }
+    }
+
+    public static void overrideJobJarLocations() {
+        Pair<File, File> files = getJobJarFiles();
+        File jobJar = files.getFirst();
+        File coprocessorJar = files.getSecond();
+
+        config().overrideKylinJobJarPath(jobJar.getAbsolutePath());
+        config().overrideCoprocessorLocalJar(coprocessorJar.getAbsolutePath());
+    }
+    
+    public static void deployJobJars() throws IOException {
+        Pair<File, File> files = getJobJarFiles();
+        File jobJar = files.getFirst();
+        File coprocessorJar = files.getSecond();
+
+        File jobJarRemote = new File(config().getKylinJobJarPath());
+        File jobJarLocal = new File(jobJar.getParentFile(), jobJarRemote.getName());
+        if (jobJar.equals(jobJarLocal) == false) {
+            FileUtils.copyFile(jobJar, jobJarLocal);
+        }
+        
+        File coprocessorJarRemote = new File(config().getCoprocessorLocalJar());
+        File coprocessorJarLocal = new File(coprocessorJar.getParentFile(), coprocessorJarRemote.getName());
+        if (coprocessorJar.equals(coprocessorJarLocal) == false) {
+            FileUtils.copyFile(coprocessorJar, coprocessorJarLocal);
+        }
+        
+        CliCommandExecutor cmdExec = config().getCliCommandExecutor();
+        cmdExec.copyFile(jobJarLocal.getAbsolutePath(), jobJarRemote.getParent());
+        cmdExec.copyFile(coprocessorJar.getAbsolutePath(), coprocessorJarRemote.getParent());
+    }
+    
+    private static Pair<File, File> getJobJarFiles() {
+        String version;
+        try {
+            MavenXpp3Reader pomReader = new MavenXpp3Reader();
+            Model model = pomReader.read(new FileReader("../pom.xml"));
+            version = model.getVersion();
+        } catch (Exception e) {
+            throw new RuntimeException(e.getMessage(), e);
+        }
+
+        File jobJar = new File("../job/target", "kylin-job-" + version + "-job.jar");
+        File coprocessorJar = new File("../storage/target", "kylin-storage-" + version + "-coprocessor.jar");
+        return new Pair<File, File>(jobJar, coprocessorJar);
+    }
+    
+    public static void overrideJobConf(String confDir) throws IOException {
+        boolean enableLzo = LZOSupportnessChecker.getSupportness();
+        overrideJobConf(confDir, enableLzo);
+    }
+
+    public static void overrideJobConf(String confDir, boolean enableLzo) throws IOException {
+        File src = new File(confDir, JobEngineConfig.HADOOP_JOB_CONF_FILENAME + (enableLzo ? ".lzo_enabled" : ".lzo_disabled") + ".xml");
+        File dst = new File(confDir, JobEngineConfig.HADOOP_JOB_CONF_FILENAME + ".xml");
+        FileUtils.copyFile(src, dst);
+    }
+
+    private static void execCliCommand(String cmd) throws IOException {
+        config().getCliCommandExecutor().execute(cmd);
+    }
+
+    private static String getHadoopCliWorkingDir() {
+        return config().getCliWorkingDir();
+    }
+
+    private static KylinConfig config() {
+        return KylinConfig.getInstanceFromEnv();
+    }
+
+    // ============================================================================
+
+    static final String TABLE_CAL_DT = "test_cal_dt";
+    static final String TABLE_CATEGORY_GROUPINGS = "test_category_groupings";
+    static final String TABLE_KYLIN_FACT = "test_kylin_fact";
+    static final String TABLE_SELLER_TYPE_DIM = "test_seller_type_dim";
+    static final String TABLE_SITES = "test_sites";
+
+    static final String[] TABLE_NAMES = new String[] { TABLE_CAL_DT, TABLE_CATEGORY_GROUPINGS, TABLE_KYLIN_FACT, TABLE_SELLER_TYPE_DIM, TABLE_SITES };
+
+    public static void prepareTestData(String joinType, String cubeName) throws Exception {
+        // data is generated according to cube descriptor and saved in resource store
+        if (joinType.equalsIgnoreCase("inner")) {
+            FactTableGenerator.generate(cubeName, "10000", "1", null, "inner");
+        } else if (joinType.equalsIgnoreCase("left")) {
+            FactTableGenerator.generate(cubeName, "10000", "0.6", null, "left");
+        } else {
+            throw new IllegalArgumentException("Unsupported join type : " + joinType);
+        }
+
+        deployHiveTables();
+    }
+
+    private static void deployHiveTables() throws Exception {
+
+        MetadataManager metaMgr = MetadataManager.getInstance(config());
+
+        // scp data files, use the data from hbase, instead of local files
+        File temp = File.createTempFile("temp", ".csv");
+        temp.createNewFile();
+        for (String tablename : TABLE_NAMES) {
+            tablename = tablename.toUpperCase();
+
+            File localBufferFile = new File(temp.getParent() + "/" + tablename + ".csv");
+            localBufferFile.createNewFile();
+
+            InputStream hbaseDataStream = metaMgr.getStore().getResource("/data/" + tablename + ".csv");
+            FileOutputStream localFileStream = new FileOutputStream(localBufferFile);
+            IOUtils.copy(hbaseDataStream, localFileStream);
+
+            hbaseDataStream.close();
+            localFileStream.close();
+
+            config().getCliCommandExecutor().copyFile(localBufferFile.getPath(), config().getCliWorkingDir());
+            localBufferFile.delete();
+        }
+        temp.delete();
+
+        // create hive tables
+        execHiveCommand(generateCreateTableHql(metaMgr.getTableDesc(TABLE_CAL_DT.toUpperCase())));
+        execHiveCommand(generateCreateTableHql(metaMgr.getTableDesc(TABLE_CATEGORY_GROUPINGS.toUpperCase())));
+        execHiveCommand(generateCreateTableHql(metaMgr.getTableDesc(TABLE_KYLIN_FACT.toUpperCase())));
+        execHiveCommand(generateCreateTableHql(metaMgr.getTableDesc(TABLE_SELLER_TYPE_DIM.toUpperCase())));
+        execHiveCommand(generateCreateTableHql(metaMgr.getTableDesc(TABLE_SITES.toUpperCase())));
+
+        // load data to hive tables
+        // LOAD DATA LOCAL INPATH 'filepath' [OVERWRITE] INTO TABLE tablename
+        execHiveCommand(generateLoadDataHql(TABLE_CAL_DT));
+        execHiveCommand(generateLoadDataHql(TABLE_CATEGORY_GROUPINGS));
+        execHiveCommand(generateLoadDataHql(TABLE_KYLIN_FACT));
+        execHiveCommand(generateLoadDataHql(TABLE_SELLER_TYPE_DIM));
+        execHiveCommand(generateLoadDataHql(TABLE_SITES));
+    }
+
+    private static void execHiveCommand(String hql) throws IOException {
+        String hiveCmd = "hive -e \"" + hql + "\"";
+        config().getCliCommandExecutor().execute(hiveCmd);
+    }
+
+    private static String generateLoadDataHql(String tableName) {
+        return "LOAD DATA LOCAL INPATH '" + config().getCliWorkingDir() + "/" + tableName.toUpperCase() + ".csv' OVERWRITE INTO TABLE " + tableName.toUpperCase();
+    }
+
+    private static String generateCreateTableHql(TableDesc tableDesc) {
+        StringBuilder ddl = new StringBuilder();
+
+        ddl.append("DROP TABLE IF EXISTS " + tableDesc.getName() + ";\n");
+        ddl.append("CREATE TABLE " + tableDesc.getName() + "\n");
+        ddl.append("(" + "\n");
+
+        for (int i = 0; i < tableDesc.getColumns().length; i++) {
+            ColumnDesc col = tableDesc.getColumns()[i];
+            if (i > 0) {
+                ddl.append(",");
+            }
+            ddl.append(col.getName() + " " + SqlHiveDataTypeMapping.getHiveDataType((col.getDatatype())) + "\n");
+        }
+
+        ddl.append(")" + "\n");
+        ddl.append("ROW FORMAT DELIMITED FIELDS TERMINATED BY ','" + "\n");
+        ddl.append("STORED AS TEXTFILE;");
+
+        return ddl.toString();
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/job/src/test/java/com/kylinolap/job/JobDAOTest.java
----------------------------------------------------------------------
diff --git a/job/src/test/java/com/kylinolap/job/JobDAOTest.java b/job/src/test/java/com/kylinolap/job/JobDAOTest.java
new file mode 100644
index 0000000..f09c68e
--- /dev/null
+++ b/job/src/test/java/com/kylinolap/job/JobDAOTest.java
@@ -0,0 +1,107 @@
+/*
+ * 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.job;
+
+import static org.junit.Assert.*;
+
+import java.io.IOException;
+
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.databind.SerializationFeature;
+import com.kylinolap.common.util.LocalFileMetadataTestCase;
+import com.kylinolap.cube.CubeBuildTypeEnum;
+import com.kylinolap.job.exception.InvalidJobInstanceException;
+
+/**
+ * @author George Song (ysong1)
+ * 
+ */
+public class JobDAOTest extends LocalFileMetadataTestCase {
+
+    @Before
+    public void setup() throws Exception {
+        createTestMetadata();
+    }
+
+    @After
+    public void after() throws Exception {
+        cleanupTestMetadata();
+    }
+
+    @Test
+    public void test() throws IOException, InvalidJobInstanceException {
+
+        String uuid = "132432cb-8c68-42d8-aa3a-504151b39d1b";
+        JobDAO service = JobDAO.getInstance(getTestConfig());
+        JobInstance job = createDumbJobInstance(uuid);
+        assertEquals(0, job.getLastModified());
+        service.updateJobInstance(job);
+
+        // test read
+        JobInstance job2 = service.getJob(uuid);
+        // assertEquals(JobStatusEnum.PENDING, job2.getStatus());
+        assertTrue(job2.getLastModified() > 0);
+
+        // test modify
+        job2.setRelatedCube("abc");
+        service.updateJobInstance(job2);
+        JobInstance job3 = service.getJob(uuid);
+        assertEquals("abc", job3.getRelatedCube());
+        assertTrue(job3.getLastModified() > 0);
+
+        // test delete
+        service.deleteJob(job2);
+        JobInstance job4 = service.getJob(uuid);
+        assertNull(job4);
+    }
+
+    @Test
+    public void testOutput() throws IOException, InvalidJobInstanceException {
+        String uuid = "132432cb-8c68-42d8-aa3a-504151b39d1b";
+        int seq = 1;
+        String s = "this is output";
+        JobDAO service = JobDAO.getInstance(getTestConfig());
+        service.saveJobOutput(uuid, seq, s);
+
+        // test read
+        JobStepOutput output2 = service.getJobOutput(uuid, seq);
+        assertTrue(output2.getLastModified() > 0);
+        assertEquals(s, output2.getOutput());
+
+    }
+
+    private JobInstance createDumbJobInstance(String uuid) {
+        try {
+            ObjectMapper mapper = new ObjectMapper();
+            mapper.configure(SerializationFeature.INDENT_OUTPUT, true);
+
+            JobInstance jobInstance = new JobInstance();
+            jobInstance.setUuid(uuid);
+            jobInstance.setType(CubeBuildTypeEnum.BUILD);
+            jobInstance.setRelatedCube("test_kylin_cube_with_slr".toUpperCase());
+            jobInstance.setName("Dummy_Job");
+            // jobInstance.setStatus(JobStatusEnum.PENDING);
+
+            return jobInstance;
+        } catch (Exception e) {
+            return null;
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/job/src/test/java/com/kylinolap/job/JobInstanceTest.java
----------------------------------------------------------------------
diff --git a/job/src/test/java/com/kylinolap/job/JobInstanceTest.java b/job/src/test/java/com/kylinolap/job/JobInstanceTest.java
new file mode 100644
index 0000000..a1c04ef
--- /dev/null
+++ b/job/src/test/java/com/kylinolap/job/JobInstanceTest.java
@@ -0,0 +1,101 @@
+/*
+ * 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.job;
+
+import static org.junit.Assert.*;
+
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+import com.kylinolap.common.KylinConfig;
+import com.kylinolap.common.util.LocalFileMetadataTestCase;
+import com.kylinolap.cube.CubeBuildTypeEnum;
+import com.kylinolap.job.JobInstance.JobStep;
+import com.kylinolap.job.constant.JobStatusEnum;
+import com.kylinolap.job.constant.JobStepStatusEnum;
+import com.kylinolap.job.engine.JobEngineConfig;
+
+import java.util.UUID;
+
+/**
+ * @author ysong1
+ * 
+ */
+public class JobInstanceTest extends LocalFileMetadataTestCase {
+    @Before
+    public void before() throws Exception {
+        this.createTestMetadata();
+    }
+
+    @After
+    public void after() throws Exception {
+        this.cleanupTestMetadata();
+    }
+
+    @Test
+    public void testJobInstanceStatus() throws Exception {
+        KylinConfig kylinCfg = KylinConfig.getInstanceFromEnv();
+        JobManager jobManager = new JobManager("JobInstanceTest", new JobEngineConfig(kylinCfg));
+
+        JobInstance jobInstance = jobManager.createJob("test_kylin_cube_with_slr_1_new_segment", "20130331080000_20131212080000", UUID.randomUUID().toString(), CubeBuildTypeEnum.BUILD,"KylinTest");
+        // initial job status should be PENDING
+        assertEquals(JobStatusEnum.PENDING, jobInstance.getStatus());
+
+        // if a step fails, job status should be ERROR
+        jobInstance.getSteps().get(3).setStatus(JobStepStatusEnum.ERROR);
+        assertEquals(JobStatusEnum.ERROR, jobInstance.getStatus());
+
+        // then resume job, job status should be NEW
+        jobInstance.getSteps().get(0).setStatus(JobStepStatusEnum.FINISHED);
+        jobInstance.getSteps().get(1).setStatus(JobStepStatusEnum.FINISHED);
+        jobInstance.getSteps().get(2).setStatus(JobStepStatusEnum.FINISHED);
+        jobInstance.getSteps().get(3).setStatus(JobStepStatusEnum.PENDING);
+        assertEquals(JobStatusEnum.PENDING, jobInstance.getStatus());
+
+        // running job
+        jobInstance.getSteps().get(0).setStatus(JobStepStatusEnum.FINISHED);
+        jobInstance.getSteps().get(1).setStatus(JobStepStatusEnum.FINISHED);
+        jobInstance.getSteps().get(2).setStatus(JobStepStatusEnum.FINISHED);
+        jobInstance.getSteps().get(3).setStatus(JobStepStatusEnum.RUNNING);
+        assertEquals(JobStatusEnum.RUNNING, jobInstance.getStatus());
+
+        // kill job
+        jobInstance.getSteps().get(0).setStatus(JobStepStatusEnum.FINISHED);
+        jobInstance.getSteps().get(1).setStatus(JobStepStatusEnum.FINISHED);
+        jobInstance.getSteps().get(2).setStatus(JobStepStatusEnum.FINISHED);
+        jobInstance.getSteps().get(3).setStatus(JobStepStatusEnum.DISCARDED);
+        assertEquals(JobStatusEnum.DISCARDED, jobInstance.getStatus());
+
+        // finish job
+        for (JobStep step : jobInstance.getSteps()) {
+            step.setStatus(JobStepStatusEnum.FINISHED);
+        }
+        assertEquals(JobStatusEnum.FINISHED, jobInstance.getStatus());
+
+        // finish job
+        for (JobStep step : jobInstance.getSteps()) {
+            step.setStatus(JobStepStatusEnum.NEW);
+        }
+        assertEquals(JobStatusEnum.NEW, jobInstance.getStatus());
+
+        // default
+        jobInstance.getSteps().get(3).setStatus(JobStepStatusEnum.WAITING);
+        assertEquals(JobStatusEnum.RUNNING, jobInstance.getStatus());
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/job/src/test/java/com/kylinolap/job/SampleCubeSetupTest.java
----------------------------------------------------------------------
diff --git a/job/src/test/java/com/kylinolap/job/SampleCubeSetupTest.java b/job/src/test/java/com/kylinolap/job/SampleCubeSetupTest.java
new file mode 100644
index 0000000..30ed697
--- /dev/null
+++ b/job/src/test/java/com/kylinolap/job/SampleCubeSetupTest.java
@@ -0,0 +1,84 @@
+package com.kylinolap.job;
+
+import java.io.File;
+import java.io.IOException;
+
+import org.apache.commons.lang3.StringUtils;
+import org.codehaus.plexus.util.FileUtils;
+import org.junit.Before;
+import org.junit.Test;
+
+import com.kylinolap.common.KylinConfig;
+import com.kylinolap.common.util.ClasspathUtil;
+import com.kylinolap.common.util.HBaseMetadataTestCase;
+import com.kylinolap.cube.CubeInstance;
+import com.kylinolap.cube.CubeManager;
+import com.kylinolap.job.engine.JobEngineConfig;
+
+/**
+ * Created by honma on 9/24/14.
+ * <p/>
+ * This class is only used for building a sample cube in the one-line deployment tool.
+ */
+public class SampleCubeSetupTest extends HBaseMetadataTestCase {
+
+    @Before
+    public void before() throws Exception {
+
+        try {
+            this.testConnectivity();
+        } catch (Exception e) {
+            System.out.println("Failed to connect to remote CLI with given password");
+            throw e;
+        }
+
+        String confPaths = System.getenv("KYLIN_HBASE_CONF_PATH");
+        System.out.println("The conf paths is " + confPaths);
+        if (confPaths != null) {
+            String[] paths = confPaths.split(":");
+            for (String path : paths) {
+                if (!StringUtils.isEmpty(path)) {
+                    try {
+                        ClasspathUtil.addClasspath(new File(path).getAbsolutePath());
+                    } catch (Exception e) {
+                        System.out.println(e.getLocalizedMessage());
+                        System.out.println(e.getStackTrace());
+                    }
+                }
+            }
+        }
+    }
+
+    private void testConnectivity() throws Exception {
+        KylinConfig cfg = KylinConfig.getInstanceFromEnv();
+        cfg.getCliCommandExecutor().execute("echo hello");
+    }
+
+    @Test
+    public void testCubes() throws Exception {
+        DeployUtil.initCliWorkDir();
+        DeployUtil.deployMetadata();
+        DeployUtil.deployJobJars();
+        deployJobConfToEtc();
+        DeployUtil.prepareTestData("inner", "test_kylin_cube_with_slr_empty");
+
+        // remove all other cubes to keep it clean
+        CubeManager cubeManager = CubeManager.getInstance(KylinConfig.getInstanceFromEnv());
+        for (CubeInstance cubeInstance : cubeManager.listAllCubes()) {
+            if (!cubeInstance.getName().equalsIgnoreCase("test_kylin_cube_without_slr_empty") && !cubeInstance.getName().equalsIgnoreCase("test_kylin_cube_with_slr_empty"))
+                cubeManager.dropCube(cubeInstance.getName(), false);
+        }
+
+    }
+
+    private void deployJobConfToEtc() throws IOException {
+        String lzoSupportness = System.getenv("KYLIN_LZO_SUPPORTED");
+        boolean enableLzo = "true".equalsIgnoreCase(lzoSupportness);
+        DeployUtil.overrideJobConf(SANDBOX_TEST_DATA, enableLzo);
+        
+        File src = new File(SANDBOX_TEST_DATA, JobEngineConfig.HADOOP_JOB_CONF_FILENAME + ".xml");
+        File dst = new File("/etc/kylin", src.getName());
+        FileUtils.copyFile(src, dst);
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/job/src/test/java/com/kylinolap/job/engine/GenericJobEngineTest.java
----------------------------------------------------------------------
diff --git a/job/src/test/java/com/kylinolap/job/engine/GenericJobEngineTest.java b/job/src/test/java/com/kylinolap/job/engine/GenericJobEngineTest.java
new file mode 100644
index 0000000..47974fc
--- /dev/null
+++ b/job/src/test/java/com/kylinolap/job/engine/GenericJobEngineTest.java
@@ -0,0 +1,441 @@
+/*
+ * 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.job.engine;
+
+import static org.junit.Assert.*;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.commons.io.FileUtils;
+import org.junit.AfterClass;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import com.kylinolap.common.KylinConfig;
+import com.kylinolap.common.util.AbstractKylinTestCase;
+import com.kylinolap.common.util.JsonUtil;
+import com.kylinolap.common.util.SSHClient;
+import com.kylinolap.cube.CubeBuildTypeEnum;
+import com.kylinolap.cube.CubeInstance;
+import com.kylinolap.cube.CubeManager;
+import com.kylinolap.cube.CubeSegment;
+import com.kylinolap.cube.CubeSegmentStatusEnum;
+import com.kylinolap.cube.exception.CubeIntegrityException;
+import com.kylinolap.cube.project.ProjectManager;
+import com.kylinolap.job.JobDAO;
+import com.kylinolap.job.JobInstance;
+import com.kylinolap.job.JobInstance.JobStep;
+import com.kylinolap.job.JobManager;
+import com.kylinolap.job.constant.JobConstants;
+import com.kylinolap.job.constant.JobStatusEnum;
+import com.kylinolap.job.constant.JobStepCmdTypeEnum;
+import com.kylinolap.job.constant.JobStepStatusEnum;
+import com.kylinolap.job.exception.InvalidJobInstanceException;
+import com.kylinolap.metadata.MetadataManager;
+
+/**
+ * @author ysong1
+ */
+public class GenericJobEngineTest {
+    private static String cubeName = "test_kylin_cube_with_slr_empty";
+
+    private static String tempTestMetadataUrl = "../examples/sample_metadata";
+    private static JobManager jobManager;
+
+    private static JobDAO jobDAO;
+
+    private static String mrInputDir = "/tmp/mapredsmokeinput";
+    private static String mrOutputDir1 = "/tmp/mapredsmokeoutput1";
+    private static String mrOutputDir2 = "/tmp/mapredsmokeoutput2";
+    private static String mrCmd = "hadoop --config /etc/hadoop/conf jar /usr/lib/hadoop-mapreduce/hadoop-mapreduce-examples-2.*.jar wordcount " + mrInputDir + " ";
+
+    public static void removeHdfsDir(SSHClient hadoopCli, String hdfsDir) throws Exception {
+        String cmd = "hadoop fs -rm -f -r " + hdfsDir;
+        hadoopCli.execCommand(cmd);
+    }
+
+    public static String getHadoopCliHostname() {
+        return getHadoopCliAsRemote() ? KylinConfig.getInstanceFromEnv().getRemoteHadoopCliHostname() : null;
+    }
+
+    public static String getHadoopCliUsername() {
+        return getHadoopCliAsRemote() ? KylinConfig.getInstanceFromEnv().getRemoteHadoopCliUsername() : null;
+    }
+
+    public static String getHadoopCliPassword() {
+        return getHadoopCliAsRemote() ? KylinConfig.getInstanceFromEnv().getRemoteHadoopCliPassword() : null;
+    }
+
+    public static boolean getHadoopCliAsRemote() {
+        return KylinConfig.getInstanceFromEnv().getRunAsRemoteCommand();
+    }
+
+    public static void scpFilesToHdfs(SSHClient hadoopCli, String[] localFiles, String hdfsDir) throws Exception {
+        String remoteTempDir = "/tmp/";
+
+        List<String> nameList = new ArrayList<String>();
+        for (int i = 0; i < localFiles.length; i++) {
+            File file = new File(localFiles[i]);
+            String filename = file.getName();
+            nameList.add(filename);
+        }
+        for (String f : localFiles) {
+            hadoopCli.scpFileToRemote(f, remoteTempDir);
+        }
+        for (String f : nameList) {
+            hadoopCli.execCommand("hadoop fs -mkdir -p " + hdfsDir);
+            hadoopCli.execCommand("hadoop fs -put -f " + remoteTempDir + f + " " + hdfsDir + "/" + f);
+            hadoopCli.execCommand("hadoop fs -chmod 777 " + hdfsDir + "/" + f);
+        }
+    }
+
+    @BeforeClass
+    public static void beforeClass() throws Exception {
+
+        FileUtils.forceMkdir(new File(KylinConfig.getInstanceFromEnv().getKylinJobLogDir()));
+
+        FileUtils.deleteDirectory(new File(tempTestMetadataUrl));
+        FileUtils.copyDirectory(new File(AbstractKylinTestCase.LOCALMETA_TEST_DATA), new File(tempTestMetadataUrl));
+        System.setProperty(KylinConfig.KYLIN_CONF, tempTestMetadataUrl);
+
+        // deploy files to hdfs
+        SSHClient hadoopCli = new SSHClient(getHadoopCliHostname(), getHadoopCliUsername(), getHadoopCliPassword(), null);
+        scpFilesToHdfs(hadoopCli, new String[] { "src/test/resources/json/dummy_jobinstance.json" }, mrInputDir);
+        // deploy sample java jar
+        hadoopCli.scpFileToRemote("src/test/resources/jarfile/SampleJavaProgram.jarfile", "/tmp");
+        hadoopCli.scpFileToRemote("src/test/resources/jarfile/SampleBadJavaProgram.jarfile", "/tmp");
+
+        // create log dir
+        hadoopCli.execCommand("mkdir -p " + KylinConfig.getInstanceFromEnv().getKylinJobLogDir());
+        KylinConfig kylinConfig = KylinConfig.getInstanceFromEnv();
+        kylinConfig.setMetadataUrl(tempTestMetadataUrl);
+
+        jobManager = new JobManager("GenericJobEngineTest", new JobEngineConfig(KylinConfig.getInstanceFromEnv()));
+
+        jobDAO = JobDAO.getInstance(KylinConfig.getInstanceFromEnv());
+
+        jobDAO.updateJobInstance(createARunningJobInstance("a_running_job"));
+
+        jobManager.startJobEngine(2);
+        Thread.sleep(2000);
+    }
+
+    @AfterClass
+    public static void afterClass() throws Exception {
+        FileUtils.deleteDirectory(new File(tempTestMetadataUrl));
+        System.clearProperty(KylinConfig.KYLIN_CONF);
+
+        // print metrics
+        System.out.println("Job step duration seconds 80 percentile: " + jobManager.getPercentileJobStepDuration(80));
+        System.out.println("Max job step duration seconds: " + jobManager.getMaxJobStepDuration());
+        System.out.println("Min job step duration seconds: " + jobManager.getMinJobStepDuration());
+        System.out.println("# job steps executed: " + jobManager.getNumberOfJobStepsExecuted());
+        System.out.println("Engine ID: " + jobManager.getPrimaryEngineID());
+
+        jobManager.stopJobEngine();
+
+    }
+
+    @Before
+    public void before() throws Exception {
+        SSHClient hadoopCli = new SSHClient(getHadoopCliHostname(), getHadoopCliUsername(), getHadoopCliPassword(), null);
+        removeHdfsDir(hadoopCli, mrOutputDir1);
+        removeHdfsDir(hadoopCli, mrOutputDir2);
+
+        MetadataManager.removeInstance(KylinConfig.getInstanceFromEnv());
+        CubeManager.removeInstance(KylinConfig.getInstanceFromEnv());
+        ProjectManager.removeInstance(KylinConfig.getInstanceFromEnv());
+    }
+
+    @Test(expected = InvalidJobInstanceException.class)
+    public void testSubmitDuplicatedJobs() throws IOException, InvalidJobInstanceException, CubeIntegrityException {
+        String uuid = "bad_job_2";
+        JobInstance job = createASingleStepBadJobInstance(uuid);
+        // job.setStatus(JobStatusEnum.KILLED);
+        jobManager.submitJob(job);
+        jobManager.submitJob(job);
+    }
+
+    @Test
+    public void testGoodJob() throws Exception {
+        String uuid = "good_job_uuid";
+        jobManager.submitJob(createAGoodJobInstance(uuid, 5));
+
+        waitUntilJobComplete(uuid);
+
+        JobInstance savedJob1 = jobManager.getJob(uuid);
+        assertEquals(JobStatusEnum.FINISHED, savedJob1.getStatus());
+        String jobString = JsonUtil.writeValueAsIndentString(savedJob1);
+        System.out.println(jobString);
+        assertTrue(jobString.length() > 0);
+
+        // cube should be updated
+        CubeInstance cube = CubeManager.getInstance(KylinConfig.getInstanceFromEnv()).getCube(cubeName);
+        String cubeString = JsonUtil.writeValueAsIndentString(cube);
+        System.out.println(cubeString);
+        assertEquals("good_job_uuid", cube.getSegments().get(0).getLastBuildJobID());
+        assertTrue(cube.getSegments().get(0).getSizeKB() > 0);
+    }
+
+    @Test
+    public void testSingleStepBadJob() throws Exception {
+        String uuid = "single_step_bad_job_uuid";
+        jobManager.submitJob(createASingleStepBadJobInstance(uuid));
+
+        waitUntilJobComplete(uuid);
+
+        JobInstance savedJob1 = jobManager.getJob(uuid);
+        assertEquals(JobStatusEnum.ERROR, savedJob1.getStatus());
+        String jobString = JsonUtil.writeValueAsIndentString(savedJob1);
+        System.out.println(jobString);
+        assertTrue(jobString.length() > 0);
+
+        // cube should be updated
+        CubeInstance cube = CubeManager.getInstance(KylinConfig.getInstanceFromEnv()).getCube(cubeName);
+        String cubeString = JsonUtil.writeValueAsIndentString(cube);
+        System.out.println(cubeString);
+        assertEquals(CubeSegmentStatusEnum.NEW, cube.getSegments().get(0).getStatus());
+    }
+
+    @Test
+    public void testSelfCheckAndResume() throws Exception {
+        String uuid = "a_running_job";
+        assertEquals(JobStatusEnum.ERROR, jobManager.getJob(uuid).getStatus());
+        // check step status
+        assertEquals(JobStepStatusEnum.FINISHED, jobManager.getJob(uuid).getSteps().get(0).getStatus());
+        assertEquals(JobStepStatusEnum.ERROR, jobManager.getJob(uuid).getSteps().get(1).getStatus());
+        assertEquals(JobStepStatusEnum.PENDING, jobManager.getJob(uuid).getSteps().get(2).getStatus());
+
+        jobManager.resumeJob(uuid);
+
+        waitUntilJobComplete(uuid);
+        assertEquals(JobStatusEnum.FINISHED, jobManager.getJob(uuid).getStatus());
+    }
+
+    @Test
+    public void testDiscardSyncStep() throws Exception {
+        String uuid = "a_long_running_good_job_uuid";
+        JobInstance job = createAGoodJobInstance(uuid, 600);
+        jobManager.submitJob(job);
+
+        try {
+            // sleep for 5 seconds
+            Thread.sleep(5L * 1000L);
+        } catch (Exception e) {
+        }
+
+        try {
+            jobManager.discardJob(uuid);
+        } catch (RuntimeException e) {
+            throw e;
+        }
+
+        waitUntilJobComplete(uuid);
+
+        JobInstance killedJob = jobManager.getJob(uuid);
+        assertEquals(JobStepStatusEnum.DISCARDED, killedJob.getSteps().get(0).getStatus());
+        assertEquals(JobStatusEnum.DISCARDED, killedJob.getStatus());
+    }
+
+    @Test
+    public void testKillMrStep() throws Exception {
+        String uuid = "a_long_running_good_job_uuid_2";
+        JobInstance job = createAGoodJobInstance(uuid, 1);
+        jobManager.submitJob(job);
+
+        try {
+            waitUntilMrStepIsRunning(uuid);
+            jobManager.discardJob(uuid);
+        } catch (RuntimeException e) {
+            throw e;
+        }
+
+        waitUntilJobComplete(uuid);
+
+        JobInstance killedJob = jobManager.getJob(uuid);
+        assertEquals(JobStepStatusEnum.ERROR, killedJob.getSteps().get(1).getStatus());
+        assertEquals(JobStatusEnum.ERROR, killedJob.getStatus());
+
+        // cube should be updated
+        CubeInstance cube = CubeManager.getInstance(KylinConfig.getInstanceFromEnv()).getCube(cubeName);
+        String cubeString = JsonUtil.writeValueAsIndentString(cube);
+        System.out.println(cubeString);
+        assertEquals(0, cube.getSegments().size());
+    }
+
+    private void waitUntilMrStepIsRunning(String jobUuid) throws InterruptedException, IOException {
+        boolean running = false;
+        while (running == false) {
+            // sleep for 1 seconds
+            Thread.sleep(1 * 1000L);
+
+            JobInstance savedJob = jobManager.getJob(jobUuid);
+            for (JobStep step : savedJob.getSteps()) {
+                if (step.getCmdType().equals(JobStepCmdTypeEnum.SHELL_CMD_HADOOP) && step.getStatus().equals(JobStepStatusEnum.RUNNING) && step.getInfo(JobInstance.MR_JOB_ID) != null) {
+                    System.out.println("MR step is running with id " + step.getInfo(JobInstance.MR_JOB_ID));
+                    running = true;
+                    break;
+                }
+            }
+
+        }
+
+    }
+
+    private void waitUntilJobComplete(String jobUuid) throws IOException, InterruptedException {
+        boolean finished = false;
+        while (!finished) {
+            // sleep for 5 seconds
+            Thread.sleep(5 * 1000L);
+
+            finished = true;
+
+            JobInstance savedJob = jobManager.getJob(jobUuid);
+            JobStatusEnum jobStatus = savedJob.getStatus();
+            System.out.println("Job " + jobUuid + " status: " + jobStatus);
+            if (jobStatus.getCode() <= JobStatusEnum.RUNNING.getCode()) {
+                finished = false;
+            }
+        }
+    }
+
+    private JobInstance createAGoodJobInstance(String uuid, int syncCmdSleepSeconds) throws IOException, CubeIntegrityException {
+        CubeInstance cube = CubeManager.getInstance(KylinConfig.getInstanceFromEnv()).getCube(cubeName);
+        cube.getSegments().clear();
+        CubeManager.getInstance(KylinConfig.getInstanceFromEnv()).updateCube(cube);
+        CubeSegment seg = CubeManager.getInstance(KylinConfig.getInstanceFromEnv()).allocateSegments(cube, CubeBuildTypeEnum.BUILD, 0, 12345L).get(0);
+
+        JobInstance jobInstance = new JobInstance();
+        jobInstance.setUuid(uuid);
+        jobInstance.setRelatedCube(cubeName);
+        jobInstance.setRelatedSegment(seg.getName());
+        jobInstance.setName("A_Good_Job");
+        // jobInstance.setStatus(JobStatusEnum.PENDING);
+        jobInstance.setType(CubeBuildTypeEnum.BUILD);
+        // jobInstance.putInputParameter(JobConstants.PROP_STORAGE_LOCATION,
+        // "htablename");
+
+        JobStep step1 = new JobStep();
+        step1.setName("step1");
+        step1.setExecCmd("java -jar /tmp/SampleJavaProgram.jarfile " + syncCmdSleepSeconds);
+        step1.setStatus(JobStepStatusEnum.PENDING);
+        step1.setSequenceID(0);
+        step1.setCmdType(JobStepCmdTypeEnum.SHELL_CMD);
+        step1.setRunAsync(false);
+
+        // async mr step
+        JobStep step2 = new JobStep();
+        step2.setName(JobConstants.STEP_NAME_CONVERT_CUBOID_TO_HFILE);
+        step2.setExecCmd(mrCmd + mrOutputDir1);
+        step2.setStatus(JobStepStatusEnum.PENDING);
+        step2.setSequenceID(1);
+        step2.setCmdType(JobStepCmdTypeEnum.SHELL_CMD_HADOOP);
+        step2.setRunAsync(true);
+
+        // async mr step
+        JobStep step3 = new JobStep();
+        step3.setName("synced mr step");
+        step3.setExecCmd(mrCmd + mrOutputDir2);
+        step3.setStatus(JobStepStatusEnum.PENDING);
+        step3.setSequenceID(2);
+        step3.setCmdType(JobStepCmdTypeEnum.SHELL_CMD_HADOOP);
+        step3.setRunAsync(false);
+
+        jobInstance.addStep(0, step1);
+        jobInstance.addStep(1, step2);
+        jobInstance.addStep(2, step3);
+        return jobInstance;
+    }
+
+    private JobInstance createASingleStepBadJobInstance(String uuid) throws IOException, CubeIntegrityException {
+        CubeInstance cube = CubeManager.getInstance(KylinConfig.getInstanceFromEnv()).getCube(cubeName);
+        cube.getSegments().clear();
+        CubeManager.getInstance(KylinConfig.getInstanceFromEnv()).updateCube(cube);
+        CubeSegment seg = CubeManager.getInstance(KylinConfig.getInstanceFromEnv()).allocateSegments(cube, CubeBuildTypeEnum.BUILD, 0, 12345L).get(0);
+
+        JobInstance jobInstance = new JobInstance();
+        jobInstance.setUuid(uuid);
+        jobInstance.setRelatedCube(cubeName);
+        jobInstance.setRelatedSegment(seg.getName());
+        jobInstance.setName("A_Bad_Job");
+        // jobInstance.setStatus(JobStatusEnum.PENDING);
+        jobInstance.setType(CubeBuildTypeEnum.BUILD);
+        // jobInstance.putInputParameter(JobConstants.PROP_STORAGE_LOCATION,
+        // "htablename");
+
+        JobStep step1 = new JobStep();
+        step1.setName("step1");
+        step1.setExecCmd("java -jar /tmp/SampleBadJavaProgram.jarfile");
+        step1.setStatus(JobStepStatusEnum.PENDING);
+        step1.setSequenceID(0);
+        step1.setRunAsync(false);
+        step1.setCmdType(JobStepCmdTypeEnum.SHELL_CMD);
+        jobInstance.addStep(0, step1);
+
+        return jobInstance;
+    }
+
+    private static JobInstance createARunningJobInstance(String uuid) throws IOException, CubeIntegrityException {
+        CubeInstance cube = CubeManager.getInstance(KylinConfig.getInstanceFromEnv()).getCube(cubeName);
+        cube.getSegments().clear();
+        CubeManager.getInstance(KylinConfig.getInstanceFromEnv()).updateCube(cube);
+        CubeSegment seg = CubeManager.getInstance(KylinConfig.getInstanceFromEnv()).allocateSegments(cube, CubeBuildTypeEnum.BUILD, 0, 12345L).get(0);
+
+        JobInstance jobInstance = new JobInstance();
+        jobInstance.setUuid(uuid);
+        jobInstance.setRelatedCube(cubeName);
+        jobInstance.setRelatedSegment(seg.getName());
+        jobInstance.setName("A_Running_Job");
+        // jobInstance.setStatus(JobStatusEnum.RUNNING);
+        jobInstance.setType(CubeBuildTypeEnum.BUILD);
+        // jobInstance.putInputParameter(JobConstants.PROP_STORAGE_LOCATION,
+        // "htablename");
+
+        JobStep step1 = new JobStep();
+        step1.setName("step1");
+        step1.setExecCmd("echo step1");
+        step1.setStatus(JobStepStatusEnum.FINISHED);
+        step1.setSequenceID(0);
+        step1.setRunAsync(false);
+        step1.setCmdType(JobStepCmdTypeEnum.SHELL_CMD);
+
+        JobStep step2 = new JobStep();
+        step2.setName("step2");
+        step2.setExecCmd(mrCmd + mrOutputDir1);
+        step2.setStatus(JobStepStatusEnum.RUNNING);
+        step2.setSequenceID(1);
+        step2.setRunAsync(true);
+        step2.setCmdType(JobStepCmdTypeEnum.SHELL_CMD_HADOOP);
+
+        JobStep step3 = new JobStep();
+        step3.setName("step3");
+        step3.setExecCmd("java -jar /tmp/SampleJavaProgram.jarfile 3");
+        step3.setStatus(JobStepStatusEnum.PENDING);
+        step3.setSequenceID(2);
+        step3.setRunAsync(false);
+        step3.setCmdType(JobStepCmdTypeEnum.SHELL_CMD);
+
+        jobInstance.addStep(0, step1);
+        jobInstance.addStep(1, step2);
+        jobInstance.addStep(2, step3);
+        return jobInstance;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/job/src/test/java/com/kylinolap/job/engine/JobInstanceBuilderTest.java
----------------------------------------------------------------------
diff --git a/job/src/test/java/com/kylinolap/job/engine/JobInstanceBuilderTest.java b/job/src/test/java/com/kylinolap/job/engine/JobInstanceBuilderTest.java
new file mode 100644
index 0000000..3828cd2
--- /dev/null
+++ b/job/src/test/java/com/kylinolap/job/engine/JobInstanceBuilderTest.java
@@ -0,0 +1,183 @@
+/*
+ * 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.job.engine;
+
+import static org.junit.Assert.*;
+
+import java.text.SimpleDateFormat;
+import java.util.TimeZone;
+import java.util.UUID;
+
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+import com.kylinolap.common.KylinConfig;
+import com.kylinolap.common.util.JsonUtil;
+import com.kylinolap.common.util.LocalFileMetadataTestCase;
+import com.kylinolap.cube.CubeBuildTypeEnum;
+import com.kylinolap.cube.CubeInstance;
+import com.kylinolap.cube.CubeManager;
+import com.kylinolap.cube.CubeSegment;
+import com.kylinolap.cube.project.ProjectManager;
+import com.kylinolap.dict.DictionaryManager;
+import com.kylinolap.job.JobInstance;
+import com.kylinolap.job.JobInstance.JobStep;
+import com.kylinolap.job.JobManager;
+import com.kylinolap.job.constant.JobStepCmdTypeEnum;
+import com.kylinolap.metadata.MetadataManager;
+
+/**
+ * @author George Song (ysong1)
+ * 
+ */
+public class JobInstanceBuilderTest extends LocalFileMetadataTestCase {
+
+    @Before
+    public void before() throws Exception {
+        this.createTestMetadata();
+        MetadataManager.removeInstance(this.getTestConfig());
+        CubeManager.removeInstance(this.getTestConfig());
+        ProjectManager.removeInstance(this.getTestConfig());
+        DictionaryManager.removeInstance(this.getTestConfig());
+    }
+
+    @After
+    public void after() throws Exception {
+        this.cleanupTestMetadata();
+    }
+
+    @Test
+    public void testCreateSteps() throws Exception {
+        // create a new cube
+        SimpleDateFormat f = new SimpleDateFormat("yyyy-MM-dd");
+        f.setTimeZone(TimeZone.getTimeZone("GMT"));
+
+        long dateEnd = f.parse("2013-11-12").getTime();
+
+        JobManager jobManager = new JobManager("JobInstanceBuilderTest", new JobEngineConfig(KylinConfig.getInstanceFromEnv()));
+        String cubeName = "test_kylin_cube_with_slr_empty";
+        CubeManager cubeManager = CubeManager.getInstance(this.getTestConfig());
+        CubeInstance cube = cubeManager.getCube(cubeName);
+
+        // initial segment
+        CubeSegment segment = cubeManager.allocateSegments(cube, CubeBuildTypeEnum.BUILD, 0, dateEnd).get(0);
+
+        JobInstance jobInstance = jobManager.createJob(cubeName, segment.getName(), UUID.randomUUID().toString(), CubeBuildTypeEnum.BUILD,"KylinTest");
+
+        String actual = JsonUtil.writeValueAsIndentString(jobInstance);
+        System.out.println(actual);
+
+        assertEquals(13, jobInstance.getSteps().size());
+
+        assertTrue(jobInstance.getSteps().get(3).getExecCmd().contains(JobEngineConfig.HADOOP_JOB_CONF_FILENAME + ".xml"));
+
+        JobStep jobStep;
+        // check each step
+        jobStep = jobInstance.getSteps().get(0);
+        assertEquals(JobStepCmdTypeEnum.SHELL_CMD_HADOOP, jobStep.getCmdType());
+        assertEquals(false, jobStep.isRunAsync());
+
+        jobStep = jobInstance.getSteps().get(1);
+        assertEquals(JobStepCmdTypeEnum.JAVA_CMD_HADOOP_FACTDISTINCT, jobStep.getCmdType());
+        assertEquals(true, jobStep.isRunAsync());
+
+        jobStep = jobInstance.getSteps().get(2);
+        assertEquals(JobStepCmdTypeEnum.JAVA_CMD_HADOOP_NO_MR_DICTIONARY, jobStep.getCmdType());
+        assertEquals(false, jobStep.isRunAsync());
+
+        jobStep = jobInstance.getSteps().get(3);
+        assertEquals(JobStepCmdTypeEnum.JAVA_CMD_HADOOP_BASECUBOID, jobStep.getCmdType());
+        assertEquals(true, jobStep.isRunAsync());
+
+        jobStep = jobInstance.getSteps().get(4);
+        assertEquals(JobStepCmdTypeEnum.JAVA_CMD_HADOOP_NDCUBOID, jobStep.getCmdType());
+        assertEquals(true, jobStep.isRunAsync());
+
+        jobStep = jobInstance.getSteps().get(5);
+        assertEquals(JobStepCmdTypeEnum.JAVA_CMD_HADOOP_NDCUBOID, jobStep.getCmdType());
+        assertEquals(true, jobStep.isRunAsync());
+
+        jobStep = jobInstance.getSteps().get(6);
+        assertEquals(JobStepCmdTypeEnum.JAVA_CMD_HADOOP_NDCUBOID, jobStep.getCmdType());
+        assertEquals(true, jobStep.isRunAsync());
+
+        jobStep = jobInstance.getSteps().get(7);
+        assertEquals(JobStepCmdTypeEnum.JAVA_CMD_HADOOP_NDCUBOID, jobStep.getCmdType());
+        assertEquals(true, jobStep.isRunAsync());
+
+        jobStep = jobInstance.getSteps().get(8);
+        assertEquals(JobStepCmdTypeEnum.JAVA_CMD_HADOOP_NDCUBOID, jobStep.getCmdType());
+        assertEquals(true, jobStep.isRunAsync());
+
+        jobStep = jobInstance.getSteps().get(9);
+        assertEquals(JobStepCmdTypeEnum.JAVA_CMD_HADOOP_RANGEKEYDISTRIBUTION, jobStep.getCmdType());
+        assertEquals(true, jobStep.isRunAsync());
+
+        jobStep = jobInstance.getSteps().get(10);
+        assertEquals(JobStepCmdTypeEnum.JAVA_CMD_HADDOP_NO_MR_CREATEHTABLE, jobStep.getCmdType());
+        assertEquals(false, jobStep.isRunAsync());
+
+        jobStep = jobInstance.getSteps().get(11);
+        assertEquals(JobStepCmdTypeEnum.JAVA_CMD_HADOOP_CONVERTHFILE, jobStep.getCmdType());
+        assertEquals(true, jobStep.isRunAsync());
+
+        jobStep = jobInstance.getSteps().get(12);
+        assertEquals(JobStepCmdTypeEnum.JAVA_CMD_HADOOP_NO_MR_BULKLOAD, jobStep.getCmdType());
+        assertEquals(false, jobStep.isRunAsync());
+    }
+
+    @Test
+    public void testCreateMergeSteps() throws Exception {
+
+        JobManager jobManager = new JobManager("JobInstanceBuilderTest", new JobEngineConfig(KylinConfig.getInstanceFromEnv()));
+        String cubeName = "test_kylin_cube_with_slr_ready_2_segments";
+        CubeManager cubeManager = CubeManager.getInstance(this.getTestConfig());
+        CubeInstance cube = cubeManager.getCube(cubeName);
+
+        // initial segment
+        CubeSegment segment = CubeManager.getInstance(this.getTestConfig()).allocateSegments(cube, CubeBuildTypeEnum.MERGE, 1384240200000L, 1386835200000L).get(0);
+
+        JobInstance jobInstance = jobManager.createJob(cubeName, segment.getName(), UUID.randomUUID().toString(), CubeBuildTypeEnum.MERGE,"KylinTest");
+
+        String actual = JsonUtil.writeValueAsIndentString(jobInstance);
+        System.out.println(actual);
+
+        assertEquals(5, jobInstance.getSteps().size());
+
+        JobStep jobStep;
+        // check each step
+        jobStep = jobInstance.getSteps().get(0);
+        assertEquals(JobStepCmdTypeEnum.JAVA_CMD_HADOOP_MERGECUBOID, jobStep.getCmdType());
+        assertEquals(true, jobStep.isRunAsync());
+
+        jobStep = jobInstance.getSteps().get(1);
+        assertEquals(JobStepCmdTypeEnum.JAVA_CMD_HADOOP_RANGEKEYDISTRIBUTION, jobStep.getCmdType());
+        assertEquals(true, jobStep.isRunAsync());
+
+        jobStep = jobInstance.getSteps().get(2);
+        assertEquals(JobStepCmdTypeEnum.JAVA_CMD_HADDOP_NO_MR_CREATEHTABLE, jobStep.getCmdType());
+        assertEquals(false, jobStep.isRunAsync());
+
+        jobStep = jobInstance.getSteps().get(3);
+        assertEquals(JobStepCmdTypeEnum.JAVA_CMD_HADOOP_CONVERTHFILE, jobStep.getCmdType());
+        assertEquals(true, jobStep.isRunAsync());
+
+        jobStep = jobInstance.getSteps().get(4);
+        assertEquals(JobStepCmdTypeEnum.JAVA_CMD_HADOOP_NO_MR_BULKLOAD, jobStep.getCmdType());
+        assertEquals(false, jobStep.isRunAsync());
+    }
+}


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

Posted by lu...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/cube/.settings/org.eclipse.jdt.core.prefs
----------------------------------------------------------------------
diff --git a/cube/.settings/org.eclipse.jdt.core.prefs b/cube/.settings/org.eclipse.jdt.core.prefs
new file mode 100644
index 0000000..a903301
--- /dev/null
+++ b/cube/.settings/org.eclipse.jdt.core.prefs
@@ -0,0 +1,379 @@
+eclipse.preferences.version=1
+org.eclipse.jdt.core.compiler.annotation.inheritNullAnnotations=disabled
+org.eclipse.jdt.core.compiler.annotation.missingNonNullByDefaultAnnotation=ignore
+org.eclipse.jdt.core.compiler.annotation.nonnull=org.eclipse.jdt.annotation.NonNull
+org.eclipse.jdt.core.compiler.annotation.nonnullbydefault=org.eclipse.jdt.annotation.NonNullByDefault
+org.eclipse.jdt.core.compiler.annotation.nullable=org.eclipse.jdt.annotation.Nullable
+org.eclipse.jdt.core.compiler.annotation.nullanalysis=disabled
+org.eclipse.jdt.core.compiler.codegen.inlineJsrBytecode=enabled
+org.eclipse.jdt.core.compiler.codegen.targetPlatform=1.7
+org.eclipse.jdt.core.compiler.codegen.unusedLocal=preserve
+org.eclipse.jdt.core.compiler.compliance=1.7
+org.eclipse.jdt.core.compiler.debug.lineNumber=generate
+org.eclipse.jdt.core.compiler.debug.localVariable=generate
+org.eclipse.jdt.core.compiler.debug.sourceFile=generate
+org.eclipse.jdt.core.compiler.problem.annotationSuperInterface=warning
+org.eclipse.jdt.core.compiler.problem.assertIdentifier=error
+org.eclipse.jdt.core.compiler.problem.autoboxing=ignore
+org.eclipse.jdt.core.compiler.problem.comparingIdentical=warning
+org.eclipse.jdt.core.compiler.problem.deadCode=warning
+org.eclipse.jdt.core.compiler.problem.deprecation=warning
+org.eclipse.jdt.core.compiler.problem.deprecationInDeprecatedCode=disabled
+org.eclipse.jdt.core.compiler.problem.deprecationWhenOverridingDeprecatedMethod=disabled
+org.eclipse.jdt.core.compiler.problem.discouragedReference=warning
+org.eclipse.jdt.core.compiler.problem.emptyStatement=ignore
+org.eclipse.jdt.core.compiler.problem.enumIdentifier=error
+org.eclipse.jdt.core.compiler.problem.explicitlyClosedAutoCloseable=ignore
+org.eclipse.jdt.core.compiler.problem.fallthroughCase=ignore
+org.eclipse.jdt.core.compiler.problem.fatalOptionalError=disabled
+org.eclipse.jdt.core.compiler.problem.fieldHiding=ignore
+org.eclipse.jdt.core.compiler.problem.finalParameterBound=warning
+org.eclipse.jdt.core.compiler.problem.finallyBlockNotCompletingNormally=warning
+org.eclipse.jdt.core.compiler.problem.forbiddenReference=warning
+org.eclipse.jdt.core.compiler.problem.hiddenCatchBlock=warning
+org.eclipse.jdt.core.compiler.problem.includeNullInfoFromAsserts=disabled
+org.eclipse.jdt.core.compiler.problem.incompatibleNonInheritedInterfaceMethod=warning
+org.eclipse.jdt.core.compiler.problem.incompleteEnumSwitch=warning
+org.eclipse.jdt.core.compiler.problem.indirectStaticAccess=ignore
+org.eclipse.jdt.core.compiler.problem.localVariableHiding=ignore
+org.eclipse.jdt.core.compiler.problem.methodWithConstructorName=warning
+org.eclipse.jdt.core.compiler.problem.missingDefaultCase=ignore
+org.eclipse.jdt.core.compiler.problem.missingDeprecatedAnnotation=ignore
+org.eclipse.jdt.core.compiler.problem.missingEnumCaseDespiteDefault=disabled
+org.eclipse.jdt.core.compiler.problem.missingHashCodeMethod=ignore
+org.eclipse.jdt.core.compiler.problem.missingOverrideAnnotation=ignore
+org.eclipse.jdt.core.compiler.problem.missingOverrideAnnotationForInterfaceMethodImplementation=enabled
+org.eclipse.jdt.core.compiler.problem.missingSerialVersion=warning
+org.eclipse.jdt.core.compiler.problem.missingSynchronizedOnInheritedMethod=ignore
+org.eclipse.jdt.core.compiler.problem.noEffectAssignment=warning
+org.eclipse.jdt.core.compiler.problem.noImplicitStringConversion=warning
+org.eclipse.jdt.core.compiler.problem.nonExternalizedStringLiteral=ignore
+org.eclipse.jdt.core.compiler.problem.nonnullParameterAnnotationDropped=warning
+org.eclipse.jdt.core.compiler.problem.nullAnnotationInferenceConflict=error
+org.eclipse.jdt.core.compiler.problem.nullReference=warning
+org.eclipse.jdt.core.compiler.problem.nullSpecViolation=error
+org.eclipse.jdt.core.compiler.problem.nullUncheckedConversion=warning
+org.eclipse.jdt.core.compiler.problem.overridingPackageDefaultMethod=warning
+org.eclipse.jdt.core.compiler.problem.parameterAssignment=ignore
+org.eclipse.jdt.core.compiler.problem.possibleAccidentalBooleanAssignment=ignore
+org.eclipse.jdt.core.compiler.problem.potentialNullReference=ignore
+org.eclipse.jdt.core.compiler.problem.potentiallyUnclosedCloseable=ignore
+org.eclipse.jdt.core.compiler.problem.rawTypeReference=warning
+org.eclipse.jdt.core.compiler.problem.redundantNullAnnotation=warning
+org.eclipse.jdt.core.compiler.problem.redundantNullCheck=ignore
+org.eclipse.jdt.core.compiler.problem.redundantSpecificationOfTypeArguments=ignore
+org.eclipse.jdt.core.compiler.problem.redundantSuperinterface=ignore
+org.eclipse.jdt.core.compiler.problem.reportMethodCanBePotentiallyStatic=ignore
+org.eclipse.jdt.core.compiler.problem.reportMethodCanBeStatic=ignore
+org.eclipse.jdt.core.compiler.problem.specialParameterHidingField=disabled
+org.eclipse.jdt.core.compiler.problem.staticAccessReceiver=warning
+org.eclipse.jdt.core.compiler.problem.suppressOptionalErrors=disabled
+org.eclipse.jdt.core.compiler.problem.suppressWarnings=enabled
+org.eclipse.jdt.core.compiler.problem.syntacticNullAnalysisForFields=disabled
+org.eclipse.jdt.core.compiler.problem.syntheticAccessEmulation=ignore
+org.eclipse.jdt.core.compiler.problem.typeParameterHiding=warning
+org.eclipse.jdt.core.compiler.problem.unavoidableGenericTypeProblems=enabled
+org.eclipse.jdt.core.compiler.problem.uncheckedTypeOperation=warning
+org.eclipse.jdt.core.compiler.problem.unclosedCloseable=warning
+org.eclipse.jdt.core.compiler.problem.undocumentedEmptyBlock=ignore
+org.eclipse.jdt.core.compiler.problem.unhandledWarningToken=warning
+org.eclipse.jdt.core.compiler.problem.unnecessaryElse=ignore
+org.eclipse.jdt.core.compiler.problem.unnecessaryTypeCheck=ignore
+org.eclipse.jdt.core.compiler.problem.unqualifiedFieldAccess=ignore
+org.eclipse.jdt.core.compiler.problem.unusedDeclaredThrownException=ignore
+org.eclipse.jdt.core.compiler.problem.unusedDeclaredThrownExceptionExemptExceptionAndThrowable=enabled
+org.eclipse.jdt.core.compiler.problem.unusedDeclaredThrownExceptionIncludeDocCommentReference=enabled
+org.eclipse.jdt.core.compiler.problem.unusedDeclaredThrownExceptionWhenOverriding=disabled
+org.eclipse.jdt.core.compiler.problem.unusedImport=warning
+org.eclipse.jdt.core.compiler.problem.unusedLabel=warning
+org.eclipse.jdt.core.compiler.problem.unusedLocal=warning
+org.eclipse.jdt.core.compiler.problem.unusedObjectAllocation=ignore
+org.eclipse.jdt.core.compiler.problem.unusedParameter=ignore
+org.eclipse.jdt.core.compiler.problem.unusedParameterIncludeDocCommentReference=enabled
+org.eclipse.jdt.core.compiler.problem.unusedParameterWhenImplementingAbstract=disabled
+org.eclipse.jdt.core.compiler.problem.unusedParameterWhenOverridingConcrete=disabled
+org.eclipse.jdt.core.compiler.problem.unusedPrivateMember=warning
+org.eclipse.jdt.core.compiler.problem.unusedTypeParameter=ignore
+org.eclipse.jdt.core.compiler.problem.unusedWarningToken=warning
+org.eclipse.jdt.core.compiler.problem.varargsArgumentNeedCast=warning
+org.eclipse.jdt.core.compiler.source=1.7
+org.eclipse.jdt.core.formatter.align_type_members_on_columns=false
+org.eclipse.jdt.core.formatter.alignment_for_arguments_in_allocation_expression=16
+org.eclipse.jdt.core.formatter.alignment_for_arguments_in_annotation=0
+org.eclipse.jdt.core.formatter.alignment_for_arguments_in_enum_constant=16
+org.eclipse.jdt.core.formatter.alignment_for_arguments_in_explicit_constructor_call=16
+org.eclipse.jdt.core.formatter.alignment_for_arguments_in_method_invocation=16
+org.eclipse.jdt.core.formatter.alignment_for_arguments_in_qualified_allocation_expression=16
+org.eclipse.jdt.core.formatter.alignment_for_assignment=0
+org.eclipse.jdt.core.formatter.alignment_for_binary_expression=16
+org.eclipse.jdt.core.formatter.alignment_for_compact_if=16
+org.eclipse.jdt.core.formatter.alignment_for_conditional_expression=80
+org.eclipse.jdt.core.formatter.alignment_for_enum_constants=0
+org.eclipse.jdt.core.formatter.alignment_for_expressions_in_array_initializer=16
+org.eclipse.jdt.core.formatter.alignment_for_method_declaration=0
+org.eclipse.jdt.core.formatter.alignment_for_multiple_fields=16
+org.eclipse.jdt.core.formatter.alignment_for_parameters_in_constructor_declaration=16
+org.eclipse.jdt.core.formatter.alignment_for_parameters_in_method_declaration=16
+org.eclipse.jdt.core.formatter.alignment_for_resources_in_try=80
+org.eclipse.jdt.core.formatter.alignment_for_selector_in_method_invocation=16
+org.eclipse.jdt.core.formatter.alignment_for_superclass_in_type_declaration=16
+org.eclipse.jdt.core.formatter.alignment_for_superinterfaces_in_enum_declaration=16
+org.eclipse.jdt.core.formatter.alignment_for_superinterfaces_in_type_declaration=16
+org.eclipse.jdt.core.formatter.alignment_for_throws_clause_in_constructor_declaration=16
+org.eclipse.jdt.core.formatter.alignment_for_throws_clause_in_method_declaration=16
+org.eclipse.jdt.core.formatter.alignment_for_union_type_in_multicatch=16
+org.eclipse.jdt.core.formatter.blank_lines_after_imports=1
+org.eclipse.jdt.core.formatter.blank_lines_after_package=1
+org.eclipse.jdt.core.formatter.blank_lines_before_field=0
+org.eclipse.jdt.core.formatter.blank_lines_before_first_class_body_declaration=0
+org.eclipse.jdt.core.formatter.blank_lines_before_imports=1
+org.eclipse.jdt.core.formatter.blank_lines_before_member_type=1
+org.eclipse.jdt.core.formatter.blank_lines_before_method=1
+org.eclipse.jdt.core.formatter.blank_lines_before_new_chunk=1
+org.eclipse.jdt.core.formatter.blank_lines_before_package=0
+org.eclipse.jdt.core.formatter.blank_lines_between_import_groups=1
+org.eclipse.jdt.core.formatter.blank_lines_between_type_declarations=1
+org.eclipse.jdt.core.formatter.brace_position_for_annotation_type_declaration=end_of_line
+org.eclipse.jdt.core.formatter.brace_position_for_anonymous_type_declaration=end_of_line
+org.eclipse.jdt.core.formatter.brace_position_for_array_initializer=end_of_line
+org.eclipse.jdt.core.formatter.brace_position_for_block=end_of_line
+org.eclipse.jdt.core.formatter.brace_position_for_block_in_case=end_of_line
+org.eclipse.jdt.core.formatter.brace_position_for_constructor_declaration=end_of_line
+org.eclipse.jdt.core.formatter.brace_position_for_enum_constant=end_of_line
+org.eclipse.jdt.core.formatter.brace_position_for_enum_declaration=end_of_line
+org.eclipse.jdt.core.formatter.brace_position_for_method_declaration=end_of_line
+org.eclipse.jdt.core.formatter.brace_position_for_switch=end_of_line
+org.eclipse.jdt.core.formatter.brace_position_for_type_declaration=end_of_line
+org.eclipse.jdt.core.formatter.comment.clear_blank_lines_in_block_comment=false
+org.eclipse.jdt.core.formatter.comment.clear_blank_lines_in_javadoc_comment=false
+org.eclipse.jdt.core.formatter.comment.format_block_comments=false
+org.eclipse.jdt.core.formatter.comment.format_header=false
+org.eclipse.jdt.core.formatter.comment.format_html=true
+org.eclipse.jdt.core.formatter.comment.format_javadoc_comments=false
+org.eclipse.jdt.core.formatter.comment.format_line_comments=false
+org.eclipse.jdt.core.formatter.comment.format_source_code=true
+org.eclipse.jdt.core.formatter.comment.indent_parameter_description=true
+org.eclipse.jdt.core.formatter.comment.indent_root_tags=true
+org.eclipse.jdt.core.formatter.comment.insert_new_line_before_root_tags=insert
+org.eclipse.jdt.core.formatter.comment.insert_new_line_for_parameter=insert
+org.eclipse.jdt.core.formatter.comment.line_length=80
+org.eclipse.jdt.core.formatter.comment.new_lines_at_block_boundaries=true
+org.eclipse.jdt.core.formatter.comment.new_lines_at_javadoc_boundaries=true
+org.eclipse.jdt.core.formatter.comment.preserve_white_space_between_code_and_line_comments=false
+org.eclipse.jdt.core.formatter.compact_else_if=true
+org.eclipse.jdt.core.formatter.continuation_indentation=2
+org.eclipse.jdt.core.formatter.continuation_indentation_for_array_initializer=2
+org.eclipse.jdt.core.formatter.disabling_tag=@formatter\:off
+org.eclipse.jdt.core.formatter.enabling_tag=@formatter\:on
+org.eclipse.jdt.core.formatter.format_guardian_clause_on_one_line=false
+org.eclipse.jdt.core.formatter.format_line_comment_starting_on_first_column=true
+org.eclipse.jdt.core.formatter.indent_body_declarations_compare_to_annotation_declaration_header=true
+org.eclipse.jdt.core.formatter.indent_body_declarations_compare_to_enum_constant_header=true
+org.eclipse.jdt.core.formatter.indent_body_declarations_compare_to_enum_declaration_header=true
+org.eclipse.jdt.core.formatter.indent_body_declarations_compare_to_type_header=true
+org.eclipse.jdt.core.formatter.indent_breaks_compare_to_cases=true
+org.eclipse.jdt.core.formatter.indent_empty_lines=false
+org.eclipse.jdt.core.formatter.indent_statements_compare_to_block=true
+org.eclipse.jdt.core.formatter.indent_statements_compare_to_body=true
+org.eclipse.jdt.core.formatter.indent_switchstatements_compare_to_cases=true
+org.eclipse.jdt.core.formatter.indent_switchstatements_compare_to_switch=false
+org.eclipse.jdt.core.formatter.indentation.size=4
+org.eclipse.jdt.core.formatter.insert_new_line_after_annotation_on_field=insert
+org.eclipse.jdt.core.formatter.insert_new_line_after_annotation_on_local_variable=insert
+org.eclipse.jdt.core.formatter.insert_new_line_after_annotation_on_method=insert
+org.eclipse.jdt.core.formatter.insert_new_line_after_annotation_on_package=insert
+org.eclipse.jdt.core.formatter.insert_new_line_after_annotation_on_parameter=do not insert
+org.eclipse.jdt.core.formatter.insert_new_line_after_annotation_on_type=insert
+org.eclipse.jdt.core.formatter.insert_new_line_after_label=do not insert
+org.eclipse.jdt.core.formatter.insert_new_line_after_opening_brace_in_array_initializer=do not insert
+org.eclipse.jdt.core.formatter.insert_new_line_at_end_of_file_if_missing=do not insert
+org.eclipse.jdt.core.formatter.insert_new_line_before_catch_in_try_statement=do not insert
+org.eclipse.jdt.core.formatter.insert_new_line_before_closing_brace_in_array_initializer=do not insert
+org.eclipse.jdt.core.formatter.insert_new_line_before_else_in_if_statement=do not insert
+org.eclipse.jdt.core.formatter.insert_new_line_before_finally_in_try_statement=do not insert
+org.eclipse.jdt.core.formatter.insert_new_line_before_while_in_do_statement=do not insert
+org.eclipse.jdt.core.formatter.insert_new_line_in_empty_annotation_declaration=insert
+org.eclipse.jdt.core.formatter.insert_new_line_in_empty_anonymous_type_declaration=insert
+org.eclipse.jdt.core.formatter.insert_new_line_in_empty_block=insert
+org.eclipse.jdt.core.formatter.insert_new_line_in_empty_enum_constant=insert
+org.eclipse.jdt.core.formatter.insert_new_line_in_empty_enum_declaration=insert
+org.eclipse.jdt.core.formatter.insert_new_line_in_empty_method_body=insert
+org.eclipse.jdt.core.formatter.insert_new_line_in_empty_type_declaration=insert
+org.eclipse.jdt.core.formatter.insert_space_after_and_in_type_parameter=insert
+org.eclipse.jdt.core.formatter.insert_space_after_assignment_operator=insert
+org.eclipse.jdt.core.formatter.insert_space_after_at_in_annotation=do not insert
+org.eclipse.jdt.core.formatter.insert_space_after_at_in_annotation_type_declaration=do not insert
+org.eclipse.jdt.core.formatter.insert_space_after_binary_operator=insert
+org.eclipse.jdt.core.formatter.insert_space_after_closing_angle_bracket_in_type_arguments=insert
+org.eclipse.jdt.core.formatter.insert_space_after_closing_angle_bracket_in_type_parameters=insert
+org.eclipse.jdt.core.formatter.insert_space_after_closing_brace_in_block=insert
+org.eclipse.jdt.core.formatter.insert_space_after_closing_paren_in_cast=insert
+org.eclipse.jdt.core.formatter.insert_space_after_colon_in_assert=insert
+org.eclipse.jdt.core.formatter.insert_space_after_colon_in_case=insert
+org.eclipse.jdt.core.formatter.insert_space_after_colon_in_conditional=insert
+org.eclipse.jdt.core.formatter.insert_space_after_colon_in_for=insert
+org.eclipse.jdt.core.formatter.insert_space_after_colon_in_labeled_statement=insert
+org.eclipse.jdt.core.formatter.insert_space_after_comma_in_allocation_expression=insert
+org.eclipse.jdt.core.formatter.insert_space_after_comma_in_annotation=insert
+org.eclipse.jdt.core.formatter.insert_space_after_comma_in_array_initializer=insert
+org.eclipse.jdt.core.formatter.insert_space_after_comma_in_constructor_declaration_parameters=insert
+org.eclipse.jdt.core.formatter.insert_space_after_comma_in_constructor_declaration_throws=insert
+org.eclipse.jdt.core.formatter.insert_space_after_comma_in_enum_constant_arguments=insert
+org.eclipse.jdt.core.formatter.insert_space_after_comma_in_enum_declarations=insert
+org.eclipse.jdt.core.formatter.insert_space_after_comma_in_explicitconstructorcall_arguments=insert
+org.eclipse.jdt.core.formatter.insert_space_after_comma_in_for_increments=insert
+org.eclipse.jdt.core.formatter.insert_space_after_comma_in_for_inits=insert
+org.eclipse.jdt.core.formatter.insert_space_after_comma_in_method_declaration_parameters=insert
+org.eclipse.jdt.core.formatter.insert_space_after_comma_in_method_declaration_throws=insert
+org.eclipse.jdt.core.formatter.insert_space_after_comma_in_method_invocation_arguments=insert
+org.eclipse.jdt.core.formatter.insert_space_after_comma_in_multiple_field_declarations=insert
+org.eclipse.jdt.core.formatter.insert_space_after_comma_in_multiple_local_declarations=insert
+org.eclipse.jdt.core.formatter.insert_space_after_comma_in_parameterized_type_reference=insert
+org.eclipse.jdt.core.formatter.insert_space_after_comma_in_superinterfaces=insert
+org.eclipse.jdt.core.formatter.insert_space_after_comma_in_type_arguments=insert
+org.eclipse.jdt.core.formatter.insert_space_after_comma_in_type_parameters=insert
+org.eclipse.jdt.core.formatter.insert_space_after_ellipsis=insert
+org.eclipse.jdt.core.formatter.insert_space_after_opening_angle_bracket_in_parameterized_type_reference=do not insert
+org.eclipse.jdt.core.formatter.insert_space_after_opening_angle_bracket_in_type_arguments=do not insert
+org.eclipse.jdt.core.formatter.insert_space_after_opening_angle_bracket_in_type_parameters=do not insert
+org.eclipse.jdt.core.formatter.insert_space_after_opening_brace_in_array_initializer=insert
+org.eclipse.jdt.core.formatter.insert_space_after_opening_bracket_in_array_allocation_expression=do not insert
+org.eclipse.jdt.core.formatter.insert_space_after_opening_bracket_in_array_reference=do not insert
+org.eclipse.jdt.core.formatter.insert_space_after_opening_paren_in_annotation=do not insert
+org.eclipse.jdt.core.formatter.insert_space_after_opening_paren_in_cast=do not insert
+org.eclipse.jdt.core.formatter.insert_space_after_opening_paren_in_catch=do not insert
+org.eclipse.jdt.core.formatter.insert_space_after_opening_paren_in_constructor_declaration=do not insert
+org.eclipse.jdt.core.formatter.insert_space_after_opening_paren_in_enum_constant=do not insert
+org.eclipse.jdt.core.formatter.insert_space_after_opening_paren_in_for=do not insert
+org.eclipse.jdt.core.formatter.insert_space_after_opening_paren_in_if=do not insert
+org.eclipse.jdt.core.formatter.insert_space_after_opening_paren_in_method_declaration=do not insert
+org.eclipse.jdt.core.formatter.insert_space_after_opening_paren_in_method_invocation=do not insert
+org.eclipse.jdt.core.formatter.insert_space_after_opening_paren_in_parenthesized_expression=do not insert
+org.eclipse.jdt.core.formatter.insert_space_after_opening_paren_in_switch=do not insert
+org.eclipse.jdt.core.formatter.insert_space_after_opening_paren_in_synchronized=do not insert
+org.eclipse.jdt.core.formatter.insert_space_after_opening_paren_in_try=do not insert
+org.eclipse.jdt.core.formatter.insert_space_after_opening_paren_in_while=do not insert
+org.eclipse.jdt.core.formatter.insert_space_after_postfix_operator=do not insert
+org.eclipse.jdt.core.formatter.insert_space_after_prefix_operator=do not insert
+org.eclipse.jdt.core.formatter.insert_space_after_question_in_conditional=insert
+org.eclipse.jdt.core.formatter.insert_space_after_question_in_wildcard=do not insert
+org.eclipse.jdt.core.formatter.insert_space_after_semicolon_in_for=insert
+org.eclipse.jdt.core.formatter.insert_space_after_semicolon_in_try_resources=insert
+org.eclipse.jdt.core.formatter.insert_space_after_unary_operator=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_and_in_type_parameter=insert
+org.eclipse.jdt.core.formatter.insert_space_before_assignment_operator=insert
+org.eclipse.jdt.core.formatter.insert_space_before_at_in_annotation_type_declaration=insert
+org.eclipse.jdt.core.formatter.insert_space_before_binary_operator=insert
+org.eclipse.jdt.core.formatter.insert_space_before_closing_angle_bracket_in_parameterized_type_reference=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_closing_angle_bracket_in_type_arguments=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_closing_angle_bracket_in_type_parameters=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_closing_brace_in_array_initializer=insert
+org.eclipse.jdt.core.formatter.insert_space_before_closing_bracket_in_array_allocation_expression=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_closing_bracket_in_array_reference=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_closing_paren_in_annotation=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_closing_paren_in_cast=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_closing_paren_in_catch=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_closing_paren_in_constructor_declaration=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_closing_paren_in_enum_constant=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_closing_paren_in_for=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_closing_paren_in_if=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_closing_paren_in_method_declaration=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_closing_paren_in_method_invocation=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_closing_paren_in_parenthesized_expression=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_closing_paren_in_switch=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_closing_paren_in_synchronized=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_closing_paren_in_try=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_closing_paren_in_while=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_colon_in_assert=insert
+org.eclipse.jdt.core.formatter.insert_space_before_colon_in_case=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_colon_in_conditional=insert
+org.eclipse.jdt.core.formatter.insert_space_before_colon_in_default=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_colon_in_for=insert
+org.eclipse.jdt.core.formatter.insert_space_before_colon_in_labeled_statement=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_comma_in_allocation_expression=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_comma_in_annotation=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_comma_in_array_initializer=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_comma_in_constructor_declaration_parameters=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_comma_in_constructor_declaration_throws=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_comma_in_enum_constant_arguments=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_comma_in_enum_declarations=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_comma_in_explicitconstructorcall_arguments=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_comma_in_for_increments=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_comma_in_for_inits=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_comma_in_method_declaration_parameters=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_comma_in_method_declaration_throws=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_comma_in_method_invocation_arguments=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_comma_in_multiple_field_declarations=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_comma_in_multiple_local_declarations=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_comma_in_parameterized_type_reference=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_comma_in_superinterfaces=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_comma_in_type_arguments=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_comma_in_type_parameters=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_ellipsis=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_opening_angle_bracket_in_parameterized_type_reference=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_opening_angle_bracket_in_type_arguments=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_opening_angle_bracket_in_type_parameters=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_opening_brace_in_annotation_type_declaration=insert
+org.eclipse.jdt.core.formatter.insert_space_before_opening_brace_in_anonymous_type_declaration=insert
+org.eclipse.jdt.core.formatter.insert_space_before_opening_brace_in_array_initializer=insert
+org.eclipse.jdt.core.formatter.insert_space_before_opening_brace_in_block=insert
+org.eclipse.jdt.core.formatter.insert_space_before_opening_brace_in_constructor_declaration=insert
+org.eclipse.jdt.core.formatter.insert_space_before_opening_brace_in_enum_constant=insert
+org.eclipse.jdt.core.formatter.insert_space_before_opening_brace_in_enum_declaration=insert
+org.eclipse.jdt.core.formatter.insert_space_before_opening_brace_in_method_declaration=insert
+org.eclipse.jdt.core.formatter.insert_space_before_opening_brace_in_switch=insert
+org.eclipse.jdt.core.formatter.insert_space_before_opening_brace_in_type_declaration=insert
+org.eclipse.jdt.core.formatter.insert_space_before_opening_bracket_in_array_allocation_expression=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_opening_bracket_in_array_reference=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_opening_bracket_in_array_type_reference=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_opening_paren_in_annotation=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_opening_paren_in_annotation_type_member_declaration=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_opening_paren_in_catch=insert
+org.eclipse.jdt.core.formatter.insert_space_before_opening_paren_in_constructor_declaration=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_opening_paren_in_enum_constant=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_opening_paren_in_for=insert
+org.eclipse.jdt.core.formatter.insert_space_before_opening_paren_in_if=insert
+org.eclipse.jdt.core.formatter.insert_space_before_opening_paren_in_method_declaration=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_opening_paren_in_method_invocation=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_opening_paren_in_parenthesized_expression=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_opening_paren_in_switch=insert
+org.eclipse.jdt.core.formatter.insert_space_before_opening_paren_in_synchronized=insert
+org.eclipse.jdt.core.formatter.insert_space_before_opening_paren_in_try=insert
+org.eclipse.jdt.core.formatter.insert_space_before_opening_paren_in_while=insert
+org.eclipse.jdt.core.formatter.insert_space_before_parenthesized_expression_in_return=insert
+org.eclipse.jdt.core.formatter.insert_space_before_parenthesized_expression_in_throw=insert
+org.eclipse.jdt.core.formatter.insert_space_before_postfix_operator=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_prefix_operator=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_question_in_conditional=insert
+org.eclipse.jdt.core.formatter.insert_space_before_question_in_wildcard=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_semicolon=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_semicolon_in_for=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_semicolon_in_try_resources=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_unary_operator=do not insert
+org.eclipse.jdt.core.formatter.insert_space_between_brackets_in_array_type_reference=do not insert
+org.eclipse.jdt.core.formatter.insert_space_between_empty_braces_in_array_initializer=do not insert
+org.eclipse.jdt.core.formatter.insert_space_between_empty_brackets_in_array_allocation_expression=do not insert
+org.eclipse.jdt.core.formatter.insert_space_between_empty_parens_in_annotation_type_member_declaration=do not insert
+org.eclipse.jdt.core.formatter.insert_space_between_empty_parens_in_constructor_declaration=do not insert
+org.eclipse.jdt.core.formatter.insert_space_between_empty_parens_in_enum_constant=do not insert
+org.eclipse.jdt.core.formatter.insert_space_between_empty_parens_in_method_declaration=do not insert
+org.eclipse.jdt.core.formatter.insert_space_between_empty_parens_in_method_invocation=do not insert
+org.eclipse.jdt.core.formatter.join_lines_in_comments=true
+org.eclipse.jdt.core.formatter.join_wrapped_lines=true
+org.eclipse.jdt.core.formatter.keep_else_statement_on_same_line=false
+org.eclipse.jdt.core.formatter.keep_empty_array_initializer_on_one_line=false
+org.eclipse.jdt.core.formatter.keep_imple_if_on_one_line=false
+org.eclipse.jdt.core.formatter.keep_then_statement_on_same_line=false
+org.eclipse.jdt.core.formatter.lineSplit=999
+org.eclipse.jdt.core.formatter.never_indent_block_comments_on_first_column=false
+org.eclipse.jdt.core.formatter.never_indent_line_comments_on_first_column=false
+org.eclipse.jdt.core.formatter.number_of_blank_lines_at_beginning_of_method_body=0
+org.eclipse.jdt.core.formatter.number_of_empty_lines_to_preserve=1
+org.eclipse.jdt.core.formatter.put_empty_statement_on_new_line=true
+org.eclipse.jdt.core.formatter.tabulation.char=space
+org.eclipse.jdt.core.formatter.tabulation.size=4
+org.eclipse.jdt.core.formatter.use_on_off_tags=false
+org.eclipse.jdt.core.formatter.use_tabs_only_for_leading_indentations=false
+org.eclipse.jdt.core.formatter.wrap_before_binary_operator=true
+org.eclipse.jdt.core.formatter.wrap_before_or_operator_multicatch=true
+org.eclipse.jdt.core.formatter.wrap_outer_expressions_when_nested=true

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/cube/.settings/org.eclipse.jdt.ui.prefs
----------------------------------------------------------------------
diff --git a/cube/.settings/org.eclipse.jdt.ui.prefs b/cube/.settings/org.eclipse.jdt.ui.prefs
new file mode 100644
index 0000000..dece0e6
--- /dev/null
+++ b/cube/.settings/org.eclipse.jdt.ui.prefs
@@ -0,0 +1,7 @@
+eclipse.preferences.version=1
+formatter_profile=_Space Indent & Long Lines
+formatter_settings_version=12
+org.eclipse.jdt.ui.ignorelowercasenames=true
+org.eclipse.jdt.ui.importorder=java;javax;org;com;
+org.eclipse.jdt.ui.ondemandthreshold=99
+org.eclipse.jdt.ui.staticondemandthreshold=1

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/cube/pom.xml
----------------------------------------------------------------------
diff --git a/cube/pom.xml b/cube/pom.xml
new file mode 100644
index 0000000..f177c07
--- /dev/null
+++ b/cube/pom.xml
@@ -0,0 +1,157 @@
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+    <modelVersion>4.0.0</modelVersion>
+
+    <artifactId>kylin-cube</artifactId>
+    <packaging>jar</packaging>
+    <name>Kylin:Cube</name>
+    <parent>
+        <groupId>com.kylinolap</groupId>
+        <artifactId>kylin</artifactId>
+        <version>0.6.3-SNAPSHOT</version>
+    </parent>
+
+    <dependencies>
+        <!--Kylin Jar -->
+        <dependency>
+            <groupId>com.kylinolap</groupId>
+            <artifactId>kylin-metadata</artifactId>
+            <version>${project.parent.version}</version>
+        </dependency>
+        <dependency>
+            <groupId>com.kylinolap</groupId>
+            <artifactId>kylin-dictionary</artifactId>
+            <version>${project.parent.version}</version>
+        </dependency>
+
+        <dependency>
+            <groupId>commons-cli</groupId>
+            <artifactId>commons-cli</artifactId>
+        </dependency>
+        <dependency>
+            <groupId>commons-lang</groupId>
+            <artifactId>commons-lang</artifactId>
+        </dependency>
+        <dependency>
+            <groupId>commons-io</groupId>
+            <artifactId>commons-io</artifactId>
+        </dependency>
+        <dependency>
+            <groupId>commons-configuration</groupId>
+            <artifactId>commons-configuration</artifactId>
+        </dependency>
+        <dependency>
+            <groupId>log4j</groupId>
+            <artifactId>log4j</artifactId>
+        </dependency>
+        <dependency>
+            <groupId>com.fasterxml.jackson.core</groupId>
+            <artifactId>jackson-databind</artifactId>
+        </dependency>
+        <dependency>
+            <groupId>commons-httpclient</groupId>
+            <artifactId>commons-httpclient</artifactId>
+        </dependency>
+        <dependency>
+            <groupId>com.google.guava</groupId>
+            <artifactId>guava</artifactId>
+        </dependency>
+        <dependency>
+            <groupId>com.ning</groupId>
+            <artifactId>compress-lzf</artifactId>
+        </dependency>
+        <dependency>
+            <groupId>com.n3twork.druid</groupId>
+            <artifactId>extendedset</artifactId>
+        </dependency>
+
+        <!-- Env & Test -->
+
+        <dependency>
+            <groupId>org.apache.hadoop</groupId>
+            <artifactId>hadoop-common</artifactId>
+            <scope>provided</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.hadoop</groupId>
+            <artifactId>hadoop-annotations</artifactId>
+            <scope>provided</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.hadoop</groupId>
+            <artifactId>hadoop-mapreduce-client-core</artifactId>
+            <scope>provided</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.hadoop</groupId>
+            <artifactId>hadoop-minicluster</artifactId>
+            <scope>test</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.mrunit</groupId>
+            <artifactId>mrunit</artifactId>
+            <classifier>hadoop2</classifier>
+            <scope>test</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.hbase</groupId>
+            <artifactId>hbase-hadoop2-compat</artifactId>
+            <scope>provided</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.hbase</groupId>
+            <artifactId>hbase-common</artifactId>
+            <scope>provided</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.hbase</groupId>
+            <artifactId>hbase-client</artifactId>
+            <scope>provided</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.hbase</groupId>
+            <artifactId>hbase-server</artifactId>
+            <scope>provided</scope>
+            <!-- version conflict with hadoop2.2 -->
+            <exclusions>
+                <exclusion>
+                    <groupId>org.apache.hadoop</groupId>
+                    <artifactId>hadoop-mapreduce-client-jobclient</artifactId>
+                </exclusion>
+            </exclusions>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.hadoop</groupId>
+            <artifactId>hadoop-mapreduce-client-jobclient</artifactId>
+            <scope>provided</scope>
+        </dependency>
+        <dependency>
+            <groupId>junit</groupId>
+            <artifactId>junit</artifactId>
+            <scope>test</scope>
+        </dependency>
+    </dependencies>
+
+    <!--
+        <build>
+            <plugins>
+                <plugin>
+                    <artifactId>maven-assembly-plugin</artifactId>
+                    <executions>
+                        <execution>
+                            <id>jar-with-dependencies</id>
+                            <phase>package</phase>
+                            <goals>
+                                <goal>single</goal>
+                            </goals>
+                            <configuration>
+                                <descriptors>
+                                    <descriptor>src/main/assembly/assemble.xml</descriptor>
+                                </descriptors>
+                            </configuration>
+                        </execution>
+                    </executions>
+                </plugin>
+            </plugins>
+        </build>
+     -->
+</project>

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/cube/src/main/assembly/assemble.xml
----------------------------------------------------------------------
diff --git a/cube/src/main/assembly/assemble.xml b/cube/src/main/assembly/assemble.xml
new file mode 100644
index 0000000..c26c222
--- /dev/null
+++ b/cube/src/main/assembly/assemble.xml
@@ -0,0 +1,19 @@
+<assembly xmlns="http://maven.apache.org/plugins/maven-assembly-plugin/assembly/1.1.2" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+          xsi:schemaLocation="http://maven.apache.org/plugins/maven-assembly-plugin/assembly/1.1.2 http://maven.apache.org/xsd/assembly-1.1.2.xsd">
+    <id>jar-with-dependencies</id>
+    <formats>
+        <format>jar</format>
+    </formats>
+    <includeBaseDirectory>false</includeBaseDirectory>
+    <dependencySets>
+        <dependencySet>
+            <outputDirectory>/</outputDirectory>
+            <useProjectArtifact>true</useProjectArtifact>
+            <unpack>true</unpack>
+            <scope>runtime</scope>
+            <excludes>
+                <exclude>javax.servlet*:*</exclude>
+            </excludes>
+        </dependencySet>
+    </dependencySets>
+</assembly>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/cube/src/main/java/com/kylinolap/cube/CubeBuildTypeEnum.java
----------------------------------------------------------------------
diff --git a/cube/src/main/java/com/kylinolap/cube/CubeBuildTypeEnum.java b/cube/src/main/java/com/kylinolap/cube/CubeBuildTypeEnum.java
new file mode 100644
index 0000000..7299747
--- /dev/null
+++ b/cube/src/main/java/com/kylinolap/cube/CubeBuildTypeEnum.java
@@ -0,0 +1,32 @@
+/*
+ * 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.cube;
+
+/**
+ * @author xduo
+ * 
+ */
+public enum CubeBuildTypeEnum {
+    /**
+     * rebuild a segment or incremental build
+     */
+    BUILD,
+    /**
+     * merge segments
+     */
+    MERGE;
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/cube/src/main/java/com/kylinolap/cube/CubeInstance.java
----------------------------------------------------------------------
diff --git a/cube/src/main/java/com/kylinolap/cube/CubeInstance.java b/cube/src/main/java/com/kylinolap/cube/CubeInstance.java
new file mode 100644
index 0000000..942332c
--- /dev/null
+++ b/cube/src/main/java/com/kylinolap/cube/CubeInstance.java
@@ -0,0 +1,435 @@
+/*
+ * 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.cube;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+
+import com.fasterxml.jackson.annotation.JsonAutoDetect;
+import com.fasterxml.jackson.annotation.JsonAutoDetect.Visibility;
+import com.fasterxml.jackson.annotation.JsonIgnore;
+import com.fasterxml.jackson.annotation.JsonManagedReference;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.google.common.base.Objects;
+import com.google.common.collect.Lists;
+import com.kylinolap.common.KylinConfig;
+import com.kylinolap.common.persistence.ResourceStore;
+import com.kylinolap.common.persistence.RootPersistentEntity;
+import com.kylinolap.metadata.MetadataManager;
+import com.kylinolap.metadata.model.cube.CubeDesc;
+import com.kylinolap.metadata.model.cube.CubePartitionDesc;
+import com.kylinolap.metadata.model.invertedindex.InvertedIndexDesc;
+
+@JsonAutoDetect(fieldVisibility = Visibility.NONE, getterVisibility = Visibility.NONE, isGetterVisibility = Visibility.NONE, setterVisibility = Visibility.NONE)
+public class CubeInstance extends RootPersistentEntity {
+
+    public static CubeInstance create(String cubeName, String projectName, CubeDesc cubeDesc) {
+        CubeInstance cubeInstance = new CubeInstance();
+
+        cubeInstance.setConfig(cubeDesc.getConfig());
+        cubeInstance.setName(cubeName);
+        cubeInstance.setDescName(cubeDesc.getName());
+        cubeInstance.setCreateTime(formatTime(System.currentTimeMillis()));
+        cubeInstance.setSegments(new ArrayList<CubeSegment>());
+        cubeInstance.setStatus(CubeStatusEnum.DISABLED);
+        cubeInstance.updateRandomUuid();
+
+        return cubeInstance;
+    }
+
+    @JsonIgnore
+    private KylinConfig config;
+    @JsonProperty("name")
+    private String name;
+    @JsonProperty("owner")
+    private String owner;
+    @JsonProperty("version")
+    private String version; // user info only, we don't do version control
+    @JsonProperty("descriptor")
+    private String descName;
+    // Mark cube priority for query
+    @JsonProperty("cost")
+    private int cost = 50;
+    @JsonProperty("status")
+    private CubeStatusEnum status;
+
+    @JsonManagedReference
+    @JsonProperty("segments")
+    private List<CubeSegment> segments = new ArrayList<CubeSegment>();
+
+    @JsonProperty("create_time")
+    private String createTime;
+
+    public List<CubeSegment> getBuildingSegments() {
+        List<CubeSegment> buildingSegments = new ArrayList<CubeSegment>();
+        if (null != segments) {
+            for (CubeSegment segment : segments) {
+                if (CubeSegmentStatusEnum.NEW == segment.getStatus() || CubeSegmentStatusEnum.READY_PENDING == segment.getStatus()) {
+                    buildingSegments.add(segment);
+                }
+            }
+        }
+
+        return buildingSegments;
+    }
+
+    public long getAllocatedEndDate() {
+        if (null == segments || segments.size() == 0) {
+            return 0;
+        }
+
+        Collections.sort(segments);
+
+        return segments.get(segments.size() - 1).getDateRangeEnd();
+    }
+
+    public long getAllocatedStartDate() {
+        if (null == segments || segments.size() == 0) {
+            return 0;
+        }
+
+        Collections.sort(segments);
+
+        return segments.get(0).getDateRangeStart();
+    }
+
+    public List<CubeSegment> getMergingSegments() {
+        return this.getMergingSegments(null);
+    }
+
+    public List<CubeSegment> getMergingSegments(CubeSegment cubeSegment) {
+        CubeSegment buildingSegment;
+        if (cubeSegment == null) {
+            List<CubeSegment> buildingSegments = getBuildingSegments();
+            if (buildingSegments.size() == 0) {
+                return Collections.emptyList();
+            }
+            buildingSegment = buildingSegments.get(0);
+        } else {
+            buildingSegment = cubeSegment;
+        }
+
+        List<CubeSegment> mergingSegments = new ArrayList<CubeSegment>();
+        if (null != this.segments) {
+            for (CubeSegment segment : this.segments) {
+                if (segment.getStatus() == CubeSegmentStatusEnum.READY) {
+                    if (buildingSegment.getDateRangeStart() <= segment.getDateRangeStart() && buildingSegment.getDateRangeEnd() >= segment.getDateRangeEnd()) {
+                        mergingSegments.add(segment);
+                    }
+                }
+            }
+        }
+        return mergingSegments;
+
+    }
+
+    public List<CubeSegment> getRebuildingSegments() {
+        List<CubeSegment> buildingSegments = getBuildingSegments();
+        if (buildingSegments.size() == 0) {
+            return Collections.emptyList();
+        } else {
+            List<CubeSegment> rebuildingSegments = new ArrayList<CubeSegment>();
+            if (null != this.segments) {
+                long startDate = buildingSegments.get(0).getDateRangeStart();
+                long endDate = buildingSegments.get(buildingSegments.size() - 1).getDateRangeEnd();
+                for (CubeSegment segment : this.segments) {
+                    if (segment.getStatus() == CubeSegmentStatusEnum.READY) {
+                        if (startDate >= segment.getDateRangeStart() && startDate < segment.getDateRangeEnd() && segment.getDateRangeEnd() < endDate) {
+                            rebuildingSegments.add(segment);
+                            continue;
+                        }
+                        if (startDate <= segment.getDateRangeStart() && endDate >= segment.getDateRangeEnd()) {
+                            rebuildingSegments.add(segment);
+                            continue;
+                        }
+                    }
+                }
+            }
+
+            return rebuildingSegments;
+        }
+    }
+
+    public CubeDesc getDescriptor() {
+        return MetadataManager.getInstance(config).getCubeDesc(descName);
+    }
+
+    public InvertedIndexDesc getInvertedIndexDesc() {
+        return MetadataManager.getInstance(config).getInvertedIndexDesc(name);
+    }
+
+    public boolean isInvertedIndex() {
+        return getInvertedIndexDesc() != null;
+    }
+
+    public boolean isReady() {
+        return getStatus() == CubeStatusEnum.READY;
+    }
+
+    public String getResourcePath() {
+        return concatResourcePath(name);
+    }
+
+    public static String concatResourcePath(String cubeName) {
+        return ResourceStore.CUBE_RESOURCE_ROOT + "/" + cubeName + ".json";
+    }
+
+    @Override
+    public String toString() {
+        return "Cube [name=" + name + "]";
+    }
+
+    // ============================================================================
+
+    @JsonProperty("size_kb")
+    public long getSizeKB() {
+        long sizeKb = 0L;
+
+        for (CubeSegment cubeSegment : this.getSegments(CubeSegmentStatusEnum.READY)) {
+            sizeKb += cubeSegment.getSizeKB();
+        }
+
+        return sizeKb;
+    }
+
+    @JsonProperty("source_records_count")
+    public long getSourceRecordCount() {
+        long sizeRecordCount = 0L;
+
+        for (CubeSegment cubeSegment : this.getSegments(CubeSegmentStatusEnum.READY)) {
+            sizeRecordCount += cubeSegment.getSourceRecords();
+        }
+
+        return sizeRecordCount;
+    }
+
+    @JsonProperty("source_records_size")
+    public long getSourceRecordSize() {
+        long sizeRecordSize = 0L;
+
+        for (CubeSegment cubeSegment : this.getSegments(CubeSegmentStatusEnum.READY)) {
+            sizeRecordSize += cubeSegment.getSourceRecordsSize();
+        }
+
+        return sizeRecordSize;
+    }
+
+    public KylinConfig getConfig() {
+        return config;
+    }
+
+    public void setConfig(KylinConfig config) {
+        this.config = config;
+    }
+
+    public String getName() {
+        return name;
+    }
+
+    public void setName(String name) {
+        this.name = name;
+    }
+
+    public String getOwner() {
+        return owner;
+    }
+
+    public void setOwner(String owner) {
+        this.owner = owner;
+    }
+
+    public String getVersion() {
+        return version;
+    }
+
+    public void setVersion(String version) {
+        this.version = version;
+    }
+
+    public String getDescName() {
+        return descName.toUpperCase();
+    }
+
+    public String getOriginDescName() {
+        return descName;
+    }
+
+    public void setDescName(String descName) {
+        this.descName = descName;
+    }
+
+    public int getCost() {
+        return cost;
+    }
+
+    public void setCost(int cost) {
+        this.cost = cost;
+    }
+
+    public CubeStatusEnum getStatus() {
+        return status;
+    }
+
+    public void setStatus(CubeStatusEnum status) {
+        this.status = status;
+    }
+
+    public CubeSegment getFirstSegment() {
+        if (this.segments == null || this.segments.size() == 0) {
+            return null;
+        } else {
+            return this.segments.get(0);
+        }
+    }
+
+    public CubeSegment getLatestReadySegment() {
+        CubeSegment latest = null;
+        for (int i = segments.size() - 1; i >= 0; i--) {
+            CubeSegment seg = segments.get(i);
+            if (seg.getStatus() != CubeSegmentStatusEnum.READY)
+                continue;
+            if (latest == null || latest.getDateRangeEnd() < seg.getDateRangeEnd()) {
+                latest = seg;
+            }
+        }
+        return latest;
+    }
+
+    public List<CubeSegment> getSegments() {
+        return segments;
+    }
+
+    public List<CubeSegment> getSegments(CubeSegmentStatusEnum status) {
+        List<CubeSegment> result = new ArrayList<CubeSegment>();
+
+        for (CubeSegment segment : segments) {
+            if (segment.getStatus() == status) {
+                result.add(segment);
+            }
+        }
+
+        return result;
+    }
+
+    public List<CubeSegment> getSegment(CubeSegmentStatusEnum status) {
+        List<CubeSegment> result = Lists.newArrayList();
+        for (CubeSegment segment : segments) {
+            if (segment.getStatus() == status) {
+                result.add(segment);
+            }
+        }
+        return result;
+    }
+
+    public CubeSegment getSegment(String name, CubeSegmentStatusEnum status) {
+        for (CubeSegment segment : segments) {
+            if ((null != segment.getName() && segment.getName().equals(name)) && segment.getStatus() == status) {
+                return segment;
+            }
+        }
+
+        return null;
+    }
+
+    public void setSegments(List<CubeSegment> segments) {
+        this.segments = segments;
+    }
+
+    public CubeSegment getSegmentById(String segmentId) {
+        for (CubeSegment segment : segments) {
+            if (Objects.equal(segment.getUuid(), segmentId)) {
+                return segment;
+            }
+        }
+        return null;
+    }
+
+    public String getCreateTime() {
+        return createTime;
+    }
+
+    public void setCreateTime(String createTime) {
+        this.createTime = createTime;
+    }
+
+    public long[] getDateRange() {
+        List<CubeSegment> readySegments = getSegment(CubeSegmentStatusEnum.READY);
+        if (readySegments.isEmpty()) {
+            return new long[]{0L, 0L};
+        }
+        long start = Long.MAX_VALUE;
+        long end = Long.MIN_VALUE;
+        for (CubeSegment segment : readySegments) {
+            if (segment.getDateRangeStart() < start) {
+                start = segment.getDateRangeStart();
+            }
+            if (segment.getDateRangeEnd() > end) {
+                end = segment.getDateRangeEnd();
+            }
+        }
+        return new long[]{start, end};
+    }
+
+    private boolean appendOnHll() {
+        CubePartitionDesc cubePartitionDesc = getDescriptor().getCubePartitionDesc();
+        if (cubePartitionDesc == null) {
+            return false;
+        }
+        if (cubePartitionDesc.getPartitionDateColumn() == null) {
+            return false;
+        }
+        if (cubePartitionDesc.getCubePartitionType() != CubePartitionDesc.CubePartitionType.APPEND) {
+            return false;
+        }
+        return getDescriptor().hasHolisticCountDistinctMeasures();
+    }
+
+    public boolean appendBuildOnHllMeasure(long startDate, long endDate) {
+        if (!appendOnHll()) {
+            return false;
+        }
+        List<CubeSegment> readySegments = getSegment(CubeSegmentStatusEnum.READY);
+        if (readySegments.isEmpty()) {
+            return false;
+        }
+        for (CubeSegment readySegment: readySegments) {
+            if (readySegment.getDateRangeStart() == startDate && readySegment.getDateRangeEnd() == endDate) {
+                //refresh build
+                return false;
+            }
+        }
+        return true;
+    }
+
+    public boolean needMergeImmediatelyAfterBuild(CubeSegment segment) {
+        if (!appendOnHll()) {
+            return false;
+        }
+        List<CubeSegment> readySegments = getSegment(CubeSegmentStatusEnum.READY);
+        if (readySegments.isEmpty()) {
+            return false;
+        }
+        for (CubeSegment readySegment: readySegments) {
+            if (readySegment.getDateRangeEnd() > segment.getDateRangeStart()) {
+                //has overlap and not refresh
+                return true;
+            }
+        }
+        return false;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/cube/src/main/java/com/kylinolap/cube/CubeManager.java
----------------------------------------------------------------------
diff --git a/cube/src/main/java/com/kylinolap/cube/CubeManager.java b/cube/src/main/java/com/kylinolap/cube/CubeManager.java
new file mode 100644
index 0000000..a1d2f27
--- /dev/null
+++ b/cube/src/main/java/com/kylinolap/cube/CubeManager.java
@@ -0,0 +1,660 @@
+/*
+ * 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.cube;
+
+import java.io.IOException;
+import java.util.*;
+import java.util.concurrent.ConcurrentHashMap;
+
+import com.kylinolap.dict.DateStrDictionary;
+import org.apache.commons.lang3.StringUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.kylinolap.common.KylinConfig;
+import com.kylinolap.common.persistence.JsonSerializer;
+import com.kylinolap.common.persistence.ResourceStore;
+import com.kylinolap.common.persistence.Serializer;
+import com.kylinolap.common.restclient.Broadcaster;
+import com.kylinolap.common.restclient.SingleValueCache;
+import com.kylinolap.cube.exception.CubeIntegrityException;
+import com.kylinolap.cube.project.ProjectInstance;
+import com.kylinolap.cube.project.ProjectManager;
+import com.kylinolap.dict.Dictionary;
+import com.kylinolap.dict.DictionaryInfo;
+import com.kylinolap.dict.DictionaryManager;
+import com.kylinolap.dict.lookup.HiveTable;
+import com.kylinolap.dict.lookup.LookupStringTable;
+import com.kylinolap.dict.lookup.SnapshotManager;
+import com.kylinolap.dict.lookup.SnapshotTable;
+import com.kylinolap.metadata.MetadataManager;
+import com.kylinolap.metadata.model.cube.CubeDesc;
+import com.kylinolap.metadata.model.cube.DimensionDesc;
+import com.kylinolap.metadata.model.cube.TblColRef;
+import com.kylinolap.metadata.model.invertedindex.InvertedIndexDesc;
+import com.kylinolap.metadata.model.schema.ColumnDesc;
+import com.kylinolap.metadata.model.schema.TableDesc;
+
+/**
+ * @author yangli9
+ */
+public class CubeManager {
+
+    private static String ALPHA_NUM = "0123456789ABCDEFGHIJKLMNOPQRSTUVWXYZ";
+
+    private static int HBASE_TABLE_LENGTH = 10;
+
+    private static final Serializer<CubeInstance> CUBE_SERIALIZER = new JsonSerializer<CubeInstance>(CubeInstance.class);
+
+    private static final Logger logger = LoggerFactory.getLogger(CubeManager.class);
+
+    // static cached instances
+    private static final ConcurrentHashMap<KylinConfig, CubeManager> CACHE = new ConcurrentHashMap<KylinConfig, CubeManager>();
+
+    public static CubeManager getInstance(KylinConfig config) {
+        CubeManager r = CACHE.get(config);
+        if (r != null) {
+            return r;
+        }
+
+        synchronized (CubeManager.class) {
+            r = CACHE.get(config);
+            if (r != null) {
+                return r;
+            }
+            try {
+                r = new CubeManager(config);
+                CACHE.put(config, r);
+                if (CACHE.size() > 1) {
+                    logger.warn("More than one singleton exist");
+                }
+                return r;
+            } catch (IOException e) {
+                throw new IllegalStateException("Failed to init CubeManager from " + config, e);
+            }
+        }
+    }
+
+    public static void clearCache() {
+        CACHE.clear();
+    }
+
+    public static synchronized void removeInstance(KylinConfig config) {
+        CACHE.remove(config);
+    }
+
+    // ============================================================================
+
+    private KylinConfig config;
+    // cube name ==> CubeInstance
+    private SingleValueCache<String, CubeInstance> cubeMap = new SingleValueCache<String, CubeInstance>(Broadcaster.TYPE.CUBE);
+    // "table/column" ==> lookup table
+    private SingleValueCache<String, LookupStringTable> lookupTables = new SingleValueCache<String, LookupStringTable>(Broadcaster.TYPE.METADATA);
+
+    // for generation hbase table name of a new segment
+    private HashSet<String> usedStorageLocation = new HashSet<String>();
+
+    private CubeManager(KylinConfig config) throws IOException {
+        logger.info("Initializing CubeManager with config " + config);
+        this.config = config;
+
+        loadAllCubeInstance();
+    }
+
+    public List<CubeInstance> listAllCubes() {
+        return new ArrayList<CubeInstance>(cubeMap.values());
+    }
+
+    public CubeInstance getCube(String cubeName) {
+        cubeName = cubeName.toUpperCase();
+        return cubeMap.get(cubeName);
+    }
+
+    /**
+     * Get related Cubes by cubedesc name. By default, the desc name will be
+     * translated into upper case.
+     *
+     * @param descName CubeDesc name
+     * @return
+     */
+    public List<CubeInstance> getCubesByDesc(String descName) {
+
+        descName = descName.toUpperCase();
+        List<CubeInstance> list = listAllCubes();
+        List<CubeInstance> result = new ArrayList<CubeInstance>();
+        Iterator<CubeInstance> it = list.iterator();
+        while (it.hasNext()) {
+            CubeInstance ci = it.next();
+            if (descName.equals(ci.getDescName())) {
+                result.add(ci);
+            }
+        }
+        return result;
+    }
+
+    public void buildInvertedIndexDictionary(CubeSegment cubeSeg, String factColumnsPath) throws IOException {
+        DictionaryManager dictMgr = getDictionaryManager();
+
+        InvertedIndexDesc iiDesc = cubeSeg.getCubeInstance().getInvertedIndexDesc();
+        TableDesc tableDesc = iiDesc.getFactTableDesc();
+        for (ColumnDesc colDesc : tableDesc.getColumns()) {
+            TblColRef col = new TblColRef(colDesc);
+            if (iiDesc.isMetricsCol(col))
+                continue;
+            
+            DictionaryInfo dict = dictMgr.buildDictionary(null, col, factColumnsPath);
+            cubeSeg.putDictResPath(col, dict.getResourcePath());
+        }
+
+        saveResource(cubeSeg.getCubeInstance());
+    }
+
+    public DictionaryInfo buildDictionary(CubeSegment cubeSeg, TblColRef col, String factColumnsPath) throws IOException {
+        if (!cubeSeg.getCubeDesc().getRowkey().isUseDictionary(col))
+            return null;
+
+        DictionaryManager dictMgr = getDictionaryManager();
+        DictionaryInfo dictInfo = dictMgr.buildDictionary(cubeSeg.getCubeDesc(), col, factColumnsPath);
+        cubeSeg.putDictResPath(col, dictInfo.getResourcePath());
+
+        saveResource(cubeSeg.getCubeInstance());
+
+        return dictInfo;
+    }
+
+    /**
+     * return null if no dictionary for given column
+     */
+    public Dictionary<?> getDictionary(CubeSegment cubeSeg, TblColRef col) {
+        DictionaryInfo info = null;
+        try {
+            DictionaryManager dictMgr = getDictionaryManager();
+            // logger.info("Using metadata url " + metadataUrl +
+            // " for DictionaryManager");
+            String dictResPath = cubeSeg.getDictResPath(col);
+            if (dictResPath == null)
+                return null;
+
+            info = dictMgr.getDictionaryInfo(dictResPath);
+            if (info == null)
+                throw new IllegalStateException("No dictionary found by " + dictResPath + ", invalid cube state; cube segment" + cubeSeg + ", col " + col);
+        } catch (IOException e) {
+            throw new IllegalStateException("Failed to get dictionary for cube segment" + cubeSeg + ", col" + col, e);
+        }
+
+        return info.getDictionaryObject();
+    }
+
+    public SnapshotTable buildSnapshotTable(CubeSegment cubeSeg, String lookupTable) throws IOException {
+        MetadataManager metaMgr = getMetadataManager();
+        SnapshotManager snapshotMgr = getSnapshotManager();
+
+        HiveTable hiveTable = new HiveTable(metaMgr, lookupTable);
+        TableDesc tableDesc = metaMgr.getTableDesc(lookupTable);
+        SnapshotTable snapshot = snapshotMgr.buildSnapshot(hiveTable, tableDesc);
+
+        cubeSeg.putSnapshotResPath(lookupTable, snapshot.getResourcePath());
+
+        saveResource(cubeSeg.getCubeInstance());
+
+        return snapshot;
+    }
+
+    // sync on update
+    public CubeInstance dropCube(String cubeName, boolean deleteDesc) throws IOException {
+        logger.info("Dropping cube '" + cubeName + "'");
+        // load projects before remove cube from project
+        List<ProjectInstance> projects = ProjectManager.getInstance(config).getProjects(cubeName);
+
+        ResourceStore store = getStore();
+
+        // delete cube instance and cube desc
+        CubeInstance cube = getCube(cubeName);
+
+        if (deleteDesc && cube.getDescriptor() != null)
+            store.deleteResource(cube.getDescriptor().getResourcePath());
+
+        store.deleteResource(cube.getResourcePath());
+
+        // delete cube from project
+        ProjectManager.getInstance(config).removeCubeFromProjects(cubeName);
+
+        // clean cube cache
+        this.afterCubeDroped(cube, projects);
+
+        return cube;
+    }
+
+    // sync on update
+    public CubeInstance createCube(String cubeName, String projectName, CubeDesc desc, String owner) throws IOException {
+        logger.info("Creating cube '" + projectName + "-->" + cubeName + "' from desc '" + desc.getName() + "'");
+
+        // save cube resource
+        CubeInstance cube = CubeInstance.create(cubeName, projectName, desc);
+        cube.setOwner(owner);
+        saveResource(cube);
+
+        ProjectManager.getInstance(config).updateCubeToProject(cubeName, projectName, owner);
+
+        return cube;
+    }
+
+    public CubeInstance updateCube(CubeInstance cube) throws IOException {
+        logger.info("Updating cube instance '" + cube.getName());
+
+        // save resource
+        saveResource(cube);
+
+        logger.info("Cube with " + cube.getSegments().size() + " segments is saved");
+
+        return cube;
+    }
+
+    public List<CubeSegment> allocateSegments(CubeInstance cubeInstance, CubeBuildTypeEnum buildType, long startDate, long endDate) throws IOException, CubeIntegrityException {
+        if (cubeInstance.getBuildingSegments().size() > 0) {
+            throw new RuntimeException("There is already an allocating segment!");
+        }
+        List<CubeSegment> segments = new ArrayList<CubeSegment>();
+
+        final boolean appendBuildOnHllMeasure = cubeInstance.appendBuildOnHllMeasure(startDate, endDate);
+        if (null != cubeInstance.getDescriptor().getCubePartitionDesc().getPartitionDateColumn()) {
+            if (appendBuildOnHllMeasure) {
+                long[] dateRange = cubeInstance.getDateRange();
+                segments.add(buildSegment(cubeInstance, dateRange[0], endDate));
+            } else {
+
+                if (startDate == 0 && cubeInstance.getSegments().size() == 0) {
+                    startDate = cubeInstance.getDescriptor().getCubePartitionDesc().getPartitionDateStart();
+                }
+
+                // incremental build
+                CubeSegment lastSegment = null;
+                for (CubeSegment segment : cubeInstance.getSegments()) {
+                    if (segment.getDateRangeStart() == startDate) {
+                        // refresh or merge
+                        segments.add(buildSegment(cubeInstance, startDate, endDate));
+                    }
+                    if (segment.getDateRangeStart() < startDate && startDate < segment.getDateRangeEnd()) {
+                        // delete-insert
+                        segments.add(buildSegment(cubeInstance, segment.getDateRangeStart(), startDate));
+                        segments.add(buildSegment(cubeInstance, startDate, endDate));
+                    }
+                    lastSegment = segment;
+                }
+
+                // append
+                if (null == lastSegment || (lastSegment.getDateRangeEnd() == startDate)) {
+                    segments.add(buildSegment(cubeInstance, startDate, endDate));
+                }
+            }
+        } else {
+            segments.add(buildSegment(cubeInstance, 0, 0));
+        }
+
+        validateNewSegments(cubeInstance, buildType, segments);
+
+        CubeSegment newSeg = segments.get(0);
+        if (buildType == CubeBuildTypeEnum.MERGE) {
+            List<CubeSegment> mergingSegments = cubeInstance.getMergingSegments(newSeg);
+            this.makeDictForNewSegment(cubeInstance, newSeg, mergingSegments);
+            this.makeSnapshotForNewSegment(cubeInstance, newSeg, mergingSegments);
+        } else if (appendBuildOnHllMeasure) {
+            List<CubeSegment> mergingSegments = cubeInstance.getSegment(CubeSegmentStatusEnum.READY);
+            this.makeDictForNewSegment(cubeInstance, newSeg, mergingSegments);
+            this.makeSnapshotForNewSegment(cubeInstance, newSeg, mergingSegments);
+        }
+
+        cubeInstance.getSegments().addAll(segments);
+        Collections.sort(cubeInstance.getSegments());
+
+        this.updateCube(cubeInstance);
+
+        return segments;
+    }
+
+    public static String getHBaseStorageLocationPrefix() {
+        return "KYLIN_";
+        //return getHbaseStorageLocationPrefix(config.getMetadataUrl());
+    }
+
+    /**
+     * For each cube htable, we leverage htable's metadata to keep track of
+     * which kylin server(represented by its kylin_metadata prefix) owns this htable
+     */
+    public static  String getHtableMetadataKey() {
+        return "KYLIN_HOST";
+    }
+
+    public void updateSegmentOnJobSucceed(CubeInstance cubeInstance, CubeBuildTypeEnum buildType, String segmentName, String jobUuid, long lastBuildTime, long sizeKB, long sourceRecordCount, long sourceRecordsSize) throws IOException, CubeIntegrityException {
+
+        List<CubeSegment> segmentsInNewStatus = cubeInstance.getSegments(CubeSegmentStatusEnum.NEW);
+        CubeSegment cubeSegment = cubeInstance.getSegmentById(jobUuid);
+        if (cubeSegment == null) {
+            cubeSegment = cubeInstance.getSegment(segmentName, CubeSegmentStatusEnum.NEW);
+        }
+
+        switch (buildType) {
+        case BUILD:
+            if (cubeInstance.needMergeImmediatelyAfterBuild(cubeSegment)) {
+                cubeInstance.getSegments().removeAll(cubeInstance.getMergingSegments());
+            } else {
+                if (segmentsInNewStatus.size() == 1) {// if this the last segment in
+                    // status of NEW
+                    // remove all the rebuilding/impacted segments
+                    cubeInstance.getSegments().removeAll(cubeInstance.getRebuildingSegments());
+                }
+            }
+            break;
+        case MERGE:
+            cubeInstance.getSegments().removeAll(cubeInstance.getMergingSegments());
+            break;
+        }
+
+        cubeSegment.setLastBuildJobID(jobUuid);
+        cubeSegment.setLastBuildTime(lastBuildTime);
+        cubeSegment.setSizeKB(sizeKB);
+        cubeSegment.setSourceRecords(sourceRecordCount);
+        cubeSegment.setSourceRecordsSize(sourceRecordsSize);
+        if (segmentsInNewStatus.size() == 1) {
+            cubeSegment.setStatus(CubeSegmentStatusEnum.READY);
+            cubeInstance.setStatus(CubeStatusEnum.READY);
+
+            for (CubeSegment seg : cubeInstance.getSegments(CubeSegmentStatusEnum.READY_PENDING)) {
+                seg.setStatus(CubeSegmentStatusEnum.READY);
+            }
+        } else {
+            cubeSegment.setStatus(CubeSegmentStatusEnum.READY_PENDING);
+        }
+        this.updateCube(cubeInstance);
+    }
+
+    public void updateSegmentOnJobDiscard(CubeInstance cubeInstance, String segmentName) throws IOException, CubeIntegrityException {
+        for (int i = 0; i < cubeInstance.getSegments().size(); i++) {
+            CubeSegment segment = cubeInstance.getSegments().get(i);
+            if (segment.getName().equals(segmentName) && segment.getStatus() != CubeSegmentStatusEnum.READY) {
+                cubeInstance.getSegments().remove(segment);
+            }
+        }
+        updateCube(cubeInstance);
+    }
+
+    /**
+     * After cube update, reload cube related cache
+     *
+     * @param cube
+     */
+    public void loadCubeCache(CubeInstance cube) {
+        try {
+            loadCubeInstance(cube.getResourcePath());
+        } catch (IOException e) {
+            logger.error(e.getLocalizedMessage(), e);
+        }
+    }
+
+    /**
+     * After cube deletion, remove cube related cache
+     *
+     * @param cube
+     */
+    public void removeCubeCache(CubeInstance cube) {
+        cubeMap.remove(cube.getName().toUpperCase());
+
+        for (CubeSegment segment : cube.getSegments()) {
+            usedStorageLocation.remove(segment.getName());
+        }
+    }
+
+    public LookupStringTable getLookupTable(CubeSegment cubeSegment, DimensionDesc dim) {
+
+        String tableName = dim.getTable();
+        String[] pkCols = dim.getJoin().getPrimaryKey();
+        String key = tableName + "#" + StringUtils.join(pkCols, ",");
+
+        LookupStringTable r = lookupTables.get(key);
+        if (r == null) {
+            String snapshotResPath = cubeSegment.getSnapshotResPath(tableName);
+            if (snapshotResPath == null)
+                throw new IllegalStateException("No snaphot for table '" + tableName + "' found on cube segment" + cubeSegment.getCubeInstance().getName() + "/" + cubeSegment);
+
+            try {
+                SnapshotTable snapshot = getSnapshotManager().getSnapshotTable(snapshotResPath);
+                TableDesc tableDesc = getMetadataManager().getTableDesc(tableName);
+                r = new LookupStringTable(tableDesc, pkCols, snapshot);
+            } catch (IOException e) {
+                throw new IllegalStateException("Failed to load lookup table " + tableName + " from snapshot " + snapshotResPath, e);
+            }
+
+            lookupTables.putLocal(key, r);
+        }
+
+        return r;
+    }
+
+    /**
+     * For the new segment, we need to create dictionaries for it, too. For
+     * those dictionaries on fact table, create it by merging underlying
+     * dictionaries For those dictionaries on lookup table, just copy it from
+     * any one of the merging segments, it's guaranteed to be consistent(checked
+     * in CubeSegmentValidator)
+     *
+     * @param cube
+     * @param newSeg
+     * @throws IOException
+     */
+    private void makeDictForNewSegment(CubeInstance cube, CubeSegment newSeg, List<CubeSegment> mergingSegments) throws IOException {
+        HashSet<TblColRef> colsNeedMeringDict = new HashSet<TblColRef>();
+        HashSet<TblColRef> colsNeedCopyDict = new HashSet<TblColRef>();
+        DictionaryManager dictMgr = this.getDictionaryManager();
+
+        for (DimensionDesc dim : cube.getDescriptor().getDimensions()) {
+            for (TblColRef col : dim.getColumnRefs()) {
+                if (newSeg.getCubeDesc().getRowkey().isUseDictionary(col)) {
+                    if (cube.getDescriptor().getFactTable().equalsIgnoreCase((String) dictMgr.decideSourceData(cube.getDescriptor(), col, null)[0])) {
+                        colsNeedMeringDict.add(col);
+                    } else {
+                        colsNeedCopyDict.add(col);
+                    }
+                }
+            }
+        }
+
+        for (TblColRef col : colsNeedMeringDict) {
+            logger.info("Merging fact table dictionary on : " + col);
+            List<DictionaryInfo> dictInfos = new ArrayList<DictionaryInfo>();
+            for (CubeSegment segment : mergingSegments) {
+                logger.info("Including fact table dictionary of segment : " + segment);
+                DictionaryInfo dictInfo = dictMgr.getDictionaryInfo(segment.getDictResPath(col));
+                dictInfos.add(dictInfo);
+            }
+            this.mergeDictionaries(newSeg, dictInfos, col);
+        }
+
+        for (TblColRef col : colsNeedCopyDict) {
+            String path = mergingSegments.get(0).getDictResPath(col);
+            newSeg.putDictResPath(col, path);
+        }
+    }
+
+    /**
+     * make snapshots for the new segment by copying from one of the underlying
+     * merging segments. it's ganranteed to be consistent(checked in
+     * CubeSegmentValidator)
+     *
+     * @param cube
+     * @param newSeg
+     */
+    private void makeSnapshotForNewSegment(CubeInstance cube, CubeSegment newSeg, List<CubeSegment> mergingSegments) {
+        for (Map.Entry<String, String> entry : mergingSegments.get(0).getSnapshots().entrySet()) {
+            newSeg.putSnapshotResPath(entry.getKey(), entry.getValue());
+        }
+    }
+
+    private DictionaryInfo mergeDictionaries(CubeSegment cubeSeg, List<DictionaryInfo> dicts, TblColRef col) throws IOException {
+        DictionaryManager dictMgr = getDictionaryManager();
+        DictionaryInfo dictInfo = dictMgr.mergeDictionary(dicts);
+        cubeSeg.putDictResPath(col, dictInfo.getResourcePath());
+
+        return dictInfo;
+    }
+
+    private void saveResource(CubeInstance cube) throws IOException {
+        ResourceStore store = getStore();
+        store.putResource(cube.getResourcePath(), cube, CUBE_SERIALIZER);
+        this.afterCubeUpdated(cube);
+    }
+
+    private void afterCubeUpdated(CubeInstance updatedCube) {
+        MetadataManager.getInstance(config).reload();
+        cubeMap.put(updatedCube.getName().toUpperCase(), updatedCube);
+
+        for (ProjectInstance project : ProjectManager.getInstance(config).getProjects(updatedCube.getName())) {
+            try {
+                ProjectManager.getInstance(config).loadProjectCache(project, true);
+            } catch (IOException e) {
+                logger.error(e.getLocalizedMessage(), e);
+            }
+        }
+    }
+
+    private void afterCubeDroped(CubeInstance droppedCube, List<ProjectInstance> projects) {
+        MetadataManager.getInstance(config).reload();
+        removeCubeCache(droppedCube);
+
+        if (null != projects) {
+            for (ProjectInstance project : projects) {
+                try {
+                    ProjectManager.getInstance(config).loadProjectCache(project, true);
+                } catch (IOException e) {
+                    logger.error(e.getLocalizedMessage(), e);
+                }
+            }
+        }
+    }
+
+    /**
+     * @param cubeInstance
+     * @param startDate    (pass 0 if full build)
+     * @param endDate      (pass 0 if full build)
+     * @return
+     */
+    private CubeSegment buildSegment(CubeInstance cubeInstance, long startDate, long endDate) {
+        CubeSegment segment = new CubeSegment();
+        String incrementalSegName = CubeSegment.getSegmentName(startDate, endDate);
+        segment.setUuid(UUID.randomUUID().toString());
+        segment.setName(incrementalSegName);
+        segment.setCreateTime(DateStrDictionary.dateToString(new Date()));
+        segment.setDateRangeStart(startDate);
+        segment.setDateRangeEnd(endDate);
+        segment.setStatus(CubeSegmentStatusEnum.NEW);
+        segment.setStorageLocationIdentifier(generateStorageLocation());
+
+        segment.setCubeInstance(cubeInstance);
+
+        return segment;
+    }
+
+    private String generateStorageLocation() {
+        String namePrefix = getHBaseStorageLocationPrefix();
+        String tableName = "";
+        do {
+            StringBuffer sb = new StringBuffer();
+            sb.append(namePrefix);
+            for (int i = 0; i < HBASE_TABLE_LENGTH; i++) {
+                int idx = (int) (Math.random() * ALPHA_NUM.length());
+                sb.append(ALPHA_NUM.charAt(idx));
+            }
+            tableName = sb.toString();
+        } while (this.usedStorageLocation.contains(tableName));
+
+        return tableName;
+    }
+
+    /**
+     */
+    private void validateNewSegments(CubeInstance cubeInstance, CubeBuildTypeEnum buildType, List<CubeSegment> newSegments) throws CubeIntegrityException {
+        if (null == cubeInstance.getDescriptor().getCubePartitionDesc().getPartitionDateColumn()) {
+            // do nothing for non-incremental build
+            return;
+        }
+
+        if (newSegments.size() == 0) {
+            throw new CubeIntegrityException("Failed to allocate any segment.");
+        }
+
+        for (CubeSegment segment : newSegments) {
+            if (segment.getDateRangeEnd() <= segment.getDateRangeStart()) {
+                throw new CubeIntegrityException(" end date.");
+            }
+        }
+
+        CubeSegmentValidator cubeSegmentValidator = CubeSegmentValidator.getCubeSegmentValidator(buildType, cubeInstance.getDescriptor().getCubePartitionDesc().getCubePartitionType());
+        cubeSegmentValidator.validate(cubeInstance, newSegments);
+    }
+
+    private void loadAllCubeInstance() throws IOException {
+        ResourceStore store = getStore();
+        List<String> paths = store.collectResourceRecursively(ResourceStore.CUBE_RESOURCE_ROOT, ".json");
+
+        logger.debug("Loading Cube from folder " + store.getReadableResourcePath(ResourceStore.CUBE_RESOURCE_ROOT));
+
+        for (String path : paths) {
+            loadCubeInstance(path);
+        }
+
+        logger.debug("Loaded " + paths.size() + " Cube(s)");
+    }
+
+    private synchronized CubeInstance loadCubeInstance(String path) throws IOException {
+        ResourceStore store = getStore();
+
+        CubeInstance cubeInstance = null;
+        try {
+            cubeInstance = store.getResource(path, CubeInstance.class, CUBE_SERIALIZER);
+            cubeInstance.setConfig(config);
+
+            if (StringUtils.isBlank(cubeInstance.getName()))
+                throw new IllegalStateException("CubeInstance name must not be blank");
+
+            cubeMap.putLocal(cubeInstance.getName().toUpperCase(), cubeInstance);
+
+            for (CubeSegment segment : cubeInstance.getSegments()) {
+                usedStorageLocation.add(segment.getName());
+            }
+
+            return cubeInstance;
+        } catch (Exception e) {
+            logger.error("Error during load cube instance " + path, e);
+            return null;
+        }
+    }
+
+    private MetadataManager getMetadataManager() {
+        return MetadataManager.getInstance(config);
+    }
+
+    private DictionaryManager getDictionaryManager() {
+        return DictionaryManager.getInstance(config);
+    }
+
+    private SnapshotManager getSnapshotManager() {
+        return SnapshotManager.getInstance(config);
+    }
+
+    private ResourceStore getStore() {
+        return ResourceStore.getStore(this.config);
+    }
+}


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

Posted by lu...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/examples/test_case_data/localmeta/data/TEST_CATEGORY_GROUPINGS.csv
----------------------------------------------------------------------
diff --git a/examples/test_case_data/localmeta/data/TEST_CATEGORY_GROUPINGS.csv b/examples/test_case_data/localmeta/data/TEST_CATEGORY_GROUPINGS.csv
new file mode 100644
index 0000000..441bf62
--- /dev/null
+++ b/examples/test_case_data/localmeta/data/TEST_CATEGORY_GROUPINGS.csv
@@ -0,0 +1,144 @@
+48027,Coins & Paper Money:Paper Money: World:Asia:Japan,0,,,,10,Coins,2008-06-03,2013-07-16 08:23:09,USER_X,USER_Y,11116,Coins & Paper Money,3411,8008,48027,48027,48027,48027,Paper Money: World,Asia,Japan,Japan,Japan,Japan,262144,N,-999,Coins,5837,3069,48027,10,1,Collectibles
+164261,Jewelry & Watches:Fashion Jewelry:Earrings:Dangle & Chandelier:Sterling Silver (w/o Stone),0,,,,17,JewelrGemWatches,2008-08-12,2012-09-11 20:26:04,USER_X,USER_Y,281,Jewelry & Watches,10968,50647,50647,50647,50647,50647,Fashion Jewelry,Earrings,Earrings,Earrings,Earrings,Earrings,262144,N,-999,JewelrGemWatches,2928,1873,50647,17,1,Fashion
+82494,BookComics & Mags:Comic Books:Modern (1980-now):Superhero:Sandman,15,,,,3,Books,2008-06-03,2010-09-13 07:05:54,USER_X,USER_Y,267,BookMagazines,63,63,63,63,63,63,NULL,Comic Books,Comic Books,Comic Books,Comic Books,Comic Books,262144,N,-999,Books,-999,-999,63,3,1,Media
+66767,Home & Lifestyle:Pet Supplies:Dog Supplies:Dog Apparel:Dresses,15,,,,16,Home & Garden,2008-06-03,2011-06-14 07:48:40,USER_X,USER_Y,11700,Home & Garden,1281,20742,1283,1283,1283,1283,NULL,Dogs,Other Supplies,Other Supplies,Other Supplies,Other Supplies,262144,N,-999,Home Improvement,-999,-999,1283,16,1,Home & Garden
+152801,Jewelry & Watches:Earrings:Studs:DiamonEnhanced Natural:PrincesSquare,0,,,,17,JewelrGemWatches,2008-06-03,2012-09-11 20:26:04,USER_X,USER_Y,281,Jewelry & Watches,10968,50647,50647,50647,50647,50647,NULL,Earrings,Earrings,Earrings,Earrings,Earrings,262144,N,-999,JewelrGemWatches,2928,1873,50647,17,1,Fashion
+43398,Home & Garden:Food & Beverages:Cheese & Crackers,0,,,,36,Home & Living,2008-06-03,2013-02-20 23:11:43,USER_X,USER_Y,11700,Home & Garden,14308,43398,43398,43398,43398,43398,NULL,Cheese & Crackers,Cheese & Crackers,Cheese & Crackers,Cheese & Crackers,Cheese & Crackers,262144,N,-999,Home Furnishing,-999,-999,43398,36,1,Home & Garden
+95173,Health & Beauty:Bath & Body:Bath SetKits:Avon,0,,,,37,Health & Beauty,2008-06-03,2008-06-03 13:36:00,USER_X,USER_Y,26395,Health & Beauty,11838,67391,67391,67391,67391,67391,Bath & Body,Bath Sets & Kits,Bath Sets & Kits,Bath Sets & Kits,Bath Sets & Kits,Bath Sets & Kits,262144,N,-999,Health & Beauty,2919,1868,67391,37,1,Fashion
+158666,ToyHobbies:Action Figures:Anime & Manga,15,,,,28,Toys,2012-09-11,2012-09-11 20:27:22,USER_X,USER_Y,220,ToyHobbies,246,158666,158666,158666,158666,158666,Action Figures,Anime & Manga,Anime & Manga,Anime & Manga,Anime & Manga,Anime & Manga,262144,N,-999,Toys,-999,-999,158666,28,1,Collectibles
+12688,eBay Premier:Books & Manuscripts:Books: Other:Icollector,0,,2038,,57,Sothebys,2008-06-03,2008-06-03 13:36:00,USER_X,USER_Y,2038,eBay Premier,1680,1681,12688,12688,12688,12688,Books & Manuscripts,Books: Other,Icollector,Icollector,Icollector,Icollector,60,N,,Sothebys,-999,-999,0,57,1,Lifestyle
+103324,Clothing & Accessories:Women:Shoes:Casual SneakerSkate:Sizes 10+,15,,,,9,Clothing & Accessories,2008-06-03,2012-05-01 08:54:26,USER_X,USER_Y,11450,ClothinShoeAccessories,3034,63889,63889,63889,63889,63889,Women's Shoes,Mixed Items,Mixed Items,Mixed Items,Mixed Items,Mixed Items,262144,N,-999,Clothing & Accessories,2441,1634,-99,9,1,Fashion
+108782,Vehicle Parts & Accessories:CaTruck Parts:Car Care & Cleaning:Air Fresheners,15,,,,8,Auto Parts,2012-09-11,2013-09-10 16:52:46,USER_X,USER_Y,131090,Vehicle Parts & Accessories,6030,72200,108782,108782,108782,108782,CaTruck Parts,Car Care & Cleaning,Air Fresheners,Air Fresheners,Air Fresheners,Air Fresheners,262144,N,-999,Auto - Parts,-999,-999,108782,8,1,Parts & Accessories
+80287,Computers & Networking:Software:Business & Productivity:Voice Recognition,0,,,,12,Computers,2008-06-03,2012-05-01 08:54:26,USER_X,USER_Y,58058,Computers/Tablets & Networking,18793,3768,3768,3768,3768,3768,Software,Office & Business,Office & Business,Office & Business,Office & Business,Office & Business,262144,N,-999,Computers,5893,3093,3768,12,1,Electronics
+140746,eBay Motors:Parts & Accessories:Vintage Car & Truck Parts:WheelTire& Hub Caps:Tires,100,,,,8,Auto - Parts,2008-06-03,2012-06-19 21:15:09,USER_X,USER_Y,6000,eBay Motors,6028,10073,140744,140746,140746,140746,Parts & Accessories,Vintage Car & Truck Parts,WheelTire& Hub Caps,Tires,Tires,Tires,262144,N,-999,Auto - Parts,3517,1794,140746,8,1,Parts & Accessories
+87118,Sporting Goods:Paintball:Barrels:Intimidator,0,,,,24,Sporting Goods,2008-06-03,2013-04-28 20:37:19,USER_X,USER_Y,888,Sporting Goods,159043,16045,16046,16046,16046,16046,Outdoor Sports,Paintball,Barrels,Barrels,Barrels,Barrels,0,N,-999,Sporting Goods,2919,1868,16046,24,1,Lifestyle
+25147,Sports MeCards & Fan Shop:Fan Apparel & Souvenirs:Baseball-MLB:Philadelphia Phillies:Other Items,0,,888,,25,Sports Memorabilia,2008-06-03,2010-02-08 10:51:44,USER_X,USER_Y,64482,Sports MeCards & Fan Shop,24409,24410,24410,24410,24410,24410,Fan Apparel & Souvenirs,Baseball-MLB,Baseball-MLB,Baseball-MLB,Baseball-MLB,Baseball-MLB,262144,N,-999,Sports Memorabilia,2884,1845,24410,25,1,Collectibles
+170302,Crafts:Embroidery:Design CDs:Other Design Themes,15,,,,15,Crafts,2009-09-21,2010-09-13 07:13:34,USER_X,USER_Y,14339,Crafts,28141,41383,41383,41383,41383,41383,Embroidery,Design CDs,Design CDs,Design CDs,Design CDs,Design CDs,262144,N,-999,Hobbies & Crafts,-999,-999,41383,15,1,Home & Garden
+53064,Business & Industrial:Agriculture & Forestry:Antique Tractors & Equipment:Tractors:Massey HarriFerguson,0,,,,4,Business (Office & Industrial),2008-06-03,2013-04-28 20:37:19,USER_X,USER_Y,12576,Business & Industrial,177641,177642,177642,177642,177642,177642,Heavy Equipment,Antique & Vintage Farm Equip,Antique & Vintage Farm Equip,Antique & Vintage Farm Equip,Antique & Vintage Farm Equip,Antique & Vintage Farm Equip,262144,N,-999,Business (Office & Industrial),-999,-999,177642,4,1,Business & Industrial
+132939,Jewelry & Watches:Designer Brands:John Atencio,0,,,,17,JewelrGemWatches,2008-06-03,2012-09-11 20:26:04,USER_X,USER_Y,281,Jewelry & Watches,10968,499,499,499,499,499,Fashion Jewelry,Other,Other,Other,Other,Other,262144,N,-999,JewelrGemWatches,2928,1873,499,17,1,Fashion
+113593,Mobiles & Phones:Mobile Phones:Samsung:Samsung D500,15,,,,40,Cell Phones & Portable Electro,2008-06-03,2012-09-11 20:26:04,USER_X,USER_Y,15032,Phones,9355,9355,9355,9355,9355,9355,Mobile Phones,Mobile Phones,Mobile Phones,Mobile Phones,Mobile Phones,Mobile Phones,262144,N,-999,Cell Phones & Accessories,2246,1609,9355,40,1,Electronics
+34273,eBay Motors:Parts & Accessories:Motorcycle:Frame & Suspension:Frames,100,,,,8,Auto - Parts,2008-06-03,2008-06-03 13:36:00,USER_X,USER_Y,6000,eBay Motors,6028,10063,34272,34273,34273,34273,Parts & Accessories,Motorcycle,Frame & Suspension,Frames,Frames,Frames,0,N,-999,Auto - Parts,-999,-999,0,8,1,Parts & Accessories
+106340,Home & Lifestyle:Tools & Building Materials:Hand Tools:Socket Sets,15,,,,16,Home & Garden,2008-06-03,2012-09-11 20:26:04,USER_X,USER_Y,11700,Home & Garden,2032,29515,29515,29515,29515,29515,Gardening,Hand Tools,Hand Tools,Hand Tools,Hand Tools,Hand Tools,262144,N,-999,Home Improvement,-999,-999,29515,16,1,Home & Garden
+150265,Baby & Maternity:Baby Clothing & Shoes:Boys:Size 00:Outfits & Sets:Bonds,15,,,,2,Baby,2008-06-03,2011-04-05 08:25:25,USER_X,USER_Y,2984,Baby,3082,147317,147333,147333,147333,147333,Baby Clothing,Boys,OutfitSets,OutfitSets,OutfitSets,OutfitSets,0,N,-999,Baby,3090,1947,147333,2,1,Home & Garden
+24760,Sports MeCards & Fan Shop:Fan Apparel & Souvenirs:Hockey-NHL:San Jose Sharks:CapHats,0,,888,,25,Sports Memorabilia,2008-06-03,2010-02-08 10:51:44,USER_X,USER_Y,64482,Sports MeCards & Fan Shop,24409,24510,24510,24510,24510,24510,Fan Apparel & Souvenirs,Hockey-NHL,Hockey-NHL,Hockey-NHL,Hockey-NHL,Hockey-NHL,262144,N,-999,Sports Memorabilia,2884,1845,24510,25,1,Collectibles
+37831,Collectibles:Advertising:Merchandise & Memorabilia:Advertising-Print:1910-19,0,,,,11,Collectibles,2008-06-03,2012-09-11 20:26:04,USER_X,USER_Y,1,Collectibles,34,13623,37828,37831,37831,37831,Advertising,Merchandise & Memorabilia,Advertising-Print,1910-19,1910-19,1910-19,262144,N,-999,Collectibles,-999,-999,37831,11,1,Collectibles
+1120,Books:First Editions:Other,3,,267,Other,3,BookComics & Magazines,2008-06-03,2008-06-03 13:36:00,USER_X,USER_Y,267,Books,2231,1120,1120,1120,1120,1120,First Editions,Other,Other,Other,Other,Other,0,N,-999,Books,,,0,3,1,Media
+43972,eBay Motors:Parts & Accessories:ATV Parts:Other,100,,,,8,Auto - Parts,2008-06-03,2012-06-19 21:15:09,USER_X,USER_Y,6000,eBay Motors,6028,43962,43972,43972,43972,43972,Parts & Accessories,ATV Parts,Other,Other,Other,Other,262144,N,-999,Auto - Parts,5411,2941,43972,8,1,Parts & Accessories
+166013,Computers:Apple AccessorieParts:TV Tuners,15,,,,12,Computers,2008-09-09,2012-05-01 08:54:26,USER_X,USER_Y,58058,Computers,175673,3761,3761,3761,3761,3761,Computer Components & Parts,Video Capture & TV Tuner Cards,Video Capture & TV Tuner Cards,Video Capture & TV Tuner Cards,Video Capture & TV Tuner Cards,Video Capture & TV Tuner Cards,262144,N,-999,Computers,-999,-999,3761,12,1,Electronics
+15568,Baby:Baby Clothing:Unisex:Sleepwear,15,,,,2,Baby,2008-06-03,2011-04-05 08:25:25,USER_X,USER_Y,2984,Baby,3082,163397,163400,163400,163400,163400,Baby Clothing,Unisex,Sleepwear,Sleepwear,Sleepwear,Sleepwear,0,N,-999,Baby,3090,1947,163400,2,1,Home & Garden
+103178,Clothing & Accessories:Women:HandbagBags:Evening Bags,15,,,,9,Clothing & Accessories,2008-06-03,2012-05-01 08:54:26,USER_X,USER_Y,11450,ClothinShoeAccessories,169291,169291,169291,169291,169291,169291,Women's Bags,Women's Bags,Women's Bags,Women's Bags,Women's Bags,Women's Bags,262144,N,-999,Clothing & Accessories,2487,1098,-99,9,1,Fashion
+2023,Sporting Goods:Team Sports:Basketball:Other,0,,888,Sporting Goods,24,Sporting Goods,2008-06-03,2013-04-28 20:37:19,USER_X,USER_Y,888,Sporting Goods,159049,21194,2023,2023,2023,2023,Team Sports,Basketball,Other,Other,Other,Other,262144,N,-999,Sporting Goods,-999,-999,2023,24,1,Lifestyle
+94847,Consumer Electronics:Car Electronics:Car Video:Car Video Monitors Only:Headrest Monitors:6-6.9 inches,0,,,,13,Consumer Electronics - Other,2008-06-03,2012-05-01 08:54:26,USER_X,USER_Y,293,Consumer Electronics,3270,48604,48606,48606,48606,48606,Vehicle Electronics & GPS,Car Video,Car Monitors w/o Player,Car Monitors w/o Player,Car Monitors w/o Player,Car Monitors w/o Player,262144,N,-999,Consumer Electronics - Other,2920,1869,48606,13,1,Electronics
+15868,Real Estate:Land:Central U.S.:Illinois,0,,10542,,23,Real Estate,2008-06-03,2008-06-03 13:36:00,USER_X,USER_Y,10542,Real Estate,15841,15841,15841,15841,15841,15841,Land,Land,Land,Land,Land,Land,0,N,-999,Real Estate,-999,-999,15841,23,1,Real Estate
+32876,Home & Garden:Home Improvement:Plumbing & Fixtures:Toilet Paper Holders-Mounted,0,,,,36,Home & Living,2008-06-03,2012-09-11 20:26:04,USER_X,USER_Y,11700,Home & Garden,159907,20601,32876,32876,32876,32876,Home Improvement,Plumbing & Fixtures,Toilet Paper Holders-Mounted,Toilet Paper Holders-Mounted,Toilet Paper Holders-Mounted,Toilet Paper Holders-Mounted,262144,N,-999,Home Furnishing,-999,-999,32876,36,1,Home & Garden
+62179,Sporting Goods:Skiing & Snowboarding:Apparel:Ski Suits:Women,0,,,,9,Apparel,2008-06-03,2008-06-03 13:36:00,USER_X,USER_Y,11450,ClothinShoes & Accessories,15724,137085,137085,137085,137085,137085,Women's Clothing,Athletic Apparel,Athletic Apparel,Athletic Apparel,Athletic Apparel,Athletic Apparel,0,N,-999,Clothing & Accessories,2919,1868,137085,9,1,Fashion
+45333,ClothinShoes & Accessories:Women's Shoes:Flats & Oxfords,0,,,,9,Apparel,2008-06-03,2012-09-11 20:26:04,USER_X,USER_Y,11450,ClothinShoes & Accessories,3034,45333,45333,45333,45333,45333,Women's Shoes,Flats & Oxfords,Flats & Oxfords,Flats & Oxfords,Flats & Oxfords,Flats & Oxfords,262144,N,-999,Clothing & Accessories,-999,-999,45333,9,1,Fashion
+50677,Jewelry & Watches:Fashion Jewelry:Pins & Brooches,0,,,,17,JewelrGemWatches,2008-06-03,2012-09-11 20:26:04,USER_X,USER_Y,281,Jewelry & Watches,10968,50677,50677,50677,50677,50677,Fashion Jewelry,Pins & Brooches,Pins & Brooches,Pins & Brooches,Pins & Brooches,Pins & Brooches,262144,N,-999,JewelrGemWatches,-999,-999,50677,17,1,Fashion
+33038,Musical Instruments:Guitars:Electric:ESP,15,,,,30,Musical Instruments,2008-06-03,2010-06-14 07:48:40,USER_X,USER_Y,619,Musical Instruments,166070,33034,33034,33034,33034,33034,Instruments,Guitars (Electric),Guitars (Electric),Guitars (Electric),Guitars (Electric),Guitars (Electric),262144,N,-999,Musical Instruments,-999,-999,33034,30,1,Lifestyle
+156614,Toys & Hobbies:Diecast Toy Vehicles:CarTrucks-Diecast:NASCAR:Revell:Dale Jarrett,0,,,,28,Toys,2008-06-03,2010-03-22 10:34:30,USER_X,USER_Y,220,Toys & Hobbies,222,171127,762,762,762,762,Diecast & Toy Vehicles,Cars: RacinNASCAR,NASCAR,NASCAR,NASCAR,NASCAR,262144,N,-999,Toys,1740,1380,762,28,1,Collectibles
+106246,Health & Beauty:Hair Care:Conditioner:Biolage,0,,,,37,Health & Beauty,2008-06-03,2013-04-28 20:37:19,USER_X,USER_Y,26395,Health & Beauty,11854,177661,177661,177661,177661,177661,Hair Care & Styling,Shampoo & Conditioning,Shampoo & Conditioning,Shampoo & Conditioning,Shampoo & Conditioning,Shampoo & Conditioning,262144,N,-999,Health & Beauty,2919,1868,177661,37,1,Fashion
+20865,Sporting Goods:Soccer:Apparel & Footwear:Jerseys:Other,0,,888,,9,Apparel,2008-06-03,2008-06-03 13:36:00,USER_X,USER_Y,11450,ClothinShoes & Accessories,1059,137084,137084,137084,137084,137084,Men's Clothing,Athletic Apparel,Athletic Apparel,Athletic Apparel,Athletic Apparel,Athletic Apparel,0,N,-999,Clothing & Accessories,2919,1868,137084,9,1,Fashion
+15115,Entertainment:Video Games:Games:Sega Dreamcast:Sports,0,,293,,32,Video Games,2008-06-03,2012-06-19 21:15:09,USER_X,USER_Y,1249,Video Games & Consoles,139973,139973,139973,139973,139973,139973,Video Games,Video Games,Video Games,Video Games,Video Games,Video Games,0,N,-999,Video Games,-999,-999,139973,32,1,Electronics
+3838,JewelrGemstones:Jewelry:Charms:Plated,0,,281,Other,17,JewelrGemWatches,2008-06-03,2012-09-11 20:26:04,USER_X,USER_Y,281,Jewelry & Watches,10968,140944,140944,140944,140944,140944,Fashion Jewelry,Charms & Charm Bracelets,Charms & Charm Bracelets,Charms & Charm Bracelets,Charms & Charm Bracelets,Charms & Charm Bracelets,0,N,-999,JewelrGemWatches,-999,-999,140944,17,1,Fashion
+759,Toys & Hobbies:Diecast Toy Vehicles:CarTrucks-Diecast:Franklin Mint:Modern (1970-Now),0,,220,Other,28,Toys,2008-06-03,2010-03-22 10:34:30,USER_X,USER_Y,220,Toys & Hobbies,222,19036,223,223,223,223,Diecast & Toy Vehicles,CarTrucks & Vans,Diecast-Modern Manufacture,Diecast-Modern Manufacture,Diecast-Modern Manufacture,Diecast-Modern Manufacture,262144,N,-999,Toys,1740,1380,223,28,1,Collectibles
+61323,Consumer Electronics:A/V Accessories & Cables:Remote Controls:Universal Remotes,0,,,,13,Consumer Electronics - Other,2008-06-03,2012-05-01 08:54:26,USER_X,USER_Y,293,Consumer Electronics,32852,14961,61312,61312,61312,61312,TVideo & Home Audio,TVideo & Audio Accessories,Remote Controls,Remote Controls,Remote Controls,Remote Controls,262144,N,-999,Consumer Electronics - Other,5344,2248,61312,13,1,Electronics
+121153,Baby:Nursery Decor:Night Lights,0,,,,2,Baby,2008-06-03,2012-09-11 20:26:04,USER_X,USER_Y,2984,Baby,66697,121153,121153,121153,121153,121153,Nursery Decor,Night Lights,Night Lights,Night Lights,Night Lights,Night Lights,262144,N,-999,Baby,-999,-999,121153,2,1,Home & Garden
+88750,Consumer Electronics:Car Electronics:RadaLaser Detectors:Whistler:17174 Series,0,,,,13,Consumer Electronics - Other,2008-06-03,2012-05-01 08:54:26,USER_X,USER_Y,293,Consumer Electronics,3270,14935,14935,14935,14935,14935,Vehicle Electronics & GPS,Radar & Laser Detectors,Radar & Laser Detectors,Radar & Laser Detectors,Radar & Laser Detectors,Radar & Laser Detectors,262144,N,-999,Consumer Electronics - Other,2920,1869,14935,13,1,Electronics
+161567,Computers:Laptop Accessories:Batteries:Hewlett Packard,15,,,,12,Computers,2008-06-03,2012-05-01 08:54:26,USER_X,USER_Y,58058,Computers,31530,14295,14295,14295,14295,14295,Laptop & Desktop Accessories,Laptop Batteries,Laptop Batteries,Laptop Batteries,Laptop Batteries,Laptop Batteries,262144,N,-999,Computers,3699,2194,14295,12,1,Electronics
+113802,Lots More...:Metaphysical:Herbs,15,,,,99,Everything Else,2008-06-03,2008-06-03 13:36:00,USER_X,USER_Y,99,Lots More...,19266,113802,113802,113802,113802,113802,Metaphysical,Herbs,Herbs,Herbs,Herbs,Herbs,262144,N,-999,Everything Else,-999,-999,113802,99,1,Lifestyle
+15808,Clothing & Accessories:Women:Plus Sizes:Tops:Other Tops,15,,,,9,Clothing & Accessories,2008-06-03,2012-09-11 20:26:04,USER_X,USER_Y,11450,ClothinShoeAccessories,15724,53159,53159,53159,53159,53159,Women's Clothing,Tops & Blouses,Tops & Blouses,Tops & Blouses,Tops & Blouses,Tops & Blouses,262144,N,-999,Clothing & Accessories,2264,1551,53159,9,1,Fashion
+174053,Vehicle Parts & Accessories:Car Parts:External & Body Parts:SunrooConvertible & Hardtop,3,,,,8,Auto - Parts,2011-08-23,2013-04-28 20:37:19,USER_X,USER_Y,131090,Vehicle Parts & Accessories,9884,14768,174053,174053,174053,174053,Car Parts,External & Body Parts,SunrooConvertible & Hardtop,SunrooConvertible & Hardtop,SunrooConvertible & Hardtop,SunrooConvertible & Hardtop,262144,N,-999,Auto - Parts,4075,2390,174053,8,1,Parts & Accessories
+2635,Toys & Hobbies:Toy Soldiers:1970-Now:Plastic,0,,220,Other,28,Toys,2008-06-03,2010-03-22 10:34:30,USER_X,USER_Y,220,Toys & Hobbies,2631,2638,2638,2638,2638,2638,Toy Soldiers,1970-Now,1970-Now,1970-Now,1970-Now,1970-Now,262144,N,-999,Toys,2919,1868,2638,28,1,Collectibles
+1161,DVFilm & TV:Other Formats:Videos: NTSC  (US):ActioAdventure,3,,267,Other,18,DVDFilm & TV,2008-06-03,2008-06-03 13:36:00,USER_X,USER_Y,11232,DVFilm & TV,380,1151,1161,1161,1161,1161,Other Formats,Videos: NTSC  (US),ActioAdventure,ActioAdventure,ActioAdventure,ActioAdventure,0,N,-999,DVDs & Movies,-999,-999,0,18,1,Media
+64076,Computers & Networking:Networking:Switches:3Com:SuperStack,0,,,,12,Computers,2008-06-03,2012-05-01 08:54:26,USER_X,USER_Y,58058,Computers/Tablets & Networking,175698,175706,51268,51268,51268,51268,Enterprise NetworkinServers,Switches & Hubs,Network Switches,Network Switches,Network Switches,Network Switches,262144,N,-999,Computers,2919,1868,51268,12,1,Electronics
+33977,Crafts:Scrapbooking:Albums:12x12 Albums,15,,,,15,Crafts,2008-06-03,2010-06-14 07:56:25,USER_X,USER_Y,14339,Crafts,31253,33875,33875,33875,33875,33875,Scrapbooking,Albums,Albums,Albums,Albums,Albums,262144,N,-999,Hobbies & Crafts,-999,-999,33875,15,1,Home & Garden
+31673,Sports:Memorabilia:Racing-NASCAR:Plaques,0,,,,25,Sports Memorabilia,2008-06-03,2010-02-08 10:48:39,USER_X,USER_Y,64482,Sports MeCards & Fan Shop,24409,46156,46156,46156,46156,46156,Fan Apparel & Souvenirs,Racing-NASCAR,Racing-NASCAR,Racing-NASCAR,Racing-NASCAR,Racing-NASCAR,0,N,-999,Sports Memorabilia,-999,-999,46156,25,1,Collectibles
+174106,Vehicle Parts & Accessories:Car Parts:Transmission & Drivetrain:Sensors,3,,,,8,Auto - Parts,2011-08-23,2013-04-28 20:37:19,USER_X,USER_Y,131090,Vehicle Parts & Accessories,9884,33726,174106,174106,174106,174106,Car Parts,Transmission & Drivetrain,Sensors,Sensors,Sensors,Sensors,262144,N,-999,Auto - Parts,4075,2390,174106,8,1,Parts & Accessories
+26249,Business & Industrial:Printing & Graphic Arts:Commercial Printing Presses:Sheet-fed Offset Presses,0,,9428,,4,Business (Office & Industrial),2008-06-03,2012-09-11 20:26:04,USER_X,USER_Y,12576,Business & Industrial,26238,26247,26249,26249,26249,26249,Printing & Graphic Arts,Commercial Printing Presses,Sheet-fed Offset Presses,Sheet-fed Offset Presses,Sheet-fed Offset Presses,Sheet-fed Offset Presses,262144,N,-999,Business (Office & Industrial),-999,-999,26249,4,1,Business & Industrial
+159184,Sporting Goods:Winter Sports:Snowboarding:Stomp Pads,0,,,,24,Sporting Goods,2008-06-03,2013-09-10 16:52:46,USER_X,USER_Y,888,Sporting Goods,36259,21247,159184,159184,159184,159184,Winter Sports,Snowboarding,Stomp Pads,Stomp Pads,Stomp Pads,Stomp Pads,262144,N,-999,Sporting Goods,-999,-999,159184,24,1,Lifestyle
+10058,Tickets & Travel:Tickets:Sporting Events:Wrestling,3,,11730,,27,Tickets,2008-06-03,2013-09-10 16:52:46,USER_X,USER_Y,1305,Events Tickets,1306,1306,1306,1306,1306,1306,Other Tickets,Other Tickets,Other Tickets,Other Tickets,Other Tickets,Other Tickets,262144,N,-999,Tickets,2937,1879,1306,27,1,Lifestyle
+48904,ClothinShoes & Accessories:Vintage:Women's Vintage Clothing:1977-89 (PunNew Wav80s):Sweaters,0,,,,9,Apparel,2008-06-03,2012-05-01 08:54:26,USER_X,USER_Y,11450,ClothinShoes & Accessories,175759,175762,52451,48904,48904,48904,Vintage,Women's Vintage Clothing,1977-89 (PunNew Wav80s),Sweaters,Sweaters,Sweaters,262144,N,-999,Clothing & Accessories,-999,-999,48904,9,1,Fashion
+145970,Toys & Hobbies:ModelKits:Automotive:Vintage:Plymouth,0,,,,28,Toys,2008-06-03,2008-10-08 07:18:40,USER_X,USER_Y,220,Toys & Hobbies,1188,2580,2585,2585,2585,2585,Models & Kits,Automotive,Vintage,Vintage,Vintage,Vintage,262144,N,-999,Toys,2974,1905,2585,28,1,Collectibles
+963,ClothinShoes & Accessories:Vintage:Clothing (Pre-1980):Women:ClothinShoes & Accessories:Shoes,0,,1,Other,9,Apparel,2008-06-03,2012-05-01 08:54:26,USER_X,USER_Y,11450,ClothinShoes & Accessories,175759,74976,48561,48561,48561,48561,Vintage,Women's Vintage Shoes,Other,Other,Other,Other,0,N,-999,Clothing & Accessories,-999,-999,48561,9,1,Fashion
+118687,Health & Beauty:Fragrances:Women's Fragrances:Benetton,3,,,,37,Health & Beauty,2008-06-03,2013-04-28 20:37:19,USER_X,USER_Y,26395,Health & Beauty,26396,11848,11848,11848,11848,11848,Fragrances,Women's Fragrances,Women's Fragrances,Women's Fragrances,Women's Fragrances,Women's Fragrances,262144,N,-999,Health & Beauty,6778,3189,11848,37,1,Fashion
+20886,Toys & Hobbies:Diecast Toy Vehicles:CarTrucks-Diecast:NASCAR:Action/ Motorsports Authentic:Darrell Waltrip,0,,220,,28,Toys,2008-06-03,2010-03-22 10:34:30,USER_X,USER_Y,220,Toys & Hobbies,222,171127,762,762,762,762,Diecast & Toy Vehicles,Cars: RacinNASCAR,NASCAR,NASCAR,NASCAR,NASCAR,262144,N,-999,Toys,1740,1380,762,28,1,Collectibles
+148324,Mobiles & Phones:Mobile Accessories:Covers / Faceplates:Sony Ericsson:W900i,15,,,,31,Phones,2008-06-03,2012-09-18 00:08:03,USER_X,USER_XIANZHU,15032,Phones,9394,20349,20349,20349,20349,20349,Mobile Accessories,CaseCoverSkins,CaseCoverSkins,CaseCoverSkins,CaseCoverSkins,CaseCoverSkins,262144,N,-999,Telecomm,-999,-999,20349,31,1,Electronics
+139255,JewellerGemWatches:Fine Jewellery:Earrings:Gemstone - Created:Lapis,15,,,,17,JewelrGemWatches,2008-06-03,2012-05-01 08:54:26,USER_X,USER_Y,281,Jewellery & Watches,4196,10985,10985,10985,10985,10985,Fine Jewellery,Earrings,Earrings,Earrings,Earrings,Earrings,262144,N,-999,JewelrGemWatches,-999,-999,-99,17,1,Fashion
+20213,Collectibles:Postcards:US StateCities & Towns:Maine,0,,1,,11,Collectibles,2008-06-03,2008-09-09 22:08:47,USER_X,USER_Y,1,Collectibles,914,20193,20213,20213,20213,20213,Postcards,US StateCities & Towns,Maine,Maine,Maine,Maine,262144,N,-999,Collectibles,-999,-999,20213,11,1,Collectibles
+32996,Movies:Television Memorabilia:Clippings,15,,,,42,Entertainment Memorabilia,2008-06-03,2012-09-11 20:26:04,USER_X,USER_Y,11232,Movies,1424,32996,32996,32996,32996,32996,Television Memorabilia,Clippings,Clippings,Clippings,Clippings,Clippings,262144,N,-999,Entertainment Memorabilia,-999,-999,32996,42,1,Collectibles
+99985,Collectibles:Trading Cards:Sci-FFantasy:Star Wars:Sets:Other,0,,,,11,Collectibles,2008-06-03,2008-09-09 22:08:47,USER_X,USER_Y,1,Collectibles,868,37897,100899,99980,99985,99985,Trading Cards,Sci-FFantasy,Star Wars,Sets,Other,Other,262144,N,-999,Collectibles,-999,-999,99985,11,1,Collectibles
+67703,Jewellery & Watches:Jewellery Boxes & Supplies:Jewellery Display,3,,,,17,Jewellery & Watches,2008-06-03,2012-06-19 21:15:09,USER_X,USER_Y,281,Jewellery & Watches,10321,67703,67703,67703,67703,67703,Jewellery Boxes & Supplies,Jewellery Display,Jewellery Display,Jewellery Display,Jewellery Display,Jewellery Display,262144,N,-999,JewelrGemWatches,4435,2261,67703,17,1,Fashion
+164,Computers/Tablets & Networking:Computer Components & Parts:CPUProcessors,0,,160,Computer Hardware,12,Computers,2008-06-03,2012-06-19 21:15:09,USER_X,USER_Y,58058,Computers/Tablets & Networking,175673,164,164,164,164,164,Computer Components & Parts,CPUProcessors,CPUProcessors,CPUProcessors,CPUProcessors,CPUProcessors,262144,N,-999,Computers,5730,2011,164,12,1,Electronics
+279,BookMagazines:Children's Books,15,,267,Other,3,Books,2008-06-03,2013-09-10 16:52:46,USER_X,USER_Y,267,BookMagazines,1093,1093,1093,1093,1093,1093,Children's Books,Children's Books,Children's Books,Children's Books,Children's Books,Children's Books,262144,N,-999,Books,1415,1128,1093,3,1,Media
+65,Collectibles:Comics:Platinum Age (1897-1937),0,,1,Other,11,Collectibles,2008-06-03,2013-02-20 23:11:43,USER_X,USER_Y,1,Collectibles,63,65,65,65,65,65,Comics,Platinum Age (1897-1937),Platinum Age (1897-1937),Platinum Age (1897-1937),Platinum Age (1897-1937),Platinum Age (1897-1937),262144,N,-999,Collectibles,6014,2967,65,11,1,Collectibles
+24541,Sports MeCards & Fan Shop:Fan Apparel & Souvenirs:College-NCAA,0,,,,25,Sports Memorabilia,2010-02-08,2013-07-16 08:23:09,USER_X,USER_Y,64482,Sports MeCards & Fan Shop,24409,24541,24541,24541,24541,24541,Fan Apparel & Souvenirs,College-NCAA,College-NCAA,College-NCAA,College-NCAA,College-NCAA,262144,N,-999,Sports Memorabilia,7721,2482,24541,25,1,Collectibles
+23446,Mode & Accessoires:Chaussures de femme:Sandales & Sandalettes:Taille 37-38,23,,,,9,Vtements et Accessoires,2008-06-03,2012-06-19 21:15:09,USER_X,USER_Y,11450,Mode & Accessoires,35107,23418,23446,23446,23446,23446,Chaussures de femme,Sandales & Sandalettes,Taille 37-38,Taille 37-38,Taille 37-38,Taille 37-38,262144,N,-999,Clothing & Accessories,2438,1634,23446,9,1,Fashion
+130,Collectibles:Transportation:Railroadiana & Trains:Other,0,,1,Other,11,Collectibles,2008-06-03,2012-09-11 20:26:04,USER_X,USER_Y,1,Collectibles,417,1444,130,130,130,130,Transportation,Railroadiana & Trains,Other,Other,Other,Other,262144,N,-999,Collectibles,-999,-999,130,11,1,Collectibles
+216,Sports MeCards & Fan Shop:Cards:Hockey,0,,11800,Other,25,Sports Memorabilia,2008-06-03,2013-02-20 23:11:43,USER_X,USER_Y,64482,Sports MeCards & Fan Shop,212,216,216,216,216,216,Cards,Hockey,Hockey,Hockey,Hockey,Hockey,262144,N,-999,Sports Memorabilia,3636,2152,216,25,1,Collectibles
+10866,Collectibles:Animals:Farm & Countryside:Pigs,0,,1,,11,Collectibles,2008-06-03,2012-09-11 20:26:04,USER_X,USER_Y,1,Collectibles,1335,45602,10866,10866,10866,10866,Animals,Farm & Countryside,Pigs,Pigs,Pigs,Pigs,262144,N,-999,Collectibles,-999,-999,10866,11,1,Collectibles
+57013,Business & Industrial:MRO & Industrial Supply:Pumps & Plumbing:Pumps:Pump Accessories & Parts,0,,,,4,Business (Office & Industrial),2008-06-03,2013-04-28 20:37:19,USER_X,USER_Y,12576,Business & Industrial,1266,109619,42928,57013,57013,57013,MRO & Industrial Supply,Pumps & Plumbing,Pumps,Pump Accessories & Parts,Pump Accessories & Parts,Pump Accessories & Parts,262144,N,-999,Business (Office & Industrial),-999,-999,57013,4,1,Business & Industrial
+15687,ClothinShoes & Accessories:Men's Clothing:T-Shirts,0,,11450,,9,Apparel,2008-06-03,2013-07-16 08:23:09,USER_X,USER_Y,11450,ClothinShoes & Accessories,1059,15687,15687,15687,15687,15687,Men's Clothing,T-Shirts,T-Shirts,T-Shirts,T-Shirts,T-Shirts,262144,N,2515,Clothing & Accessories,7692,1115,15687,9,1,Fashion
+80135,Computers/Tablets & Networking:DriveStorage & Blank Media:Blank Media & Accessories:CDVD & Blu-ray Discs,0,,,,12,Computers,2008-06-03,2013-07-16 08:23:09,USER_X,USER_Y,58058,Computers/Tablets & Networking,165,80133,80135,80135,80135,80135,DriveStorage & Blank Media,Blank Media & Accessories,CDVD & Blu-ray Discs,CDVD & Blu-ray Discs,CDVD & Blu-ray Discs,CDVD & Blu-ray Discs,262144,N,-999,Computers,3374,2018,80135,12,1,Electronics
+31519,Computers/Tablets & Networking:Laptop & Desktop Accessories:Laptop Cases & Bags,0,,,,12,Computers,2012-05-01,2012-06-19 21:15:09,USER_X,USER_Y,58058,Computers/Tablets & Networking,31530,31519,31519,31519,31519,31519,Laptop & Desktop Accessories,Laptop Cases & Bags,Laptop Cases & Bags,Laptop Cases & Bags,Laptop Cases & Bags,Laptop Cases & Bags,262144,N,-999,Computers,6404,3189,31519,12,1,Electronics
+175750,Home & Garden:Bedding:Blankets & Throws,0,,,,36,Home & Living,2012-05-01,2012-05-01 08:57:38,USER_X,USER_Y,11700,Home & Garden,20444,175750,175750,175750,175750,175750,Bedding,Blankets & Throws,Blankets & Throws,Blankets & Throws,Blankets & Throws,Blankets & Throws,262144,N,-999,Home Furnishing,-999,-999,175750,36,1,Home & Garden
+63864,ClotheShoes & Accessories:Women's Clothing:Skirts,3,,,,9,Clothing & Accessories,2009-06-15,2012-06-19 21:15:09,USER_X,USER_Y,11450,ClotheShoes & Accessories,15724,63864,63864,63864,63864,63864,Women's Clothing,Skirts,Skirts,Skirts,Skirts,Skirts,262144,N,-999,Clothing & Accessories,6334,1551,63864,9,1,Fashion
+13836,Collectibles:Decorative Collectibles:Spoons,0,,1,,11,Collectibles,2008-06-03,2012-09-11 20:26:04,USER_X,USER_Y,1,Collectibles,13777,13836,13836,13836,13836,13836,Decorative Collectibles,Spoons,Spoons,Spoons,Spoons,Spoons,262144,N,-999,Collectibles,-999,-999,13836,11,1,Collectibles
+63861,ClothinShoes & Accessories:Women's Clothing:Dresses,0,,,,9,Apparel,2008-06-03,2013-07-16 08:23:09,USER_X,USER_Y,11450,ClothinShoes & Accessories,15724,63861,63861,63861,63861,63861,Women's Clothing,Dresses,Dresses,Dresses,Dresses,Dresses,262144,N,-999,Clothing & Accessories,1263,1112,63861,9,1,Fashion
+150047,Crafts:Jewellery Making:Findings:Bails,3,,,,15,Hobbies & Crafts,2008-06-03,2012-06-19 21:15:09,USER_X,USER_Y,14339,Crafts,150045,150046,150047,150047,150047,150047,Jewellery Making,Findings,Bails,Bails,Bails,Bails,262144,N,-999,Hobbies & Crafts,4618,2740,150047,15,1,Home & Garden
+95672,ClotheShoes & Accessories:Women's Shoes:Trainers,3,,,,9,Clothing & Accessories,2011-04-05,2013-07-16 08:23:09,USER_X,USER_Y,11450,ClotheShoes & Accessories,3034,95672,95672,95672,95672,95672,Women's Shoes,Trainers,Trainers,Trainers,Trainers,Trainers,262144,N,-999,Clothing & Accessories,2314,1634,95672,9,1,Fashion
+57990,ClotheShoes & Accessories:Men's Clothing:Casual Shirts & Tops,3,,,,9,Clothing & Accessories,2009-06-15,2012-06-19 21:15:09,USER_X,USER_Y,11450,ClotheShoes & Accessories,1059,57990,57990,57990,57990,57990,Men's Clothing,Casual Shirts & Tops,Casual Shirts & Tops,Casual Shirts & Tops,Casual Shirts & Tops,Casual Shirts & Tops,262144,N,-999,Clothing & Accessories,6331,1623,57990,9,1,Fashion
+75708,Toys & Games:Action Figures:TMovies & Video Games,3,,,,28,Toys & Games,2012-05-01,2012-05-01 08:57:38,USER_X,USER_Y,220,Toys & Games,246,75708,75708,75708,75708,75708,Action Figures,TMovies & Video Games,TMovies & Video Games,TMovies & Video Games,TMovies & Video Games,TMovies & Video Games,262144,N,-999,Toys,-999,-999,75708,28,1,Collectibles
+4943,Toys & Hobbies:Diecast & Toy Vehicles:CarTrucks & Vans:Plastic:Vintage Manufacture (Pre-1970),0,,220,Other,28,Toys,2008-06-03,2013-04-28 20:37:19,USER_X,USER_Y,220,Toys & Hobbies,222,19036,19060,4943,4943,4943,Diecast & Toy Vehicles,CarTrucks & Vans,Plastic,Vintage Manufacture (Pre-1970),Vintage Manufacture (Pre-1970),Vintage Manufacture (Pre-1970),262144,N,-999,Toys,1740,1380,4943,28,1,Collectibles
+40059,Mobile Phones & Communication:Radio Communication Equipment:Parts & Accessories:Other Parts & Accessories,3,,,,33,Consumer Electronics - Audio,2012-05-01,2012-05-01 08:57:38,USER_X,USER_Y,15032,Mobile Phones & Communication,1500,175734,40059,40059,40059,40059,Radio Communication Equipment,Parts & Accessories,Other Parts & Accessories,Other Parts & Accessories,Other Parts & Accessories,Other Parts & Accessories,262144,N,-999,Consumer Electronics - Audio,-999,-999,40059,33,1,Electronics
+43479,Cameras & Photo:Film Photography:Other,0,,,,21,Photo,2008-06-03,2012-09-11 20:26:04,USER_X,USER_Y,625,Cameras & Photo,69323,43479,43479,43479,43479,43479,Film Photography,Other,Other,Other,Other,Other,262144,N,-999,Photo,-999,-999,43479,21,1,Electronics
+11848,Health & Beauty:Fragrances:Women,0,,,,37,Health & Beauty,2008-06-03,2012-06-19 21:15:09,USER_X,USER_Y,26395,Health & Beauty,26396,11848,11848,11848,11848,11848,Fragrances,Women,Women,Women,Women,Women,262144,N,-999,Health & Beauty,5884,3087,11848,37,1,Fashion
+165888,Jewelry & Watches:Vintage & Antique Jewelry:Costume:RetrVintage 1930s-1980s:PinBrooches,0,,,,17,JewelrGemWatches,2008-09-09,2009-01-12 07:05:17,USER_X,USER_Y,281,Jewelry & Watches,48579,500,58559,165888,165888,165888,Vintage & Antique Jewelry,Costume,RetrVintage 1930s-1980s,PinBrooches,PinBrooches,PinBrooches,262144,N,-999,JewelrGemWatches,-999,-999,165888,17,1,Fashion
+26262,Collectibles:Advertising:Food & Beverage:Distillery:Bacardi,0,,1,,11,Collectibles,2008-06-03,2012-09-11 20:26:04,USER_X,USER_Y,1,Collectibles,34,13587,820,26262,26262,26262,Advertising,Food & Beverage,Distillery,Bacardi,Bacardi,Bacardi,262144,N,-999,Collectibles,-999,-999,26262,11,1,Collectibles
+67698,Business & Industrial:Retail & Services:Jewellery Packaging & Display:Cases & Displays:Bracelet,2,,,,4,Business (Office & Industrial),2008-06-03,2012-09-11 20:26:04,USER_X,USER_Y,12576,Business & Industrial,11890,67694,67697,67698,67698,67698,Retail & Services,Jewellery Packaging & Display,Cases & Displays,Bracelet,Bracelet,Bracelet,262144,N,-999,Business (Office & Industrial),-999,-999,67698,4,1,Business & Industrial
+57990,ClothinShoes & Accessories:Men's Clothing:Casual Shirts,0,,,,9,Apparel,2008-06-03,2013-09-10 16:52:46,USER_X,USER_Y,11450,ClothinShoes & Accessories,1059,57990,57990,57990,57990,57990,Men's Clothing,Casual Shirts,Casual Shirts,Casual Shirts,Casual Shirts,Casual Shirts,262144,N,-999,Clothing & Accessories,-999,-999,57990,9,1,Fashion
+67698,Business & Industrial:Retail & Services:Jewelry Packaging & Display:Cases & Displays:Bracelet,0,,,,4,Business (Office & Industrial),2008-06-03,2012-09-11 20:26:04,USER_X,USER_Y,12576,Business & Industrial,11890,67694,67697,67698,67698,67698,Retail & Services,Jewelry Packaging & Display,Cases & Displays,Bracelet,Bracelet,Bracelet,262144,N,-999,Business (Office & Industrial),-999,-999,67698,4,1,Business & Industrial
+46575,Business & Industrial:Light Equipment & Tools:Air Tools:Air Tool Accessories,0,,,,4,Business (Office & Industrial),2008-06-03,2013-04-28 20:37:19,USER_X,USER_Y,12576,Business & Industrial,61573,46573,46575,46575,46575,46575,Light Equipment & Tools,Air Tools,Air Tool Accessories,Air Tool Accessories,Air Tool Accessories,Air Tool Accessories,262144,N,-999,Business (Office & Industrial),-999,-999,46575,4,1,Business & Industrial
+41940,Business & Industrial:Manufacturing & Metalworking:Metalworking Tooling:Toolholding:Indexable Inserts,0,,,,4,Business (Office & Industrial),2008-06-03,2012-09-11 20:26:04,USER_X,USER_Y,12576,Business & Industrial,11804,92084,45017,41940,41940,41940,Manufacturing & Metalworking,Metalworking Tooling,Toolholding,Indexable Inserts,Indexable Inserts,Indexable Inserts,262144,N,-999,Business (Office & Industrial),-999,-999,41940,4,1,Business & Industrial
+51582,ClothinShoes & Accessories:Kids' ClothinShoes & Accs:Girls' Clothing (Sizes 4 & Up):Sweaters,0,,,,9,Apparel,2012-05-01,2012-09-11 20:26:04,USER_X,USER_Y,11450,ClothinShoes & Accessories,171146,11462,51582,51582,51582,51582,Kids' ClothinShoes & Accs,Girls' Clothing (Sizes 4 & Up),Sweaters,Sweaters,Sweaters,Sweaters,262144,N,-999,Clothing & Accessories,-999,-999,51582,9,1,Fashion
+80053,Computers/Tablets & Networking:MonitorProjectors & Accs:Monitors,0,,,,12,Computers,2009-09-21,2012-06-19 21:15:09,USER_X,USER_Y,58058,Computers/Tablets & Networking,162497,80053,80053,80053,80053,80053,MonitorProjectors & Accs,Monitors,Monitors,Monitors,Monitors,Monitors,262144,N,-999,Computers,2982,1911,80053,12,1,Electronics
+31387,Jewellery & Watches:Watches:Wristwatches,3,,,,17,Jewellery & Watches,2008-06-03,2013-04-28 20:37:19,USER_X,USER_Y,281,Jewellery & Watches,14324,31387,31387,31387,31387,31387,Watches,Wristwatches,Wristwatches,Wristwatches,Wristwatches,Wristwatches,262144,N,-999,JewelrGemWatches,7489,32,31387,17,1,Fashion
+20485,Home & Garden:Inside the Home:Furniture:Dining Room:Other,0,,11700,,36,Home & Living,2008-06-03,2012-05-01 08:54:26,USER_X,USER_Y,11700,Home & Garden,3197,175752,175752,175752,175752,175752,Furniture,Other,Other,Other,Other,Other,262144,N,-999,Home Furnishing,2919,1868,175752,36,1,Home & Garden
+44079,Sporting Goods:Exercise & Fitness:GyWorkout & Yoga:Fitness Equipment:Exercise Mats,0,,,,24,Sporting Goods,2008-06-03,2013-04-28 20:37:19,USER_X,USER_Y,888,Sporting Goods,15273,158932,28064,44079,44079,44079,Exercise & Fitness,GyWorkout & Yoga,Fitness Equipment,Exercise Mats,Exercise Mats,Exercise Mats,262144,N,-999,Sporting Goods,-999,-999,44079,24,1,Lifestyle
+175750,HomFurniture & DIY:Bedding:Blankets,3,,,,16,Home,2012-05-01,2012-09-11 20:26:04,USER_X,USER_Y,11700,HomFurniture & DIY,20444,175750,175750,175750,175750,175750,Bedding,Blankets,Blankets,Blankets,Blankets,Blankets,262144,N,-999,Home Improvement,-999,-999,175750,16,1,Home & Garden
+16509,Toys & Hobbies:Model Railroads & Trains:S Scale:Other,0,,220,,28,Toys,2008-06-03,2012-09-11 20:26:04,USER_X,USER_Y,220,Toys & Hobbies,479,19143,16509,16509,16509,16509,Model Railroads & Trains,S Scale,Other,Other,Other,Other,262144,N,-999,Toys,-999,-999,16509,28,1,Collectibles
+139973,Video Games & Consoles:Games,3,,,,32,PC & Video Gaming,2011-04-05,2012-09-11 20:26:04,USER_X,USER_Y,1249,Video Games & Consoles,139973,139973,139973,139973,139973,139973,Games,Games,Games,Games,Games,Games,262144,N,-999,Video Games,1655,1135,139973,32,1,Electronics
+170083,Computers/Tablets & Networking:Computer Components & Parts:Memory (RAM),3,,,,12,Computers,2012-05-01,2012-06-19 21:15:09,USER_X,USER_Y,58058,Computers/Tablets & Networking,175673,170083,170083,170083,170083,170083,Computer Components & Parts,Memory (RAM),Memory (RAM),Memory (RAM),Memory (RAM),Memory (RAM),262144,N,-999,Computers,3369,2042,170083,12,1,Electronics
+16145,Computers/Tablets & Networking:Computer Components & Parts:Other Components & Parts,3,,160,,12,Computers,2008-06-03,2013-02-20 23:11:43,USER_X,USER_Y,58058,Computers/Tablets & Networking,175673,16145,16145,16145,16145,16145,Computer Components & Parts,Other Components & Parts,Other Components & Parts,Other Components & Parts,Other Components & Parts,Other Components & Parts,262144,N,-999,Computers,6976,3189,16145,12,1,Electronics
+158798,Toys & Hobbies:Vintage & Antique Toys:Spinning Tops,0,,,,28,Toys,2008-06-03,2008-09-09 22:08:47,USER_X,USER_Y,220,Toys & Hobbies,717,158798,158798,158798,158798,158798,Vintage & Antique Toys,Spinning Tops,Spinning Tops,Spinning Tops,Spinning Tops,Spinning Tops,262144,N,-999,Toys,-999,-999,158798,28,1,Collectibles
+20485,CasArredamento e Bricolage:Cucina:Altro per cucina,101,,,,36,Mobili per la casa,2008-06-03,2008-06-03 13:36:00,USER_X,USER_Y,11700,CasArredamento e Bricolage,20625,20485,20485,20485,20485,20485,Cucina,Altro per cucina,Altro per cucina,Altro per cucina,Altro per cucina,Altro per cucina,0,N,-999,Home Furnishing,-999,-999,20485,36,1,Home & Garden
+100847,Half Books,0,,,,3,Books,2008-06-03,2008-06-03 13:36:00,USER_X,USER_Y,100847,Half Books,100847,100847,100847,100847,100847,100847,Half Books,Half Books,Half Books,Half Books,Half Books,Half Books,0,N,-999,Books,-999,-999,100847,3,2,Media
+156356,Collectibles:Postcards:BuildingArchitecture,0,,,,11,Collectibles,2008-06-03,2008-06-03 13:36:00,USER_X,USER_Y,1,Collectibles,914,156356,156356,156356,156356,156356,Postcards,BuildingArchitecture,BuildingArchitecture,BuildingArchitecture,BuildingArchitecture,BuildingArchitecture,262144,N,-999,Collectibles,-999,-999,156356,11,1,Collectibles
+60606,Collectables:Badges/ Patches:Golly Badges:Robertsons:Modern (Post-1990),3,,,,11,Collectables,2008-06-03,2012-06-19 21:15:09,USER_X,USER_Y,1,Collectables,32615,98793,106160,60606,60606,60606,Badges/ Patches,Golly Badges,Robertsons,Modern (Post-1990),Modern (Post-1990),Modern (Post-1990),262144,N,-999,Collectibles,4390,2640,60606,11,1,Collectibles
+36250,Sporting Goods:Outdoor Sports:Hunting:Decoys:Duck,0,,,,24,Sporting Goods,2008-06-03,2013-04-28 20:37:19,USER_X,USER_Y,888,Sporting Goods,7301,36249,36249,36249,36249,36249,Hunting,Decoys,Decoys,Decoys,Decoys,Decoys,262144,N,-999,Sporting Goods,-999,-999,36249,24,1,Lifestyle
+50508,Cameras & Photo:Camera & Photo Accessories:LCD Hoods,0,,,,21,Photo,2008-06-03,2013-02-20 23:11:43,USER_X,USER_Y,625,Cameras & Photo,15200,50508,50508,50508,50508,50508,Camera & Photo Accessories,LCD Hoods,LCD Hoods,LCD Hoods,LCD Hoods,LCD Hoods,262144,N,-999,Photo,6256,3170,50508,21,1,Electronics
+9426,Mobile Phones & Communication:Home Phones & Accessories:Phone Accessories:Other Phone Accessories,3,,,,31,Mobile & Home Phones,2011-04-05,2012-05-01 08:54:26,USER_X,USER_Y,15032,Mobile Phones & Communication,3286,9422,9426,9426,9426,9426,Home Phones & Accessories,Phone Accessories,Other Phone Accessories,Other Phone Accessories,Other Phone Accessories,Other Phone Accessories,262144,N,-999,Telecomm,-999,-999,9426,31,1,Electronics
+45238,Abbigliamento e accessori:Donna: Accessori:SciarpFoulard e Scialli,101,,,,9,Vestiti ed Accessori,2012-09-11,2012-09-11 20:27:22,USER_X,USER_Y,11450,Abbigliamento e accessori,4251,45238,45238,45238,45238,45238,Donna: Accessori,SciarpFoulard e Scialli,SciarpFoulard e Scialli,SciarpFoulard e Scialli,SciarpFoulard e Scialli,SciarpFoulard e Scialli,0,N,-999,Clothing & Accessories,-999,-999,45238,9,1,Fashion
+75665,Home & Garden:YarGarden & Outdoor Living:Gardening Supplies:Garden Tools & Equipment:Composting,0,,,,16,Home Improvement,2008-06-03,2012-09-11 20:26:04,USER_X,USER_Y,11700,Home & Garden,159912,2032,29515,75665,75665,75665,YarGarden & Outdoor Living,Gardening Supplies,Garden Tools & Equipment,Composting,Composting,Composting,262144,N,-999,Home Improvement,-999,-999,75665,16,1,Home & Garden
+57784,ClothinShoes & Accessories:Baby & Toddler Clothing:Boys' Clothing (Newborn-5T):One-Pieces,0,,,,9,Apparel,2008-06-03,2012-09-11 20:26:04,USER_X,USER_Y,11450,ClothinShoes & Accessories,3082,147317,57784,57784,57784,57784,Baby & Toddler Clothing,Boys' Clothing (Newborn-5T),One-Pieces,One-Pieces,One-Pieces,One-Pieces,262144,N,-999,Clothing & Accessories,-999,-999,57784,9,1,Fashion
+6762,eBay Motors:Parts & Accessories:Manuals:Manuals:Other Makes,0,,6001,Other,-999,Unknown,2008-06-03,2008-06-03 13:36:00,USER_X,USER_Y,-999,Unknown,-999,-999,-999,-999,-999,-999,Unknown,Unknown,Unknown,Unknown,Unknown,Unknown,0,N,-999,Unknown,-999,-999,-999,-999,1,Unknown
+60340,Entertainment Memorabilia:Movie Memorabilia:Pressbooks:1970-79,0,,,,42,Entertainment Memorabilia,2008-06-03,2008-06-03 13:36:00,USER_X,USER_Y,45100,Entertainment Memorabilia,196,18830,60340,60340,60340,60340,Movie Memorabilia,Pressbooks,1970-79,1970-79,1970-79,1970-79,262144,N,-999,Entertainment Memorabilia,-999,-999,60340,42,1,Collectibles
+35570,eBay Motors:Parts & Accessories:Motorcycle Parts:DecalEmblems,100,,,,8,Auto - Parts,2008-06-03,2012-06-19 21:15:09,USER_X,USER_Y,6000,eBay Motors,6028,10063,35570,35570,35570,35570,Parts & Accessories,Motorcycle Parts,DecalEmblems,DecalEmblems,DecalEmblems,DecalEmblems,262144,N,-999,Auto - Parts,5407,2940,35570,8,1,Parts & Accessories
+11554,ClothinShoes & Accessories:Women's Clothing:Jeans,0,,11450,,9,Apparel,2008-06-03,2013-07-16 08:23:09,USER_X,USER_Y,11450,ClothinShoes & Accessories,15724,11554,11554,11554,11554,11554,Women's Clothing,Jeans,Jeans,Jeans,Jeans,Jeans,262144,N,-999,Clothing & Accessories,7720,1113,11554,9,1,Fashion
+1357,Collectibles:Decorative Collectibles:Decorative Collectible Brands:Hamilton Collector Plates,0,,1,Other,11,Collectibles,2008-06-03,2012-09-11 20:26:04,USER_X,USER_Y,1,Collectibles,13777,156286,1357,1357,1357,1357,Decorative Collectibles,Decorative Collectible Brands,Hamilton Collector Plates,Hamilton Collector Plates,Hamilton Collector Plates,Hamilton Collector Plates,262144,N,-999,Collectibles,-999,-999,1357,11,1,Collectibles
+13987,Collectibles:Paper:Booklets:1920-39,0,,1,,11,Collectibles,2008-06-03,2008-06-03 13:36:00,USER_X,USER_Y,1,Collectibles,124,13985,13987,13987,13987,13987,Paper,Booklets,1920-39,1920-39,1920-39,1920-39,262144,N,-999,Collectibles,-999,-999,13987,11,1,Collectibles
+63861,ClotheShoes & Accessories:Women's Clothing:Dresses,3,,,,9,Clothing & Accessories,2009-06-15,2013-07-16 08:23:09,USER_X,USER_Y,11450,ClotheShoes & Accessories,15724,63861,63861,63861,63861,63861,Women's Clothing,Dresses,Dresses,Dresses,Dresses,Dresses,262144,N,-999,Clothing & Accessories,-999,-999,63861,9,1,Fashion
+314,ClothinShoes & Accessories:Womens' Clothing:Other,211,,,,9,ClothinShoes & Accessories,2008-06-03,2013-02-20 23:11:43,USER_X,USER_Y,11450,ClothinShoes & Accessories,15724,314,314,314,314,314,Womens' Clothing,Other,Other,Other,Other,Other,0,N,-999,Clothing & Accessories,-999,-999,314,9,1,Fashion
+139973,Video Games & Consoles:Video Games,0,,,,32,Video Games,2008-06-03,2012-06-19 21:15:09,USER_X,USER_Y,1249,Video Games & Consoles,139973,139973,139973,139973,139973,139973,Video Games,Video Games,Video Games,Video Games,Video Games,Video Games,262144,N,-999,Video Games,1453,1135,139973,32,1,Electronics
+30059,Cameras & Photography:Lenses & Filters:Lens AdapterMounts & Tubes,3,,,,21,Photography,2008-06-03,2012-09-11 20:26:04,USER_X,USER_Y,625,Cameras & Photography,78997,30059,30059,30059,30059,30059,Lenses & Filters,Lens AdapterMounts & Tubes,Lens AdapterMounts & Tubes,Lens AdapterMounts & Tubes,Lens AdapterMounts & Tubes,Lens AdapterMounts & Tubes,262144,N,-999,Photo,4379,2630,30059,21,1,Electronics
+16145,Computers/Tablets & Networking:Computer Components & Parts:Other,0,,,,12,Computers,2008-06-03,2013-02-20 23:11:43,USER_X,USER_Y,58058,Computers/Tablets & Networking,175673,16145,16145,16145,16145,16145,Computer Components & Parts,Other,Other,Other,Other,Other,262144,N,-999,Computers,7312,3189,16145,12,1,Electronics
+95672,ClothinShoes & Accessories:Women's Shoes:Athletic,0,,,,9,Apparel,2011-04-05,2013-02-20 23:11:43,USER_X,USER_Y,11450,ClothinShoes & Accessories,3034,95672,95672,95672,95672,95672,Women's Shoes,Athletic,Athletic,Athletic,Athletic,Athletic,262144,N,-999,Clothing & Accessories,1251,1100,95672,9,1,Fashion
+1349,Collectibles:Decorative Collectibles:Decorative Collectible Brands:Avon:Bottles,0,,1,Other,11,Collectibles,2008-06-03,2012-09-11 20:26:04,USER_X,USER_Y,1,Collectibles,13777,156286,13779,1349,1349,1349,Decorative Collectibles,Decorative Collectible Brands,Avon,Bottles,Bottles,Bottles,262144,N,-999,Collectibles,-999,-999,1349,11,1,Collectibles
+1504,Business & Industrial:Electrical & Test Equipment:Test Equipment:Other,0,,9428,Other,4,Business (Office & Industrial),2008-06-03,2012-09-11 20:26:04,USER_X,USER_Y,12576,Business & Industrial,92074,4676,1504,1504,1504,1504,Electrical & Test Equipment,Test Equipment,Other,Other,Other,Other,262144,N,-999,Business (Office & Industrial),-999,-999,1504,4,1,Business & Industrial
+73506,Collectibles:Decorative Collectibles:Tea PotSets,0,,,,11,Collectibles,2008-06-03,2012-09-11 20:26:04,USER_X,USER_Y,1,Collectibles,13777,73506,73506,73506,73506,73506,Decorative Collectibles,Tea PotSets,Tea PotSets,Tea PotSets,Tea PotSets,Tea PotSets,262144,N,-999,Collectibles,-999,-999,73506,11,1,Collectibles
+38238,Home & Garden:Home Decor & Accents:Wall Decor:Other,0,,,,36,Home & Living,2008-06-03,2012-09-11 20:26:04,USER_X,USER_Y,11700,Home & Garden,10033,10034,10034,10034,10034,10034,Home Decor,Other,Other,Other,Other,Other,262144,N,-999,Home Furnishing,2919,1868,10034,36,1,Home & Garden
+533,Coins & Paper Money:Coins: World:Africa:Other,0,,11116,Other,10,Coins,2008-06-03,2012-06-19 21:15:09,USER_X,USER_Y,11116,Coins & Paper Money,256,127144,533,533,533,533,Coins: World,Africa,Other,Other,Other,Other,262144,N,2431,Coins,6213,1620,533,10,1,Collectibles
+223,Toys & Hobbies:Diecast & Toy Vehicles:CarTrucks & Vans:Diecast-Modern Manufacture,0,,220,Other,28,Toys,2008-06-03,2013-04-28 20:37:19,USER_X,USER_Y,220,Toys & Hobbies,222,19036,223,223,223,223,Diecast & Toy Vehicles,CarTrucks & Vans,Diecast-Modern Manufacture,Diecast-Modern Manufacture,Diecast-Modern Manufacture,Diecast-Modern Manufacture,262144,N,-999,Toys,1740,1380,223,28,1,Collectibles
+31519,Computers/Tablets & Networking:Laptop & Desktop Accessories:Laptop Cases & Bags,3,,,,12,Computers,2012-05-01,2012-06-19 21:15:09,USER_X,USER_Y,58058,Computers/Tablets & Networking,31530,31519,31519,31519,31519,31519,Laptop & Desktop Accessories,Laptop Cases & Bags,Laptop Cases & Bags,Laptop Cases & Bags,Laptop Cases & Bags,Laptop Cases & Bags,262144,N,-999,Computers,6592,3189,31519,12,1,Electronics
+155226,ClothinShoes & Accessories:Women's Clothing:Sweats & Hoodies,0,,,,9,Apparel,2008-06-03,2012-09-11 20:26:04,USER_X,USER_Y,11450,ClothinShoes & Accessories,15724,155226,155226,155226,155226,155226,Women's Clothing,Sweats & Hoodies,Sweats & Hoodies,Sweats & Hoodies,Sweats & Hoodies,Sweats & Hoodies,262144,N,-999,Clothing & Accessories,-999,-999,155226,9,1,Fashion
+63889,ClothinShoes & Accessories:Women's Shoes:Mixed Items & Lots,0,,,,9,Apparel,2008-06-03,2012-09-11 20:26:04,USER_X,USER_Y,11450,ClothinShoes & Accessories,3034,63889,63889,63889,63889,63889,Women's Shoes,Mixed Items & Lots,Mixed Items & Lots,Mixed Items & Lots,Mixed Items & Lots,Mixed Items & Lots,262144,N,-999,Clothing & Accessories,-999,-999,63889,9,1,Fashion
+314,ClothinShoes & Accessories:Women's Clothing:Other,0,,11450,Other,9,Apparel,2008-06-03,2012-09-11 20:26:04,USER_X,USER_Y,11450,ClothinShoes & Accessories,15724,314,314,314,314,314,Women's Clothing,Other,Other,Other,Other,Other,262144,N,-999,Clothing & Accessories,-999,-999,314,9,1,Fashion

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/examples/test_case_data/localmeta/data/TEST_KYLIN_FACT.csv
----------------------------------------------------------------------
diff --git a/examples/test_case_data/localmeta/data/TEST_KYLIN_FACT.csv b/examples/test_case_data/localmeta/data/TEST_KYLIN_FACT.csv
new file mode 100644
index 0000000..e03d72c
--- /dev/null
+++ b/examples/test_case_data/localmeta/data/TEST_KYLIN_FACT.csv
@@ -0,0 +1,402 @@
+10000000157,2013-03-31,Auction,48028,0,12,184.21,1,10000001
+10000000158,2013-11-12,Others,164262,0,5,172.03,1,10000002
+10000000161,2013-04-06,Auction,82494,15,14,66.6,1,10000003
+10000000137,2013-05-17,Auction,66767,15,12,92.98,1,10000004
+10000000147,2013-05-20,FP-GTC,152801,0,5,132.33,1,10000005
+10000000155,2013-06-16,FP-GTC,43398,0,13,7.12,1,10000006
+10000000170,2013-06-14,Auction,95173,0,14,204.28,1,10000007
+10000000173,2013-03-22,Auction,158666,15,13,35.72,1,10000008
+10000000178,2013-03-10,Auction,12688,0,12,4.13,1,10000009
+10000000163,2013-11-01,FP-GTC,103324,15,5,27.48,1,10000010
+10000000166,2013-06-16,FP-GTC,108782,15,14,9.26,1,10000011
+10000000167,2013-09-12,Auction,80287,0,12,3.18,1,10000012
+10000000086,2013-09-28,Others,140746,100,13,3.18,1,10000013
+10000000110,2013-06-15,ABIN,87118,0,14,377.94,1,10000014
+10000000113,2013-03-14,Auction,25147,0,12,146.33,1,10000015
+10000000069,2013-09-01,FP-GTC,170302,15,5,51.23,1,10000016
+10000000079,2013-05-29,FP-non GTC,53064,0,13,72.65,1,10000017
+10000000080,2013-05-31,Auction,132939,0,13,66.6,1,10000018
+10000000130,2013-03-18,Auction,113593,15,12,9.26,1,10000019
+10000000268,2013-07-19,Auction,34273,100,14,583.44,1,10000020
+10000000132,2013-06-23,FP-GTC,106340,15,14,638.72,1,10000021
+10000000115,2013-05-20,FP-GTC,150265,15,14,4.54,1,10000022
+10000000117,2013-05-17,FP-GTC,24760,0,12,319.79,1,10000023
+10000000118,2013-03-11,Auction,37831,0,12,20.35,1,10000024
+10000000223,2013-01-30,FP-non GTC,1120,3,5,223.63,1,10000025
+10000000224,2013-01-26,FP-GTC,43972,100,13,204.28,1,10000026
+10000000243,2013-03-22,Auction,166013,15,14,5.48,1,10000027
+10000000217,2013-07-23,Auction,15568,15,14,27.48,1,10000028
+10000000218,2013-07-27,FP-GTC,103178,15,5,21.72,1,10000029
+10000000221,2013-10-29,ABIN,2023,0,12,3.18,1,10000030
+10000000256,2013-10-08,FP-GTC,94847,0,11,491.32,1,10000031
+10000000257,2013-04-26,Auction,15868,0,14,448.8,1,10000032
+10000000263,2013-01-01,Auction,32876,0,13,415.73,1,10000033
+10000000245,2013-01-15,Auction,62179,0,13,377.94,1,10000034
+10000000248,2013-05-27,FP-GTC,33038,15,14,146.33,1,10000035
+10000000254,2013-11-11,FP-GTC,156614,0,5,7.12,1,10000036
+10000000192,2013-03-08,Auction,106246,0,13,42.99,1,10000037
+10000000196,2013-03-25,Auction,20865,0,13,12.85,1,10000038
+10000000203,2013-08-20,FP-GTC,15115,0,13,55.89,1,10000039
+10000000179,2013-05-17,FP-GTC,3838,0,14,73.26,1,10000040
+10000000183,2013-06-05,Auction,759,0,11,112.56,1,10000041
+10000000185,2013-10-08,FP-non GTC,61323,0,11,3.49,1,10000042
+10000000211,2013-08-14,Auction,121153,0,13,184.21,1,10000043
+10000000213,2013-08-14,Auction,88750,0,13,157.14,1,10000044
+10000000214,2013-05-17,FP-GTC,161567,15,14,72.65,1,10000045
+10000000204,2013-08-09,FP-GTC,113802,15,14,51.23,1,10000046
+10000000208,2013-06-30,FP-non GTC,15808,15,14,15.85,1,10000047
+10000000209,2013-06-03,FP-GTC,174053,3,13,7.12,1,10000048
+10000000042,2013-12-31,Auction,2635,0,14,12.04,1,10000049
+10000000044,2013-12-25,Auction,1161,3,13,73.26,1,10000050
+10000000022,2013-03-28,FP-non GTC,64076,0,5,184.21,1,10000051
+10000000023,2013-01-30,FP-GTC,33977,15,13,172.03,1,10000052
+10000000047,2013-12-05,FP-GTC,31673,0,14,122.78,1,10000053
+10000000056,2013-10-08,Auction,174106,3,14,92.98,1,10000054
+10000000062,2013-12-27,Auction,26249,0,13,12.19,1,10000055
+10000000053,2013-12-16,FP-GTC,159184,0,5,15.65,1,10000056
+10000000055,2013-10-17,FP-GTC,10058,3,11,101.79,1,10000057
+10000000020,2013-11-17,ABIN,48904,0,12,7.12,1,10000058
+10000000007,2013-09-18,FP-non GTC,145970,0,14,12.85,1,10000059
+10000000008,2013-06-30,FP-GTC,963,0,13,12.19,1,10000060
+10000000002,2013-10-12,FP-GTC,118687,3,13,92.98,1,10000061
+10000000003,2013-08-20,FP-GTC,20886,0,14,42.99,1,10000062
+10000000010,2013-08-29,Auction,148324,15,13,1.88,1,10000063
+10000000016,2013-07-17,Auction,139255,15,14,21.14,1,10000064
+10000000017,2013-07-23,FP-GTC,20213,0,5,21.14,1,10000065
+10000000012,2013-01-06,Auction,32996,15,13,132.33,1,10000066
+10000000013,2013-08-14,FP-GTC,99985,0,14,120.87,1,10000067
+10000000067,2013-08-10,Auction,67703,3,14,120.87,1,10000068
+10000000085,2013-09-28,FP-non GTC,65,0,11,9.26,1,10000069
+10000000141,2013-08-21,FP-non GTC,130,0,14,16.26,1,10000070
+10000000078,2013-03-11,FP-GTC,164,0,14,157.14,1,10000071
+10000000109,2013-05-06,FP-GTC,216,0,11,1.88,1,10000072
+10000000096,2013-05-17,FP-non GTC,223,0,5,12.04,1,10000073
+10000000095,2013-01-10,FP-non GTC,223,0,14,189.23,1,10000074
+10000000098,2013-05-05,FP-non GTC,223,0,5,73.26,1,10000075
+10000000097,2013-02-03,FP-non GTC,223,0,5,4.13,1,10000076
+10000000099,2013-11-26,FP-non GTC,223,0,5,290.72,1,10000077
+10000000100,2013-08-30,FP-non GTC,223,0,5,265.56,1,10000078
+10000000126,2013-04-26,FP-GTC,279,15,5,5.91,1,10000079
+10000000252,2013-06-30,Auction,314,0,5,319.79,1,10000080
+10000000052,2013-06-30,Auction,314,211,5,246,1,10000081
+10000000253,2013-12-16,Auction,314,211,5,20.35,1,10000082
+10000000051,2013-12-15,Auction,314,0,5,36.7,1,10000083
+10000000190,2013-08-17,Auction,533,0,13,101.79,1,10000084
+10000000251,2013-12-15,ABIN,1349,0,5,47.71,1,10000085
+10000000050,2013-02-04,ABIN,1349,0,5,3.49,1,10000086
+10000000049,2013-01-11,ABIN,1349,0,13,46.44,1,10000087
+10000000250,2013-02-04,ABIN,1349,0,13,4.54,1,10000088
+10000000131,2013-05-17,ABIN,1357,0,14,3.18,1,10000089
+10000000172,2013-11-12,FP-GTC,1504,0,14,86.58,1,10000090
+10000000142,2013-08-21,FP-GTC,4943,0,13,12.85,1,10000091
+10000000195,2013-05-09,ABIN,6762,0,13,16.26,1,10000092
+10000000070,2013-09-19,Auction,9426,3,13,21.14,1,10000093
+10000000165,2013-02-06,FP-non GTC,10866,0,14,20.6,1,10000094
+10000000187,2013-02-02,Auction,11554,0,13,246,1,10000095
+10000000189,2013-08-23,FP-GTC,11848,0,14,109,1,10000096
+10000000139,2013-08-03,Auction,13836,0,13,39.41,1,10000097
+10000000140,2013-05-17,Auction,13836,0,14,16.26,1,10000098
+10000000102,2013-06-06,FP-GTC,13987,0,13,112.56,1,10000099
+10000000076,2013-07-02,Auction,15687,0,14,184.21,1,10000100
+10000000082,2013-10-25,Auction,15687,0,11,27.48,1,10000001
+10000000129,2013-04-20,FP-non GTC,16145,3,12,26.45,1,10000002
+10000000128,2013-03-12,FP-non GTC,16145,0,13,415.73,1,10000003
+10000000222,2013-03-28,ABIN,16509,0,5,56.36,1,10000004
+10000000021,2013-10-29,ABIN,16509,0,5,2.44,1,10000005
+10000000134,2013-05-22,FP-GTC,20485,0,14,269.76,1,10000006
+10000000135,2013-01-25,FP-GTC,20485,101,12,109,1,10000007
+10000000136,2013-06-12,FP-GTC,20485,101,12,101.79,1,10000008
+10000000241,2013-12-26,Auction,23446,23,14,246,1,10000009
+10000000041,2013-12-26,Auction,23446,23,14,189.23,1,10000010
+10000000242,2013-12-31,Auction,23446,23,14,15.65,1,10000011
+10000000040,2013-10-04,Auction,23446,23,14,28.23,1,10000012
+10000000194,2013-03-16,FP-GTC,24541,0,5,16.26,1,10000013
+10000000101,2013-05-21,FP-GTC,26262,0,5,122.78,1,10000014
+10000000077,2013-01-28,FP-GTC,30059,3,14,172.03,1,10000015
+10000000057,2013-04-26,Auction,31387,3,14,42.99,1,10000016
+10000000258,2013-10-06,Auction,31387,3,14,207.5,1,10000017
+10000000261,2013-11-06,FP-GTC,31519,0,14,5.91,1,10000018
+10000000058,2013-10-06,FP-GTC,31519,3,14,39.41,1,10000019
+10000000059,2013-12-28,FP-GTC,31519,0,14,16.26,1,10000020
+10000000060,2013-11-06,FP-GTC,31519,0,14,16.26,1,10000021
+10000000260,2013-11-06,FP-GTC,31519,0,14,78.48,1,10000022
+10000000259,2013-12-28,FP-GTC,31519,3,14,190.22,1,10000023
+10000000156,2013-06-11,FP-GTC,35570,100,12,2.44,1,10000024
+10000000119,2013-01-10,Auction,36250,0,5,7.12,1,10000025
+10000000186,2013-09-17,FP-non GTC,38238,0,14,36.7,1,10000026
+10000000038,2013-08-14,FP-GTC,40059,3,14,35.72,1,10000027
+10000000239,2013-08-09,FP-GTC,40059,3,14,3.49,1,10000028
+10000000034,2013-12-02,FP-GTC,41940,0,13,223.63,1,10000029
+10000000235,2013-02-01,FP-GTC,41940,0,13,265.56,1,10000030
+10000000127,2013-07-28,FP-non GTC,43479,0,13,62.02,1,10000031
+10000000103,2013-06-16,FP-GTC,44079,0,12,46.44,1,10000032
+10000000201,2013-08-23,Auction,45238,101,14,132.33,1,10000033
+10000000122,2013-06-15,Auction,45333,0,13,448.8,1,10000034
+10000000123,2013-06-15,FP-non GTC,45333,0,14,207.5,1,10000035
+10000000124,2013-06-01,FP-non GTC,45333,0,14,190.22,1,10000036
+10000000072,2013-08-10,FP-GTC,46575,0,14,16.71,1,10000037
+10000000043,2013-03-22,FP-non GTC,50508,0,13,4.13,1,10000038
+10000000244,2013-12-25,FP-non GTC,50508,0,13,1.88,1,10000039
+10000000121,2013-07-22,FP-GTC,50677,0,13,491.32,1,10000040
+10000000120,2013-04-13,FP-GTC,50677,0,5,2.44,1,10000041
+10000000168,2013-04-16,Auction,51582,0,14,56.36,1,10000042
+10000000073,2013-08-21,FP-GTC,57013,0,13,15.85,1,10000043
+10000000075,2013-04-22,FP-non GTC,57013,0,14,2.44,1,10000044
+10000000074,2013-08-29,FP-GTC,57013,0,14,7.12,1,10000045
+10000000093,2013-05-16,Auction,57784,0,14,35.72,1,10000046
+10000000265,2013-08-23,Auction,57990,3,11,9.26,1,10000047
+10000000266,2013-07-10,Auction,57990,3,14,3.18,1,10000048
+10000000267,2013-08-10,Auction,57990,3,14,638.72,1,10000049
+10000000065,2013-08-23,Auction,57990,3,14,141.7,1,10000050
+10000000143,2013-04-18,ABIN,57990,0,13,12.19,1,10000051
+10000000066,2013-07-10,Auction,57990,3,14,132.33,1,10000052
+10000000144,2013-06-16,ABIN,57990,3,5,5.48,1,10000053
+10000000064,2013-07-15,Auction,57990,3,11,1.88,1,10000054
+10000000061,2013-11-06,FP-GTC,60340,0,14,12.85,1,10000055
+10000000262,2013-12-27,FP-GTC,60340,0,14,62.02,1,10000056
+10000000019,2013-07-29,FP-GTC,60606,3,12,15.85,1,10000057
+10000000220,2013-11-17,FP-GTC,60606,3,12,9.26,1,10000058
+10000000018,2013-07-27,FP-GTC,60606,3,12,16.71,1,10000059
+10000000219,2013-07-29,FP-GTC,60606,3,12,20.6,1,10000060
+10000000145,2013-01-09,Auction,63861,3,5,1.88,1,10000061
+10000000200,2013-06-11,ABIN,63861,0,5,141.7,1,10000062
+10000000199,2013-01-10,ABIN,63861,0,5,1.88,1,10000063
+10000000237,2013-09-16,Others,63861,0,11,112.56,1,10000064
+10000000036,2013-01-14,Others,63861,0,11,94.45,1,10000065
+10000000125,2013-05-17,Auction,63861,0,14,78.48,1,10000066
+10000000198,2013-06-05,ABIN,63861,0,13,5.48,1,10000067
+10000000094,2013-05-24,Auction,63864,3,14,28.23,1,10000068
+10000000104,2013-05-15,Others,63889,0,13,3.49,1,10000069
+10000000107,2013-03-25,FP-GTC,67698,2,11,15.65,1,10000070
+10000000108,2013-03-09,FP-GTC,67698,0,11,5.48,1,10000071
+10000000106,2013-12-05,FP-GTC,67698,0,11,246,1,10000072
+10000000182,2013-04-18,FP-non GTC,73506,0,13,122.78,1,10000073
+10000000169,2013-11-01,FP-GTC,75665,0,14,223.63,1,10000074
+10000000146,2013-05-03,ABIN,75708,3,5,141.7,1,10000075
+10000000151,2013-04-21,FP-non GTC,80053,0,11,21.14,1,10000076
+10000000149,2013-03-12,FP-non GTC,80053,0,11,55.89,1,10000077
+10000000150,2013-05-19,FP-non GTC,80053,0,11,51.23,1,10000078
+10000000083,2013-11-23,Auction,80135,0,14,21.72,1,10000079
+10000000089,2013-10-19,Auction,95672,3,14,204.28,1,10000080
+10000000152,2013-05-18,Others,95672,0,11,21.14,1,10000081
+10000000035,2013-02-01,Others,100847,0,5,204.28,1,10000082
+10000000236,2013-01-14,Others,100847,0,5,122.78,1,10000083
+10000000090,2013-08-05,ABIN,139973,3,14,94.45,1,10000084
+10000000091,2013-05-19,ABIN,139973,0,11,86.58,1,10000085
+10000000033,2013-12-01,Auction,150047,3,14,56.36,1,10000086
+10000000234,2013-12-02,Auction,150047,3,14,290.72,1,10000087
+10000000249,2013-01-11,FP-GTC,155226,0,13,60.37,1,10000088
+10000000048,2013-05-27,FP-GTC,155226,0,13,112.56,1,10000089
+10000000181,2013-09-01,FP-GTC,156356,0,13,265.56,1,10000090
+10000000092,2013-04-11,FP-GTC,158798,0,11,35.72,1,10000091
+10000000191,2013-05-05,FP-non GTC,165888,0,13,92.98,1,10000092
+10000000229,2013-11-21,Auction,170083,3,11,28.23,1,10000093
+10000000028,2013-10-07,Auction,170083,3,11,27.48,1,10000094
+10000000031,2013-07-12,Auction,175750,3,14,9.26,1,10000095
+10000000032,2013-06-07,Auction,175750,3,14,3.18,1,10000096
+10000000177,2013-05-22,FP-GTC,175750,0,14,12.04,1,10000097
+10000000030,2013-11-28,Auction,175750,3,13,20.6,1,10000098
+10000000231,2013-07-12,Auction,175750,3,13,12.04,1,10000099
+10000000232,2013-06-07,Auction,175750,3,14,4.13,1,10000100
+10000000233,2013-12-01,Auction,175750,3,14,73.26,1,10000201
+10000000002,2012-10-12,Auction,48027,0,12,184.21,1,10000001
+10000000003,2012-08-20,Others,164261,0,5,172.03,1,10000002
+10000000007,2012-09-18,Auction,82494,15,14,66.6,1,10000003
+10000000008,2012-06-30,Auction,66767,15,12,92.98,1,10000004
+10000000010,2012-08-29,FP-GTC,152801,0,5,132.33,1,10000005
+10000000012,2012-01-06,FP-GTC,43398,0,13,7.12,1,10000006
+10000000013,2012-08-14,Auction,95173,0,14,204.28,1,10000007
+10000000016,2012-07-17,Auction,158666,15,13,35.72,1,10000008
+10000000017,2012-07-23,Auction,12688,0,12,4.13,1,10000009
+10000000018,2012-07-27,FP-GTC,103324,15,5,27.48,1,10000010
+10000000019,2012-07-29,FP-GTC,108782,15,14,9.26,1,10000011
+10000000020,2012-11-17,Auction,80287,0,12,3.18,1,10000012
+10000000021,2012-10-29,Others,140746,100,13,3.18,1,10000013
+10000000022,2012-03-28,ABIN,87118,0,14,377.94,1,10000014
+10000000023,2012-01-30,Auction,25147,0,12,146.33,1,10000015
+10000000028,2012-10-07,FP-GTC,170302,15,5,51.23,1,10000016
+10000000030,2012-11-28,FP-non GTC,53064,0,13,72.65,1,10000017
+10000000031,2012-07-12,Auction,132939,0,13,66.6,1,10000018
+10000000032,2012-06-07,Auction,113593,15,12,9.26,1,10000019
+10000000033,2012-12-01,Auction,34273,100,14,583.44,1,10000020
+10000000034,2012-12-02,FP-GTC,106340,15,14,638.72,1,10000021
+10000000035,2012-02-01,FP-GTC,150265,15,14,4.54,1,10000022
+10000000036,2012-01-14,FP-GTC,24760,0,12,319.79,1,10000023
+10000000038,2012-08-14,Auction,37831,0,12,20.35,1,10000024
+10000000040,2012-10-04,FP-non GTC,1120,3,5,223.63,1,10000025
+10000000041,2012-12-26,FP-GTC,43972,100,13,204.28,1,10000026
+10000000042,2012-12-31,Auction,166013,15,14,5.48,1,10000027
+10000000043,2012-03-22,Auction,15568,15,14,27.48,1,10000028
+10000000044,2012-12-25,FP-GTC,103178,15,5,21.72,1,10000029
+10000000047,2012-12-05,ABIN,2023,0,12,3.18,1,10000030
+10000000048,2012-05-27,FP-GTC,94847,0,11,491.32,1,10000031
+10000000049,2012-01-11,Auction,15868,0,14,448.8,1,10000032
+10000000050,2012-02-04,Auction,32876,0,13,415.73,1,10000033
+10000000051,2012-12-15,Auction,62179,0,13,377.94,1,10000034
+10000000052,2012-06-30,FP-GTC,33038,15,14,146.33,1,10000035
+10000000053,2012-12-16,FP-GTC,156614,0,5,7.12,1,10000036
+10000000055,2012-10-17,Auction,106246,0,13,42.99,1,10000037
+10000000056,2012-10-08,Auction,20865,0,13,12.85,1,10000038
+10000000057,2012-04-26,FP-GTC,15115,0,13,55.89,1,10000039
+10000000058,2012-10-06,FP-GTC,3838,0,14,73.26,1,10000040
+10000000059,2012-12-28,Auction,759,0,11,112.56,1,10000041
+10000000060,2012-11-06,FP-non GTC,61323,0,11,3.49,1,10000042
+10000000061,2012-11-06,Auction,121153,0,13,184.21,1,10000043
+10000000062,2012-12-27,Auction,88750,0,13,157.14,1,10000044
+10000000064,2012-07-15,FP-GTC,161567,15,14,72.65,1,10000045
+10000000065,2012-08-23,FP-GTC,113802,15,14,51.23,1,10000046
+10000000066,2012-07-10,FP-non GTC,15808,15,14,15.85,1,10000047
+10000000067,2012-08-10,FP-GTC,174053,3,13,7.12,1,10000048
+10000000069,2012-09-01,Auction,2635,0,14,12.04,1,10000049
+10000000070,2012-09-19,Auction,1161,3,13,73.26,1,10000050
+10000000072,2012-08-10,FP-non GTC,64076,0,5,184.21,1,10000051
+10000000073,2012-08-21,FP-GTC,33977,15,13,172.03,1,10000052
+10000000074,2012-08-29,FP-GTC,31673,0,14,122.78,1,10000053
+10000000075,2012-04-22,Auction,174106,3,14,92.98,1,10000054
+10000000076,2012-07-02,Auction,26249,0,13,12.19,1,10000055
+10000000077,2012-01-28,FP-GTC,159184,0,5,15.65,1,10000056
+10000000078,2012-03-11,FP-GTC,10058,3,11,101.79,1,10000057
+10000000079,2012-05-29,ABIN,48904,0,12,7.12,1,10000058
+10000000080,2012-05-31,FP-non GTC,145970,0,14,12.85,1,10000059
+10000000082,2012-10-25,FP-GTC,963,0,13,12.19,1,10000060
+10000000083,2012-11-23,FP-GTC,118687,3,13,92.98,1,10000061
+10000000085,2012-09-28,FP-GTC,20886,0,14,42.99,1,10000062
+10000000086,2012-09-28,Auction,148324,15,13,1.88,1,10000063
+10000000089,2012-10-19,Auction,139255,15,14,21.14,1,10000064
+10000000090,2012-08-05,FP-GTC,20213,0,5,21.14,1,10000065
+10000000091,2012-05-19,Auction,32996,15,13,132.33,1,10000066
+10000000092,2012-04-11,FP-GTC,99985,0,14,120.87,1,10000067
+10000000093,2012-05-16,Auction,67703,3,14,120.87,1,10000068
+10000000094,2012-05-24,FP-non GTC,65,0,11,9.26,1,10000069
+10000000095,2012-01-10,FP-non GTC,130,0,14,16.26,1,10000070
+10000000096,2012-05-17,FP-GTC,164,0,14,157.14,1,10000071
+10000000097,2012-02-03,FP-GTC,216,0,11,1.88,1,10000072
+10000000098,2012-05-05,FP-non GTC,223,0,5,12.04,1,10000073
+10000000099,2012-11-26,FP-non GTC,223,0,14,189.23,1,10000074
+10000000100,2012-08-30,FP-non GTC,223,0,5,73.26,1,10000075
+10000000101,2012-05-21,FP-non GTC,223,0,5,4.13,1,10000076
+10000000102,2012-06-06,FP-non GTC,223,0,5,290.72,1,10000077
+10000000103,2012-06-16,FP-non GTC,223,0,5,265.56,1,10000078
+10000000104,2012-05-15,FP-GTC,279,15,5,5.91,1,10000079
+10000000106,2012-12-05,Auction,314,0,5,319.79,1,10000080
+10000000107,2012-03-25,Auction,314,211,5,246,1,10000081
+10000000108,2012-03-09,Auction,314,211,5,20.35,1,10000082
+10000000109,2012-05-06,Auction,314,0,5,36.7,1,10000083
+10000000110,2012-06-15,Auction,533,0,13,101.79,1,10000084
+10000000113,2012-03-14,ABIN,1349,0,5,47.71,1,10000085
+10000000115,2012-05-20,ABIN,1349,0,5,3.49,1,10000086
+10000000117,2012-05-17,ABIN,1349,0,13,46.44,1,10000087
+10000000118,2012-03-11,ABIN,1349,0,13,4.54,1,10000088
+10000000119,2012-01-10,ABIN,1357,0,14,3.18,1,10000089
+10000000120,2012-04-13,FP-GTC,1504,0,14,86.58,1,10000090
+10000000121,2012-07-22,FP-GTC,4943,0,13,12.85,1,10000091
+10000000122,2012-06-15,ABIN,6762,0,13,16.26,1,10000092
+10000000123,2012-06-15,Auction,9426,3,13,21.14,1,10000093
+10000000124,2012-06-01,FP-non GTC,10866,0,14,20.6,1,10000094
+10000000125,2012-05-17,Auction,11554,0,13,246,1,10000095
+10000000126,2012-04-26,FP-GTC,11848,0,14,109,1,10000096
+10000000127,2012-07-28,Auction,13836,0,13,39.41,1,10000097
+10000000128,2012-03-12,Auction,13836,0,14,16.26,1,10000098
+10000000129,2012-04-20,FP-GTC,13987,0,13,112.56,1,10000099
+10000000130,2012-03-18,Auction,15687,0,14,184.21,1,10000100
+10000000131,2012-05-17,Auction,15687,0,11,27.48,1,10000001
+10000000132,2012-06-23,FP-non GTC,16145,3,12,26.45,1,10000002
+10000000134,2012-05-22,FP-non GTC,16145,0,13,415.73,1,10000003
+10000000135,2012-01-25,ABIN,16509,0,5,56.36,1,10000004
+10000000136,2012-06-12,ABIN,16509,0,5,2.44,1,10000005
+10000000137,2012-05-17,FP-GTC,20485,0,14,269.76,1,10000006
+10000000139,2012-08-03,FP-GTC,20485,101,12,109,1,10000007
+10000000140,2012-05-17,FP-GTC,20485,101,12,101.79,1,10000008
+10000000141,2012-08-21,Auction,23446,23,14,246,1,10000009
+10000000142,2012-08-21,Auction,23446,23,14,189.23,1,10000010
+10000000143,2012-04-18,Auction,23446,23,14,15.65,1,10000011
+10000000144,2012-06-16,Auction,23446,23,14,28.23,1,10000012
+10000000145,2012-01-09,FP-GTC,24541,0,5,16.26,1,10000013
+10000000146,2012-05-03,FP-GTC,26262,0,5,122.78,1,10000014
+10000000147,2012-05-20,FP-GTC,30059,3,14,172.03,1,10000015
+10000000149,2012-03-12,Auction,31387,3,14,42.99,1,10000016
+10000000150,2012-05-19,Auction,31387,3,14,207.5,1,10000017
+10000000151,2012-04-21,FP-GTC,31519,0,14,5.91,1,10000018
+10000000152,2012-05-18,FP-GTC,31519,3,14,39.41,1,10000019
+10000000155,2012-06-16,FP-GTC,31519,0,14,16.26,1,10000020
+10000000156,2012-06-11,FP-GTC,31519,0,14,16.26,1,10000021
+10000000157,2012-03-31,FP-GTC,31519,0,14,78.48,1,10000022
+10000000158,2012-11-12,FP-GTC,31519,3,14,190.22,1,10000023
+10000000161,2012-04-06,FP-GTC,35570,100,12,2.44,1,10000024
+10000000163,2012-11-01,Auction,36250,0,5,7.12,1,10000025
+10000000165,2012-02-06,FP-non GTC,38238,0,14,36.7,1,10000026
+10000000166,2012-06-16,FP-GTC,40059,3,14,35.72,1,10000027
+10000000167,2012-09-12,FP-GTC,40059,3,14,3.49,1,10000028
+10000000168,2012-04-16,FP-GTC,41940,0,13,223.63,1,10000029
+10000000169,2012-11-01,FP-GTC,41940,0,13,265.56,1,10000030
+10000000170,2012-06-14,FP-non GTC,43479,0,13,62.02,1,10000031
+10000000172,2012-11-12,FP-GTC,44079,0,12,46.44,1,10000032
+10000000173,2012-03-22,Auction,45238,101,14,132.33,1,10000033
+10000000177,2012-05-22,Auction,45333,0,13,448.8,1,10000034
+10000000178,2012-03-10,FP-non GTC,45333,0,14,207.5,1,10000035
+10000000179,2012-05-17,FP-non GTC,45333,0,14,190.22,1,10000036
+10000000181,2012-09-01,FP-GTC,46575,0,14,16.71,1,10000037
+10000000182,2012-04-18,FP-non GTC,50508,0,13,4.13,1,10000038
+10000000183,2012-06-05,FP-non GTC,50508,0,13,1.88,1,10000039
+10000000185,2012-10-08,FP-GTC,50677,0,13,491.32,1,10000040
+10000000186,2012-09-17,FP-GTC,50677,0,5,2.44,1,10000041
+10000000187,2012-02-02,Auction,51582,0,14,56.36,1,10000042
+10000000189,2012-08-23,FP-GTC,57013,0,13,15.85,1,10000043
+10000000190,2012-08-17,FP-non GTC,57013,0,14,2.44,1,10000044
+10000000191,2012-05-05,FP-GTC,57013,0,14,7.12,1,10000045
+10000000192,2012-03-08,Auction,57784,0,14,35.72,1,10000046
+10000000194,2012-03-16,Auction,57990,3,11,9.26,1,10000047
+10000000195,2012-05-09,Auction,57990,3,14,3.18,1,10000048
+10000000196,2012-03-25,Auction,57990,3,14,638.72,1,10000049
+10000000198,2012-06-05,Auction,57990,3,14,141.7,1,10000050
+10000000199,2012-01-10,ABIN,57990,0,13,12.19,1,10000051
+10000000200,2012-06-11,Auction,57990,3,14,132.33,1,10000052
+10000000201,2012-08-23,ABIN,57990,3,5,5.48,1,10000053
+10000000203,2012-08-20,Auction,57990,3,11,1.88,1,10000054
+10000000204,2012-08-09,FP-GTC,60340,0,14,12.85,1,10000055
+10000000208,2012-06-30,FP-GTC,60340,0,14,62.02,1,10000056
+10000000209,2012-06-03,FP-GTC,60606,3,12,15.85,1,10000057
+10000000211,2012-08-14,FP-GTC,60606,3,12,9.26,1,10000058
+10000000213,2012-08-14,FP-GTC,60606,3,12,16.71,1,10000059
+10000000214,2012-05-17,FP-GTC,60606,3,12,20.6,1,10000060
+10000000217,2012-07-23,Auction,63861,3,5,1.88,1,10000061
+10000000218,2012-07-27,ABIN,63861,0,5,141.7,1,10000062
+10000000219,2012-07-29,ABIN,63861,0,5,1.88,1,10000063
+10000000220,2012-11-17,Others,63861,0,11,112.56,1,10000064
+10000000221,2012-10-29,Others,63861,0,11,94.45,1,10000065
+10000000222,2012-03-28,Auction,63861,0,14,78.48,1,10000066
+10000000223,2012-01-30,ABIN,63861,0,13,5.48,1,10000067
+10000000224,2012-01-26,Auction,63864,3,14,28.23,1,10000068
+10000000229,2012-11-21,Others,63889,0,13,3.49,1,10000069
+10000000231,2012-07-12,FP-GTC,67698,2,11,15.65,1,10000070
+10000000232,2012-06-07,FP-GTC,67698,0,11,5.48,1,10000071
+10000000233,2012-12-01,FP-GTC,67698,0,11,246,1,10000072
+10000000234,2012-12-02,FP-non GTC,73506,0,13,122.78,1,10000073
+10000000235,2012-02-01,FP-GTC,75665,0,14,223.63,1,10000074
+10000000236,2012-01-14,ABIN,75708,3,5,141.7,1,10000075
+10000000237,2012-09-16,FP-non GTC,80053,0,11,21.14,1,10000076
+10000000239,2012-08-09,FP-non GTC,80053,0,11,55.89,1,10000077
+10000000241,2012-12-26,FP-non GTC,80053,0,11,51.23,1,10000078
+10000000242,2012-12-31,Auction,80135,0,14,21.72,1,10000079
+10000000243,2012-03-22,Auction,95672,3,14,204.28,1,10000080
+10000000244,2012-12-25,Others,95672,0,11,21.14,1,10000081
+10000000245,2012-01-15,Others,100847,0,5,204.28,1,10000082
+10000000248,2012-05-27,Others,100847,0,5,122.78,1,10000083
+10000000249,2012-01-11,ABIN,139973,3,14,94.45,1,10000084
+10000000250,2012-02-04,ABIN,139973,0,11,86.58,1,10000085
+10000000251,2012-12-15,Auction,150047,3,14,56.36,1,10000086
+10000000252,2012-06-30,Auction,150047,3,14,290.72,1,10000087
+10000000253,2012-12-16,FP-GTC,155226,0,13,60.37,1,10000088
+10000000254,2012-11-11,FP-GTC,155226,0,13,112.56,1,10000089
+10000000256,2012-10-08,FP-GTC,156356,0,13,265.56,1,10000090
+10000000257,2012-04-26,FP-GTC,158798,0,11,35.72,1,10000091
+10000000258,2012-10-06,FP-non GTC,165888,0,13,92.98,1,10000092
+10000000259,2012-12-28,Auction,170083,3,11,28.23,1,10000093
+10000000260,2012-11-06,Auction,170083,3,11,27.48,1,10000094
+10000000261,2012-11-06,Auction,175750,3,14,9.26,1,10000095
+10000000262,2012-12-27,Auction,175750,3,14,3.18,1,10000096
+10000000263,2012-01-01,FP-GTC,175750,0,14,12.04,1,10000097
+10000000265,2012-08-23,Auction,175750,3,13,20.6,1,10000098
+10000000266,2012-07-10,Auction,175750,3,13,12.04,1,10000099
+10000000267,2012-08-10,Auction,175750,3,14,4.13,1,10000100
+10000000268,2012-07-19,Auction,175750,3,14,73.26,1,10000201

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/examples/test_case_data/localmeta/data/TEST_SELLER_TYPE_DIM.csv
----------------------------------------------------------------------
diff --git a/examples/test_case_data/localmeta/data/TEST_SELLER_TYPE_DIM.csv b/examples/test_case_data/localmeta/data/TEST_SELLER_TYPE_DIM.csv
new file mode 100644
index 0000000..73abe26
--- /dev/null
+++ b/examples/test_case_data/localmeta/data/TEST_SELLER_TYPE_DIM.csv
@@ -0,0 +1,8 @@
+16,Consumer-Other,16,102,Consumer,2012-08-09,USER_P,,USER_A
+5,Occasional,5,102,Consumer,2010-05-11,USER_H,2011-03-01 11:30:52,USER_A
+15,Professional-Other,15,101,Professional,2012-08-09,USER_P,,USER_A
+-99,Not Applicable,-99,-99,Not Applicable,2010-05-11,USER_H,,USER_A
+14,Regular,14,102,Consumer,2011-03-01,USER_H,,USER_A
+12,Merchant,12,101,Professional,2011-03-01,USER_H,,USER_A
+11,Large Merchant,11,101,Professional,2011-03-01,USER_H,,USER_A
+13,Entrepreneur,13,101,Professional,2011-03-01,USER_H,,USER_A


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

Posted by lu...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/metadata/src/main/java/com/kylinolap/metadata/model/cube/TblColRef.java
----------------------------------------------------------------------
diff --git a/metadata/src/main/java/com/kylinolap/metadata/model/cube/TblColRef.java b/metadata/src/main/java/com/kylinolap/metadata/model/cube/TblColRef.java
new file mode 100644
index 0000000..a57c209
--- /dev/null
+++ b/metadata/src/main/java/com/kylinolap/metadata/model/cube/TblColRef.java
@@ -0,0 +1,141 @@
+/*
+ * 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.metadata.model.cube;
+
+import org.apache.commons.lang.StringUtils;
+
+import com.kylinolap.metadata.model.schema.ColumnDesc;
+import com.kylinolap.metadata.model.schema.DataType;
+import com.kylinolap.metadata.model.schema.TableDesc;
+
+/**
+ * Created with IntelliJ IDEA. User: lukhan Date: 9/26/13 Time: 1:30 PM To
+ * change this template use File | Settings | File Templates.
+ */
+public class TblColRef {
+
+    private static final String INNER_TABLE_NAME = "_kylin_table";
+
+    // used by projection rewrite, see OLAPProjectRel
+    public enum InnerDataTypeEnum {
+
+        LITERAL("_literal_type"), DERIVED("_derived_type");
+
+        private final String dateType;
+
+        private InnerDataTypeEnum(String name) {
+            this.dateType = name;
+        }
+
+        public String getDataType() {
+            return dateType;
+        }
+
+        public static boolean contains(String name) {
+            return LITERAL.getDataType().equals(name) || DERIVED.getDataType().equals(name);
+        }
+    }
+
+    // used by projection rewrite, see OLAPProjectRel
+    public static TblColRef newInnerColumn(String columnName, InnerDataTypeEnum dataType) {
+        ColumnDesc column = new ColumnDesc();
+        column.setName(columnName);
+        TableDesc table = new TableDesc();
+        column.setTable(table);
+        TblColRef colRef = new TblColRef(column);
+        colRef.markInnerColumn(dataType);
+        return colRef;
+    }
+
+    // ============================================================================
+
+    private ColumnDesc column;
+
+    public TblColRef(ColumnDesc column) {
+        this.column = column;
+    }
+
+    public ColumnDesc getColumn() {
+        return column;
+    }
+
+    public void setColumn(ColumnDesc column) {
+        this.column = column;
+    }
+
+    public String getName() {
+        return column.getName();
+    }
+
+    public String getTable() {
+        if (column.getTable() == null) {
+            return null;
+        }
+        return column.getTable().getName();
+    }
+
+    public String getDatatype() {
+        return column.getDatatype();
+    }
+
+    public DataType getType() {
+        return column.getType();
+    }
+
+    public void markInnerColumn(InnerDataTypeEnum dataType) {
+        this.column.setDatatype(dataType.getDataType());
+        this.column.getTable().setName(INNER_TABLE_NAME);
+    }
+
+    public boolean isInnerColumn() {
+        return InnerDataTypeEnum.contains(getDatatype());
+    }
+
+    public boolean isDerivedDataType() {
+        return InnerDataTypeEnum.DERIVED.getDataType().equals(getDatatype());
+    }
+
+    @Override
+    public int hashCode() {
+        final int prime = 31;
+        int result = 1;
+
+        result = prime * result + column.getTable().getName().hashCode();
+        result = prime * result + column.getName().hashCode();
+        return result;
+    }
+
+    @Override
+    public boolean equals(Object obj) {
+        if (this == obj)
+            return true;
+        if (obj == null)
+            return false;
+        if (getClass() != obj.getClass())
+            return false;
+        TblColRef other = (TblColRef) obj;
+        if (!StringUtils.equals(column.getTable().getName(), other.column.getTable().getName()))
+            return false;
+        if (!StringUtils.equals(column.getName(), other.column.getName()))
+            return false;
+        return true;
+    }
+
+    @Override
+    public String toString() {
+        return (column.getTable() == null ? null : column.getTable().getName()) + "." + column.getName();
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/metadata/src/main/java/com/kylinolap/metadata/model/invertedindex/InvertedIndexDesc.java
----------------------------------------------------------------------
diff --git a/metadata/src/main/java/com/kylinolap/metadata/model/invertedindex/InvertedIndexDesc.java b/metadata/src/main/java/com/kylinolap/metadata/model/invertedindex/InvertedIndexDesc.java
new file mode 100644
index 0000000..324adb7
--- /dev/null
+++ b/metadata/src/main/java/com/kylinolap/metadata/model/invertedindex/InvertedIndexDesc.java
@@ -0,0 +1,146 @@
+package com.kylinolap.metadata.model.invertedindex;
+
+import java.util.BitSet;
+
+import org.apache.commons.lang.ArrayUtils;
+import org.apache.hadoop.hbase.util.Bytes;
+
+import com.fasterxml.jackson.annotation.JsonAutoDetect;
+import com.fasterxml.jackson.annotation.JsonAutoDetect.Visibility;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.kylinolap.common.KylinConfig;
+import com.kylinolap.common.persistence.ResourceStore;
+import com.kylinolap.common.persistence.RootPersistentEntity;
+import com.kylinolap.common.util.StringUtil;
+import com.kylinolap.metadata.MetadataManager;
+import com.kylinolap.metadata.model.cube.TblColRef;
+import com.kylinolap.metadata.model.schema.ColumnDesc;
+import com.kylinolap.metadata.model.schema.TableDesc;
+
+/**
+ * @author yangli9
+ */
+@JsonAutoDetect(fieldVisibility = Visibility.NONE, getterVisibility = Visibility.NONE, isGetterVisibility = Visibility.NONE, setterVisibility = Visibility.NONE)
+public class InvertedIndexDesc extends RootPersistentEntity {
+
+    public static final String HBASE_FAMILY = "f";
+    public static final String HBASE_QUALIFIER = "c";
+    public static final byte[] HBASE_FAMILY_BYTES = Bytes.toBytes(HBASE_FAMILY);
+    public static final byte[] HBASE_QUALIFIER_BYTES = Bytes.toBytes(HBASE_QUALIFIER);
+
+    private KylinConfig config;
+
+    @JsonProperty("name")
+    private String name;
+    @JsonProperty("fact_table")
+    private String factTable;
+    @JsonProperty("timestamp_dimension")
+    private String timestampDimension;
+    @JsonProperty("bitmap_dimensions")
+    private String[] bitmapDimensions;
+    @JsonProperty("value_dimensions")
+    private String[] valueDimensions;
+    @JsonProperty("metrics")
+    private String[] metrics;
+    @JsonProperty("sharding")
+    private short sharding = 1; // parallelism
+    @JsonProperty("slice_size")
+    private int sliceSize = 50000; // no. rows
+
+    // computed
+    private TableDesc tableDesc;
+    private int tsCol;
+    private int[] bitmapCols;
+    private int[] valueCols;
+    private int[] metricsCols;
+    private BitSet metricsColSet;
+
+    public void init(MetadataManager mgr) {
+        config = mgr.getConfig();
+
+        factTable = factTable.toUpperCase();
+        timestampDimension = timestampDimension.toUpperCase();
+        StringUtil.toUpperCaseArray(bitmapDimensions, bitmapDimensions);
+        StringUtil.toUpperCaseArray(valueDimensions, valueDimensions);
+        StringUtil.toUpperCaseArray(metrics, metrics);
+
+        tableDesc = mgr.getTableDesc(factTable);
+        bitmapCols = new int[bitmapDimensions.length];
+        valueCols = new int[valueDimensions.length];
+        metricsCols = new int[metrics.length];
+        metricsColSet = new BitSet(tableDesc.getColumnCount());
+        int i = 0, j = 0, k = 0;
+        for (ColumnDesc col : tableDesc.getColumns()) {
+            if (ArrayUtils.contains(bitmapDimensions, col.getName())) {
+                bitmapCols[i++] = col.getZeroBasedIndex();
+            }
+            if (ArrayUtils.contains(valueDimensions, col.getName())) {
+                valueCols[j++] = col.getZeroBasedIndex();
+            }
+            if (ArrayUtils.contains(metrics, col.getName())) {
+                metricsCols[k++] = col.getZeroBasedIndex();
+                metricsColSet.set(col.getZeroBasedIndex());
+            }
+        }
+
+        tsCol = tableDesc.findColumnByName(timestampDimension).getZeroBasedIndex();
+    }
+
+    public KylinConfig getConfig() {
+        return config;
+    }
+
+    public String getName() {
+        return name;
+    }
+
+    public int getTimestampColumn() {
+        return tsCol;
+    }
+
+    public int[] getBitmapColumns() {
+        return bitmapCols;
+    }
+
+    public int[] getValueColumns() {
+        return valueCols;
+    }
+    
+    public int[] getMetricsColumns() {
+        return metricsCols;
+    }
+    
+    public short getSharding() {
+        return sharding;
+    }
+    
+    public int getSliceSize() {
+        return sliceSize;
+    }
+
+    public boolean isMetricsCol(TblColRef col) {
+        assert col.getTable().equals(factTable);
+        return isMetricsCol(col.getColumn().getZeroBasedIndex());
+    }
+
+    public boolean isMetricsCol(int colZeroBasedIndex) {
+        return metricsColSet.get(colZeroBasedIndex);
+    }
+
+    public String getResourcePath() {
+        return ResourceStore.IIDESC_RESOURCE_ROOT + "/" + name + ".json";
+    }
+
+    public TableDesc getFactTableDesc() {
+        return tableDesc;
+    }
+
+    public String getFactTable() {
+        return factTable;
+    }
+
+    public String getTimestampDimension() {
+        return timestampDimension;
+    }
+    
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/metadata/src/main/java/com/kylinolap/metadata/model/schema/ColumnDesc.java
----------------------------------------------------------------------
diff --git a/metadata/src/main/java/com/kylinolap/metadata/model/schema/ColumnDesc.java b/metadata/src/main/java/com/kylinolap/metadata/model/schema/ColumnDesc.java
new file mode 100644
index 0000000..1facf04
--- /dev/null
+++ b/metadata/src/main/java/com/kylinolap/metadata/model/schema/ColumnDesc.java
@@ -0,0 +1,125 @@
+/*
+ * 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.metadata.model.schema;
+
+import com.fasterxml.jackson.annotation.JsonAutoDetect;
+import com.fasterxml.jackson.annotation.JsonAutoDetect.Visibility;
+import com.fasterxml.jackson.annotation.JsonProperty;
+
+/**
+ * Column Metadata from Source. All name should be uppercase.
+ * <p/>
+ * User: lukhan Date: 10/15/13 Time: 9:07 AM To change this template use File |
+ * Settings | File Templates.
+ */
+@JsonAutoDetect(fieldVisibility = Visibility.NONE, getterVisibility = Visibility.NONE, isGetterVisibility = Visibility.NONE, setterVisibility = Visibility.NONE)
+public class ColumnDesc {
+    @JsonProperty("id")
+    private String id;
+    @JsonProperty("name")
+    private String name;
+    @JsonProperty("datatype")
+    private String datatype;
+
+    // parsed from data type
+    private DataType type;
+
+    private TableDesc table;
+    private int zeroBasedIndex = -1;
+    private boolean isNullable = true;
+
+    public ColumnDesc() { // default constructor for Jackson
+    }
+
+    public int getZeroBasedIndex() {
+        return zeroBasedIndex;
+    }
+
+    public String getDatatype() {
+        return datatype;
+    }
+
+    public void setDatatype(String datatype) {
+        this.datatype = datatype;
+        type = DataType.getInstance(datatype);
+    }
+
+    public String getId() {
+        return id;
+    }
+
+    public void setId(String id) {
+        this.id = id;
+    }
+
+    public String getName() {
+        return name;
+    }
+
+    public void setName(String name) {
+        this.name = name;
+    }
+
+    public TableDesc getTable() {
+        return table;
+    }
+
+    public void setTable(TableDesc table) {
+        this.table = table;
+    }
+
+    public DataType getType() {
+        return type;
+    }
+
+    public String getTypeName() {
+        return type.getName();
+    }
+
+    public int getTypePrecision() {
+        return type.getPrecision();
+    }
+
+    public int getTypeScale() {
+        return type.getScale();
+    }
+
+    public void setNullable(boolean nullable) {
+        this.isNullable = nullable;
+    }
+
+    public boolean isNullable() {
+        return this.isNullable;
+    }
+
+    public void init(TableDesc table) {
+        this.table = table;
+
+        if (name != null)
+            name = name.toUpperCase();
+
+        if (id != null)
+            zeroBasedIndex = Integer.parseInt(id) - 1;
+
+        type = DataType.getInstance(datatype);
+    }
+
+    @Override
+    public String toString() {
+        return "ColumnDesc [name=" + name + ",table=" + table.getName() + "]";
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/metadata/src/main/java/com/kylinolap/metadata/model/schema/DataType.java
----------------------------------------------------------------------
diff --git a/metadata/src/main/java/com/kylinolap/metadata/model/schema/DataType.java b/metadata/src/main/java/com/kylinolap/metadata/model/schema/DataType.java
new file mode 100644
index 0000000..b955fae
--- /dev/null
+++ b/metadata/src/main/java/com/kylinolap/metadata/model/schema/DataType.java
@@ -0,0 +1,288 @@
+/*
+ * 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.metadata.model.schema;
+
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+
+import com.kylinolap.metadata.model.cube.TblColRef.InnerDataTypeEnum;
+
+/**
+ * @author yangli9
+ * 
+ */
+public class DataType {
+
+    public static final String VALID_TYPES_STRING = "any|char|varchar|boolean|binary" //
+            + "|integer|tinyint|smallint|bigint|decimal|numeric|float|real|double" //
+            + "|date|time|datetime|timestamp|byte|int|short|long|string|hllc" //
+            + "|" + InnerDataTypeEnum.LITERAL.getDataType() //
+            + "|" + InnerDataTypeEnum.DERIVED.getDataType();
+
+    private static final Pattern TYPE_PATTERN = Pattern.compile(
+    // standard sql types, ref:
+    // http://www.w3schools.com/sql/sql_datatypes_general.asp
+            "(" + VALID_TYPES_STRING + ")" + "\\s*" //
+                    + "(?:" + "[(]" + "([\\d\\s,]+)" + "[)]" + ")?", Pattern.CASE_INSENSITIVE);
+
+    public static final Set<String> INTEGER_FAMILY = new HashSet<String>();
+    public static final Set<String> NUMBER_FAMILY = new HashSet<String>();
+    public static final Set<String> DATETIME_FAMILY = new HashSet<String>();
+    public static final Set<String> STRING_FAMILY = new HashSet<String>();
+    private static final Set<Integer> HLLC_PRECISIONS = new HashSet<Integer>();
+    private static final Map<String, String> LEGACY_TYPE_MAP = new HashMap<String, String>();
+    static {
+        INTEGER_FAMILY.add("tinyint");
+        INTEGER_FAMILY.add("smallint");
+        INTEGER_FAMILY.add("integer");
+        INTEGER_FAMILY.add("bigint");
+
+        NUMBER_FAMILY.addAll(INTEGER_FAMILY);
+        NUMBER_FAMILY.add("float");
+        NUMBER_FAMILY.add("double");
+        NUMBER_FAMILY.add("decimal");
+        NUMBER_FAMILY.add("real");
+        NUMBER_FAMILY.add("numeric");
+
+        DATETIME_FAMILY.add("date");
+        DATETIME_FAMILY.add("time");
+        DATETIME_FAMILY.add("datetime");
+        DATETIME_FAMILY.add("timestamp");
+
+        STRING_FAMILY.add("varchar");
+        STRING_FAMILY.add("char");
+
+        LEGACY_TYPE_MAP.put("byte", "tinyint");
+        LEGACY_TYPE_MAP.put("int", "integer");
+        LEGACY_TYPE_MAP.put("short", "smallint");
+        LEGACY_TYPE_MAP.put("long", "bigint");
+        LEGACY_TYPE_MAP.put("string", "varchar");
+        LEGACY_TYPE_MAP.put("hllc10", "hllc(10)");
+        LEGACY_TYPE_MAP.put("hllc12", "hllc(12)");
+        LEGACY_TYPE_MAP.put("hllc14", "hllc(14)");
+        LEGACY_TYPE_MAP.put("hllc15", "hllc(15)");
+        LEGACY_TYPE_MAP.put("hllc16", "hllc(16)");
+
+        for (int i = 10; i <= 16; i++)
+            HLLC_PRECISIONS.add(i);
+    }
+
+    private static final ConcurrentMap<DataType, DataType> CACHE = new ConcurrentHashMap<DataType, DataType>();
+
+    public static DataType getInstance(String type) {
+        if (type == null)
+            return null;
+
+        DataType dataType = new DataType(type);
+        DataType cached = CACHE.get(dataType);
+        if (cached == null) {
+            CACHE.put(dataType, dataType);
+            cached = dataType;
+        }
+        return cached;
+    }
+
+    // ============================================================================
+
+    private String name;
+    private int precision;
+    private int scale;
+
+    DataType(String datatype) {
+        parseDataType(datatype);
+    }
+
+    private void parseDataType(String datatype) {
+        datatype = datatype.trim().toLowerCase();
+        datatype = replaceLegacy(datatype);
+
+        Matcher m = TYPE_PATTERN.matcher(datatype);
+        if (m.matches() == false)
+            throw new IllegalArgumentException("bad data type -- " + datatype + ", does not match " + TYPE_PATTERN);
+
+        name = replaceLegacy(m.group(1));
+        precision = -1;
+        scale = -1;
+
+        String leftover = m.group(2);
+        if (leftover != null) {
+            String[] parts = leftover.split("\\s*,\\s*");
+            for (int i = 0; i < parts.length; i++) {
+                int n;
+                try {
+                    n = Integer.parseInt(parts[i]);
+                } catch (NumberFormatException e) {
+                    throw new IllegalArgumentException("bad data type -- " + datatype + ", precision/scale not numeric");
+                }
+                if (i == 0)
+                    precision = n;
+                else if (i == 1)
+                    scale = n;
+                else
+                    throw new IllegalArgumentException("bad data type -- " + datatype + ", too many precision/scale parts");
+            }
+        }
+
+        // FIXME 256 for unknown string precision
+        if ((name.equals("char") || name.equals("varchar")) && precision == -1) {
+            precision = 256; // to save memory at frontend, e.g. tableau will
+                             // allocate memory according to this
+        }
+
+        // FIXME (19,4) for unknown decimal precision
+        if ((name.equals("decimal") || name.equals("numeric")) && precision == -1) {
+            precision = 19;
+            scale = 4;
+        }
+
+        if (isHLLC() && HLLC_PRECISIONS.contains(precision) == false)
+            throw new IllegalArgumentException("HLLC precision must be one of " + HLLC_PRECISIONS);
+    }
+
+    private String replaceLegacy(String str) {
+        String replace = LEGACY_TYPE_MAP.get(str);
+        return replace == null ? str : replace;
+    }
+
+    public int getSpaceEstimate() {
+        if (isTinyInt()) {
+            return 1;
+        } else if (isSmallInt()) {
+            return 2;
+        } else if (isInt()) {
+            return 4;
+        } else if (isBigInt()) {
+            return 8;
+        } else if (isFloat()) {
+            return 4;
+        } else if (isDouble()) {
+            return 8;
+        } else if (isDecimal()) {
+            return 8;
+        } else if (isHLLC()) {
+            return 1 << precision;
+        }
+        throw new IllegalStateException("The return type : " + name + " is not recognized;");
+    }
+
+    public boolean isStringFamily() {
+        return STRING_FAMILY.contains(name);
+    }
+
+    public boolean isIntegerFamily() {
+        return INTEGER_FAMILY.contains(name);
+    }
+
+    public boolean isNumberFamily() {
+        return NUMBER_FAMILY.contains(name);
+    }
+
+    public boolean isDateTimeFamily() {
+        return DATETIME_FAMILY.contains(name);
+    }
+
+    public boolean isTinyInt() {
+        return name.equals("tinyint");
+    }
+
+    public boolean isSmallInt() {
+        return name.equals("smallint");
+    }
+
+    public boolean isInt() {
+        return name.equals("integer");
+    }
+
+    public boolean isBigInt() {
+        return name.equals("bigint");
+    }
+
+    public boolean isFloat() {
+        return name.equals("float");
+    }
+
+    public boolean isDouble() {
+        return name.equals("double");
+    }
+
+    public boolean isDecimal() {
+        return name.equals("decimal");
+    }
+
+    public boolean isHLLC() {
+        return name.equals("hllc");
+    }
+
+    public String getName() {
+        return name;
+    }
+
+    public int getPrecision() {
+        return precision;
+    }
+
+    public int getScale() {
+        return scale;
+    }
+
+    @Override
+    public int hashCode() {
+        final int prime = 31;
+        int result = 1;
+        result = prime * result + ((name == null) ? 0 : name.hashCode());
+        result = prime * result + precision;
+        result = prime * result + scale;
+        return result;
+    }
+
+    @Override
+    public boolean equals(Object obj) {
+        if (this == obj)
+            return true;
+        if (obj == null)
+            return false;
+        if (getClass() != obj.getClass())
+            return false;
+        DataType other = (DataType) obj;
+        if (name == null) {
+            if (other.name != null)
+                return false;
+        } else if (!name.equals(other.name))
+            return false;
+        if (precision != other.precision)
+            return false;
+        if (scale != other.scale)
+            return false;
+        return true;
+    }
+
+    @Override
+    public String toString() {
+        if (precision < 0 && scale < 0)
+            return name;
+        else if (scale < 0)
+            return name + "(" + precision + ")";
+        else
+            return name + "(" + precision + "," + scale + ")";
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/metadata/src/main/java/com/kylinolap/metadata/model/schema/DatabaseDesc.java
----------------------------------------------------------------------
diff --git a/metadata/src/main/java/com/kylinolap/metadata/model/schema/DatabaseDesc.java b/metadata/src/main/java/com/kylinolap/metadata/model/schema/DatabaseDesc.java
new file mode 100644
index 0000000..8183264
--- /dev/null
+++ b/metadata/src/main/java/com/kylinolap/metadata/model/schema/DatabaseDesc.java
@@ -0,0 +1,75 @@
+/*
+ * 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.metadata.model.schema;
+
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+
+/**
+ * @author xjiang
+ */
+public class DatabaseDesc {
+    private String name;
+
+    /**
+     * @return the name
+     */
+    public String getName() {
+        return name.toUpperCase();
+    }
+
+    /**
+     * @param name
+     *            the name to set
+     */
+    public void setName(String name) {
+        this.name = name;
+    }
+
+    /*
+     * (non-Javadoc)
+     * 
+     * @see java.lang.Object#toString()
+     */
+    @Override
+    public String toString() {
+
+        return "DatabaseDesc [name=" + name + "]";
+    }
+
+    public static HashMap<String, Integer> extractDatabaseOccurenceCounts(List<TableDesc> tables) {
+        HashMap<String, Integer> databaseCounts = new HashMap<String, Integer>();
+        for (TableDesc tableDesc : tables) {
+            String databaseName = tableDesc.getDatabase();
+            Integer counter = databaseCounts.get(databaseName);
+            if (counter != null)
+                databaseCounts.put(databaseName, counter + 1);
+            else
+                databaseCounts.put(databaseName, 1);
+        }
+        return databaseCounts;
+    }
+
+    public static HashSet<String> extractDatabaseNames(List<TableDesc> tables) {
+        HashSet<String> databaseNames = new HashSet<String>();
+        for (TableDesc tableDesc : tables) {
+            databaseNames.add(tableDesc.getDatabase());
+        }
+        return databaseNames;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/metadata/src/main/java/com/kylinolap/metadata/model/schema/TableDesc.java
----------------------------------------------------------------------
diff --git a/metadata/src/main/java/com/kylinolap/metadata/model/schema/TableDesc.java b/metadata/src/main/java/com/kylinolap/metadata/model/schema/TableDesc.java
new file mode 100644
index 0000000..e10fe19
--- /dev/null
+++ b/metadata/src/main/java/com/kylinolap/metadata/model/schema/TableDesc.java
@@ -0,0 +1,129 @@
+/*
+ * 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.metadata.model.schema;
+
+import java.util.Arrays;
+import java.util.Comparator;
+
+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;
+
+/**
+ * Table Metadata from Source. All name should be uppercase.
+ * <p/>
+ * User: lukhan Date: 10/15/13 Time: 9:06 AM To change this template use File |
+ * Settings | File Templates.
+ */
+@JsonAutoDetect(fieldVisibility = Visibility.NONE, getterVisibility = Visibility.NONE, isGetterVisibility = Visibility.NONE, setterVisibility = Visibility.NONE)
+public class TableDesc extends RootPersistentEntity {
+    @JsonProperty("name")
+    private String name;
+    @JsonProperty("columns")
+    private ColumnDesc[] columns;
+
+    private DatabaseDesc database;
+
+    public ColumnDesc findColumnByName(String name) {
+        for (ColumnDesc c : columns) {
+            // return first matched column
+            if (name.equalsIgnoreCase(c.getName())) {
+                return c;
+            }
+        }
+        return null;
+    }
+
+    public String getResourcePath() {
+        return ResourceStore.TABLE_RESOURCE_ROOT + "/" + name + ".json";
+    }
+
+    // ============================================================================
+
+    public String getName() {
+        return this.name;
+    }
+
+    public void setName(String name) {
+        this.name = name;
+    }
+
+    @JsonProperty("database")
+    public String getDatabase() {
+        if (database == null) {
+            return "DEFAULT";
+        }
+        return database.getName();
+    }
+
+    @JsonProperty("database")
+    public void setDatabase(String database) {
+        this.database = new DatabaseDesc();
+        this.database.setName(database);
+    }
+
+    public ColumnDesc[] getColumns() {
+        return columns;
+    }
+
+    public void setColumns(ColumnDesc[] columns) {
+        this.columns = columns;
+    }
+
+    public int getMaxColumnIndex() {
+        int max = -1;
+        for (ColumnDesc col : columns) {
+            int idx = col.getZeroBasedIndex();
+            max = Math.max(max, idx);
+        }
+        return max;
+    }
+
+    public int getColumnCount() {
+        return getMaxColumnIndex() + 1;
+    }
+
+    public void init() {
+        if (name != null)
+            name = name.toUpperCase();
+
+        if (getDatabase() != null)
+            setDatabase(getDatabase().toUpperCase());
+
+        if (columns != null) {
+            Arrays.sort(columns, new Comparator<ColumnDesc>() {
+                @Override
+                public int compare(ColumnDesc col1, ColumnDesc col2) {
+                    Integer id1 = Integer.parseInt(col1.getId());
+                    Integer id2 = Integer.parseInt(col2.getId());
+                    return id1.compareTo(id2);
+                }
+            });
+
+            for (ColumnDesc col : columns) {
+                col.init(this);
+            }
+        }
+    }
+
+    @Override
+    public String toString() {
+        return "TableDesc [name=" + name + "]";
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/metadata/src/main/java/com/kylinolap/metadata/tool/HiveSourceTableLoader.java
----------------------------------------------------------------------
diff --git a/metadata/src/main/java/com/kylinolap/metadata/tool/HiveSourceTableLoader.java b/metadata/src/main/java/com/kylinolap/metadata/tool/HiveSourceTableLoader.java
new file mode 100644
index 0000000..3e4cca3
--- /dev/null
+++ b/metadata/src/main/java/com/kylinolap/metadata/tool/HiveSourceTableLoader.java
@@ -0,0 +1,251 @@
+package com.kylinolap.metadata.tool;
+
+/*
+ * 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.
+ */
+
+import java.io.BufferedReader;
+import java.io.File;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.io.StringReader;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
+
+import com.kylinolap.metadata.MetadataManager;
+import org.apache.commons.io.IOUtils;
+import org.apache.commons.lang.StringUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.google.common.collect.Lists;
+import com.google.common.collect.Maps;
+import com.google.common.collect.Sets;
+import com.kylinolap.common.KylinConfig;
+import com.kylinolap.common.persistence.ResourceTool;
+import com.kylinolap.common.util.CliCommandExecutor;
+import com.kylinolap.common.util.JsonUtil;
+import com.kylinolap.metadata.model.schema.ColumnDesc;
+import com.kylinolap.metadata.model.schema.TableDesc;
+
+/**
+ * Management class to sync hive table metadata with command See main method for
+ * how to use the class
+ * 
+ * @author jianliu
+ */
+public class HiveSourceTableLoader {
+    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) {
+            int cut = table.indexOf('.');
+            String database = cut >= 0 ? table.substring(0, cut).trim() : "DEFAULT";
+            String tableName = cut >= 0 ? table.substring(cut + 1).trim() : table.trim();
+            Set<String> set = db2tables.get(database);
+            if (set == null) {
+                set = Sets.newHashSet();
+                db2tables.put(database, set);
+            }
+            set.add(tableName);
+        }
+
+        // metadata tmp dir
+        File metaTmpDir = File.createTempFile("meta_tmp", null);
+        metaTmpDir.delete();
+        metaTmpDir.mkdirs();
+
+        for (String database: db2tables.keySet()) {
+            for (String table: db2tables.get(database)) {
+                TableDesc tableDesc = MetadataManager.getInstance(config).getTableDesc(table);
+                if (tableDesc == null) {
+                    continue;
+                }
+                if (tableDesc.getDatabase().equalsIgnoreCase(database)) {
+                    continue;
+                } else {
+                    throw new UnsupportedOperationException(String.format("there is already a table[%s] in database[%s]", tableDesc.getName(), tableDesc.getDatabase()));
+                }
+            }
+        }
+
+        // extract from hive
+        Set<String> loadedTables = Sets.newHashSet();
+        for (String database : db2tables.keySet()) {
+            List<String> loaded = extractHiveTables(database, db2tables.get(database), metaTmpDir, config);
+            loadedTables.addAll(loaded);
+        }
+
+        // save loaded tables
+        ResourceTool.copy(KylinConfig.createInstanceFromUri(metaTmpDir.getAbsolutePath()), config);
+
+        return loadedTables;
+    }
+
+    private static List<String> extractHiveTables(String database, Set<String> tables, File metaTmpDir, KylinConfig config) throws IOException {
+        StringBuilder cmd = new StringBuilder();
+        cmd.append("hive -e \"");
+        if (StringUtils.isEmpty(database) == false) {
+            cmd.append("use " + database + "; ");
+        }
+        for (String table : tables) {
+            cmd.append("show table extended like " + table + "; ");
+        }
+        cmd.append("\"");
+
+        CliCommandExecutor cmdExec = config.getCliCommandExecutor();
+        String output = cmdExec.execute(cmd.toString());
+
+        return extractTableDescFromHiveOutput(database, output, metaTmpDir);
+    }
+
+    private static List<String> extractTableDescFromHiveOutput(String database, String hiveOutput, File metaTmpDir) throws IOException {
+        BufferedReader reader = new BufferedReader(new StringReader(hiveOutput));
+        try {
+            return extractTables(database, reader, metaTmpDir);
+        } finally {
+            IOUtils.closeQuietly(reader);
+        }
+    }
+
+    private static List<String> extractTables(String database, BufferedReader reader, File metaTmpDir) throws IOException {
+
+        File tableDescDir = new File(metaTmpDir, TABLE_FOLDER_NAME);
+        File tableExdDir = new File(metaTmpDir, TABLE_EXD_FOLDER_NAME);
+        mkdirs(tableDescDir);
+        mkdirs(tableExdDir);
+
+        List<TableDesc> tableDescList = new ArrayList<TableDesc>();
+        List<Map<String, String>> tableAttrsList = new ArrayList<Map<String, String>>();
+        getTables(database, reader, tableDescList, tableAttrsList);
+
+        List<String> loadedTables = Lists.newArrayList();
+        
+        for (TableDesc table : tableDescList) {
+            File file = new File(tableDescDir, table.getName().toUpperCase() + "." + OUTPUT_SURFIX);
+            JsonUtil.writeValueIndent(new FileOutputStream(file), table);
+            loadedTables.add(table.getDatabase() + "." + table.getName());
+        }
+
+        for (Map<String, String> tableAttrs : tableAttrsList) {
+            File file = new File(tableExdDir, tableAttrs.get("tableName").toUpperCase() + "." + OUTPUT_SURFIX);
+            JsonUtil.writeValueIndent(new FileOutputStream(file), tableAttrs);
+        }
+        return loadedTables;
+    }
+
+    private static void mkdirs(File metaTmpDir) {
+        if (!metaTmpDir.exists()) {
+            if (!metaTmpDir.mkdirs()) {
+                throw new IllegalArgumentException("Failed to create Output dir : " + metaTmpDir.getAbsolutePath());
+            }
+        }
+    }
+
+    private static void getTables(String database, BufferedReader reader, //
+            List<TableDesc> tableDescList, List<Map<String, String>> tableAttrsList) throws IOException {
+
+        Map<String, String> tableAttrs = new HashMap<String, String>();
+        TableDesc tableDesc = new TableDesc();
+        String line;
+        boolean hit = false;
+        
+        while ((line = reader.readLine()) != null) {
+            logger.info(line);
+            int i = line.indexOf(":");
+            if (i == -1) {
+                continue;
+            }
+            String key = line.substring(0, i);
+            String value = line.substring(i + 1, line.length());
+            if (key.equals("tableName")) {// Create a new table object
+                hit = true;
+                tableAttrs = new HashMap<String, String>();
+                tableAttrsList.add(tableAttrs);
+                tableDesc = new TableDesc();
+                tableDescList.add(tableDesc);
+            }
+
+            if (!hit) {
+                continue;
+            }
+
+            if (line.startsWith("columns")) {// geneate source table metadata
+                String tname = tableAttrs.get("tableName");
+
+                tableDesc.setDatabase(database.toUpperCase());
+                tableDesc.setName(tname.toUpperCase());
+                tableDesc.setUuid(UUID.randomUUID().toString());
+                addColumns(tableDesc, value);
+            }
+            tableAttrs.put(key, value);
+            if (key.equals("lastUpdateTime")) {
+                hit = false;
+            }
+        }
+
+    }
+
+    private static void addColumns(TableDesc sTable, String value) {
+        List<ColumnDesc> columns = new ArrayList<ColumnDesc>();
+        int i1 = value.indexOf("{");
+        int i2 = value.indexOf("}");
+        if (i1 < 0 || i2 < 0 || i1 > i2) {
+            return;
+        }
+        String temp = value.substring(i1 + 1, i2);
+        String[] strArr = temp.split(", ");
+        for (int i = 0; i < strArr.length; i++) {
+            String t1 = strArr[i].trim();
+            int pos = t1.indexOf(" ");
+            String colType = t1.substring(0, pos).trim();
+            String colName = t1.substring(pos).trim();
+            ColumnDesc cdesc = new ColumnDesc();
+            cdesc.setName(colName.toUpperCase());
+            cdesc.setDatatype(convertType(colType));
+            cdesc.setId(String.valueOf(i + 1));
+            columns.add(cdesc);
+        }
+        sTable.setColumns(columns.toArray(new ColumnDesc[0]));
+    }
+
+    private static String convertType(String colType) {
+        if ("i32".equals(colType)) {
+            return "int";
+        } else if ("i64".equals(colType)) {
+            return "bigint";
+        } else if ("i16".equals(colType)) {
+            return "smallint";
+        } else if ("byte".equals(colType)) {
+            return "tinyint";
+        } else if ("bool".equals(colType))
+            return "boolean";
+        return colType;
+    }
+
+    /**
+     */
+    public static void main(String[] args) {
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/metadata/src/main/java/com/kylinolap/metadata/validation/CubeMetadataValidator.java
----------------------------------------------------------------------
diff --git a/metadata/src/main/java/com/kylinolap/metadata/validation/CubeMetadataValidator.java b/metadata/src/main/java/com/kylinolap/metadata/validation/CubeMetadataValidator.java
new file mode 100644
index 0000000..7ee2f97
--- /dev/null
+++ b/metadata/src/main/java/com/kylinolap/metadata/validation/CubeMetadataValidator.java
@@ -0,0 +1,73 @@
+/*
+ * 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.metadata.validation;
+
+import com.kylinolap.metadata.model.cube.CubeDesc;
+import com.kylinolap.metadata.validation.ValidateContext.Result;
+import com.kylinolap.metadata.validation.rule.AggregationGroupSizeRule;
+import com.kylinolap.metadata.validation.rule.FunctionRule;
+import com.kylinolap.metadata.validation.rule.MandatoryColumnRule;
+import com.kylinolap.metadata.validation.rule.RowKeyAttrRule;
+
+/**
+ * For cube metadata validator
+ * 
+ * @author jianliu
+ * 
+ */
+public class CubeMetadataValidator {
+    @SuppressWarnings("unchecked")
+    private IValidatorRule<CubeDesc>[] rules = new IValidatorRule[] { new FunctionRule(), new AggregationGroupSizeRule(), new MandatoryColumnRule(), new RowKeyAttrRule() };
+
+    public ValidateContext validate(CubeDesc cube) {
+        return validate(cube, false);
+    }
+
+    /**
+     * @param cubeDesc
+     * @param inject
+     *            inject error into cube desc
+     * @return
+     */
+    public ValidateContext validate(CubeDesc cube, boolean inject) {
+        ValidateContext context = new ValidateContext();
+        for (int i = 0; i < rules.length; i++) {
+            IValidatorRule<CubeDesc> rule = rules[i];
+            rule.validate(cube, context);
+        }
+        if (inject) {
+            injectResult(cube, context);
+        }
+        return context;
+    }
+
+    /**
+     * 
+     * Inject errors info into cubeDesc
+     * 
+     * @param cubeDesc
+     * @param context
+     */
+    public void injectResult(CubeDesc cubeDesc, ValidateContext context) {
+        Result[] results = context.getResults();
+        for (int i = 0; i < results.length; i++) {
+            Result result = results[i];
+            cubeDesc.addError(result.getLevel() + " : " + result.getMessage(), true);
+        }
+
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/metadata/src/main/java/com/kylinolap/metadata/validation/IValidatorRule.java
----------------------------------------------------------------------
diff --git a/metadata/src/main/java/com/kylinolap/metadata/validation/IValidatorRule.java b/metadata/src/main/java/com/kylinolap/metadata/validation/IValidatorRule.java
new file mode 100644
index 0000000..1091775
--- /dev/null
+++ b/metadata/src/main/java/com/kylinolap/metadata/validation/IValidatorRule.java
@@ -0,0 +1,28 @@
+/*
+ * 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.metadata.validation;
+
+import com.kylinolap.metadata.validation.rule.IKylinValidationConstances;
+
+/**
+ * @author jianliu
+ * 
+ */
+public interface IValidatorRule<T> extends IKylinValidationConstances {
+
+    public void validate(T element, ValidateContext context);
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/metadata/src/main/java/com/kylinolap/metadata/validation/ResultLevel.java
----------------------------------------------------------------------
diff --git a/metadata/src/main/java/com/kylinolap/metadata/validation/ResultLevel.java b/metadata/src/main/java/com/kylinolap/metadata/validation/ResultLevel.java
new file mode 100644
index 0000000..32af9d7
--- /dev/null
+++ b/metadata/src/main/java/com/kylinolap/metadata/validation/ResultLevel.java
@@ -0,0 +1,36 @@
+/*
+ * 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.metadata.validation;
+
+/**
+ * Validation result level
+ * 
+ * @author jianliu
+ * 
+ */
+public enum ResultLevel {
+    ERROR("ERROR"), WARN("WARN");
+    private String level;
+
+    ResultLevel(String level) {
+        this.level = level;
+    }
+
+    public String toString() {
+        return level;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/metadata/src/main/java/com/kylinolap/metadata/validation/SourceTableMetadataValidator.java
----------------------------------------------------------------------
diff --git a/metadata/src/main/java/com/kylinolap/metadata/validation/SourceTableMetadataValidator.java b/metadata/src/main/java/com/kylinolap/metadata/validation/SourceTableMetadataValidator.java
new file mode 100644
index 0000000..deff50b
--- /dev/null
+++ b/metadata/src/main/java/com/kylinolap/metadata/validation/SourceTableMetadataValidator.java
@@ -0,0 +1,32 @@
+/*
+ * 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.metadata.validation;
+
+import com.kylinolap.metadata.model.schema.TableDesc;
+
+/**
+ * Validate Table metadata from source.
+ * <p/>
+ * User: lukhan Date: 12/2/13 Time: 10:45 AM To change this template use File |
+ * Settings | File Templates.
+ */
+public class SourceTableMetadataValidator {
+
+    public static boolean validate(TableDesc table) {
+        // table.get
+        return true;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/metadata/src/main/java/com/kylinolap/metadata/validation/ValidateContext.java
----------------------------------------------------------------------
diff --git a/metadata/src/main/java/com/kylinolap/metadata/validation/ValidateContext.java b/metadata/src/main/java/com/kylinolap/metadata/validation/ValidateContext.java
new file mode 100644
index 0000000..bd27244
--- /dev/null
+++ b/metadata/src/main/java/com/kylinolap/metadata/validation/ValidateContext.java
@@ -0,0 +1,101 @@
+/*
+ * 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.metadata.validation;
+
+import java.io.PrintStream;
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.List;
+
+/**
+ * Context. Supply all dependent objects for validator
+ * 
+ * @author jianliu
+ * 
+ */
+public class ValidateContext {
+    private List<Result> results = new ArrayList<ValidateContext.Result>();
+
+    public void addResult(ResultLevel level, String message) {
+        results.add(new Result(level, message));
+    }
+
+    public void addResult(Result result) {
+        results.add(result);
+    }
+
+    public class Result {
+        private ResultLevel level;
+        private String message;
+
+        /**
+         * @param level
+         * @param message
+         */
+        public Result(ResultLevel level, String message) {
+            this.level = level;
+            this.message = message;
+        }
+
+        /**
+         * @return the level
+         */
+        public ResultLevel getLevel() {
+            return level;
+        }
+
+        /**
+         * @return the message
+         */
+        public String getMessage() {
+            return message;
+        }
+    }
+
+    /**
+     * Get validation result
+     * 
+     * @return
+     */
+    public Result[] getResults() {
+        Result[] rs = new Result[0];
+        rs = results.toArray(rs);
+        return rs;
+    }
+
+    /**
+     * 
+     */
+    public void print(PrintStream out) {
+        if (results.isEmpty()) {
+            out.print("The element is perfect.");
+        }
+        Iterator<Result> it = results.iterator();
+        while (it.hasNext()) {
+            Result result = it.next();
+            out.println(result.level + " : " + result.message);
+        }
+    }
+
+    /**
+     * @return if there is not validation errors
+     */
+    public boolean ifPass() {
+        return results.isEmpty();
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/metadata/src/main/java/com/kylinolap/metadata/validation/rule/AggregationGroupSizeRule.java
----------------------------------------------------------------------
diff --git a/metadata/src/main/java/com/kylinolap/metadata/validation/rule/AggregationGroupSizeRule.java b/metadata/src/main/java/com/kylinolap/metadata/validation/rule/AggregationGroupSizeRule.java
new file mode 100644
index 0000000..8ab5a7e
--- /dev/null
+++ b/metadata/src/main/java/com/kylinolap/metadata/validation/rule/AggregationGroupSizeRule.java
@@ -0,0 +1,64 @@
+/*
+ * 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.metadata.validation.rule;
+
+import com.kylinolap.common.KylinConfig;
+import com.kylinolap.metadata.model.cube.CubeDesc;
+import com.kylinolap.metadata.validation.IValidatorRule;
+import com.kylinolap.metadata.validation.ResultLevel;
+import com.kylinolap.metadata.validation.ValidateContext;
+
+/**
+ * Rule to validate: 1. The aggregationGroup size must be less than 20
+ * 
+ * @author jianliu
+ * 
+ */
+public class AggregationGroupSizeRule implements IValidatorRule<CubeDesc> {
+
+    /*
+     * (non-Javadoc)
+     * 
+     * @see
+     * com.kylinolap.metadata.validation.IValidatorRule#validate(java.lang.Object
+     * , com.kylinolap.metadata.validation.ValidateContext)
+     */
+    @Override
+    public void validate(CubeDesc cube, ValidateContext context) {
+        innerValidateMaxSize(cube, context);
+    }
+
+    /**
+     * @param cube
+     * @param context
+     */
+    private void innerValidateMaxSize(CubeDesc cube, ValidateContext context) {
+        int maxSize = getMaxAgrGroupSize();
+        String[][] groups = cube.getRowkey().getAggregationGroups();
+        for (int i = 0; i < groups.length; i++) {
+            String[] group = groups[i];
+            if (group.length >= maxSize) {
+                context.addResult(ResultLevel.ERROR, "Length of the number " + i + " aggregation group's length should be less than " + maxSize);
+            }
+        }
+    }
+
+    protected int getMaxAgrGroupSize() {
+        String size = KylinConfig.getInstanceFromEnv().getProperty(KEY_MAX_AGR_GROUP_SIZE, String.valueOf(DEFAULT_MAX_AGR_GROUP_SIZE));
+        return Integer.parseInt(size);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/metadata/src/main/java/com/kylinolap/metadata/validation/rule/FunctionRule.java
----------------------------------------------------------------------
diff --git a/metadata/src/main/java/com/kylinolap/metadata/validation/rule/FunctionRule.java b/metadata/src/main/java/com/kylinolap/metadata/validation/rule/FunctionRule.java
new file mode 100644
index 0000000..4d2543e
--- /dev/null
+++ b/metadata/src/main/java/com/kylinolap/metadata/validation/rule/FunctionRule.java
@@ -0,0 +1,181 @@
+/*
+ * 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.metadata.validation.rule;
+
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Set;
+
+import org.apache.commons.lang.StringUtils;
+
+import com.kylinolap.common.KylinConfig;
+import com.kylinolap.metadata.MetadataManager;
+import com.kylinolap.metadata.model.cube.CubeDesc;
+import com.kylinolap.metadata.model.cube.FunctionDesc;
+import com.kylinolap.metadata.model.cube.MeasureDesc;
+import com.kylinolap.metadata.model.cube.ParameterDesc;
+import com.kylinolap.metadata.model.schema.ColumnDesc;
+import com.kylinolap.metadata.model.schema.DataType;
+import com.kylinolap.metadata.model.schema.TableDesc;
+import com.kylinolap.metadata.validation.IValidatorRule;
+import com.kylinolap.metadata.validation.ResultLevel;
+import com.kylinolap.metadata.validation.ValidateContext;
+
+/**
+ * Validate function parameter. Ticket:
+ * https://github.scm.corp.ebay.com/Kylin/Kylin/issues/268
+ * <p/>
+ * if type is column, check values are valid fact table columns if type is
+ * constant, the value only can be numberic
+ * <p/>
+ * the return type only can be int/bigint/long/double/decimal
+ *
+ * @author jianliu
+ */
+public class FunctionRule implements IValidatorRule<CubeDesc> {
+
+    /*
+     * (non-Javadoc)
+     * 
+     * @see
+     * com.kylinolap.metadata.validation.IValidatorRule#validate(java.lang.Object
+     * , com.kylinolap.metadata.validation.ValidateContext)
+     */
+    @Override
+    public void validate(CubeDesc cube, ValidateContext context) {
+        List<MeasureDesc> measures = cube.getMeasures();
+
+        List<FunctionDesc> countFuncs = new ArrayList<FunctionDesc>();
+
+        Iterator<MeasureDesc> it = measures.iterator();
+        while (it.hasNext()) {
+            MeasureDesc measure = it.next();
+            FunctionDesc func = measure.getFunction();
+            ParameterDesc parameter = func.getParameter();
+            if (parameter == null) {
+                context.addResult(ResultLevel.ERROR, "Must define parameter for function " + func.getExpression() + " in " + measure.getName());
+                return;
+            }
+
+            String type = func.getParameter().getType();
+            String value = func.getParameter().getValue();
+            if (StringUtils.isEmpty(type)) {
+                context.addResult(ResultLevel.ERROR, "Must define type for parameter type " + func.getExpression() + " in " + measure.getName());
+                return;
+            }
+            if (StringUtils.isEmpty(value)) {
+                context.addResult(ResultLevel.ERROR, "Must define type for parameter value " + func.getExpression() + " in " + measure.getName());
+                return;
+            }
+            if (StringUtils.isEmpty(func.getReturnType())) {
+                context.addResult(ResultLevel.ERROR, "Must define return type for function " + func.getExpression() + " in " + measure.getName());
+                return;
+            }
+
+            if (StringUtils.equalsIgnoreCase(FunctionDesc.PARAMETER_TYPE_COLUMN, type)) {
+                validateColumnParameter(context, cube, value);
+            } else if (StringUtils.equals(FunctionDesc.PARAMTER_TYPE_CONSTANT, type)) {
+                validateCostantParameter(context, cube, value);
+            }
+            validateReturnType(context, cube, func);
+
+            if (func.isCount())
+                countFuncs.add(func);
+        }
+
+        if (countFuncs.size() != 1) {
+            context.addResult(ResultLevel.ERROR, "Must define one and only one count(1) function, but there are " + countFuncs.size() + " -- " + countFuncs);
+        }
+    }
+
+    private void validateReturnType(ValidateContext context, CubeDesc cube, FunctionDesc funcDesc) {
+
+        String func = funcDesc.getExpression();
+        DataType rtype = funcDesc.getReturnDataType();
+
+        if (funcDesc.isCount()) {
+            if (rtype.isIntegerFamily() == false) {
+                context.addResult(ResultLevel.ERROR, "Return type for function " + func + " must be one of " + DataType.INTEGER_FAMILY);
+            }
+        } else if (funcDesc.isCountDistinct()) {
+            if (rtype.isHLLC() == false && funcDesc.isHolisticCountDistinct() == false) {
+                context.addResult(ResultLevel.ERROR, "Return type for function " + func + " must be hllc(10), hllc(12) etc.");
+            }
+        } else if (funcDesc.isMax() || funcDesc.isMin() || funcDesc.isSum()) {
+            if (rtype.isNumberFamily() == false) {
+                context.addResult(ResultLevel.ERROR, "Return type for function " + func + " must be one of " + DataType.NUMBER_FAMILY);
+            }
+        } else {
+            if (StringUtils.equalsIgnoreCase(KylinConfig.getInstanceFromEnv().getProperty(KEY_IGNORE_UNKNOWN_FUNC, "false"), "false")) {
+                context.addResult(ResultLevel.ERROR, "Unrecognized function: [" + func + "]");
+            }
+        }
+
+    }
+
+    /**
+     * @param context
+     * @param cube
+     * @param value
+     */
+    private void validateCostantParameter(ValidateContext context, CubeDesc cube, String value) {
+        try {
+            Integer.parseInt(value);
+        } catch (Exception e) {
+            context.addResult(ResultLevel.ERROR, "Parameter value must be number, but it is " + value);
+        }
+    }
+
+    /**
+     * @param context
+     * @param cube
+     * @param value
+     */
+    private void validateColumnParameter(ValidateContext context, CubeDesc cube, String value) {
+        String factTable = cube.getFactTable();
+        if (StringUtils.isEmpty(factTable)) {
+            context.addResult(ResultLevel.ERROR, "Fact table can not be null.");
+            return;
+        }
+        TableDesc table = MetadataManager.getInstance(cube.getConfig()).getTableDesc(factTable);
+        if (table == null) {
+            context.addResult(ResultLevel.ERROR, "Fact table can not be found: " + cube);
+            return;
+        }
+        // Prepare column set
+        Set<String> set = new HashSet<String>();
+        ColumnDesc[] cdesc = table.getColumns();
+        for (int i = 0; i < cdesc.length; i++) {
+            ColumnDesc columnDesc = cdesc[i];
+            set.add(columnDesc.getName());
+        }
+
+        String[] items = value.split(",");
+        for (int i = 0; i < items.length; i++) {
+            String item = items[i].trim();
+            if (StringUtils.isEmpty(item)) {
+                continue;
+            }
+            if (!set.contains(item)) {
+                context.addResult(ResultLevel.ERROR, "Column [" + item + "] does not exist in factable table" + factTable);
+            }
+        }
+
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/metadata/src/main/java/com/kylinolap/metadata/validation/rule/IKylinValidationConstances.java
----------------------------------------------------------------------
diff --git a/metadata/src/main/java/com/kylinolap/metadata/validation/rule/IKylinValidationConstances.java b/metadata/src/main/java/com/kylinolap/metadata/validation/rule/IKylinValidationConstances.java
new file mode 100644
index 0000000..bc1bbc6
--- /dev/null
+++ b/metadata/src/main/java/com/kylinolap/metadata/validation/rule/IKylinValidationConstances.java
@@ -0,0 +1,31 @@
+/*
+ * 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.metadata.validation.rule;
+
+import com.kylinolap.metadata.MetadataConstances;
+
+/**
+ * @author jianliu
+ * 
+ */
+public interface IKylinValidationConstances extends MetadataConstances {
+
+    public static final int DEFAULT_MAX_AGR_GROUP_SIZE = 20;
+    public static final String KEY_MAX_AGR_GROUP_SIZE = "rule_max.arggregation.group.size";
+    public static final String KEY_IGNORE_UNKNOWN_FUNC = "rule_ignore_unknown_func";
+
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/metadata/src/main/java/com/kylinolap/metadata/validation/rule/MandatoryColumnRule.java
----------------------------------------------------------------------
diff --git a/metadata/src/main/java/com/kylinolap/metadata/validation/rule/MandatoryColumnRule.java b/metadata/src/main/java/com/kylinolap/metadata/validation/rule/MandatoryColumnRule.java
new file mode 100644
index 0000000..2e2d43e
--- /dev/null
+++ b/metadata/src/main/java/com/kylinolap/metadata/validation/rule/MandatoryColumnRule.java
@@ -0,0 +1,74 @@
+/*
+ * 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.metadata.validation.rule;
+
+import java.util.HashSet;
+import java.util.Set;
+
+import org.apache.commons.lang.ArrayUtils;
+
+import com.kylinolap.metadata.model.cube.CubeDesc;
+import com.kylinolap.metadata.model.cube.RowKeyColDesc;
+import com.kylinolap.metadata.validation.IValidatorRule;
+import com.kylinolap.metadata.validation.ResultLevel;
+import com.kylinolap.metadata.validation.ValidateContext;
+
+/**
+ * Validate that mandatory column must NOT appear in aggregation group.
+ * 
+ * @author jianliu
+ * 
+ */
+public class MandatoryColumnRule implements IValidatorRule<CubeDesc> {
+
+    /*
+     * (non-Javadoc)
+     * 
+     * @see
+     * com.kylinolap.metadata.validation.IValidatorRule#validate(java.lang.Object
+     * , com.kylinolap.metadata.validation.ValidateContext)
+     */
+    @Override
+    public void validate(CubeDesc cube, ValidateContext context) {
+        Set<String> mands = new HashSet<String>();
+        RowKeyColDesc[] cols = cube.getRowkey().getRowKeyColumns();
+        if (cols == null || cols.length == 0) {
+            return;
+        }
+        for (int i = 0; i < cols.length; i++) {
+            RowKeyColDesc rowKeyColDesc = cols[i];
+            if (rowKeyColDesc.isMandatory()) {
+                mands.add(rowKeyColDesc.getColumn());
+            }
+        }
+        if (mands.isEmpty()) {
+            return;
+        }
+        String[][] groups = cube.getRowkey().getAggregationGroups();
+        for (int i = 0; i < groups.length; i++) {
+            String[] group = groups[i];
+            for (int j = 0; j < group.length; j++) {
+                String col = group[j];
+                if (mands.contains(col)) {
+                    context.addResult(ResultLevel.ERROR, "mandatory column " + col + " must not be in aggregation group [" + ArrayUtils.toString(group) + "]");
+                }
+            }
+        }
+
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/metadata/src/main/java/com/kylinolap/metadata/validation/rule/RowKeyAttrRule.java
----------------------------------------------------------------------
diff --git a/metadata/src/main/java/com/kylinolap/metadata/validation/rule/RowKeyAttrRule.java b/metadata/src/main/java/com/kylinolap/metadata/validation/rule/RowKeyAttrRule.java
new file mode 100644
index 0000000..5253cea
--- /dev/null
+++ b/metadata/src/main/java/com/kylinolap/metadata/validation/rule/RowKeyAttrRule.java
@@ -0,0 +1,73 @@
+/*
+ * 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.metadata.validation.rule;
+
+import org.apache.commons.lang.StringUtils;
+
+import com.kylinolap.metadata.model.cube.CubeDesc;
+import com.kylinolap.metadata.model.cube.RowKeyColDesc;
+import com.kylinolap.metadata.model.cube.RowKeyDesc;
+import com.kylinolap.metadata.validation.IValidatorRule;
+import com.kylinolap.metadata.validation.ResultLevel;
+import com.kylinolap.metadata.validation.ValidateContext;
+
+/**
+ * Validate that only one of "length" and "dictionary" appears on rowkey_column
+ * 
+ * @author jianliu
+ * 
+ */
+public class RowKeyAttrRule implements IValidatorRule<CubeDesc> {
+
+    /*
+     * (non-Javadoc)
+     * 
+     * @see
+     * com.kylinolap.metadata.validation.IValidatorRule#validate(java.lang.Object
+     * , com.kylinolap.metadata.validation.ValidateContext)
+     */
+    @Override
+    public void validate(CubeDesc cube, ValidateContext context) {
+        RowKeyDesc row = cube.getRowkey();
+        if (row == null) {
+            context.addResult(ResultLevel.ERROR, "Rowkey does not exist");
+            return;
+        }
+
+        RowKeyColDesc[] rcd = row.getRowKeyColumns();
+        if (rcd == null) {
+            context.addResult(ResultLevel.ERROR, "Rowkey columns do not exist");
+            return;
+        }
+        if(rcd.length == 0){
+            context.addResult(ResultLevel.ERROR, "Rowkey columns is empty");
+            return;       	
+        }
+
+        for (int i = 0; i < rcd.length; i++) {
+            RowKeyColDesc rd = rcd[i];
+            if (rd.getLength() != 0 && !StringUtils.isEmpty(rd.getDictionary())) {
+                context.addResult(ResultLevel.ERROR, "Rowkey column " + rd.getColumn() + " must not have both 'length' and 'dictionary' attribute");
+            }
+            if (rd.getLength() == 0 && StringUtils.isEmpty(rd.getDictionary())) {
+                context.addResult(ResultLevel.ERROR, "Rowkey column " + rd.getColumn() + " must not have both 'length' and 'dictionary' empty");
+            }
+        }
+
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/metadata/src/test/java/com/kylinolap/metadata/CubeDescTest.java
----------------------------------------------------------------------
diff --git a/metadata/src/test/java/com/kylinolap/metadata/CubeDescTest.java b/metadata/src/test/java/com/kylinolap/metadata/CubeDescTest.java
new file mode 100644
index 0000000..1320b73
--- /dev/null
+++ b/metadata/src/test/java/com/kylinolap/metadata/CubeDescTest.java
@@ -0,0 +1,50 @@
+/*
+ * 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.metadata;
+
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+import com.kylinolap.common.util.JsonUtil;
+import com.kylinolap.common.util.LocalFileMetadataTestCase;
+import com.kylinolap.metadata.model.cube.CubeDesc;
+
+/**
+ * @author yangli9
+ */
+public class CubeDescTest extends LocalFileMetadataTestCase {
+
+    @Before
+    public void setUp() throws Exception {
+        this.createTestMetadata();
+    }
+
+    @After
+    public void after() throws Exception {
+        this.cleanupTestMetadata();
+    }
+
+    @Test
+    public void testSerialize() throws Exception {
+        CubeDesc desc = MetadataManager.getInstance(this.getTestConfig()).getCubeDesc("test_kylin_cube_with_slr_desc");
+        String str = JsonUtil.writeValueAsIndentString(desc);
+        System.out.println(str);
+        @SuppressWarnings("unused")
+        CubeDesc desc2 = JsonUtil.readValue(str, CubeDesc.class);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/metadata/src/test/java/com/kylinolap/metadata/MetadataManagerTest.java
----------------------------------------------------------------------
diff --git a/metadata/src/test/java/com/kylinolap/metadata/MetadataManagerTest.java b/metadata/src/test/java/com/kylinolap/metadata/MetadataManagerTest.java
new file mode 100644
index 0000000..a559ac7
--- /dev/null
+++ b/metadata/src/test/java/com/kylinolap/metadata/MetadataManagerTest.java
@@ -0,0 +1,73 @@
+/*
+ * 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.metadata;
+
+import java.util.List;
+
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+
+import com.kylinolap.common.util.LocalFileMetadataTestCase;
+import com.kylinolap.metadata.model.cube.CubeDesc;
+import com.kylinolap.metadata.model.schema.TableDesc;
+
+/**
+ * Created with IntelliJ IDEA. User: lukhan Date: 9/24/13 Time: 2:38 PM To
+ * change this template use File | Settings | File Templates.
+ */
+public class MetadataManagerTest extends LocalFileMetadataTestCase {
+
+    @Before
+    public void setUp() throws Exception {
+        this.createTestMetadata();
+    }
+
+    @After
+    public void after() throws Exception {
+        this.cleanupTestMetadata();
+    }
+
+    @Test
+    public void testListAllTables() throws Exception {
+        List<TableDesc> tables = MetadataManager.getInstance(this.getTestConfig()).listAllTables();
+        Assert.assertNotNull(tables);
+        Assert.assertTrue(tables.size() > 0);
+    }
+
+    @Test
+    public void testFindTableByName() throws Exception {
+        TableDesc table = MetadataManager.getInstance(this.getTestConfig()).getTableDesc("TEST_CAL_DT");
+        Assert.assertNotNull(table);
+        Assert.assertEquals("TEST_CAL_DT", table.getName());
+    }
+
+    @Test
+    public void testGetInstance() throws Exception {
+
+        Assert.assertNotNull(MetadataManager.getInstance(this.getTestConfig()));
+        Assert.assertNotNull(MetadataManager.getInstance(this.getTestConfig()).listAllTables());
+        Assert.assertTrue(MetadataManager.getInstance(this.getTestConfig()).listAllTables().size() > 0);
+    }
+
+    @Test
+    public void testGetCubeDesc() throws Exception {
+        CubeDesc cubeDesc = MetadataManager.getInstance(this.getTestConfig()).getCubeDesc("test_kylin_cube_with_slr_desc");
+        Assert.assertNotNull(cubeDesc);
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/metadata/src/test/java/com/kylinolap/metadata/validation/rule/AggregationGroupSizeRuleTest.java
----------------------------------------------------------------------
diff --git a/metadata/src/test/java/com/kylinolap/metadata/validation/rule/AggregationGroupSizeRuleTest.java b/metadata/src/test/java/com/kylinolap/metadata/validation/rule/AggregationGroupSizeRuleTest.java
new file mode 100644
index 0000000..356c6e7
--- /dev/null
+++ b/metadata/src/test/java/com/kylinolap/metadata/validation/rule/AggregationGroupSizeRuleTest.java
@@ -0,0 +1,98 @@
+/*
+ * 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.metadata.validation.rule;
+
+import static org.junit.Assert.*;
+
+import org.junit.Before;
+import org.junit.Test;
+
+import com.kylinolap.common.util.JsonUtil;
+import com.kylinolap.metadata.model.cube.CubeDesc;
+import com.kylinolap.metadata.validation.IValidatorRule;
+import com.kylinolap.metadata.validation.ValidateContext;
+
+/**
+ * @author jianliu
+ * 
+ */
+public class AggregationGroupSizeRuleTest {
+
+    private CubeDesc cube;
+    private ValidateContext vContext = new ValidateContext();
+
+    /**
+     * @throws java.lang.Exception
+     */
+    @Before
+    public void setUp() throws Exception {
+        CubeDesc desc2 = JsonUtil.readValue(getClass().getClassLoader().getResourceAsStream("data/TEST2_desc.json"), CubeDesc.class);
+        this.cube = desc2;
+
+    }
+
+    @Test
+    public void testOneMandatoryColumn() {
+        IValidatorRule<CubeDesc> rule = new AggregationGroupSizeRule() {
+            /*
+             * (non-Javadoc)
+             * 
+             * @see
+             * com.kylinolap.metadata.validation.rule.AggregationGroupSizeRule
+             * #getMaxAgrGroupSize()
+             */
+            @Override
+            protected int getMaxAgrGroupSize() {
+                return 3;
+            }
+        };
+        rule.validate(cube, vContext);
+        vContext.print(System.out);
+        assertEquals("Failed to validate aggragation group error", vContext.getResults().length, 2);
+        assertTrue("Failed to validate aggragation group error", vContext.getResults()[0].getMessage().startsWith("Length of the number"));
+        assertTrue("Failed to validate aggragation group error", vContext.getResults()[1].getMessage().startsWith("Length of the number"));
+        // assertTrue("Failed to validate aggragation group error",
+        // vContext.getResults()[2].getMessage()
+        // .startsWith("Hierachy column"));
+    }
+
+    @Test
+    public void testAggColumnSize() {
+        AggregationGroupSizeRule rule = new AggregationGroupSizeRule() {
+            /*
+             * (non-Javadoc)
+             * 
+             * @see
+             * com.kylinolap.metadata.validation.rule.AggregationGroupSizeRule
+             * #getMaxAgrGroupSize()
+             */
+            @Override
+            protected int getMaxAgrGroupSize() {
+                return 20;
+            }
+        };
+        rule.validate(cube, vContext);
+        vContext.print(System.out);
+        assertEquals("Failed to validate aggragation group error", vContext.getResults().length, 0);
+        // assertTrue("Failed to validate aggragation group error",
+        // vContext.getResults()[0].getMessage()
+        // .startsWith("Aggregation group"));
+        // assertTrue("Failed to validate aggragation group error",
+        // vContext.getResults()[0].getMessage()
+        // .startsWith("Hierachy column"));
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/metadata/src/test/java/com/kylinolap/metadata/validation/rule/MandatoryColumnRuleTest.java
----------------------------------------------------------------------
diff --git a/metadata/src/test/java/com/kylinolap/metadata/validation/rule/MandatoryColumnRuleTest.java b/metadata/src/test/java/com/kylinolap/metadata/validation/rule/MandatoryColumnRuleTest.java
new file mode 100644
index 0000000..a78bfe4
--- /dev/null
+++ b/metadata/src/test/java/com/kylinolap/metadata/validation/rule/MandatoryColumnRuleTest.java
@@ -0,0 +1,55 @@
+/*
+ * 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.metadata.validation.rule;
+
+import static org.junit.Assert.*;
+
+import org.junit.Before;
+import org.junit.Test;
+
+import com.kylinolap.common.util.JsonUtil;
+import com.kylinolap.metadata.model.cube.CubeDesc;
+import com.kylinolap.metadata.validation.IValidatorRule;
+import com.kylinolap.metadata.validation.ValidateContext;
+
+/**
+ * @author jianliu
+ * 
+ */
+public class MandatoryColumnRuleTest {
+
+    private CubeDesc cube;
+    private ValidateContext vContext = new ValidateContext();
+
+    /**
+     * @throws java.lang.Exception
+     */
+    @Before
+    public void setUp() throws Exception {
+        CubeDesc desc2 = JsonUtil.readValue(getClass().getClassLoader().getResourceAsStream("data/TEST1_desc.json"), CubeDesc.class);
+        this.cube = desc2;
+
+    }
+
+    @Test
+    public void testOneMandatoryColumn() {
+        IValidatorRule<CubeDesc> rule = new MandatoryColumnRule();
+        rule.validate(cube, vContext);
+        assertTrue("Failed to validate mandatory error", vContext.getResults().length == 1);
+        assertTrue("Failed to validate mandatory error", vContext.getResults()[0].getMessage().startsWith("mandatory column"));
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/metadata/src/test/java/com/kylinolap/metadata/validation/rule/RowKeyAttrRuleTest.java
----------------------------------------------------------------------
diff --git a/metadata/src/test/java/com/kylinolap/metadata/validation/rule/RowKeyAttrRuleTest.java b/metadata/src/test/java/com/kylinolap/metadata/validation/rule/RowKeyAttrRuleTest.java
new file mode 100644
index 0000000..4f9ebe3
--- /dev/null
+++ b/metadata/src/test/java/com/kylinolap/metadata/validation/rule/RowKeyAttrRuleTest.java
@@ -0,0 +1,56 @@
+/*
+ * 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.metadata.validation.rule;
+
+import static org.junit.Assert.*;
+
+import org.junit.Before;
+import org.junit.Test;
+
+import com.kylinolap.common.util.JsonUtil;
+import com.kylinolap.metadata.model.cube.CubeDesc;
+import com.kylinolap.metadata.validation.IValidatorRule;
+import com.kylinolap.metadata.validation.ValidateContext;
+
+/**
+ * @author jianliu
+ * 
+ */
+public class RowKeyAttrRuleTest {
+
+    private CubeDesc cube;
+    private ValidateContext vContext = new ValidateContext();
+
+    /**
+     * @throws java.lang.Exception
+     */
+    @Before
+    public void setUp() throws Exception {
+        CubeDesc desc2 = JsonUtil.readValue(getClass().getClassLoader().getResourceAsStream("data/TEST3_desc.json"), CubeDesc.class);
+        this.cube = desc2;
+
+    }
+
+    @Test
+    public void testOneMandatoryColumn() {
+        IValidatorRule<CubeDesc> rule = new RowKeyAttrRule();
+        rule.validate(cube, vContext);
+        vContext.print(System.out);
+        assertTrue("Failed to validate rowkey", vContext.getResults().length == 1);
+        assertTrue("Failed to validate mandatory error", vContext.getResults()[0].getMessage().startsWith("Rowkey column"));
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/metadata/src/test/resources/com/kylinolap/metadata/kylin_env.properties
----------------------------------------------------------------------
diff --git a/metadata/src/test/resources/com/kylinolap/metadata/kylin_env.properties b/metadata/src/test/resources/com/kylinolap/metadata/kylin_env.properties
new file mode 100644
index 0000000..b9acf90
--- /dev/null
+++ b/metadata/src/test/resources/com/kylinolap/metadata/kylin_env.properties
@@ -0,0 +1,5 @@
+PROD=/etc/kylin
+QA=/etc/kylin
+DEV=/etc/kylin
+
+YADEV=hbase:yadesk00:2181:/hbase-unsecure
\ No newline at end of file


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

Posted by lu...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/cube/src/main/java/com/kylinolap/cube/CubeSegment.java
----------------------------------------------------------------------
diff --git a/cube/src/main/java/com/kylinolap/cube/CubeSegment.java b/cube/src/main/java/com/kylinolap/cube/CubeSegment.java
new file mode 100644
index 0000000..744ee9a
--- /dev/null
+++ b/cube/src/main/java/com/kylinolap/cube/CubeSegment.java
@@ -0,0 +1,307 @@
+/*
+ * 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.cube;
+
+import java.text.SimpleDateFormat;
+import java.util.Collection;
+import java.util.Map;
+import java.util.TimeZone;
+import java.util.concurrent.ConcurrentHashMap;
+
+import com.fasterxml.jackson.annotation.JsonAutoDetect;
+import com.fasterxml.jackson.annotation.JsonAutoDetect.Visibility;
+import com.fasterxml.jackson.annotation.JsonBackReference;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.google.common.base.Objects;
+import com.kylinolap.metadata.model.cube.CubeDesc;
+import com.kylinolap.metadata.model.cube.TblColRef;
+
+@JsonAutoDetect(fieldVisibility = Visibility.NONE, getterVisibility = Visibility.NONE, isGetterVisibility = Visibility.NONE, setterVisibility = Visibility.NONE)
+public class CubeSegment implements Comparable<CubeSegment> {
+
+    @JsonBackReference
+    private CubeInstance cubeInstance;
+    @JsonProperty("uuid")
+    private String uuid;
+    @JsonProperty("name")
+    private String name;
+    @JsonProperty("storage_location_identifier")
+    private String storageLocationIdentifier; // HTable name
+    @JsonProperty("date_range_start")
+    private long dateRangeStart;
+    @JsonProperty("date_range_end")
+    private long dateRangeEnd;
+    @JsonProperty("status")
+    private CubeSegmentStatusEnum status;
+    @JsonProperty("size_kb")
+    private long sizeKB;
+    @JsonProperty("source_records")
+    private long sourceRecords;
+    @JsonProperty("source_records_size")
+    private long sourceRecordsSize;
+    @JsonProperty("last_build_time")
+    private long lastBuildTime;
+    @JsonProperty("last_build_job_id")
+    private String lastBuildJobID;
+    @JsonProperty("create_time")
+    private String createTime;
+
+    @JsonProperty("binary_signature")
+    private String binarySignature; // a hash of cube schema and dictionary ID,
+                                    // used for sanity check
+
+    @JsonProperty("dictionaries")
+    private ConcurrentHashMap<String, String> dictionaries; // table/column ==> dictionary resource path
+    @JsonProperty("snapshots")
+    private ConcurrentHashMap<String, String> snapshots; // table name ==> snapshot resource path
+
+    public CubeDesc getCubeDesc() {
+        return getCubeInstance().getDescriptor();
+    }
+
+    /**
+     * @param startDate
+     * @param endDate
+     * @return if(startDate == 0 && endDate == 0), returns "FULL_BUILD", else
+     *         returns "yyyyMMddHHmmss_yyyyMMddHHmmss"
+     */
+    public static String getSegmentName(long startDate, long endDate) {
+        if (startDate == 0 && endDate == 0) {
+            return "FULL_BUILD";
+        }
+
+        SimpleDateFormat dateFormat = new SimpleDateFormat("yyyyMMddHHmmss");
+        dateFormat.setTimeZone(TimeZone.getTimeZone("GMT"));
+
+        return dateFormat.format(startDate) + "_" + dateFormat.format(endDate);
+    }
+
+    // ============================================================================
+
+
+    public String getUuid() {
+        return uuid;
+    }
+
+    public void setUuid(String id) {
+        this.uuid = id;
+    }
+
+    public String getName() {
+        return name;
+    }
+
+    public void setName(String name) {
+        this.name = name;
+    }
+
+    public long getDateRangeStart() {
+        return dateRangeStart;
+    }
+
+    public void setDateRangeStart(long dateRangeStart) {
+        this.dateRangeStart = dateRangeStart;
+    }
+
+    public long getDateRangeEnd() {
+        return dateRangeEnd;
+    }
+
+    public void setDateRangeEnd(long dateRangeEnd) {
+        this.dateRangeEnd = dateRangeEnd;
+    }
+
+    public CubeSegmentStatusEnum getStatus() {
+        return status;
+    }
+
+    public void setStatus(CubeSegmentStatusEnum status) {
+        this.status = status;
+    }
+
+    public long getSizeKB() {
+        return sizeKB;
+    }
+
+    public void setSizeKB(long sizeKB) {
+        this.sizeKB = sizeKB;
+    }
+
+    public long getSourceRecords() {
+        return sourceRecords;
+    }
+
+    public void setSourceRecords(long sourceRecords) {
+        this.sourceRecords = sourceRecords;
+    }
+
+    public long getSourceRecordsSize() {
+        return sourceRecordsSize;
+    }
+
+    public void setSourceRecordsSize(long sourceRecordsSize) {
+        this.sourceRecordsSize = sourceRecordsSize;
+    }
+
+    public long getLastBuildTime() {
+        return lastBuildTime;
+    }
+
+    public void setLastBuildTime(long lastBuildTime) {
+        this.lastBuildTime = lastBuildTime;
+    }
+
+    public String getLastBuildJobID() {
+        return lastBuildJobID;
+    }
+
+    public void setLastBuildJobID(String lastBuildJobID) {
+        this.lastBuildJobID = lastBuildJobID;
+    }
+
+    public String getCreateTime() {
+        return createTime;
+    }
+
+    public void setCreateTime(String createTime) {
+        this.createTime = createTime;
+    }
+
+    public String getBinarySignature() {
+        return binarySignature;
+    }
+
+    public void setBinarySignature(String binarySignature) {
+        this.binarySignature = binarySignature;
+    }
+
+    public CubeInstance getCubeInstance() {
+        return cubeInstance;
+    }
+
+    public void setCubeInstance(CubeInstance cubeInstance) {
+        this.cubeInstance = cubeInstance;
+    }
+
+    public String getStorageLocationIdentifier() {
+
+        return storageLocationIdentifier;
+    }
+
+    public Map<String, String> getDictionaries() {
+        if (dictionaries == null)
+            dictionaries = new ConcurrentHashMap<String, String>();
+        return dictionaries;
+    }
+
+    public Map<String, String> getSnapshots() {
+        if (snapshots == null)
+            snapshots = new ConcurrentHashMap<String, String>();
+        return snapshots;
+    }
+
+    public String getSnapshotResPath(String table) {
+        return getSnapshots().get(table);
+    }
+
+    public void putSnapshotResPath(String table, String snapshotResPath) {
+        getSnapshots().put(table, snapshotResPath);
+    }
+
+    public Collection<String> getDictionaryPaths() {
+        return getDictionaries().values();
+    }
+
+    public Collection<String> getSnapshotPaths() {
+        return getSnapshots().values();
+    }
+
+    public String getDictResPath(TblColRef col) {
+        return getDictionaries().get(dictKey(col));
+    }
+
+    public void putDictResPath(TblColRef col, String dictResPath) {
+        getDictionaries().put(dictKey(col), dictResPath);
+    }
+
+    private String dictKey(TblColRef col) {
+        return col.getTable() + "/" + col.getName();
+    }
+
+    /**
+     * @param storageLocationIdentifier
+     *            the storageLocationIdentifier to set
+     */
+    public void setStorageLocationIdentifier(String storageLocationIdentifier) {
+        this.storageLocationIdentifier = storageLocationIdentifier;
+    }
+
+    @Override
+    public int compareTo(CubeSegment other) {
+        if (this.dateRangeEnd < other.dateRangeEnd) {
+            return -1;
+        } else if (this.dateRangeEnd > other.dateRangeEnd) {
+            return 1;
+        } else {
+            return 0;
+        }
+    }
+
+    @Override
+    public int hashCode() {
+        final int prime = 31;
+        int result = 1;
+        result = prime * result + ((cubeInstance == null) ? 0 : cubeInstance.hashCode());
+        result = prime * result + ((name == null) ? 0 : name.hashCode());
+        result = prime * result + ((status == null) ? 0 : status.hashCode());
+        return result;
+    }
+
+    @Override
+    public boolean equals(Object obj) {
+        if (this == obj)
+            return true;
+        if (obj == null)
+            return false;
+        if (getClass() != obj.getClass())
+            return false;
+        CubeSegment other = (CubeSegment) obj;
+        if (cubeInstance == null) {
+            if (other.cubeInstance != null)
+                return false;
+        } else if (!cubeInstance.equals(other.cubeInstance))
+            return false;
+        if (name == null) {
+            if (other.name != null)
+                return false;
+        } else if (!name.equals(other.name))
+            return false;
+        if (status != other.status)
+            return false;
+        return true;
+    }
+
+    @Override
+    public String toString() {
+        return Objects.toStringHelper(this)
+                .add("uuid", uuid)
+                .add("create_time:", createTime)
+                .add("name", name)
+                .add("last_build_job_id", lastBuildJobID)
+                .add("status", status)
+                .toString();
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/cube/src/main/java/com/kylinolap/cube/CubeSegmentStatusEnum.java
----------------------------------------------------------------------
diff --git a/cube/src/main/java/com/kylinolap/cube/CubeSegmentStatusEnum.java b/cube/src/main/java/com/kylinolap/cube/CubeSegmentStatusEnum.java
new file mode 100644
index 0000000..88fb852
--- /dev/null
+++ b/cube/src/main/java/com/kylinolap/cube/CubeSegmentStatusEnum.java
@@ -0,0 +1,25 @@
+/*
+ * 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.cube;
+
+/**
+ * @author xduo
+ * 
+ */
+public enum CubeSegmentStatusEnum {
+    NEW, READY, READY_PENDING
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/cube/src/main/java/com/kylinolap/cube/CubeSegmentTypeEnum.java
----------------------------------------------------------------------
diff --git a/cube/src/main/java/com/kylinolap/cube/CubeSegmentTypeEnum.java b/cube/src/main/java/com/kylinolap/cube/CubeSegmentTypeEnum.java
new file mode 100644
index 0000000..e1f6928
--- /dev/null
+++ b/cube/src/main/java/com/kylinolap/cube/CubeSegmentTypeEnum.java
@@ -0,0 +1,25 @@
+/*
+ * 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.cube;
+
+/**
+ * @author ysong1
+ * 
+ */
+public enum CubeSegmentTypeEnum {
+    TRANSIENT, PERMANENT
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/cube/src/main/java/com/kylinolap/cube/CubeSegmentValidator.java
----------------------------------------------------------------------
diff --git a/cube/src/main/java/com/kylinolap/cube/CubeSegmentValidator.java b/cube/src/main/java/com/kylinolap/cube/CubeSegmentValidator.java
new file mode 100644
index 0000000..c2ceb25
--- /dev/null
+++ b/cube/src/main/java/com/kylinolap/cube/CubeSegmentValidator.java
@@ -0,0 +1,240 @@
+/*
+ * 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.cube;
+
+import java.io.IOException;
+import java.util.Collection;
+import java.util.HashSet;
+import java.util.List;
+
+import com.kylinolap.cube.exception.CubeIntegrityException;
+import com.kylinolap.dict.DictionaryManager;
+import com.kylinolap.metadata.model.cube.CubePartitionDesc.CubePartitionType;
+import com.kylinolap.metadata.model.cube.DimensionDesc;
+import com.kylinolap.metadata.model.cube.TblColRef;
+
+/**
+ * @author xduo
+ */
+public class CubeSegmentValidator {
+
+    private CubeSegmentValidator() {
+    }
+
+    public static CubeSegmentValidator getCubeSegmentValidator(CubeBuildTypeEnum buildType, CubePartitionType partitionType) {
+        switch (buildType) {
+        case MERGE:
+            return new MergeOperationValidator();
+        case BUILD:
+            switch (partitionType) {
+            case APPEND:
+                return new IncrementalBuildOperationValidator();
+            case UPDATE_INSERT:
+                return new UpdateBuildOperationValidator();
+            }
+        default:
+            return new CubeSegmentValidator();
+        }
+    }
+
+    void validate(CubeInstance cubeInstance, List<CubeSegment> newSegments) throws CubeIntegrityException {
+    }
+
+    public static class MergeOperationValidator extends CubeSegmentValidator {
+        private void checkContingency(CubeInstance cubeInstance, List<CubeSegment> newSegments) throws CubeIntegrityException {
+            if (cubeInstance.getSegments().size() < 2) {
+                throw new CubeIntegrityException("No segments to merge.");
+            }
+            if (newSegments.size() != 1) {
+                throw new CubeIntegrityException("Invalid date range.");
+            }
+
+            CubeSegment newSegment = newSegments.get(0);
+            CubeSegment startSeg = null;
+            CubeSegment endSeg = null;
+            for (CubeSegment segment : cubeInstance.getSegments()) {
+                if (segment.getDateRangeStart() == newSegment.getDateRangeStart()) {
+                    startSeg = segment;
+                }
+                if (segment.getDateRangeEnd() == newSegment.getDateRangeEnd()) {
+                    endSeg = segment;
+                }
+            }
+
+            if (null == startSeg || null == endSeg || startSeg.getDateRangeStart() >= endSeg.getDateRangeStart()) {
+                throw new CubeIntegrityException("Invalid date range.");
+            }
+        }
+
+        private void checkLoopTableConsistency(CubeInstance cube, List<CubeSegment> newSegments) throws CubeIntegrityException {
+
+            CubeSegment cubeSeg = newSegments.get(0);
+            DictionaryManager dictMgr = DictionaryManager.getInstance(cube.getConfig());
+            List<CubeSegment> segmentList = cube.getMergingSegments(cubeSeg);
+
+            HashSet<TblColRef> cols = new HashSet<TblColRef>();
+            for (DimensionDesc dim : cube.getDescriptor().getDimensions()) {
+                for (TblColRef col : dim.getColumnRefs()) {
+                    // include those dictionaries that do not need mergning
+                    try {
+                        if (cubeSeg.getCubeDesc().getRowkey().isUseDictionary(col) && !cube.getDescriptor().getFactTable().equalsIgnoreCase((String) dictMgr.decideSourceData(cube.getDescriptor(), col, null)[0])) {
+                            cols.add(col);
+                        }
+                    } catch (IOException e) {
+                        throw new CubeIntegrityException("checkLoopTableConsistency not passed when allocating a new segment.");
+                    }
+                }
+            }
+
+            // check if all dictionaries on lookup table columns are identical
+            for (TblColRef col : cols) {
+                String dictOfFirstSegment = null;
+                for (CubeSegment segment : segmentList) {
+                    String temp = segment.getDictResPath(col);
+                    if (temp == null) {
+                        throw new CubeIntegrityException("Dictionary is null on column: " + col + " Segment: " + segment);
+                    }
+
+                    if (dictOfFirstSegment == null) {
+                        dictOfFirstSegment = temp;
+                    } else {
+                        if (!dictOfFirstSegment.equalsIgnoreCase(temp)) {
+                            throw new CubeIntegrityException("Segments with different dictionaries(on lookup table) cannot be merged");
+                        }
+                    }
+                }
+            }
+
+            // check if all segments' snapshot are identical
+            CubeSegment firstSegment = null;
+            for (CubeSegment segment : segmentList) {
+                if (firstSegment == null) {
+                    firstSegment = segment;
+                } else {
+                    Collection<String> a = firstSegment.getSnapshots().values();
+                    Collection<String> b = segment.getSnapshots().values();
+                    if (!((a.size() == b.size()) && a.containsAll(b)))
+                        throw new CubeIntegrityException("Segments with different snapshots cannot be merged");
+                }
+            }
+
+        }
+
+        @Override
+        public void validate(CubeInstance cubeInstance, List<CubeSegment> newSegments) throws CubeIntegrityException {
+            this.checkContingency(cubeInstance, newSegments);
+            this.checkLoopTableConsistency(cubeInstance, newSegments);
+        }
+    }
+
+    public static class IncrementalBuildOperationValidator extends CubeSegmentValidator {
+        /*
+         * (non-Javadoc)
+         *
+         * @see
+         * com.kylinolap.cube.CubeSegmentValidator#validate(com.kylinolap.cube
+         * .CubeInstance, java.util.List)
+         */
+        @Override
+        void validate(CubeInstance cubeInstance, List<CubeSegment> newSegments) throws CubeIntegrityException {
+            if (newSegments.size() != 1) {
+                throw new CubeIntegrityException("Invalid date range.");
+            }
+            CubeSegment newSegment = newSegments.get(0);
+            if (cubeInstance.needMergeImmediatelyAfterBuild(newSegment)) {
+
+            } else {
+                // check if user will rebuild one specified segment
+                boolean hasMatchSegment = false;
+                for (CubeSegment segment : cubeInstance.getSegments()) {
+                    if (segment.getDateRangeStart() == newSegment.getDateRangeStart()) {
+                        if (segment.getDateRangeEnd() == newSegment.getDateRangeEnd()) {
+                            hasMatchSegment = true;
+                        } else {
+                            throw new CubeIntegrityException("Invalid date range.");
+                        }
+                    }
+                }
+
+                if (!hasMatchSegment) {
+                    if (cubeInstance.getSegments().size() == 0) {
+                        if (cubeInstance.getDescriptor().getCubePartitionDesc().getPartitionDateStart() != newSegment.getDateRangeStart()) {
+                            throw new CubeIntegrityException("Invalid start date.");
+                        }
+                    } else {
+                        CubeSegment lastSegment = cubeInstance.getSegments().get(cubeInstance.getSegments().size() - 1);
+                        if (newSegment.getDateRangeStart() != lastSegment.getDateRangeEnd()) {
+                            throw new CubeIntegrityException("Invalid start date.");
+                        }
+                    }
+                }
+            }
+        }
+
+    }
+
+    public static class UpdateBuildOperationValidator extends CubeSegmentValidator {
+
+        /*
+         * (non-Javadoc)
+         *
+         * @see
+         * com.kylinolap.cube.CubeSegmentValidator#validate(com.kylinolap.cube
+         * .CubeInstance, java.util.List)
+         */
+        @Override
+        void validate(CubeInstance cubeInstance, List<CubeSegment> newSegments) throws CubeIntegrityException {
+            if (newSegments.size() != 1 && newSegments.size() != 2) {
+                throw new CubeIntegrityException("Invalid new segment count, got " + newSegments.size());
+            }
+
+            CubeSegment previousSeg = null;
+            for (CubeSegment newSegment : newSegments) {
+                if (null == previousSeg) {
+                    previousSeg = newSegment;
+                } else {
+                    if (previousSeg.getDateRangeEnd() != newSegment.getDateRangeStart()) {
+                        throw new CubeIntegrityException("Invalid date range.");
+                    }
+                }
+            }
+
+            if (cubeInstance.getSegments().size() == 0) {
+                if (cubeInstance.getDescriptor().getCubePartitionDesc().getPartitionDateStart() != newSegments.get(0).getDateRangeStart()) {
+                    throw new CubeIntegrityException("Invalid start date.");
+                }
+            } else {
+                CubeSegment startSegment = newSegments.get(0);
+                CubeSegment matchSeg = null;
+                for (CubeSegment segment : cubeInstance.getSegments()) {
+                    if (segment.getDateRangeStart() == startSegment.getDateRangeStart()) {
+                        matchSeg = segment;
+                    }
+                }
+
+                if (newSegments.size() == 2 && null == matchSeg) {
+                    throw new CubeIntegrityException("Invalid date range.");
+                }
+
+                if (newSegments.size() == 2 && newSegments.get(newSegments.size() - 1).getDateRangeEnd() < matchSeg.getDateRangeEnd()) {
+                    throw new CubeIntegrityException("Invalid date range.");
+                }
+            }
+        }
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/cube/src/main/java/com/kylinolap/cube/CubeStatusEnum.java
----------------------------------------------------------------------
diff --git a/cube/src/main/java/com/kylinolap/cube/CubeStatusEnum.java b/cube/src/main/java/com/kylinolap/cube/CubeStatusEnum.java
new file mode 100644
index 0000000..16376ae
--- /dev/null
+++ b/cube/src/main/java/com/kylinolap/cube/CubeStatusEnum.java
@@ -0,0 +1,22 @@
+/*
+ * 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.cube;
+
+public enum CubeStatusEnum {
+
+    DISABLED, BUILDING, READY, DESCBROKEN
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/cube/src/main/java/com/kylinolap/cube/cli/DictionaryGeneratorCLI.java
----------------------------------------------------------------------
diff --git a/cube/src/main/java/com/kylinolap/cube/cli/DictionaryGeneratorCLI.java b/cube/src/main/java/com/kylinolap/cube/cli/DictionaryGeneratorCLI.java
new file mode 100644
index 0000000..c3dbe1a
--- /dev/null
+++ b/cube/src/main/java/com/kylinolap/cube/cli/DictionaryGeneratorCLI.java
@@ -0,0 +1,66 @@
+/*
+ * 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.cube.cli;
+
+import java.io.IOException;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.kylinolap.common.KylinConfig;
+import com.kylinolap.cube.CubeInstance;
+import com.kylinolap.cube.CubeManager;
+import com.kylinolap.cube.CubeSegment;
+import com.kylinolap.cube.CubeSegmentStatusEnum;
+import com.kylinolap.metadata.model.cube.DimensionDesc;
+import com.kylinolap.metadata.model.cube.TblColRef;
+
+public class DictionaryGeneratorCLI {
+
+    private static final Logger logger = LoggerFactory.getLogger(DictionaryGeneratorCLI.class);
+
+    public static void processSegment(KylinConfig config, String cubeName, String segmentName, String factColumnsPath) throws IOException {
+        CubeInstance cube = CubeManager.getInstance(config).getCube(cubeName);
+        CubeSegment segment = cube.getSegment(segmentName, CubeSegmentStatusEnum.NEW);
+
+        processSegment(config, segment, factColumnsPath);
+    }
+
+    private static void processSegment(KylinConfig config, CubeSegment cubeSeg, String factColumnsPath) throws IOException {
+        CubeManager cubeMgr = CubeManager.getInstance(config);
+
+        for (DimensionDesc dim : cubeSeg.getCubeDesc().getDimensions()) {
+            // dictionary
+            for (TblColRef col : dim.getColumnRefs()) {
+                if (cubeSeg.getCubeDesc().getRowkey().isUseDictionary(col)) {
+                    logger.info("Building dictionary for " + col);
+                    cubeMgr.buildDictionary(cubeSeg, col, factColumnsPath);
+                }
+            }
+
+            // build snapshot
+            if (dim.getTable() != null && !dim.getTable().equalsIgnoreCase(cubeSeg.getCubeDesc().getFactTable())) {
+                // CubeSegment seg = cube.getTheOnlySegment();
+                logger.info("Building snapshot of " + dim.getTable());
+                cubeMgr.buildSnapshotTable(cubeSeg, dim.getTable());
+                logger.info("Checking snapshot of " + dim.getTable());
+                cubeMgr.getLookupTable(cubeSeg, dim); // load the table for
+                                                      // sanity check
+            }
+        }
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/cube/src/main/java/com/kylinolap/cube/cli/DumpDictionaryCLI.java
----------------------------------------------------------------------
diff --git a/cube/src/main/java/com/kylinolap/cube/cli/DumpDictionaryCLI.java b/cube/src/main/java/com/kylinolap/cube/cli/DumpDictionaryCLI.java
new file mode 100644
index 0000000..cde8c97
--- /dev/null
+++ b/cube/src/main/java/com/kylinolap/cube/cli/DumpDictionaryCLI.java
@@ -0,0 +1,55 @@
+/*
+ * 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.cube.cli;
+
+import java.io.DataInputStream;
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.IOException;
+import java.util.Date;
+
+import com.kylinolap.common.util.JsonUtil;
+import com.kylinolap.dict.DictionaryInfo;
+import com.kylinolap.dict.DictionaryInfoSerializer;
+
+public class DumpDictionaryCLI {
+
+    public static void main(String[] args) throws IOException {
+        for (String path : args) {
+            dump(new File(path));
+        }
+    }
+    
+    public static void dump(File f) throws IOException {
+        if (f.isDirectory()) {
+            for (File c : f.listFiles())
+                dump(c);
+            return;
+        }
+        
+        if (f.getName().endsWith(".dict")) {
+            DictionaryInfoSerializer ser = new DictionaryInfoSerializer();
+            DictionaryInfo dictInfo = ser.deserialize(new DataInputStream(new FileInputStream(f)));
+
+            System.out.println("============================================================================");
+            System.out.println("File: " + f.getAbsolutePath());
+            System.out.println(new Date(dictInfo.getLastModified()));
+            System.out.println(JsonUtil.writeValueAsIndentString(dictInfo));
+            dictInfo.getDictionaryObject().dump(System.out);
+            System.out.println();
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/cube/src/main/java/com/kylinolap/cube/common/BytesSplitter.java
----------------------------------------------------------------------
diff --git a/cube/src/main/java/com/kylinolap/cube/common/BytesSplitter.java b/cube/src/main/java/com/kylinolap/cube/common/BytesSplitter.java
new file mode 100644
index 0000000..fec0db1
--- /dev/null
+++ b/cube/src/main/java/com/kylinolap/cube/common/BytesSplitter.java
@@ -0,0 +1,141 @@
+/*
+ * 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.cube.common;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.io.Text;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * @author xjiang
+ */
+public class BytesSplitter {
+    private static final Logger logger = LoggerFactory.getLogger(BytesSplitter.class);
+
+    private static final int[] COMMON_DELIMS = new int[] { "\177".codePointAt(0), "|".codePointAt(0), "\t".codePointAt(0), ",".codePointAt(0) };
+
+    private SplittedBytes[] splitBuffers;
+    private int bufferSize;
+
+    public SplittedBytes[] getSplitBuffers() {
+        return splitBuffers;
+    }
+
+    public SplittedBytes getSplitBuffer(int index) {
+        return splitBuffers[index];
+    }
+
+    public int getBufferSize() {
+        return bufferSize;
+    }
+
+    public BytesSplitter(int splitLen, int bytesLen) {
+        this.splitBuffers = new SplittedBytes[splitLen];
+        for (int i = 0; i < splitLen; i++) {
+            this.splitBuffers[i] = new SplittedBytes(bytesLen);
+        }
+        this.bufferSize = 0;
+    }
+
+    public int split(byte[] bytes, int byteLen, byte delimiter) {
+        this.bufferSize = 0;
+        int offset = 0;
+        int length = 0;
+        for (int i = 0; i < byteLen; i++) {
+            if (bytes[i] == delimiter) {
+                SplittedBytes split = this.splitBuffers[this.bufferSize++];
+                split.length = length;
+                System.arraycopy(bytes, offset, split.value, 0, length);
+                offset = i + 1;
+                length = 0;
+            } else {
+                length++;
+            }
+        }
+        SplittedBytes split = this.splitBuffers[this.bufferSize++];
+        System.arraycopy(bytes, offset, split.value, 0, length);
+        split.length = length;
+
+        return bufferSize;
+    }
+
+    public static List<String> splitToString(byte[] bytes, int offset, byte delimiter) {
+        List<String> splitStrings = new ArrayList<String>();
+        int splitOffset = 0;
+        int splitLength = 0;
+        for (int i = offset; i < bytes.length; i++) {
+            if (bytes[i] == delimiter) {
+                String str = Bytes.toString(bytes, splitOffset, splitLength);
+                splitStrings.add(str);
+                splitOffset = i + 1;
+                splitLength = 0;
+            } else {
+                splitLength++;
+            }
+        }
+        String str = Bytes.toString(bytes, splitOffset, splitLength);
+        splitStrings.add(str);
+        return splitStrings;
+    }
+
+    public byte inferByteRowDelimiter(byte[] bytes, int byteLen, int expectedSplits) throws IOException {
+
+        if (expectedSplits > this.splitBuffers.length)
+            throw new IOException("expectSplits can not be greater than split buffer size");
+
+        boolean delimiterFound = false;
+        byte foundDelimiter = 0;
+        for (int i = 0; i < bytes.length; ++i) {
+            byte c = bytes[i];
+            if (!Character.isLetterOrDigit((char) c)) {
+                try {
+                    int actualSplits = this.split(bytes, byteLen, c);
+                    if (actualSplits == expectedSplits) {
+                        if (!delimiterFound) {
+                            logger.info("Delimiter found, value is : " + c);
+                            delimiterFound = true;
+                            foundDelimiter = c;
+                        } else if (c != foundDelimiter) {
+                            throw new IOException("Duplicate delimiter found, found delimiter is : " + foundDelimiter + " new delimiter is " + c);
+                        }
+                    }
+                } catch (Exception e) {
+                    logger.info("Unqualified delimiter pruned, value is " + c);
+                }
+            }
+        }
+
+        if (delimiterFound)
+            return foundDelimiter;
+        else
+            throw new IOException("No delimiter found");
+    }
+
+    public int detectDelim(Text value, int expectedParts) {
+        for (int i = 0; i < COMMON_DELIMS.length; i++) {
+            int nParts = split(value.getBytes(), value.getLength(), (byte) COMMON_DELIMS[i]);
+            if (nParts == expectedParts)
+                return COMMON_DELIMS[i];
+        }
+        throw new RuntimeException("Cannot detect delimeter from first line -- " + value.toString() + " -- expect " + expectedParts + " columns");
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/cube/src/main/java/com/kylinolap/cube/common/RowKeySplitter.java
----------------------------------------------------------------------
diff --git a/cube/src/main/java/com/kylinolap/cube/common/RowKeySplitter.java b/cube/src/main/java/com/kylinolap/cube/common/RowKeySplitter.java
new file mode 100644
index 0000000..c049b3a
--- /dev/null
+++ b/cube/src/main/java/com/kylinolap/cube/common/RowKeySplitter.java
@@ -0,0 +1,88 @@
+/*
+ * 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.cube.common;
+
+import org.apache.hadoop.hbase.util.Bytes;
+
+import com.kylinolap.cube.CubeSegment;
+import com.kylinolap.cube.cuboid.Cuboid;
+import com.kylinolap.cube.kv.RowConstants;
+import com.kylinolap.cube.kv.RowKeyColumnIO;
+import com.kylinolap.metadata.model.cube.CubeDesc;
+import com.kylinolap.metadata.model.cube.TblColRef;
+
+/**
+ * @author George Song (ysong1)
+ * 
+ */
+public class RowKeySplitter {
+
+    private CubeDesc cubeDesc;
+    private RowKeyColumnIO colIO;
+
+    private SplittedBytes[] splitBuffers;
+    private int bufferSize;
+
+    public SplittedBytes[] getSplitBuffers() {
+        return splitBuffers;
+    }
+
+    public int getBufferSize() {
+        return bufferSize;
+    }
+
+    public RowKeySplitter(CubeSegment cubeSeg, int splitLen, int bytesLen) {
+        this.cubeDesc = cubeSeg.getCubeDesc();
+        this.colIO = new RowKeyColumnIO(cubeSeg);
+
+        this.splitBuffers = new SplittedBytes[splitLen];
+        for (int i = 0; i < splitLen; i++) {
+            this.splitBuffers[i] = new SplittedBytes(bytesLen);
+        }
+        this.bufferSize = 0;
+    }
+
+    /**
+     * @param bytes
+     * @param byteLen
+     * @return cuboid ID
+     */
+    public long split(byte[] bytes, int byteLen) {
+        this.bufferSize = 0;
+        int offset = 0;
+
+        // extract cuboid id
+        SplittedBytes cuboidIdSplit = this.splitBuffers[this.bufferSize++];
+        cuboidIdSplit.length = RowConstants.ROWKEY_CUBOIDID_LEN;
+        System.arraycopy(bytes, offset, cuboidIdSplit.value, 0, RowConstants.ROWKEY_CUBOIDID_LEN);
+        offset += RowConstants.ROWKEY_CUBOIDID_LEN;
+
+        long cuboidId = Bytes.toLong(cuboidIdSplit.value, 0, cuboidIdSplit.length);
+        Cuboid cuboid = Cuboid.findById(cubeDesc, cuboidId);
+
+        // rowkey columns
+        for (int i = 0; i < cuboid.getColumns().size(); i++) {
+            TblColRef col = cuboid.getColumns().get(i);
+            int colLength = colIO.getColumnLength(col);
+            SplittedBytes split = this.splitBuffers[this.bufferSize++];
+            split.length = colLength;
+            System.arraycopy(bytes, offset, split.value, 0, colLength);
+            offset += colLength;
+        }
+
+        return cuboidId;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/cube/src/main/java/com/kylinolap/cube/common/SplittedBytes.java
----------------------------------------------------------------------
diff --git a/cube/src/main/java/com/kylinolap/cube/common/SplittedBytes.java b/cube/src/main/java/com/kylinolap/cube/common/SplittedBytes.java
new file mode 100644
index 0000000..00a0206
--- /dev/null
+++ b/cube/src/main/java/com/kylinolap/cube/common/SplittedBytes.java
@@ -0,0 +1,30 @@
+/*
+ * 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.cube.common;
+
+/**
+ * @author George Song (ysong1)
+ * 
+ */
+public class SplittedBytes {
+    public SplittedBytes(int length) {
+        value = new byte[length];
+        length = 0;
+    }
+
+    public byte[] value;
+    public int length;
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/cube/src/main/java/com/kylinolap/cube/cuboid/Cuboid.java
----------------------------------------------------------------------
diff --git a/cube/src/main/java/com/kylinolap/cube/cuboid/Cuboid.java b/cube/src/main/java/com/kylinolap/cube/cuboid/Cuboid.java
new file mode 100644
index 0000000..ea673a5
--- /dev/null
+++ b/cube/src/main/java/com/kylinolap/cube/cuboid/Cuboid.java
@@ -0,0 +1,323 @@
+/*
+ * 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.cube.cuboid;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Queue;
+import java.util.concurrent.ConcurrentHashMap;
+
+import org.apache.hadoop.hbase.util.Bytes;
+
+import com.kylinolap.metadata.model.cube.CubeDesc;
+import com.kylinolap.metadata.model.cube.RowKeyColDesc;
+import com.kylinolap.metadata.model.cube.RowKeyDesc;
+import com.kylinolap.metadata.model.cube.RowKeyDesc.AggrGroupMask;
+import com.kylinolap.metadata.model.cube.RowKeyDesc.HierarchyMask;
+import com.kylinolap.metadata.model.cube.TblColRef;
+
+/**
+ * @author George Song (ysong1)
+ */
+public class Cuboid implements Comparable<Cuboid> {
+
+    private final static Map<String, Map<Long, Cuboid>> CUBOID_CACHE = new ConcurrentHashMap<String, Map<Long, Cuboid>>();
+
+    public static Cuboid findById(CubeDesc cube, byte[] cuboidID) {
+        return findById(cube, Bytes.toLong(cuboidID));
+    }
+
+    public static Cuboid findById(CubeDesc cube, long cuboidID) {
+        Map<Long, Cuboid> cubeCache = CUBOID_CACHE.get(cube.getName());
+        if (cubeCache == null) {
+            cubeCache = new ConcurrentHashMap<Long, Cuboid>();
+            CUBOID_CACHE.put(cube.getName(), cubeCache);
+        }
+        Cuboid cuboid = cubeCache.get(cuboidID);
+        if (cuboid == null) {
+            long validCuboidID = translateToValidCuboid(cube, cuboidID);
+            cuboid = new Cuboid(cube, cuboidID, validCuboidID);
+            cubeCache.put(cuboidID, cuboid);
+        }
+        return cuboid;
+    }
+
+    public static boolean isValid(CubeDesc cube, long cuboidID) {
+        RowKeyDesc rowkey = cube.getRowkey();
+
+        if (cuboidID < 0) {
+            throw new IllegalArgumentException("Cuboid " + cuboidID + " should be greater than 0");
+        }
+
+        if (checkBaseCuboid(rowkey, cuboidID)) {
+            return true;
+        }
+
+        if (checkMandatoryColumns(rowkey, cuboidID) == false) {
+            return false;
+        }
+
+        if (checkAggregationGroup(rowkey, cuboidID) == false) {
+            return false;
+        }
+
+        if (checkHierarchy(rowkey, cuboidID) == false) {
+            return false;
+        }
+
+        return true;
+    }
+
+    public static long getBaseCuboidId(CubeDesc cube) {
+        return cube.getRowkey().getFullMask();
+    }
+
+    // Breadth-First-Search
+    private static long translateToValidCuboid(CubeDesc cube, long cuboidID) {
+        if (Cuboid.isValid(cube, cuboidID)) {
+            return cuboidID;
+        }
+
+        HashSet<Long> dedupped = new HashSet<Long>();
+        Queue<Long> queue = new LinkedList<Long>();
+        List<Long> parents = Cuboid.getAllPossibleParents(cube, cuboidID);
+
+        // check each parent
+        addToQueue(queue, parents, dedupped);
+        while (queue.size() > 0) {
+            long parent = pollFromQueue(queue, dedupped);
+            if (Cuboid.isValid(cube, parent)) {
+                return parent;
+            } else {
+                addToQueue(queue, Cuboid.getAllPossibleParents(cube, parent), dedupped);
+            }
+        }
+        return -1;
+    }
+
+    private static List<Long> getAllPossibleParents(CubeDesc cube, long cuboidID) {
+        List<Long> allPossibleParents = new ArrayList<Long>();
+
+        for (int i = 0; i < cube.getRowkey().getRowKeyColumns().length; i++) {
+            long mask = 1L << i;
+            long parentId = cuboidID | mask;
+            if (parentId != cuboidID) {
+                allPossibleParents.add(parentId);
+            }
+        }
+
+        return allPossibleParents;
+    }
+
+    private static void addToQueue(Queue<Long> queue, List<Long> parents, HashSet<Long> dedupped) {
+        Collections.sort(parents);
+        for (Long p : parents) {
+            if (!dedupped.contains(p)) {
+                dedupped.add(p);
+                queue.offer(p);
+            }
+        }
+    }
+
+    private static long pollFromQueue(Queue<Long> queue, HashSet<Long> dedupped) {
+        long element = queue.poll();
+        dedupped.remove(element);
+        return element;
+    }
+
+    private static boolean checkBaseCuboid(RowKeyDesc rowkey, long cuboidID) {
+        long baseCuboidId = rowkey.getFullMask();
+        if (cuboidID > baseCuboidId) {
+            throw new IllegalArgumentException("Cubiod " + cuboidID + " is out of scope 0-" + baseCuboidId);
+        }
+        return baseCuboidId == cuboidID;
+    }
+
+    private static boolean checkMandatoryColumns(RowKeyDesc rowkey, long cuboidID) {
+        long mandatoryColumnMask = rowkey.getMandatoryColumnMask();
+
+        // note the all-zero cuboid (except for mandatory) is not valid
+        if (cuboidID <= mandatoryColumnMask)
+            return false;
+
+        return (cuboidID & mandatoryColumnMask) == mandatoryColumnMask;
+    }
+
+    private static boolean checkHierarchy(RowKeyDesc rowkey, long cuboidID) {
+        List<HierarchyMask> hierarchyMaskList = rowkey.getHierarchyMasks();
+        // if no hierarchy defined in metadata
+        if (hierarchyMaskList == null || hierarchyMaskList.size() == 0) {
+            return true;
+        }
+
+        hier: for (HierarchyMask hierarchyMasks : hierarchyMaskList) {
+            long result = cuboidID & hierarchyMasks.fullMask;
+            if (result > 0) {
+                // if match one of the hierarchy constrains, return true;
+                for (long mask : hierarchyMasks.allMasks) {
+                    if (result == mask) {
+                        continue hier;
+                    }
+                }
+                return false;
+            }
+        }
+        return true;
+    }
+
+    private static boolean checkAggregationGroup(RowKeyDesc rowkey, long cuboidID) {
+        long cuboidWithoutMandatory = cuboidID & ~rowkey.getMandatoryColumnMask();
+        long leftover;
+        for (AggrGroupMask mask : rowkey.getAggrGroupMasks()) {
+            if ((cuboidWithoutMandatory & mask.uniqueMask) != 0) {
+                leftover = cuboidWithoutMandatory & ~mask.groupMask;
+                return leftover == 0 || leftover == mask.leftoverMask;
+            }
+        }
+
+        leftover = cuboidWithoutMandatory & rowkey.getTailMask();
+        return leftover == 0 || leftover == rowkey.getTailMask();
+    }
+
+    private CubeDesc cube;
+    private final long inputID;
+    private final long id;
+    private final byte[] idBytes;
+    private final boolean requirePostAggregation;
+    private List<TblColRef> dimensionColumns;
+
+    // will translate the cuboidID if it is not valid
+    private Cuboid(CubeDesc cube, long originalID, long validID) {
+        this.cube = cube;
+        this.inputID = originalID;
+        this.id = validID;
+        this.idBytes = Bytes.toBytes(id);
+        this.dimensionColumns = translateIdToColumns(this.id);
+        this.requirePostAggregation = calcExtraAggregation(this.inputID, this.id) != 0;
+    }
+
+    private List<TblColRef> translateIdToColumns(long cuboidID) {
+        List<TblColRef> dimesnions = new ArrayList<TblColRef>();
+        RowKeyColDesc[] allColumns = cube.getRowkey().getRowKeyColumns();
+        for (int i = 0; i < allColumns.length; i++) {
+            // NOTE: the order of column in list!!!
+            long bitmask = 1L << allColumns[i].getBitIndex();
+            if ((cuboidID & bitmask) != 0) {
+                TblColRef colRef = allColumns[i].getColRef();
+                dimesnions.add(colRef);
+            }
+        }
+        return dimesnions;
+    }
+
+    private long calcExtraAggregation(long inputID, long id) {
+        long diff = id ^ inputID;
+        return eliminateHierarchyAggregation(diff);
+    }
+    
+    // higher level in hierarchy can be ignored when counting aggregation columns
+    private long eliminateHierarchyAggregation(long id) {
+        List<HierarchyMask> hierarchyMaskList = cube.getRowkey().getHierarchyMasks();
+        if (hierarchyMaskList != null && hierarchyMaskList.size() > 0) {
+            for (HierarchyMask hierMask : hierarchyMaskList) {
+                long[] allMasks = hierMask.allMasks;
+                for (int i = allMasks.length - 1; i > 0; i--) {
+                    long bit = allMasks[i] ^ allMasks[i - 1];
+                    if ((inputID & bit) != 0) {
+                        id &= ~allMasks[i - 1];
+                    }
+                }
+            }
+        }
+        return id;
+    }
+    
+    public CubeDesc getCube() {
+        return cube;
+    }
+
+    public List<TblColRef> getColumns() {
+        return dimensionColumns;
+    }
+    
+    public List<TblColRef> getAggregationColumns() {
+        long aggrColsID = eliminateHierarchyAggregation(id);
+        return translateIdToColumns(aggrColsID);
+    }
+
+    public long getId() {
+        return id;
+    }
+
+    public byte[] getBytes() {
+        return idBytes;
+    }
+
+    public long getInputID() {
+        return inputID;
+    }
+
+    public boolean useAncestor() {
+        return inputID != id;
+    }
+
+    public boolean requirePostAggregation() {
+        return requirePostAggregation;
+    }
+
+    @Override
+    public String toString() {
+        return "Cuboid [id=" + id + "]";
+    }
+
+    @Override
+    public int hashCode() {
+        final int prime = 31;
+        int result = 1;
+        result = prime * result + (int) (id ^ (id >>> 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;
+        Cuboid other = (Cuboid) obj;
+        if (id != other.id)
+            return false;
+        return true;
+    }
+
+    @Override
+    public int compareTo(Cuboid o) {
+        if (this.id < o.id) {
+            return -1;
+        } else if (this.id > o.id) {
+            return 1;
+        } else {
+            return 0;
+        }
+    }
+
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/cube/src/main/java/com/kylinolap/cube/cuboid/CuboidCLI.java
----------------------------------------------------------------------
diff --git a/cube/src/main/java/com/kylinolap/cube/cuboid/CuboidCLI.java b/cube/src/main/java/com/kylinolap/cube/cuboid/CuboidCLI.java
new file mode 100644
index 0000000..95759c8
--- /dev/null
+++ b/cube/src/main/java/com/kylinolap/cube/cuboid/CuboidCLI.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.cube.cuboid;
+
+import java.io.IOException;
+import java.util.Collection;
+import java.util.LinkedList;
+import java.util.TreeSet;
+
+import com.kylinolap.common.KylinConfig;
+import com.kylinolap.metadata.MetadataManager;
+import com.kylinolap.metadata.model.cube.CubeDesc;
+import com.kylinolap.metadata.model.cube.RowKeyDesc;
+import com.kylinolap.metadata.model.cube.RowKeyDesc.AggrGroupMask;
+import com.kylinolap.metadata.model.cube.RowKeyDesc.HierarchyMask;
+
+/**
+ * @author yangli9
+ * 
+ */
+public class CuboidCLI {
+
+    public static void main(String[] args) throws IOException {
+        MetadataManager metaMgr = MetadataManager.getInstance(KylinConfig.getInstanceFromEnv());
+
+        if ("test".equals(args[0])) {
+            CubeDesc cubeDesc = metaMgr.getCubeDesc(args[1]);
+            simulateCuboidGeneration(cubeDesc);
+        }
+    }
+
+    public static int simulateCuboidGeneration(CubeDesc cube) {
+        CuboidScheduler scheduler = new CuboidScheduler(cube);
+
+        long baseCuboid = Cuboid.getBaseCuboidId(cube);
+        Collection<Long> cuboidSet = new TreeSet<Long>();
+        cuboidSet.add(baseCuboid);
+        LinkedList<Long> cuboidQueue = new LinkedList<Long>();
+        cuboidQueue.push(baseCuboid);
+        while (!cuboidQueue.isEmpty()) {
+            long cuboid = cuboidQueue.pop();
+            Collection<Long> spnanningCuboids = scheduler.getSpanningCuboid(cuboid);
+            for (Long sc : spnanningCuboids) {
+                boolean notfound = cuboidSet.add(sc);
+                if (!notfound) {
+                    throw new IllegalStateException("Find duplicate spanning cuboid " + sc + " from cuboid " + cuboid);
+                }
+                cuboidQueue.push(sc);
+            }
+        }
+
+        TreeSet<Long> enumCuboids = enumCalcCuboidCount(cube);
+        if (enumCuboids.equals(cuboidSet) == false) {
+            throw new IllegalStateException("Expected cuboid set " + enumCuboids + "; but actual cuboid set " + cuboidSet);
+        }
+
+        int mathCount = mathCalcCuboidCount(cube);
+        if (mathCount != enumCuboids.size()) {
+            throw new IllegalStateException("Math cuboid count " + mathCount + ", but actual cuboid count " + enumCuboids.size());
+        }
+
+        return mathCount;
+
+    }
+
+    public static TreeSet<Long> enumCalcCuboidCount(CubeDesc cube) {
+        long baseCuboid = Cuboid.getBaseCuboidId(cube);
+        TreeSet<Long> expectedCuboids = new TreeSet<Long>();
+        for (long cuboid = 0; cuboid <= baseCuboid; cuboid++) {
+            if (Cuboid.isValid(cube, cuboid)) {
+                expectedCuboids.add(cuboid);
+            }
+        }
+        return expectedCuboids;
+    }
+
+    public static int[] calculateAllLevelCount(CubeDesc cube) {
+        int levels = cube.getRowkey().getNCuboidBuildLevels();
+        int[] allLevelCounts = new int[levels + 1];
+
+        CuboidScheduler scheduler = new CuboidScheduler(cube);
+        LinkedList<Long> nextQueue = new LinkedList<Long>();
+        LinkedList<Long> currentQueue = new LinkedList<Long>();
+        long baseCuboid = Cuboid.getBaseCuboidId(cube);
+        currentQueue.push(baseCuboid);
+
+        for (int i = 0; i <= levels; i++) {
+            allLevelCounts[i] = currentQueue.size();
+            while (!currentQueue.isEmpty()) {
+                long cuboid = currentQueue.pop();
+                Collection<Long> spnanningCuboids = scheduler.getSpanningCuboid(cuboid);
+                nextQueue.addAll(spnanningCuboids);
+            }
+            currentQueue = nextQueue;
+            nextQueue = new LinkedList<Long>();
+        }
+
+        return allLevelCounts;
+    }
+
+    public static int mathCalcCuboidCount(CubeDesc cube) {
+        int result = 1; // 1 for base cuboid
+
+        RowKeyDesc rowkey = cube.getRowkey();
+        AggrGroupMask[] aggrGroupMasks = rowkey.getAggrGroupMasks();
+        for (int i = 0; i < aggrGroupMasks.length; i++) {
+            boolean hasTail = i < aggrGroupMasks.length - 1 || rowkey.getTailMask() > 0;
+            result += mathCalcCuboidCount_aggrGroup(rowkey, aggrGroupMasks[i], hasTail);
+        }
+
+        return result;
+    }
+
+    private static int mathCalcCuboidCount_aggrGroup(RowKeyDesc rowkey, AggrGroupMask aggrGroupMask, boolean hasTail) {
+        long groupMask = aggrGroupMask.groupMask;
+        int n = mathCalcCuboidCount_combination(rowkey, groupMask);
+        n -= 2; // exclude group all 1 and all 0
+        
+        long nonUniqueMask = groupMask & (~aggrGroupMask.uniqueMask);
+        if (nonUniqueMask > 0) {
+            // exclude duplicates caused by non-unique columns
+            // FIXME this assumes non-unique masks consolidates in ONE following group which maybe not be true
+            n -= mathCalcCuboidCount_combination(rowkey, nonUniqueMask) - 1; // exclude all 0
+        }
+        
+        if (hasTail) {
+            n *= 2; // tail being 1 and 0
+            n += 2; // +1 for group all 1 and tail 0; +1 for group all 0 and tail 1
+        }
+
+        return n;
+    }
+
+    private static int mathCalcCuboidCount_combination(RowKeyDesc rowkey, long colMask) {
+        if (colMask == 0) // no column selected
+            return 0;
+
+        int count = 1;
+
+        for (HierarchyMask hierMask : rowkey.getHierarchyMasks()) {
+            long hierBits = colMask & hierMask.fullMask;
+            if (hierBits != 0) {
+                count *= Long.bitCount(hierBits) + 1; // +1 is for all-zero case
+                colMask &= ~hierBits;
+            }
+        }
+
+        count *= Math.pow(2, Long.bitCount(colMask));
+
+        return count;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/cube/src/main/java/com/kylinolap/cube/cuboid/CuboidScheduler.java
----------------------------------------------------------------------
diff --git a/cube/src/main/java/com/kylinolap/cube/cuboid/CuboidScheduler.java b/cube/src/main/java/com/kylinolap/cube/cuboid/CuboidScheduler.java
new file mode 100644
index 0000000..9f77770
--- /dev/null
+++ b/cube/src/main/java/com/kylinolap/cube/cuboid/CuboidScheduler.java
@@ -0,0 +1,190 @@
+/*
+ * 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.cube.cuboid;
+
+/** 
+ * @author George Song (ysong1)
+ * 
+ */
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+
+import com.kylinolap.metadata.model.cube.CubeDesc;
+import com.kylinolap.metadata.model.cube.RowKeyDesc;
+import com.kylinolap.metadata.model.cube.RowKeyDesc.AggrGroupMask;
+
+public class CuboidScheduler {
+
+    private final CubeDesc cubeDef;
+    private final int size;
+    private final long max;
+    private final Map<Long, Collection<Long>> cache;
+
+    public CuboidScheduler(CubeDesc cube) {
+        this.cubeDef = cube;
+        this.size = cube.getRowkey().getRowKeyColumns().length;
+        this.max = (long) Math.pow(2, size) - 1;
+        this.cache = new ConcurrentHashMap<Long, Collection<Long>>();
+    }
+
+    public Collection<Long> getSpanningCuboid(long cuboid) {
+        if (cuboid > max || cuboid < 0) {
+            throw new IllegalArgumentException("Cuboid " + cuboid + " is out of scope 0-" + max);
+        }
+
+        Collection<Long> result = cache.get(cuboid);
+        if (result != null) {
+            return result;
+        }
+
+        // smaller sibling's children
+        Collection<Long> allPrevOffspring = new HashSet<Long>();
+        for (Long sibling : findSmallerSibling(cuboid)) {
+            Collection<Long> prevOffsprings = generateChildren(sibling);
+            allPrevOffspring.addAll(prevOffsprings);
+        }
+
+        // my children is my generation excluding smaller sibling's generation
+        result = new ArrayList<Long>();
+        for (Long offspring : generateChildren(cuboid)) {
+            if (!allPrevOffspring.contains(offspring)) {
+                result.add(offspring);
+            }
+        }
+
+        cache.put(cuboid, result);
+        return result;
+    }
+
+    private Collection<Long> generateChildren(long cuboid) {
+        Collection<Long> result = new HashSet<Long>();
+
+        // generate zero tail cuboid -- the one with all 1 in the first
+        // aggregation group and all 0 for the rest bits
+        generateZeroTailBase(cuboid, result);
+
+        RowKeyDesc rowkey = cubeDef.getRowkey();
+        long cuboidWithoutMandatory = cuboid & ~rowkey.getMandatoryColumnMask();
+        for (AggrGroupMask mask : rowkey.getAggrGroupMasks()) {
+            if (belongTo(cuboidWithoutMandatory, mask) == false)
+                continue;
+
+            long[] groupOneBitMasks = mask.groupOneBitMasks;
+            for (int i = 0; i < groupOneBitMasks.length; i++) {
+                long oneBit = groupOneBitMasks[i];
+                if ((cuboid & oneBit) == 0)
+                    continue;
+
+                long child = cuboid ^ oneBit;
+                if (Cuboid.isValid(cubeDef, child)) {
+                    result.add(child);
+                }
+            }
+
+            if ((cuboidWithoutMandatory & mask.uniqueMask) > 0)
+                break;
+        }
+
+        return result;
+    }
+
+    private void generateZeroTailBase(long cuboid, Collection<Long> result) {
+        RowKeyDesc rowkey = cubeDef.getRowkey();
+
+        long cuboidWithoutMandatory = cuboid & ~rowkey.getMandatoryColumnMask();
+
+        for (AggrGroupMask mask : rowkey.getAggrGroupMasks()) {
+            if ((cuboidWithoutMandatory & mask.groupMask) == mask.groupMask && (cuboidWithoutMandatory & mask.leftoverMask) == mask.leftoverMask) {
+                long zeroTail = rowkey.getMandatoryColumnMask() | mask.groupMask;
+                if (zeroTail > 0 && zeroTail != cuboid) {
+                    result.add(zeroTail);
+                }
+            }
+            if ((cuboidWithoutMandatory & mask.uniqueMask) > 0)
+                break;
+        }
+    }
+
+    public Collection<Long> findSmallerSibling(long cuboid) {
+        if (!Cuboid.isValid(cubeDef, cuboid)) {
+            return Collections.emptyList();
+        }
+
+        RowKeyDesc rowkey = cubeDef.getRowkey();
+
+        // do combination in all related groups
+        long groupAllBitMask = 0;
+        for (AggrGroupMask mask : rowkey.getAggrGroupMasks()) {
+            if ((mask.groupMask & cuboid) > 0) {
+                groupAllBitMask |= mask.groupMask;
+            }
+        }
+
+        long groupBitValue = cuboid & groupAllBitMask;
+        long leftBitValue = cuboid & ~groupAllBitMask;
+        long[] groupOneBits = bits(groupAllBitMask);
+
+        Collection<Long> siblings = new HashSet<Long>();
+        combination(cuboid, siblings, groupOneBits, 0, leftBitValue, Long.bitCount(groupBitValue));
+        return siblings;
+    }
+
+    private long[] bits(long groupAllBitMask) {
+        int size = Long.bitCount(groupAllBitMask);
+        long[] r = new long[size];
+        long l = groupAllBitMask;
+        int i = 0;
+        while (l != 0) {
+            long bit = Long.highestOneBit(l);
+            r[i++] = bit;
+            l ^= bit;
+        }
+        return r;
+    }
+
+    private void combination(long cuboid, Collection<Long> siblings, long[] bitMasks, int offset, long bitValue, int k) {
+        if (k == 0) {
+            if (Cuboid.isValid(cubeDef, bitValue)) {
+                siblings.add(bitValue);
+            }
+        } else {
+            for (int i = offset; i < bitMasks.length; i++) {
+                long newBitValue = bitValue | bitMasks[i];
+                if (newBitValue < cuboid) {
+                    combination(cuboid, siblings, bitMasks, i + 1, newBitValue, k - 1);
+                }
+            }
+        }
+    }
+
+    private boolean belongTo(long cuboidWithoutMandatory, AggrGroupMask mask) {
+        long groupBits = cuboidWithoutMandatory & mask.groupMask;
+        long leftoverBits = cuboidWithoutMandatory & mask.leftoverMask;
+        return groupBits > 0 && (leftoverBits == 0 || leftoverBits == mask.leftoverMask);
+    }
+
+    public int getCardinality(long cuboid) {
+        if (cuboid > max || cuboid < 0) {
+            throw new IllegalArgumentException("Cubiod " + cuboid + " is out of scope 0-" + max);
+        }
+
+        return Long.bitCount(cuboid);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/cube/src/main/java/com/kylinolap/cube/dataGen/ColumnConfig.java
----------------------------------------------------------------------
diff --git a/cube/src/main/java/com/kylinolap/cube/dataGen/ColumnConfig.java b/cube/src/main/java/com/kylinolap/cube/dataGen/ColumnConfig.java
new file mode 100644
index 0000000..3ee1cf0
--- /dev/null
+++ b/cube/src/main/java/com/kylinolap/cube/dataGen/ColumnConfig.java
@@ -0,0 +1,54 @@
+package com.kylinolap.cube.dataGen;
+
+import java.util.ArrayList;
+
+import com.fasterxml.jackson.annotation.JsonAutoDetect;
+import com.fasterxml.jackson.annotation.JsonProperty;
+
+/**
+ * Created by honma on 5/29/14.
+ */
+@JsonAutoDetect(fieldVisibility = JsonAutoDetect.Visibility.NONE, getterVisibility = JsonAutoDetect.Visibility.NONE, isGetterVisibility = JsonAutoDetect.Visibility.NONE, setterVisibility = JsonAutoDetect.Visibility.NONE)
+public class ColumnConfig {
+    @JsonProperty("columnName")
+    private String columnName;
+    @JsonProperty("valueSet")
+    private ArrayList<String> valueSet;
+    @JsonProperty("exclusive")
+    private boolean exclusive;
+    @JsonProperty("asRange")
+    private boolean asRange;
+
+    public boolean isAsRange() {
+        return asRange;
+    }
+
+    public void setAsRange(boolean asRange) {
+        this.asRange = asRange;
+    }
+
+    public boolean isExclusive() {
+        return exclusive;
+    }
+
+    public void setExclusive(boolean exclusive) {
+        this.exclusive = exclusive;
+    }
+
+    public String getColumnName() {
+        return columnName;
+    }
+
+    public void setColumnName(String columnName) {
+        this.columnName = columnName;
+    }
+
+    public ArrayList<String> getValueSet() {
+        return valueSet;
+    }
+
+    public void setValueSet(ArrayList<String> valueSet) {
+        this.valueSet = valueSet;
+    }
+
+}


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

Posted by lu...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/job/src/main/java/com/kylinolap/job/hadoop/cardinality/ColumnCardinalityReducer.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/com/kylinolap/job/hadoop/cardinality/ColumnCardinalityReducer.java b/job/src/main/java/com/kylinolap/job/hadoop/cardinality/ColumnCardinalityReducer.java
new file mode 100644
index 0000000..06c98ee
--- /dev/null
+++ b/job/src/main/java/com/kylinolap/job/hadoop/cardinality/ColumnCardinalityReducer.java
@@ -0,0 +1,86 @@
+/*
+ * 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.job.hadoop.cardinality;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.hadoop.io.BytesWritable;
+import org.apache.hadoop.io.IntWritable;
+import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.mapreduce.Reducer;
+
+import com.kylinolap.common.hll.HyperLogLogPlusCounter;
+import com.kylinolap.cube.kv.RowConstants;
+
+/**
+ * @author Jack
+ * 
+ */
+public class ColumnCardinalityReducer extends Reducer<IntWritable, BytesWritable, IntWritable, LongWritable> {
+
+    public static final int ONE = 1;
+    private Map<Integer, HyperLogLogPlusCounter> hllcMap = new HashMap<Integer, HyperLogLogPlusCounter>();
+
+    @Override
+    public void reduce(IntWritable key, Iterable<BytesWritable> values, Context context) throws IOException, InterruptedException {
+        for (BytesWritable v : values) {
+            int skey = key.get();
+            ByteBuffer buffer = ByteBuffer.wrap(v.getBytes());
+            HyperLogLogPlusCounter hll = new HyperLogLogPlusCounter();
+            hll.readRegisters(buffer);
+            getHllc(skey).merge(hll);
+            hll.clear();
+        }
+    }
+
+    private HyperLogLogPlusCounter getHllc(Integer key) {
+        if (!hllcMap.containsKey(key)) {
+            hllcMap.put(key, new HyperLogLogPlusCounter());
+        }
+        return hllcMap.get(key);
+    }
+
+    @Override
+    protected void cleanup(Context context) throws IOException, InterruptedException {
+        List<Integer> keys = new ArrayList<Integer>();
+        Iterator<Integer> it = hllcMap.keySet().iterator();
+        while (it.hasNext()) {
+            keys.add(it.next());
+        }
+        Collections.sort(keys);
+        it = keys.iterator();
+        while (it.hasNext()) {
+            int key = it.next();
+            HyperLogLogPlusCounter hllc = hllcMap.get(key);
+            ByteBuffer buf = ByteBuffer.allocate(RowConstants.ROWVALUE_BUFFER_SIZE);
+            buf.clear();
+            hllc.writeRegisters(buf);
+            buf.flip();
+            context.write(new IntWritable(key), new LongWritable(hllc.getCountEstimate()));
+            // context.write(new Text("ErrorRate_" + key), new
+            // LongWritable((long)hllc.getErrorRate()));
+        }
+
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/job/src/main/java/com/kylinolap/job/hadoop/cardinality/HiveColumnCardinalityJob.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/com/kylinolap/job/hadoop/cardinality/HiveColumnCardinalityJob.java b/job/src/main/java/com/kylinolap/job/hadoop/cardinality/HiveColumnCardinalityJob.java
new file mode 100644
index 0000000..b6ea002
--- /dev/null
+++ b/job/src/main/java/com/kylinolap/job/hadoop/cardinality/HiveColumnCardinalityJob.java
@@ -0,0 +1,254 @@
+package com.kylinolap.job.hadoop.cardinality;
+
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.StringWriter;
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.commons.cli.Option;
+import org.apache.commons.cli.OptionBuilder;
+import org.apache.commons.cli.Options;
+import org.apache.commons.io.IOUtils;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.io.BytesWritable;
+import org.apache.hadoop.io.IntWritable;
+import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.io.compress.CompressionCodec;
+import org.apache.hadoop.io.compress.CompressionCodecFactory;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.mapreduce.Job;
+import org.apache.hadoop.mapreduce.MRJobConfig;
+import org.apache.hadoop.mapreduce.lib.input.FileInputFormat;
+import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
+import org.apache.hadoop.mapreduce.lib.output.TextOutputFormat;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.util.ToolRunner;
+
+import com.kylinolap.job.hadoop.AbstractHadoopJob;
+
+public class HiveColumnCardinalityJob extends AbstractHadoopJob {
+    public static final String JOB_TITLE = "Kylin Hive Column Cardinality Job";
+
+    @SuppressWarnings("static-access")
+    protected static final Option OPTION_FORMAT = OptionBuilder.withArgName("input format").hasArg().isRequired(true).withDescription("The file format").create("iformat");
+
+    @SuppressWarnings("static-access")
+    protected static final Option OPTION_INPUT_DELIM = OptionBuilder.withArgName("input_dilim").hasArg().isRequired(false).withDescription("Input delim").create("idelim");
+
+    public static final String KEY_INPUT_DELIM = "INPUT_DELIM";
+    public static final String OUTPUT_PATH = "/tmp/cardinality";
+
+    /**
+     * This is the jar path
+     */
+    private String jarPath;
+    private Configuration conf;
+
+    /**
+     * MRJobConfig.MAPREDUCE_JOB_CREDENTIALS_BINARY
+     */
+    private String tokenPath;
+
+    public HiveColumnCardinalityJob() {
+
+    }
+
+    public HiveColumnCardinalityJob(String path, String tokenPath) {
+        this.jarPath = path;
+        this.tokenPath = tokenPath;
+    }
+
+    /*
+     * (non-Javadoc)
+     * 
+     * @see org.apache.hadoop.conf.Configured#getConf()
+     */
+    @Override
+    public Configuration getConf() {
+        if (conf != null) {
+            return conf;
+        }
+        conf = new JobConf();
+        String path = "/apache/hadoop/conf/";
+        File file = new File(path);
+        if (file.isDirectory()) {
+            File[] files = file.listFiles();
+            for (int i = 0; i < files.length; i++) {
+                File tmp = files[i];
+                if (tmp.getName().endsWith(".xml")) {
+                    try {
+                        conf.addResource(new FileInputStream(tmp));
+                    } catch (FileNotFoundException e) {
+                        e.printStackTrace();
+                    }
+                }
+            }
+        }
+
+        // conf.addResource("/apache/hadoop/conf/mapred-site.xml");
+        if (tokenPath != null) {
+            conf.set(MRJobConfig.MAPREDUCE_JOB_CREDENTIALS_BINARY, tokenPath);
+            conf.set("hadoop.security.authentication", "kerberos");
+            UserGroupInformation.setConfiguration(conf);
+            try {
+                UserGroupInformation.loginUserFromKeytab("b_kylin@CORP.EBAY.COM", "~/.keytabs/b_kylin.keytab");
+                System.out.println("###" + UserGroupInformation.getLoginUser());
+            } catch (IOException e) {
+                e.printStackTrace();
+            }
+        }
+        return conf;
+    }
+
+    @SuppressWarnings("unchecked")
+    @Override
+    public int run(String[] args) throws Exception {
+
+        Options options = new Options();
+
+        try {
+            options.addOption(OPTION_INPUT_PATH);
+            options.addOption(OPTION_OUTPUT_PATH);
+            options.addOption(OPTION_FORMAT);
+            options.addOption(OPTION_INPUT_DELIM);
+
+            parseOptions(options, args);
+
+            // start job
+            String jobName = JOB_TITLE + getOptionsAsString();
+            System.out.println("Starting: " + jobName);
+            Configuration conf = getConf();
+            job = Job.getInstance(conf, jobName);
+
+            // set job configuration - basic
+            if (jarPath == null || !new File(jarPath).exists()) {
+                job.setJarByClass(getClass());
+            } else {
+                job.setJar(jarPath);
+            }
+            FileInputFormat.setInputDirRecursive(job, true);
+            addInputDirs(getOptionValue(OPTION_INPUT_PATH), job);
+
+            Path output = new Path(getOptionValue(OPTION_OUTPUT_PATH));
+            FileOutputFormat.setOutputPath(job, output);
+            job.getConfiguration().set("dfs.block.size", "67108864");
+
+            String format = getOptionValue(OPTION_FORMAT);
+            @SuppressWarnings("rawtypes")
+            Class cformat = getFormat(format);
+            String delim = getOptionValue(OPTION_INPUT_DELIM);
+            if (delim != null) {
+                if (delim.equals("t")) {
+                    delim = "\t";
+                } else if (delim.equals("177")) {
+                    delim = "\177";
+                }
+                job.getConfiguration().set(KEY_INPUT_DELIM, delim);
+            }
+
+            // Mapper
+            job.setInputFormatClass(cformat);
+            job.setMapperClass(ColumnCardinalityMapper.class);
+            job.setMapOutputKeyClass(IntWritable.class);
+            job.setMapOutputValueClass(BytesWritable.class);
+
+            // Reducer - only one
+            job.setReducerClass(ColumnCardinalityReducer.class);
+            job.setOutputFormatClass(TextOutputFormat.class);
+            job.setOutputKeyClass(IntWritable.class);
+            job.setOutputValueClass(LongWritable.class);
+            job.setNumReduceTasks(1);
+
+            this.deletePath(job.getConfiguration(), output);
+
+            int result = waitForCompletion(job);
+            return result;
+        } catch (Exception e) {
+            printUsage(options);
+            e.printStackTrace(System.err);
+            log.error(e.getLocalizedMessage(), e);
+            return 2;
+        }
+
+    }
+
+    /**
+     * @param format
+     * @throws ClassNotFoundException
+     */
+    @SuppressWarnings("rawtypes")
+    private Class getFormat(String format) throws ClassNotFoundException {
+        if (format.endsWith(".TextInputFormat")) {
+            return Class.forName("org.apache.hadoop.mapreduce.lib.input.TextInputFormat");
+        } else if (format.endsWith(".SequenceFileInputFormat")) {
+            return Class.forName("org.apache.hadoop.mapreduce.lib.input.SequenceFileAsTextInputFormat");
+        } else {
+            return Class.forName(format);
+        }
+
+    }
+
+    public static void main1(String[] args) throws Exception {
+        int exitCode = ToolRunner.run(new HiveColumnCardinalityJob(), args);
+        System.exit(exitCode);
+    }
+
+    public static void main(String[] args) {
+
+        String location = "hdfs://apollo-phx-nn.vip.ebay.com:8020/tmp/f1a98d8a-26b9-452e-ab7b-9f01e5a6459b/shipping_sisense_cube_desc_intermediate_table";
+        String tempName = "test";
+        String inputFormat = "org.apache.hadoop.mapred.SequenceFileInputFormat";
+        String delim = "177";
+        String jarPath = "/usr/lib/kylin/kylin-index-latest.jar";
+
+        args = new String[] { "-input", location, "-output", "/tmp/cardinality/" + tempName, "-iformat", inputFormat, "-idelim", delim };
+        HiveColumnCardinalityJob job = new HiveColumnCardinalityJob(jarPath, "/tmp/krb5cc_882");
+        try {
+            ToolRunner.run(job, args);
+        } catch (Exception e) {
+            e.printStackTrace();
+        }
+    }
+
+    public List<String> readLines(Path location, Configuration conf) throws Exception {
+        FileSystem fileSystem = FileSystem.get(location.toUri(), conf);
+        CompressionCodecFactory factory = new CompressionCodecFactory(conf);
+        FileStatus[] items = fileSystem.listStatus(location);
+        if (items == null)
+            return new ArrayList<String>();
+        List<String> results = new ArrayList<String>();
+        for (FileStatus item : items) {
+
+            // ignoring files like _SUCCESS
+            if (item.getPath().getName().startsWith("_")) {
+                continue;
+            }
+
+            CompressionCodec codec = factory.getCodec(item.getPath());
+            InputStream stream = null;
+
+            // check if we have a compression codec we need to use
+            if (codec != null) {
+                stream = codec.createInputStream(fileSystem.open(item.getPath()));
+            } else {
+                stream = fileSystem.open(item.getPath());
+            }
+
+            StringWriter writer = new StringWriter();
+            IOUtils.copy(stream, writer, "UTF-8");
+            String raw = writer.toString();
+            for (String str : raw.split("\n")) {
+                results.add(str);
+            }
+        }
+        return results;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/job/src/main/java/com/kylinolap/job/hadoop/cube/BaseCuboidJob.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/com/kylinolap/job/hadoop/cube/BaseCuboidJob.java b/job/src/main/java/com/kylinolap/job/hadoop/cube/BaseCuboidJob.java
new file mode 100644
index 0000000..cc25b13
--- /dev/null
+++ b/job/src/main/java/com/kylinolap/job/hadoop/cube/BaseCuboidJob.java
@@ -0,0 +1,35 @@
+/*
+ * 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.job.hadoop.cube;
+
+import org.apache.hadoop.util.ToolRunner;
+
+/**
+ * @author honma
+ * 
+ */
+
+public class BaseCuboidJob extends CuboidJob {
+    public BaseCuboidJob() {
+        this.setMapperClass(BaseCuboidMapper.class);
+    }
+
+    public static void main(String[] args) throws Exception {
+        CuboidJob job = new BaseCuboidJob();
+        int exitCode = ToolRunner.run(job, args);
+        System.exit(exitCode);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/job/src/main/java/com/kylinolap/job/hadoop/cube/BaseCuboidMapper.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/com/kylinolap/job/hadoop/cube/BaseCuboidMapper.java b/job/src/main/java/com/kylinolap/job/hadoop/cube/BaseCuboidMapper.java
new file mode 100644
index 0000000..39b3918
--- /dev/null
+++ b/job/src/main/java/com/kylinolap/job/hadoop/cube/BaseCuboidMapper.java
@@ -0,0 +1,246 @@
+/*
+ * 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.job.hadoop.cube;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import java.util.List;
+
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.mapreduce.Mapper;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.google.common.collect.Lists;
+import com.kylinolap.common.KylinConfig;
+import com.kylinolap.cube.CubeInstance;
+import com.kylinolap.cube.CubeManager;
+import com.kylinolap.cube.CubeSegment;
+import com.kylinolap.cube.CubeSegmentStatusEnum;
+import com.kylinolap.cube.common.BytesSplitter;
+import com.kylinolap.cube.common.SplittedBytes;
+import com.kylinolap.cube.cuboid.Cuboid;
+import com.kylinolap.cube.kv.AbstractRowKeyEncoder;
+import com.kylinolap.cube.kv.RowConstants;
+import com.kylinolap.cube.measure.MeasureCodec;
+import com.kylinolap.job.constant.BatchConstants;
+import com.kylinolap.job.hadoop.AbstractHadoopJob;
+import com.kylinolap.job.hadoop.hive.JoinedFlatTableDesc;
+import com.kylinolap.metadata.model.cube.CubeDesc;
+import com.kylinolap.metadata.model.cube.FunctionDesc;
+import com.kylinolap.metadata.model.cube.MeasureDesc;
+import com.kylinolap.metadata.model.cube.ParameterDesc;
+
+/**
+ * @author George Song (ysong1)
+ */
+public class BaseCuboidMapper<KEYIN> extends Mapper<KEYIN, Text, Text, Text> {
+
+    private static final Logger logger = LoggerFactory.getLogger(BaseCuboidMapper.class);
+
+    public static final byte[] HIVE_NULL = Bytes.toBytes("\\N");
+    public static final byte[] ONE = Bytes.toBytes("1");
+
+    private String cubeName;
+    private String segmentName;
+    private Cuboid baseCuboid;
+    private CubeInstance cube;
+    private CubeDesc cubeDesc;
+    private CubeSegment cubeSegment;
+    private List<byte[]> nullBytes;
+
+    private JoinedFlatTableDesc intermediateTableDesc;
+    private String intermediateTableRowDelimiter;
+    private byte byteRowDelimiter;
+
+    private int counter;
+    private int errorRecordCounter;
+    private Text outputKey = new Text();
+    private Text outputValue = new Text();
+    private Object[] measures;
+    private byte[][] keyBytesBuf;
+    private ByteBuffer valueBuf = ByteBuffer.allocate(RowConstants.ROWVALUE_BUFFER_SIZE);
+
+    private BytesSplitter bytesSplitter;
+    private AbstractRowKeyEncoder rowKeyEncoder;
+    private MeasureCodec measureCodec;
+
+    @Override
+    protected void setup(Context context) throws IOException {
+        cubeName = context.getConfiguration().get(BatchConstants.CFG_CUBE_NAME).toUpperCase();
+        segmentName = context.getConfiguration().get(BatchConstants.CFG_CUBE_SEGMENT_NAME);
+        intermediateTableRowDelimiter = context.getConfiguration().get(BatchConstants.CFG_CUBE_INTERMEDIATE_TABLE_ROW_DELIMITER, Character.toString(BatchConstants.INTERMEDIATE_TABLE_ROW_DELIMITER));
+        if (Bytes.toBytes(intermediateTableRowDelimiter).length > 1) {
+            throw new RuntimeException("Expected delimiter byte length is 1, but got " + Bytes.toBytes(intermediateTableRowDelimiter).length);
+        }
+
+        byteRowDelimiter = Bytes.toBytes(intermediateTableRowDelimiter)[0];
+
+        KylinConfig config = AbstractHadoopJob.loadKylinPropsAndMetadata(context.getConfiguration());
+
+        cube = CubeManager.getInstance(config).getCube(cubeName);
+        cubeDesc = cube.getDescriptor();
+        cubeSegment = cube.getSegment(segmentName, CubeSegmentStatusEnum.NEW);
+
+        long baseCuboidId = Cuboid.getBaseCuboidId(cubeDesc);
+        baseCuboid = Cuboid.findById(cubeDesc, baseCuboidId);
+
+        intermediateTableDesc = new JoinedFlatTableDesc(cube.getDescriptor(), cubeSegment);
+
+        bytesSplitter = new BytesSplitter(200, 4096);
+        rowKeyEncoder = AbstractRowKeyEncoder.createInstance(cubeSegment, baseCuboid);
+
+        measureCodec = new MeasureCodec(cubeDesc.getMeasures());
+        measures = new Object[cubeDesc.getMeasures().size()];
+
+        int colCount = cubeDesc.getRowkey().getRowKeyColumns().length;
+        keyBytesBuf = new byte[colCount][];
+
+        initNullBytes();
+    }
+
+    private void initNullBytes() {
+        nullBytes = Lists.newArrayList();
+        nullBytes.add(HIVE_NULL);
+        String[] nullStrings = cubeDesc.getNullStrings();
+        if (nullStrings != null) {
+            for (String s : nullStrings) {
+                nullBytes.add(Bytes.toBytes(s));
+            }
+        }
+    }
+
+    private boolean isNull(byte[] v) {
+        for (byte[] nullByte : nullBytes) {
+            if (Bytes.equals(v, nullByte))
+                return true;
+        }
+        return false;
+    }
+
+    private byte[] buildKey(SplittedBytes[] splitBuffers) {
+        int[] rowKeyColumnIndexes = intermediateTableDesc.getRowKeyColumnIndexes();
+        for (int i = 0; i < baseCuboid.getColumns().size(); i++) {
+            int index = rowKeyColumnIndexes[i];
+            keyBytesBuf[i] = Arrays.copyOf(splitBuffers[index].value, splitBuffers[index].length);
+            if (isNull(keyBytesBuf[i])) {
+                keyBytesBuf[i] = null;
+            }
+        }
+        return rowKeyEncoder.encode(keyBytesBuf);
+    }
+
+    private void buildValue(SplittedBytes[] splitBuffers) {
+
+        for (int i = 0; i < measures.length; i++) {
+            byte[] valueBytes = getValueBytes(splitBuffers, i);
+            measures[i] = measureCodec.getSerializer(i).valueOf(valueBytes);
+        }
+
+        valueBuf.clear();
+        measureCodec.encode(measures, valueBuf);
+    }
+
+    private byte[] getValueBytes(SplittedBytes[] splitBuffers, int measureIdx) {
+        MeasureDesc desc = cubeDesc.getMeasures().get(measureIdx);
+        FunctionDesc func = desc.getFunction();
+        ParameterDesc paramDesc = func.getParameter();
+        int[] flatTableIdx = intermediateTableDesc.getMeasureColumnIndexes()[measureIdx];
+
+        byte[] result = null;
+
+        // constant
+        if (flatTableIdx == null) {
+            result = Bytes.toBytes(paramDesc.getValue());
+        }
+        // column values
+        else {
+            // for multiple columns, their values are joined
+            for (int i = 0; i < flatTableIdx.length; i++) {
+                SplittedBytes split = splitBuffers[flatTableIdx[i]];
+                if (result == null) {
+                    result = Arrays.copyOf(split.value, split.length);
+                } else {
+                    byte[] newResult = new byte[result.length + split.length];
+                    System.arraycopy(result, 0, newResult, 0, result.length);
+                    System.arraycopy(split.value, 0, newResult, result.length, split.length);
+                    result = newResult;
+                }
+            }
+        }
+
+        if (func.isCount() || func.isHolisticCountDistinct()) {
+            // note for holistic count distinct, this value will be ignored
+            result = ONE;
+        }
+
+        if (isNull(result)) {
+            result = null;
+        }
+
+        return result;
+    }
+
+    @Override
+    public void map(KEYIN key, Text value, Context context) throws IOException, InterruptedException {
+        counter++;
+        if (counter % BatchConstants.COUNTER_MAX == 0) {
+            logger.info("Handled " + counter + " records!");
+        }
+
+        bytesSplitter.split(value.getBytes(), value.getLength(), byteRowDelimiter);
+
+        try {
+            byte[] rowKey = buildKey(bytesSplitter.getSplitBuffers());
+            outputKey.set(rowKey, 0, rowKey.length);
+
+            buildValue(bytesSplitter.getSplitBuffers());
+            outputValue.set(valueBuf.array(), 0, valueBuf.position());
+
+            context.write(outputKey, outputValue);
+        } catch (Exception ex) {
+            handleErrorRecord(bytesSplitter.getSplitBuffers(), ex);
+        }
+    }
+
+    private void handleErrorRecord(SplittedBytes[] splitBuffers, Exception ex) throws IOException {
+        
+        StringBuilder buf = new StringBuilder();
+        buf.append("Error record: [");
+        for (int i = 0; i < splitBuffers.length; i++) {
+            if (i > 0)
+                buf.append(", ");
+            
+            buf.append(Bytes.toString(splitBuffers[i].value, 0, splitBuffers[i].length));
+        }
+        buf.append("] -- ");
+        buf.append(ex.toString());
+        System.err.println(buf.toString());
+        ex.printStackTrace(System.err);
+        
+        errorRecordCounter++;
+        if (errorRecordCounter > BatchConstants.ERROR_RECORD_THRESHOLD) {
+            if (ex instanceof IOException)
+                throw (IOException) ex;
+            else if (ex instanceof RuntimeException)
+                throw (RuntimeException) ex;
+            else
+                throw new RuntimeException("", ex);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/job/src/main/java/com/kylinolap/job/hadoop/cube/CubeHFileJob.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/com/kylinolap/job/hadoop/cube/CubeHFileJob.java b/job/src/main/java/com/kylinolap/job/hadoop/cube/CubeHFileJob.java
new file mode 100644
index 0000000..1f939d1
--- /dev/null
+++ b/job/src/main/java/com/kylinolap/job/hadoop/cube/CubeHFileJob.java
@@ -0,0 +1,113 @@
+/*
+ * 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.job.hadoop.cube;
+
+import java.io.File;
+
+import org.apache.commons.cli.Options;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.HBaseConfiguration;
+import org.apache.hadoop.hbase.client.HTable;
+import org.apache.hadoop.hbase.mapreduce.HFileOutputFormat;
+import org.apache.hadoop.hbase.mapreduce.KeyValueSortReducer;
+import org.apache.hadoop.hdfs.DFSConfigKeys;
+import org.apache.hadoop.mapreduce.Job;
+import org.apache.hadoop.mapreduce.lib.input.SequenceFileInputFormat;
+import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
+import org.apache.hadoop.util.ToolRunner;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.kylinolap.common.KylinConfig;
+import com.kylinolap.cube.CubeInstance;
+import com.kylinolap.cube.CubeManager;
+import com.kylinolap.job.constant.BatchConstants;
+import com.kylinolap.job.hadoop.AbstractHadoopJob;
+
+/**
+ * @author George Song (ysong1)
+ * 
+ */
+
+public class CubeHFileJob extends AbstractHadoopJob {
+
+    protected static final Logger log = LoggerFactory.getLogger(CubeHFileJob.class);
+
+    public int run(String[] args) throws Exception {
+        Options options = new Options();
+
+        try {
+            options.addOption(OPTION_JOB_NAME);
+            options.addOption(OPTION_CUBE_NAME);
+            options.addOption(OPTION_INPUT_PATH);
+            options.addOption(OPTION_OUTPUT_PATH);
+            options.addOption(OPTION_HTABLE_NAME);
+            parseOptions(options, args);
+
+            Path output = new Path(getOptionValue(OPTION_OUTPUT_PATH));
+            String cubeName = getOptionValue(OPTION_CUBE_NAME).toUpperCase();
+
+            CubeManager cubeMgr = CubeManager.getInstance(KylinConfig.getInstanceFromEnv());
+
+            CubeInstance cube = cubeMgr.getCube(cubeName);
+            job = Job.getInstance(getConf(), getOptionValue(OPTION_JOB_NAME));
+
+            File JarFile = new File(KylinConfig.getInstanceFromEnv().getKylinJobJarPath());
+            if (JarFile.exists()) {
+                job.setJar(KylinConfig.getInstanceFromEnv().getKylinJobJarPath());
+            } else {
+                job.setJarByClass(this.getClass());
+            }
+
+            addInputDirs(getOptionValue(OPTION_INPUT_PATH), job);
+            FileOutputFormat.setOutputPath(job, output);
+
+            job.setInputFormatClass(SequenceFileInputFormat.class);
+            job.setMapperClass(CubeHFileMapper.class);
+            job.setReducerClass(KeyValueSortReducer.class);
+
+            // set job configuration
+            job.getConfiguration().set(BatchConstants.CFG_CUBE_NAME, cubeName);
+            Configuration conf = HBaseConfiguration.create(getConf());
+            // add metadata to distributed cache
+            attachKylinPropsAndMetadata(cube, job.getConfiguration());
+
+            String tableName = getOptionValue(OPTION_HTABLE_NAME).toUpperCase();
+            HTable htable = new HTable(conf, tableName);
+
+            //Automatic config !
+            HFileOutputFormat.configureIncrementalLoad(job, htable);
+
+            // set block replication to 3 for hfiles
+            conf.set(DFSConfigKeys.DFS_REPLICATION_KEY, "3");
+
+            this.deletePath(job.getConfiguration(), output);
+
+            return waitForCompletion(job);
+        } catch (Exception e) {
+            printUsage(options);
+            log.error(e.getLocalizedMessage(), e);
+            return 2;
+        }
+    }
+
+    public static void main(String[] args) throws Exception {
+        int exitCode = ToolRunner.run(new CubeHFileJob(), args);
+        System.exit(exitCode);
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/job/src/main/java/com/kylinolap/job/hadoop/cube/CubeHFileMapper.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/com/kylinolap/job/hadoop/cube/CubeHFileMapper.java b/job/src/main/java/com/kylinolap/job/hadoop/cube/CubeHFileMapper.java
new file mode 100644
index 0000000..4f9b5c8
--- /dev/null
+++ b/job/src/main/java/com/kylinolap/job/hadoop/cube/CubeHFileMapper.java
@@ -0,0 +1,178 @@
+/*
+ * 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.job.hadoop.cube;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import java.util.List;
+
+import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.KeyValue.Type;
+import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.mapreduce.Mapper;
+
+import com.google.common.collect.Lists;
+import com.kylinolap.common.KylinConfig;
+import com.kylinolap.cube.CubeManager;
+import com.kylinolap.cube.kv.RowConstants;
+import com.kylinolap.cube.measure.MeasureCodec;
+import com.kylinolap.job.constant.BatchConstants;
+import com.kylinolap.job.hadoop.AbstractHadoopJob;
+import com.kylinolap.metadata.model.cube.CubeDesc;
+import com.kylinolap.metadata.model.cube.HBaseColumnDesc;
+import com.kylinolap.metadata.model.cube.HBaseColumnFamilyDesc;
+import com.kylinolap.metadata.model.cube.MeasureDesc;
+
+/**
+ * @author George Song (ysong1)
+ * 
+ */
+public class CubeHFileMapper extends Mapper<Text, Text, ImmutableBytesWritable, KeyValue> {
+
+    ImmutableBytesWritable outputKey = new ImmutableBytesWritable();
+
+    String cubeName;
+    CubeDesc cubeDesc;
+
+    MeasureCodec inputCodec;
+    Object[] inputMeasures;
+    List<KeyValueCreator> keyValueCreators;
+
+    @Override
+    protected void setup(Context context) throws IOException {
+        cubeName = context.getConfiguration().get(BatchConstants.CFG_CUBE_NAME);
+
+        KylinConfig config = AbstractHadoopJob.loadKylinPropsAndMetadata(context.getConfiguration());
+
+        CubeManager cubeMgr = CubeManager.getInstance(config);
+        cubeDesc = cubeMgr.getCube(cubeName).getDescriptor();
+
+        inputCodec = new MeasureCodec(cubeDesc.getMeasures());
+        inputMeasures = new Object[cubeDesc.getMeasures().size()];
+        keyValueCreators = Lists.newArrayList();
+
+        for (HBaseColumnFamilyDesc cfDesc : cubeDesc.getHBaseMapping().getColumnFamily()) {
+            for (HBaseColumnDesc colDesc : cfDesc.getColumns()) {
+                keyValueCreators.add(new KeyValueCreator(cubeDesc, colDesc));
+            }
+        }
+    }
+
+    @Override
+    public void map(Text key, Text value, Context context) throws IOException, InterruptedException {
+        outputKey.set(key.getBytes(), 0, key.getLength());
+        KeyValue outputValue;
+
+        int n = keyValueCreators.size();
+        if (n == 1 && keyValueCreators.get(0).isFullCopy) { // shortcut for
+                                                            // simple full copy
+
+            outputValue = keyValueCreators.get(0).create(key, value.getBytes(), 0, value.getLength());
+            context.write(outputKey, outputValue);
+
+        } else { // normal (complex) case that distributes measures to multiple
+                 // HBase columns
+
+            inputCodec.decode(value, inputMeasures);
+
+            for (int i = 0; i < n; i++) {
+                outputValue = keyValueCreators.get(i).create(key, inputMeasures);
+                context.write(outputKey, outputValue);
+            }
+        }
+    }
+
+    class KeyValueCreator {
+        byte[] cfBytes;
+        byte[] qBytes;
+        long timestamp;
+
+        int[] refIndex;
+        MeasureDesc[] refMeasures;
+
+        MeasureCodec codec;
+        Object[] colValues;
+        ByteBuffer valueBuf = ByteBuffer.allocate(RowConstants.ROWVALUE_BUFFER_SIZE);
+
+        boolean isFullCopy;
+
+        public KeyValueCreator(CubeDesc cubeDesc, HBaseColumnDesc colDesc) {
+
+            cfBytes = Bytes.toBytes(colDesc.getColumnFamilyName());
+            qBytes = Bytes.toBytes(colDesc.getQualifier());
+            timestamp = System.currentTimeMillis();
+
+            List<MeasureDesc> measures = cubeDesc.getMeasures();
+            String[] measureNames = getMeasureNames(cubeDesc);
+            String[] refs = colDesc.getMeasureRefs();
+
+            refIndex = new int[refs.length];
+            refMeasures = new MeasureDesc[refs.length];
+            for (int i = 0; i < refs.length; i++) {
+                refIndex[i] = indexOf(measureNames, refs[i]);
+                refMeasures[i] = measures.get(refIndex[i]);
+            }
+
+            codec = new MeasureCodec(refMeasures);
+            colValues = new Object[refs.length];
+
+            isFullCopy = true;
+            for (int i = 0; i < measures.size(); i++) {
+                if (refIndex.length <= i || refIndex[i] != i)
+                    isFullCopy = false;
+            }
+        }
+
+        public KeyValue create(Text key, Object[] measureValues) {
+            for (int i = 0; i < colValues.length; i++) {
+                colValues[i] = measureValues[refIndex[i]];
+            }
+
+            valueBuf.clear();
+            codec.encode(colValues, valueBuf);
+
+            return create(key, valueBuf.array(), 0, valueBuf.position());
+        }
+
+        public KeyValue create(Text key, byte[] value, int voffset, int vlen) {
+            return new KeyValue(key.getBytes(), 0, key.getLength(), //
+                    cfBytes, 0, cfBytes.length, //
+                    qBytes, 0, qBytes.length, //
+                    timestamp, Type.Put, //
+                    value, voffset, vlen);
+        }
+
+        private int indexOf(String[] measureNames, String ref) {
+            for (int i = 0; i < measureNames.length; i++)
+                if (measureNames[i].equalsIgnoreCase(ref))
+                    return i;
+
+            throw new IllegalArgumentException("Measure '" + ref + "' not found in " + Arrays.toString(measureNames));
+        }
+
+        private String[] getMeasureNames(CubeDesc cubeDesc) {
+            List<MeasureDesc> measures = cubeDesc.getMeasures();
+            String[] result = new String[measures.size()];
+            for (int i = 0; i < measures.size(); i++)
+                result[i] = measures.get(i).getName();
+            return result;
+        }
+
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/job/src/main/java/com/kylinolap/job/hadoop/cube/CuboidJob.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/com/kylinolap/job/hadoop/cube/CuboidJob.java b/job/src/main/java/com/kylinolap/job/hadoop/cube/CuboidJob.java
new file mode 100644
index 0000000..687fe10
--- /dev/null
+++ b/job/src/main/java/com/kylinolap/job/hadoop/cube/CuboidJob.java
@@ -0,0 +1,194 @@
+/*
+ * 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.job.hadoop.cube;
+
+import java.io.File;
+import java.io.IOException;
+
+import org.apache.commons.cli.Options;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.mapreduce.Job;
+import org.apache.hadoop.mapreduce.Mapper;
+import org.apache.hadoop.mapreduce.lib.input.FileInputFormat;
+import org.apache.hadoop.mapreduce.lib.input.SequenceFileInputFormat;
+import org.apache.hadoop.mapreduce.lib.input.TextInputFormat;
+import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
+import org.apache.hadoop.mapreduce.lib.output.SequenceFileOutputFormat;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.kylinolap.common.KylinConfig;
+import com.kylinolap.cube.CubeInstance;
+import com.kylinolap.cube.CubeManager;
+import com.kylinolap.cube.cuboid.CuboidCLI;
+import com.kylinolap.job.constant.BatchConstants;
+import com.kylinolap.job.exception.JobException;
+import com.kylinolap.job.hadoop.AbstractHadoopJob;
+import com.kylinolap.metadata.model.cube.CubeDesc;
+
+/**
+ * @author ysong1
+ */
+public class CuboidJob extends AbstractHadoopJob {
+
+    protected static final Logger log = LoggerFactory.getLogger(CuboidJob.class);
+    private static final String MAPRED_REDUCE_TASKS = "mapred.reduce.tasks";
+
+    @SuppressWarnings("rawtypes")
+    private Class<? extends Mapper> mapperClass;
+
+    @Override
+    public int run(String[] args) throws Exception {
+        Options options = new Options();
+
+        try {
+            options.addOption(OPTION_JOB_NAME);
+            options.addOption(OPTION_CUBE_NAME);
+            options.addOption(OPTION_SEGMENT_NAME);
+            options.addOption(OPTION_INPUT_PATH);
+            options.addOption(OPTION_OUTPUT_PATH);
+            options.addOption(OPTION_NCUBOID_LEVEL);
+            options.addOption(OPTION_INPUT_FORMAT);
+            parseOptions(options, args);
+
+            Path input = new Path(getOptionValue(OPTION_INPUT_PATH));
+            Path output = new Path(getOptionValue(OPTION_OUTPUT_PATH));
+            String cubeName = getOptionValue(OPTION_CUBE_NAME).toUpperCase();
+            int nCuboidLevel = Integer.parseInt(getOptionValue(OPTION_NCUBOID_LEVEL));
+            String segmentName = getOptionValue(OPTION_SEGMENT_NAME);
+
+            KylinConfig config = KylinConfig.getInstanceFromEnv();
+            CubeManager cubeMgr = CubeManager.getInstance(config);
+            CubeInstance cube = cubeMgr.getCube(cubeName);
+
+            job = Job.getInstance(getConf(), getOptionValue(OPTION_JOB_NAME));
+            System.out.println("Starting: " + job.getJobName());
+            FileInputFormat.setInputPaths(job, input);
+
+            File jarFile = new File(config.getKylinJobJarPath());
+            if (jarFile.exists()) {
+                job.setJar(config.getKylinJobJarPath());
+            } else {
+                job.setJarByClass(this.getClass());
+            }
+
+            // Mapper
+            if (this.mapperClass == null) {
+                throw new Exception("Mapper class is not set!");
+            }
+
+            boolean isInputTextFormat = false;
+            if (hasOption(OPTION_INPUT_FORMAT) && ("textinputformat".equalsIgnoreCase(getOptionValue(OPTION_INPUT_FORMAT)))) {
+                isInputTextFormat = true;
+            }
+
+            if (isInputTextFormat) {
+                job.setInputFormatClass(TextInputFormat.class);
+
+            } else {
+                job.setInputFormatClass(SequenceFileInputFormat.class);
+            }
+            job.setMapperClass(this.mapperClass);
+            job.setMapOutputKeyClass(Text.class);
+            job.setMapOutputValueClass(Text.class);
+            job.setCombinerClass(CuboidReducer.class); // for base cuboid shuffle skew, some rowkey aggregates far more records than others
+
+            // Reducer
+            job.setReducerClass(CuboidReducer.class);
+            job.setOutputFormatClass(SequenceFileOutputFormat.class);
+            job.setOutputKeyClass(Text.class);
+            job.setOutputValueClass(Text.class);
+
+            FileOutputFormat.setOutputPath(job, output);
+
+            // set job configuration
+            job.getConfiguration().set(BatchConstants.CFG_CUBE_NAME, cubeName);
+            job.getConfiguration().set(BatchConstants.CFG_CUBE_SEGMENT_NAME, segmentName);
+            // add metadata to distributed cache
+            attachKylinPropsAndMetadata(cube, job.getConfiguration());
+
+            setReduceTaskNum(job, config, cubeName, nCuboidLevel);
+
+            this.deletePath(job.getConfiguration(), output);
+
+            return waitForCompletion(job);
+        } catch (Exception e) {
+            printUsage(options);
+            log.error(e.getLocalizedMessage(), e);
+            return 2;
+        }
+    }
+
+    protected void setReduceTaskNum(Job job, KylinConfig config, String cubeName, int level) throws ClassNotFoundException, IOException, InterruptedException, JobException {
+        Configuration jobConf = job.getConfiguration();
+        KylinConfig kylinConfig = KylinConfig.getInstanceFromEnv();
+
+        CubeDesc cubeDesc = CubeManager.getInstance(config).getCube(cubeName).getDescriptor();
+
+        double perReduceInputMB = kylinConfig.getDefaultHadoopJobReducerInputMB();
+        double reduceCountRatio = kylinConfig.getDefaultHadoopJobReducerCountRatio();
+
+        // total map input MB
+        double totalMapInputMB = this.getTotalMapInputMB();
+
+        // output / input ratio
+        int preLevelCuboids, thisLevelCuboids;
+        if (level == 0) { // base cuboid
+            preLevelCuboids = thisLevelCuboids = 1;
+        } else { // n-cuboid
+            int[] allLevelCount = CuboidCLI.calculateAllLevelCount(cubeDesc);
+            preLevelCuboids = allLevelCount[level - 1];
+            thisLevelCuboids = allLevelCount[level];
+        }
+
+        // total reduce input MB
+        double totalReduceInputMB = totalMapInputMB * thisLevelCuboids / preLevelCuboids;
+
+        // number of reduce tasks
+        int numReduceTasks = (int) Math.round(totalReduceInputMB / perReduceInputMB * reduceCountRatio);
+
+        // adjust reducer number for cube which has DISTINCT_COUNT measures for
+        // better performance
+        if (cubeDesc.hasHolisticCountDistinctMeasures()) {
+            numReduceTasks = numReduceTasks * 4;
+        }
+
+        // at least 1 reducer
+        numReduceTasks = Math.max(1, numReduceTasks);
+        // no more than 5000 reducer by default
+        numReduceTasks = Math.min(kylinConfig.getHadoopJobMaxReducerNumber(), numReduceTasks);
+
+        jobConf.setInt(MAPRED_REDUCE_TASKS, numReduceTasks);
+
+        System.out.println("Having total map input MB " + Math.round(totalMapInputMB));
+        System.out.println("Having level " + level + ", pre-level cuboids " + preLevelCuboids + ", this level cuboids " + thisLevelCuboids);
+        System.out.println("Having per reduce MB " + perReduceInputMB + ", reduce count ratio " + reduceCountRatio);
+        System.out.println("Setting " + MAPRED_REDUCE_TASKS + "=" + numReduceTasks);
+    }
+
+    /**
+     * @param mapperClass
+     *            the mapperClass to set
+     */
+    @SuppressWarnings("rawtypes")
+    public void setMapperClass(Class<? extends Mapper> mapperClass) {
+        this.mapperClass = mapperClass;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/job/src/main/java/com/kylinolap/job/hadoop/cube/CuboidReducer.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/com/kylinolap/job/hadoop/cube/CuboidReducer.java b/job/src/main/java/com/kylinolap/job/hadoop/cube/CuboidReducer.java
new file mode 100644
index 0000000..50af652
--- /dev/null
+++ b/job/src/main/java/com/kylinolap/job/hadoop/cube/CuboidReducer.java
@@ -0,0 +1,98 @@
+/*
+ * 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.job.hadoop.cube;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.List;
+
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.mapreduce.Reducer;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.kylinolap.common.KylinConfig;
+import com.kylinolap.cube.CubeManager;
+import com.kylinolap.cube.kv.RowConstants;
+import com.kylinolap.cube.measure.MeasureAggregators;
+import com.kylinolap.cube.measure.MeasureCodec;
+import com.kylinolap.job.constant.BatchConstants;
+import com.kylinolap.job.hadoop.AbstractHadoopJob;
+import com.kylinolap.metadata.model.cube.CubeDesc;
+import com.kylinolap.metadata.model.cube.MeasureDesc;
+
+/**
+ * @author George Song (ysong1)
+ * 
+ */
+public class CuboidReducer extends Reducer<Text, Text, Text, Text> {
+
+    private static final Logger logger = LoggerFactory.getLogger(CuboidReducer.class);
+
+    private String cubeName;
+    private CubeDesc cubeDesc;
+    private List<MeasureDesc> measuresDescs;
+
+    private MeasureCodec codec;
+    private MeasureAggregators aggs;
+
+    private int counter;
+    private Object[] input;
+    private Object[] result;
+
+    private ByteBuffer valueBuf = ByteBuffer.allocate(RowConstants.ROWVALUE_BUFFER_SIZE);
+    private Text outputValue = new Text();
+
+    @Override
+    protected void setup(Context context) throws IOException {
+        cubeName = context.getConfiguration().get(BatchConstants.CFG_CUBE_NAME).toUpperCase();
+
+        KylinConfig config = AbstractHadoopJob.loadKylinPropsAndMetadata(context.getConfiguration());
+
+        cubeDesc = CubeManager.getInstance(config).getCube(cubeName).getDescriptor();
+        measuresDescs = cubeDesc.getMeasures();
+
+        codec = new MeasureCodec(measuresDescs);
+        aggs = new MeasureAggregators(measuresDescs);
+
+        input = new Object[measuresDescs.size()];
+        result = new Object[measuresDescs.size()];
+    }
+
+    @Override
+    public void reduce(Text key, Iterable<Text> values, Context context) throws IOException, InterruptedException {
+
+        aggs.reset();
+
+        for (Text value : values) {
+            codec.decode(value, input);
+            aggs.aggregate(input);
+        }
+        aggs.collectStates(result);
+
+        valueBuf.clear();
+        codec.encode(result, valueBuf);
+
+        outputValue.set(valueBuf.array(), 0, valueBuf.position());
+        context.write(key, outputValue);
+
+        counter++;
+        if (counter % BatchConstants.COUNTER_MAX == 0) {
+            logger.info("Handled " + counter + " records!");
+        }
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/job/src/main/java/com/kylinolap/job/hadoop/cube/FactDistinctColumnsCombiner.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/com/kylinolap/job/hadoop/cube/FactDistinctColumnsCombiner.java b/job/src/main/java/com/kylinolap/job/hadoop/cube/FactDistinctColumnsCombiner.java
new file mode 100644
index 0000000..916dd9e
--- /dev/null
+++ b/job/src/main/java/com/kylinolap/job/hadoop/cube/FactDistinctColumnsCombiner.java
@@ -0,0 +1,54 @@
+/*
+ * 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.job.hadoop.cube;
+
+import java.io.IOException;
+import java.util.HashSet;
+
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.io.ShortWritable;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.mapreduce.Reducer;
+
+import com.kylinolap.common.util.ByteArray;
+
+/**
+ * @author yangli9
+ */
+public class FactDistinctColumnsCombiner extends Reducer<ShortWritable, Text, ShortWritable, Text> {
+
+    private Text outputValue = new Text();
+
+    @Override
+    protected void setup(Context context) throws IOException {
+    }
+
+    @Override
+    public void reduce(ShortWritable key, Iterable<Text> values, Context context) throws IOException, InterruptedException {
+
+        HashSet<ByteArray> set = new HashSet<ByteArray>();
+        for (Text textValue : values) {
+            ByteArray value = new ByteArray(Bytes.copy(textValue.getBytes(), 0, textValue.getLength()));
+            set.add(value);
+        }
+
+        for (ByteArray value : set) {
+            outputValue.set(value.data);
+            context.write(key, outputValue);
+        }
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/job/src/main/java/com/kylinolap/job/hadoop/cube/FactDistinctColumnsJob.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/com/kylinolap/job/hadoop/cube/FactDistinctColumnsJob.java b/job/src/main/java/com/kylinolap/job/hadoop/cube/FactDistinctColumnsJob.java
new file mode 100644
index 0000000..556f690
--- /dev/null
+++ b/job/src/main/java/com/kylinolap/job/hadoop/cube/FactDistinctColumnsJob.java
@@ -0,0 +1,129 @@
+/*
+ * 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.job.hadoop.cube;
+
+import java.io.File;
+import java.io.IOException;
+
+import org.apache.commons.cli.Options;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.io.NullWritable;
+import org.apache.hadoop.io.ShortWritable;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.mapreduce.Job;
+import org.apache.hadoop.mapreduce.lib.input.FileInputFormat;
+import org.apache.hadoop.mapreduce.lib.input.SequenceFileInputFormat;
+import org.apache.hadoop.mapreduce.lib.input.TextInputFormat;
+import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
+import org.apache.hadoop.mapreduce.lib.output.SequenceFileOutputFormat;
+import org.apache.hadoop.util.ToolRunner;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.kylinolap.common.KylinConfig;
+import com.kylinolap.cube.CubeManager;
+import com.kylinolap.job.constant.BatchConstants;
+import com.kylinolap.job.hadoop.AbstractHadoopJob;
+
+/**
+ * @author yangli9
+ */
+public class FactDistinctColumnsJob extends AbstractHadoopJob {
+    protected static final Logger log = LoggerFactory.getLogger(FactDistinctColumnsJob.class);
+
+    @Override
+    public int run(String[] args) throws Exception {
+        Options options = new Options();
+
+        try {
+            options.addOption(OPTION_JOB_NAME);
+            options.addOption(OPTION_CUBE_NAME);
+            options.addOption(OPTION_INPUT_PATH);
+            options.addOption(OPTION_INPUT_FORMAT);
+            options.addOption(OPTION_OUTPUT_PATH);
+            parseOptions(options, args);
+
+            job = Job.getInstance(getConf(), getOptionValue(OPTION_JOB_NAME));
+            String cubeName = getOptionValue(OPTION_CUBE_NAME);
+            Path input = new Path(getOptionValue(OPTION_INPUT_PATH));
+            String inputFormat = getOptionValue(OPTION_INPUT_FORMAT);
+            Path output = new Path(getOptionValue(OPTION_OUTPUT_PATH));
+
+            // ----------------------------------------------------------------------------
+
+            job.getConfiguration().set(BatchConstants.CFG_CUBE_NAME, cubeName);
+            System.out.println("Starting: " + job.getJobName());
+
+            setupMapInput(input, inputFormat);
+            setupReduceOutput(output);
+
+            // add metadata to distributed cache
+            CubeManager cubeMgr = CubeManager.getInstance(KylinConfig.getInstanceFromEnv());
+            // CubeSegment seg = cubeMgr.getCube(cubeName).getTheOnlySegment();
+            attachKylinPropsAndMetadata(cubeMgr.getCube(cubeName), job.getConfiguration());
+
+            return waitForCompletion(job);
+
+        } catch (Exception e) {
+            printUsage(options);
+            log.error(e.getLocalizedMessage(), e);
+            return 2;
+        }
+
+    }
+
+    private void setupMapInput(Path input, String inputFormat) throws IOException {
+        FileInputFormat.setInputPaths(job, input);
+
+        File JarFile = new File(KylinConfig.getInstanceFromEnv().getKylinJobJarPath());
+        if (JarFile.exists()) {
+            job.setJar(KylinConfig.getInstanceFromEnv().getKylinJobJarPath());
+        } else {
+            job.setJarByClass(this.getClass());
+        }
+
+        if ("text".equalsIgnoreCase(inputFormat) || "textinputformat".equalsIgnoreCase(inputFormat)) {
+            job.setInputFormatClass(TextInputFormat.class);
+        } else {
+            job.setInputFormatClass(SequenceFileInputFormat.class);
+        }
+        job.setMapperClass(FactDistinctColumnsMapper.class);
+        job.setCombinerClass(FactDistinctColumnsCombiner.class);
+        job.setMapOutputKeyClass(ShortWritable.class);
+        job.setMapOutputValueClass(Text.class);
+    }
+
+    private void setupReduceOutput(Path output) throws IOException {
+        job.setReducerClass(FactDistinctColumnsReducer.class);
+        job.setOutputFormatClass(SequenceFileOutputFormat.class);
+        job.setOutputKeyClass(NullWritable.class);
+        job.setOutputValueClass(Text.class);
+
+        FileOutputFormat.setOutputPath(job, output);
+        job.getConfiguration().set(BatchConstants.OUTPUT_PATH, output.toString());
+
+        job.setNumReduceTasks(1);
+
+        deletePath(job.getConfiguration(), output);
+    }
+
+    public static void main(String[] args) throws Exception {
+        FactDistinctColumnsJob job = new FactDistinctColumnsJob();
+        int exitCode = ToolRunner.run(job, args);
+        System.exit(exitCode);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/job/src/main/java/com/kylinolap/job/hadoop/cube/FactDistinctColumnsMapper.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/com/kylinolap/job/hadoop/cube/FactDistinctColumnsMapper.java b/job/src/main/java/com/kylinolap/job/hadoop/cube/FactDistinctColumnsMapper.java
new file mode 100644
index 0000000..236658d
--- /dev/null
+++ b/job/src/main/java/com/kylinolap/job/hadoop/cube/FactDistinctColumnsMapper.java
@@ -0,0 +1,111 @@
+/*
+ * 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.job.hadoop.cube;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.io.ShortWritable;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.mapreduce.Mapper;
+
+import com.kylinolap.common.KylinConfig;
+import com.kylinolap.cube.CubeInstance;
+import com.kylinolap.cube.CubeManager;
+import com.kylinolap.cube.common.BytesSplitter;
+import com.kylinolap.cube.common.SplittedBytes;
+import com.kylinolap.cube.cuboid.Cuboid;
+import com.kylinolap.dict.DictionaryManager;
+import com.kylinolap.job.constant.BatchConstants;
+import com.kylinolap.job.hadoop.AbstractHadoopJob;
+import com.kylinolap.job.hadoop.hive.JoinedFlatTableDesc;
+import com.kylinolap.metadata.model.cube.CubeDesc;
+import com.kylinolap.metadata.model.cube.RowKeyDesc;
+import com.kylinolap.metadata.model.cube.TblColRef;
+
+/**
+ * @author yangli9
+ */
+public class FactDistinctColumnsMapper<KEYIN> extends Mapper<KEYIN, Text, ShortWritable, Text> {
+
+    private String cubeName;
+    private CubeInstance cube;
+    private CubeDesc cubeDesc;
+    private int[] factDictCols;
+
+    private JoinedFlatTableDesc intermediateTableDesc;
+    private String intermediateTableRowDelimiter;
+    private byte byteRowDelimiter;
+    private BytesSplitter bytesSplitter;
+
+    private ShortWritable outputKey = new ShortWritable();
+    private Text outputValue = new Text();
+
+    @Override
+    protected void setup(Context context) throws IOException {
+        Configuration conf = context.getConfiguration();
+        intermediateTableRowDelimiter = conf.get(BatchConstants.CFG_CUBE_INTERMEDIATE_TABLE_ROW_DELIMITER, Character.toString(BatchConstants.INTERMEDIATE_TABLE_ROW_DELIMITER));
+        byteRowDelimiter = intermediateTableRowDelimiter.getBytes("UTF-8")[0];
+        bytesSplitter = new BytesSplitter(200, 4096);
+
+        KylinConfig config = AbstractHadoopJob.loadKylinPropsAndMetadata(conf);
+        cubeName = conf.get(BatchConstants.CFG_CUBE_NAME);
+        cube = CubeManager.getInstance(config).getCube(cubeName);
+        cubeDesc = cube.getDescriptor();
+        intermediateTableDesc = new JoinedFlatTableDesc(cubeDesc, null);
+
+        long baseCuboidId = Cuboid.getBaseCuboidId(cubeDesc);
+        Cuboid baseCuboid = Cuboid.findById(cubeDesc, baseCuboidId);
+        List<TblColRef> columns = baseCuboid.getColumns();
+
+        ArrayList<Integer> factDictCols = new ArrayList<Integer>();
+        RowKeyDesc rowkey = cubeDesc.getRowkey();
+        DictionaryManager dictMgr = DictionaryManager.getInstance(config);
+        for (int i = 0; i < columns.size(); i++) {
+            TblColRef col = columns.get(i);
+            if (rowkey.isUseDictionary(col) == false)
+                continue;
+
+            String scanTable = (String) dictMgr.decideSourceData(cubeDesc, col, null)[0];
+            if (cubeDesc.isFactTable(scanTable)) {
+                System.out.println(col + " -- " + i);
+                factDictCols.add(i);
+            }
+        }
+        this.factDictCols = new int[factDictCols.size()];
+        for (int i = 0; i < factDictCols.size(); i++)
+            this.factDictCols[i] = factDictCols.get(i);
+    }
+
+    @Override
+    public void map(KEYIN key, Text value, Context context) throws IOException, InterruptedException {
+
+        bytesSplitter.split(value.getBytes(), value.getLength(), byteRowDelimiter);
+        SplittedBytes[] splitBuffers = bytesSplitter.getSplitBuffers();
+
+        int[] flatTableIndexes = intermediateTableDesc.getRowKeyColumnIndexes();
+        for (int i : factDictCols) {
+            outputKey.set((short) i);
+            SplittedBytes bytes = splitBuffers[flatTableIndexes[i]];
+            outputValue.set(bytes.value, 0, bytes.length);
+            System.out.println(i + " -- " + outputValue);
+            context.write(outputKey, outputValue);
+        }
+
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/job/src/main/java/com/kylinolap/job/hadoop/cube/FactDistinctColumnsReducer.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/com/kylinolap/job/hadoop/cube/FactDistinctColumnsReducer.java b/job/src/main/java/com/kylinolap/job/hadoop/cube/FactDistinctColumnsReducer.java
new file mode 100644
index 0000000..e701781
--- /dev/null
+++ b/job/src/main/java/com/kylinolap/job/hadoop/cube/FactDistinctColumnsReducer.java
@@ -0,0 +1,89 @@
+/*
+ * 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.job.hadoop.cube;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.List;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.io.NullWritable;
+import org.apache.hadoop.io.ShortWritable;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.mapreduce.Reducer;
+
+import com.kylinolap.common.KylinConfig;
+import com.kylinolap.common.util.ByteArray;
+import com.kylinolap.cube.CubeInstance;
+import com.kylinolap.cube.CubeManager;
+import com.kylinolap.cube.cuboid.Cuboid;
+import com.kylinolap.job.constant.BatchConstants;
+import com.kylinolap.job.hadoop.AbstractHadoopJob;
+import com.kylinolap.metadata.model.cube.CubeDesc;
+import com.kylinolap.metadata.model.cube.TblColRef;
+
+/**
+ * @author yangli9
+ */
+public class FactDistinctColumnsReducer extends Reducer<ShortWritable, Text, NullWritable, Text> {
+
+    private List<TblColRef> columnList = new ArrayList<TblColRef>();
+
+    @Override
+    protected void setup(Context context) throws IOException {
+        Configuration conf = context.getConfiguration();
+        KylinConfig config = AbstractHadoopJob.loadKylinPropsAndMetadata(conf);
+        String cubeName = conf.get(BatchConstants.CFG_CUBE_NAME);
+        CubeInstance cube = CubeManager.getInstance(config).getCube(cubeName);
+        CubeDesc cubeDesc = cube.getDescriptor();
+
+        long baseCuboidId = Cuboid.getBaseCuboidId(cubeDesc);
+        Cuboid baseCuboid = Cuboid.findById(cubeDesc, baseCuboidId);
+        columnList = baseCuboid.getColumns();
+    }
+
+    @Override
+    public void reduce(ShortWritable key, Iterable<Text> values, Context context) throws IOException, InterruptedException {
+        TblColRef col = columnList.get(key.get());
+
+        HashSet<ByteArray> set = new HashSet<ByteArray>();
+        for (Text textValue : values) {
+            ByteArray value = new ByteArray(Bytes.copy(textValue.getBytes(), 0, textValue.getLength()));
+            set.add(value);
+        }
+
+        Configuration conf = context.getConfiguration();
+        FileSystem fs = FileSystem.get(conf);
+        String outputPath = conf.get(BatchConstants.OUTPUT_PATH);
+        FSDataOutputStream out = fs.create(new Path(outputPath, col.getName()));
+
+        try {
+            for (ByteArray value : set) {
+                out.write(value.data);
+                out.write('\n');
+            }
+        } finally {
+            out.close();
+        }
+
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/job/src/main/java/com/kylinolap/job/hadoop/cube/KeyDistributionJob.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/com/kylinolap/job/hadoop/cube/KeyDistributionJob.java b/job/src/main/java/com/kylinolap/job/hadoop/cube/KeyDistributionJob.java
new file mode 100644
index 0000000..0c14abb
--- /dev/null
+++ b/job/src/main/java/com/kylinolap/job/hadoop/cube/KeyDistributionJob.java
@@ -0,0 +1,139 @@
+package com.kylinolap.job.hadoop.cube;
+
+///*
+// * 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.index.cube;
+//
+//import org.apache.commons.cli.Options;
+//import org.apache.hadoop.fs.FileSystem;
+//import org.apache.hadoop.fs.Path;
+//import org.apache.hadoop.io.LongWritable;
+//import org.apache.hadoop.io.Text;
+//import org.apache.hadoop.mapreduce.Job;
+//import org.apache.hadoop.mapreduce.lib.input.SequenceFileInputFormat;
+//import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
+//import org.apache.hadoop.mapreduce.lib.output.SequenceFileOutputFormat;
+//import org.apache.hadoop.util.ToolRunner;
+//
+//import com.kylinolap.cube.CubeInstance;
+//import com.kylinolap.cube.CubeManager;
+//import com.kylinolap.cube.cuboid.Cuboid;
+//import com.kylinolap.cube.kv.AbstractRowKeyEncoder;
+//import com.kylinolap.cube.kv.RowKeyEncoder;
+//import com.kylinolap.index.AbstractHadoopJob;
+//import com.kylinolap.metadata.model.cube.CubeDesc;
+//
+///**
+// * @author xjiang
+// *
+// */
+//
+//public class KeyDistributionJob extends AbstractHadoopJob {
+//
+//    public static final String JOB_TITLE = "Kylin Row Key Distribution Job";
+//    public static final String KEY_HEADER_LENGTH = "key_header_length";
+//    public static final String KEY_COLUMN_PERCENTAGE = "key_column_percentage";
+//    public static final String KEY_SPLIT_NUMBER = "key_split_number";
+//
+//    /* (non-Javadoc)
+//     * @see org.apache.hadoop.util.Tool#run(java.lang.String[])
+//     */
+//    @Override
+//    public int run(String[] args) throws Exception {
+//        Options options = new Options();
+//
+//        try {
+//            options.addOption(OPTION_INPUT_PATH);
+//            options.addOption(OPTION_OUTPUT_PATH);
+//            options.addOption(OPTION_METADATA_URL);
+//            options.addOption(OPTION_CUBE_NAME);
+//            options.addOption(OPTION_KEY_COLUMN_PERCENTAGE);
+//            options.addOption(OPTION_KEY_SPLIT_NUMBER);
+//            parseOptions(options, args);
+//
+//            // start job
+//            String jobName = JOB_TITLE + getOptionsAsString();
+//            System.out.println("Starting: " + jobName);
+//            Job job = Job.getInstanceFromEnv(getConf(), jobName);
+//
+//            // set job configuration - basic 
+//            job.setJarByClass(getClass());
+//            addInputDirs(getOptionValue(OPTION_INPUT_PATH), job);
+//
+//            Path output = new Path(getOptionValue(OPTION_OUTPUT_PATH));
+//            FileOutputFormat.setOutputPath(job, output);
+//            //job.getConfiguration().set("dfs.block.size", "67108864");
+//
+//            // set job configuration - key prefix size & key split number
+//            String keyColumnPercentage = getOptionValue(OPTION_KEY_COLUMN_PERCENTAGE);
+//            job.getConfiguration().set(KEY_COLUMN_PERCENTAGE, keyColumnPercentage);
+//            String metadataUrl = validateMetadataUrl(getOptionValue(OPTION_METADATA_URL));
+//            String cubeName = getOptionValue(OPTION_CUBE_NAME).toUpperCase();
+//            int keyHeaderLen = getKeyHeaderLength(metadataUrl, cubeName);
+//            job.getConfiguration().set(KEY_HEADER_LENGTH, String.valueOf(keyHeaderLen));
+//            job.getConfiguration().set(KEY_SPLIT_NUMBER, getOptionValue(OPTION_KEY_SPLIT_NUMBER));
+//
+//            // Mapper
+//            job.setInputFormatClass(SequenceFileInputFormat.class);
+//            job.setMapperClass(KeyDistributionMapper.class);
+//            job.setMapOutputKeyClass(Text.class);
+//            job.setMapOutputValueClass(LongWritable.class);
+//
+//            // Combiner, not needed any more as mapper now does the groping
+//            //job.setCombinerClass(KeyDistributionCombiner.class);
+//
+//            // Reducer - only one
+//            job.setReducerClass(KeyDistributionReducer.class);
+//            // use sequence file as output
+//            job.setOutputFormatClass(SequenceFileOutputFormat.class);
+//            // key is text
+//            job.setOutputKeyClass(Text.class);
+//            // value is long
+//            job.setOutputValueClass(LongWritable.class);
+//            job.setNumReduceTasks(1);
+//
+//            FileSystem fs = FileSystem.get(job.getConfiguration());
+//            if (fs.exists(output))
+//                fs.delete(output, true);
+//
+//            return waitForCompletion(job);
+//        } catch (Exception e) {
+//            printUsage(options);
+//            e.printStackTrace(System.err);
+//            return 2;
+//        }
+//    }
+//
+//    private int getKeyHeaderLength(String metadataUrl, String cubeName) {
+//        CubeManager cubeMgr = CubeManager.getInstanceFromEnv(metadataUrl);
+//        CubeInstance cubeInstance = cubeMgr.getCube(cubeName);
+//        CubeDesc cubeDesc = cubeInstance.getDescriptor();
+//        long baseCuboidId = Cuboid.getBaseCuboidId(cubeDesc);
+//        Cuboid baseCuboid = Cuboid.findById(cubeDesc, baseCuboidId);
+//        RowKeyEncoder rowKeyEncoder =
+//                (RowKeyEncoder) AbstractRowKeyEncoder.createInstance(cubeInstance.getTheOnlySegment(),
+//                        baseCuboid);
+//
+//        return rowKeyEncoder.getHeaderLength();
+//
+//    }
+//
+//    public static void main(String[] args) throws Exception {
+//        int exitCode = ToolRunner.run(new KeyDistributionJob(), args);
+//        System.exit(exitCode);
+//    }
+// }

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/job/src/main/java/com/kylinolap/job/hadoop/cube/KeyDistributionMapper.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/com/kylinolap/job/hadoop/cube/KeyDistributionMapper.java b/job/src/main/java/com/kylinolap/job/hadoop/cube/KeyDistributionMapper.java
new file mode 100644
index 0000000..037c3d9
--- /dev/null
+++ b/job/src/main/java/com/kylinolap/job/hadoop/cube/KeyDistributionMapper.java
@@ -0,0 +1,104 @@
+package com.kylinolap.job.hadoop.cube;
+
+///*
+// * 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.index.cube;
+//
+//import java.io.IOException;
+//
+//import org.apache.hadoop.io.LongWritable;
+//import org.apache.hadoop.io.Text;
+//import org.apache.hadoop.mapreduce.Mapper;
+//
+///**
+// * @author xjiang
+// *
+// */
+//public class KeyDistributionMapper extends Mapper<Text, Text, Text, LongWritable> {
+//
+//    private int headerLength;
+//
+//    private Text currentKey;
+//    private long outputLong;
+//    private Text outputKey;
+//    private LongWritable outputValue;
+//    private int columnPercentage;
+//    private int allRowCount;
+//
+//    @Override
+//    protected void setup(Context context) throws IOException {
+//        String percentStr = context.getConfiguration().get(KeyDistributionJob.KEY_COLUMN_PERCENTAGE);
+//        this.columnPercentage = Integer.valueOf(percentStr).intValue();
+//        if (this.columnPercentage <= 0 || this.columnPercentage >= 100) {
+//            this.columnPercentage = 20;
+//        }
+//        String headerLenStr = context.getConfiguration().get(KeyDistributionJob.KEY_HEADER_LENGTH);
+//        this.headerLength = Integer.valueOf(headerLenStr).intValue();
+//
+//        currentKey = new Text();
+//        outputLong = 0;
+//        outputKey = new Text();
+//        outputValue = new LongWritable(1);
+//        allRowCount = 0;
+//    }
+//
+//    @Override
+//    protected void cleanup(Context context) throws IOException, InterruptedException {
+//        emit(context); // emit the last holding record
+//
+//        byte[] zerokey = new byte[] { 0 };
+//        outputKey.set(zerokey);
+//        outputValue.set(allRowCount);
+//        context.write(outputKey, outputValue);
+//    }
+//
+//    @Override
+//    public void map(Text key, Text value, Context context) throws IOException, InterruptedException {
+//        byte[] bytes = key.getBytes();
+//        int columnLength = bytes.length - this.headerLength;
+//        int columnPrefixLen = columnLength * this.columnPercentage / 100;
+//        if (columnPrefixLen == 0 && columnLength > 0) {
+//            columnPrefixLen = 1;
+//        }
+//        if (columnPrefixLen > 0) {
+//            currentKey.set(bytes, 0, this.headerLength + columnPrefixLen);
+//        } else {
+//            currentKey.set(bytes);
+//        }
+//
+//        allRowCount++;
+//
+//        if (outputKey.getLength() == 0) { // first record
+//            outputKey.set(currentKey);
+//            outputLong = 1;
+//        } else if (outputKey.equals(currentKey)) { // same key, note input is sorted
+//            outputLong++;
+//        } else { // the next key
+//            emit(context);
+//            outputKey.set(currentKey);
+//            outputLong = 1;
+//        }
+//    }
+//
+//    private void emit(Context context) throws IOException, InterruptedException {
+//        if (outputLong == 0)
+//            return;
+//
+//        outputValue.set(outputLong);
+//        context.write(outputKey, outputValue);
+//    }
+// }

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/job/src/main/java/com/kylinolap/job/hadoop/cube/KeyDistributionReducer.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/com/kylinolap/job/hadoop/cube/KeyDistributionReducer.java b/job/src/main/java/com/kylinolap/job/hadoop/cube/KeyDistributionReducer.java
new file mode 100644
index 0000000..e0edc20
--- /dev/null
+++ b/job/src/main/java/com/kylinolap/job/hadoop/cube/KeyDistributionReducer.java
@@ -0,0 +1,93 @@
+package com.kylinolap.job.hadoop.cube;
+
+///*
+// * 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.index.cube;
+//
+//import java.io.IOException;
+//
+//import org.apache.hadoop.io.LongWritable;
+//import org.apache.hadoop.io.Text;
+//import org.apache.hadoop.mapreduce.Reducer;
+//import org.apache.hadoop.util.StringUtils;
+//import org.slf4j.Logger;
+//import org.slf4j.LoggerFactory;
+//
+///**
+// * @author xjiang
+// *
+// */
+//public class KeyDistributionReducer extends Reducer<Text, LongWritable, Text, LongWritable> {
+//
+//    private static final Logger logger = LoggerFactory.getLogger(KeyDistributionReducer.class);
+//
+//    private LongWritable outputValue;
+//    private boolean isTotalCount;
+//    private long totalCount;
+//    private int splitNumber;
+//    private long splitQuota;
+//    private long splitRemain;
+//
+//    @Override
+//    protected void setup(Context context) throws IOException, InterruptedException {
+//        String splitStr = context.getConfiguration().get(KeyDistributionJob.KEY_SPLIT_NUMBER);
+//        splitNumber = Integer.valueOf(splitStr).intValue();
+//        outputValue = new LongWritable();
+//        isTotalCount = true;
+//        totalCount = 0;
+//        splitQuota = 0;
+//        splitRemain = 0;
+//    }
+//
+//    @Override
+//    protected void cleanup(Context context) throws IOException, InterruptedException {
+//        logger.info("---------------");
+//        long splitCount = splitQuota - splitRemain;
+//        logger.info("Total Count = " + totalCount + ", Left Count = " + splitCount);
+//    }
+//
+//    @Override
+//    public void reduce(Text key, Iterable<LongWritable> values, Context context) throws IOException,
+//            InterruptedException {
+//
+//        // calculate split quota
+//        if (isTotalCount) {
+//            for (LongWritable count : values) {
+//                totalCount += count.get();
+//            }
+//            splitQuota = totalCount / splitNumber;
+//            splitRemain = splitQuota;
+//            isTotalCount = false;
+//            return;
+//        }
+//
+//        // output key when split quota is used up 
+//        for (LongWritable count : values) {
+//            splitRemain -= count.get();
+//        }
+//        if (splitRemain <= 0) {
+//            long splitCount = splitQuota - splitRemain;
+//            String hexKey = StringUtils.byteToHexString(key.getBytes());
+//            logger.info(hexKey + "\t\t" + splitCount);
+//
+//            outputValue.set(splitCount);
+//            context.write(key, outputValue);
+//            splitRemain = splitQuota;
+//        }
+//
+//    }
+// }

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/job/src/main/java/com/kylinolap/job/hadoop/cube/MergeCuboidJob.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/com/kylinolap/job/hadoop/cube/MergeCuboidJob.java b/job/src/main/java/com/kylinolap/job/hadoop/cube/MergeCuboidJob.java
new file mode 100644
index 0000000..d97e135
--- /dev/null
+++ b/job/src/main/java/com/kylinolap/job/hadoop/cube/MergeCuboidJob.java
@@ -0,0 +1,112 @@
+/*
+ * 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.job.hadoop.cube;
+
+import java.io.File;
+
+import org.apache.commons.cli.Options;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.mapreduce.Job;
+import org.apache.hadoop.mapreduce.lib.input.SequenceFileInputFormat;
+import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
+import org.apache.hadoop.mapreduce.lib.output.SequenceFileOutputFormat;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.kylinolap.common.KylinConfig;
+import com.kylinolap.cube.CubeInstance;
+import com.kylinolap.cube.CubeManager;
+import com.kylinolap.job.constant.BatchConstants;
+
+/**
+ * @author ysong1
+ */
+public class MergeCuboidJob extends CuboidJob {
+
+    private static final Logger log = LoggerFactory.getLogger(MergeCuboidJob.class);
+
+    @Override
+    public int run(String[] args) throws Exception {
+        Options options = new Options();
+
+        try {
+            options.addOption(OPTION_JOB_NAME);
+            options.addOption(OPTION_CUBE_NAME);
+            options.addOption(OPTION_SEGMENT_NAME);
+            options.addOption(OPTION_INPUT_PATH);
+            options.addOption(OPTION_OUTPUT_PATH);
+            parseOptions(options, args);
+
+            String cubeName = getOptionValue(OPTION_CUBE_NAME).toUpperCase();
+            String segmentName = getOptionValue(OPTION_SEGMENT_NAME).toUpperCase();
+            KylinConfig config = KylinConfig.getInstanceFromEnv();
+            CubeManager cubeMgr = CubeManager.getInstance(config);
+            CubeInstance cube = cubeMgr.getCube(cubeName);
+            // CubeSegment cubeSeg = cubeMgr.findSegment(cube, segmentName);
+
+            // start job
+            String jobName = getOptionValue(OPTION_JOB_NAME);
+            System.out.println("Starting: " + jobName);
+            job = Job.getInstance(getConf(), jobName);
+
+            // set job configuration - basic
+            File JarFile = new File(config.getKylinJobJarPath());
+            if (JarFile.exists()) {
+                job.setJar(config.getKylinJobJarPath());
+            } else {
+                job.setJarByClass(this.getClass());
+            }
+
+            // setJobJar(job);
+            addInputDirs(getOptionValue(OPTION_INPUT_PATH), job);
+
+            Path output = new Path(getOptionValue(OPTION_OUTPUT_PATH));
+            FileOutputFormat.setOutputPath(job, output);
+
+            // Mapper
+            job.setInputFormatClass(SequenceFileInputFormat.class);
+            job.setMapperClass(MergeCuboidMapper.class);
+            job.setMapOutputKeyClass(Text.class);
+            job.setMapOutputValueClass(Text.class);
+
+            // Reducer - only one
+            job.setReducerClass(CuboidReducer.class);
+            job.setOutputFormatClass(SequenceFileOutputFormat.class);
+            job.setOutputKeyClass(Text.class);
+            job.setOutputValueClass(Text.class);
+
+            // set job configuration
+            job.getConfiguration().set(BatchConstants.CFG_CUBE_NAME, cubeName);
+            job.getConfiguration().set(BatchConstants.CFG_CUBE_SEGMENT_NAME, segmentName);
+
+            // add metadata to distributed cache
+            attachKylinPropsAndMetadata(cube, job.getConfiguration());
+
+            setReduceTaskNum(job, config, cubeName, 0);
+
+            this.deletePath(job.getConfiguration(), output);
+
+            return waitForCompletion(job);
+        } catch (Exception e) {
+            printUsage(options);
+            log.error(e.getLocalizedMessage(), e);
+            return 2;
+        }
+    }
+
+}


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

Posted by lu...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/metadata/src/test/resources/data/TEST1_desc.json
----------------------------------------------------------------------
diff --git a/metadata/src/test/resources/data/TEST1_desc.json b/metadata/src/test/resources/data/TEST1_desc.json
new file mode 100644
index 0000000..175b105
--- /dev/null
+++ b/metadata/src/test/resources/data/TEST1_desc.json
@@ -0,0 +1,192 @@
+{
+  "uuid" : "9c53506d-d7b9-4ad4-b3b1-53ea42673c4b",
+  "last_modified" : 1401429176099,
+  "name" : "TEST1_desc",
+  "fact_table" : "TEST_KYLIN_FACT",
+  "dimensions" : [ {
+    "id" : 1,
+    "name" : "CAL_DT",
+    "join" : {
+      "type" : "inner",
+      "primary_key" : [ "CAL_DT" ],
+      "foreign_key" : [ "CAL_DT" ]
+    },
+    "hierarchy" : null,
+    "table" : "TEST_CAL_DT",
+    "column" : "CAL_DT",
+    "datatype" : "date",
+    "derived" : [ "WEEK_BEG_DT" ]
+  }, {
+    "id" : 2,
+    "name" : "CATEGORY",
+    "join" : {
+      "type" : "inner",
+      "primary_key" : [ "LEAF_CATEG_ID", "SITE_ID" ],
+      "foreign_key" : [ "LEAF_CATEG_ID", "LSTG_SITE_ID" ]
+    },
+    "hierarchy" : [ {
+      "level" : "1",
+      "column" : "META_CATEG_NAME"
+    }, {
+      "level" : "2",
+      "column" : "CATEG_LVL2_NAME"
+    }, {
+      "level" : "3",
+      "column" : "CATEG_LVL3_NAME"
+    } ],
+    "table" : "TEST_CATEGORY_GROUPINGS",
+    "column" : null,
+    "datatype" : null,
+    "derived" : null
+  }, {
+    "id" : 3,
+    "name" : "LSTG_FORMAT_NAME",
+    "join" : null,
+    "hierarchy" : null,
+    "table" : "TEST_KYLIN_FACT",
+    "column" : "LSTG_FORMAT_NAME",
+    "datatype" : "string",
+    "derived" : null
+  }, {
+    "id" : 4,
+    "name" : "SITE_ID",
+    "join" : {
+      "type" : "inner",
+      "primary_key" : [ "SITE_ID" ],
+      "foreign_key" : [ "LSTG_SITE_ID" ]
+    },
+    "hierarchy" : null,
+    "table" : "TEST_SITES",
+    "column" : "SITE_ID",
+    "datatype" : "string",
+    "derived" : [ "SITE_NAME", "CRE_USER" ]
+  }, {
+    "id" : 5,
+    "name" : "SELLER_TYPE_CD",
+    "join" : {
+      "type" : "inner",
+      "primary_key" : [ "SELLER_TYPE_CD" ],
+      "foreign_key" : [ "SLR_SEGMENT_CD" ]
+    },
+    "hierarchy" : null,
+    "table" : "TEST_SELLER_TYPE_DIM",
+    "column" : "SELLER_TYPE_CD",
+    "datatype" : "string",
+    "derived" : [ "SELLER_TYPE_DESC" ]
+  } ],
+  "measures" : [ {
+    "id" : 1,
+    "name" : "GMV_SUM",
+    "function" : {
+      "expression" : "SUM",
+      "parameter" : {
+        "type" : "column",
+        "value" : "PRICE"
+      },
+      "returntype" : "decimal"
+    }
+  }, {
+    "id" : 2,
+    "name" : "GMV_MIN",
+    "function" : {
+      "expression" : "MIN",
+      "parameter" : {
+        "type" : "column",
+        "value" : "PRICE"
+      },
+      "returntype" : "decimal"
+    }
+  }, {
+    "id" : 3,
+    "name" : "GMV_MAX",
+    "function" : {
+      "expression" : "MAX",
+      "parameter" : {
+        "type" : "column",
+        "value" : "PRICE"
+      },
+      "returntype" : "decimal"
+    }
+  }, {
+    "id" : 4,
+    "name" : "TRANS_CNT",
+    "function" : {
+      "expression" : "COUNT",
+      "parameter" : {
+        "type" : "constant",
+        "value" : "1"
+      },
+      "returntype" : "long"
+    }
+  }, {
+    "id" : 5,
+    "name" : "SELLER_CNT",
+    "function" : {
+      "expression" : "COUNT_DISTINCT",
+      "parameter" : {
+        "type" : "column",
+        "value" : "SELLER_ID"
+      },
+      "returntype" : "hllc10"
+    }
+  }, {
+    "id" : 6,
+    "name" : "SELLER_FORMAT_CNT",
+    "function" : {
+      "expression" : "COUNT_DISTINCT",
+      "parameter" : {
+        "type" : "column",
+        "value" : "LSTG_FORMAT_NAME,SELLER_ID"
+      },
+      "returntype" : "hllc10"
+    }
+  } ],
+  "rowkey" : {
+    "rowkey_columns" : [ {
+      "column" : "CAL_DT",
+      "length" : 0,
+      "dictionary" : "date(yyyy-mm-dd)",
+      "mandatory" : true
+    }, {
+      "column" : "META_CATEG_NAME",
+      "length" : 0,
+      "dictionary" : "string",
+      "mandatory" : false
+    }, {
+      "column" : "CATEG_LVL2_NAME",
+      "length" : 0,
+      "dictionary" : "string",
+      "mandatory" : false
+    }, {
+      "column" : "CATEG_LVL3_NAME",
+      "length" : 0,
+      "dictionary" : "string",
+      "mandatory" : false
+    }, {
+      "column" : "LSTG_FORMAT_NAME",
+      "length" : 12,
+      "dictionary" : null,
+      "mandatory" : false
+    }, {
+      "column" : "SITE_ID",
+      "length" : 0,
+      "dictionary" : "string",
+      "mandatory" : false
+    }, {
+      "column" : "SELLER_TYPE_CD",
+      "length" : 0,
+      "dictionary" : "string",
+      "mandatory" : false
+    } ],
+    "aggregation_groups" : [ [ "META_CATEG_NAME", "CATEG_LVL3_NAME", "CATEG_LVL2_NAME", "CAL_DT" ], [ "LSTG_FORMAT_NAME", "SITE_ID", "SELLER_TYPE_CD" ] ]
+  },
+  "hbase_mapping" : {
+    "column_family" : [ {
+      "name" : "F1",
+      "columns" : [ {
+        "qualifier" : "M",
+        "measure_refs" : [ "GMV_SUM", "GMV_MIN", "GMV_MAX", "TRANS_CNT", "SELLER_CNT", "SELLER_FORMAT_CNT" ]
+      } ]
+    } ]
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/metadata/src/test/resources/data/TEST2_desc.json
----------------------------------------------------------------------
diff --git a/metadata/src/test/resources/data/TEST2_desc.json b/metadata/src/test/resources/data/TEST2_desc.json
new file mode 100644
index 0000000..81bde27
--- /dev/null
+++ b/metadata/src/test/resources/data/TEST2_desc.json
@@ -0,0 +1,192 @@
+{
+  "uuid" : "9c53506d-d7b9-4ad4-b3b1-53ea42673c4b",
+  "last_modified" : 1401429176099,
+  "name" : "TEST2_desc",
+  "fact_table" : "TEST_KYLIN_FACT",
+  "dimensions" : [ {
+    "id" : 1,
+    "name" : "CAL_DT",
+    "join" : {
+      "type" : "inner",
+      "primary_key" : [ "CAL_DT" ],
+      "foreign_key" : [ "CAL_DT" ]
+    },
+    "hierarchy" : null,
+    "table" : "TEST_CAL_DT",
+    "column" : "CAL_DT",
+    "datatype" : "date",
+    "derived" : [ "WEEK_BEG_DT" ]
+  }, {
+    "id" : 2,
+    "name" : "CATEGORY",
+    "join" : {
+      "type" : "inner",
+      "primary_key" : [ "LEAF_CATEG_ID", "SITE_ID" ],
+      "foreign_key" : [ "LEAF_CATEG_ID", "LSTG_SITE_ID" ]
+    },
+    "hierarchy" : [ {
+      "level" : "1",
+      "column" : "META_CATEG_NAME"
+    }, {
+      "level" : "2",
+      "column" : "CATEG_LVL2_NAME"
+    }, {
+      "level" : "3",
+      "column" : "CATEG_LVL3_NAME"
+    } ],
+    "table" : "TEST_CATEGORY_GROUPINGS",
+    "column" : null,
+    "datatype" : null,
+    "derived" : null
+  }, {
+    "id" : 3,
+    "name" : "LSTG_FORMAT_NAME",
+    "join" : null,
+    "hierarchy" : null,
+    "table" : "TEST_KYLIN_FACT",
+    "column" : "LSTG_FORMAT_NAME",
+    "datatype" : "string",
+    "derived" : null
+  }, {
+    "id" : 4,
+    "name" : "SITE_ID",
+    "join" : {
+      "type" : "inner",
+      "primary_key" : [ "SITE_ID" ],
+      "foreign_key" : [ "LSTG_SITE_ID" ]
+    },
+    "hierarchy" : null,
+    "table" : "TEST_SITES",
+    "column" : "SITE_ID",
+    "datatype" : "string",
+    "derived" : [ "SITE_NAME", "CRE_USER" ]
+  }, {
+    "id" : 5,
+    "name" : "SELLER_TYPE_CD",
+    "join" : {
+      "type" : "inner",
+      "primary_key" : [ "SELLER_TYPE_CD" ],
+      "foreign_key" : [ "SLR_SEGMENT_CD" ]
+    },
+    "hierarchy" : null,
+    "table" : "TEST_SELLER_TYPE_DIM",
+    "column" : "SELLER_TYPE_CD",
+    "datatype" : "string",
+    "derived" : [ "SELLER_TYPE_DESC" ]
+  } ],
+  "measures" : [ {
+    "id" : 1,
+    "name" : "GMV_SUM",
+    "function" : {
+      "expression" : "SUM",
+      "parameter" : {
+        "type" : "column",
+        "value" : "PRICE"
+      },
+      "returntype" : "decimal"
+    }
+  }, {
+    "id" : 2,
+    "name" : "GMV_MIN",
+    "function" : {
+      "expression" : "MIN",
+      "parameter" : {
+        "type" : "column",
+        "value" : "PRICE"
+      },
+      "returntype" : "decimal"
+    }
+  }, {
+    "id" : 3,
+    "name" : "GMV_MAX",
+    "function" : {
+      "expression" : "MAX",
+      "parameter" : {
+        "type" : "column",
+        "value" : "PRICE"
+      },
+      "returntype" : "decimal"
+    }
+  }, {
+    "id" : 4,
+    "name" : "TRANS_CNT",
+    "function" : {
+      "expression" : "COUNT",
+      "parameter" : {
+        "type" : "constant",
+        "value" : "1"
+      },
+      "returntype" : "long"
+    }
+  }, {
+    "id" : 5,
+    "name" : "SELLER_CNT",
+    "function" : {
+      "expression" : "COUNT_DISTINCT",
+      "parameter" : {
+        "type" : "column",
+        "value" : "SELLER_ID"
+      },
+      "returntype" : "hllc10"
+    }
+  }, {
+    "id" : 6,
+    "name" : "SELLER_FORMAT_CNT",
+    "function" : {
+      "expression" : "COUNT_DISTINCT",
+      "parameter" : {
+        "type" : "column",
+        "value" : "LSTG_FORMAT_NAME,SELLER_ID"
+      },
+      "returntype" : "hllc10"
+    }
+  } ],
+  "rowkey" : {
+    "rowkey_columns" : [ {
+      "column" : "CAL_DT",
+      "length" : 0,
+      "dictionary" : "date(yyyy-mm-dd)",
+      "mandatory" : true
+    }, {
+      "column" : "META_CATEG_NAME",
+      "length" : 0,
+      "dictionary" : "string",
+      "mandatory" : false
+    }, {
+      "column" : "CATEG_LVL2_NAME",
+      "length" : 0,
+      "dictionary" : "string",
+      "mandatory" : false
+    }, {
+      "column" : "CATEG_LVL3_NAME",
+      "length" : 0,
+      "dictionary" : "string",
+      "mandatory" : false
+    }, {
+      "column" : "LSTG_FORMAT_NAME",
+      "length" : 12,
+      "dictionary" : null,
+      "mandatory" : false
+    }, {
+      "column" : "SITE_ID",
+      "length" : 0,
+      "dictionary" : "string",
+      "mandatory" : false
+    }, {
+      "column" : "SELLER_TYPE_CD",
+      "length" : 0,
+      "dictionary" : "string",
+      "mandatory" : false
+    } ],
+    "aggregation_groups" : [ [ "META_CATEG_NAME", "CATEG_LVL3_NAME", "CAL_DT" ], [ "LSTG_FORMAT_NAME", "SITE_ID", "SELLER_TYPE_CD", "CATEG_LVL2_NAME" ] ]
+  },
+  "hbase_mapping" : {
+    "column_family" : [ {
+      "name" : "F1",
+      "columns" : [ {
+        "qualifier" : "M",
+        "measure_refs" : [ "GMV_SUM", "GMV_MIN", "GMV_MAX", "TRANS_CNT", "SELLER_CNT", "SELLER_FORMAT_CNT" ]
+      } ]
+    } ]
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/metadata/src/test/resources/data/TEST3_desc.json
----------------------------------------------------------------------
diff --git a/metadata/src/test/resources/data/TEST3_desc.json b/metadata/src/test/resources/data/TEST3_desc.json
new file mode 100644
index 0000000..df4d60b
--- /dev/null
+++ b/metadata/src/test/resources/data/TEST3_desc.json
@@ -0,0 +1,192 @@
+{
+  "uuid" : "9c53506d-d7b9-4ad4-b3b1-53ea42673c4b",
+  "last_modified" : 1401429176099,
+  "name" : "TEST1_desc",
+  "fact_table" : "TEST_KYLIN_FACT",
+  "dimensions" : [ {
+    "id" : 1,
+    "name" : "CAL_DT",
+    "join" : {
+      "type" : "inner",
+      "primary_key" : [ "CAL_DT" ],
+      "foreign_key" : [ "CAL_DT" ]
+    },
+    "hierarchy" : null,
+    "table" : "TEST_CAL_DT",
+    "column" : "CAL_DT",
+    "datatype" : "date",
+    "derived" : [ "WEEK_BEG_DT" ]
+  }, {
+    "id" : 2,
+    "name" : "CATEGORY",
+    "join" : {
+      "type" : "inner",
+      "primary_key" : [ "LEAF_CATEG_ID", "SITE_ID" ],
+      "foreign_key" : [ "LEAF_CATEG_ID", "LSTG_SITE_ID" ]
+    },
+    "hierarchy" : [ {
+      "level" : "1",
+      "column" : "META_CATEG_NAME"
+    }, {
+      "level" : "2",
+      "column" : "CATEG_LVL2_NAME"
+    }, {
+      "level" : "3",
+      "column" : "CATEG_LVL3_NAME"
+    } ],
+    "table" : "TEST_CATEGORY_GROUPINGS",
+    "column" : null,
+    "datatype" : null,
+    "derived" : null
+  }, {
+    "id" : 3,
+    "name" : "LSTG_FORMAT_NAME",
+    "join" : null,
+    "hierarchy" : null,
+    "table" : "TEST_KYLIN_FACT",
+    "column" : "LSTG_FORMAT_NAME",
+    "datatype" : "string",
+    "derived" : null
+  }, {
+    "id" : 4,
+    "name" : "SITE_ID",
+    "join" : {
+      "type" : "inner",
+      "primary_key" : [ "SITE_ID" ],
+      "foreign_key" : [ "LSTG_SITE_ID" ]
+    },
+    "hierarchy" : null,
+    "table" : "TEST_SITES",
+    "column" : "SITE_ID",
+    "datatype" : "string",
+    "derived" : [ "SITE_NAME", "CRE_USER" ]
+  }, {
+    "id" : 5,
+    "name" : "SELLER_TYPE_CD",
+    "join" : {
+      "type" : "inner",
+      "primary_key" : [ "SELLER_TYPE_CD" ],
+      "foreign_key" : [ "SLR_SEGMENT_CD" ]
+    },
+    "hierarchy" : null,
+    "table" : "TEST_SELLER_TYPE_DIM",
+    "column" : "SELLER_TYPE_CD",
+    "datatype" : "string",
+    "derived" : [ "SELLER_TYPE_DESC" ]
+  } ],
+  "measures" : [ {
+    "id" : 1,
+    "name" : "GMV_SUM",
+    "function" : {
+      "expression" : "SUM",
+      "parameter" : {
+        "type" : "column",
+        "value" : "PRICE"
+      },
+      "returntype" : "decimal"
+    }
+  }, {
+    "id" : 2,
+    "name" : "GMV_MIN",
+    "function" : {
+      "expression" : "MIN",
+      "parameter" : {
+        "type" : "column",
+        "value" : "PRICE"
+      },
+      "returntype" : "decimal"
+    }
+  }, {
+    "id" : 3,
+    "name" : "GMV_MAX",
+    "function" : {
+      "expression" : "MAX",
+      "parameter" : {
+        "type" : "column",
+        "value" : "PRICE"
+      },
+      "returntype" : "decimal"
+    }
+  }, {
+    "id" : 4,
+    "name" : "TRANS_CNT",
+    "function" : {
+      "expression" : "COUNT",
+      "parameter" : {
+        "type" : "constant",
+        "value" : "1"
+      },
+      "returntype" : "long"
+    }
+  }, {
+    "id" : 5,
+    "name" : "SELLER_CNT",
+    "function" : {
+      "expression" : "COUNT_DISTINCT",
+      "parameter" : {
+        "type" : "column",
+        "value" : "SELLER_ID"
+      },
+      "returntype" : "hllc10"
+    }
+  }, {
+    "id" : 6,
+    "name" : "SELLER_FORMAT_CNT",
+    "function" : {
+      "expression" : "COUNT_DISTINCT",
+      "parameter" : {
+        "type" : "column",
+        "value" : "LSTG_FORMAT_NAME,SELLER_ID"
+      },
+      "returntype" : "hllc10"
+    }
+  } ],
+  "rowkey" : {
+    "rowkey_columns" : [ {
+      "column" : "CAL_DT",
+      "length" : 10,
+      "dictionary" : "date(yyyy-mm-dd)",
+      "mandatory" : false
+    }, {
+      "column" : "META_CATEG_NAME",
+      "length" : 0,
+      "dictionary" : "string",
+      "mandatory" : false
+    }, {
+      "column" : "CATEG_LVL2_NAME",
+      "length" : 0,
+      "dictionary" : "string",
+      "mandatory" : false
+    }, {
+      "column" : "CATEG_LVL3_NAME",
+      "length" : 0,
+      "dictionary" : "string",
+      "mandatory" : false
+    }, {
+      "column" : "LSTG_FORMAT_NAME",
+      "length" : 12,
+      "dictionary" : null,
+      "mandatory" : false
+    }, {
+      "column" : "SITE_ID",
+      "length" : 0,
+      "dictionary" : "string",
+      "mandatory" : false
+    }, {
+      "column" : "SELLER_TYPE_CD",
+      "length" : 0,
+      "dictionary" : "string",
+      "mandatory" : false
+    } ],
+    "aggregation_groups" : [ [ "META_CATEG_NAME", "CATEG_LVL3_NAME", "CATEG_LVL2_NAME", "CAL_DT" ], [ "LSTG_FORMAT_NAME", "SITE_ID", "SELLER_TYPE_CD" ] ]
+  },
+  "hbase_mapping" : {
+    "column_family" : [ {
+      "name" : "F1",
+      "columns" : [ {
+        "qualifier" : "M",
+        "measure_refs" : [ "GMV_SUM", "GMV_MIN", "GMV_MAX", "TRANS_CNT", "SELLER_CNT", "SELLER_FORMAT_CNT" ]
+      } ]
+    } ]
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/metadata/src/test/test.json
----------------------------------------------------------------------
diff --git a/metadata/src/test/test.json b/metadata/src/test/test.json
new file mode 100644
index 0000000..9811a33
--- /dev/null
+++ b/metadata/src/test/test.json
@@ -0,0 +1,181 @@
+{"id": 2, "name": "STAR_ITEM_CUBE", "status": "active", "fact_table": {
+    "name": "ITEM",
+    "type": "fact",
+    "database": "EDW"
+}, "dimensions": [
+    {
+        "id": 1,
+        "name": "CALENDAR",
+        "column": null,
+        "join": {
+            "type": "inner",
+            "primary_key": [
+                {
+                    "id": null,
+                    "name": "ID",
+                    "datatype": null,
+                    "table": null
+                }
+            ],
+            "foreign_key": [
+                {
+                    "id": null,
+                    "name": "CALENDAR_ID",
+                    "datatype": null,
+                    "table": null
+                }
+            ]
+        },
+        "hierarchy": [
+            {
+                "level": "1",
+                "column": {
+                    "id": null,
+                    "name": "CAL_YEAR",
+                    "datatype": "string",
+                    "table": {
+                        "name": "CALENDAR",
+                        "type": "lookup",
+                        "database": null
+                    }
+                }
+            }
+        ],
+        "table": {
+            "name": "CALENDAR",
+            "type": "lookup",
+            "database": null
+        },
+        "datatype": "string"
+    },
+    {
+        "id": 2,
+        "name": "SITE",
+        "column": {
+            "id": null,
+            "name": "SITE",
+            "datatype": "string",
+            "table": {
+                "name": "SITE",
+                "type": "lookup",
+                "database": null
+            }
+        },
+        "join": {
+            "type": "inner",
+            "primary_key": [
+                {
+                    "id": null,
+                    "name": "ID",
+                    "datatype": null,
+                    "table": null
+                }
+            ],
+            "foreign_key": [
+                {
+                    "id": null,
+                    "name": "SITE_ID",
+                    "datatype": null,
+                    "table": null
+                }
+            ]
+        },
+        "hierarchy": null,
+        "table": {
+            "name": "SITE",
+            "type": "lookup",
+            "database": null
+        },
+        "datatype": "string"
+    },
+    {
+        "id": 3,
+        "name": "CATEGORY",
+        "column": {
+            "id": null,
+            "name": "CATEGORY",
+            "datatype": "string",
+            "table": {
+                "name": "CATEGORY",
+                "type": "lookup",
+                "database": null
+            }
+        },
+        "join": {
+            "type": "inner",
+            "primary_key": [
+                {
+                    "id": null,
+                    "name": "ID",
+                    "datatype": null,
+                    "table": null
+                }
+            ],
+            "foreign_key": [
+                {
+                    "id": null,
+                    "name": "CATEGORY_ID",
+                    "datatype": null,
+                    "table": null
+                }
+            ]
+        },
+        "hierarchy": null,
+        "table": {
+            "name": "CATEGORY",
+            "type": "lookup",
+            "database": null
+        },
+        "datatype": "string"
+    }
+], "measures": [
+    {
+        "id": 1,
+        "name": "USDAMTSUM",
+        "function": {
+            "expression": "SUM",
+            "parameter": {
+                "type": "column",
+                "value": "USD_AMT"
+            },
+            "returntype": "double"
+        }
+    },
+    {
+        "id": 2,
+        "name": "ITEMCOUNT",
+        "function": {
+            "expression": "SUM",
+            "parameter": {
+                "type": "column",
+                "value": "LISTING_CNT"
+            },
+            "returntype": "long"
+        }
+    }
+], "hbase_mapping": {
+    "type": "full_materialization",
+    "hbase_table": {
+        "name": "SAMPLE_CUBE",
+        "type": "hbase_table",
+        "database": "DEFAULT"
+    },
+    "rowkey_order": ["CAL_YEAR", "SITE", "CATEGORY"],
+    "rowkey_length": ["4", "2", "4"],
+    "high_cardinality_columns": null,
+    "column_family": [
+        {
+            "name": "CF1",
+            "columns": [
+                {
+                    "qualifier": "USD_AMT",
+                    "measure_refs": ["USDAMTSUM"]
+                },
+                {
+                    "qualifier": "ITEM_COUNT",
+                    "measure_refs": ["ITEMCOUNT"]
+                }
+            ]
+        }
+    ]
+}}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/package.sh
----------------------------------------------------------------------
diff --git a/package.sh b/package.sh
new file mode 100644
index 0000000..9a17430
--- /dev/null
+++ b/package.sh
@@ -0,0 +1,19 @@
+pwd
+
+mvn clean
+mvn install -DskipTests
+
+#Copy war to kylin.war
+ls -t server/target/*.war| head -1 | awk '{print "cp " $1 " server/target/kylin.war"}' | sh
+#Copy index jar
+ls -t job/target/*-job.jar| head -1 | awk '{print "cp " $1 " job/target/kylin-job-latest.jar"}' | sh
+#Copy query jar
+ls -t storage/target/*-coprocessor.jar | head -1 | awk '{print "cp " $1 " storage/target/kylin-coprocessor-latest.jar"}' | sh
+
+#package webapp
+cd webapp/
+npm install
+npm install -g grunt-cli
+grunt dev --buildEnv=dev
+cd dist
+tar -cvf Web.tar * .htaccess

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
new file mode 100644
index 0000000..c38a8cc
--- /dev/null
+++ b/pom.xml
@@ -0,0 +1,542 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/maven-v4_0_0.xsd">
+    <modelVersion>4.0.0</modelVersion>
+    <groupId>com.kylinolap</groupId>
+    <artifactId>kylin</artifactId>
+    <packaging>pom</packaging>
+    <version>0.6.3-SNAPSHOT</version>
+    <name>Kylin:HadoopOLAPEngine</name>
+
+    <properties>
+        <!-- General Properties -->
+        <javaVersion>1.7</javaVersion>
+        <maven-model.version>3.1.1</maven-model.version>
+         <maven-release.version>2.5.1</maven-release.version>
+        <project.build.sourceEncoding>UTF-8</project.build.sourceEncoding>
+        <project.reporting.outputEncoding>UTF-8</project.reporting.outputEncoding>
+
+        <!-- Hadoop versions -->
+        <hadoop2.version>2.4.1</hadoop2.version>
+        <yarn.version>2.4.1</yarn.version>
+        <hbase-hadoop2.version>0.98.0-hadoop2</hbase-hadoop2.version>
+        <zookeeper.version>3.4.5</zookeeper.version>
+        <hive.version>0.13.0</hive.version>
+
+        <!-- Dependency versions -->
+        <antlr.version>3.4</antlr.version>
+        <junit.version>4.11</junit.version>
+        <mrunit.version>1.0.0</mrunit.version>
+        <dbunit.version>2.5.0</dbunit.version>
+        <h2.version>1.3.174</h2.version>
+
+        <!-- Commons -->
+        <commons-cli.version>1.2</commons-cli.version>
+        <commons-lang.version>2.6</commons-lang.version>
+        <commons-lang3.version>3.1</commons-lang3.version>
+        <commons-io.version>2.4</commons-io.version>
+        <commons-configuration.version>1.9</commons-configuration.version>
+        <commons-daemon.version>1.0.15</commons-daemon.version>
+        <commons-httpclient.version>3.1</commons-httpclient.version>
+
+        <!-- Utility -->
+        <log4j.version>1.2.17</log4j.version>
+        <slf4j.version>1.6.4</slf4j.version>
+        <jackson.version>2.2.3</jackson.version>
+        <guava.version>12.0.1</guava.version>
+        <jsch.version>0.1.51</jsch.version>
+        <xerces.version>2.9.1</xerces.version>
+        <xalan.version>2.7.1</xalan.version>
+        <compress-lzf.version>0.8.4</compress-lzf.version>
+        <extendedset.version>1.3.4</extendedset.version>
+
+        <!-- REST Service -->
+        <spring.framework.version>3.1.2.RELEASE</spring.framework.version>
+
+        <!-- Optiq Version -->
+        <optiq.version>0.9.1-incubating</optiq.version>
+        <linq4j.version>0.4</linq4j.version>
+
+        <!-- Metrics Codahale Version -->
+        <metrics.version>3.0.1</metrics.version>
+
+        <!-- Quartz Version -->
+        <quartz.version>2.2.1</quartz.version>
+
+        <!-- Curator.version Version -->
+        <curator.version>2.6.0</curator.version>
+
+        <!-- Sonar -->
+        <sonar.java.coveragePlugin>jacoco</sonar.java.coveragePlugin>
+        <sonar.dynamicAnalysis>reuseReports</sonar.dynamicAnalysis>
+        <sonar.jacoco.reportPath>${project.basedir}/../target/jacoco.exec</sonar.jacoco.reportPath>
+        <sonar.language>java</sonar.language>
+        <sonar.jacoco.excludes>com/kylinolap/**/tools/**:net/hydromatic/optiq/**:org/eigenbase/sql2rel/**</sonar.jacoco.excludes>
+    </properties>
+
+    <dependencyManagement>
+        <dependencies>
+            <!-- Hadoop2 dependencies -->
+            <dependency>
+                <groupId>org.apache.hadoop</groupId>
+                <artifactId>hadoop-common</artifactId>
+                <version>${hadoop2.version}</version>
+                <scope>compile</scope>
+                <exclusions>
+                    <exclusion>
+                        <groupId>javax.servlet</groupId>
+                        <artifactId>servlet-api</artifactId>
+                    </exclusion>
+                </exclusions>
+            </dependency>
+            <dependency>
+                <groupId>org.apache.hadoop</groupId>
+                <artifactId>hadoop-hdfs</artifactId>
+                <version>${hadoop2.version}</version>
+            </dependency>
+            <dependency>
+                <groupId>org.apache.hadoop</groupId>
+                <artifactId>hadoop-mapreduce-client-app</artifactId>
+                <version>${hadoop2.version}</version>
+            </dependency>
+            <dependency>
+                <groupId>org.apache.hadoop</groupId>
+                <artifactId>hadoop-yarn-api</artifactId>
+                <version>${hadoop2.version}</version>
+            </dependency>
+            <dependency>
+                <groupId>org.apache.hadoop</groupId>
+                <artifactId>hadoop-mapreduce-client-core</artifactId>
+                <version>${hadoop2.version}</version>
+            </dependency>
+            <dependency>
+                <groupId>org.apache.hadoop</groupId>
+                <artifactId>hadoop-mapreduce-client-jobclient</artifactId>
+                <version>${hadoop2.version}</version>
+            </dependency>
+            <dependency>
+                <groupId>org.apache.hadoop</groupId>
+                <artifactId>hadoop-annotations</artifactId>
+                <version>${hadoop2.version}</version>
+            </dependency>
+            <dependency>
+                <groupId>org.apache.hadoop</groupId>
+                <artifactId>hadoop-auth</artifactId>
+                <version>${hadoop2.version}</version>
+            </dependency>
+            <dependency>
+                <groupId>org.apache.hadoop</groupId>
+                <artifactId>hadoop-minicluster</artifactId>
+                <version>${hadoop2.version}</version>
+                <optional>true</optional>
+            </dependency>
+
+            <!-- HBase2 dependencies -->
+            <dependency>
+                <groupId>org.apache.hbase</groupId>
+                <artifactId>hbase-hadoop2-compat</artifactId>
+                <version>${hbase-hadoop2.version}</version>
+            </dependency>
+            <dependency>
+                <groupId>org.apache.hbase</groupId>
+                <artifactId>hbase-common</artifactId>
+                <version>${hbase-hadoop2.version}</version>
+            </dependency>
+            <dependency>
+                <groupId>org.apache.hbase</groupId>
+                <artifactId>hbase-client</artifactId>
+                <version>${hbase-hadoop2.version}</version>
+            </dependency>
+            <dependency>
+                <groupId>org.apache.hbase</groupId>
+                <artifactId>hbase-server</artifactId>
+                <version>${hbase-hadoop2.version}</version>
+            </dependency>
+            <dependency>
+                <groupId>org.apache.mrunit</groupId>
+                <artifactId>mrunit</artifactId>
+                <version>${mrunit.version}</version>
+                <classifier>hadoop2</classifier>
+            </dependency>
+
+            <!-- Hive dependencies -->
+            <dependency>
+                <groupId>org.apache.hive</groupId>
+                <artifactId>hive-jdbc</artifactId>
+                <version>${hive.version}</version>
+            </dependency>
+
+            <!-- Yarn dependencies -->
+            <dependency>
+                <groupId>org.apache.hadoop</groupId>
+                <artifactId>hadoop-yarn-server-resourcemanager</artifactId>
+                <version>${yarn.version}</version>
+            </dependency>
+
+            <!-- optiq dependencies -->
+            <dependency>
+                <groupId>org.apache.calcite</groupId>
+                <artifactId>calcite-core</artifactId>
+                <version>${optiq.version}</version>
+            </dependency>
+            <dependency>
+                <groupId>org.apache.calcite</groupId>
+                <artifactId>calcite-avatica</artifactId>
+                <version>${optiq.version}</version>
+            </dependency>
+            <dependency>
+                <groupId>net.hydromatic</groupId>
+                <artifactId>linq4j</artifactId>
+                <version>${linq4j.version}</version>
+            </dependency>
+
+            <!-- Other dependencies -->
+            <dependency>
+                <groupId>junit</groupId>
+                <artifactId>junit</artifactId>
+                <version>${junit.version}</version>
+            </dependency>
+            <dependency>
+                <groupId>org.apache.zookeeper</groupId>
+                <artifactId>zookeeper</artifactId>
+                <version>${zookeeper.version}</version>
+            </dependency>
+            <dependency>
+                <groupId>commons-cli</groupId>
+                <artifactId>commons-cli</artifactId>
+                <version>${commons-cli.version}</version>
+            </dependency>
+            <dependency>
+                <groupId>commons-lang</groupId>
+                <artifactId>commons-lang</artifactId>
+                <version>${commons-lang.version}</version>
+            </dependency>
+            <dependency>
+                <groupId>org.apache.commons</groupId>
+                <artifactId>commons-lang3</artifactId>
+                <version>${commons-lang3.version}</version>
+            </dependency>
+            <dependency>
+                <groupId>commons-io</groupId>
+                <artifactId>commons-io</artifactId>
+                <version>${commons-io.version}</version>
+            </dependency>
+            <dependency>
+                <groupId>commons-configuration</groupId>
+                <artifactId>commons-configuration</artifactId>
+                <version>${commons-configuration.version}</version>
+            </dependency>
+            <dependency>
+                <groupId>commons-daemon</groupId>
+                <artifactId>commons-daemon</artifactId>
+                <version>${commons-daemon.version}</version>
+            </dependency>
+            <dependency>
+                <groupId>log4j</groupId>
+                <artifactId>log4j</artifactId>
+                <version>${log4j.version}</version>
+            </dependency>
+            <dependency>
+                <groupId>org.slf4j</groupId>
+                <artifactId>jcl-over-slf4j</artifactId>
+                <version>${slf4j.version}</version>
+            </dependency>
+            <dependency>
+                <groupId>org.slf4j</groupId>
+                <artifactId>slf4j-api</artifactId>
+                <version>${slf4j.version}</version>
+            </dependency>
+            <dependency>
+                <groupId>org.slf4j</groupId>
+                <artifactId>slf4j-log4j12</artifactId>
+                <version>${slf4j.version}</version>
+            </dependency>
+            <dependency>
+                <groupId>com.fasterxml.jackson.core</groupId>
+                <artifactId>jackson-databind</artifactId>
+                <version>${jackson.version}</version>
+            </dependency>
+            <dependency>
+                <groupId>com.fasterxml.jackson.core</groupId>
+                <artifactId>jackson-core</artifactId>
+                <version>${jackson.version}</version>
+            </dependency>
+            <dependency>
+                <groupId>commons-httpclient</groupId>
+                <artifactId>commons-httpclient</artifactId>
+                <version>${commons-httpclient.version}</version>
+            </dependency>
+            <dependency>
+                <groupId>com.google.guava</groupId>
+                <artifactId>guava</artifactId>
+                <version>${guava.version}</version>
+            </dependency>
+            <dependency>
+                <groupId>com.jcraft</groupId>
+                <artifactId>jsch</artifactId>
+                <version>${jsch.version}</version>
+            </dependency>
+            <dependency>
+                <groupId>org.dbunit</groupId>
+                <artifactId>dbunit</artifactId>
+                <version>${dbunit.version}</version>
+            </dependency>
+            <dependency>
+                <groupId>org.apache.maven</groupId>
+                <artifactId>maven-model</artifactId>
+                <version>${maven-model.version}</version>
+            </dependency>
+            <dependency>
+                <groupId>com.h2database</groupId>
+                <artifactId>h2</artifactId>
+                <version>${h2.version}</version>
+            </dependency>
+            <dependency>
+                <groupId>xerces</groupId>
+                <artifactId>xercesImpl</artifactId>
+                <version>${xerces.version}</version>
+            </dependency>
+            <dependency>
+                <groupId>xalan</groupId>
+                <artifactId>xalan</artifactId>
+                <version>${xalan.version}</version>
+            </dependency>
+            <dependency>
+                <groupId>com.ning</groupId>
+                <artifactId>compress-lzf</artifactId>
+                <version>${compress-lzf.version}</version>
+            </dependency>
+            <dependency>
+                <groupId>com.n3twork.druid</groupId>
+                <artifactId>extendedset</artifactId>
+                <version>${extendedset.version}</version>
+            </dependency>
+
+            <dependency>
+                <groupId>org.quartz-scheduler</groupId>
+                <artifactId>quartz</artifactId>
+                <version>${quartz.version}</version>
+            </dependency>
+            <dependency>
+                <groupId>org.quartz-scheduler</groupId>
+                <artifactId>quartz-jobs</artifactId>
+                <version>${quartz.version}</version>
+            </dependency>
+
+            <dependency>
+                <groupId>org.apache.curator</groupId>
+                <artifactId>curator-framework</artifactId>
+                <version>${curator.version}</version>
+            </dependency>
+            <dependency>
+                <groupId>org.apache.curator</groupId>
+                <artifactId>curator-recipes</artifactId>
+                <version>${curator.version}</version>
+            </dependency>
+
+        </dependencies>
+    </dependencyManagement>
+
+    <repositories>
+        <repository>
+            <id>central</id>
+            <url>http://repo.maven.apache.org/maven2</url>
+        </repository>
+        <repository>
+            <id>conjars</id>
+            <url>http://conjars.org/repo/</url>
+        </repository>
+        <repository>
+            <releases>
+                <enabled>true</enabled>
+                <updatePolicy>always</updatePolicy>
+                <checksumPolicy>warn</checksumPolicy>
+            </releases>
+            <snapshots>
+                <enabled>false</enabled>
+                <updatePolicy>never</updatePolicy>
+                <checksumPolicy>fail</checksumPolicy>
+            </snapshots>
+            <id>HDPReleases</id>
+            <name>HDP Releases</name>
+            <url>http://repo.hortonworks.com/content/repositories/releases/</url>
+            <layout>default</layout>
+        </repository>
+    </repositories>
+
+    <build>
+        <pluginManagement>
+            <plugins>
+                <plugin>
+                    <groupId>org.apache.maven.plugins</groupId>
+                    <artifactId>maven-compiler-plugin</artifactId>
+                    <version>3.1</version>
+                    <configuration>
+                        <source>${javaVersion}</source>
+                        <target>${javaVersion}</target>
+                    </configuration>
+                </plugin>
+                <plugin>
+                    <groupId>org.apache.maven.plugins</groupId>
+                    <artifactId>maven-site-plugin</artifactId>
+                    <version>2.0</version>
+                </plugin>
+                <plugin>
+                    <groupId>org.apache.maven.plugins</groupId>
+                    <artifactId>maven-install-plugin</artifactId>
+                    <version>2.2</version>
+                </plugin>
+                <plugin>
+                    <groupId>org.apache.maven.plugins</groupId>
+                    <artifactId>maven-resources-plugin</artifactId>
+                    <version>2.4</version>
+                    <configuration>
+                        <encoding>UTF-8</encoding>
+                    </configuration>
+                </plugin>
+                <plugin>
+                    <groupId>org.apache.maven.plugins</groupId>
+                    <artifactId>maven-jar-plugin</artifactId>
+                    <version>2.4</version>
+                    <executions>
+                        <execution>
+                            <goals>
+                                <goal>test-jar</goal>
+                            </goals>
+                        </execution>
+                    </executions>
+                </plugin>
+                <plugin>
+                    <groupId>org.apache.maven.plugins</groupId>
+                    <artifactId>maven-war-plugin</artifactId>
+                    <version>2.0.2</version>
+                </plugin>
+                <plugin>
+                    <groupId>org.apache.maven.plugins</groupId>
+                    <artifactId>maven-dependency-plugin</artifactId>
+                </plugin>
+                <plugin>
+                    <groupId>org.apache.maven.plugins</groupId>
+                    <artifactId>maven-antrun-plugin</artifactId>
+                </plugin>
+                <plugin>
+                    <groupId>org.apache.maven.plugins</groupId>
+                    <artifactId>maven-source-plugin</artifactId>
+                    <version>2.1.2</version>
+                    <executions>
+                        <execution>
+                            <id>attach-sources</id>
+                            <phase>package</phase>
+                            <goals>
+                                <goal>jar-no-fork</goal>
+                            </goals>
+                        </execution>
+                    </executions>
+                    <configuration>
+                        <includePom>true</includePom>
+                    </configuration>
+                </plugin>
+                <plugin>
+                    <groupId>org.antlr</groupId>
+                    <artifactId>antlr3-maven-plugin</artifactId>
+                    <version>${antlr.version}</version>
+                    <executions>
+                        <execution>
+                            <configuration>
+                                <goals>
+                                    <goal>antlr</goal>
+                                </goals>
+                                <debug>false</debug>
+                            </configuration>
+                        </execution>
+                    </executions>
+                </plugin>
+                <plugin>
+                    <groupId>org.apache.maven.plugins</groupId>
+                    <artifactId>maven-failsafe-plugin</artifactId>
+                    <version>2.6</version>
+                    <executions>
+                        <execution>
+                            <goals>
+                                <goal>integration-test</goal>
+                                <goal>verify</goal>
+                            </goals>
+                        </execution>
+                    </executions>
+                </plugin>
+                <plugin>
+                    <groupId>org.apache.maven.plugins</groupId>
+                    <artifactId>maven-release-plugin</artifactId>
+                    <version>${maven-release.version}</version>
+                </plugin>
+                <plugin>
+                    <groupId>org.jacoco</groupId>
+                    <artifactId>jacoco-maven-plugin</artifactId>
+                    <version>0.7.0.201403182114</version>
+                    <configuration>
+                        <destFile>${sonar.jacoco.reportPath}</destFile>
+                        <append>true</append>
+                    </configuration>
+                    <executions>
+                        <execution>
+                            <id>agent</id>
+                            <goals>
+                                <goal>prepare-agent</goal>
+                            </goals>
+                        </execution>
+                    </executions>
+                </plugin>
+            </plugins>
+        </pluginManagement>
+    </build>
+
+    <scm>
+        <connection>scm:git:git@github.com:KylinOLAP/Kylin.git</connection>
+        <url>scm:git:git@github.com:KylinOLAP/Kylin.git</url>
+        <developerConnection>scm:git:git@github.com:KylinOLAP/Kylin.git</developerConnection>
+      <tag>HEAD</tag>
+  </scm>
+
+    <modules>
+        <module>atopcalcite</module>
+        <module>common</module>
+        <module>metadata</module>
+        <module>dictionary</module>
+        <module>cube</module>
+        <module>job</module>
+        <module>storage</module>
+        <module>query</module>
+        <module>server</module>
+		<module>jdbc</module>
+    </modules>
+    <profiles>
+        <profile>
+            <id>default</id>
+            <activation>
+                <activeByDefault>true</activeByDefault>
+            </activation>
+            <build>
+                <plugins>
+                    <plugin>
+                        <groupId>org.apache.maven.plugins</groupId>
+                        <artifactId>maven-surefire-plugin</artifactId>
+                        <version>2.16</version>
+                        <configuration>
+                            <excludes>
+                                <exclude>**/com/kylinolap/job/**</exclude>
+                                <exclude>**/BuildCubeWithEngineTest.java</exclude>
+                                <exclude>**/SampleCubeSetupTest.java</exclude>
+                                <exclude>**/InvertedIndexHBaseTest.java</exclude>
+                                <exclude>**/StorageTest.java</exclude>
+                                <exclude>**/KylinQueryTest.java</exclude>
+                            </excludes>
+                        </configuration>
+                    </plugin>
+                </plugins>
+            </build>
+        </profile>
+
+        <profile>
+            <id>e2e</id>
+        </profile>
+    </profiles>
+</project>

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/query/.settings/org.eclipse.core.resources.prefs
----------------------------------------------------------------------
diff --git a/query/.settings/org.eclipse.core.resources.prefs b/query/.settings/org.eclipse.core.resources.prefs
new file mode 100644
index 0000000..04cfa2c
--- /dev/null
+++ b/query/.settings/org.eclipse.core.resources.prefs
@@ -0,0 +1,6 @@
+eclipse.preferences.version=1
+encoding//src/main/java=UTF-8
+encoding//src/main/resources=UTF-8
+encoding//src/test/java=UTF-8
+encoding//src/test/resources=UTF-8
+encoding/<project>=UTF-8

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/query/.settings/org.eclipse.jdt.core.prefs
----------------------------------------------------------------------
diff --git a/query/.settings/org.eclipse.jdt.core.prefs b/query/.settings/org.eclipse.jdt.core.prefs
new file mode 100644
index 0000000..a903301
--- /dev/null
+++ b/query/.settings/org.eclipse.jdt.core.prefs
@@ -0,0 +1,379 @@
+eclipse.preferences.version=1
+org.eclipse.jdt.core.compiler.annotation.inheritNullAnnotations=disabled
+org.eclipse.jdt.core.compiler.annotation.missingNonNullByDefaultAnnotation=ignore
+org.eclipse.jdt.core.compiler.annotation.nonnull=org.eclipse.jdt.annotation.NonNull
+org.eclipse.jdt.core.compiler.annotation.nonnullbydefault=org.eclipse.jdt.annotation.NonNullByDefault
+org.eclipse.jdt.core.compiler.annotation.nullable=org.eclipse.jdt.annotation.Nullable
+org.eclipse.jdt.core.compiler.annotation.nullanalysis=disabled
+org.eclipse.jdt.core.compiler.codegen.inlineJsrBytecode=enabled
+org.eclipse.jdt.core.compiler.codegen.targetPlatform=1.7
+org.eclipse.jdt.core.compiler.codegen.unusedLocal=preserve
+org.eclipse.jdt.core.compiler.compliance=1.7
+org.eclipse.jdt.core.compiler.debug.lineNumber=generate
+org.eclipse.jdt.core.compiler.debug.localVariable=generate
+org.eclipse.jdt.core.compiler.debug.sourceFile=generate
+org.eclipse.jdt.core.compiler.problem.annotationSuperInterface=warning
+org.eclipse.jdt.core.compiler.problem.assertIdentifier=error
+org.eclipse.jdt.core.compiler.problem.autoboxing=ignore
+org.eclipse.jdt.core.compiler.problem.comparingIdentical=warning
+org.eclipse.jdt.core.compiler.problem.deadCode=warning
+org.eclipse.jdt.core.compiler.problem.deprecation=warning
+org.eclipse.jdt.core.compiler.problem.deprecationInDeprecatedCode=disabled
+org.eclipse.jdt.core.compiler.problem.deprecationWhenOverridingDeprecatedMethod=disabled
+org.eclipse.jdt.core.compiler.problem.discouragedReference=warning
+org.eclipse.jdt.core.compiler.problem.emptyStatement=ignore
+org.eclipse.jdt.core.compiler.problem.enumIdentifier=error
+org.eclipse.jdt.core.compiler.problem.explicitlyClosedAutoCloseable=ignore
+org.eclipse.jdt.core.compiler.problem.fallthroughCase=ignore
+org.eclipse.jdt.core.compiler.problem.fatalOptionalError=disabled
+org.eclipse.jdt.core.compiler.problem.fieldHiding=ignore
+org.eclipse.jdt.core.compiler.problem.finalParameterBound=warning
+org.eclipse.jdt.core.compiler.problem.finallyBlockNotCompletingNormally=warning
+org.eclipse.jdt.core.compiler.problem.forbiddenReference=warning
+org.eclipse.jdt.core.compiler.problem.hiddenCatchBlock=warning
+org.eclipse.jdt.core.compiler.problem.includeNullInfoFromAsserts=disabled
+org.eclipse.jdt.core.compiler.problem.incompatibleNonInheritedInterfaceMethod=warning
+org.eclipse.jdt.core.compiler.problem.incompleteEnumSwitch=warning
+org.eclipse.jdt.core.compiler.problem.indirectStaticAccess=ignore
+org.eclipse.jdt.core.compiler.problem.localVariableHiding=ignore
+org.eclipse.jdt.core.compiler.problem.methodWithConstructorName=warning
+org.eclipse.jdt.core.compiler.problem.missingDefaultCase=ignore
+org.eclipse.jdt.core.compiler.problem.missingDeprecatedAnnotation=ignore
+org.eclipse.jdt.core.compiler.problem.missingEnumCaseDespiteDefault=disabled
+org.eclipse.jdt.core.compiler.problem.missingHashCodeMethod=ignore
+org.eclipse.jdt.core.compiler.problem.missingOverrideAnnotation=ignore
+org.eclipse.jdt.core.compiler.problem.missingOverrideAnnotationForInterfaceMethodImplementation=enabled
+org.eclipse.jdt.core.compiler.problem.missingSerialVersion=warning
+org.eclipse.jdt.core.compiler.problem.missingSynchronizedOnInheritedMethod=ignore
+org.eclipse.jdt.core.compiler.problem.noEffectAssignment=warning
+org.eclipse.jdt.core.compiler.problem.noImplicitStringConversion=warning
+org.eclipse.jdt.core.compiler.problem.nonExternalizedStringLiteral=ignore
+org.eclipse.jdt.core.compiler.problem.nonnullParameterAnnotationDropped=warning
+org.eclipse.jdt.core.compiler.problem.nullAnnotationInferenceConflict=error
+org.eclipse.jdt.core.compiler.problem.nullReference=warning
+org.eclipse.jdt.core.compiler.problem.nullSpecViolation=error
+org.eclipse.jdt.core.compiler.problem.nullUncheckedConversion=warning
+org.eclipse.jdt.core.compiler.problem.overridingPackageDefaultMethod=warning
+org.eclipse.jdt.core.compiler.problem.parameterAssignment=ignore
+org.eclipse.jdt.core.compiler.problem.possibleAccidentalBooleanAssignment=ignore
+org.eclipse.jdt.core.compiler.problem.potentialNullReference=ignore
+org.eclipse.jdt.core.compiler.problem.potentiallyUnclosedCloseable=ignore
+org.eclipse.jdt.core.compiler.problem.rawTypeReference=warning
+org.eclipse.jdt.core.compiler.problem.redundantNullAnnotation=warning
+org.eclipse.jdt.core.compiler.problem.redundantNullCheck=ignore
+org.eclipse.jdt.core.compiler.problem.redundantSpecificationOfTypeArguments=ignore
+org.eclipse.jdt.core.compiler.problem.redundantSuperinterface=ignore
+org.eclipse.jdt.core.compiler.problem.reportMethodCanBePotentiallyStatic=ignore
+org.eclipse.jdt.core.compiler.problem.reportMethodCanBeStatic=ignore
+org.eclipse.jdt.core.compiler.problem.specialParameterHidingField=disabled
+org.eclipse.jdt.core.compiler.problem.staticAccessReceiver=warning
+org.eclipse.jdt.core.compiler.problem.suppressOptionalErrors=disabled
+org.eclipse.jdt.core.compiler.problem.suppressWarnings=enabled
+org.eclipse.jdt.core.compiler.problem.syntacticNullAnalysisForFields=disabled
+org.eclipse.jdt.core.compiler.problem.syntheticAccessEmulation=ignore
+org.eclipse.jdt.core.compiler.problem.typeParameterHiding=warning
+org.eclipse.jdt.core.compiler.problem.unavoidableGenericTypeProblems=enabled
+org.eclipse.jdt.core.compiler.problem.uncheckedTypeOperation=warning
+org.eclipse.jdt.core.compiler.problem.unclosedCloseable=warning
+org.eclipse.jdt.core.compiler.problem.undocumentedEmptyBlock=ignore
+org.eclipse.jdt.core.compiler.problem.unhandledWarningToken=warning
+org.eclipse.jdt.core.compiler.problem.unnecessaryElse=ignore
+org.eclipse.jdt.core.compiler.problem.unnecessaryTypeCheck=ignore
+org.eclipse.jdt.core.compiler.problem.unqualifiedFieldAccess=ignore
+org.eclipse.jdt.core.compiler.problem.unusedDeclaredThrownException=ignore
+org.eclipse.jdt.core.compiler.problem.unusedDeclaredThrownExceptionExemptExceptionAndThrowable=enabled
+org.eclipse.jdt.core.compiler.problem.unusedDeclaredThrownExceptionIncludeDocCommentReference=enabled
+org.eclipse.jdt.core.compiler.problem.unusedDeclaredThrownExceptionWhenOverriding=disabled
+org.eclipse.jdt.core.compiler.problem.unusedImport=warning
+org.eclipse.jdt.core.compiler.problem.unusedLabel=warning
+org.eclipse.jdt.core.compiler.problem.unusedLocal=warning
+org.eclipse.jdt.core.compiler.problem.unusedObjectAllocation=ignore
+org.eclipse.jdt.core.compiler.problem.unusedParameter=ignore
+org.eclipse.jdt.core.compiler.problem.unusedParameterIncludeDocCommentReference=enabled
+org.eclipse.jdt.core.compiler.problem.unusedParameterWhenImplementingAbstract=disabled
+org.eclipse.jdt.core.compiler.problem.unusedParameterWhenOverridingConcrete=disabled
+org.eclipse.jdt.core.compiler.problem.unusedPrivateMember=warning
+org.eclipse.jdt.core.compiler.problem.unusedTypeParameter=ignore
+org.eclipse.jdt.core.compiler.problem.unusedWarningToken=warning
+org.eclipse.jdt.core.compiler.problem.varargsArgumentNeedCast=warning
+org.eclipse.jdt.core.compiler.source=1.7
+org.eclipse.jdt.core.formatter.align_type_members_on_columns=false
+org.eclipse.jdt.core.formatter.alignment_for_arguments_in_allocation_expression=16
+org.eclipse.jdt.core.formatter.alignment_for_arguments_in_annotation=0
+org.eclipse.jdt.core.formatter.alignment_for_arguments_in_enum_constant=16
+org.eclipse.jdt.core.formatter.alignment_for_arguments_in_explicit_constructor_call=16
+org.eclipse.jdt.core.formatter.alignment_for_arguments_in_method_invocation=16
+org.eclipse.jdt.core.formatter.alignment_for_arguments_in_qualified_allocation_expression=16
+org.eclipse.jdt.core.formatter.alignment_for_assignment=0
+org.eclipse.jdt.core.formatter.alignment_for_binary_expression=16
+org.eclipse.jdt.core.formatter.alignment_for_compact_if=16
+org.eclipse.jdt.core.formatter.alignment_for_conditional_expression=80
+org.eclipse.jdt.core.formatter.alignment_for_enum_constants=0
+org.eclipse.jdt.core.formatter.alignment_for_expressions_in_array_initializer=16
+org.eclipse.jdt.core.formatter.alignment_for_method_declaration=0
+org.eclipse.jdt.core.formatter.alignment_for_multiple_fields=16
+org.eclipse.jdt.core.formatter.alignment_for_parameters_in_constructor_declaration=16
+org.eclipse.jdt.core.formatter.alignment_for_parameters_in_method_declaration=16
+org.eclipse.jdt.core.formatter.alignment_for_resources_in_try=80
+org.eclipse.jdt.core.formatter.alignment_for_selector_in_method_invocation=16
+org.eclipse.jdt.core.formatter.alignment_for_superclass_in_type_declaration=16
+org.eclipse.jdt.core.formatter.alignment_for_superinterfaces_in_enum_declaration=16
+org.eclipse.jdt.core.formatter.alignment_for_superinterfaces_in_type_declaration=16
+org.eclipse.jdt.core.formatter.alignment_for_throws_clause_in_constructor_declaration=16
+org.eclipse.jdt.core.formatter.alignment_for_throws_clause_in_method_declaration=16
+org.eclipse.jdt.core.formatter.alignment_for_union_type_in_multicatch=16
+org.eclipse.jdt.core.formatter.blank_lines_after_imports=1
+org.eclipse.jdt.core.formatter.blank_lines_after_package=1
+org.eclipse.jdt.core.formatter.blank_lines_before_field=0
+org.eclipse.jdt.core.formatter.blank_lines_before_first_class_body_declaration=0
+org.eclipse.jdt.core.formatter.blank_lines_before_imports=1
+org.eclipse.jdt.core.formatter.blank_lines_before_member_type=1
+org.eclipse.jdt.core.formatter.blank_lines_before_method=1
+org.eclipse.jdt.core.formatter.blank_lines_before_new_chunk=1
+org.eclipse.jdt.core.formatter.blank_lines_before_package=0
+org.eclipse.jdt.core.formatter.blank_lines_between_import_groups=1
+org.eclipse.jdt.core.formatter.blank_lines_between_type_declarations=1
+org.eclipse.jdt.core.formatter.brace_position_for_annotation_type_declaration=end_of_line
+org.eclipse.jdt.core.formatter.brace_position_for_anonymous_type_declaration=end_of_line
+org.eclipse.jdt.core.formatter.brace_position_for_array_initializer=end_of_line
+org.eclipse.jdt.core.formatter.brace_position_for_block=end_of_line
+org.eclipse.jdt.core.formatter.brace_position_for_block_in_case=end_of_line
+org.eclipse.jdt.core.formatter.brace_position_for_constructor_declaration=end_of_line
+org.eclipse.jdt.core.formatter.brace_position_for_enum_constant=end_of_line
+org.eclipse.jdt.core.formatter.brace_position_for_enum_declaration=end_of_line
+org.eclipse.jdt.core.formatter.brace_position_for_method_declaration=end_of_line
+org.eclipse.jdt.core.formatter.brace_position_for_switch=end_of_line
+org.eclipse.jdt.core.formatter.brace_position_for_type_declaration=end_of_line
+org.eclipse.jdt.core.formatter.comment.clear_blank_lines_in_block_comment=false
+org.eclipse.jdt.core.formatter.comment.clear_blank_lines_in_javadoc_comment=false
+org.eclipse.jdt.core.formatter.comment.format_block_comments=false
+org.eclipse.jdt.core.formatter.comment.format_header=false
+org.eclipse.jdt.core.formatter.comment.format_html=true
+org.eclipse.jdt.core.formatter.comment.format_javadoc_comments=false
+org.eclipse.jdt.core.formatter.comment.format_line_comments=false
+org.eclipse.jdt.core.formatter.comment.format_source_code=true
+org.eclipse.jdt.core.formatter.comment.indent_parameter_description=true
+org.eclipse.jdt.core.formatter.comment.indent_root_tags=true
+org.eclipse.jdt.core.formatter.comment.insert_new_line_before_root_tags=insert
+org.eclipse.jdt.core.formatter.comment.insert_new_line_for_parameter=insert
+org.eclipse.jdt.core.formatter.comment.line_length=80
+org.eclipse.jdt.core.formatter.comment.new_lines_at_block_boundaries=true
+org.eclipse.jdt.core.formatter.comment.new_lines_at_javadoc_boundaries=true
+org.eclipse.jdt.core.formatter.comment.preserve_white_space_between_code_and_line_comments=false
+org.eclipse.jdt.core.formatter.compact_else_if=true
+org.eclipse.jdt.core.formatter.continuation_indentation=2
+org.eclipse.jdt.core.formatter.continuation_indentation_for_array_initializer=2
+org.eclipse.jdt.core.formatter.disabling_tag=@formatter\:off
+org.eclipse.jdt.core.formatter.enabling_tag=@formatter\:on
+org.eclipse.jdt.core.formatter.format_guardian_clause_on_one_line=false
+org.eclipse.jdt.core.formatter.format_line_comment_starting_on_first_column=true
+org.eclipse.jdt.core.formatter.indent_body_declarations_compare_to_annotation_declaration_header=true
+org.eclipse.jdt.core.formatter.indent_body_declarations_compare_to_enum_constant_header=true
+org.eclipse.jdt.core.formatter.indent_body_declarations_compare_to_enum_declaration_header=true
+org.eclipse.jdt.core.formatter.indent_body_declarations_compare_to_type_header=true
+org.eclipse.jdt.core.formatter.indent_breaks_compare_to_cases=true
+org.eclipse.jdt.core.formatter.indent_empty_lines=false
+org.eclipse.jdt.core.formatter.indent_statements_compare_to_block=true
+org.eclipse.jdt.core.formatter.indent_statements_compare_to_body=true
+org.eclipse.jdt.core.formatter.indent_switchstatements_compare_to_cases=true
+org.eclipse.jdt.core.formatter.indent_switchstatements_compare_to_switch=false
+org.eclipse.jdt.core.formatter.indentation.size=4
+org.eclipse.jdt.core.formatter.insert_new_line_after_annotation_on_field=insert
+org.eclipse.jdt.core.formatter.insert_new_line_after_annotation_on_local_variable=insert
+org.eclipse.jdt.core.formatter.insert_new_line_after_annotation_on_method=insert
+org.eclipse.jdt.core.formatter.insert_new_line_after_annotation_on_package=insert
+org.eclipse.jdt.core.formatter.insert_new_line_after_annotation_on_parameter=do not insert
+org.eclipse.jdt.core.formatter.insert_new_line_after_annotation_on_type=insert
+org.eclipse.jdt.core.formatter.insert_new_line_after_label=do not insert
+org.eclipse.jdt.core.formatter.insert_new_line_after_opening_brace_in_array_initializer=do not insert
+org.eclipse.jdt.core.formatter.insert_new_line_at_end_of_file_if_missing=do not insert
+org.eclipse.jdt.core.formatter.insert_new_line_before_catch_in_try_statement=do not insert
+org.eclipse.jdt.core.formatter.insert_new_line_before_closing_brace_in_array_initializer=do not insert
+org.eclipse.jdt.core.formatter.insert_new_line_before_else_in_if_statement=do not insert
+org.eclipse.jdt.core.formatter.insert_new_line_before_finally_in_try_statement=do not insert
+org.eclipse.jdt.core.formatter.insert_new_line_before_while_in_do_statement=do not insert
+org.eclipse.jdt.core.formatter.insert_new_line_in_empty_annotation_declaration=insert
+org.eclipse.jdt.core.formatter.insert_new_line_in_empty_anonymous_type_declaration=insert
+org.eclipse.jdt.core.formatter.insert_new_line_in_empty_block=insert
+org.eclipse.jdt.core.formatter.insert_new_line_in_empty_enum_constant=insert
+org.eclipse.jdt.core.formatter.insert_new_line_in_empty_enum_declaration=insert
+org.eclipse.jdt.core.formatter.insert_new_line_in_empty_method_body=insert
+org.eclipse.jdt.core.formatter.insert_new_line_in_empty_type_declaration=insert
+org.eclipse.jdt.core.formatter.insert_space_after_and_in_type_parameter=insert
+org.eclipse.jdt.core.formatter.insert_space_after_assignment_operator=insert
+org.eclipse.jdt.core.formatter.insert_space_after_at_in_annotation=do not insert
+org.eclipse.jdt.core.formatter.insert_space_after_at_in_annotation_type_declaration=do not insert
+org.eclipse.jdt.core.formatter.insert_space_after_binary_operator=insert
+org.eclipse.jdt.core.formatter.insert_space_after_closing_angle_bracket_in_type_arguments=insert
+org.eclipse.jdt.core.formatter.insert_space_after_closing_angle_bracket_in_type_parameters=insert
+org.eclipse.jdt.core.formatter.insert_space_after_closing_brace_in_block=insert
+org.eclipse.jdt.core.formatter.insert_space_after_closing_paren_in_cast=insert
+org.eclipse.jdt.core.formatter.insert_space_after_colon_in_assert=insert
+org.eclipse.jdt.core.formatter.insert_space_after_colon_in_case=insert
+org.eclipse.jdt.core.formatter.insert_space_after_colon_in_conditional=insert
+org.eclipse.jdt.core.formatter.insert_space_after_colon_in_for=insert
+org.eclipse.jdt.core.formatter.insert_space_after_colon_in_labeled_statement=insert
+org.eclipse.jdt.core.formatter.insert_space_after_comma_in_allocation_expression=insert
+org.eclipse.jdt.core.formatter.insert_space_after_comma_in_annotation=insert
+org.eclipse.jdt.core.formatter.insert_space_after_comma_in_array_initializer=insert
+org.eclipse.jdt.core.formatter.insert_space_after_comma_in_constructor_declaration_parameters=insert
+org.eclipse.jdt.core.formatter.insert_space_after_comma_in_constructor_declaration_throws=insert
+org.eclipse.jdt.core.formatter.insert_space_after_comma_in_enum_constant_arguments=insert
+org.eclipse.jdt.core.formatter.insert_space_after_comma_in_enum_declarations=insert
+org.eclipse.jdt.core.formatter.insert_space_after_comma_in_explicitconstructorcall_arguments=insert
+org.eclipse.jdt.core.formatter.insert_space_after_comma_in_for_increments=insert
+org.eclipse.jdt.core.formatter.insert_space_after_comma_in_for_inits=insert
+org.eclipse.jdt.core.formatter.insert_space_after_comma_in_method_declaration_parameters=insert
+org.eclipse.jdt.core.formatter.insert_space_after_comma_in_method_declaration_throws=insert
+org.eclipse.jdt.core.formatter.insert_space_after_comma_in_method_invocation_arguments=insert
+org.eclipse.jdt.core.formatter.insert_space_after_comma_in_multiple_field_declarations=insert
+org.eclipse.jdt.core.formatter.insert_space_after_comma_in_multiple_local_declarations=insert
+org.eclipse.jdt.core.formatter.insert_space_after_comma_in_parameterized_type_reference=insert
+org.eclipse.jdt.core.formatter.insert_space_after_comma_in_superinterfaces=insert
+org.eclipse.jdt.core.formatter.insert_space_after_comma_in_type_arguments=insert
+org.eclipse.jdt.core.formatter.insert_space_after_comma_in_type_parameters=insert
+org.eclipse.jdt.core.formatter.insert_space_after_ellipsis=insert
+org.eclipse.jdt.core.formatter.insert_space_after_opening_angle_bracket_in_parameterized_type_reference=do not insert
+org.eclipse.jdt.core.formatter.insert_space_after_opening_angle_bracket_in_type_arguments=do not insert
+org.eclipse.jdt.core.formatter.insert_space_after_opening_angle_bracket_in_type_parameters=do not insert
+org.eclipse.jdt.core.formatter.insert_space_after_opening_brace_in_array_initializer=insert
+org.eclipse.jdt.core.formatter.insert_space_after_opening_bracket_in_array_allocation_expression=do not insert
+org.eclipse.jdt.core.formatter.insert_space_after_opening_bracket_in_array_reference=do not insert
+org.eclipse.jdt.core.formatter.insert_space_after_opening_paren_in_annotation=do not insert
+org.eclipse.jdt.core.formatter.insert_space_after_opening_paren_in_cast=do not insert
+org.eclipse.jdt.core.formatter.insert_space_after_opening_paren_in_catch=do not insert
+org.eclipse.jdt.core.formatter.insert_space_after_opening_paren_in_constructor_declaration=do not insert
+org.eclipse.jdt.core.formatter.insert_space_after_opening_paren_in_enum_constant=do not insert
+org.eclipse.jdt.core.formatter.insert_space_after_opening_paren_in_for=do not insert
+org.eclipse.jdt.core.formatter.insert_space_after_opening_paren_in_if=do not insert
+org.eclipse.jdt.core.formatter.insert_space_after_opening_paren_in_method_declaration=do not insert
+org.eclipse.jdt.core.formatter.insert_space_after_opening_paren_in_method_invocation=do not insert
+org.eclipse.jdt.core.formatter.insert_space_after_opening_paren_in_parenthesized_expression=do not insert
+org.eclipse.jdt.core.formatter.insert_space_after_opening_paren_in_switch=do not insert
+org.eclipse.jdt.core.formatter.insert_space_after_opening_paren_in_synchronized=do not insert
+org.eclipse.jdt.core.formatter.insert_space_after_opening_paren_in_try=do not insert
+org.eclipse.jdt.core.formatter.insert_space_after_opening_paren_in_while=do not insert
+org.eclipse.jdt.core.formatter.insert_space_after_postfix_operator=do not insert
+org.eclipse.jdt.core.formatter.insert_space_after_prefix_operator=do not insert
+org.eclipse.jdt.core.formatter.insert_space_after_question_in_conditional=insert
+org.eclipse.jdt.core.formatter.insert_space_after_question_in_wildcard=do not insert
+org.eclipse.jdt.core.formatter.insert_space_after_semicolon_in_for=insert
+org.eclipse.jdt.core.formatter.insert_space_after_semicolon_in_try_resources=insert
+org.eclipse.jdt.core.formatter.insert_space_after_unary_operator=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_and_in_type_parameter=insert
+org.eclipse.jdt.core.formatter.insert_space_before_assignment_operator=insert
+org.eclipse.jdt.core.formatter.insert_space_before_at_in_annotation_type_declaration=insert
+org.eclipse.jdt.core.formatter.insert_space_before_binary_operator=insert
+org.eclipse.jdt.core.formatter.insert_space_before_closing_angle_bracket_in_parameterized_type_reference=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_closing_angle_bracket_in_type_arguments=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_closing_angle_bracket_in_type_parameters=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_closing_brace_in_array_initializer=insert
+org.eclipse.jdt.core.formatter.insert_space_before_closing_bracket_in_array_allocation_expression=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_closing_bracket_in_array_reference=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_closing_paren_in_annotation=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_closing_paren_in_cast=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_closing_paren_in_catch=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_closing_paren_in_constructor_declaration=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_closing_paren_in_enum_constant=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_closing_paren_in_for=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_closing_paren_in_if=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_closing_paren_in_method_declaration=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_closing_paren_in_method_invocation=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_closing_paren_in_parenthesized_expression=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_closing_paren_in_switch=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_closing_paren_in_synchronized=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_closing_paren_in_try=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_closing_paren_in_while=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_colon_in_assert=insert
+org.eclipse.jdt.core.formatter.insert_space_before_colon_in_case=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_colon_in_conditional=insert
+org.eclipse.jdt.core.formatter.insert_space_before_colon_in_default=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_colon_in_for=insert
+org.eclipse.jdt.core.formatter.insert_space_before_colon_in_labeled_statement=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_comma_in_allocation_expression=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_comma_in_annotation=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_comma_in_array_initializer=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_comma_in_constructor_declaration_parameters=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_comma_in_constructor_declaration_throws=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_comma_in_enum_constant_arguments=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_comma_in_enum_declarations=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_comma_in_explicitconstructorcall_arguments=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_comma_in_for_increments=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_comma_in_for_inits=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_comma_in_method_declaration_parameters=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_comma_in_method_declaration_throws=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_comma_in_method_invocation_arguments=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_comma_in_multiple_field_declarations=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_comma_in_multiple_local_declarations=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_comma_in_parameterized_type_reference=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_comma_in_superinterfaces=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_comma_in_type_arguments=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_comma_in_type_parameters=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_ellipsis=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_opening_angle_bracket_in_parameterized_type_reference=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_opening_angle_bracket_in_type_arguments=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_opening_angle_bracket_in_type_parameters=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_opening_brace_in_annotation_type_declaration=insert
+org.eclipse.jdt.core.formatter.insert_space_before_opening_brace_in_anonymous_type_declaration=insert
+org.eclipse.jdt.core.formatter.insert_space_before_opening_brace_in_array_initializer=insert
+org.eclipse.jdt.core.formatter.insert_space_before_opening_brace_in_block=insert
+org.eclipse.jdt.core.formatter.insert_space_before_opening_brace_in_constructor_declaration=insert
+org.eclipse.jdt.core.formatter.insert_space_before_opening_brace_in_enum_constant=insert
+org.eclipse.jdt.core.formatter.insert_space_before_opening_brace_in_enum_declaration=insert
+org.eclipse.jdt.core.formatter.insert_space_before_opening_brace_in_method_declaration=insert
+org.eclipse.jdt.core.formatter.insert_space_before_opening_brace_in_switch=insert
+org.eclipse.jdt.core.formatter.insert_space_before_opening_brace_in_type_declaration=insert
+org.eclipse.jdt.core.formatter.insert_space_before_opening_bracket_in_array_allocation_expression=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_opening_bracket_in_array_reference=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_opening_bracket_in_array_type_reference=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_opening_paren_in_annotation=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_opening_paren_in_annotation_type_member_declaration=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_opening_paren_in_catch=insert
+org.eclipse.jdt.core.formatter.insert_space_before_opening_paren_in_constructor_declaration=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_opening_paren_in_enum_constant=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_opening_paren_in_for=insert
+org.eclipse.jdt.core.formatter.insert_space_before_opening_paren_in_if=insert
+org.eclipse.jdt.core.formatter.insert_space_before_opening_paren_in_method_declaration=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_opening_paren_in_method_invocation=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_opening_paren_in_parenthesized_expression=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_opening_paren_in_switch=insert
+org.eclipse.jdt.core.formatter.insert_space_before_opening_paren_in_synchronized=insert
+org.eclipse.jdt.core.formatter.insert_space_before_opening_paren_in_try=insert
+org.eclipse.jdt.core.formatter.insert_space_before_opening_paren_in_while=insert
+org.eclipse.jdt.core.formatter.insert_space_before_parenthesized_expression_in_return=insert
+org.eclipse.jdt.core.formatter.insert_space_before_parenthesized_expression_in_throw=insert
+org.eclipse.jdt.core.formatter.insert_space_before_postfix_operator=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_prefix_operator=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_question_in_conditional=insert
+org.eclipse.jdt.core.formatter.insert_space_before_question_in_wildcard=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_semicolon=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_semicolon_in_for=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_semicolon_in_try_resources=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_unary_operator=do not insert
+org.eclipse.jdt.core.formatter.insert_space_between_brackets_in_array_type_reference=do not insert
+org.eclipse.jdt.core.formatter.insert_space_between_empty_braces_in_array_initializer=do not insert
+org.eclipse.jdt.core.formatter.insert_space_between_empty_brackets_in_array_allocation_expression=do not insert
+org.eclipse.jdt.core.formatter.insert_space_between_empty_parens_in_annotation_type_member_declaration=do not insert
+org.eclipse.jdt.core.formatter.insert_space_between_empty_parens_in_constructor_declaration=do not insert
+org.eclipse.jdt.core.formatter.insert_space_between_empty_parens_in_enum_constant=do not insert
+org.eclipse.jdt.core.formatter.insert_space_between_empty_parens_in_method_declaration=do not insert
+org.eclipse.jdt.core.formatter.insert_space_between_empty_parens_in_method_invocation=do not insert
+org.eclipse.jdt.core.formatter.join_lines_in_comments=true
+org.eclipse.jdt.core.formatter.join_wrapped_lines=true
+org.eclipse.jdt.core.formatter.keep_else_statement_on_same_line=false
+org.eclipse.jdt.core.formatter.keep_empty_array_initializer_on_one_line=false
+org.eclipse.jdt.core.formatter.keep_imple_if_on_one_line=false
+org.eclipse.jdt.core.formatter.keep_then_statement_on_same_line=false
+org.eclipse.jdt.core.formatter.lineSplit=999
+org.eclipse.jdt.core.formatter.never_indent_block_comments_on_first_column=false
+org.eclipse.jdt.core.formatter.never_indent_line_comments_on_first_column=false
+org.eclipse.jdt.core.formatter.number_of_blank_lines_at_beginning_of_method_body=0
+org.eclipse.jdt.core.formatter.number_of_empty_lines_to_preserve=1
+org.eclipse.jdt.core.formatter.put_empty_statement_on_new_line=true
+org.eclipse.jdt.core.formatter.tabulation.char=space
+org.eclipse.jdt.core.formatter.tabulation.size=4
+org.eclipse.jdt.core.formatter.use_on_off_tags=false
+org.eclipse.jdt.core.formatter.use_tabs_only_for_leading_indentations=false
+org.eclipse.jdt.core.formatter.wrap_before_binary_operator=true
+org.eclipse.jdt.core.formatter.wrap_before_or_operator_multicatch=true
+org.eclipse.jdt.core.formatter.wrap_outer_expressions_when_nested=true

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/query/.settings/org.eclipse.jdt.ui.prefs
----------------------------------------------------------------------
diff --git a/query/.settings/org.eclipse.jdt.ui.prefs b/query/.settings/org.eclipse.jdt.ui.prefs
new file mode 100644
index 0000000..dece0e6
--- /dev/null
+++ b/query/.settings/org.eclipse.jdt.ui.prefs
@@ -0,0 +1,7 @@
+eclipse.preferences.version=1
+formatter_profile=_Space Indent & Long Lines
+formatter_settings_version=12
+org.eclipse.jdt.ui.ignorelowercasenames=true
+org.eclipse.jdt.ui.importorder=java;javax;org;com;
+org.eclipse.jdt.ui.ondemandthreshold=99
+org.eclipse.jdt.ui.staticondemandthreshold=1

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/query/pom.xml
----------------------------------------------------------------------
diff --git a/query/pom.xml b/query/pom.xml
new file mode 100644
index 0000000..0f035b3
--- /dev/null
+++ b/query/pom.xml
@@ -0,0 +1,183 @@
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+    <modelVersion>4.0.0</modelVersion>
+    <!-- The basics. -->
+    <artifactId>kylin-query</artifactId>
+    <packaging>jar</packaging>
+    <name>Kylin:Query</name>
+    <description>kylin query engine based on optiq</description>
+
+    <parent>
+        <groupId>com.kylinolap</groupId>
+        <artifactId>kylin</artifactId>
+        <version>0.6.3-SNAPSHOT</version>
+    </parent>
+
+    <properties>
+    </properties>
+
+    <!-- Dependencies. -->
+    <dependencies>
+        <dependency>
+            <groupId>com.kylinolap</groupId>
+            <artifactId>atopcalcite</artifactId>
+            <version>${project.parent.version}</version>
+        </dependency>
+        <dependency>
+            <groupId>com.kylinolap</groupId>
+            <artifactId>kylin-storage</artifactId>
+            <version>${project.parent.version}</version>
+        </dependency>
+        <dependency>
+            <groupId>net.hydromatic</groupId>
+            <artifactId>linq4j</artifactId>
+        </dependency>
+        <dependency>
+            <groupId>log4j</groupId>
+            <artifactId>log4j</artifactId>
+        </dependency>
+        <dependency>
+            <groupId>org.slf4j</groupId>
+            <artifactId>jcl-over-slf4j</artifactId>
+        </dependency>
+        <dependency>
+            <groupId>org.slf4j</groupId>
+            <artifactId>slf4j-api</artifactId>
+        </dependency>
+        <dependency>
+            <groupId>commons-cli</groupId>
+            <artifactId>commons-cli</artifactId>
+        </dependency>
+
+        <!-- Env & Test -->
+
+        <dependency>
+            <groupId>xerces</groupId>
+            <artifactId>xercesImpl</artifactId>
+            <scope>test</scope>
+        </dependency>
+        <dependency>
+            <groupId>xalan</groupId>
+            <artifactId>xalan</artifactId>
+            <scope>test</scope>
+        </dependency>
+        <dependency>
+            <groupId>com.h2database</groupId>
+            <artifactId>h2</artifactId>
+            <scope>test</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.dbunit</groupId>
+            <artifactId>dbunit</artifactId>
+            <scope>test</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.hadoop</groupId>
+            <artifactId>hadoop-common</artifactId>
+            <scope>provided</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.hadoop</groupId>
+            <artifactId>hadoop-mapreduce-client-core</artifactId>
+            <scope>provided</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.hadoop</groupId>
+            <artifactId>hadoop-hdfs</artifactId>
+            <scope>provided</scope>
+            <!-- protobuf version conflict with hbase -->
+            <exclusions>
+                <exclusion>
+                    <groupId>com.google.protobuf</groupId>
+                    <artifactId>protobuf-java</artifactId>
+                </exclusion>
+            </exclusions>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.hbase</groupId>
+            <artifactId>hbase-client</artifactId>
+            <scope>provided</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.hbase</groupId>
+            <artifactId>hbase-server</artifactId>
+            <scope>provided</scope>
+            <!-- version conflict with hadoop2.2 -->
+            <exclusions>
+                <exclusion>
+                    <groupId>org.apache.hadoop</groupId>
+                    <artifactId>hadoop-mapreduce-client-jobclient</artifactId>
+                </exclusion>
+            </exclusions>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.hadoop</groupId>
+            <artifactId>hadoop-mapreduce-client-jobclient</artifactId>
+            <scope>provided</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.hive</groupId>
+            <artifactId>hive-jdbc</artifactId>
+            <scope>provided</scope>
+        </dependency>
+    </dependencies>
+
+    <build>
+        <plugins>
+            <!-- experiment "shade" instead of "assembly"
+            <plugin>
+                <groupId>org.apache.maven.plugins</groupId>
+                <artifactId>maven-shade-plugin</artifactId>
+                <version>2.2</version>
+                <executions>
+                    <execution>
+                        <phase>package</phase>
+                        <goals>
+                            <goal>shade</goal>
+                        </goals>
+                        <configuration>
+                            <shadedArtifactAttached>true</shadedArtifactAttached>
+                            <shadedClassifierName>jar-with-dependencies</shadedClassifierName>
+                            <transformers>
+                                <transformer implementation="org.apache.maven.plugins.shade.resource.ServicesResourceTransformer" />
+                            </transformers>
+                            <filters>
+                                <filter>
+                                    <artifact>*:*</artifact>
+                                    <excludes>
+                                        <exclude>META-INF/*.SF</exclude>
+                                        <exclude>META-INF/*.DSA</exclude>
+                                        <exclude>META-INF/*.RSA</exclude>
+                                    </excludes>
+                                </filter>
+                            </filters>
+                        </configuration>
+                    </execution>
+                </executions>
+            </plugin>
+            <plugin>
+                <artifactId>maven-assembly-plugin</artifactId>
+                <configuration>
+                    <archive>
+                        <manifest>
+                            <mainClass>fully.qualified.MainClass</mainClass>
+                        </manifest>
+                    </archive>
+                    <descriptorRefs>
+                        <descriptorRef>jar-with-dependencies</descriptorRef>
+                    </descriptorRefs>
+                </configuration>
+                <executions>
+                    <execution>
+                        <id>make-assembly</id>
+                        <phase>package</phase>
+                        <goals>
+                            <goal>single</goal>
+                        </goals>
+                    </execution>
+                </executions>
+            </plugin>
+             -->
+        </plugins>
+    </build>
+
+</project>

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/query/src/main/java/com/kylinolap/query/QueryCli.java
----------------------------------------------------------------------
diff --git a/query/src/main/java/com/kylinolap/query/QueryCli.java b/query/src/main/java/com/kylinolap/query/QueryCli.java
new file mode 100644
index 0000000..a476628
--- /dev/null
+++ b/query/src/main/java/com/kylinolap/query/QueryCli.java
@@ -0,0 +1,86 @@
+/*
+ * 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.query;
+
+import java.io.File;
+import java.sql.Connection;
+import java.sql.DriverManager;
+import java.sql.ResultSet;
+import java.sql.ResultSetMetaData;
+import java.sql.Statement;
+
+import org.apache.commons.cli.CommandLine;
+import org.apache.commons.cli.CommandLineParser;
+import org.apache.commons.cli.GnuParser;
+import org.apache.commons.cli.Option;
+import org.apache.commons.cli.OptionBuilder;
+import org.apache.commons.cli.Options;
+
+import com.kylinolap.common.KylinConfig;
+import com.kylinolap.query.schema.OLAPSchemaFactory;
+
+public class QueryCli {
+
+    @SuppressWarnings("static-access")
+    private static final Option OPTION_METADATA = OptionBuilder.withArgName("metadata url").hasArg().isRequired().withDescription("Metadata URL").create("metadata");
+
+    @SuppressWarnings("static-access")
+    private static final Option OPTION_SQL = OptionBuilder.withArgName("input sql").hasArg().isRequired().withDescription("SQL").create("sql");
+
+    public static void main(String[] args) throws Exception {
+
+        Options options = new Options();
+        options.addOption(OPTION_METADATA);
+        options.addOption(OPTION_SQL);
+
+        CommandLineParser parser = new GnuParser();
+        CommandLine commandLine = parser.parse(options, args);
+        KylinConfig config = KylinConfig.createInstanceFromUri(commandLine.getOptionValue(OPTION_METADATA.getOpt()));
+        String sql = commandLine.getOptionValue(OPTION_SQL.getOpt());
+
+        Class.forName("net.hydromatic.optiq.jdbc.Driver");
+        File olapTmp = OLAPSchemaFactory.createTempOLAPJson(null, config);
+
+        Connection conn = null;
+        Statement stmt = null;
+        ResultSet rs = null;
+        try {
+            conn = DriverManager.getConnection("jdbc:calcite:model=" + olapTmp.getAbsolutePath());
+
+            stmt = conn.createStatement();
+            rs = stmt.executeQuery(sql);
+            int n = 0;
+            ResultSetMetaData meta = rs.getMetaData();
+            while (rs.next()) {
+                n++;
+                for (int i = 1; i <= meta.getColumnCount(); i++) {
+                    System.out.println(n + " - " + meta.getColumnLabel(i) + ":\t" + rs.getObject(i));
+                }
+            }
+        } finally {
+            if (rs != null) {
+                rs.close();
+            }
+            if (stmt != null) {
+                stmt.close();
+            }
+            if (conn != null) {
+                conn.close();
+            }
+        }
+
+    }
+}


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

Posted by lu...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/dictionary/.settings/org.eclipse.core.resources.prefs
----------------------------------------------------------------------
diff --git a/dictionary/.settings/org.eclipse.core.resources.prefs b/dictionary/.settings/org.eclipse.core.resources.prefs
new file mode 100644
index 0000000..04cfa2c
--- /dev/null
+++ b/dictionary/.settings/org.eclipse.core.resources.prefs
@@ -0,0 +1,6 @@
+eclipse.preferences.version=1
+encoding//src/main/java=UTF-8
+encoding//src/main/resources=UTF-8
+encoding//src/test/java=UTF-8
+encoding//src/test/resources=UTF-8
+encoding/<project>=UTF-8

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/dictionary/.settings/org.eclipse.jdt.core.prefs
----------------------------------------------------------------------
diff --git a/dictionary/.settings/org.eclipse.jdt.core.prefs b/dictionary/.settings/org.eclipse.jdt.core.prefs
new file mode 100644
index 0000000..a903301
--- /dev/null
+++ b/dictionary/.settings/org.eclipse.jdt.core.prefs
@@ -0,0 +1,379 @@
+eclipse.preferences.version=1
+org.eclipse.jdt.core.compiler.annotation.inheritNullAnnotations=disabled
+org.eclipse.jdt.core.compiler.annotation.missingNonNullByDefaultAnnotation=ignore
+org.eclipse.jdt.core.compiler.annotation.nonnull=org.eclipse.jdt.annotation.NonNull
+org.eclipse.jdt.core.compiler.annotation.nonnullbydefault=org.eclipse.jdt.annotation.NonNullByDefault
+org.eclipse.jdt.core.compiler.annotation.nullable=org.eclipse.jdt.annotation.Nullable
+org.eclipse.jdt.core.compiler.annotation.nullanalysis=disabled
+org.eclipse.jdt.core.compiler.codegen.inlineJsrBytecode=enabled
+org.eclipse.jdt.core.compiler.codegen.targetPlatform=1.7
+org.eclipse.jdt.core.compiler.codegen.unusedLocal=preserve
+org.eclipse.jdt.core.compiler.compliance=1.7
+org.eclipse.jdt.core.compiler.debug.lineNumber=generate
+org.eclipse.jdt.core.compiler.debug.localVariable=generate
+org.eclipse.jdt.core.compiler.debug.sourceFile=generate
+org.eclipse.jdt.core.compiler.problem.annotationSuperInterface=warning
+org.eclipse.jdt.core.compiler.problem.assertIdentifier=error
+org.eclipse.jdt.core.compiler.problem.autoboxing=ignore
+org.eclipse.jdt.core.compiler.problem.comparingIdentical=warning
+org.eclipse.jdt.core.compiler.problem.deadCode=warning
+org.eclipse.jdt.core.compiler.problem.deprecation=warning
+org.eclipse.jdt.core.compiler.problem.deprecationInDeprecatedCode=disabled
+org.eclipse.jdt.core.compiler.problem.deprecationWhenOverridingDeprecatedMethod=disabled
+org.eclipse.jdt.core.compiler.problem.discouragedReference=warning
+org.eclipse.jdt.core.compiler.problem.emptyStatement=ignore
+org.eclipse.jdt.core.compiler.problem.enumIdentifier=error
+org.eclipse.jdt.core.compiler.problem.explicitlyClosedAutoCloseable=ignore
+org.eclipse.jdt.core.compiler.problem.fallthroughCase=ignore
+org.eclipse.jdt.core.compiler.problem.fatalOptionalError=disabled
+org.eclipse.jdt.core.compiler.problem.fieldHiding=ignore
+org.eclipse.jdt.core.compiler.problem.finalParameterBound=warning
+org.eclipse.jdt.core.compiler.problem.finallyBlockNotCompletingNormally=warning
+org.eclipse.jdt.core.compiler.problem.forbiddenReference=warning
+org.eclipse.jdt.core.compiler.problem.hiddenCatchBlock=warning
+org.eclipse.jdt.core.compiler.problem.includeNullInfoFromAsserts=disabled
+org.eclipse.jdt.core.compiler.problem.incompatibleNonInheritedInterfaceMethod=warning
+org.eclipse.jdt.core.compiler.problem.incompleteEnumSwitch=warning
+org.eclipse.jdt.core.compiler.problem.indirectStaticAccess=ignore
+org.eclipse.jdt.core.compiler.problem.localVariableHiding=ignore
+org.eclipse.jdt.core.compiler.problem.methodWithConstructorName=warning
+org.eclipse.jdt.core.compiler.problem.missingDefaultCase=ignore
+org.eclipse.jdt.core.compiler.problem.missingDeprecatedAnnotation=ignore
+org.eclipse.jdt.core.compiler.problem.missingEnumCaseDespiteDefault=disabled
+org.eclipse.jdt.core.compiler.problem.missingHashCodeMethod=ignore
+org.eclipse.jdt.core.compiler.problem.missingOverrideAnnotation=ignore
+org.eclipse.jdt.core.compiler.problem.missingOverrideAnnotationForInterfaceMethodImplementation=enabled
+org.eclipse.jdt.core.compiler.problem.missingSerialVersion=warning
+org.eclipse.jdt.core.compiler.problem.missingSynchronizedOnInheritedMethod=ignore
+org.eclipse.jdt.core.compiler.problem.noEffectAssignment=warning
+org.eclipse.jdt.core.compiler.problem.noImplicitStringConversion=warning
+org.eclipse.jdt.core.compiler.problem.nonExternalizedStringLiteral=ignore
+org.eclipse.jdt.core.compiler.problem.nonnullParameterAnnotationDropped=warning
+org.eclipse.jdt.core.compiler.problem.nullAnnotationInferenceConflict=error
+org.eclipse.jdt.core.compiler.problem.nullReference=warning
+org.eclipse.jdt.core.compiler.problem.nullSpecViolation=error
+org.eclipse.jdt.core.compiler.problem.nullUncheckedConversion=warning
+org.eclipse.jdt.core.compiler.problem.overridingPackageDefaultMethod=warning
+org.eclipse.jdt.core.compiler.problem.parameterAssignment=ignore
+org.eclipse.jdt.core.compiler.problem.possibleAccidentalBooleanAssignment=ignore
+org.eclipse.jdt.core.compiler.problem.potentialNullReference=ignore
+org.eclipse.jdt.core.compiler.problem.potentiallyUnclosedCloseable=ignore
+org.eclipse.jdt.core.compiler.problem.rawTypeReference=warning
+org.eclipse.jdt.core.compiler.problem.redundantNullAnnotation=warning
+org.eclipse.jdt.core.compiler.problem.redundantNullCheck=ignore
+org.eclipse.jdt.core.compiler.problem.redundantSpecificationOfTypeArguments=ignore
+org.eclipse.jdt.core.compiler.problem.redundantSuperinterface=ignore
+org.eclipse.jdt.core.compiler.problem.reportMethodCanBePotentiallyStatic=ignore
+org.eclipse.jdt.core.compiler.problem.reportMethodCanBeStatic=ignore
+org.eclipse.jdt.core.compiler.problem.specialParameterHidingField=disabled
+org.eclipse.jdt.core.compiler.problem.staticAccessReceiver=warning
+org.eclipse.jdt.core.compiler.problem.suppressOptionalErrors=disabled
+org.eclipse.jdt.core.compiler.problem.suppressWarnings=enabled
+org.eclipse.jdt.core.compiler.problem.syntacticNullAnalysisForFields=disabled
+org.eclipse.jdt.core.compiler.problem.syntheticAccessEmulation=ignore
+org.eclipse.jdt.core.compiler.problem.typeParameterHiding=warning
+org.eclipse.jdt.core.compiler.problem.unavoidableGenericTypeProblems=enabled
+org.eclipse.jdt.core.compiler.problem.uncheckedTypeOperation=warning
+org.eclipse.jdt.core.compiler.problem.unclosedCloseable=warning
+org.eclipse.jdt.core.compiler.problem.undocumentedEmptyBlock=ignore
+org.eclipse.jdt.core.compiler.problem.unhandledWarningToken=warning
+org.eclipse.jdt.core.compiler.problem.unnecessaryElse=ignore
+org.eclipse.jdt.core.compiler.problem.unnecessaryTypeCheck=ignore
+org.eclipse.jdt.core.compiler.problem.unqualifiedFieldAccess=ignore
+org.eclipse.jdt.core.compiler.problem.unusedDeclaredThrownException=ignore
+org.eclipse.jdt.core.compiler.problem.unusedDeclaredThrownExceptionExemptExceptionAndThrowable=enabled
+org.eclipse.jdt.core.compiler.problem.unusedDeclaredThrownExceptionIncludeDocCommentReference=enabled
+org.eclipse.jdt.core.compiler.problem.unusedDeclaredThrownExceptionWhenOverriding=disabled
+org.eclipse.jdt.core.compiler.problem.unusedImport=warning
+org.eclipse.jdt.core.compiler.problem.unusedLabel=warning
+org.eclipse.jdt.core.compiler.problem.unusedLocal=warning
+org.eclipse.jdt.core.compiler.problem.unusedObjectAllocation=ignore
+org.eclipse.jdt.core.compiler.problem.unusedParameter=ignore
+org.eclipse.jdt.core.compiler.problem.unusedParameterIncludeDocCommentReference=enabled
+org.eclipse.jdt.core.compiler.problem.unusedParameterWhenImplementingAbstract=disabled
+org.eclipse.jdt.core.compiler.problem.unusedParameterWhenOverridingConcrete=disabled
+org.eclipse.jdt.core.compiler.problem.unusedPrivateMember=warning
+org.eclipse.jdt.core.compiler.problem.unusedTypeParameter=ignore
+org.eclipse.jdt.core.compiler.problem.unusedWarningToken=warning
+org.eclipse.jdt.core.compiler.problem.varargsArgumentNeedCast=warning
+org.eclipse.jdt.core.compiler.source=1.7
+org.eclipse.jdt.core.formatter.align_type_members_on_columns=false
+org.eclipse.jdt.core.formatter.alignment_for_arguments_in_allocation_expression=16
+org.eclipse.jdt.core.formatter.alignment_for_arguments_in_annotation=0
+org.eclipse.jdt.core.formatter.alignment_for_arguments_in_enum_constant=16
+org.eclipse.jdt.core.formatter.alignment_for_arguments_in_explicit_constructor_call=16
+org.eclipse.jdt.core.formatter.alignment_for_arguments_in_method_invocation=16
+org.eclipse.jdt.core.formatter.alignment_for_arguments_in_qualified_allocation_expression=16
+org.eclipse.jdt.core.formatter.alignment_for_assignment=0
+org.eclipse.jdt.core.formatter.alignment_for_binary_expression=16
+org.eclipse.jdt.core.formatter.alignment_for_compact_if=16
+org.eclipse.jdt.core.formatter.alignment_for_conditional_expression=80
+org.eclipse.jdt.core.formatter.alignment_for_enum_constants=0
+org.eclipse.jdt.core.formatter.alignment_for_expressions_in_array_initializer=16
+org.eclipse.jdt.core.formatter.alignment_for_method_declaration=0
+org.eclipse.jdt.core.formatter.alignment_for_multiple_fields=16
+org.eclipse.jdt.core.formatter.alignment_for_parameters_in_constructor_declaration=16
+org.eclipse.jdt.core.formatter.alignment_for_parameters_in_method_declaration=16
+org.eclipse.jdt.core.formatter.alignment_for_resources_in_try=80
+org.eclipse.jdt.core.formatter.alignment_for_selector_in_method_invocation=16
+org.eclipse.jdt.core.formatter.alignment_for_superclass_in_type_declaration=16
+org.eclipse.jdt.core.formatter.alignment_for_superinterfaces_in_enum_declaration=16
+org.eclipse.jdt.core.formatter.alignment_for_superinterfaces_in_type_declaration=16
+org.eclipse.jdt.core.formatter.alignment_for_throws_clause_in_constructor_declaration=16
+org.eclipse.jdt.core.formatter.alignment_for_throws_clause_in_method_declaration=16
+org.eclipse.jdt.core.formatter.alignment_for_union_type_in_multicatch=16
+org.eclipse.jdt.core.formatter.blank_lines_after_imports=1
+org.eclipse.jdt.core.formatter.blank_lines_after_package=1
+org.eclipse.jdt.core.formatter.blank_lines_before_field=0
+org.eclipse.jdt.core.formatter.blank_lines_before_first_class_body_declaration=0
+org.eclipse.jdt.core.formatter.blank_lines_before_imports=1
+org.eclipse.jdt.core.formatter.blank_lines_before_member_type=1
+org.eclipse.jdt.core.formatter.blank_lines_before_method=1
+org.eclipse.jdt.core.formatter.blank_lines_before_new_chunk=1
+org.eclipse.jdt.core.formatter.blank_lines_before_package=0
+org.eclipse.jdt.core.formatter.blank_lines_between_import_groups=1
+org.eclipse.jdt.core.formatter.blank_lines_between_type_declarations=1
+org.eclipse.jdt.core.formatter.brace_position_for_annotation_type_declaration=end_of_line
+org.eclipse.jdt.core.formatter.brace_position_for_anonymous_type_declaration=end_of_line
+org.eclipse.jdt.core.formatter.brace_position_for_array_initializer=end_of_line
+org.eclipse.jdt.core.formatter.brace_position_for_block=end_of_line
+org.eclipse.jdt.core.formatter.brace_position_for_block_in_case=end_of_line
+org.eclipse.jdt.core.formatter.brace_position_for_constructor_declaration=end_of_line
+org.eclipse.jdt.core.formatter.brace_position_for_enum_constant=end_of_line
+org.eclipse.jdt.core.formatter.brace_position_for_enum_declaration=end_of_line
+org.eclipse.jdt.core.formatter.brace_position_for_method_declaration=end_of_line
+org.eclipse.jdt.core.formatter.brace_position_for_switch=end_of_line
+org.eclipse.jdt.core.formatter.brace_position_for_type_declaration=end_of_line
+org.eclipse.jdt.core.formatter.comment.clear_blank_lines_in_block_comment=false
+org.eclipse.jdt.core.formatter.comment.clear_blank_lines_in_javadoc_comment=false
+org.eclipse.jdt.core.formatter.comment.format_block_comments=false
+org.eclipse.jdt.core.formatter.comment.format_header=false
+org.eclipse.jdt.core.formatter.comment.format_html=true
+org.eclipse.jdt.core.formatter.comment.format_javadoc_comments=false
+org.eclipse.jdt.core.formatter.comment.format_line_comments=false
+org.eclipse.jdt.core.formatter.comment.format_source_code=true
+org.eclipse.jdt.core.formatter.comment.indent_parameter_description=true
+org.eclipse.jdt.core.formatter.comment.indent_root_tags=true
+org.eclipse.jdt.core.formatter.comment.insert_new_line_before_root_tags=insert
+org.eclipse.jdt.core.formatter.comment.insert_new_line_for_parameter=insert
+org.eclipse.jdt.core.formatter.comment.line_length=80
+org.eclipse.jdt.core.formatter.comment.new_lines_at_block_boundaries=true
+org.eclipse.jdt.core.formatter.comment.new_lines_at_javadoc_boundaries=true
+org.eclipse.jdt.core.formatter.comment.preserve_white_space_between_code_and_line_comments=false
+org.eclipse.jdt.core.formatter.compact_else_if=true
+org.eclipse.jdt.core.formatter.continuation_indentation=2
+org.eclipse.jdt.core.formatter.continuation_indentation_for_array_initializer=2
+org.eclipse.jdt.core.formatter.disabling_tag=@formatter\:off
+org.eclipse.jdt.core.formatter.enabling_tag=@formatter\:on
+org.eclipse.jdt.core.formatter.format_guardian_clause_on_one_line=false
+org.eclipse.jdt.core.formatter.format_line_comment_starting_on_first_column=true
+org.eclipse.jdt.core.formatter.indent_body_declarations_compare_to_annotation_declaration_header=true
+org.eclipse.jdt.core.formatter.indent_body_declarations_compare_to_enum_constant_header=true
+org.eclipse.jdt.core.formatter.indent_body_declarations_compare_to_enum_declaration_header=true
+org.eclipse.jdt.core.formatter.indent_body_declarations_compare_to_type_header=true
+org.eclipse.jdt.core.formatter.indent_breaks_compare_to_cases=true
+org.eclipse.jdt.core.formatter.indent_empty_lines=false
+org.eclipse.jdt.core.formatter.indent_statements_compare_to_block=true
+org.eclipse.jdt.core.formatter.indent_statements_compare_to_body=true
+org.eclipse.jdt.core.formatter.indent_switchstatements_compare_to_cases=true
+org.eclipse.jdt.core.formatter.indent_switchstatements_compare_to_switch=false
+org.eclipse.jdt.core.formatter.indentation.size=4
+org.eclipse.jdt.core.formatter.insert_new_line_after_annotation_on_field=insert
+org.eclipse.jdt.core.formatter.insert_new_line_after_annotation_on_local_variable=insert
+org.eclipse.jdt.core.formatter.insert_new_line_after_annotation_on_method=insert
+org.eclipse.jdt.core.formatter.insert_new_line_after_annotation_on_package=insert
+org.eclipse.jdt.core.formatter.insert_new_line_after_annotation_on_parameter=do not insert
+org.eclipse.jdt.core.formatter.insert_new_line_after_annotation_on_type=insert
+org.eclipse.jdt.core.formatter.insert_new_line_after_label=do not insert
+org.eclipse.jdt.core.formatter.insert_new_line_after_opening_brace_in_array_initializer=do not insert
+org.eclipse.jdt.core.formatter.insert_new_line_at_end_of_file_if_missing=do not insert
+org.eclipse.jdt.core.formatter.insert_new_line_before_catch_in_try_statement=do not insert
+org.eclipse.jdt.core.formatter.insert_new_line_before_closing_brace_in_array_initializer=do not insert
+org.eclipse.jdt.core.formatter.insert_new_line_before_else_in_if_statement=do not insert
+org.eclipse.jdt.core.formatter.insert_new_line_before_finally_in_try_statement=do not insert
+org.eclipse.jdt.core.formatter.insert_new_line_before_while_in_do_statement=do not insert
+org.eclipse.jdt.core.formatter.insert_new_line_in_empty_annotation_declaration=insert
+org.eclipse.jdt.core.formatter.insert_new_line_in_empty_anonymous_type_declaration=insert
+org.eclipse.jdt.core.formatter.insert_new_line_in_empty_block=insert
+org.eclipse.jdt.core.formatter.insert_new_line_in_empty_enum_constant=insert
+org.eclipse.jdt.core.formatter.insert_new_line_in_empty_enum_declaration=insert
+org.eclipse.jdt.core.formatter.insert_new_line_in_empty_method_body=insert
+org.eclipse.jdt.core.formatter.insert_new_line_in_empty_type_declaration=insert
+org.eclipse.jdt.core.formatter.insert_space_after_and_in_type_parameter=insert
+org.eclipse.jdt.core.formatter.insert_space_after_assignment_operator=insert
+org.eclipse.jdt.core.formatter.insert_space_after_at_in_annotation=do not insert
+org.eclipse.jdt.core.formatter.insert_space_after_at_in_annotation_type_declaration=do not insert
+org.eclipse.jdt.core.formatter.insert_space_after_binary_operator=insert
+org.eclipse.jdt.core.formatter.insert_space_after_closing_angle_bracket_in_type_arguments=insert
+org.eclipse.jdt.core.formatter.insert_space_after_closing_angle_bracket_in_type_parameters=insert
+org.eclipse.jdt.core.formatter.insert_space_after_closing_brace_in_block=insert
+org.eclipse.jdt.core.formatter.insert_space_after_closing_paren_in_cast=insert
+org.eclipse.jdt.core.formatter.insert_space_after_colon_in_assert=insert
+org.eclipse.jdt.core.formatter.insert_space_after_colon_in_case=insert
+org.eclipse.jdt.core.formatter.insert_space_after_colon_in_conditional=insert
+org.eclipse.jdt.core.formatter.insert_space_after_colon_in_for=insert
+org.eclipse.jdt.core.formatter.insert_space_after_colon_in_labeled_statement=insert
+org.eclipse.jdt.core.formatter.insert_space_after_comma_in_allocation_expression=insert
+org.eclipse.jdt.core.formatter.insert_space_after_comma_in_annotation=insert
+org.eclipse.jdt.core.formatter.insert_space_after_comma_in_array_initializer=insert
+org.eclipse.jdt.core.formatter.insert_space_after_comma_in_constructor_declaration_parameters=insert
+org.eclipse.jdt.core.formatter.insert_space_after_comma_in_constructor_declaration_throws=insert
+org.eclipse.jdt.core.formatter.insert_space_after_comma_in_enum_constant_arguments=insert
+org.eclipse.jdt.core.formatter.insert_space_after_comma_in_enum_declarations=insert
+org.eclipse.jdt.core.formatter.insert_space_after_comma_in_explicitconstructorcall_arguments=insert
+org.eclipse.jdt.core.formatter.insert_space_after_comma_in_for_increments=insert
+org.eclipse.jdt.core.formatter.insert_space_after_comma_in_for_inits=insert
+org.eclipse.jdt.core.formatter.insert_space_after_comma_in_method_declaration_parameters=insert
+org.eclipse.jdt.core.formatter.insert_space_after_comma_in_method_declaration_throws=insert
+org.eclipse.jdt.core.formatter.insert_space_after_comma_in_method_invocation_arguments=insert
+org.eclipse.jdt.core.formatter.insert_space_after_comma_in_multiple_field_declarations=insert
+org.eclipse.jdt.core.formatter.insert_space_after_comma_in_multiple_local_declarations=insert
+org.eclipse.jdt.core.formatter.insert_space_after_comma_in_parameterized_type_reference=insert
+org.eclipse.jdt.core.formatter.insert_space_after_comma_in_superinterfaces=insert
+org.eclipse.jdt.core.formatter.insert_space_after_comma_in_type_arguments=insert
+org.eclipse.jdt.core.formatter.insert_space_after_comma_in_type_parameters=insert
+org.eclipse.jdt.core.formatter.insert_space_after_ellipsis=insert
+org.eclipse.jdt.core.formatter.insert_space_after_opening_angle_bracket_in_parameterized_type_reference=do not insert
+org.eclipse.jdt.core.formatter.insert_space_after_opening_angle_bracket_in_type_arguments=do not insert
+org.eclipse.jdt.core.formatter.insert_space_after_opening_angle_bracket_in_type_parameters=do not insert
+org.eclipse.jdt.core.formatter.insert_space_after_opening_brace_in_array_initializer=insert
+org.eclipse.jdt.core.formatter.insert_space_after_opening_bracket_in_array_allocation_expression=do not insert
+org.eclipse.jdt.core.formatter.insert_space_after_opening_bracket_in_array_reference=do not insert
+org.eclipse.jdt.core.formatter.insert_space_after_opening_paren_in_annotation=do not insert
+org.eclipse.jdt.core.formatter.insert_space_after_opening_paren_in_cast=do not insert
+org.eclipse.jdt.core.formatter.insert_space_after_opening_paren_in_catch=do not insert
+org.eclipse.jdt.core.formatter.insert_space_after_opening_paren_in_constructor_declaration=do not insert
+org.eclipse.jdt.core.formatter.insert_space_after_opening_paren_in_enum_constant=do not insert
+org.eclipse.jdt.core.formatter.insert_space_after_opening_paren_in_for=do not insert
+org.eclipse.jdt.core.formatter.insert_space_after_opening_paren_in_if=do not insert
+org.eclipse.jdt.core.formatter.insert_space_after_opening_paren_in_method_declaration=do not insert
+org.eclipse.jdt.core.formatter.insert_space_after_opening_paren_in_method_invocation=do not insert
+org.eclipse.jdt.core.formatter.insert_space_after_opening_paren_in_parenthesized_expression=do not insert
+org.eclipse.jdt.core.formatter.insert_space_after_opening_paren_in_switch=do not insert
+org.eclipse.jdt.core.formatter.insert_space_after_opening_paren_in_synchronized=do not insert
+org.eclipse.jdt.core.formatter.insert_space_after_opening_paren_in_try=do not insert
+org.eclipse.jdt.core.formatter.insert_space_after_opening_paren_in_while=do not insert
+org.eclipse.jdt.core.formatter.insert_space_after_postfix_operator=do not insert
+org.eclipse.jdt.core.formatter.insert_space_after_prefix_operator=do not insert
+org.eclipse.jdt.core.formatter.insert_space_after_question_in_conditional=insert
+org.eclipse.jdt.core.formatter.insert_space_after_question_in_wildcard=do not insert
+org.eclipse.jdt.core.formatter.insert_space_after_semicolon_in_for=insert
+org.eclipse.jdt.core.formatter.insert_space_after_semicolon_in_try_resources=insert
+org.eclipse.jdt.core.formatter.insert_space_after_unary_operator=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_and_in_type_parameter=insert
+org.eclipse.jdt.core.formatter.insert_space_before_assignment_operator=insert
+org.eclipse.jdt.core.formatter.insert_space_before_at_in_annotation_type_declaration=insert
+org.eclipse.jdt.core.formatter.insert_space_before_binary_operator=insert
+org.eclipse.jdt.core.formatter.insert_space_before_closing_angle_bracket_in_parameterized_type_reference=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_closing_angle_bracket_in_type_arguments=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_closing_angle_bracket_in_type_parameters=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_closing_brace_in_array_initializer=insert
+org.eclipse.jdt.core.formatter.insert_space_before_closing_bracket_in_array_allocation_expression=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_closing_bracket_in_array_reference=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_closing_paren_in_annotation=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_closing_paren_in_cast=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_closing_paren_in_catch=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_closing_paren_in_constructor_declaration=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_closing_paren_in_enum_constant=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_closing_paren_in_for=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_closing_paren_in_if=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_closing_paren_in_method_declaration=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_closing_paren_in_method_invocation=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_closing_paren_in_parenthesized_expression=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_closing_paren_in_switch=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_closing_paren_in_synchronized=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_closing_paren_in_try=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_closing_paren_in_while=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_colon_in_assert=insert
+org.eclipse.jdt.core.formatter.insert_space_before_colon_in_case=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_colon_in_conditional=insert
+org.eclipse.jdt.core.formatter.insert_space_before_colon_in_default=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_colon_in_for=insert
+org.eclipse.jdt.core.formatter.insert_space_before_colon_in_labeled_statement=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_comma_in_allocation_expression=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_comma_in_annotation=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_comma_in_array_initializer=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_comma_in_constructor_declaration_parameters=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_comma_in_constructor_declaration_throws=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_comma_in_enum_constant_arguments=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_comma_in_enum_declarations=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_comma_in_explicitconstructorcall_arguments=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_comma_in_for_increments=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_comma_in_for_inits=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_comma_in_method_declaration_parameters=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_comma_in_method_declaration_throws=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_comma_in_method_invocation_arguments=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_comma_in_multiple_field_declarations=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_comma_in_multiple_local_declarations=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_comma_in_parameterized_type_reference=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_comma_in_superinterfaces=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_comma_in_type_arguments=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_comma_in_type_parameters=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_ellipsis=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_opening_angle_bracket_in_parameterized_type_reference=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_opening_angle_bracket_in_type_arguments=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_opening_angle_bracket_in_type_parameters=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_opening_brace_in_annotation_type_declaration=insert
+org.eclipse.jdt.core.formatter.insert_space_before_opening_brace_in_anonymous_type_declaration=insert
+org.eclipse.jdt.core.formatter.insert_space_before_opening_brace_in_array_initializer=insert
+org.eclipse.jdt.core.formatter.insert_space_before_opening_brace_in_block=insert
+org.eclipse.jdt.core.formatter.insert_space_before_opening_brace_in_constructor_declaration=insert
+org.eclipse.jdt.core.formatter.insert_space_before_opening_brace_in_enum_constant=insert
+org.eclipse.jdt.core.formatter.insert_space_before_opening_brace_in_enum_declaration=insert
+org.eclipse.jdt.core.formatter.insert_space_before_opening_brace_in_method_declaration=insert
+org.eclipse.jdt.core.formatter.insert_space_before_opening_brace_in_switch=insert
+org.eclipse.jdt.core.formatter.insert_space_before_opening_brace_in_type_declaration=insert
+org.eclipse.jdt.core.formatter.insert_space_before_opening_bracket_in_array_allocation_expression=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_opening_bracket_in_array_reference=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_opening_bracket_in_array_type_reference=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_opening_paren_in_annotation=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_opening_paren_in_annotation_type_member_declaration=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_opening_paren_in_catch=insert
+org.eclipse.jdt.core.formatter.insert_space_before_opening_paren_in_constructor_declaration=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_opening_paren_in_enum_constant=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_opening_paren_in_for=insert
+org.eclipse.jdt.core.formatter.insert_space_before_opening_paren_in_if=insert
+org.eclipse.jdt.core.formatter.insert_space_before_opening_paren_in_method_declaration=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_opening_paren_in_method_invocation=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_opening_paren_in_parenthesized_expression=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_opening_paren_in_switch=insert
+org.eclipse.jdt.core.formatter.insert_space_before_opening_paren_in_synchronized=insert
+org.eclipse.jdt.core.formatter.insert_space_before_opening_paren_in_try=insert
+org.eclipse.jdt.core.formatter.insert_space_before_opening_paren_in_while=insert
+org.eclipse.jdt.core.formatter.insert_space_before_parenthesized_expression_in_return=insert
+org.eclipse.jdt.core.formatter.insert_space_before_parenthesized_expression_in_throw=insert
+org.eclipse.jdt.core.formatter.insert_space_before_postfix_operator=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_prefix_operator=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_question_in_conditional=insert
+org.eclipse.jdt.core.formatter.insert_space_before_question_in_wildcard=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_semicolon=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_semicolon_in_for=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_semicolon_in_try_resources=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_unary_operator=do not insert
+org.eclipse.jdt.core.formatter.insert_space_between_brackets_in_array_type_reference=do not insert
+org.eclipse.jdt.core.formatter.insert_space_between_empty_braces_in_array_initializer=do not insert
+org.eclipse.jdt.core.formatter.insert_space_between_empty_brackets_in_array_allocation_expression=do not insert
+org.eclipse.jdt.core.formatter.insert_space_between_empty_parens_in_annotation_type_member_declaration=do not insert
+org.eclipse.jdt.core.formatter.insert_space_between_empty_parens_in_constructor_declaration=do not insert
+org.eclipse.jdt.core.formatter.insert_space_between_empty_parens_in_enum_constant=do not insert
+org.eclipse.jdt.core.formatter.insert_space_between_empty_parens_in_method_declaration=do not insert
+org.eclipse.jdt.core.formatter.insert_space_between_empty_parens_in_method_invocation=do not insert
+org.eclipse.jdt.core.formatter.join_lines_in_comments=true
+org.eclipse.jdt.core.formatter.join_wrapped_lines=true
+org.eclipse.jdt.core.formatter.keep_else_statement_on_same_line=false
+org.eclipse.jdt.core.formatter.keep_empty_array_initializer_on_one_line=false
+org.eclipse.jdt.core.formatter.keep_imple_if_on_one_line=false
+org.eclipse.jdt.core.formatter.keep_then_statement_on_same_line=false
+org.eclipse.jdt.core.formatter.lineSplit=999
+org.eclipse.jdt.core.formatter.never_indent_block_comments_on_first_column=false
+org.eclipse.jdt.core.formatter.never_indent_line_comments_on_first_column=false
+org.eclipse.jdt.core.formatter.number_of_blank_lines_at_beginning_of_method_body=0
+org.eclipse.jdt.core.formatter.number_of_empty_lines_to_preserve=1
+org.eclipse.jdt.core.formatter.put_empty_statement_on_new_line=true
+org.eclipse.jdt.core.formatter.tabulation.char=space
+org.eclipse.jdt.core.formatter.tabulation.size=4
+org.eclipse.jdt.core.formatter.use_on_off_tags=false
+org.eclipse.jdt.core.formatter.use_tabs_only_for_leading_indentations=false
+org.eclipse.jdt.core.formatter.wrap_before_binary_operator=true
+org.eclipse.jdt.core.formatter.wrap_before_or_operator_multicatch=true
+org.eclipse.jdt.core.formatter.wrap_outer_expressions_when_nested=true

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/dictionary/.settings/org.eclipse.jdt.ui.prefs
----------------------------------------------------------------------
diff --git a/dictionary/.settings/org.eclipse.jdt.ui.prefs b/dictionary/.settings/org.eclipse.jdt.ui.prefs
new file mode 100644
index 0000000..dece0e6
--- /dev/null
+++ b/dictionary/.settings/org.eclipse.jdt.ui.prefs
@@ -0,0 +1,7 @@
+eclipse.preferences.version=1
+formatter_profile=_Space Indent & Long Lines
+formatter_settings_version=12
+org.eclipse.jdt.ui.ignorelowercasenames=true
+org.eclipse.jdt.ui.importorder=java;javax;org;com;
+org.eclipse.jdt.ui.ondemandthreshold=99
+org.eclipse.jdt.ui.staticondemandthreshold=1

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/dictionary/pom.xml
----------------------------------------------------------------------
diff --git a/dictionary/pom.xml b/dictionary/pom.xml
new file mode 100644
index 0000000..1d09903
--- /dev/null
+++ b/dictionary/pom.xml
@@ -0,0 +1,78 @@
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+    <modelVersion>4.0.0</modelVersion>
+
+    <artifactId>kylin-dictionary</artifactId>
+    <packaging>jar</packaging>
+    <name>Kylin:Dictionary</name>
+
+    <parent>
+        <groupId>com.kylinolap</groupId>
+        <artifactId>kylin</artifactId>
+        <version>0.6.3-SNAPSHOT</version>
+    </parent>
+
+    <properties>
+    </properties>
+
+    <dependencies>
+        <!--Kylin Jar -->
+        <dependency>
+            <groupId>com.kylinolap</groupId>
+            <artifactId>kylin-metadata</artifactId>
+            <version>${project.parent.version}</version>
+        </dependency>
+
+        <dependency>
+            <groupId>com.fasterxml.jackson.core</groupId>
+            <artifactId>jackson-databind</artifactId>
+        </dependency>
+        <dependency>
+            <groupId>commons-io</groupId>
+            <artifactId>commons-io</artifactId>
+        </dependency>
+        <dependency>
+            <groupId>commons-configuration</groupId>
+            <artifactId>commons-configuration</artifactId>
+        </dependency>
+        <dependency>
+            <groupId>com.google.guava</groupId>
+            <artifactId>guava</artifactId>
+        </dependency>
+
+        <!-- Env & Test -->
+
+        <dependency>
+            <groupId>org.apache.hadoop</groupId>
+            <artifactId>hadoop-common</artifactId>
+            <scope>provided</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.hadoop</groupId>
+            <artifactId>hadoop-hdfs</artifactId>
+            <scope>provided</scope>
+            <!-- protobuf version conflict with hbase-->
+            <exclusions>
+                <exclusion>
+                    <groupId>com.google.protobuf</groupId>
+                    <artifactId>protobuf-java</artifactId>
+                </exclusion>
+            </exclusions>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.hbase</groupId>
+            <artifactId>hbase-common</artifactId>
+            <scope>provided</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.hbase</groupId>
+            <artifactId>hbase-client</artifactId>
+            <scope>provided</scope>
+        </dependency>
+        <dependency>
+            <groupId>junit</groupId>
+            <artifactId>junit</artifactId>
+            <scope>test</scope>
+        </dependency>
+    </dependencies>
+
+</project>

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/dictionary/src/main/java/com/kylinolap/dict/BytesConverter.java
----------------------------------------------------------------------
diff --git a/dictionary/src/main/java/com/kylinolap/dict/BytesConverter.java b/dictionary/src/main/java/com/kylinolap/dict/BytesConverter.java
new file mode 100644
index 0000000..cff65c5
--- /dev/null
+++ b/dictionary/src/main/java/com/kylinolap/dict/BytesConverter.java
@@ -0,0 +1,23 @@
+/*
+ * 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;
+
+public interface BytesConverter<T> {
+
+    public byte[] convertToBytes(T v);
+
+    public T convertFromBytes(byte[] b, int offset, int length);
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/dictionary/src/main/java/com/kylinolap/dict/DateStrDictionary.java
----------------------------------------------------------------------
diff --git a/dictionary/src/main/java/com/kylinolap/dict/DateStrDictionary.java b/dictionary/src/main/java/com/kylinolap/dict/DateStrDictionary.java
new file mode 100644
index 0000000..ed26aae
--- /dev/null
+++ b/dictionary/src/main/java/com/kylinolap/dict/DateStrDictionary.java
@@ -0,0 +1,240 @@
+/*
+ * 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 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;
+
+/**
+ * A dictionary for date string (date only, no time).
+ * 
+ * Dates are numbered from 1970-1-1 -- 0 for 1970-1-1, 1 for 1-2, 2 for 1-3 and
+ * so on. With 2 bytes, 65536 states, can express dates up to the year of 2149.
+ * 
+ * Note the implementation is not thread-safe.
+ * 
+ * @author yangli9
+ */
+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 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 (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 String pattern;
+    private int baseId;
+
+    public DateStrDictionary() {
+        init(DEFAULT_DATE_PATTERN, 0);
+    }
+
+    public DateStrDictionary(String datePattern, int baseId) {
+        init(datePattern, baseId);
+    }
+
+    private void init(String datePattern, int baseId) {
+        this.pattern = datePattern;
+        this.baseId = baseId;
+    }
+
+    @Override
+    public int getMinId() {
+        return baseId;
+    }
+
+    @Override
+    public int getMaxId() {
+        return Integer.MAX_VALUE;
+    }
+
+    @Override
+    public int getSizeOfId() {
+        return 3;
+    }
+
+    @Override
+    public int getSizeOfValue() {
+        return pattern.length();
+    }
+
+    @Override
+    protected boolean isNullByteForm(byte[] value, int offset, int len) {
+        return value == null || len == 0;
+    }
+
+    @Override
+    final protected int getIdFromValueImpl(String value, int roundFlag) {
+        Date date = stringToDate(value, pattern);
+        int id = calcIdFromSeqNo(getNumOfDaysSince0000(date));
+        if (id < 0 || id >= 16777216)
+            throw new IllegalArgumentException("'" + value + "' encodes to '" + id + "' which is out of range of 3 bytes");
+
+        return id;
+    }
+
+    @Override
+    final protected String getValueFromIdImpl(int id) {
+        if (id < baseId)
+            throw new IllegalArgumentException("ID '" + id + "' must not be less than base ID " + baseId);
+        Date d = getDateFromNumOfDaysSince0000(calcSeqNoFromId(id));
+        return dateToString(d, pattern);
+    }
+
+    private int getNumOfDaysSince0000(Date d) {
+        // 86400000 = 1000 * 60 * 60 * 24
+        // -719530 is offset of 0000-01-01
+        return (int) (d.getTime() / 86400000 + 719530);
+    }
+
+    private Date getDateFromNumOfDaysSince0000(int n) {
+        long millis = ((long) n - 719530) * 86400000;
+        return new Date(millis);
+    }
+
+    @Override
+    final protected int getIdFromValueBytesImpl(byte[] value, int offset, int len, int roundingFlag) {
+        try {
+            return getIdFromValue(new String(value, offset, len, "ISO-8859-1"));
+        } catch (UnsupportedEncodingException e) {
+            throw new RuntimeException(e); // never happen
+        }
+    }
+
+    @Override
+    final protected int getValueBytesFromIdImpl(int id, byte[] returnValue, int offset) {
+        String date = getValueFromId(id);
+        byte bytes[];
+        try {
+            bytes = date.getBytes("ISO-8859-1");
+        } catch (UnsupportedEncodingException e) {
+            throw new RuntimeException(e); // never happen
+        }
+        System.arraycopy(bytes, 0, returnValue, offset, bytes.length);
+        return bytes.length;
+    }
+
+    private int calcIdFromSeqNo(int seq) {
+        return seq < 0 ? seq : baseId + seq;
+    }
+
+    private int calcSeqNoFromId(int id) {
+        return id - baseId;
+    }
+
+    @Override
+    public void write(DataOutput out) throws IOException {
+        out.writeUTF(pattern);
+        out.writeInt(baseId);
+    }
+
+    @Override
+    public void readFields(DataInput in) throws IOException {
+        String pattern = in.readUTF();
+        int baseId = in.readInt();
+        init(pattern, baseId);
+    }
+
+    @Override
+    public int hashCode() {
+        return 31 * baseId + pattern.hashCode();
+    }
+
+    @Override
+    public boolean equals(Object o) {
+        if ((o instanceof DateStrDictionary) == false)
+            return false;
+        DateStrDictionary that = (DateStrDictionary) o;
+        return StringUtils.equals(this.pattern, that.pattern) && this.baseId == that.baseId;
+    }
+
+    @Override
+    public void dump(PrintStream out) {
+        out.println(this.toString());
+    }
+
+    @Override
+    public String toString() {
+        return "DateStrDictionary [pattern=" + pattern + ", baseId=" + baseId + "]";
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/dictionary/src/main/java/com/kylinolap/dict/Dictionary.java
----------------------------------------------------------------------
diff --git a/dictionary/src/main/java/com/kylinolap/dict/Dictionary.java b/dictionary/src/main/java/com/kylinolap/dict/Dictionary.java
new file mode 100644
index 0000000..b54c6a3
--- /dev/null
+++ b/dictionary/src/main/java/com/kylinolap/dict/Dictionary.java
@@ -0,0 +1,189 @@
+/*
+ * 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 java.io.PrintStream;
+import java.io.UnsupportedEncodingException;
+
+import org.apache.hadoop.io.Writable;
+
+import com.kylinolap.common.util.BytesUtil;
+
+/**
+ * A bi-way dictionary that maps from dimension/column values to IDs and vice
+ * versa. By storing IDs instead of real values, the size of cube is
+ * significantly reduced.
+ * 
+ * - IDs are smallest integers possible for the cardinality of a column, for the
+ * purpose of minimal storage space - IDs preserve ordering of values, such that
+ * range query can be applied to IDs directly
+ * 
+ * A dictionary once built, is immutable. This allows optimal memory footprint
+ * by e.g. flatten the Trie structure into a byte array, replacing node pointers
+ * with array offsets.
+ * 
+ * @author yangli9
+ */
+abstract public class Dictionary<T> implements Writable {
+
+    public static final byte NULL = (byte) 0xff;
+
+    // ID with all bit-1 (0xff e.g.) reserved for NULL value
+    public static final int NULL_ID[] = new int[] { 0, 0xff, 0xffff, 0xffffff, 0xffffff };
+
+    abstract public int getMinId();
+
+    abstract public int getMaxId();
+
+    /**
+     * @return the size of an ID in bytes, determined by the cardinality of
+     *         column
+     */
+    abstract public int getSizeOfId();
+
+    /**
+     * @return the (maximum) size of value in bytes, determined by the longest
+     *         value of column
+     */
+    abstract public int getSizeOfValue();
+    
+    /**
+     * Convenient form of <code>getIdFromValue(value, 0)</code>
+     */
+    final public int getIdFromValue(T value) {
+        return getIdFromValue(value, 0);
+    }
+
+    /**
+     * Returns the ID integer of given value. In case of not found - if
+     * roundingFlag=0, throw IllegalArgumentException; - if roundingFlag<0, the
+     * closest smaller ID integer if exist; - if roundingFlag>0, the closest
+     * bigger ID integer if exist. The implementation often has cache, thus
+     * faster than the byte[] version getIdFromValueBytes()
+     * 
+     * @throws IllegalArgumentException
+     *             if value is not found in dictionary and rounding is off or
+     *             failed
+     */
+    final public int getIdFromValue(T value, int roundingFlag) {
+        if (isNullObjectForm(value))
+            return nullId();
+        else
+            return getIdFromValueImpl(value, roundingFlag);
+    }
+
+    protected boolean isNullObjectForm(T value) {
+        return value == null;
+    }
+
+    abstract protected int getIdFromValueImpl(T value, int roundingFlag);
+
+    /**
+     * @return the value corresponds to the given ID
+     * @throws IllegalArgumentException
+     *             if ID is not found in dictionary
+     */
+    final public T getValueFromId(int id) {
+        if (isNullId(id))
+            return null;
+        else
+            return getValueFromIdImpl(id);
+    }
+
+    abstract protected T getValueFromIdImpl(int id);
+
+    /**
+     * Convenient form of
+     * <code>getIdFromValueBytes(value, offset, len, 0)</code>
+     */
+    final public int getIdFromValueBytes(byte[] value, int offset, int len) {
+        return getIdFromValueBytes(value, offset, len, 0);
+    }
+
+    /**
+     * A lower level API, return ID integer from raw value bytes. In case of not
+     * found - if roundingFlag=0, throw IllegalArgumentException; - if
+     * roundingFlag<0, the closest smaller ID integer if exist; - if
+     * roundingFlag>0, the closest bigger ID integer if exist. Bypassing the
+     * cache layer, this could be significantly slower than getIdFromValue(T
+     * value).
+     * 
+     * @throws IllegalArgumentException
+     *             if value is not found in dictionary and rounding is off or
+     *             failed
+     */
+    final public int getIdFromValueBytes(byte[] value, int offset, int len, int roundingFlag) {
+        if (isNullByteForm(value, offset, len))
+            return nullId();
+        else
+            return getIdFromValueBytesImpl(value, offset, len, roundingFlag);
+    }
+    
+    protected boolean isNullByteForm(byte[] value, int offset, int len) {
+        return value == null;
+    }
+
+    abstract protected int getIdFromValueBytesImpl(byte[] value, int offset, int len, int roundingFlag);
+
+    /**
+     * A lower level API, get byte values from ID, return the number of bytes
+     * written. Bypassing the cache layer, this could be significantly slower
+     * than getIdFromValue(T value).
+     * 
+     * @throws IllegalArgumentException
+     *             if ID is not found in dictionary
+     */
+    final public int getValueBytesFromId(int id, byte[] returnValue, int offset) {
+        if (isNullId(id))
+            return 0;
+        else
+            return getValueBytesFromIdImpl(id, returnValue, offset);
+    }
+
+    abstract protected int getValueBytesFromIdImpl(int id, byte[] returnValue, int offset);
+
+    abstract public void dump(PrintStream out);
+
+    public int nullId() {
+        return NULL_ID[getSizeOfId()];
+    }
+
+    public boolean isNullId(int id) {
+        int nullId = NULL_ID[getSizeOfId()];
+        return (nullId & id) == nullId;
+    }
+
+    /** utility that converts a dictionary ID to string, preserving order */
+    public static String dictIdToString(byte[] idBytes, int offset, int length) {
+        try {
+            return new String(idBytes, offset, length, "ISO-8859-1");
+        } catch (UnsupportedEncodingException e) {
+            // never happen
+            return null;
+        }
+    }
+
+    /** the reverse of dictIdToString(), returns integer ID */
+    public static int stringToDictId(String str) {
+        try {
+            byte[] bytes = str.getBytes("ISO-8859-1");
+            return BytesUtil.readUnsigned(bytes, 0, bytes.length);
+        } catch (UnsupportedEncodingException e) {
+            // never happen
+            return 0;
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/dictionary/src/main/java/com/kylinolap/dict/DictionaryGenerator.java
----------------------------------------------------------------------
diff --git a/dictionary/src/main/java/com/kylinolap/dict/DictionaryGenerator.java b/dictionary/src/main/java/com/kylinolap/dict/DictionaryGenerator.java
new file mode 100644
index 0000000..16ae61c
--- /dev/null
+++ b/dictionary/src/main/java/com/kylinolap/dict/DictionaryGenerator.java
@@ -0,0 +1,203 @@
+/*
+ * 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 java.io.IOException;
+import java.text.ParseException;
+import java.text.SimpleDateFormat;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashSet;
+import java.util.List;
+
+import org.apache.hadoop.hbase.util.Bytes;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.google.common.collect.Lists;
+import com.kylinolap.common.util.JsonUtil;
+import com.kylinolap.dict.lookup.ReadableTable;
+import com.kylinolap.dict.lookup.TableReader;
+import com.kylinolap.metadata.model.schema.DataType;
+
+/**
+ * @author yangli9
+ */
+@SuppressWarnings({ "rawtypes", "unchecked" })
+public class DictionaryGenerator {
+
+    private static final Logger logger = LoggerFactory.getLogger(DictionaryGenerator.class);
+
+    private static final String[] DATE_PATTERNS = new String[] { "yyyy-MM-dd" };
+
+    public static Dictionary<?> buildDictionaryFromValueList(DictionaryInfo info, List<byte[]> values) {
+        info.setCardinality(values.size());
+
+        Dictionary dict = null;
+        int baseId = 0; // always 0 for now
+        int nSamples = 5;
+        ArrayList samples = new ArrayList();
+
+        // build dict, case by data type
+        DataType dataType = DataType.getInstance(info.getDataType());
+        if (dataType.isDateTimeFamily())
+            dict = buildDateStrDict(values, baseId, nSamples, samples);
+        else if (dataType.isNumberFamily())
+            dict = buildNumberDict(values, baseId, nSamples, samples);
+        else
+            dict = buildStringDict(values, baseId, nSamples, samples);
+
+        // log a few samples
+        StringBuilder buf = new StringBuilder();
+        for (Object s : samples) {
+            if (buf.length() > 0)
+                buf.append(", ");
+            buf.append(s.toString()).append("=>").append(dict.getIdFromValue(s));
+        }
+        logger.info("Dictionary value samples: " + buf.toString());
+        logger.info("Dictionary cardinality " + info.getCardinality());
+
+        if (values.size() > 1000000)
+            throw new IllegalArgumentException("Too high cardinality is not suitable for dictionary! Are the values stable enough for incremental load??");
+
+        return dict;
+    }
+
+    public static Dictionary mergeDictionaries(DictionaryInfo targetInfo, List<DictionaryInfo> sourceDicts) {
+
+        HashSet<byte[]> dedup = new HashSet<byte[]>();
+
+        for (DictionaryInfo info : sourceDicts) {
+            Dictionary<?> dict = info.getDictionaryObject();
+            int minkey = dict.getMinId();
+            int maxkey = dict.getMaxId();
+            byte[] buffer = new byte[dict.getSizeOfValue()];
+            for (int i = minkey; i <= maxkey; ++i) {
+                int size = dict.getValueBytesFromId(i, buffer, 0);
+                dedup.add(Bytes.copy(buffer, 0, size));
+            }
+        }
+
+        List<byte[]> valueList = new ArrayList<byte[]>();
+        valueList.addAll(dedup);
+
+        return buildDictionaryFromValueList(targetInfo, valueList);
+    }
+
+    public static Dictionary<?> buildDictionary(DictionaryInfo info, ReadableTable inpTable) throws IOException {
+
+        // currently all data types are casted to string to build dictionary
+        // String dataType = info.getDataType();
+
+        logger.info("Building dictionary " + JsonUtil.writeValueAsString(info));
+
+        ArrayList<byte[]> values = loadColumnValues(inpTable, info.getSourceColumnIndex());
+
+        return buildDictionaryFromValueList(info, values);
+    }
+
+    private static Dictionary buildDateStrDict(List<byte[]> values, int baseId, int nSamples, ArrayList samples) {
+        final int BAD_THRESHOLD = 2;
+        String matchPattern = null;
+        
+        for (String ptn : DATE_PATTERNS) {
+            matchPattern = ptn; // be optimistic
+            int badCount = 0;
+            SimpleDateFormat sdf = new SimpleDateFormat(ptn);
+            for (byte[] value : values) {
+                if (value.length == 0)
+                    continue;
+
+                String str = Bytes.toString(value);
+                try {
+                    sdf.parse(str);
+                    if (samples.size() < nSamples && samples.contains(str) == false)
+                        samples.add(str);
+                } catch (ParseException e) {
+                    logger.info("Unrecognized datetime value: " + str);
+                    badCount++;
+                    if (badCount > BAD_THRESHOLD) {
+                        matchPattern = null;
+                        break;
+                    }
+                }
+            }
+            if (matchPattern != null)
+                return new DateStrDictionary(matchPattern, baseId);
+        }
+        throw new IllegalStateException("Unrecognized datetime value");
+    }
+
+    private static Dictionary buildStringDict(List<byte[]> values, int baseId, int nSamples, ArrayList samples) {
+        TrieDictionaryBuilder builder = new TrieDictionaryBuilder(new StringBytesConverter());
+        for (byte[] value : values) {
+            String v = Bytes.toString(value);
+            builder.addValue(v);
+            if (samples.size() < nSamples && samples.contains(v) == false)
+                samples.add(v);
+        }
+        return builder.build(baseId);
+    }
+
+    private static Dictionary buildNumberDict(List<byte[]> values, int baseId, int nSamples, ArrayList samples) {
+        NumberDictionaryBuilder builder = new NumberDictionaryBuilder(new StringBytesConverter());
+        for (byte[] value : values) {
+            String v = Bytes.toString(value);
+            builder.addValue(v);
+            if (samples.size() < nSamples && samples.contains(v) == false)
+                samples.add(v);
+        }
+        return builder.build(baseId);
+    }
+
+    static ArrayList<byte[]> loadColumnValues(ReadableTable inpTable, int colIndex) throws IOException {
+
+        TableReader reader = inpTable.getReader();
+
+        try {
+            ArrayList<byte[]> result = Lists.newArrayList();
+            HashSet<String> dedup = new HashSet<String>();
+
+            while (reader.next()) {
+                String[] split = reader.getRow();
+
+                String colValue;
+                // special single column file, e.g. common_indicator.txt
+                if (split.length == 1) {
+                    colValue = split[0];
+                }
+                // normal case
+                else {
+                    if (split.length <= colIndex) {
+                        throw new ArrayIndexOutOfBoundsException("Column no. " + colIndex + " not found, line split is " + Arrays.asList(split));
+                    }
+                    colValue = split[colIndex];
+                }
+
+                if (dedup.contains(colValue) == false) {
+                    dedup.add(colValue);
+                    result.add(Bytes.toBytes(colValue));
+                }
+            }
+            return result;
+
+        } finally {
+            reader.close();
+        }
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/dictionary/src/main/java/com/kylinolap/dict/DictionaryInfo.java
----------------------------------------------------------------------
diff --git a/dictionary/src/main/java/com/kylinolap/dict/DictionaryInfo.java b/dictionary/src/main/java/com/kylinolap/dict/DictionaryInfo.java
new file mode 100644
index 0000000..89e01b2
--- /dev/null
+++ b/dictionary/src/main/java/com/kylinolap/dict/DictionaryInfo.java
@@ -0,0 +1,169 @@
+/*
+ * 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 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.dict.lookup.TableSignature;
+
+@JsonAutoDetect(fieldVisibility = Visibility.NONE, getterVisibility = Visibility.NONE, isGetterVisibility = Visibility.NONE, setterVisibility = Visibility.NONE)
+public class DictionaryInfo extends RootPersistentEntity {
+
+    @JsonProperty("source_table")
+    private String sourceTable;
+    @JsonProperty("source_column")
+    private String sourceColumn;
+    @JsonProperty("source_column_index")
+    private int sourceColumnIndex; // 0 based
+    @JsonProperty("data_type")
+    private String dataType;
+    @JsonProperty("input")
+    private TableSignature input;
+    @JsonProperty("input_delimeter")
+    private String inputDelimeter;
+    @JsonProperty("dictionary_class")
+    private String dictionaryClass;
+    @JsonProperty("cardinality")
+    private int cardinality;
+
+    transient Dictionary<?> dictionaryObject;
+
+    public DictionaryInfo() {
+    }
+
+    public DictionaryInfo(String sourceTable, String sourceColumn, int sourceColumnIndex, String dataType, TableSignature input, String inputDelimeter) {
+
+        this.updateRandomUuid();
+
+        this.sourceTable = sourceTable;
+        this.sourceColumn = sourceColumn;
+        this.sourceColumnIndex = sourceColumnIndex;
+        this.dataType = dataType;
+        this.input = input;
+        this.inputDelimeter = inputDelimeter;
+    }
+
+    public DictionaryInfo(DictionaryInfo other) {
+
+        this.updateRandomUuid();
+
+        this.sourceTable = other.sourceTable;
+        this.sourceColumn = other.sourceColumn;
+        this.sourceColumnIndex = other.sourceColumnIndex;
+        this.dataType = other.dataType;
+        this.input = other.input;
+        this.inputDelimeter = other.inputDelimeter;
+    }
+
+    // ----------------------------------------------------------------------------
+
+    public String getResourcePath() {
+        return ResourceStore.DICT_RESOURCE_ROOT + "/" + sourceTable + "/" + sourceColumn + "/" + uuid + ".dict";
+    }
+
+    public String getResourceDir() {
+        return ResourceStore.DICT_RESOURCE_ROOT + "/" + sourceTable + "/" + sourceColumn;
+    }
+
+    // ----------------------------------------------------------------------------
+
+    // to decide if two dictionaries are built on the same table/column,
+    // regardless of their signature
+    public boolean isDictOnSameColumn(DictionaryInfo other) {
+        return this.sourceTable.equalsIgnoreCase(other.sourceTable) &&
+                this.sourceColumn.equalsIgnoreCase(other.sourceColumn) &&
+                this.sourceColumnIndex == other.sourceColumnIndex &&
+                this.dataType.equalsIgnoreCase(other.dataType) &&
+                this.inputDelimeter.equalsIgnoreCase(other.inputDelimeter) &&
+                this.dictionaryClass.equalsIgnoreCase(other.dictionaryClass);
+    }
+
+    public String getSourceTable() {
+        return sourceTable;
+    }
+
+    public void setSourceTable(String sourceTable) {
+        this.sourceTable = sourceTable;
+    }
+
+    public String getSourceColumn() {
+        return sourceColumn;
+    }
+
+    public void setSourceColumn(String sourceColumn) {
+        this.sourceColumn = sourceColumn;
+    }
+
+    public int getSourceColumnIndex() {
+        return sourceColumnIndex;
+    }
+
+    public void setSourceColumnIndex(int sourceColumnIndex) {
+        this.sourceColumnIndex = sourceColumnIndex;
+    }
+
+    public String getDataType() {
+        return dataType;
+    }
+
+    public void setDataType(String dataType) {
+        this.dataType = dataType;
+    }
+
+    public TableSignature getInput() {
+        return input;
+    }
+
+    public void setInput(TableSignature input) {
+        this.input = input;
+    }
+
+    public String getInputDelimeter() {
+        return inputDelimeter;
+    }
+
+    public void setInputDelimeter(String inputDelimeter) {
+        this.inputDelimeter = inputDelimeter;
+    }
+
+    public String getDictionaryClass() {
+        return dictionaryClass;
+    }
+
+    public void setDictionaryClass(String dictionaryClass) {
+        this.dictionaryClass = dictionaryClass;
+    }
+
+    public Dictionary<?> getDictionaryObject() {
+        return dictionaryObject;
+    }
+
+    public void setDictionaryObject(Dictionary<?> dictionaryObject) {
+        this.dictionaryObject = dictionaryObject;
+    }
+
+    public int getCardinality() {
+        return cardinality;
+    }
+
+    public void setCardinality(int cardinality) {
+        this.cardinality = cardinality;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/dictionary/src/main/java/com/kylinolap/dict/DictionaryInfoSerializer.java
----------------------------------------------------------------------
diff --git a/dictionary/src/main/java/com/kylinolap/dict/DictionaryInfoSerializer.java b/dictionary/src/main/java/com/kylinolap/dict/DictionaryInfoSerializer.java
new file mode 100644
index 0000000..623d25a
--- /dev/null
+++ b/dictionary/src/main/java/com/kylinolap/dict/DictionaryInfoSerializer.java
@@ -0,0 +1,76 @@
+/*
+ * 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 java.io.DataInputStream;
+import java.io.DataOutputStream;
+import java.io.IOException;
+
+import com.kylinolap.common.persistence.Serializer;
+import com.kylinolap.common.util.JsonUtil;
+
+/**
+ * @author yangli9
+ * 
+ */
+public class DictionaryInfoSerializer implements Serializer<DictionaryInfo> {
+
+    public static final DictionaryInfoSerializer FULL_SERIALIZER = new DictionaryInfoSerializer(false);
+    public static final DictionaryInfoSerializer INFO_SERIALIZER = new DictionaryInfoSerializer(true);
+
+    private boolean infoOnly;
+
+    public DictionaryInfoSerializer() {
+        this(false);
+    }
+
+    public DictionaryInfoSerializer(boolean infoOnly) {
+        this.infoOnly = infoOnly;
+    }
+
+    @Override
+    public void serialize(DictionaryInfo obj, DataOutputStream out) throws IOException {
+        String json = JsonUtil.writeValueAsIndentString(obj);
+        out.writeUTF(json);
+
+        if (infoOnly == false)
+            obj.getDictionaryObject().write(out);
+    }
+
+    @Override
+    public DictionaryInfo deserialize(DataInputStream in) throws IOException {
+        String json = in.readUTF();
+        DictionaryInfo obj = JsonUtil.readValue(json, DictionaryInfo.class);
+
+        if (infoOnly == false) {
+            Dictionary<?> dict;
+            try {
+                dict = (Dictionary<?>) Class.forName(obj.getDictionaryClass()).newInstance();
+            } catch (InstantiationException e) {
+                throw new RuntimeException(e);
+            } catch (IllegalAccessException e) {
+                throw new RuntimeException(e);
+            } catch (ClassNotFoundException e) {
+                throw new RuntimeException(e);
+            }
+            dict.readFields(in);
+            obj.setDictionaryObject(dict);
+        }
+        return obj;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/dictionary/src/main/java/com/kylinolap/dict/DictionaryManager.java
----------------------------------------------------------------------
diff --git a/dictionary/src/main/java/com/kylinolap/dict/DictionaryManager.java b/dictionary/src/main/java/com/kylinolap/dict/DictionaryManager.java
new file mode 100644
index 0000000..73bc9d9
--- /dev/null
+++ b/dictionary/src/main/java/com/kylinolap/dict/DictionaryManager.java
@@ -0,0 +1,345 @@
+/*
+ * 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 java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.ConcurrentHashMap;
+
+import org.apache.commons.compress.utils.IOUtils;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.kylinolap.common.KylinConfig;
+import com.kylinolap.common.persistence.ResourceStore;
+import com.kylinolap.common.util.HadoopUtil;
+import com.kylinolap.dict.lookup.FileTable;
+import com.kylinolap.dict.lookup.HiveTable;
+import com.kylinolap.dict.lookup.ReadableTable;
+import com.kylinolap.dict.lookup.TableSignature;
+import com.kylinolap.metadata.MetadataManager;
+import com.kylinolap.metadata.model.cube.CubeDesc;
+import com.kylinolap.metadata.model.cube.TblColRef;
+
+public class DictionaryManager {
+
+    private static final Logger logger = LoggerFactory.getLogger(DictionaryManager.class);
+
+    private static final DictionaryInfo NONE_INDICATOR = new DictionaryInfo();
+
+    // static cached instances
+    private static final ConcurrentHashMap<KylinConfig, DictionaryManager> SERVICE_CACHE = new ConcurrentHashMap<KylinConfig, DictionaryManager>();
+
+    public static DictionaryManager getInstance(KylinConfig config) {
+        DictionaryManager r = SERVICE_CACHE.get(config);
+        if (r == null) {
+            r = new DictionaryManager(config);
+            SERVICE_CACHE.put(config, r);
+            if (SERVICE_CACHE.size() > 1) {
+                logger.warn("More than one singleton exist");
+            }
+        }
+        return r;
+    }
+
+    public static void removeInstance(KylinConfig config) {
+        SERVICE_CACHE.remove(config);
+    }
+
+    // ============================================================================
+
+    private KylinConfig config;
+    private ConcurrentHashMap<String, DictionaryInfo> dictCache; // resource
+    // path ==>
+    // DictionaryInfo
+
+    private DictionaryManager(KylinConfig config) {
+        this.config = config;
+        dictCache = new ConcurrentHashMap<String, DictionaryInfo>();
+    }
+
+    public Dictionary<?> getDictionary(String resourcePath) throws IOException {
+        DictionaryInfo dictInfo = getDictionaryInfo(resourcePath);
+        return dictInfo == null ? null : dictInfo.getDictionaryObject();
+    }
+
+    public DictionaryInfo getDictionaryInfo(String resourcePath) throws IOException {
+        DictionaryInfo dictInfo = dictCache.get(resourcePath);
+        if (dictInfo == null) {
+            dictInfo = load(resourcePath, true);
+            if (dictInfo == null)
+                dictInfo = NONE_INDICATOR;
+            dictCache.put(resourcePath, dictInfo);
+        }
+        return dictInfo == NONE_INDICATOR ? null : dictInfo;
+    }
+
+    public DictionaryInfo trySaveNewDict(Dictionary<?> newDict, DictionaryInfo newDictInfo) throws IOException {
+
+        String dupDict = checkDupByContent(newDictInfo, newDict);
+        if (dupDict != null) {
+            logger.info("Identical dictionary content " + newDict + ", reuse existing dictionary at " + dupDict);
+            return getDictionaryInfo(dupDict);
+        }
+
+        newDictInfo.setDictionaryObject(newDict);
+        newDictInfo.setDictionaryClass(newDict.getClass().getName());
+
+        save(newDictInfo);
+        dictCache.put(newDictInfo.getResourcePath(), newDictInfo);
+
+        return newDictInfo;
+    }
+
+    public DictionaryInfo mergeDictionary(List<DictionaryInfo> dicts) throws IOException {
+        DictionaryInfo firstDictInfo = null;
+        int totalSize = 0;
+        for (DictionaryInfo info : dicts) {
+            // check
+            if (firstDictInfo == null) {
+                firstDictInfo = info;
+            } else {
+                if (!firstDictInfo.isDictOnSameColumn(info)) {
+                    throw new IllegalArgumentException("Merging dictionaries are not structurally equal(regardless of signature).");
+                }
+            }
+            totalSize += info.getInput().getSize();
+        }
+
+        if (firstDictInfo == null) {
+            throw new IllegalArgumentException("DictionaryManager.mergeDictionary input cannot be null");
+        }
+
+        DictionaryInfo newDictInfo = new DictionaryInfo(firstDictInfo);
+        TableSignature signature = newDictInfo.getInput();
+        signature.setSize(totalSize);
+        signature.setLastModifiedTime(System.currentTimeMillis());
+        signature.setPath("merged_with_no_original_path");
+
+        String dupDict = checkDupByInfo(newDictInfo);
+        if (dupDict != null) {
+            logger.info("Identical dictionary input " + newDictInfo.getInput() + ", reuse existing dictionary at " + dupDict);
+            return getDictionaryInfo(dupDict);
+        }
+
+        Dictionary<?> newDict = DictionaryGenerator.mergeDictionaries(newDictInfo, dicts);
+
+        return trySaveNewDict(newDict, newDictInfo);
+    }
+
+    public DictionaryInfo buildDictionary(CubeDesc cube, TblColRef col, String factColumnsPath) throws IOException {
+
+        Object[] tmp = decideSourceData(cube, col, factColumnsPath);
+        String srcTable = (String) tmp[0];
+        String srcCol = (String) tmp[1];
+        int srcColIdx = (Integer) tmp[2];
+        ReadableTable inpTable = (ReadableTable) tmp[3];
+
+        DictionaryInfo dictInfo = new DictionaryInfo(srcTable, srcCol, srcColIdx, col.getDatatype(), inpTable.getSignature(), inpTable.getColumnDelimeter());
+
+        String dupDict = checkDupByInfo(dictInfo);
+        if (dupDict != null) {
+            logger.info("Identical dictionary input " + dictInfo.getInput() + ", reuse existing dictionary at " + dupDict);
+            return getDictionaryInfo(dupDict);
+        }
+
+        Dictionary<?> dict = DictionaryGenerator.buildDictionary(dictInfo, inpTable);
+
+        return trySaveNewDict(dict, dictInfo);
+    }
+
+    /**
+     * Get column origin
+     *
+     * @return 1. source table name
+     * 2. column name
+     * 3. column cardinal in source table
+     * 4. ReadableTable object
+     */
+    public Object[] decideSourceData(CubeDesc cube, TblColRef col, String factColumnsPath) throws IOException {
+        String srcTable;
+        String srcCol;
+        int srcColIdx;
+        ReadableTable table;
+        MetadataManager metaMgr = MetadataManager.getInstance(config);
+
+        // case of full table (dict on fact table)
+        if (cube == null) {
+            srcTable = col.getTable();
+            srcCol = col.getName();
+            srcColIdx = col.getColumn().getZeroBasedIndex();
+            int nColumns = metaMgr.getTableDesc(col.getTable()).getColumnCount();
+            table = new FileTable(factColumnsPath + "/" + col.getName(), nColumns);
+            return new Object[] { srcTable, srcCol, srcColIdx, table };
+        }
+
+        // Decide source data of dictionary:
+        // 1. If 'useDict' specifies pre-defined data set, use that
+        // 2. Otherwise find a lookup table to scan through
+
+        // Note FK on fact table is supported by scan the related PK on lookup
+        // table
+
+        String useDict = cube.getRowkey().getDictionary(col);
+
+        // normal case, source from lookup table
+        if ("true".equals(useDict) || "string".equals(useDict) || "number".equals(useDict) || "any".equals(useDict)) {
+            // FK on fact table, use PK from lookup instead
+            if (cube.isFactTable(col.getTable())) {
+                TblColRef pkCol = cube.findPKByFK(col);
+                if (pkCol != null)
+                    col = pkCol; // scan the counterparty PK on lookup table
+                // instead
+            }
+            srcTable = col.getTable();
+            srcCol = col.getName();
+            srcColIdx = col.getColumn().getZeroBasedIndex();
+            if (cube.isFactTable(col.getTable())) {
+                table = new FileTable(factColumnsPath + "/" + col.getName(), -1);
+            } else {
+                table = new HiveTable(metaMgr, col.getTable());
+            }
+        }
+        // otherwise could refer to a data set, e.g. common_indicators.txt
+        // (LEGACY PATH, since distinct values are collected from fact table)
+        else {
+            String dictDataSetPath = unpackDataSet(this.config.getTempHDFSDir(), useDict);
+            if (dictDataSetPath == null)
+                throw new IllegalArgumentException("Unknown dictionary data set '" + useDict + "', referred from " + col);
+            srcTable = "PREDEFINED";
+            srcCol = useDict;
+            srcColIdx = 0;
+            table = new FileTable(dictDataSetPath, -1);
+        }
+
+        return new Object[] { srcTable, srcCol, srcColIdx, table };
+    }
+
+    private String unpackDataSet(String tempHDFSDir, String dataSetName) throws IOException {
+
+        InputStream in = this.getClass().getResourceAsStream("/com/kylinolap/dict/" + dataSetName + ".txt");
+        if (in == null) // data set resource not found
+            return null;
+
+        ByteArrayOutputStream buf = new ByteArrayOutputStream();
+        IOUtils.copy(in, buf);
+        in.close();
+        byte[] bytes = buf.toByteArray();
+
+        Path tmpDataSetPath = new Path(tempHDFSDir + "/dict/temp_dataset/" + dataSetName + "_" + bytes.length + ".txt");
+
+        FileSystem fs = HadoopUtil.getFileSystem(tempHDFSDir);
+        boolean writtenNewFile = false;
+        if (fs.exists(tmpDataSetPath) == false || fs.getFileStatus(tmpDataSetPath).getLen() != bytes.length) {
+            fs.mkdirs(tmpDataSetPath.getParent());
+            FSDataOutputStream out = fs.create(tmpDataSetPath);
+            IOUtils.copy(new ByteArrayInputStream(bytes), out);
+            out.close();
+            writtenNewFile = true;
+        }
+
+        String qualifiedPath = tmpDataSetPath.makeQualified(fs.getUri(), new Path("/")).toString();
+        if (writtenNewFile)
+            logger.info("Dictionary temp data set file written to " + qualifiedPath);
+        return qualifiedPath;
+    }
+
+    private String checkDupByInfo(DictionaryInfo dictInfo) throws IOException {
+        ResourceStore store = MetadataManager.getInstance(config).getStore();
+        ArrayList<String> existings = store.listResources(dictInfo.getResourceDir());
+        if (existings == null)
+            return null;
+
+        TableSignature input = dictInfo.getInput();
+        for (String existing : existings) {
+            DictionaryInfo existingInfo = load(existing, false); // skip cache,
+            // direct
+            // load from
+            // store
+            if (input.equals(existingInfo.getInput()))
+                return existing;
+        }
+
+        return null;
+    }
+
+    private String checkDupByContent(DictionaryInfo dictInfo, Dictionary<?> dict) throws IOException {
+        ResourceStore store = MetadataManager.getInstance(config).getStore();
+        ArrayList<String> existings = store.listResources(dictInfo.getResourceDir());
+        if (existings == null)
+            return null;
+
+        for (String existing : existings) {
+            logger.info("Checking dup dict :" + existing);
+            DictionaryInfo existingInfo = load(existing, true); // skip cache,
+            // direct load
+            // from store
+            if(existingInfo == null)
+                logger.info("existingInfo is null");
+
+            if (existingInfo != null && dict.equals(existingInfo.getDictionaryObject()))
+                return existing;
+        }
+
+        return null;
+    }
+
+    public void removeDictionary(String resourcePath) throws IOException {
+        ResourceStore store = MetadataManager.getInstance(config).getStore();
+        store.deleteResource(resourcePath);
+        dictCache.remove(resourcePath);
+    }
+
+    public void removeDictionaries(String srcTable, String srcCol) throws IOException {
+        DictionaryInfo info = new DictionaryInfo();
+        info.setSourceTable(srcTable);
+        info.setSourceColumn(srcCol);
+
+        ResourceStore store = MetadataManager.getInstance(config).getStore();
+        ArrayList<String> existings = store.listResources(info.getResourceDir());
+        if (existings == null)
+            return;
+
+        for (String existing : existings)
+            removeDictionary(existing);
+    }
+
+    void save(DictionaryInfo dict) throws IOException {
+        ResourceStore store = MetadataManager.getInstance(config).getStore();
+        String path = dict.getResourcePath();
+        logger.info("Saving dictionary at " + path);
+        store.putResource(path, dict, DictionaryInfoSerializer.FULL_SERIALIZER);
+    }
+
+    DictionaryInfo load(String resourcePath, boolean loadDictObj) throws IOException {
+        ResourceStore store = MetadataManager.getInstance(config).getStore();
+
+        DictionaryInfo info = store.getResource(resourcePath, DictionaryInfo.class, loadDictObj ? DictionaryInfoSerializer.FULL_SERIALIZER : DictionaryInfoSerializer.INFO_SERIALIZER);
+
+        if (loadDictObj)
+            logger.debug("Loaded dictionary at " + resourcePath);
+
+        return info;
+    }
+
+}


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

Posted by lu...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/job/src/test/resources/data/test_cal_dt/part-r-00000
----------------------------------------------------------------------
diff --git a/job/src/test/resources/data/test_cal_dt/part-r-00000 b/job/src/test/resources/data/test_cal_dt/part-r-00000
new file mode 100644
index 0000000..206c349
--- /dev/null
+++ b/job/src/test/resources/data/test_cal_dt/part-r-00000
@@ -0,0 +1,366 @@
+2013-08-16,2013-01-01,2013-07-01,2013-08-01,2013-08-11,0,-1,-3,-15,-103,0,-1,-4,-15,-15,41501,228,47,16,6,33,5928,2,8,1364,3,455,114,2013-12-31,2013-09-30,2013-08-31,2013-08-17,16-Aug-2013,Aug 16th 2013,Fri 08-16-13,1,0,0,0,2013-06-21,365,92,31,7,2012-12-30,2013-06-30,2013-07-28,2013-08-11,2013-08-12,2013-08-16,Fri       ,2013M08,Aug-2013,N,2013M08   ,N,Year 2013 - Quarter 03,2013Q03   ,N,33,2013,2013-08-11,2013-08-17,N,Wk.33 - 13,2013-08-11 00:00:00,2013-08-17 00:00:00,2013W33   ,2013W33   ,08/11/13 - 08/17/13,08/11 - 08/17,2013,N,2012-08-16,2011-08-16,2013-05-16,2013-02-16,2013-07-16,2013-06-16,2013-08-09,2013-08-02,0,0,0,0,0,0,0,0,8,3,33,3,1,1,1,2005-09-07,DW_BATCH  ,2013-11-27 00:16:56,DW_BATCH
+2013-01-03,2013-01-01,2013-01-01,2013-01-01,2013-01-01,0,-3,-10,-47,-328,0,-3,-11,-47,-47,41276,3,3,3,5,1,5896,1,1,1357,1,453,114,2013-12-31,2013-03-31,2013-01-31,2013-01-05,03-Jan-2013,Jan 3rd 2013,Thu 01-03-13,1,0,0,0,2012-12-21,365,90,31,5,2012-12-30,2012-12-30,2012-12-30,2012-12-30,2012-12-31,2013-01-03,Thu       ,2013M01,Jan-2013,N,2013M01   ,N,Year 2013 - Quarter 01,2013Q01   ,N,1,2013,2012-12-30,2013-01-05,N,Wk.01 - 13,2013-01-01 00:00:00,2013-01-05 00:00:00,2013W01   ,2013W01   ,01/01/13 - 01/05/13,01/01 - 01/05,2013,N,2012-01-03,2011-01-03,2012-10-03,2012-07-03,2012-12-03,2012-11-03,2012-12-27,2012-12-20,0,0,0,0,0,0,0,0,1,1,1,1,1,1,1,2005-09-07,DW_BATCH  ,2013-11-27 00:16:56,DW_BATCH
+2013-04-10,2013-01-01,2013-04-01,2013-04-01,2013-04-07,0,-2,-7,-33,-231,0,-2,-8,-33,-33,41373,100,10,10,4,15,5910,1,4,1360,2,454,114,2013-12-31,2013-06-30,2013-04-30,2013-04-13,10-Apr-2013,Apr 10th 2013,Wed 04-10-13,1,0,0,0,2013-03-21,365,91,30,7,2012-12-30,2013-03-31,2013-03-31,2013-04-07,2013-04-08,2013-04-10,Wed       ,2013M04,Apr-2013,N,2013M04   ,N,Year 2013 - Quarter 02,2013Q02   ,N,15,2013,2013-04-07,2013-04-13,N,Wk.15 - 13,2013-04-07 00:00:00,2013-04-13 00:00:00,2013W15   ,2013W15   ,04/07/13 - 04/13/13,04/07 - 04/13,2013,N,2012-04-10,2011-04-10,2013-01-10,2012-10-10,2013-03-10,2013-02-10,2013-04-03,2013-03-27,0,0,0,0,0,0,0,0,4,2,15,2,1,1,1,2005-09-07,DW_BATCH  ,2013-11-27 00:16:56,DW_BATCH
+2013-06-12,2013-01-01,2013-04-01,2013-06-01,2013-06-09,0,-2,-5,-24,-168,0,-2,-6,-24,-24,41436,163,73,12,4,24,5919,3,6,1362,2,454,114,2013-12-31,2013-06-30,2013-06-30,2013-06-15,12-Jun-2013,Jun 12th 2013,Wed 06-12-13,1,0,0,0,2013-03-21,365,91,30,7,2012-12-30,2013-03-31,2013-05-26,2013-06-09,2013-06-10,2013-06-12,Wed       ,2013M06,Jun-2013,N,2013M06   ,N,Year 2013 - Quarter 02,2013Q02   ,N,24,2013,2013-06-09,2013-06-15,N,Wk.24 - 13,2013-06-09 00:00:00,2013-06-15 00:00:00,2013W24   ,2013W24   ,06/09/13 - 06/15/13,06/09 - 06/15,2013,N,2012-06-12,2011-06-12,2013-03-12,2012-12-12,2013-05-12,2013-04-12,2013-06-05,2013-05-29,0,0,0,0,0,0,0,0,6,2,24,2,1,1,1,2005-09-07,DW_BATCH  ,2013-11-27 00:16:56,DW_BATCH
+2013-04-27,2013-01-01,2013-04-01,2013-04-01,2013-04-21,0,-2,-7,-31,-214,0,-2,-8,-31,-31,41390,117,27,27,7,17,5912,1,4,1360,2,454,114,2013-12-31,2013-06-30,2013-04-30,2013-04-27,27-Apr-2013,Apr 27th 2013,Sat 04-27-13,1,0,0,0,2013-03-21,365,91,30,7,2012-12-30,2013-03-31,2013-03-31,2013-04-21,2013-04-22,2013-04-27,Sat       ,2013M04,Apr-2013,N,2013M04   ,N,Year 2013 - Quarter 02,2013Q02   ,N,17,2013,2013-04-21,2013-04-27,N,Wk.17 - 13,2013-04-21 00:00:00,2013-04-27 00:00:00,2013W17   ,2013W17   ,04/21/13 - 04/27/13,04/21 - 04/27,2013,N,2012-04-27,2011-04-27,2013-01-27,2012-10-27,2013-03-27,2013-02-27,2013-04-20,2013-04-13,0,0,0,0,0,0,0,0,4,2,17,2,1,1,1,2005-09-07,DW_BATCH  ,2013-11-27 00:16:56,DW_BATCH
+2013-09-19,2013-01-01,2013-07-01,2013-09-01,2013-09-15,0,-1,-2,-10,-69,0,-1,-3,-10,-10,41535,262,81,19,5,38,5933,3,9,1365,3,455,114,2013-12-31,2013-09-30,2013-09-30,2013-09-21,19-Sep-2013,Sep 19th 2013,Thu 09-19-13,1,0,0,0,2013-06-21,365,92,30,7,2012-12-30,2013-06-30,2013-08-25,2013-09-15,2013-09-16,2013-09-19,Thu       ,2013M09,Sep-2013,N,2013M09   ,N,Year 2013 - Quarter 03,2013Q03   ,N,38,2013,2013-09-15,2013-09-21,N,Wk.38 - 13,2013-09-15 00:00:00,2013-09-21 00:00:00,2013W38   ,2013W38   ,09/15/13 - 09/21/13,09/15 - 09/21,2013,N,2012-09-19,2011-09-19,2013-06-19,2013-03-19,2013-08-19,2013-07-19,2013-09-12,2013-09-05,0,0,0,0,0,0,0,0,9,3,38,3,1,1,1,2005-09-07,DW_BATCH  ,2013-11-27 00:16:56,DW_BATCH
+2013-03-07,2013-01-01,2013-01-01,2013-03-01,2013-03-03,0,-3,-8,-38,-265,0,-3,-9,-38,-38,41339,66,66,7,5,10,5905,3,3,1359,1,453,114,2013-12-31,2013-03-31,2013-03-31,2013-03-09,07-Mar-2013,Mar 7th 2013,Thu 03-07-13,1,0,0,0,2012-12-21,365,90,31,7,2012-12-30,2012-12-30,2013-02-24,2013-03-03,2013-03-04,2013-03-07,Thu       ,2013M03,Mar-2013,N,2013M03   ,N,Year 2013 - Quarter 01,2013Q01   ,N,10,2013,2013-03-03,2013-03-09,N,Wk.10 - 13,2013-03-03 00:00:00,2013-03-09 00:00:00,2013W10   ,2013W10   ,03/03/13 - 03/09/13,03/03 - 03/09,2013,N,2012-03-07,2011-03-07,2012-12-07,2012-09-07,2013-02-07,2013-01-07,2013-02-28,2013-02-21,0,0,0,0,0,0,0,0,3,1,10,1,1,1,1,2005-09-07,DW_BATCH  ,2013-11-27 00:16:56,DW_BATCH
+2013-03-28,2013-01-01,2013-01-01,2013-03-01,2013-03-24,0,-3,-8,-35,-244,0,-3,-9,-35,-35,41360,87,87,28,5,13,5908,3,3,1359,1,453,114,2013-12-31,2013-03-31,2013-03-31,2013-03-30,28-Mar-2013,Mar 28th 2013,Thu 03-28-13,1,0,0,0,2013-03-21,365,90,31,7,2012-12-30,2012-12-30,2013-02-24,2013-03-24,2013-03-25,2013-03-28,Thu       ,2013M03,Mar-2013,N,2013M03   ,N,Year 2013 - Quarter 01,2013Q01   ,N,13,2013,2013-03-24,2013-03-30,N,Wk.13 - 13,2013-03-24 00:00:00,2013-03-30 00:00:00,2013W13   ,2013W13   ,03/24/13 - 03/30/13,03/24 - 03/30,2013,N,2012-03-28,2011-03-28,2012-12-28,2012-09-28,2013-02-28,2013-01-28,2013-03-21,2013-03-14,0,0,0,0,0,0,0,0,3,1,13,2,1,1,1,2005-09-07,DW_BATCH  ,2013-11-27 00:16:56,DW_BATCH
+2013-11-23,2013-01-01,2013-10-01,2013-11-01,2013-11-17,0,0,0,-1,-4,0,0,-1,-1,-1,41600,327,54,23,7,47,5942,2,11,1367,4,456,114,2013-12-31,2013-12-31,2013-11-30,2013-11-23,23-Nov-2013,Nov 23rd 2013,Sat 11-23-13,1,1,1,0,2013-09-21,365,92,30,7,2012-12-30,2013-09-29,2013-10-27,2013-11-17,2013-11-18,2013-11-23,Sat       ,2013M11,Nov-2013,N,2013M11   ,N,Year 2013 - Quarter 04,2013Q04   ,N,47,2013,2013-11-17,2013-11-23,N,Wk.47 - 13,2013-11-17 00:00:00,2013-11-23 00:00:00,2013W47   ,2013W47   ,11/17/13 - 11/23/13,11/17 - 11/23,2013,N,2012-11-23,2011-11-23,2013-08-23,2013-05-23,2013-10-23,2013-09-23,2013-11-16,2013-11-09,0,0,0,0,0,0,0,0,11,4,47,4,1,1,1,2005-09-07,DW_BATCH  ,2013-11-27 00:16:56,DW_BATCH
+2013-03-09,2013-01-01,2013-01-01,2013-03-01,2013-03-03,0,-3,-8,-38,-263,0,-3,-9,-38,-38,41341,68,68,9,7,10,5905,3,3,1359,1,453,114,2013-12-31,2013-03-31,2013-03-31,2013-03-09,09-Mar-2013,Mar 9th 2013,Sat 03-09-13,1,0,0,0,2012-12-21,365,90,31,7,2012-12-30,2012-12-30,2013-02-24,2013-03-03,2013-03-04,2013-03-09,Sat       ,2013M03,Mar-2013,N,2013M03   ,N,Year 2013 - Quarter 01,2013Q01   ,N,10,2013,2013-03-03,2013-03-09,N,Wk.10 - 13,2013-03-03 00:00:00,2013-03-09 00:00:00,2013W10   ,2013W10   ,03/03/13 - 03/09/13,03/03 - 03/09,2013,N,2012-03-09,2011-03-09,2012-12-09,2012-09-09,2013-02-09,2013-01-09,2013-03-02,2013-02-23,0,0,0,0,0,0,0,0,3,1,10,1,1,1,1,2005-09-07,DW_BATCH  ,2013-11-27 00:16:56,DW_BATCH
+2013-11-21,2013-01-01,2013-10-01,2013-11-01,2013-11-17,0,0,0,-1,-6,0,0,-1,-1,-1,41598,325,52,21,5,47,5942,2,11,1367,4,456,114,2013-12-31,2013-12-31,2013-11-30,2013-11-23,21-Nov-2013,Nov 21st 2013,Thu 11-21-13,1,1,1,0,2013-09-21,365,92,30,7,2012-12-30,2013-09-29,2013-10-27,2013-11-17,2013-11-18,2013-11-21,Thu       ,2013M11,Nov-2013,N,2013M11   ,N,Year 2013 - Quarter 04,2013Q04   ,N,47,2013,2013-11-17,2013-11-23,N,Wk.47 - 13,2013-11-17 00:00:00,2013-11-23 00:00:00,2013W47   ,2013W47   ,11/17/13 - 11/23/13,11/17 - 11/23,2013,N,2012-11-21,2011-11-21,2013-08-21,2013-05-21,2013-10-21,2013-09-21,2013-11-14,2013-11-07,0,0,0,0,0,0,0,0,11,4,47,4,1,1,1,2005-09-07,DW_BATCH  ,2013-11-27 00:16:56,DW_BATCH
+2013-03-26,2013-01-01,2013-01-01,2013-03-01,2013-03-24,0,-3,-8,-35,-246,0,-3,-9,-35,-35,41358,85,85,26,3,13,5908,3,3,1359,1,453,114,2013-12-31,2013-03-31,2013-03-31,2013-03-30,26-Mar-2013,Mar 26th 2013,Tue 03-26-13,1,0,0,0,2013-03-21,365,90,31,7,2012-12-30,2012-12-30,2013-02-24,2013-03-24,2013-03-25,2013-03-26,Tue       ,2013M03,Mar-2013,N,2013M03   ,N,Year 2013 - Quarter 01,2013Q01   ,N,13,2013,2013-03-24,2013-03-30,N,Wk.13 - 13,2013-03-24 00:00:00,2013-03-30 00:00:00,2013W13   ,2013W13   ,03/24/13 - 03/30/13,03/24 - 03/30,2013,N,2012-03-26,2011-03-26,2012-12-26,2012-09-26,2013-02-26,2013-01-26,2013-03-19,2013-03-12,0,0,0,0,0,0,0,0,3,1,13,2,1,1,1,2005-09-07,DW_BATCH  ,2013-11-27 00:16:56,DW_BATCH
+2013-01-24,2013-01-01,2013-01-01,2013-01-01,2013-01-20,0,-3,-10,-44,-307,0,-3,-11,-44,-44,41297,24,24,24,5,4,5899,1,1,1357,1,453,114,2013-12-31,2013-03-31,2013-01-31,2013-01-26,24-Jan-2013,Jan 24th 2013,Thu 01-24-13,1,0,0,0,2012-12-21,365,90,31,7,2012-12-30,2012-12-30,2012-12-30,2013-01-20,2013-01-21,2013-01-24,Thu       ,2013M01,Jan-2013,N,2013M01   ,N,Year 2013 - Quarter 01,2013Q01   ,N,4,2013,2013-01-20,2013-01-26,N,Wk.04 - 13,2013-01-20 00:00:00,2013-01-26 00:00:00,2013W04   ,2013W04   ,01/20/13 - 01/26/13,01/20 - 01/26,2013,N,2012-01-24,2011-01-24,2012-10-24,2012-07-24,2012-12-24,2012-11-24,2013-01-17,2013-01-10,0,0,0,0,0,0,0,0,1,1,4,1,1,1,1,2005-09-07,DW_BATCH  ,2013-11-27 00:16:56,DW_BATCH
+2013-01-01,2013-01-01,2013-01-01,2013-01-01,2013-01-01,0,-3,-10,-47,-330,0,-3,-11,-47,-47,41274,1,1,1,3,1,5896,1,1,1357,1,453,114,2013-12-31,2013-03-31,2013-01-31,2013-01-05,01-Jan-2013,Jan 1st 2013,Tue 01-01-13,1,0,0,0,2012-12-21,365,90,31,5,2012-12-30,2012-12-30,2012-12-30,2012-12-30,2012-12-31,2013-01-01,Tue       ,2013M01,Jan-2013,Y,2013M01   ,Y,Year 2013 - Quarter 01,2013Q01   ,Y,1,2013,2012-12-30,2013-01-05,Y,Wk.01 - 13,2013-01-01 00:00:00,2013-01-05 00:00:00,2013W01   ,2013W01   ,01/01/13 - 01/05/13,01/01 - 01/05,2013,Y,2012-01-01,2011-01-01,2012-10-01,2012-07-01,2012-12-01,2012-11-01,2012-12-25,2012-12-18,0,0,0,0,0,0,0,0,1,1,1,1,1,1,1,2005-09-07,DW_BATCH  ,2013-11-27 00:16:56,DW_BATCH
+2013-04-29,2013-01-01,2013-04-01,2013-04-01,2013-04-28,0,-2,-7,-30,-212,0,-2,-7,-30,-30,41392,119,29,29,2,18,5913,1,4,1360,2,454,114,2013-12-31,2013-06-30,2013-04-30,2013-05-04,29-Apr-2013,Apr 29th 2013,Mon 04-29-13,1,0,0,0,2013-03-21,365,91,30,7,2012-12-30,2013-03-31,2013-04-28,2013-04-28,2013-04-29,2013-04-29,Mon       ,2013M04,Apr-2013,N,2013M04   ,N,Year 2013 - Quarter 02,2013Q02   ,N,18,2013,2013-04-28,2013-05-04,N,Wk.18 - 13,2013-04-28 00:00:00,2013-05-04 00:00:00,2013W18   ,2013W18   ,04/28/13 - 05/04/13,04/28 - 05/04,2013,N,2012-04-29,2011-04-29,2013-01-29,2012-10-29,2013-03-29,2013-02-28,2013-04-22,2013-04-15,0,0,0,0,0,0,0,0,5,2,18,2,1,1,1,2005-09-07,DW_BATCH  ,2013-11-27 00:16:56,DW_BATCH
+2013-05-11,2013-01-01,2013-04-01,2013-05-01,2013-05-05,0,-2,-6,-29,-200,0,-2,-7,-29,-29,41404,131,41,11,7,19,5914,2,5,1361,2,454,114,2013-12-31,2013-06-30,2013-05-31,2013-05-11,11-May-2013,May 11th 2013,Sat 05-11-13,1,0,0,0,2013-03-21,365,91,31,7,2012-12-30,2013-03-31,2013-04-28,2013-05-05,2013-05-06,2013-05-11,Sat       ,2013M05,May-2013,N,2013M05   ,N,Year 2013 - Quarter 02,2013Q02   ,N,19,2013,2013-05-05,2013-05-11,N,Wk.19 - 13,2013-05-05 00:00:00,2013-05-11 00:00:00,2013W19   ,2013W19   ,05/05/13 - 05/11/13,05/05 - 05/11,2013,N,2012-05-11,2011-05-11,2013-02-11,2012-11-11,2013-04-11,2013-03-11,2013-05-04,2013-04-27,0,0,0,0,0,0,0,0,5,2,19,2,1,1,1,2005-09-07,DW_BATCH  ,2013-11-27 00:16:56,DW_BATCH
+2013-10-01,2013-01-01,2013-10-01,2013-10-01,2013-09-29,0,0,-1,-8,-57,0,0,-2,-8,-8,41547,274,1,1,3,40,5935,1,10,1366,4,456,114,2013-12-31,2013-12-31,2013-10-31,2013-10-05,01-Oct-2013,Oct 1st 2013,Tue 10-01-13,1,1,0,0,2013-09-21,365,92,31,7,2012-12-30,2013-09-29,2013-09-29,2013-09-29,2013-09-30,2013-10-01,Tue       ,2013M10,Oct-2013,Y,2013M10   ,Y,Year 2013 - Quarter 04,2013Q04   ,Y,40,2013,2013-09-29,2013-10-05,N,Wk.40 - 13,2013-09-29 00:00:00,2013-10-05 00:00:00,2013W40   ,2013W40   ,09/29/13 - 10/05/13,09/29 - 10/05,2013,N,2012-10-01,2011-10-01,2013-07-01,2013-04-01,2013-09-01,2013-08-01,2013-09-24,2013-09-17,0,0,0,0,0,0,0,0,10,4,40,4,1,1,1,2005-09-07,DW_BATCH  ,2013-11-27 00:16:56,DW_BATCH
+2013-12-05,2013-01-01,2013-10-01,2013-12-01,2013-12-01,0,0,1,1,8,0,0,0,1,1,41612,339,66,5,5,49,5944,3,12,1368,4,456,114,2013-12-31,2013-12-31,2013-12-31,2013-12-07,05-Dec-2013,Dec 5th 2013,Thu 12-05-13,0,0,0,0,2013-09-21,365,92,31,7,2012-12-30,2013-09-29,2013-11-24,2013-12-01,2013-12-02,2013-12-05,Thu       ,2013M12,Dec-2013,N,2013M12   ,N,Year 2013 - Quarter 04,2013Q04   ,N,49,2013,2013-12-01,2013-12-07,N,Wk.49 - 13,2013-12-01 00:00:00,2013-12-07 00:00:00,2013W49   ,2013W49   ,12/01/13 - 12/07/13,12/01 - 12/07,2013,N,2012-12-05,2011-12-05,2013-09-05,2013-06-05,2013-11-05,2013-10-05,2013-11-28,2013-11-21,0,0,0,0,0,0,0,0,12,4,49,4,0,1,0,2005-09-07,DW_BATCH  ,2013-11-27 00:16:56,DW_BATCH
+2013-03-05,2013-01-01,2013-01-01,2013-03-01,2013-03-03,0,-3,-8,-38,-267,0,-3,-9,-38,-38,41337,64,64,5,3,10,5905,3,3,1359,1,453,114,2013-12-31,2013-03-31,2013-03-31,2013-03-09,05-Mar-2013,Mar 5th 2013,Tue 03-05-13,1,0,0,0,2012-12-21,365,90,31,7,2012-12-30,2012-12-30,2013-02-24,2013-03-03,2013-03-04,2013-03-05,Tue       ,2013M03,Mar-2013,N,2013M03   ,N,Year 2013 - Quarter 01,2013Q01   ,N,10,2013,2013-03-03,2013-03-09,N,Wk.10 - 13,2013-03-03 00:00:00,2013-03-09 00:00:00,2013W10   ,2013W10   ,03/03/13 - 03/09/13,03/03 - 03/09,2013,N,2012-03-05,2011-03-05,2012-12-05,2012-09-05,2013-02-05,2013-01-05,2013-02-26,2013-02-19,0,0,0,0,0,0,0,0,3,1,10,1,1,1,1,2005-09-07,DW_BATCH  ,2013-11-27 00:16:56,DW_BATCH
+2013-10-03,2013-01-01,2013-10-01,2013-10-01,2013-09-29,0,0,-1,-8,-55,0,0,-2,-8,-8,41549,276,3,3,5,40,5935,1,10,1366,4,456,114,2013-12-31,2013-12-31,2013-10-31,2013-10-05,03-Oct-2013,Oct 3rd 2013,Thu 10-03-13,1,1,0,0,2013-09-21,365,92,31,7,2012-12-30,2013-09-29,2013-09-29,2013-09-29,2013-09-30,2013-10-03,Thu       ,2013M10,Oct-2013,N,2013M10   ,N,Year 2013 - Quarter 04,2013Q04   ,N,40,2013,2013-09-29,2013-10-05,N,Wk.40 - 13,2013-09-29 00:00:00,2013-10-05 00:00:00,2013W40   ,2013W40   ,09/29/13 - 10/05/13,09/29 - 10/05,2013,N,2012-10-03,2011-10-03,2013-07-03,2013-04-03,2013-09-03,2013-08-03,2013-09-26,2013-09-19,0,0,0,0,0,0,0,0,10,4,40,4,1,1,1,2005-09-07,DW_BATCH  ,2013-11-27 00:16:56,DW_BATCH
+2013-01-22,2013-01-01,2013-01-01,2013-01-01,2013-01-20,0,-3,-10,-44,-309,0,-3,-11,-44,-44,41295,22,22,22,3,4,5899,1,1,1357,1,453,114,2013-12-31,2013-03-31,2013-01-31,2013-01-26,22-Jan-2013,Jan 22nd 2013,Tue 01-22-13,1,0,0,0,2012-12-21,365,90,31,7,2012-12-30,2012-12-30,2012-12-30,2013-01-20,2013-01-21,2013-01-22,Tue       ,2013M01,Jan-2013,N,2013M01   ,N,Year 2013 - Quarter 01,2013Q01   ,N,4,2013,2013-01-20,2013-01-26,N,Wk.04 - 13,2013-01-20 00:00:00,2013-01-26 00:00:00,2013W04   ,2013W04   ,01/20/13 - 01/26/13,01/20 - 01/26,2013,N,2012-01-22,2011-01-22,2012-10-22,2012-07-22,2012-12-22,2012-11-22,2013-01-15,2013-01-08,0,0,0,0,0,0,0,0,1,1,4,1,1,1,1,2005-09-07,DW_BATCH  ,2013-11-27 00:16:56,DW_BATCH
+2013-12-26,2013-01-01,2013-10-01,2013-12-01,2013-12-22,0,0,1,4,29,0,0,0,4,4,41633,360,87,26,5,52,5947,3,12,1368,4,456,114,2013-12-31,2013-12-31,2013-12-31,2013-12-28,26-Dec-2013,Dec 26th 2013,Thu 12-26-13,0,0,0,0,2013-12-21,365,92,31,7,2012-12-30,2013-09-29,2013-11-24,2013-12-22,2013-12-23,2013-12-26,Thu       ,2013M12,Dec-2013,N,2013M12   ,N,Year 2013 - Quarter 04,2013Q04   ,N,52,2013,2013-12-22,2013-12-28,N,Wk.52 - 13,2013-12-22 00:00:00,2013-12-28 00:00:00,2013W52   ,2013W52   ,12/22/13 - 12/28/13,12/22 - 12/28,2013,N,2012-12-26,2011-12-26,2013-09-26,2013-06-26,2013-11-26,2013-10-26,2013-12-19,2013-12-12,0,0,0,0,0,0,0,0,12,4,52,1,0,1,0,2005-09-07,DW_BATCH  ,2013-11-27 00:16:56,DW_BATCH
+2013-09-17,2013-01-01,2013-07-01,2013-09-01,2013-09-15,0,-1,-2,-10,-71,0,-1,-3,-10,-10,41533,260,79,17,3,38,5933,3,9,1365,3,455,114,2013-12-31,2013-09-30,2013-09-30,2013-09-21,17-Sep-2013,Sep 17th 2013,Tue 09-17-13,1,0,0,0,2013-06-21,365,92,30,7,2012-12-30,2013-06-30,2013-08-25,2013-09-15,2013-09-16,2013-09-17,Tue       ,2013M09,Sep-2013,N,2013M09   ,N,Year 2013 - Quarter 03,2013Q03   ,N,38,2013,2013-09-15,2013-09-21,N,Wk.38 - 13,2013-09-15 00:00:00,2013-09-21 00:00:00,2013W38   ,2013W38   ,09/15/13 - 09/21/13,09/15 - 09/21,2013,N,2012-09-17,2011-09-17,2013-06-17,2013-03-17,2013-08-17,2013-07-17,2013-09-10,2013-09-03,0,0,0,0,0,0,0,0,9,3,38,3,1,1,1,2005-09-07,DW_BATCH  ,2013-11-27 00:16:56,DW_BATCH
+2013-10-20,2013-01-01,2013-10-01,2013-10-01,2013-10-20,0,0,-1,-5,-38,0,0,-2,-5,-6,41566,293,20,20,1,43,5938,1,10,1366,4,456,114,2013-12-31,2013-12-31,2013-10-31,2013-10-26,20-Oct-2013,Oct 20th 2013,Sun 10-20-13,1,1,0,0,2013-09-21,365,92,31,7,2012-12-30,2013-09-29,2013-09-29,2013-10-20,2013-10-14,2013-10-20,Sun       ,2013M10,Oct-2013,N,2013M10   ,N,Year 2013 - Quarter 04,2013Q04   ,N,43,2013,2013-10-20,2013-10-26,Y,Wk.43 - 13,2013-10-20 00:00:00,2013-10-26 00:00:00,2013W43   ,2013W43   ,10/20/13 - 10/26/13,10/20 - 10/26,2013,N,2012-10-20,2011-10-20,2013-07-20,2013-04-20,2013-09-20,2013-08-20,2013-10-13,2013-10-06,0,0,0,0,0,0,0,0,10,4,43,4,1,1,1,2005-09-07,DW_BATCH  ,2013-11-27 00:16:56,DW_BATCH
+2013-11-25,2013-01-01,2013-10-01,2013-11-01,2013-11-24,0,0,0,0,-2,0,0,0,0,0,41602,329,56,25,2,48,5943,2,11,1367,4,456,114,2013-12-31,2013-12-31,2013-11-30,2013-11-30,25-Nov-2013,Nov 25th 2013,Mon 11-25-13,1,1,1,1,2013-09-21,365,92,30,7,2012-12-30,2013-09-29,2013-11-24,2013-11-24,2013-11-25,2013-11-25,Mon       ,2013M11,Nov-2013,N,2013M11   ,N,Year 2013 - Quarter 04,2013Q04   ,N,48,2013,2013-11-24,2013-11-30,N,Wk.48 - 13,2013-11-24 00:00:00,2013-11-30 00:00:00,2013W48   ,2013W48   ,11/24/13 - 11/30/13,11/24 - 11/30,2013,N,2012-11-25,2011-11-25,2013-08-25,2013-05-25,2013-10-25,2013-09-25,2013-11-18,2013-11-11,0,0,0,0,0,0,0,0,12,4,48,4,1,1,1,2005-09-07,DW_BATCH  ,2013-11-27 00:16:56,DW_BATCH
+2013-01-26,2013-01-01,2013-01-01,2013-01-01,2013-01-20,0,-3,-10,-44,-305,0,-3,-11,-44,-44,41299,26,26,26,7,4,5899,1,1,1357,1,453,114,2013-12-31,2013-03-31,2013-01-31,2013-01-26,26-Jan-2013,Jan 26th 2013,Sat 01-26-13,1,0,0,0,2012-12-21,365,90,31,7,2012-12-30,2012-12-30,2012-12-30,2013-01-20,2013-01-21,2013-01-26,Sat       ,2013M01,Jan-2013,N,2013M01   ,N,Year 2013 - Quarter 01,2013Q01   ,N,4,2013,2013-01-20,2013-01-26,N,Wk.04 - 13,2013-01-20 00:00:00,2013-01-26 00:00:00,2013W04   ,2013W04   ,01/20/13 - 01/26/13,01/20 - 01/26,2013,N,2012-01-26,2011-01-26,2012-10-26,2012-07-26,2012-12-26,2012-11-26,2013-01-19,2013-01-12,0,0,0,0,0,0,0,0,1,1,4,1,1,1,1,2005-09-07,DW_BATCH  ,2013-11-27 00:16:56,DW_BATCH
+2013-12-24,2013-01-01,2013-10-01,2013-12-01,2013-12-22,0,0,1,4,27,0,0,0,4,4,41631,358,85,24,3,52,5947,3,12,1368,4,456,114,2013-12-31,2013-12-31,2013-12-31,2013-12-28,24-Dec-2013,Dec 24th 2013,Tue 12-24-13,0,0,0,0,2013-12-21,365,92,31,7,2012-12-30,2013-09-29,2013-11-24,2013-12-22,2013-12-23,2013-12-24,Tue       ,2013M12,Dec-2013,N,2013M12   ,N,Year 2013 - Quarter 04,2013Q04   ,N,52,2013,2013-12-22,2013-12-28,N,Wk.52 - 13,2013-12-22 00:00:00,2013-12-28 00:00:00,2013W52   ,2013W52   ,12/22/13 - 12/28/13,12/22 - 12/28,2013,N,2012-12-24,2011-12-24,2013-09-24,2013-06-24,2013-11-24,2013-10-24,2013-12-17,2013-12-10,0,0,0,0,0,0,0,0,12,4,52,1,0,1,0,2005-09-07,DW_BATCH  ,2013-11-27 00:16:56,DW_BATCH
+2013-02-04,2013-01-01,2013-01-01,2013-02-01,2013-02-03,0,-3,-9,-42,-296,0,-3,-10,-42,-42,41308,35,35,4,2,6,5901,2,2,1358,1,453,114,2013-12-31,2013-03-31,2013-02-28,2013-02-09,04-Feb-2013,Feb 4th 2013,Mon 02-04-13,1,0,0,0,2012-12-21,365,90,28,7,2012-12-30,2012-12-30,2013-01-27,2013-02-03,2013-02-04,2013-02-04,Mon       ,2013M02,Feb-2013,N,2013M02   ,N,Year 2013 - Quarter 01,2013Q01   ,N,6,2013,2013-02-03,2013-02-09,N,Wk.06 - 13,2013-02-03 00:00:00,2013-02-09 00:00:00,2013W06   ,2013W06   ,02/03/13 - 02/09/13,02/03 - 02/09,2013,N,2012-02-04,2011-02-04,2012-11-04,2012-08-04,2013-01-04,2012-12-04,2013-01-28,2013-01-21,0,0,0,0,0,0,0,0,2,1,6,1,1,1,1,2005-09-07,DW_BATCH  ,2013-11-27 00:16:56,DW_BATCH
+2013-05-30,2013-01-01,2013-04-01,2013-05-01,2013-05-26,0,-2,-6,-26,-181,0,-2,-6,-26,-26,41423,150,60,30,5,22,5917,2,5,1361,2,454,114,2013-12-31,2013-06-30,2013-05-31,2013-06-01,30-May-2013,May 30th 2013,Thu 05-30-13,1,0,0,0,2013-03-21,365,91,31,7,2012-12-30,2013-03-31,2013-05-26,2013-05-26,2013-05-27,2013-05-30,Thu       ,2013M05,May-2013,N,2013M05   ,N,Year 2013 - Quarter 02,2013Q02   ,N,22,2013,2013-05-26,2013-06-01,N,Wk.22 - 13,2013-05-26 00:00:00,2013-06-01 00:00:00,2013W22   ,2013W22   ,05/26/13 - 06/01/13,05/26 - 06/01,2013,N,2012-05-30,2011-05-30,2013-02-28,2012-11-30,2013-04-30,2013-03-30,2013-05-23,2013-05-16,0,0,0,0,0,0,0,0,6,2,22,2,1,1,1,2005-09-07,DW_BATCH  ,2013-11-27 00:16:56,DW_BATCH
+2013-04-12,2013-01-01,2013-04-01,2013-04-01,2013-04-07,0,-2,-7,-33,-229,0,-2,-8,-33,-33,41375,102,12,12,6,15,5910,1,4,1360,2,454,114,2013-12-31,2013-06-30,2013-04-30,2013-04-13,12-Apr-2013,Apr 12th 2013,Fri 04-12-13,1,0,0,0,2013-03-21,365,91,30,7,2012-12-30,2013-03-31,2013-03-31,2013-04-07,2013-04-08,2013-04-12,Fri       ,2013M04,Apr-2013,N,2013M04   ,N,Year 2013 - Quarter 02,2013Q02   ,N,15,2013,2013-04-07,2013-04-13,N,Wk.15 - 13,2013-04-07 00:00:00,2013-04-13 00:00:00,2013W15   ,2013W15   ,04/07/13 - 04/13/13,04/07 - 04/13,2013,N,2012-04-12,2011-04-12,2013-01-12,2012-10-12,2013-03-12,2013-02-12,2013-04-05,2013-03-29,0,0,0,0,0,0,0,0,4,2,15,2,1,1,1,2005-09-07,DW_BATCH  ,2013-11-27 00:16:56,DW_BATCH
+2013-04-08,2013-01-01,2013-04-01,2013-04-01,2013-04-07,0,-2,-7,-33,-233,0,-2,-8,-33,-33,41371,98,8,8,2,15,5910,1,4,1360,2,454,114,2013-12-31,2013-06-30,2013-04-30,2013-04-13,08-Apr-2013,Apr 8th 2013,Mon 04-08-13,1,0,0,0,2013-03-21,365,91,30,7,2012-12-30,2013-03-31,2013-03-31,2013-04-07,2013-04-08,2013-04-08,Mon       ,2013M04,Apr-2013,N,2013M04   ,N,Year 2013 - Quarter 02,2013Q02   ,N,15,2013,2013-04-07,2013-04-13,N,Wk.15 - 13,2013-04-07 00:00:00,2013-04-13 00:00:00,2013W15   ,2013W15   ,04/07/13 - 04/13/13,04/07 - 04/13,2013,N,2012-04-08,2011-04-08,2013-01-08,2012-10-08,2013-03-08,2013-02-08,2013-04-01,2013-03-25,0,0,0,0,0,0,0,0,4,2,15,2,1,1,1,2005-09-07,DW_BATCH  ,2013-11-27 00:16:56,DW_BATCH
+2013-12-28,2013-01-01,2013-10-01,2013-12-01,2013-12-22,0,0,1,4,31,0,0,0,4,4,41635,362,89,28,7,52,5947,3,12,1368,4,456,114,2013-12-31,2013-12-31,2013-12-31,2013-12-28,28-Dec-2013,Dec 28th 2013,Sat 12-28-13,0,0,0,0,2013-12-21,365,92,31,7,2012-12-30,2013-09-29,2013-11-24,2013-12-22,2013-12-23,2013-12-28,Sat       ,2013M12,Dec-2013,N,2013M12   ,N,Year 2013 - Quarter 04,2013Q04   ,N,52,2013,2013-12-22,2013-12-28,N,Wk.52 - 13,2013-12-22 00:00:00,2013-12-28 00:00:00,2013W52   ,2013W52   ,12/22/13 - 12/28/13,12/22 - 12/28,2013,N,2012-12-28,2011-12-28,2013-09-28,2013-06-28,2013-11-28,2013-10-28,2013-12-21,2013-12-14,0,0,0,0,0,0,0,0,12,4,52,1,0,1,0,2005-09-07,DW_BATCH  ,2013-11-27 00:16:56,DW_BATCH
+2013-12-03,2013-01-01,2013-10-01,2013-12-01,2013-12-01,0,0,1,1,6,0,0,0,1,1,41610,337,64,3,3,49,5944,3,12,1368,4,456,114,2013-12-31,2013-12-31,2013-12-31,2013-12-07,03-Dec-2013,Dec 3rd 2013,Tue 12-03-13,0,0,0,0,2013-09-21,365,92,31,7,2012-12-30,2013-09-29,2013-11-24,2013-12-01,2013-12-02,2013-12-03,Tue       ,2013M12,Dec-2013,N,2013M12   ,N,Year 2013 - Quarter 04,2013Q04   ,N,49,2013,2013-12-01,2013-12-07,N,Wk.49 - 13,2013-12-01 00:00:00,2013-12-07 00:00:00,2013W49   ,2013W49   ,12/01/13 - 12/07/13,12/01 - 12/07,2013,N,2012-12-03,2011-12-03,2013-09-03,2013-06-03,2013-11-03,2013-10-03,2013-11-26,2013-11-19,0,0,0,0,0,0,0,0,12,4,49,4,0,1,0,2005-09-07,DW_BATCH  ,2013-11-27 00:16:56,DW_BATCH
+2013-07-15,2013-01-01,2013-07-01,2013-07-01,2013-07-14,0,-1,-4,-19,-135,0,-1,-5,-19,-19,41469,196,15,15,2,29,5924,1,7,1363,3,455,114,2013-12-31,2013-09-30,2013-07-31,2013-07-20,15-Jul-2013,Jul 15th 2013,Mon 07-15-13,1,0,0,0,2013-06-21,365,92,31,7,2012-12-30,2013-06-30,2013-06-30,2013-07-14,2013-07-15,2013-07-15,Mon       ,2013M07,Jul-2013,N,2013M07   ,N,Year 2013 - Quarter 03,2013Q03   ,N,29,2013,2013-07-14,2013-07-20,N,Wk.29 - 13,2013-07-14 00:00:00,2013-07-20 00:00:00,2013W29   ,2013W29   ,07/14/13 - 07/20/13,07/14 - 07/20,2013,N,2012-07-15,2011-07-15,2013-04-15,2013-01-15,2013-06-15,2013-05-15,2013-07-08,2013-07-01,0,0,0,0,0,0,0,0,7,3,29,3,1,1,1,2005-09-07,DW_BATCH  ,2013-11-27 00:16:56,DW_BATCH
+2013-05-13,2013-01-01,2013-04-01,2013-05-01,2013-05-12,0,-2,-6,-28,-198,0,-2,-7,-28,-28,41406,133,43,13,2,20,5915,2,5,1361,2,454,114,2013-12-31,2013-06-30,2013-05-31,2013-05-18,13-May-2013,May 13th 2013,Mon 05-13-13,1,0,0,0,2013-03-21,365,91,31,7,2012-12-30,2013-03-31,2013-04-28,2013-05-12,2013-05-13,2013-05-13,Mon       ,2013M05,May-2013,N,2013M05   ,N,Year 2013 - Quarter 02,2013Q02   ,N,20,2013,2013-05-12,2013-05-18,N,Wk.20 - 13,2013-05-12 00:00:00,2013-05-18 00:00:00,2013W20   ,2013W20   ,05/12/13 - 05/18/13,05/12 - 05/18,2013,N,2012-05-13,2011-05-13,2013-02-13,2012-11-13,2013-04-13,2013-03-13,2013-05-06,2013-04-29,0,0,0,0,0,0,0,0,5,2,20,2,1,1,1,2005-09-07,DW_BATCH  ,2013-11-27 00:16:56,DW_BATCH
+2013-03-30,2013-01-01,2013-01-01,2013-03-01,2013-03-24,0,-3,-8,-35,-242,0,-3,-9,-35,-35,41362,89,89,30,7,13,5908,3,3,1359,1,453,114,2013-12-31,2013-03-31,2013-03-31,2013-03-30,30-Mar-2013,Mar 30th 2013,Sat 03-30-13,1,0,0,0,2013-03-21,365,90,31,7,2012-12-30,2012-12-30,2013-02-24,2013-03-24,2013-03-25,2013-03-30,Sat       ,2013M03,Mar-2013,N,2013M03   ,N,Year 2013 - Quarter 01,2013Q01   ,N,13,2013,2013-03-24,2013-03-30,N,Wk.13 - 13,2013-03-24 00:00:00,2013-03-30 00:00:00,2013W13   ,2013W13   ,03/24/13 - 03/30/13,03/24 - 03/30,2013,N,2012-03-30,2011-03-30,2012-12-30,2012-09-30,2013-02-28,2013-01-30,2013-03-23,2013-03-16,0,0,0,0,0,0,0,0,3,1,13,2,1,1,1,2005-09-07,DW_BATCH  ,2013-11-27 00:16:56,DW_BATCH
+2013-12-09,2013-01-01,2013-10-01,2013-12-01,2013-12-08,0,0,1,2,12,0,0,0,2,2,41616,343,70,9,2,50,5945,3,12,1368,4,456,114,2013-12-31,2013-12-31,2013-12-31,2013-12-14,09-Dec-2013,Dec 9th 2013,Mon 12-09-13,0,0,0,0,2013-09-21,365,92,31,7,2012-12-30,2013-09-29,2013-11-24,2013-12-08,2013-12-09,2013-12-09,Mon       ,2013M12,Dec-2013,N,2013M12   ,N,Year 2013 - Quarter 04,2013Q04   ,N,50,2013,2013-12-08,2013-12-14,N,Wk.50 - 13,2013-12-08 00:00:00,2013-12-14 00:00:00,2013W50   ,2013W50   ,12/08/13 - 12/14/13,12/08 - 12/14,2013,N,2012-12-09,2011-12-09,2013-09-09,2013-06-09,2013-11-09,2013-10-09,2013-12-02,2013-11-25,0,0,0,0,0,0,0,0,12,4,50,4,0,1,0,2005-09-07,DW_BATCH  ,2013-11-27 00:16:56,DW_BATCH
+2013-02-25,2013-01-01,2013-01-01,2013-02-01,2013-02-24,0,-3,-9,-39,-275,0,-3,-9,-39,-39,41329,56,56,25,2,9,5904,2,2,1358,1,453,114,2013-12-31,2013-03-31,2013-02-28,2013-03-02,25-Feb-2013,Feb 25th 2013,Mon 02-25-13,1,0,0,0,2012-12-21,365,90,28,7,2012-12-30,2012-12-30,2013-02-24,2013-02-24,2013-02-25,2013-02-25,Mon       ,2013M02,Feb-2013,N,2013M02   ,N,Year 2013 - Quarter 01,2013Q01   ,N,9,2013,2013-02-24,2013-03-02,N,Wk.09 - 13,2013-02-24 00:00:00,2013-03-02 00:00:00,2013W09   ,2013W09   ,02/24/13 - 03/02/13,02/24 - 03/02,2013,N,2012-02-25,2011-02-25,2012-11-25,2012-08-25,2013-01-25,2012-12-25,2013-02-18,2013-02-11,0,0,0,0,0,0,0,0,3,1,9,1,1,1,1,2005-09-07,DW_BATCH  ,2013-11-27 00:16:56,DW_BATCH
+2013-11-04,2013-01-01,2013-10-01,2013-11-01,2013-11-03,0,0,0,-3,-23,0,0,-1,-3,-3,41581,308,35,4,2,45,5940,2,11,1367,4,456,114,2013-12-31,2013-12-31,2013-11-30,2013-11-09,04-Nov-2013,Nov 4th 2013,Mon 11-04-13,1,1,1,0,2013-09-21,365,92,30,7,2012-12-30,2013-09-29,2013-10-27,2013-11-03,2013-11-04,2013-11-04,Mon       ,2013M11,Nov-2013,N,2013M11   ,N,Year 2013 - Quarter 04,2013Q04   ,N,45,2013,2013-11-03,2013-11-09,N,Wk.45 - 13,2013-11-03 00:00:00,2013-11-09 00:00:00,2013W45   ,2013W45   ,11/03/13 - 11/09/13,11/03 - 11/09,2013,N,2012-11-04,2011-11-04,2013-08-04,2013-05-04,2013-10-04,2013-09-04,2013-10-28,2013-10-21,0,0,0,0,0,0,0,0,11,4,45,4,1,1,1,2005-09-07,DW_BATCH  ,2013-11-27 00:16:56,DW_BATCH
+2013-11-08,2013-01-01,2013-10-01,2013-11-01,2013-11-03,0,0,0,-3,-19,0,0,-1,-3,-3,41585,312,39,8,6,45,5940,2,11,1367,4,456,114,2013-12-31,2013-12-31,2013-11-30,2013-11-09,08-Nov-2013,Nov 8th 2013,Fri 11-08-13,1,1,1,0,2013-09-21,365,92,30,7,2012-12-30,2013-09-29,2013-10-27,2013-11-03,2013-11-04,2013-11-08,Fri       ,2013M11,Nov-2013,N,2013M11   ,N,Year 2013 - Quarter 04,2013Q04   ,N,45,2013,2013-11-03,2013-11-09,N,Wk.45 - 13,2013-11-03 00:00:00,2013-11-09 00:00:00,2013W45   ,2013W45   ,11/03/13 - 11/09/13,11/03 - 11/09,2013,N,2012-11-08,2011-11-08,2013-08-08,2013-05-08,2013-10-08,2013-09-08,2013-11-01,2013-10-25,0,0,0,0,0,0,0,0,11,4,45,4,1,1,1,2005-09-07,DW_BATCH  ,2013-11-27 00:16:56,DW_BATCH
+2013-08-22,2013-01-01,2013-07-01,2013-08-01,2013-08-18,0,-1,-3,-14,-97,0,-1,-4,-14,-14,41507,234,53,22,5,34,5929,2,8,1364,3,455,114,2013-12-31,2013-09-30,2013-08-31,2013-08-24,22-Aug-2013,Aug 22nd 2013,Thu 08-22-13,1,0,0,0,2013-06-21,365,92,31,7,2012-12-30,2013-06-30,2013-07-28,2013-08-18,2013-08-19,2013-08-22,Thu       ,2013M08,Aug-2013,N,2013M08   ,N,Year 2013 - Quarter 03,2013Q03   ,N,34,2013,2013-08-18,2013-08-24,N,Wk.34 - 13,2013-08-18 00:00:00,2013-08-24 00:00:00,2013W34   ,2013W34   ,08/18/13 - 08/24/13,08/18 - 08/24,2013,N,2012-08-22,2011-08-22,2013-05-22,2013-02-22,2013-07-22,2013-06-22,2013-08-15,2013-08-08,0,0,0,0,0,0,0,0,8,3,34,3,1,1,1,2005-09-07,DW_BATCH  ,2013-11-27 00:16:56,DW_BATCH
+2013-06-18,2013-01-01,2013-04-01,2013-06-01,2013-06-16,0,-2,-5,-23,-162,0,-2,-6,-23,-23,41442,169,79,18,3,25,5920,3,6,1362,2,454,114,2013-12-31,2013-06-30,2013-06-30,2013-06-22,18-Jun-2013,Jun 18th 2013,Tue 06-18-13,1,0,0,0,2013-03-21,365,91,30,7,2012-12-30,2013-03-31,2013-05-26,2013-06-16,2013-06-17,2013-06-18,Tue       ,2013M06,Jun-2013,N,2013M06   ,N,Year 2013 - Quarter 02,2013Q02   ,N,25,2013,2013-06-16,2013-06-22,N,Wk.25 - 13,2013-06-16 00:00:00,2013-06-22 00:00:00,2013W25   ,2013W25   ,06/16/13 - 06/22/13,06/16 - 06/22,2013,N,2012-06-18,2011-06-18,2013-03-18,2012-12-18,2013-05-18,2013-04-18,2013-06-11,2013-06-04,0,0,0,0,0,0,0,0,6,2,25,2,1,1,1,2005-09-07,DW_BATCH  ,2013-11-27 00:16:56,DW_BATCH
+2013-07-19,2013-01-01,2013-07-01,2013-07-01,2013-07-14,0,-1,-4,-19,-131,0,-1,-5,-19,-19,41473,200,19,19,6,29,5924,1,7,1363,3,455,114,2013-12-31,2013-09-30,2013-07-31,2013-07-20,19-Jul-2013,Jul 19th 2013,Fri 07-19-13,1,0,0,0,2013-06-21,365,92,31,7,2012-12-30,2013-06-30,2013-06-30,2013-07-14,2013-07-15,2013-07-19,Fri       ,2013M07,Jul-2013,N,2013M07   ,N,Year 2013 - Quarter 03,2013Q03   ,N,29,2013,2013-07-14,2013-07-20,N,Wk.29 - 13,2013-07-14 00:00:00,2013-07-20 00:00:00,2013W29   ,2013W29   ,07/14/13 - 07/20/13,07/14 - 07/20,2013,N,2012-07-19,2011-07-19,2013-04-19,2013-01-19,2013-06-19,2013-05-19,2013-07-12,2013-07-05,0,0,0,0,0,0,0,0,7,3,29,3,1,1,1,2005-09-07,DW_BATCH  ,2013-11-27 00:16:56,DW_BATCH
+2013-08-20,2013-01-01,2013-07-01,2013-08-01,2013-08-18,0,-1,-3,-14,-99,0,-1,-4,-14,-14,41505,232,51,20,3,34,5929,2,8,1364,3,455,114,2013-12-31,2013-09-30,2013-08-31,2013-08-24,20-Aug-2013,Aug 20th 2013,Tue 08-20-13,1,0,0,0,2013-06-21,365,92,31,7,2012-12-30,2013-06-30,2013-07-28,2013-08-18,2013-08-19,2013-08-20,Tue       ,2013M08,Aug-2013,N,2013M08   ,N,Year 2013 - Quarter 03,2013Q03   ,N,34,2013,2013-08-18,2013-08-24,N,Wk.34 - 13,2013-08-18 00:00:00,2013-08-24 00:00:00,2013W34   ,2013W34   ,08/18/13 - 08/24/13,08/18 - 08/24,2013,N,2012-08-20,2011-08-20,2013-05-20,2013-02-20,2013-07-20,2013-06-20,2013-08-13,2013-08-06,0,0,0,0,0,0,0,0,8,3,34,3,1,1,1,2005-09-07,DW_BATCH  ,2013-11-27 00:16:56,DW_BATCH
+2013-08-24,2013-01-01,2013-07-01,2013-08-01,2013-08-18,0,-1,-3,-14,-95,0,-1,-4,-14,-14,41509,236,55,24,7,34,5929,2,8,1364,3,455,114,2013-12-31,2013-09-30,2013-08-31,2013-08-24,24-Aug-2013,Aug 24th 2013,Sat 08-24-13,1,0,0,0,2013-06-21,365,92,31,7,2012-12-30,2013-06-30,2013-07-28,2013-08-18,2013-08-19,2013-08-24,Sat       ,2013M08,Aug-2013,N,2013M08   ,N,Year 2013 - Quarter 03,2013Q03   ,N,34,2013,2013-08-18,2013-08-24,N,Wk.34 - 13,2013-08-18 00:00:00,2013-08-24 00:00:00,2013W34   ,2013W34   ,08/18/13 - 08/24/13,08/18 - 08/24,2013,N,2012-08-24,2011-08-24,2013-05-24,2013-02-24,2013-07-24,2013-06-24,2013-08-17,2013-08-10,0,0,0,0,0,0,0,0,8,3,34,3,1,1,1,2005-09-07,DW_BATCH  ,2013-11-27 00:16:56,DW_BATCH
+2013-07-13,2013-01-01,2013-07-01,2013-07-01,2013-07-07,0,-1,-4,-20,-137,0,-1,-5,-20,-20,41467,194,13,13,7,28,5923,1,7,1363,3,455,114,2013-12-31,2013-09-30,2013-07-31,2013-07-13,13-Jul-2013,Jul 13th 2013,Sat 07-13-13,1,0,0,0,2013-06-21,365,92,31,7,2012-12-30,2013-06-30,2013-06-30,2013-07-07,2013-07-08,2013-07-13,Sat       ,2013M07,Jul-2013,N,2013M07   ,N,Year 2013 - Quarter 03,2013Q03   ,N,28,2013,2013-07-07,2013-07-13,N,Wk.28 - 13,2013-07-07 00:00:00,2013-07-13 00:00:00,2013W28   ,2013W28   ,07/07/13 - 07/13/13,07/07 - 07/13,2013,N,2012-07-13,2011-07-13,2013-04-13,2013-01-13,2013-06-13,2013-05-13,2013-07-06,2013-06-29,0,0,0,0,0,0,0,0,7,3,28,3,1,1,1,2005-09-07,DW_BATCH  ,2013-11-27 00:16:56,DW_BATCH
+2013-11-06,2013-01-01,2013-10-01,2013-11-01,2013-11-03,0,0,0,-3,-21,0,0,-1,-3,-3,41583,310,37,6,4,45,5940,2,11,1367,4,456,114,2013-12-31,2013-12-31,2013-11-30,2013-11-09,06-Nov-2013,Nov 6th 2013,Wed 11-06-13,1,1,1,0,2013-09-21,365,92,30,7,2012-12-30,2013-09-29,2013-10-27,2013-11-03,2013-11-04,2013-11-06,Wed       ,2013M11,Nov-2013,N,2013M11   ,N,Year 2013 - Quarter 04,2013Q04   ,N,45,2013,2013-11-03,2013-11-09,N,Wk.45 - 13,2013-11-03 00:00:00,2013-11-09 00:00:00,2013W45   ,2013W45   ,11/03/13 - 11/09/13,11/03 - 11/09,2013,N,2012-11-06,2011-11-06,2013-08-06,2013-05-06,2013-10-06,2013-09-06,2013-10-30,2013-10-23,0,0,0,0,0,0,0,0,11,4,45,4,1,1,1,2005-09-07,DW_BATCH  ,2013-11-27 00:16:56,DW_BATCH
+2013-12-07,2013-01-01,2013-10-01,2013-12-01,2013-12-01,0,0,1,1,10,0,0,0,1,1,41614,341,68,7,7,49,5944,3,12,1368,4,456,114,2013-12-31,2013-12-31,2013-12-31,2013-12-07,07-Dec-2013,Dec 7th 2013,Sat 12-07-13,0,0,0,0,2013-09-21,365,92,31,7,2012-12-30,2013-09-29,2013-11-24,2013-12-01,2013-12-02,2013-12-07,Sat       ,2013M12,Dec-2013,N,2013M12   ,N,Year 2013 - Quarter 04,2013Q04   ,N,49,2013,2013-12-01,2013-12-07,N,Wk.49 - 13,2013-12-01 00:00:00,2013-12-07 00:00:00,2013W49   ,2013W49   ,12/01/13 - 12/07/13,12/01 - 12/07,2013,N,2012-12-07,2011-12-07,2013-09-07,2013-06-07,2013-11-07,2013-10-07,2013-11-30,2013-11-23,0,0,0,0,0,0,0,0,12,4,49,4,0,1,0,2005-09-07,DW_BATCH  ,2013-11-27 00:16:56,DW_BATCH
+2013-07-04,2013-01-01,2013-07-01,2013-07-01,2013-06-30,0,-1,-4,-21,-146,0,-1,-5,-21,-21,41458,185,4,4,5,27,5922,1,7,1363,3,455,114,2013-12-31,2013-09-30,2013-07-31,2013-07-06,04-Jul-2013,Jul 4th 2013,Thu 07-04-13,1,0,0,0,2013-06-21,365,92,31,7,2012-12-30,2013-06-30,2013-06-30,2013-06-30,2013-07-01,2013-07-04,Thu       ,2013M07,Jul-2013,N,2013M07   ,N,Year 2013 - Quarter 03,2013Q03   ,N,27,2013,2013-06-30,2013-07-06,N,Wk.27 - 13,2013-06-30 00:00:00,2013-07-06 00:00:00,2013W27   ,2013W27   ,06/30/13 - 07/06/13,06/30 - 07/06,2013,N,2012-07-04,2011-07-04,2013-04-04,2013-01-04,2013-06-04,2013-05-04,2013-06-27,2013-06-20,0,0,0,0,0,0,0,0,7,3,27,3,1,1,1,2005-09-07,DW_BATCH  ,2013-11-27 00:16:56,DW_BATCH
+2013-10-07,2013-01-01,2013-10-01,2013-10-01,2013-10-06,0,0,-1,-7,-51,0,0,-2,-7,-7,41553,280,7,7,2,41,5936,1,10,1366,4,456,114,2013-12-31,2013-12-31,2013-10-31,2013-10-12,07-Oct-2013,Oct 7th 2013,Mon 10-07-13,1,1,0,0,2013-09-21,365,92,31,7,2012-12-30,2013-09-29,2013-09-29,2013-10-06,2013-10-07,2013-10-07,Mon       ,2013M10,Oct-2013,N,2013M10   ,N,Year 2013 - Quarter 04,2013Q04   ,N,41,2013,2013-10-06,2013-10-12,N,Wk.41 - 13,2013-10-06 00:00:00,2013-10-12 00:00:00,2013W41   ,2013W41   ,10/06/13 - 10/12/13,10/06 - 10/12,2013,N,2012-10-07,2011-10-07,2013-07-07,2013-04-07,2013-09-07,2013-08-07,2013-09-30,2013-09-23,0,0,0,0,0,0,0,0,10,4,41,4,1,1,1,2005-09-07,DW_BATCH  ,2013-11-27 00:16:56,DW_BATCH
+2013-11-10,2013-01-01,2013-10-01,2013-11-01,2013-11-10,0,0,0,-2,-17,0,0,-1,-2,-3,41587,314,41,10,1,46,5941,2,11,1367,4,456,114,2013-12-31,2013-12-31,2013-11-30,2013-11-16,10-Nov-2013,Nov 10th 2013,Sun 11-10-13,1,1,1,0,2013-09-21,365,92,30,7,2012-12-30,2013-09-29,2013-10-27,2013-11-10,2013-11-04,2013-11-10,Sun       ,2013M11,Nov-2013,N,2013M11   ,N,Year 2013 - Quarter 04,2013Q04   ,N,46,2013,2013-11-10,2013-11-16,Y,Wk.46 - 13,2013-11-10 00:00:00,2013-11-16 00:00:00,2013W46   ,2013W46   ,11/10/13 - 11/16/13,11/10 - 11/16,2013,N,2012-11-10,2011-11-10,2013-08-10,2013-05-10,2013-10-10,2013-09-10,2013-11-03,2013-10-27,0,0,0,0,0,0,0,0,11,4,46,4,1,1,1,2005-09-07,DW_BATCH  ,2013-11-27 00:16:56,DW_BATCH
+2013-08-07,2013-01-01,2013-07-01,2013-08-01,2013-08-04,0,-1,-3,-16,-112,0,-1,-4,-16,-16,41492,219,38,7,4,32,5927,2,8,1364,3,455,114,2013-12-31,2013-09-30,2013-08-31,2013-08-10,07-Aug-2013,Aug 7th 2013,Wed 08-07-13,1,0,0,0,2013-06-21,365,92,31,7,2012-12-30,2013-06-30,2013-07-28,2013-08-04,2013-08-05,2013-08-07,Wed       ,2013M08,Aug-2013,N,2013M08   ,N,Year 2013 - Quarter 03,2013Q03   ,N,32,2013,2013-08-04,2013-08-10,N,Wk.32 - 13,2013-08-04 00:00:00,2013-08-10 00:00:00,2013W32   ,2013W32   ,08/04/13 - 08/10/13,08/04 - 08/10,2013,N,2012-08-07,2011-08-07,2013-05-07,2013-02-07,2013-07-07,2013-06-07,2013-07-31,2013-07-24,0,0,0,0,0,0,0,0,8,3,32,3,1,1,1,2005-09-07,DW_BATCH  ,2013-11-27 00:16:56,DW_BATCH
+2013-09-25,2013-01-01,2013-07-01,2013-09-01,2013-09-22,0,-1,-2,-9,-63,0,-1,-3,-9,-9,41541,268,87,25,4,39,5934,3,9,1365,3,455,114,2013-12-31,2013-09-30,2013-09-30,2013-09-28,25-Sep-2013,Sep 25th 2013,Wed 09-25-13,1,0,0,0,2013-09-21,365,92,30,7,2012-12-30,2013-06-30,2013-08-25,2013-09-22,2013-09-23,2013-09-25,Wed       ,2013M09,Sep-2013,N,2013M09   ,N,Year 2013 - Quarter 03,2013Q03   ,N,39,2013,2013-09-22,2013-09-28,N,Wk.39 - 13,2013-09-22 00:00:00,2013-09-28 00:00:00,2013W39   ,2013W39   ,09/22/13 - 09/28/13,09/22 - 09/28,2013,N,2012-09-25,2011-09-25,2013-06-25,2013-03-25,2013-08-25,2013-07-25,2013-09-18,2013-09-11,0,0,0,0,0,0,0,0,9,3,39,4,1,1,1,2005-09-07,DW_BATCH  ,2013-11-27 00:16:56,DW_BATCH
+2013-10-05,2013-01-01,2013-10-01,2013-10-01,2013-09-29,0,0,-1,-8,-53,0,0,-2,-8,-8,41551,278,5,5,7,40,5935,1,10,1366,4,456,114,2013-12-31,2013-12-31,2013-10-31,2013-10-05,05-Oct-2013,Oct 5th 2013,Sat 10-05-13,1,1,0,0,2013-09-21,365,92,31,7,2012-12-30,2013-09-29,2013-09-29,2013-09-29,2013-09-30,2013-10-05,Sat       ,2013M10,Oct-2013,N,2013M10   ,N,Year 2013 - Quarter 04,2013Q04   ,N,40,2013,2013-09-29,2013-10-05,N,Wk.40 - 13,2013-09-29 00:00:00,2013-10-05 00:00:00,2013W40   ,2013W40   ,09/29/13 - 10/05/13,09/29 - 10/05,2013,N,2012-10-05,2011-10-05,2013-07-05,2013-04-05,2013-09-05,2013-08-05,2013-09-28,2013-09-21,0,0,0,0,0,0,0,0,10,4,40,4,1,1,1,2005-09-07,DW_BATCH  ,2013-11-27 00:16:56,DW_BATCH
+2013-09-02,2013-01-01,2013-07-01,2013-09-01,2013-09-01,0,-1,-2,-12,-86,0,-1,-3,-12,-12,41518,245,64,2,2,36,5931,3,9,1365,3,455,114,2013-12-31,2013-09-30,2013-09-30,2013-09-07,02-Sep-2013,Sep 2nd 2013,Mon 09-02-13,1,0,0,0,2013-06-21,365,92,30,7,2012-12-30,2013-06-30,2013-08-25,2013-09-01,2013-09-02,2013-09-02,Mon       ,2013M09,Sep-2013,N,2013M09   ,N,Year 2013 - Quarter 03,2013Q03   ,N,36,2013,2013-09-01,2013-09-07,N,Wk.36 - 13,2013-09-01 00:00:00,2013-09-07 00:00:00,2013W36   ,2013W36   ,09/01/13 - 09/07/13,09/01 - 09/07,2013,N,2012-09-02,2011-09-02,2013-06-02,2013-03-02,2013-08-02,2013-07-02,2013-08-26,2013-08-19,0,0,0,0,0,0,0,0,9,3,36,3,1,1,1,2005-09-07,DW_BATCH  ,2013-11-27 00:16:56,DW_BATCH
+2013-05-15,2013-01-01,2013-04-01,2013-05-01,2013-05-12,0,-2,-6,-28,-196,0,-2,-7,-28,-28,41408,135,45,15,4,20,5915,2,5,1361,2,454,114,2013-12-31,2013-06-30,2013-05-31,2013-05-18,15-May-2013,May 15th 2013,Wed 05-15-13,1,0,0,0,2013-03-21,365,91,31,7,2012-12-30,2013-03-31,2013-04-28,2013-05-12,2013-05-13,2013-05-15,Wed       ,2013M05,May-2013,N,2013M05   ,N,Year 2013 - Quarter 02,2013Q02   ,N,20,2013,2013-05-12,2013-05-18,N,Wk.20 - 13,2013-05-12 00:00:00,2013-05-18 00:00:00,2013W20   ,2013W20   ,05/12/13 - 05/18/13,05/12 - 05/18,2013,N,2012-05-15,2011-05-15,2013-02-15,2012-11-15,2013-04-15,2013-03-15,2013-05-08,2013-05-01,0,0,0,0,0,0,0,0,5,2,20,2,1,1,1,2005-09-07,DW_BATCH  ,2013-11-27 00:16:56,DW_BATCH
+2013-02-10,2013-01-01,2013-01-01,2013-02-01,2013-02-10,0,-3,-9,-41,-290,0,-3,-10,-41,-42,41314,41,41,10,1,7,5902,2,2,1358,1,453,114,2013-12-31,2013-03-31,2013-02-28,2013-02-16,10-Feb-2013,Feb 10th 2013,Sun 02-10-13,1,0,0,0,2012-12-21,365,90,28,7,2012-12-30,2012-12-30,2013-01-27,2013-02-10,2013-02-04,2013-02-10,Sun       ,2013M02,Feb-2013,N,2013M02   ,N,Year 2013 - Quarter 01,2013Q01   ,N,7,2013,2013-02-10,2013-02-16,Y,Wk.07 - 13,2013-02-10 00:00:00,2013-02-16 00:00:00,2013W07   ,2013W07   ,02/10/13 - 02/16/13,02/10 - 02/16,2013,N,2012-02-10,2011-02-10,2012-11-10,2012-08-10,2013-01-10,2012-12-10,2013-02-03,2013-01-27,0,0,0,0,0,0,0,0,2,1,7,1,1,1,1,2005-09-07,DW_BATCH  ,2013-11-27 00:16:56,DW_BATCH
+2013-12-21,2013-01-01,2013-10-01,2013-12-01,2013-12-15,0,0,1,3,24,0,0,0,3,3,41628,355,82,21,7,51,5946,3,12,1368,4,456,114,2013-12-31,2013-12-31,2013-12-31,2013-12-21,21-Dec-2013,Dec 21st 2013,Sat 12-21-13,0,0,0,0,2013-12-21,365,92,31,7,2012-12-30,2013-09-29,2013-11-24,2013-12-15,2013-12-16,2013-12-21,Sat       ,2013M12,Dec-2013,N,2013M12   ,N,Year 2013 - Quarter 04,2013Q04   ,N,51,2013,2013-12-15,2013-12-21,N,Wk.51 - 13,2013-12-15 00:00:00,2013-12-21 00:00:00,2013W51   ,2013W51   ,12/15/13 - 12/21/13,12/15 - 12/21,2013,N,2012-12-21,2011-12-21,2013-09-21,2013-06-21,2013-11-21,2013-10-21,2013-12-14,2013-12-07,0,0,0,0,0,0,0,0,12,4,51,1,0,1,0,2005-09-07,DW_BATCH  ,2013-11-27 00:16:56,DW_BATCH
+2013-10-22,2013-01-01,2013-10-01,2013-10-01,2013-10-20,0,0,-1,-5,-36,0,0,-2,-5,-5,41568,295,22,22,3,43,5938,1,10,1366,4,456,114,2013-12-31,2013-12-31,2013-10-31,2013-10-26,22-Oct-2013,Oct 22nd 2013,Tue 10-22-13,1,1,0,0,2013-09-21,365,92,31,7,2012-12-30,2013-09-29,2013-09-29,2013-10-20,2013-10-21,2013-10-22,Tue       ,2013M10,Oct-2013,N,2013M10   ,N,Year 2013 - Quarter 04,2013Q04   ,N,43,2013,2013-10-20,2013-10-26,N,Wk.43 - 13,2013-10-20 00:00:00,2013-10-26 00:00:00,2013W43   ,2013W43   ,10/20/13 - 10/26/13,10/20 - 10/26,2013,N,2012-10-22,2011-10-22,2013-07-22,2013-04-22,2013-09-22,2013-08-22,2013-10-15,2013-10-08,0,0,0,0,0,0,0,0,10,4,43,4,1,1,1,2005-09-07,DW_BATCH  ,2013-11-27 00:16:56,DW_BATCH
+2013-10-24,2013-01-01,2013-10-01,2013-10-01,2013-10-20,0,0,-1,-5,-34,0,0,-2,-5,-5,41570,297,24,24,5,43,5938,1,10,1366,4,456,114,2013-12-31,2013-12-31,2013-10-31,2013-10-26,24-Oct-2013,Oct 24th 2013,Thu 10-24-13,1,1,0,0,2013-09-21,365,92,31,7,2012-12-30,2013-09-29,2013-09-29,2013-10-20,2013-10-21,2013-10-24,Thu       ,2013M10,Oct-2013,N,2013M10   ,N,Year 2013 - Quarter 04,2013Q04   ,N,43,2013,2013-10-20,2013-10-26,N,Wk.43 - 13,2013-10-20 00:00:00,2013-10-26 00:00:00,2013W43   ,2013W43   ,10/20/13 - 10/26/13,10/20 - 10/26,2013,N,2012-10-24,2011-10-24,2013-07-24,2013-04-24,2013-09-24,2013-08-24,2013-10-17,2013-10-10,0,0,0,0,0,0,0,0,10,4,43,4,1,1,1,2005-09-07,DW_BATCH  ,2013-11-27 00:16:56,DW_BATCH
+2013-07-10,2013-01-01,2013-07-01,2013-07-01,2013-07-07,0,-1,-4,-20,-140,0,-1,-5,-20,-20,41464,191,10,10,4,28,5923,1,7,1363,3,455,114,2013-12-31,2013-09-30,2013-07-31,2013-07-13,10-Jul-2013,Jul 10th 2013,Wed 07-10-13,1,0,0,0,2013-06-21,365,92,31,7,2012-12-30,2013-06-30,2013-06-30,2013-07-07,2013-07-08,2013-07-10,Wed       ,2013M07,Jul-2013,N,2013M07   ,N,Year 2013 - Quarter 03,2013Q03   ,N,28,2013,2013-07-07,2013-07-13,N,Wk.28 - 13,2013-07-07 00:00:00,2013-07-13 00:00:00,2013W28   ,2013W28   ,07/07/13 - 07/13/13,07/07 - 07/13,2013,N,2012-07-10,2011-07-10,2013-04-10,2013-01-10,2013-06-10,2013-05-10,2013-07-03,2013-06-26,0,0,0,0,0,0,0,0,7,3,28,3,1,1,1,2005-09-07,DW_BATCH  ,2013-11-27 00:16:56,DW_BATCH
+2013-02-27,2013-01-01,2013-01-01,2013-02-01,2013-02-24,0,-3,-9,-39,-273,0,-3,-9,-39,-39,41331,58,58,27,4,9,5904,2,2,1358,1,453,114,2013-12-31,2013-03-31,2013-02-28,2013-03-02,27-Feb-2013,Feb 27th 2013,Wed 02-27-13,1,0,0,0,2012-12-21,365,90,28,7,2012-12-30,2012-12-30,2013-02-24,2013-02-24,2013-02-25,2013-02-27,Wed       ,2013M02,Feb-2013,N,2013M02   ,N,Year 2013 - Quarter 01,2013Q01   ,N,9,2013,2013-02-24,2013-03-02,N,Wk.09 - 13,2013-02-24 00:00:00,2013-03-02 00:00:00,2013W09   ,2013W09   ,02/24/13 - 03/02/13,02/24 - 03/02,2013,N,2012-02-27,2011-02-27,2012-11-27,2012-08-27,2013-01-27,2012-12-27,2013-02-20,2013-02-13,0,0,0,0,0,0,0,0,3,1,9,1,1,1,1,2005-09-07,DW_BATCH  ,2013-11-27 00:16:56,DW_BATCH
+2013-03-02,2013-01-01,2013-01-01,2013-03-01,2013-02-24,0,-3,-8,-39,-270,0,-3,-9,-39,-39,41334,61,61,2,7,9,5904,3,3,1359,1,453,114,2013-12-31,2013-03-31,2013-03-31,2013-03-02,02-Mar-2013,Mar 2nd 2013,Sat 03-02-13,1,0,0,0,2012-12-21,365,90,31,7,2012-12-30,2012-12-30,2013-02-24,2013-02-24,2013-02-25,2013-03-02,Sat       ,2013M03,Mar-2013,N,2013M03   ,N,Year 2013 - Quarter 01,2013Q01   ,N,9,2013,2013-02-24,2013-03-02,N,Wk.09 - 13,2013-02-24 00:00:00,2013-03-02 00:00:00,2013W09   ,2013W09   ,02/24/13 - 03/02/13,02/24 - 03/02,2013,N,2012-03-02,2011-03-02,2012-12-02,2012-09-02,2013-02-02,2013-01-02,2013-02-23,2013-02-16,0,0,0,0,0,0,0,0,3,1,9,1,1,1,1,2005-09-07,DW_BATCH  ,2013-11-27 00:16:56,DW_BATCH
+2013-11-02,2013-01-01,2013-10-01,2013-11-01,2013-10-27,0,0,0,-4,-25,0,0,-1,-4,-4,41579,306,33,2,7,44,5939,2,11,1367,4,456,114,2013-12-31,2013-12-31,2013-11-30,2013-11-02,02-Nov-2013,Nov 2nd 2013,Sat 11-02-13,1,1,1,0,2013-09-21,365,92,30,7,2012-12-30,2013-09-29,2013-10-27,2013-10-27,2013-10-28,2013-11-02,Sat       ,2013M11,Nov-2013,N,2013M11   ,N,Year 2013 - Quarter 04,2013Q04   ,N,44,2013,2013-10-27,2013-11-02,N,Wk.44 - 13,2013-10-27 00:00:00,2013-11-02 00:00:00,2013W44   ,2013W44   ,10/27/13 - 11/02/13,10/27 - 11/02,2013,N,2012-11-02,2011-11-02,2013-08-02,2013-05-02,2013-10-02,2013-09-02,2013-10-26,2013-10-19,0,0,0,0,0,0,0,0,11,4,44,4,1,1,1,2005-09-07,DW_BATCH  ,2013-11-27 00:16:56,DW_BATCH
+2013-03-21,2013-01-01,2013-01-01,2013-03-01,2013-03-17,0,-3,-8,-36,-251,0,-3,-9,-36,-36,41353,80,80,21,5,12,5907,3,3,1359,1,453,114,2013-12-31,2013-03-31,2013-03-31,2013-03-23,21-Mar-2013,Mar 21st 2013,Thu 03-21-13,1,0,0,0,2013-03-21,365,90,31,7,2012-12-30,2012-12-30,2013-02-24,2013-03-17,2013-03-18,2013-03-21,Thu       ,2013M03,Mar-2013,N,2013M03   ,N,Year 2013 - Quarter 01,2013Q01   ,N,12,2013,2013-03-17,2013-03-23,N,Wk.12 - 13,2013-03-17 00:00:00,2013-03-23 00:00:00,2013W12   ,2013W12   ,03/17/13 - 03/23/13,03/17 - 03/23,2013,N,2012-03-21,2011-03-21,2012-12-21,2012-09-21,2013-02-21,2013-01-21,2013-03-14,2013-03-07,0,0,0,0,0,0,0,0,3,1,12,2,1,1,1,2005-09-07,DW_BATCH  ,2013-11-27 00:16:56,DW_BATCH
+2013-04-03,2013-01-01,2013-04-01,2013-04-01,2013-03-31,0,-2,-7,-34,-238,0,-2,-8,-34,-34,41366,93,3,3,4,14,5909,1,4,1360,2,454,114,2013-12-31,2013-06-30,2013-04-30,2013-04-06,03-Apr-2013,Apr 3rd 2013,Wed 04-03-13,1,0,0,0,2013-03-21,365,91,30,7,2012-12-30,2013-03-31,2013-03-31,2013-03-31,2013-04-01,2013-04-03,Wed       ,2013M04,Apr-2013,N,2013M04   ,N,Year 2013 - Quarter 02,2013Q02   ,N,14,2013,2013-03-31,2013-04-06,N,Wk.14 - 13,2013-03-31 00:00:00,2013-04-06 00:00:00,2013W14   ,2013W14   ,03/31/13 - 04/06/13,03/31 - 04/06,2013,N,2012-04-03,2011-04-03,2013-01-03,2012-10-03,2013-03-03,2013-02-03,2013-03-27,2013-03-20,0,0,0,0,0,0,0,0,4,2,14,2,1,1,1,2005-09-07,DW_BATCH  ,2013-11-27 00:16:56,DW_BATCH
+2013-07-06,2013-01-01,2013-07-01,2013-07-01,2013-06-30,0,-1,-4,-21,-144,0,-1,-5,-21,-21,41460,187,6,6,7,27,5922,1,7,1363,3,455,114,2013-12-31,2013-09-30,2013-07-31,2013-07-06,06-Jul-2013,Jul 6th 2013,Sat 07-06-13,1,0,0,0,2013-06-21,365,92,31,7,2012-12-30,2013-06-30,2013-06-30,2013-06-30,2013-07-01,2013-07-06,Sat       ,2013M07,Jul-2013,N,2013M07   ,N,Year 2013 - Quarter 03,2013Q03   ,N,27,2013,2013-06-30,2013-07-06,N,Wk.27 - 13,2013-06-30 00:00:00,2013-07-06 00:00:00,2013W27   ,2013W27   ,06/30/13 - 07/06/13,06/30 - 07/06,2013,N,2012-07-06,2011-07-06,2013-04-06,2013-01-06,2013-06-06,2013-05-06,2013-06-29,2013-06-22,0,0,0,0,0,0,0,0,7,3,27,3,1,1,1,2005-09-07,DW_BATCH  ,2013-11-27 00:16:56,DW_BATCH
+2013-06-26,2013-01-01,2013-04-01,2013-06-01,2013-06-23,0,-2,-5,-22,-154,0,-2,-6,-22,-22,41450,177,87,26,4,26,5921,3,6,1362,2,454,114,2013-12-31,2013-06-30,2013-06-30,2013-06-29,26-Jun-2013,Jun 26th 2013,Wed 06-26-13,1,0,0,0,2013-06-21,365,91,30,7,2012-12-30,2013-03-31,2013-05-26,2013-06-23,2013-06-24,2013-06-26,Wed       ,2013M06,Jun-2013,N,2013M06   ,N,Year 2013 - Quarter 02,2013Q02   ,N,26,2013,2013-06-23,2013-06-29,N,Wk.26 - 13,2013-06-23 00:00:00,2013-06-29 00:00:00,2013W26   ,2013W26   ,06/23/13 - 06/29/13,06/23 - 06/29,2013,N,2012-06-26,2011-06-26,2013-03-26,2012-12-26,2013-05-26,2013-04-26,2013-06-19,2013-06-12,0,0,0,0,0,0,0,0,6,2,26,3,1,1,1,2005-09-07,DW_BATCH  ,2013-11-27 00:16:56,DW_BATCH
+2013-08-26,2013-01-01,2013-07-01,2013-08-01,2013-08-25,0,-1,-3,-13,-93,0,-1,-3,-13,-13,41511,238,57,26,2,35,5930,2,8,1364,3,455,114,2013-12-31,2013-09-30,2013-08-31,2013-08-31,26-Aug-2013,Aug 26th 2013,Mon 08-26-13,1,0,0,0,2013-06-21,365,92,31,7,2012-12-30,2013-06-30,2013-08-25,2013-08-25,2013-08-26,2013-08-26,Mon       ,2013M08,Aug-2013,N,2013M08   ,N,Year 2013 - Quarter 03,2013Q03   ,N,35,2013,2013-08-25,2013-08-31,N,Wk.35 - 13,2013-08-25 00:00:00,2013-08-31 00:00:00,2013W35   ,2013W35   ,08/25/13 - 08/31/13,08/25 - 08/31,2013,N,2012-08-26,2011-08-26,2013-05-26,2013-02-26,2013-07-26,2013-06-26,2013-08-19,2013-08-12,0,0,0,0,0,0,0,0,9,3,35,3,1,1,1,2005-09-07,DW_BATCH  ,2013-11-27 00:16:56,DW_BATCH
+2013-12-13,2013-01-01,2013-10-01,2013-12-01,2013-12-08,0,0,1,2,16,0,0,0,2,2,41620,347,74,13,6,50,5945,3,12,1368,4,456,114,2013-12-31,2013-12-31,2013-12-31,2013-12-14,13-Dec-2013,Dec 13th 2013,Fri 12-13-13,0,0,0,0,2013-09-21,365,92,31,7,2012-12-30,2013-09-29,2013-11-24,2013-12-08,2013-12-09,2013-12-13,Fri       ,2013M12,Dec-2013,N,2013M12   ,N,Year 2013 - Quarter 04,2013Q04   ,N,50,2013,2013-12-08,2013-12-14,N,Wk.50 - 13,2013-12-08 00:00:00,2013-12-14 00:00:00,2013W50   ,2013W50   ,12/08/13 - 12/14/13,12/08 - 12/14,2013,N,2012-12-13,2011-12-13,2013-09-13,2013-06-13,2013-11-13,2013-10-13,2013-12-06,2013-11-29,0,0,0,0,0,0,0,0,12,4,50,4,0,1,0,2005-09-07,DW_BATCH  ,2013-11-27 00:16:56,DW_BATCH
+2013-07-02,2013-01-01,2013-07-01,2013-07-01,2013-06-30,0,-1,-4,-21,-148,0,-1,-5,-21,-21,41456,183,2,2,3,27,5922,1,7,1363,3,455,114,2013-12-31,2013-09-30,2013-07-31,2013-07-06,02-Jul-2013,Jul 2nd 2013,Tue 07-02-13,1,0,0,0,2013-06-21,365,92,31,7,2012-12-30,2013-06-30,2013-06-30,2013-06-30,2013-07-01,2013-07-02,Tue       ,2013M07,Jul-2013,N,2013M07   ,N,Year 2013 - Quarter 03,2013Q03   ,N,27,2013,2013-06-30,2013-07-06,N,Wk.27 - 13,2013-06-30 00:00:00,2013-07-06 00:00:00,2013W27   ,2013W27   ,06/30/13 - 07/06/13,06/30 - 07/06,2013,N,2012-07-02,2011-07-02,2013-04-02,2013-01-02,2013-06-02,2013-05-02,2013-06-25,2013-06-18,0,0,0,0,0,0,0,0,7,3,27,3,1,1,1,2005-09-07,DW_BATCH  ,2013-11-27 00:16:56,DW_BATCH
+2013-08-05,2013-01-01,2013-07-01,2013-08-01,2013-08-04,0,-1,-3,-16,-114,0,-1,-4,-16,-16,41490,217,36,5,2,32,5927,2,8,1364,3,455,114,2013-12-31,2013-09-30,2013-08-31,2013-08-10,05-Aug-2013,Aug 5th 2013,Mon 08-05-13,1,0,0,0,2013-06-21,365,92,31,7,2012-12-30,2013-06-30,2013-07-28,2013-08-04,2013-08-05,2013-08-05,Mon       ,2013M08,Aug-2013,N,2013M08   ,N,Year 2013 - Quarter 03,2013Q03   ,N,32,2013,2013-08-04,2013-08-10,N,Wk.32 - 13,2013-08-04 00:00:00,2013-08-10 00:00:00,2013W32   ,2013W32   ,08/04/13 - 08/10/13,08/04 - 08/10,2013,N,2012-08-05,2011-08-05,2013-05-05,2013-02-05,2013-07-05,2013-06-05,2013-07-29,2013-07-22,0,0,0,0,0,0,0,0,8,3,32,3,1,1,1,2005-09-07,DW_BATCH  ,2013-11-27 00:16:56,DW_BATCH
+2013-02-08,2013-01-01,2013-01-01,2013-02-01,2013-02-03,0,-3,-9,-42,-292,0,-3,-10,-42,-42,41312,39,39,8,6,6,5901,2,2,1358,1,453,114,2013-12-31,2013-03-31,2013-02-28,2013-02-09,08-Feb-2013,Feb 8th 2013,Fri 02-08-13,1,0,0,0,2012-12-21,365,90,28,7,2012-12-30,2012-12-30,2013-01-27,2013-02-03,2013-02-04,2013-02-08,Fri       ,2013M02,Feb-2013,N,2013M02   ,N,Year 2013 - Quarter 01,2013Q01   ,N,6,2013,2013-02-03,2013-02-09,N,Wk.06 - 13,2013-02-03 00:00:00,2013-02-09 00:00:00,2013W06   ,2013W06   ,02/03/13 - 02/09/13,02/03 - 02/09,2013,N,2012-02-08,2011-02-08,2012-11-08,2012-08-08,2013-01-08,2012-12-08,2013-02-01,2013-01-25,0,0,0,0,0,0,0,0,2,1,6,1,1,1,1,2005-09-07,DW_BATCH  ,2013-11-27 00:16:56,DW_BATCH
+2013-02-14,2013-01-01,2013-01-01,2013-02-01,2013-02-10,0,-3,-9,-41,-286,0,-3,-10,-41,-41,41318,45,45,14,5,7,5902,2,2,1358,1,453,114,2013-12-31,2013-03-31,2013-02-28,2013-02-16,14-Feb-2013,Feb 14th 2013,Thu 02-14-13,1,0,0,0,2012-12-21,365,90,28,7,2012-12-30,2012-12-30,2013-01-27,2013-02-10,2013-02-11,2013-02-14,Thu       ,2013M02,Feb-2013,N,2013M02   ,N,Year 2013 - Quarter 01,2013Q01   ,N,7,2013,2013-02-10,2013-02-16,N,Wk.07 - 13,2013-02-10 00:00:00,2013-02-16 00:00:00,2013W07   ,2013W07   ,02/10/13 - 02/16/13,02/10 - 02/16,2013,N,2012-02-14,2011-02-14,2012-11-14,2012-08-14,2013-01-14,2012-12-14,2013-02-07,2013-01-31,0,0,0,0,0,0,0,0,2,1,7,1,1,1,1,2005-09-07,DW_BATCH  ,2013-11-27 00:16:56,DW_BATCH
+2013-09-21,2013-01-01,2013-07-01,2013-09-01,2013-09-15,0,-1,-2,-10,-67,0,-1,-3,-10,-10,41537,264,83,21,7,38,5933,3,9,1365,3,455,114,2013-12-31,2013-09-30,2013-09-30,2013-09-21,21-Sep-2013,Sep 21st 2013,Sat 09-21-13,1,0,0,0,2013-09-21,365,92,30,7,2012-12-30,2013-06-30,2013-08-25,2013-09-15,2013-09-16,2013-09-21,Sat       ,2013M09,Sep-2013,N,2013M09   ,N,Year 2013 - Quarter 03,2013Q03   ,N,38,2013,2013-09-15,2013-09-21,N,Wk.38 - 13,2013-09-15 00:00:00,2013-09-21 00:00:00,2013W38   ,2013W38   ,09/15/13 - 09/21/13,09/15 - 09/21,2013,N,2012-09-21,2011-09-21,2013-06-21,2013-03-21,2013-08-21,2013-07-21,2013-09-14,2013-09-07,0,0,0,0,0,0,0,0,9,3,38,4,1,1,1,2005-09-07,DW_BATCH  ,2013-11-27 00:16:56,DW_BATCH
+2013-03-19,2013-01-01,2013-01-01,2013-03-01,2013-03-17,0,-3,-8,-36,-253,0,-3,-9,-36,-36,41351,78,78,19,3,12,5907,3,3,1359,1,453,114,2013-12-31,2013-03-31,2013-03-31,2013-03-23,19-Mar-2013,Mar 19th 2013,Tue 03-19-13,1,0,0,0,2012-12-21,365,90,31,7,2012-12-30,2012-12-30,2013-02-24,2013-03-17,2013-03-18,2013-03-19,Tue       ,2013M03,Mar-2013,N,2013M03   ,N,Year 2013 - Quarter 01,2013Q01   ,N,12,2013,2013-03-17,2013-03-23,N,Wk.12 - 13,2013-03-17 00:00:00,2013-03-23 00:00:00,2013W12   ,2013W12   ,03/17/13 - 03/23/13,03/17 - 03/23,2013,N,2012-03-19,2011-03-19,2012-12-19,2012-09-19,2013-02-19,2013-01-19,2013-03-12,2013-03-05,0,0,0,0,0,0,0,0,3,1,12,1,1,1,1,2005-09-07,DW_BATCH  ,2013-11-27 00:16:56,DW_BATCH
+2013-01-28,2013-01-01,2013-01-01,2013-01-01,2013-01-27,0,-3,-10,-43,-303,0,-3,-10,-43,-43,41301,28,28,28,2,5,5900,1,1,1357,1,453,114,2013-12-31,2013-03-31,2013-01-31,2013-02-02,28-Jan-2013,Jan 28th 2013,Mon 01-28-13,1,0,0,0,2012-12-21,365,90,31,7,2012-12-30,2012-12-30,2013-01-27,2013-01-27,2013-01-28,2013-01-28,Mon       ,2013M01,Jan-2013,N,2013M01   ,N,Year 2013 - Quarter 01,2013Q01   ,N,5,2013,2013-01-27,2013-02-02,N,Wk.05 - 13,2013-01-27 00:00:00,2013-02-02 00:00:00,2013W05   ,2013W05   ,01/27/13 - 02/02/13,01/27 - 02/02,2013,N,2012-01-28,2011-01-28,2012-10-28,2012-07-28,2012-12-28,2012-11-28,2013-01-21,2013-01-14,0,0,0,0,0,0,0,0,2,1,5,1,1,1,1,2005-09-07,DW_BATCH  ,2013-11-27 00:16:56,DW_BATCH
+2013-08-03,2013-01-01,2013-07-01,2013-08-01,2013-07-28,0,-1,-3,-17,-116,0,-1,-4,-17,-17,41488,215,34,3,7,31,5926,2,8,1364,3,455,114,2013-12-31,2013-09-30,2013-08-31,2013-08-03,03-Aug-2013,Aug 3rd 2013,Sat 08-03-13,1,0,0,0,2013-06-21,365,92,31,7,2012-12-30,2013-06-30,2013-07-28,2013-07-28,2013-07-29,2013-08-03,Sat       ,2013M08,Aug-2013,N,2013M08   ,N,Year 2013 - Quarter 03,2013Q03   ,N,31,2013,2013-07-28,2013-08-03,N,Wk.31 - 13,2013-07-28 00:00:00,2013-08-03 00:00:00,2013W31   ,2013W31   ,07/28/13 - 08/03/13,07/28 - 08/03,2013,N,2012-08-03,2011-08-03,2013-05-03,2013-02-03,2013-07-03,2013-06-03,2013-07-27,2013-07-20,0,0,0,0,0,0,0,0,8,3,31,3,1,1,1,2005-09-07,DW_BATCH  ,2013-11-27 00:16:56,DW_BATCH
+2013-06-14,2013-01-01,2013-04-01,2013-06-01,2013-06-09,0,-2,-5,-24,-166,0,-2,-6,-24,-24,41438,165,75,14,6,24,5919,3,6,1362,2,454,114,2013-12-31,2013-06-30,2013-06-30,2013-06-15,14-Jun-2013,Jun 14th 2013,Fri 06-14-13,1,0,0,0,2013-03-21,365,91,30,7,2012-12-30,2013-03-31,2013-05-26,2013-06-09,2013-06-10,2013-06-14,Fri       ,2013M06,Jun-2013,N,2013M06   ,N,Year 2013 - Quarter 02,2013Q02   ,N,24,2013,2013-06-09,2013-06-15,N,Wk.24 - 13,2013-06-09 00:00:00,2013-06-15 00:00:00,2013W24   ,2013W24   ,06/09/13 - 06/15/13,06/09 - 06/15,2013,N,2012-06-14,2011-06-14,2013-03-14,2012-12-14,2013-05-14,2013-04-14,2013-06-07,2013-05-31,0,0,0,0,0,0,0,0,6,2,24,2,1,1,1,2005-09-07,DW_BATCH  ,2013-11-27 00:16:56,DW_BATCH
+2013-07-08,2013-01-01,2013-07-01,2013-07-01,2013-07-07,0,-1,-4,-20,-142,0,-1,-5,-20,-20,41462,189,8,8,2,28,5923,1,7,1363,3,455,114,2013-12-31,2013-09-30,2013-07-31,2013-07-13,08-Jul-2013,Jul 8th 2013,Mon 07-08-13,1,0,0,0,2013-06-21,365,92,31,7,2012-12-30,2013-06-30,2013-06-30,2013-07-07,2013-07-08,2013-07-08,Mon       ,2013M07,Jul-2013,N,2013M07   ,N,Year 2013 - Quarter 03,2013Q03   ,N,28,2013,2013-07-07,2013-07-13,N,Wk.28 - 13,2013-07-07 00:00:00,2013-07-13 00:00:00,2013W28   ,2013W28   ,07/07/13 - 07/13/13,07/07 - 07/13,2013,N,2012-07-08,2011-07-08,2013-04-08,2013-01-08,2013-06-08,2013-05-08,2013-07-01,2013-06-24,0,0,0,0,0,0,0,0,7,3,28,3,1,1,1,2005-09-07,DW_BATCH  ,2013-11-27 00:16:56,DW_BATCH
+2013-11-30,2013-01-01,2013-10-01,2013-11-01,2013-11-24,0,0,0,0,3,0,0,0,0,0,41607,334,61,30,7,48,5943,2,11,1367,4,456,114,2013-12-31,2013-12-31,2013-11-30,2013-11-30,30-Nov-2013,Nov 30th 2013,Sat 11-30-13,0,0,0,0,2013-09-21,365,92,30,7,2012-12-30,2013-09-29,2013-11-24,2013-11-24,2013-11-25,2013-11-30,Sat       ,2013M11,Nov-2013,N,2013M11   ,N,Year 2013 - Quarter 04,2013Q04   ,N,48,2013,2013-11-24,2013-11-30,N,Wk.48 - 13,2013-11-24 00:00:00,2013-11-30 00:00:00,2013W48   ,2013W48   ,11/24/13 - 11/30/13,11/24 - 11/30,2013,N,2012-11-30,2011-11-30,2013-08-30,2013-05-30,2013-10-30,2013-09-30,2013-11-23,2013-11-16,0,0,0,0,0,0,0,0,12,4,48,4,1,1,1,2005-09-07,DW_BATCH  ,2013-11-27 00:16:56,DW_BATCH
+2013-01-10,2013-01-01,2013-01-01,2013-01-01,2013-01-06,0,-3,-10,-46,-321,0,-3,-11,-46,-46,41283,10,10,10,5,2,5897,1,1,1357,1,453,114,2013-12-31,2013-03-31,2013-01-31,2013-01-12,10-Jan-2013,Jan 10th 2013,Thu 01-10-13,1,0,0,0,2012-12-21,365,90,31,7,2012-12-30,2012-12-30,2012-12-30,2013-01-06,2013-01-07,2013-01-10,Thu       ,2013M01,Jan-2013,N,2013M01   ,N,Year 2013 - Quarter 01,2013Q01   ,N,2,2013,2013-01-06,2013-01-12,N,Wk.02 - 13,2013-01-06 00:00:00,2013-01-12 00:00:00,2013W02   ,2013W02   ,01/06/13 - 01/12/13,01/06 - 01/12,2013,N,2012-01-10,2011-01-10,2012-10-10,2012-07-10,2012-12-10,2012-11-10,2013-01-03,2012-12-27,0,0,0,0,0,0,0,0,1,1,2,1,1,1,1,2005-09-07,DW_BATCH  ,2013-11-27 00:16:56,DW_BATCH
+2013-05-18,2013-01-01,2013-04-01,2013-05-01,2013-05-12,0,-2,-6,-28,-193,0,-2,-7,-28,-28,41411,138,48,18,7,20,5915,2,5,1361,2,454,114,2013-12-31,2013-06-30,2013-05-31,2013-05-18,18-May-2013,May 18th 2013,Sat 05-18-13,1,0,0,0,2013-03-21,365,91,31,7,2012-12-30,2013-03-31,2013-04-28,2013-05-12,2013-05-13,2013-05-18,Sat       ,2013M05,May-2013,N,2013M05   ,N,Year 2013 - Quarter 02,2013Q02   ,N,20,2013,2013-05-12,2013-05-18,N,Wk.20 - 13,2013-05-12 00:00:00,2013-05-18 00:00:00,2013W20   ,2013W20   ,05/12/13 - 05/18/13,05/12 - 05/18,2013,N,2012-05-18,2011-05-18,2013-02-18,2012-11-18,2013-04-18,2013-03-18,2013-05-11,2013-05-04,0,0,0,0,0,0,0,0,5,2,20,2,1,1,1,2005-09-07,DW_BATCH  ,2013-11-27 00:16:56,DW_BATCH
+2013-04-15,2013-01-01,2013-04-01,2013-04-01,2013-04-14,0,-2,-7,-32,-226,0,-2,-8,-32,-32,41378,105,15,15,2,16,5911,1,4,1360,2,454,114,2013-12-31,2013-06-30,2013-04-30,2013-04-20,15-Apr-2013,Apr 15th 2013,Mon 04-15-13,1,0,0,0,2013-03-21,365,91,30,7,2012-12-30,2013-03-31,2013-03-31,2013-04-14,2013-04-15,2013-04-15,Mon       ,2013M04,Apr-2013,N,2013M04   ,N,Year 2013 - Quarter 02,2013Q02   ,N,16,2013,2013-04-14,2013-04-20,N,Wk.16 - 13,2013-04-14 00:00:00,2013-04-20 00:00:00,2013W16   ,2013W16   ,04/14/13 - 04/20/13,04/14 - 04/20,2013,N,2012-04-15,2011-04-15,2013-01-15,2012-10-15,2013-03-15,2013-02-15,2013-04-08,2013-04-01,0,0,0,0,0,0,0,0,4,2,16,2,1,1,1,2005-09-07,DW_BATCH  ,2013-11-27 00:16:56,DW_BATCH
+2013-03-12,2013-01-01,2013-01-01,2013-03-01,2013-03-10,0,-3,-8,-37,-260,0,-3,-9,-37,-37,41344,71,71,12,3,11,5906,3,3,1359,1,453,114,2013-12-31,2013-03-31,2013-03-31,2013-03-16,12-Mar-2013,Mar 12th 2013,Tue 03-12-13,1,0,0,0,2012-12-21,365,90,31,7,2012-12-30,2012-12-30,2013-02-24,2013-03-10,2013-03-11,2013-03-12,Tue       ,2013M03,Mar-2013,N,2013M03   ,N,Year 2013 - Quarter 01,2013Q01   ,N,11,2013,2013-03-10,2013-03-16,N,Wk.11 - 13,2013-03-10 00:00:00,2013-03-16 00:00:00,2013W11   ,2013W11   ,03/10/13 - 03/16/13,03/10 - 03/16,2013,N,2012-03-12,2011-03-12,2012-12-12,2012-09-12,2013-02-12,2013-01-12,2013-03-05,2013-02-26,0,0,0,0,0,0,0,0,3,1,11,1,1,1,1,2005-09-07,DW_BATCH  ,2013-11-27 00:16:56,DW_BATCH
+2013-11-09,2013-01-01,2013-10-01,2013-11-01,2013-11-03,0,0,0,-3,-18,0,0,-1,-3,-3,41586,313,40,9,7,45,5940,2,11,1367,4,456,114,2013-12-31,2013-12-31,2013-11-30,2013-11-09,09-Nov-2013,Nov 9th 2013,Sat 11-09-13,1,1,1,0,2013-09-21,365,92,30,7,2012-12-30,2013-09-29,2013-10-27,2013-11-03,2013-11-04,2013-11-09,Sat       ,2013M11,Nov-2013,N,2013M11   ,N,Year 2013 - Quarter 04,2013Q04   ,N,45,2013,2013-11-03,2013-11-09,N,Wk.45 - 13,2013-11-03 00:00:00,2013-11-09 00:00:00,2013W45   ,2013W45   ,11/03/13 - 11/09/13,11/03 - 11/09,2013,N,2012-11-09,2011-11-09,2013-08-09,2013-05-09,2013-10-09,2013-09-09,2013-11-02,2013-10-26,0,0,0,0,0,0,0,0,11,4,45,4,1,1,1,2005-09-07,DW_BATCH  ,2013-11-27 00:16:56,DW_BATCH
+2013-03-14,2013-01-01,2013-01-01,2013-03-01,2013-03-10,0,-3,-8,-37,-258,0,-3,-9,-37,-37,41346,73,73,14,5,11,5906,3,3,1359,1,453,114,2013-12-31,2013-03-31,2013-03-31,2013-03-16,14-Mar-2013,Mar 14th 2013,Thu 03-14-13,1,0,0,0,2012-12-21,365,90,31,7,2012-12-30,2012-12-30,2013-02-24,2013-03-10,2013-03-11,2013-03-14,Thu       ,2013M03,Mar-2013,N,2013M03   ,N,Year 2013 - Quarter 01,2013Q01   ,N,11,2013,2013-03-10,2013-03-16,N,Wk.11 - 13,2013-03-10 00:00:00,2013-03-16 00:00:00,2013W11   ,2013W11   ,03/10/13 - 03/16/13,03/10 - 03/16,2013,N,2012-03-14,2011-03-14,2012-12-14,2012-09-14,2013-02-14,2013-01-14,2013-03-07,2013-02-28,0,0,0,0,0,0,0,0,3,1,11,1,1,1,1,2005-09-07,DW_BATCH  ,2013-11-27 00:16:56,DW_BATCH
+2013-09-03,2013-01-01,2013-07-01,2013-09-01,2013-09-01,0,-1,-2,-12,-85,0,-1,-3,-12,-12,41519,246,65,3,3,36,5931,3,9,1365,3,455,114,2013-12-31,2013-09-30,2013-09-30,2013-09-07,03-Sep-2013,Sep 3rd 2013,Tue 09-03-13,1,0,0,0,2013-06-21,365,92,30,7,2012-12-30,2013-06-30,2013-08-25,2013-09-01,2013-09-02,2013-09-03,Tue       ,2013M09,Sep-2013,N,2013M09   ,N,Year 2013 - Quarter 03,2013Q03   ,N,36,2013,2013-09-01,2013-09-07,N,Wk.36 - 13,2013-09-01 00:00:00,2013-09-07 00:00:00,2013W36   ,2013W36   ,09/01/13 - 09/07/13,09/01 - 09/07,2013,N,2012-09-03,2011-09-03,2013-06-03,2013-03-03,2013-08-03,2013-07-03,2013-08-27,2013-08-20,0,0,0,0,0,0,0,0,9,3,36,3,1,1,1,2005-09-07,DW_BATCH  ,2013-11-27 00:16:56,DW_BATCH
+2013-05-16,2013-01-01,2013-04-01,2013-05-01,2013-05-12,0,-2,-6,-28,-195,0,-2,-7,-28,-28,41409,136,46,16,5,20,5915,2,5,1361,2,454,114,2013-12-31,2013-06-30,2013-05-31,2013-05-18,16-May-2013,May 16th 2013,Thu 05-16-13,1,0,0,0,2013-03-21,365,91,31,7,2012-12-30,2013-03-31,2013-04-28,2013-05-12,2013-05-13,2013-05-16,Thu       ,2013M05,May-2013,N,2013M05   ,N,Year 2013 - Quarter 02,2013Q02   ,N,20,2013,2013-05-12,2013-05-18,N,Wk.20 - 13,2013-05-12 00:00:00,2013-05-18 00:00:00,2013W20   ,2013W20   ,05/12/13 - 05/18/13,05/12 - 05/18,2013,N,2012-05-16,2011-05-16,2013-02-16,2012-11-16,2013-04-16,2013-03-16,2013-05-09,2013-05-02,0,0,0,0,0,0,0,0,5,2,20,2,1,1,1,2005-09-07,DW_BATCH  ,2013-11-27 00:16:56,DW_BATCH
+2013-10-27,2013-01-01,2013-10-01,2013-10-01,2013-10-27,0,0,-1,-4,-31,0,0,-1,-4,-5,41573,300,27,27,1,44,5939,1,10,1366,4,456,114,2013-12-31,2013-12-31,2013-10-31,2013-11-02,27-Oct-2013,Oct 27th 2013,Sun 10-27-13,1,1,0,0,2013-09-21,365,92,31,7,2012-12-30,2013-09-29,2013-10-27,2013-10-27,2013-10-21,2013-10-27,Sun       ,2013M10,Oct-2013,N,2013M10   ,N,Year 2013 - Quarter 04,2013Q04   ,N,44,2013,2013-10-27,2013-11-02,Y,Wk.44 - 13,2013-10-27 00:00:00,2013-11-02 00:00:00,2013W44   ,2013W44   ,10/27/13 - 11/02/13,10/27 - 11/02,2013,N,2012-10-27,2011-10-27,2013-07-27,2013-04-27,2013-09-27,2013-08-27,2013-10-20,2013-10-13,0,0,0,0,1,0,0,0,11,4,44,4,1,1,1,2005-09-07,DW_BATCH  ,2013-11-27 00:16:56,DW_BATCH
+2013-08-21,2013-01-01,2013-07-01,2013-08-01,2013-08-18,0,-1,-3,-14,-98,0,-1,-4,-14,-14,41506,233,52,21,4,34,5929,2,8,1364,3,455,114,2013-12-31,2013-09-30,2013-08-31,2013-08-24,21-Aug-2013,Aug 21st 2013,Wed 08-21-13,1,0,0,0,2013-06-21,365,92,31,7,2012-12-30,2013-06-30,2013-07-28,2013-08-18,2013-08-19,2013-08-21,Wed       ,2013M08,Aug-2013,N,2013M08   ,N,Year 2013 - Quarter 03,2013Q03   ,N,34,2013,2013-08-18,2013-08-24,N,Wk.34 - 13,2013-08-18 00:00:00,2013-08-24 00:00:00,2013W34   ,2013W34   ,08/18/13 - 08/24/13,08/18 - 08/24,2013,N,2012-08-21,2011-08-21,2013-05-21,2013-02-21,2013-07-21,2013-06-21,2013-08-14,2013-08-07,0,0,0,0,0,0,0,0,8,3,34,3,1,1,1,2005-09-07,DW_BATCH  ,2013-11-27 00:16:56,DW_BATCH
+2013-10-25,2013-01-01,2013-10-01,2013-10-01,2013-10-20,0,0,-1,-5,-33,0,0,-2,-5,-5,41571,298,25,25,6,43,5938,1,10,1366,4,456,114,2013-12-31,2013-12-31,2013-10-31,2013-10-26,25-Oct-2013,Oct 25th 2013,Fri 10-25-13,1,1,0,0,2013-09-21,365,92,31,7,2012-12-30,2013-09-29,2013-09-29,2013-10-20,2013-10-21,2013-10-25,Fri       ,2013M10,Oct-2013,N,2013M10   ,N,Year 2013 - Quarter 04,2013Q04   ,N,43,2013,2013-10-20,2013-10-26,N,Wk.43 - 13,2013-10-20 00:00:00,2013-10-26 00:00:00,2013W43   ,2013W43   ,10/20/13 - 10/26/13,10/20 - 10/26,2013,N,2012-10-25,2011-10-25,2013-07-25,2013-04-25,2013-09-25,2013-08-25,2013-10-18,2013-10-11,0,0,0,0,0,0,0,0,10,4,43,4,1,1,1,2005-09-07,DW_BATCH  ,2013-11-27 00:16:56,DW_BATCH
+2013-09-24,2013-01-01,2013-07-01,2013-09-01,2013-09-22,0,-1,-2,-9,-64,0,-1,-3,-9,-9,41540,267,86,24,3,39,5934,3,9,1365,3,455,114,2013-12-31,2013-09-30,2013-09-30,2013-09-28,24-Sep-2013,Sep 24th 2013,Tue 09-24-13,1,0,0,0,2013-09-21,365,92,30,7,2012-12-30,2013-06-30,2013-08-25,2013-09-22,2013-09-23,2013-09-24,Tue       ,2013M09,Sep-2013,N,2013M09   ,N,Year 2013 - Quarter 03,2013Q03   ,N,39,2013,2013-09-22,2013-09-28,N,Wk.39 - 13,2013-09-22 00:00:00,2013-09-28 00:00:00,2013W39   ,2013W39   ,09/22/13 - 09/28/13,09/22 - 09/28,2013,N,2012-09-24,2011-09-24,2013-06-24,2013-03-24,2013-08-24,2013-07-24,2013-09-17,2013-09-10,0,0,0,0,0,0,0,0,9,3,39,4,1,1,1,2005-09-07,DW_BATCH  ,2013-11-27 00:16:56,DW_BATCH
+2013-01-27,2013-01-01,2013-01-01,2013-01-01,2013-01-27,0,-3,-10,-43,-304,0,-3,-10,-43,-44,41300,27,27,27,1,5,5900,1,1,1357,1,453,114,2013-12-31,2013-03-31,2013-01-31,2013-02-02,27-Jan-2013,Jan 27th 2013,Sun 01-27-13,1,0,0,0,2012-12-21,365,90,31,7,2012-12-30,2012-12-30,2013-01-27,2013-01-27,2013-01-21,2013-01-27,Sun       ,2013M01,Jan-2013,N,2013M01   ,N,Year 2013 - Quarter 01,2013Q01   ,N,5,2013,2013-01-27,2013-02-02,Y,Wk.05 - 13,2013-01-27 00:00:00,2013-02-02 00:00:00,2013W05   ,2013W05   ,01/27/13 - 02/02/13,01/27 - 02/02,2013,N,2012-01-27,2011-01-27,2012-10-27,2012-07-27,2012-12-27,2012-11-27,2013-01-20,2013-01-13,0,0,0,0,0,0,0,0,2,1,5,1,1,1,1,2005-09-07,DW_BATCH  ,2013-11-27 00:16:56,DW_BATCH
+2013-03-10,2013-01-01,2013-01-01,2013-03-01,2013-03-10,0,-3,-8,-37,-262,0,-3,-9,-37,-38,41342,69,69,10,1,11,5906,3,3,1359,1,453,114,2013-12-31,2013-03-31,2013-03-31,2013-03-16,10-Mar-2013,Mar 10th 2013,Sun 03-10-13,1,0,0,0,2012-12-21,365,90,31,7,2012-12-30,2012-12-30,2013-02-24,2013-03-10,2013-03-04,2013-03-10,Sun       ,2013M03,Mar-2013,N,2013M03   ,N,Year 2013 - Quarter 01,2013Q01   ,N,11,2013,2013-03-10,2013-03-16,Y,Wk.11 - 13,2013-03-10 00:00:00,2013-03-16 00:00:00,2013W11   ,2013W11   ,03/10/13 - 03/16/13,03/10 - 03/16,2013,N,2012-03-10,2011-03-10,2012-12-10,2012-09-10,2013-02-10,2013-01-10,2013-03-03,2013-02-24,0,0,0,0,0,0,0,0,3,1,11,1,1,1,1,2005-09-07,DW_BATCH  ,2013-11-27 00:16:56,DW_BATCH
+2013-05-14,2013-01-01,2013-04-01,2013-05-01,2013-05-12,0,-2,-6,-28,-197,0,-2,-7,-28,-28,41407,134,44,14,3,20,5915,2,5,1361,2,454,114,2013-12-31,2013-06-30,2013-05-31,2013-05-18,14-May-2013,May 14th 2013,Tue 05-14-13,1,0,0,0,2013-03-21,365,91,31,7,2012-12-30,2013-03-31,2013-04-28,2013-05-12,2013-05-13,2013-05-14,Tue       ,2013M05,May-2013,N,2013M05   ,N,Year 2013 - Quarter 02,2013Q02   ,N,20,2013,2013-05-12,2013-05-18,N,Wk.20 - 13,2013-05-12 00:00:00,2013-05-18 00:00:00,2013W20   ,2013W20   ,05/12/13 - 05/18/13,05/12 - 05/18,2013,N,2012-05-14,2011-05-14,2013-02-14,2012-11-14,2013-04-14,2013-03-14,2013-05-07,2013-04-30,0,0,0,0,0,0,0,0,5,2,20,2,1,1,1,2005-09-07,DW_BATCH  ,2013-11-27 00:16:56,DW_BATCH
+2013-12-31,2013-01-01,2013-10-01,2013-12-01,2013-12-29,0,0,1,5,34,1,1,1,5,5,41638,365,92,31,3,53,5948,3,12,1368,4,456,114,2013-12-31,2013-12-31,2013-12-31,2013-12-31,31-Dec-2013,Dec 31st 2013,Tue 12-31-13,0,0,0,0,2013-12-21,365,92,31,3,2013-12-29,2013-12-29,2013-12-29,2013-12-29,2013-12-30,2013-12-31,Tue       ,2013M12,Dec-2013,N,2013M12   ,N,Year 2013 - Quarter 04,2013Q04   ,N,1,2014,2013-12-29,2014-01-04,N,Wk.53 - 13,2013-12-29 00:00:00,2013-12-31 00:00:00,2013W53   ,2013W53   ,12/29/13 - 12/31/13,12/29 - 12/31,2013,N,2012-12-31,2011-12-31,2013-09-30,2013-06-30,2013-11-30,2013-10-31,2013-12-24,2013-12-17,0,0,0,0,0,0,0,0,1,1,1,1,0,1,0,2005-09-07,DW_BATCH  ,2013-11-27 00:16:56,DW_BATCH
+2013-09-01,2013-01-01,2013-07-01,2013-09-01,2013-09-01,0,-1,-2,-12,-87,0,-1,-3,-12,-13,41517,244,63,1,1,36,5931,3,9,1365,3,455,114,2013-12-31,2013-09-30,2013-09-30,2013-09-07,01-Sep-2013,Sep 1st 2013,Sun 09-01-13,1,0,0,0,2013-06-21,365,92,30,7,2012-12-30,2013-06-30,2013-08-25,2013-09-01,2013-08-26,2013-09-01,Sun       ,2013M09,Sep-2013,Y,2013M09   ,Y,Year 2013 - Quarter 03,2013Q03   ,N,36,2013,2013-09-01,2013-09-07,Y,Wk.36 - 13,2013-09-01 00:00:00,2013-09-07 00:00:00,2013W36   ,2013W36   ,09/01/13 - 09/07/13,09/01 - 09/07,2013,N,2012-09-01,2011-09-01,2013-06-01,2013-03-01,2013-08-01,2013-07-01,2013-08-25,2013-08-18,0,0,0,0,0,0,0,0,9,3,36,3,1,1,1,2005-09-07,DW_BATCH  ,2013-11-27 00:16:56,DW_BATCH
+2013-04-13,2013-01-01,2013-04-01,2013-04-01,2013-04-07,0,-2,-7,-33,-228,0,-2,-8,-33,-33,41376,103,13,13,7,15,5910,1,4,1360,2,454,114,2013-12-31,2013-06-30,2013-04-30,2013-04-13,13-Apr-2013,Apr 13th 2013,Sat 04-13-13,1,0,0,0,2013-03-21,365,91,30,7,2012-12-30,2013-03-31,2013-03-31,2013-04-07,2013-04-08,2013-04-13,Sat       ,2013M04,Apr-2013,N,2013M04   ,N,Year 2013 - Quarter 02,2013Q02   ,N,15,2013,2013-04-07,2013-04-13,N,Wk.15 - 13,2013-04-07 00:00:00,2013-04-13 00:00:00,2013W15   ,2013W15   ,04/07/13 - 04/13/13,04/07 - 04/13,2013,N,2012-04-13,2011-04-13,2013-01-13,2012-10-13,2013-03-13,2013-02-13,2013-04-06,2013-03-30,0,0,0,0,0,0,0,0,4,2,15,2,1,1,1,2005-09-07,DW_BATCH  ,2013-11-27 00:16:56,DW_BATCH
+2013-02-09,2013-01-01,2013-01-01,2013-02-01,2013-02-03,0,-3,-9,-42,-291,0,-3,-10,-42,-42,41313,40,40,9,7,6,5901,2,2,1358,1,453,114,2013-12-31,2013-03-31,2013-02-28,2013-02-09,09-Feb-2013,Feb 9th 2013,Sat 02-09-13,1,0,0,0,2012-12-21,365,90,28,7,2012-12-30,2012-12-30,2013-01-27,2013-02-03,2013-02-04,2013-02-09,Sat       ,2013M02,Feb-2013,N,2013M02   ,N,Year 2013 - Quarter 01,2013Q01   ,N,6,2013,2013-02-03,2013-02-09,N,Wk.06 - 13,2013-02-03 00:00:00,2013-02-09 00:00:00,2013W06   ,2013W06   ,02/03/13 - 02/09/13,02/03 - 02/09,2013,N,2012-02-09,2011-02-09,2012-11-09,2012-08-09,2013-01-09,2012-12-09,2013-02-02,2013-01-26,0,0,0,0,0,0,0,0,2,1,6,1,1,1,1,2005-09-07,DW_BATCH  ,2013-11-27 00:16:56,DW_BATCH
+2013-12-27,2013-01-01,2013-10-01,2013-12-01,2013-12-22,0,0,1,4,30,0,0,0,4,4,41634,361,88,27,6,52,5947,3,12,1368,4,456,114,2013-12-31,2013-12-31,2013-12-31,2013-12-28,27-Dec-2013,Dec 27th 2013,Fri 12-27-13,0,0,0,0,2013-12-21,365,92,31,7,2012-12-30,2013-09-29,2013-11-24,2013-12-22,2013-12-23,2013-12-27,Fri       ,2013M12,Dec-2013,N,2013M12   ,N,Year 2013 - Quarter 04,2013Q04   ,N,52,2013,2013-12-22,2013-12-28,N,Wk.52 - 13,2013-12-22 00:00:00,2013-12-28 00:00:00,2013W52   ,2013W52   ,12/22/13 - 12/28/13,12/22 - 12/28,2013,N,2012-12-27,2011-12-27,2013-09-27,2013-06-27,2013-11-27,2013-10-27,2013-12-20,2013-12-13,0,0,0,0,0,0,0,0,12,4,52,1,0,1,0,2005-09-07,DW_BATCH  ,2013-11-27 00:16:56,DW_BATCH
+2013-08-19,2013-01-01,2013-07-01,2013-08-01,2013-08-18,0,-1,-3,-14,-100,0,-1,-4,-14,-14,41504,231,50,19,2,34,5929,2,8,1364,3,455,114,2013-12-31,2013-09-30,2013-08-31,2013-08-24,19-Aug-2013,Aug 19th 2013,Mon 08-19-13,1,0,0,0,2013-06-21,365,92,31,7,2012-12-30,2013-06-30,2013-07-28,2013-08-18,2013-08-19,2013-08-19,Mon       ,2013M08,Aug-2013,N,2013M08   ,N,Year 2013 - Quarter 03,2013Q03   ,N,34,2013,2013-08-18,2013-08-24,N,Wk.34 - 13,2013-08-18 00:00:00,2013-08-24 00:00:00,2013W34   ,2013W34   ,08/18/13 - 08/24/13,08/18 - 08/24,2013,N,2012-08-19,2011-08-19,2013-05-19,2013-02-19,2013-07-19,2013-06-19,2013-08-12,2013-08-05,0,0,0,0,0,0,0,0,8,3,34,3,1,1,1,2005-09-07,DW_BATCH  ,2013-11-27 00:16:56,DW_BATCH
+2013-04-11,2013-01-01,2013-04-01,2013-04-01,2013-04-07,0,-2,-7,-33,-230,0,-2,-8,-33,-33,41374,101,11,11,5,15,5910,1,4,1360,2,454,114,2013-12-31,2013-06-30,2013-04-30,2013-04-13,11-Apr-2013,Apr 11th 2013,Thu 04-11-13,1,0,0,0,2013-03-21,365,91,30,7,2012-12-30,2013-03-31,2013-03-31,2013-04-07,2013-04-08,2013-04-11,Thu       ,2013M04,Apr-2013,N,2013M04   ,N,Year 2013 - Quarter 02,2013Q02   ,N,15,2013,2013-04-07,2013-04-13,N,Wk.15 - 13,2013-04-07 00:00:00,2013-04-13 00:00:00,2013W15   ,2013W15   ,04/07/13 - 04/13/13,04/07 - 04/13,2013,N,2012-04-11,2011-04-11,2013-01-11,2012-10-11,2013-03-11,2013-02-11,2013-04-04,2013-03-28,0,0,0,0,0,0,0,0,4,2,15,2,1,1,1,2005-09-07,DW_BATCH  ,2013-11-27 00:16:56,DW_BATCH
+2013-12-06,2013-01-01,2013-10-01,2013-12-01,2013-12-01,0,0,1,1,9,0,0,0,1,1,41613,340,67,6,6,49,5944,3,12,1368,4,456,114,2013-12-31,2013-12-31,2013-12-31,2013-12-07,06-Dec-2013,Dec 6th 2013,Fri 12-06-13,0,0,0,0,2013-09-21,365,92,31,7,2012-12-30,2013-09-29,2013-11-24,2013-12-01,2013-12-02,2013-12-06,Fri       ,2013M12,Dec-2013,N,2013M12   ,N,Year 2013 - Quarter 04,2013Q04   ,N,49,2013,2013-12-01,2013-12-07,N,Wk.49 - 13,2013-12-01 00:00:00,2013-12-07 00:00:00,2013W49   ,2013W49   ,12/01/13 - 12/07/13,12/01 - 12/07,2013,N,2012-12-06,2011-12-06,2013-09-06,2013-06-06,2013-11-06,2013-10-06,2013-11-29,2013-11-22,0,0,0,0,0,0,0,0,12,4,49,4,0,1,0,2005-09-07,DW_BATCH  ,2013-11-27 00:16:56,DW_BATCH
+2013-03-31,2013-01-01,2013-01-01,2013-03-01,2013-03-31,0,-3,-8,-34,-241,0,-2,-8,-34,-35,41363,90,90,31,1,14,5909,3,3,1359,1,453,114,2013-12-31,2013-03-31,2013-03-31,2013-04-06,31-Mar-2013,Mar 31st 2013,Sun 03-31-13,1,0,0,0,2013-03-21,365,90,31,7,2012-12-30,2013-03-31,2013-03-31,2013-03-31,2013-03-25,2013-03-31,Sun       ,2013M03,Mar-2013,N,2013M03   ,N,Year 2013 - Quarter 01,2013Q01   ,N,14,2013,2013-03-31,2013-04-06,Y,Wk.14 - 13,2013-03-31 00:00:00,2013-04-06 00:00:00,2013W14   ,2013W14   ,03/31/13 - 04/06/13,03/31 - 04/06,2013,N,2012-03-31,2011-03-31,2012-12-31,2012-09-30,2013-02-28,2013-01-31,2013-03-24,2013-03-17,0,0,0,0,0,0,0,0,4,2,14,2,1,1,1,2005-09-07,DW_BATCH  ,2013-11-27 00:16:56,DW_BATCH
+2013-09-22,2013-01-01,2013-07-01,2013-09-01,2013-09-22,0,-1,-2,-9,-66,0,-1,-3,-9,-10,41538,265,84,22,1,39,5934,3,9,1365,3,455,114,2013-12-31,2013-09-30,2013-09-30,2013-09-28,22-Sep-2013,Sep 22nd 2013,Sun 09-22-13,1,0,0,0,2013-09-21,365,92,30,7,2012-12-30,2013-06-30,2013-08-25,2013-09-22,2013-09-16,2013-09-22,Sun       ,2013M09,Sep-2013,N,2013M09   ,N,Year 2013 - Quarter 03,2013Q03   ,N,39,2013,2013-09-22,2013-09-28,Y,Wk.39 - 13,2013-09-22 00:00:00,2013-09-28 00:00:00,2013W39   ,2013W39   ,09/22/13 - 09/28/13,09/22 - 09/28,2013,N,2012-09-22,2011-09-22,2013-06-22,2013-03-22,2013-08-22,2013-07-22,2013-09-15,2013-09-08,0,0,0,0,0,0,0,0,9,3,39,4,1,1,1,2005-09-07,DW_BATCH  ,2013-11-27 00:16:56,DW_BATCH
+2013-01-06,2013-01-01,2013-01-01,2013-01-01,2013-01-06,0,-3,-10,-46,-325,0,-3,-11,-46,-47,41279,6,6,6,1,2,5897,1,1,1357,1,453,114,2013-12-31,2013-03-31,2013-01-31,2013-01-12,06-Jan-2013,Jan 6th 2013,Sun 01-06-13,1,0,0,0,2012-12-21,365,90,31,7,2012-12-30,2012-12-30,2012-12-30,2013-01-06,2012-12-31,2013-01-06,Sun       ,2013M01,Jan-2013,N,2013M01   ,N,Year 2013 - Quarter 01,2013Q01   ,N,2,2013,2013-01-06,2013-01-12,Y,Wk.02 - 13,2013-01-06 00:00:00,2013-01-12 00:00:00,2013W02   ,2013W02   ,01/06/13 - 01/12/13,01/06 - 01/12,2013,N,2012-01-06,2011-01-06,2012-10-06,2012-07-06,2012-12-06,2012-11-06,2012-12-30,2012-12-23,0,0,0,0,0,0,0,0,1,1,2,1,1,1,1,2005-09-07,DW_BATCH  ,2013-11-27 00:16:56,DW_BATCH
+2013-01-04,2013-01-01,2013-01-01,2013-01-01,2013-01-01,0,-3,-10,-47,-327,0,-3,-11,-47,-47,41277,4,4,4,6,1,5896,1,1,1357,1,453,114,2013-12-31,2013-03-31,2013-01-31,2013-01-05,04-Jan-2013,Jan 4th 2013,Fri 01-04-13,1,0,0,0,2012-12-21,365,90,31,5,2012-12-30,2012-12-30,2012-12-30,2012-12-30,2012-12-31,2013-01-04,Fri       ,2013M01,Jan-2013,N,2013M01   ,N,Year 2013 - Quarter 01,2013Q01   ,N,1,2013,2012-12-30,2013-01-05,N,Wk.01 - 13,2013-01-01 00:00:00,2013-01-05 00:00:00,2013W01   ,2013W01   ,01/01/13 - 01/05/13,01/01 - 01/05,2013,N,2012-01-04,2011-01-04,2012-10-04,2012-07-04,2012-12-04,2012-11-04,2012-12-28,2012-12-21,0,0,0,0,0,0,0,0,1,1,1,1,1,1,1,2005-09-07,DW_BATCH  ,2013-11-27 00:16:56,DW_BATCH
+2013-07-16,2013-01-01,2013-07-01,2013-07-01,2013-07-14,0,-1,-4,-19,-134,0,-1,-5,-19,-19,41470,197,16,16,3,29,5924,1,7,1363,3,455,114,2013-12-31,2013-09-30,2013-07-31,2013-07-20,16-Jul-2013,Jul 16th 2013,Tue 07-16-13,1,0,0,0,2013-06-21,365,92,31,7,2012-12-30,2013-06-30,2013-06-30,2013-07-14,2013-07-15,2013-07-16,Tue       ,2013M07,Jul-2013,N,2013M07   ,N,Year 2013 - Quarter 03,2013Q03   ,N,29,2013,2013-07-14,2013-07-20,N,Wk.29 - 13,2013-07-14 00:00:00,2013-07-20 00:00:00,2013W29   ,2013W29   ,07/14/13 - 07/20/13,07/14 - 07/20,2013,N,2012-07-16,2011-07-16,2013-04-16,2013-01-16,2013-06-16,2013-05-16,2013-07-09,2013-07-02,0,0,0,0,0,0,0,0,7,3,29,3,1,1,1,2005-09-07,DW_BATCH  ,2013-11-27 00:16:56,DW_BATCH
+2013-11-03,2013-01-01,2013-10-01,2013-11-01,2013-11-03,0,0,0,-3,-24,0,0,-1,-3,-4,41580,307,34,3,1,45,5940,2,11,1367,4,456,114,2013-12-31,2013-12-31,2013-11-30,2013-11-09,03-Nov-2013,Nov 3rd 2013,Sun 11-03-13,1,1,1,0,2013-09-21,365,92,30,7,2012-12-30,2013-09-29,2013-10-27,2013-11-03,2013-10-28,2013-11-03,Sun       ,2013M11,Nov-2013,N,2013M11   ,N,Year 2013 - Quarter 04,2013Q04   ,N,45,2013,2013-11-03,2013-11-09,Y,Wk.45 - 13,2013-11-03 00:00:00,2013-11-09 00:00:00,2013W45   ,2013W45   ,11/03/13 - 11/09/13,11/03 - 11/09,2013,N,2012-11-03,2011-11-03,2013-08-03,2013-05-03,2013-10-03,2013-09-03,2013-10-27,2013-10-20,0,0,0,0,0,0,0,0,11,4,45,4,1,1,1,2005-09-07,DW_BATCH  ,2013-11-27 00:16:56,DW_BATCH
+2013-08-25,2013-01-01,2013-07-01,2013-08-01,2013-08-25,0,-1,-3,-13,-94,0,-1,-3,-13,-14,41510,237,56,25,1,35,5930,2,8,1364,3,455,114,2013-12-31,2013-09-30,2013-08-31,2013-08-31,25-Aug-2013,Aug 25th 2013,Sun 08-25-13,1,0,0,0,2013-06-21,365,92,31,7,2012-12-30,2013-06-30,2013-08-25,2013-08-25,2013-08-19,2013-08-25,Sun       ,2013M08,Aug-2013,N,2013M08   ,N,Year 2013 - Quarter 03,2013Q03   ,N,35,2013,2013-08-25,2013-08-31,Y,Wk.35 - 13,2013-08-25 00:00:00,2013-08-31 00:00:00,2013W35   ,2013W35   ,08/25/13 - 08/31/13,08/25 - 08/31,2013,N,2012-08-25,2011-08-25,2013-05-25,2013-02-25,2013-07-25,2013-06-25,2013-08-18,2013-08-11,0,0,0,0,0,0,0,0,9,3,35,3,1,1,1,2005-09-07,DW_BATCH  ,2013-11-27 00:16:56,DW_BATCH
+2013-11-24,2013-01-01,2013-10-01,2013-11-01,2013-11-24,0,0,0,0,-3,0,0,0,0,-1,41601,328,55,24,1,48,5943,2,11,1367,4,456,114,2013-12-31,2013-12-31,2013-11-30,2013-11-30,24-Nov-2013,Nov 24th 2013,Sun 11-24-13,1,1,1,1,2013-09-21,365,92,30,7,2012-12-30,2013-09-29,2013-11-24,2013-11-24,2013-11-18,2013-11-24,Sun       ,2013M11,Nov-2013,N,2013M11   ,N,Year 2013 - Quarter 04,2013Q04   ,N,48,2013,2013-11-24,2013-11-30,Y,Wk.48 - 13,2013-11-24 00:00:00,2013-11-30 00:00:00,2013W48   ,2013W48   ,11/24/13 - 11/30/13,11/24 - 11/30,2013,N,2012-11-24,2011-11-24,2013-08-24,2013-05-24,2013-10-24,2013-09-24,2013-11-17,2013-11-10,0,0,0,0,0,0,0,0,12,4,48,4,1,1,1,2005-09-07,DW_BATCH  ,2013-11-27 00:16:56,DW_BATCH
+2013-05-12,2013-01-01,2013-04-01,2013-05-01,2013-05-12,0,-2,-6,-28,-199,0,-2,-7,-28,-29,41405,132,42,12,1,20,5915,2,5,1361,2,454,114,2013-12-31,2013-06-30,2013-05-31,2013-05-18,12-May-2013,May 12th 2013,Sun 05-12-13,1,0,0,0,2013-03-21,365,91,31,7,2012-12-30,2013-03-31,2013-04-28,2013-05-12,2013-05-06,2013-05-12,Sun       ,2013M05,May-2013,N,2013M05   ,N,Year 2013 - Quarter 02,2013Q02   ,N,20,2013,2013-05-12,2013-05-18,Y,Wk.20 - 13,2013-05-12 00:00:00,2013-05-18 00:00:00,2013W20   ,2013W20   ,05/12/13 - 05/18/13,05/12 - 05/18,2013,N,2012-05-12,2011-05-12,2013-02-12,2012-11-12,2013-04-12,2013-03-12,2013-05-05,2013-04-28,0,0,0,0,0,0,0,0,5,2,20,2,1,1,1,2005-09-07,DW_BATCH  ,2013-11-27 00:16:56,DW_BATCH
+2013-08-17,2013-01-01,2013-07-01,2013-08-01,2013-08-11,0,-1,-3,-15,-102,0,-1,-4,-15,-15,41502,229,48,17,7,33,5928,2,8,1364,3,455,114,2013-12-31,2013-09-30,2013-08-31,2013-08-17,17-Aug-2013,Aug 17th 2013,Sat 08-17-13,1,0,0,0,2013-06-21,365,92,31,7,2012-12-30,2013-06-30,2013-07-28,2013-08-11,2013-08-12,2013-08-17,Sat       ,2013M08,Aug-2013,N,2013M08   ,N,Year 2013 - Quarter 03,2013Q03   ,N,33,2013,2013-08-11,2013-08-17,N,Wk.33 - 13,2013-08-11 00:00:00,2013-08-17 00:00:00,2013W33   ,2013W33   ,08/11/13 - 08/17/13,08/11 - 08/17,2013,N,2012-08-17,2011-08-17,2013-05-17,2013-02-17,2013-07-17,2013-06-17,2013-08-10,2013-08-03,0,0,0,0,0,0,0,0,8,3,33,3,1,1,1,2005-09-07,DW_BATCH  ,2013-11-27 00:16:56,DW_BATCH
+2013-10-21,2013-01-01,2013-10-01,2013-10-01,2013-10-20,0,0,-1,-5,-37,0,0,-2,-5,-5,41567,294,21,21,2,43,5938,1,10,1366,4,456,114,2013-12-31,2013-12-31,2013-10-31,2013-10-26,21-Oct-2013,Oct 21st 2013,Mon 10-21-13,1,1,0,0,2013-09-21,365,92,31,7,2012-12-30,2013-09-29,2013-09-29,2013-10-20,2013-10-21,2013-10-21,Mon       ,2013M10,Oct-2013,N,2013M10   ,N,Year 2013 - Quarter 04,2013Q04   ,N,43,2013,2013-10-20,2013-10-26,N,Wk.43 - 13,2013-10-20 00:00:00,2013-10-26 00:00:00,2013W43   ,2013W43   ,10/20/13 - 10/26/13,10/20 - 10/26,2013,N,2012-10-21,2011-10-21,2013-07-21,2013-04-21,2013-09-21,2013-08-21,2013-10-14,2013-10-07,0,0,0,0,0,0,0,0,10,4,43,4,1,1,1,2005-09-07,DW_BATCH  ,2013-11-27 00:16:56,DW_BATCH
+2013-03-27,2013-01-01,2013-01-01,2013-03-01,2013-03-24,0,-3,-8,-35,-245,0,-3,-9,-35,-35,41359,86,86,27,4,13,5908,3,3,1359,1,453,114,2013-12-31,2013-03-31,2013-03-31,2013-03-30,27-Mar-2013,Mar 27th 2013,Wed 03-27-13,1,0,0,0,2013-03-21,365,90,31,7,2012-12-30,2012-12-30,2013-02-24,2013-03-24,2013-03-25,2013-03-27,Wed       ,2013M03,Mar-2013,N,2013M03   ,N,Year 2013 - Quarter 01,2013Q01   ,N,13,2013,2013-03-24,2013-03-30,N,Wk.13 - 13,2013-03-24 00:00:00,2013-03-30 00:00:00,2013W13   ,2013W13   ,03/24/13 - 03/30/13,03/24 - 03/30,2013,N,2012-03-27,2011-03-27,2012-12-27,2012-09-27,2013-02-27,2013-01-27,2013-03-20,2013-03-13,0,0,0,0,0,0,0,0,3,1,13,2,1,1,1,2005-09-07,DW_BATCH  ,2013-11-27 00:16:56,DW_BATCH
+2013-11-22,2013-01-01,2013-10-01,2013-11-01,2013-11-17,0,0,0,-1,-5,0,0,-1,-1,-1,41599,326,53,22,6,47,5942,2,11,1367,4,456,114,2013-12-31,2013-12-31,2013-11-30,2013-11-23,22-Nov-2013,Nov 22nd 2013,Fri 11-22-13,1,1,1,0,2013-09-21,365,92,30,7,2012-12-30,2013-09-29,2013-10-27,2013-11-17,2013-11-18,2013-11-22,Fri       ,2013M11,Nov-2013,N,2013M11   ,N,Year 2013 - Quarter 04,2013Q04   ,N,47,2013,2013-11-17,2013-11-23,N,Wk.47 - 13,2013-11-17 00:00:00,2013-11-23 00:00:00,2013W47   ,2013W47   ,11/17/13 - 11/23/13,11/17 - 11/23,2013,N,2012-11-22,2011-11-22,2013-08-22,2013-05-22,2013-10-22,2013-09-22,2013-11-15,2013-11-08,0,0,0,0,0,0,0,0,11,4,47,4,1,1,1,2005-09-07,DW_BATCH  ,2013-11-27 00:16:56,DW_BATCH
+2013-07-07,2013-01-01,2013-07-01,2013-07-01,2013-07-07,0,-1,-4,-20,-143,0,-1,-5,-20,-21,41461,188,7,7,1,28,5923,1,7,1363,3,455,114,2013-12-31,2013-09-30,2013-07-31,2013-07-13,07-Jul-2013,Jul 7th 2013,Sun 07-07-13,1,0,0,0,2013-06-21,365,92,31,7,2012-12-30,2013-06-30,2013-06-30,2013-07-07,2013-07-01,2013-07-07,Sun       ,2013M07,Jul-2013,N,2013M07   ,N,Year 2013 - Quarter 03,2013Q03   ,N,28,2013,2013-07-07,2013-07-13,Y,Wk.28 - 13,2013-07-07 00:00:00,2013-07-13 00:00:00,2013W28   ,2013W28   ,07/07/13 - 07/13/13,07/07 - 07/13,2013,N,2012-07-07,2011-07-07,2013-04-07,2013-01-07,2013-06-07,2013-05-07,2013-06-30,2013-06-23,0,0,0,0,0,0,0,0,7,3,28,3,1,1,1,2005-09-07,DW_BATCH  ,2013-11-27 00:16:56,DW_BATCH
+2013-09-11,2013-01-01,2013-07-01,2013-09-01,2013-09-08,0,-1,-2,-11,-77,0,-1,-3,-11,-11,41527,254,73,11,4,37,5932,3,9,1365,3,455,114,2013-12-31,2013-09-30,2013-09-30,2013-09-14,11-Sep-2013,Sep 11th 2013,Wed 09-11-13,1,0,0,0,2013-06-21,365,92,30,7,2012-12-30,2013-06-30,2013-08-25,2013-09-08,2013-09-09,2013-09-11,Wed       ,2013M09,Sep-2013,N,2013M09   ,N,Year 2013 - Quarter 03,2013Q03   ,N,37,2013,2013-09-08,2013-09-14,N,Wk.37 - 13,2013-09-08 00:00:00,2013-09-14 00:00:00,2013W37   ,2013W37   ,09/08/13 - 09/14/13,09/08 - 09/14,2013,N,2012-09-11,2011-09-11,2013-06-11,2013-03-11,2013-08-11,2013-07-11,2013-09-04,2013-08-28,0,0,0,0,0,0,0,0,9,3,37,3,1,1,1,2005-09-07,DW_BATCH  ,2013-11-27 00:16:56,DW_BATCH
+2013-10-19,2013-01-01,2013-10-01,2013-10-01,2013-10-13,0,0,-1,-6,-39,0,0,-2,-6,-6,41565,292,19,19,7,42,5937,1,10,1366,4,456,114,2013-12-31,2013-12-31,2013-10-31,2013-10-19,19-Oct-2013,Oct 19th 2013,Sat 10-19-13,1,1,0,0,2013-09-21,365,92,31,7,2012-12-30,2013-09-29,2013-09-29,2013-10-13,2013-10-14,2013-10-19,Sat       ,2013M10,Oct-2013,N,2013M10   ,N,Year 2013 - Quarter 04,2013Q04   ,N,42,2013,2013-10-13,2013-10-19,N,Wk.42 - 13,2013-10-13 00:00:00,2013-10-19 00:00:00,2013W42   ,2013W42   ,10/13/13 - 10/19/13,10/13 - 10/19,2013,N,2012-10-19,2011-10-19,2013-07-19,2013-04-19,2013-09-19,2013-08-19,2013-10-12,2013-10-05,0,0,0,0,0,0,0,0,10,4,42,4,1,1,1,2005-09-07,DW_BATCH  ,2013-11-27 00:16:56,DW_BATCH
+2013-12-23,2013-01-01,2013-10-01,2013-12-01,2013-12-22,0,0,1,4,26,0,0,0,4,4,41630,357,84,23,2,52,5947,3,12,1368,4,456,114,2013-12-31,2013-12-31,2013-12-31,2013-12-28,23-Dec-2013,Dec 23rd 2013,Mon 12-23-13,0,0,0,0,2013-12-21,365,92,31,7,2012-12-30,2013-09-29,2013-11-24,2013-12-22,2013-12-23,2013-12-23,Mon       ,2013M12,Dec-2013,N,2013M12   ,N,Year 2013 - Quarter 04,2013Q04   ,N,52,2013,2013-12-22,2013-12-28,N,Wk.52 - 13,2013-12-22 00:00:00,2013-12-28 00:00:00,2013W52   ,2013W52   ,12/22/13 - 12/28/13,12/22 - 12/28,2013,N,2012-12-23,2011-12-23,2013-09-23,2013-06-23,2013-11-23,2013-10-23,2013-12-16,2013-12-09,0,0,0,0,0,0,0,0,12,4,52,1,0,1,0,2005-09-07,DW_BATCH  ,2013-11-27 00:16:56,DW_BATCH
+2013-05-24,2013-01-01,2013-04-01,2013-05-01,2013-05-19,0,-2,-6,-27,-187,0,-2,-7,-27,-27,41417,144,54,24,6,21,5916,2,5,1361,2,454,114,2013-12-31,2013-06-30,2013-05-31,2013-05-25,24-May-2013,May 24th 2013,Fri 05-24-13,1,0,0,0,2013-03-21,365,91,31,7,2012-12-30,2013-03-31,2013-04-28,2013-05-19,2013-05-20,2013-05-24,Fri       ,2013M05,May-2013,N,2013M05   ,N,Year 2013 - Quarter 02,2013Q02   ,N,21,2013,2013-05-19,2013-05-25,N,Wk.21 - 13,2013-05-19 00:00:00,2013-05-25 00:00:00,2013W21   ,2013W21   ,05/19/13 - 05/25/13,05/19 - 05/25,2013,N,2012-05-24,2011-05-24,2013-02-24,2012-11-24,2013-04-24,2013-03-24,2013-05-17,2013-05-10,0,0,0,0,0,0,0,0,5,2,21,2,1,1,1,2005-09-07,DW_BATCH  ,2013-11-27 00:16:56,DW_BATCH
+2013-11-05,2013-01-01,2013-10-01,2013-11-01,2013-11-03,0,0,0,-3,-22,0,0,-1,-3,-3,41582,309,36,5,3,45,5940,2,11,1367,4,456,114,2013-12-31,2013-12-31,2013-11-30,2013-11-09,05-Nov-2013,Nov 5th 2013,Tue 11-05-13,1,1,1,0,2013-09-21,365,92,30,7,2012-12-30,2013-09-29,2013-10-27,2013-11-03,2013-11-04,2013-11-05,Tue       ,2013M11,Nov-2013,N,2013M11   ,N,Year 2013 - Quarter 04,2013Q04   ,N,45,2013,2013-11-03,2013-11-09,N,Wk.45 - 13,2013-11-03 00:00:00,2013-11-09 00:00:00,2013W45   ,2013W45   ,11/03/13 - 11/09/13,11/03 - 11/09,2013,N,2012-11-05,2011-11-05,2013-08-05,2013-05-05,2013-10-05,2013-09-05,2013-10-29,2013-10-22,0,0,0,0,0,0,0,0,11,4,45,4,1,1,1,2005-09-07,DW_BATCH  ,2013-11-27 00:16:56,DW_BATCH
+2013-05-29,2013-01-01,2013-04-01,2013-05-01,2013-05-26,0,-2,-6,-26,-182,0,-2,-6,-26,-26,41422,149,59,29,4,22,5917,2,5,1361,2,454,114,2013-12-31,2013-06-30,2013-05-31,2013-06-01,29-May-2013,May 29th 2013,Wed 05-29-13,1,0,0,0,2013-03-21,365,91,31,7,2012-12-30,2013-03-31,2013-05-26,2013-05-26,2013-05-27,2013-05-29,Wed       ,2013M05,May-2013,N,2013M05   ,N,Year 2013 - Quarter 02,2013Q02   ,N,22,2013,2013-05-26,2013-06-01,N,Wk.22 - 13,2013-05-26 00:00:00,2013-06-01 00:00:00,2013W22   ,2013W22   ,05/26/13 - 06/01/13,05/26 - 06/01,2013,N,2012-05-29,2011-05-29,2013-02-28,2012-11-29,2013-04-29,2013-03-29,2013-05-22,2013-05-15,0,0,0,0,0,0,0,0,6,2,22,2,1,1,1,2005-09-07,DW_BATCH  ,2013-11-27 00:16:56,DW_BATCH
+2013-01-02,2013-01-01,2013-01-01,2013-01-01,2013-01-01,0,-3,-10,-47,-329,0,-3,-11,-47,-47,41275,2,2,2,4,1,5896,1,1,1357,1,453,114,2013-12-31,2013-03-31,2013-01-31,2013-01-05,02-Jan-2013,Jan 2nd 2013,Wed 01-02-13,1,0,0,0,2012-12-21,365,90,31,5,2012-12-30,2012-12-30,2012-12-30,2012-12-30,2012-12-31,2013-01-02,Wed       ,2013M01,Jan-2013,N,2013M01   ,N,Year 2013 - Quarter 01,2013Q01   ,N,1,2013,2012-12-30,2013-01-05,N,Wk.01 - 13,2013-01-01 00:00:00,2013-01-05 00:00:00,2013W01   ,2013W01   ,01/01/13 - 01/05/13,01/01 - 01/05,2013,N,2012-01-02,2011-01-02,2012-10-02,2012-07-02,2012-12-02,2012-11-02,2012-12-26,2012-12-19,0,0,0,0,0,0,0,0,1,1,1,1,1,1,1,2005-09-07,DW_BATCH  ,2013-11-27 00:16:56,DW_BATCH
+2013-12-02,2013-01-01,2013-10-01,2013-12-01,2013-12-01,0,0,1,1,5,0,0,0,1,1,41609,336,63,2,2,49,5944,3,12,1368,4,456,114,2013-12-31,2013-12-31,2013-12-31,2013-12-07,02-Dec-2013,Dec 2nd 2013,Mon 12-02-13,0,0,0,0,2013-09-21,365,92,31,7,2012-12-30,2013-09-29,2013-11-24,2013-12-01,2013-12-02,2013-12-02,Mon       ,2013M12,Dec-2013,N,2013M12   ,N,Year 2013 - Quarter 04,2013Q04   ,N,49,2013,2013-12-01,2013-12-07,N,Wk.49 - 13,2013-12-01 00:00:00,2013-12-07 00:00:00,2013W49   ,2013W49   ,12/01/13 - 12/07/13,12/01 - 12/07,2013,N,2012-12-02,2011-12-02,2013-09-02,2013-06-02,2013-11-02,2013-10-02,2013-11-25,2013-11-18,0,0,0,0,0,0,0,0,12,4,49,4,0,1,0,2005-09-07,DW_BATCH  ,2013-11-27 00:16:56,DW_BATCH
+2013-02-24,2013-01-01,2013-01-01,2013-02-01,2013-02-24,0,-3,-9,-39,-276,0,-3,-9,-39,-40,41328,55,55,24,1,9,5904,2,2,1358,1,453,114,2013-12-31,2013-03-31,2013-02-28,2013-03-02,2

<TRUNCATED>
http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/job/src/test/resources/expected_result/flat_item/part-r-00000
----------------------------------------------------------------------
diff --git a/job/src/test/resources/expected_result/flat_item/part-r-00000 b/job/src/test/resources/expected_result/flat_item/part-r-00000
new file mode 100644
index 0000000..452085d
Binary files /dev/null and b/job/src/test/resources/expected_result/flat_item/part-r-00000 differ


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

Posted by lu...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/query/src/test/resources/query/sql/query54.sql
----------------------------------------------------------------------
diff --git a/query/src/test/resources/query/sql/query54.sql b/query/src/test/resources/query/sql/query54.sql
new file mode 100644
index 0000000..8404d06
--- /dev/null
+++ b/query/src/test/resources/query/sql/query54.sql
@@ -0,0 +1,7 @@
+select test_kylin_fact.lstg_format_name, test_cal_dt.week_beg_dt,sum(test_kylin_fact.price) as GMV 
+ , count(*) as TRANS_CNT 
+ from test_kylin_fact 
+ inner JOIN test_cal_dt 
+ ON test_kylin_fact.cal_dt = test_cal_dt.cal_dt 
+ where 1 <> 1
+ group by test_kylin_fact.lstg_format_name, test_cal_dt.week_beg_dt 

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/query/src/test/resources/query/sql/query55.sql
----------------------------------------------------------------------
diff --git a/query/src/test/resources/query/sql/query55.sql b/query/src/test/resources/query/sql/query55.sql
new file mode 100644
index 0000000..05c115b
--- /dev/null
+++ b/query/src/test/resources/query/sql/query55.sql
@@ -0,0 +1 @@
+select count(*) as c from test_cal_dt where extract(YEAR from test_cal_dt.cal_dt) = 2012
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/query/src/test/resources/query/sql/query56.sql
----------------------------------------------------------------------
diff --git a/query/src/test/resources/query/sql/query56.sql b/query/src/test/resources/query/sql/query56.sql
new file mode 100644
index 0000000..62fb8b3
--- /dev/null
+++ b/query/src/test/resources/query/sql/query56.sql
@@ -0,0 +1,8 @@
+select test_cal_dt.week_beg_dt, sum(test_kylin_fact.price) as GMV 
+ , count(*) as TRANS_CNT 
+ from test_kylin_fact 
+ inner JOIN test_cal_dt 
+ ON test_kylin_fact.cal_dt = test_cal_dt.cal_dt
+ where
+ extract(MONTH from test_cal_dt.week_beg_dt) = 12
+ group by test_cal_dt.week_beg_dt
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/query/src/test/resources/query/sql/query57.sql
----------------------------------------------------------------------
diff --git a/query/src/test/resources/query/sql/query57.sql b/query/src/test/resources/query/sql/query57.sql
new file mode 100644
index 0000000..492a5b2
--- /dev/null
+++ b/query/src/test/resources/query/sql/query57.sql
@@ -0,0 +1,8 @@
+select test_cal_dt.week_beg_dt, sum(test_kylin_fact.price) as GMV 
+ , count(*) as TRANS_CNT 
+ from test_kylin_fact 
+ inner JOIN test_cal_dt 
+ ON test_kylin_fact.cal_dt = test_cal_dt.cal_dt
+ where
+ (test_kylin_fact.lstg_format_name='FP-GTC')  and extract(MONTH from test_cal_dt.week_beg_dt) = 12
+ group by test_cal_dt.week_beg_dt
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/query/src/test/resources/query/sql/query58.sql
----------------------------------------------------------------------
diff --git a/query/src/test/resources/query/sql/query58.sql b/query/src/test/resources/query/sql/query58.sql
new file mode 100644
index 0000000..d25fd23
--- /dev/null
+++ b/query/src/test/resources/query/sql/query58.sql
@@ -0,0 +1,8 @@
+select  sum(test_kylin_fact.price) as GMV
+ , count(*) as TRANS_CNT
+ from test_kylin_fact
+ inner JOIN test_cal_dt
+ ON test_kylin_fact.cal_dt = test_cal_dt.cal_dt
+ where
+ (test_kylin_fact.lstg_format_name='FP-GTC')  and extract(MONTH from test_cal_dt.week_beg_dt) = 12
+ group by test_kylin_fact.lstg_format_name
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/query/src/test/resources/query/sql/query59.sql
----------------------------------------------------------------------
diff --git a/query/src/test/resources/query/sql/query59.sql b/query/src/test/resources/query/sql/query59.sql
new file mode 100644
index 0000000..d2db28c
--- /dev/null
+++ b/query/src/test/resources/query/sql/query59.sql
@@ -0,0 +1 @@
+select sum(price) as GVM,lstg_format_name from test_kylin_fact group by lstg_format_name
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/query/src/test/resources/query/sql/query60.sql
----------------------------------------------------------------------
diff --git a/query/src/test/resources/query/sql/query60.sql b/query/src/test/resources/query/sql/query60.sql
new file mode 100644
index 0000000..73283f0
--- /dev/null
+++ b/query/src/test/resources/query/sql/query60.sql
@@ -0,0 +1,6 @@
+select test_kylin_fact.cal_dt, sum(test_kylin_fact.price) as sum_price, count(1) as cnt_1
+from test_kylin_fact 
+left join test_cal_dt on test_kylin_fact.cal_dt=test_cal_dt.cal_dt 
+group by test_kylin_fact.cal_dt 
+order by 2 desc 
+limit 3
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/query/src/test/resources/query/sql/query61.sql
----------------------------------------------------------------------
diff --git a/query/src/test/resources/query/sql/query61.sql b/query/src/test/resources/query/sql/query61.sql
new file mode 100644
index 0000000..acc3635
--- /dev/null
+++ b/query/src/test/resources/query/sql/query61.sql
@@ -0,0 +1,6 @@
+select count(1) as cnt_1
+from test_kylin_fact 
+left join test_cal_dt on test_kylin_fact.cal_dt=test_cal_dt.cal_dt 
+group by test_kylin_fact.cal_dt 
+order by 1 desc 
+limit 4
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/query/src/test/resources/query/sql/query62.sql
----------------------------------------------------------------------
diff --git a/query/src/test/resources/query/sql/query62.sql b/query/src/test/resources/query/sql/query62.sql
new file mode 100644
index 0000000..b9bbd0b
--- /dev/null
+++ b/query/src/test/resources/query/sql/query62.sql
@@ -0,0 +1,3 @@
+select test_kylin_fact.lstg_format_name,sum(test_kylin_fact.price) as GMV , min(cal_dt) as min_cal_dt
+ , count(*) as TRANS_CNT from test_kylin_fact
+ group by test_kylin_fact.lstg_format_name having sum(price)>5000 and count(*)>72
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/query/src/test/resources/query/sql/query63.sql
----------------------------------------------------------------------
diff --git a/query/src/test/resources/query/sql/query63.sql b/query/src/test/resources/query/sql/query63.sql
new file mode 100644
index 0000000..13a26bc
--- /dev/null
+++ b/query/src/test/resources/query/sql/query63.sql
@@ -0,0 +1,2 @@
+select min(cal_dt) as min_cal_dt
+ from test_kylin_fact
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/query/src/test/resources/query/sql/query64.sql
----------------------------------------------------------------------
diff --git a/query/src/test/resources/query/sql/query64.sql b/query/src/test/resources/query/sql/query64.sql
new file mode 100644
index 0000000..bbf99a8
--- /dev/null
+++ b/query/src/test/resources/query/sql/query64.sql
@@ -0,0 +1,3 @@
+ select test_kylin_fact.lstg_format_name,sum(test_kylin_fact.price) as GMV , min(cal_dt) as min_cal_dt
+ , count(*) as TRANS_CNT from test_kylin_fact
+ group by test_kylin_fact.lstg_format_name
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/query/src/test/resources/query/sql/query65.sql
----------------------------------------------------------------------
diff --git a/query/src/test/resources/query/sql/query65.sql b/query/src/test/resources/query/sql/query65.sql
new file mode 100644
index 0000000..b44cb1b
--- /dev/null
+++ b/query/src/test/resources/query/sql/query65.sql
@@ -0,0 +1 @@
+select min(cal_dt) as min_cal_dt, max(cal_dt) as max_cal_dt from test_kylin_fact
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/query/src/test/resources/query/sql/query66.sql
----------------------------------------------------------------------
diff --git a/query/src/test/resources/query/sql/query66.sql b/query/src/test/resources/query/sql/query66.sql
new file mode 100644
index 0000000..5efc8eb
--- /dev/null
+++ b/query/src/test/resources/query/sql/query66.sql
@@ -0,0 +1,2 @@
+select test_kylin_fact.cal_dt, max(test_kylin_fact.cal_dt) as mmm from test_kylin_fact left join test_cal_dt
+	on test_kylin_fact.cal_dt=test_cal_dt.cal_dt group by test_kylin_fact.cal_dt order by 2 desc limit 7
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/query/src/test/resources/query/sql/query67.sql
----------------------------------------------------------------------
diff --git a/query/src/test/resources/query/sql/query67.sql b/query/src/test/resources/query/sql/query67.sql
new file mode 100644
index 0000000..b50b518
--- /dev/null
+++ b/query/src/test/resources/query/sql/query67.sql
@@ -0,0 +1,4 @@
+SELECT (CASE "TEST_KYLIN_FACT"."LSTG_FORMAT_NAME" WHEN 'Auction' THEN '111' ELSE '222' END) AS "LSTG_FORMAT_NAME__group_",
+  SUM("TEST_KYLIN_FACT"."PRICE") AS "sum_PRICE_ok"
+FROM "TEST_KYLIN_FACT" "TEST_KYLIN_FACT"
+group by (CASE "TEST_KYLIN_FACT"."LSTG_FORMAT_NAME" WHEN 'Auction' THEN '111' ELSE '222' END)  ORDER BY 1 ASC
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/query/src/test/resources/query/sql/query68.sql
----------------------------------------------------------------------
diff --git a/query/src/test/resources/query/sql/query68.sql b/query/src/test/resources/query/sql/query68.sql
new file mode 100644
index 0000000..635c131
--- /dev/null
+++ b/query/src/test/resources/query/sql/query68.sql
@@ -0,0 +1,8 @@
+select cal_dt as bb , sum(price) as aa from test_kylin_fact fact
+inner join (
+
+
+select count(1) as cnt, min(cal_dt) as "mmm",  cal_dt as dt from test_kylin_fact group by cal_dt order by 2 desc limit 10
+
+
+) t0 on (fact.cal_dt = t0.dt) group by cal_dt
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/query/src/test/resources/query/sql/query69.sql
----------------------------------------------------------------------
diff --git a/query/src/test/resources/query/sql/query69.sql b/query/src/test/resources/query/sql/query69.sql
new file mode 100644
index 0000000..5491078
--- /dev/null
+++ b/query/src/test/resources/query/sql/query69.sql
@@ -0,0 +1,7 @@
+select lstg_format_name, 
+ sum(price) as GMV, 
+ count(1) as TRANS_CNT 
+ from test_kylin_fact 
+ where (CASE WHEN ("TEST_KYLIN_FACT"."LSTG_FORMAT_NAME" IN ('Auction', 'FP-GTC')) THEN 'Auction' ELSE "TEST_KYLIN_FACT"."LSTG_FORMAT_NAME" END) = 'Auction'
+ group by lstg_format_name 
+ order by sum(price)
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/query/src/test/resources/query/sql/query70.sql
----------------------------------------------------------------------
diff --git a/query/src/test/resources/query/sql/query70.sql b/query/src/test/resources/query/sql/query70.sql
new file mode 100644
index 0000000..413fb12
--- /dev/null
+++ b/query/src/test/resources/query/sql/query70.sql
@@ -0,0 +1,6 @@
+select lstg_format_name, 
+ sum(price) as GMV, 
+ count(1) as TRANS_CNT 
+ from test_kylin_fact 
+ where "TEST_KYLIN_FACT"."LSTG_FORMAT_NAME" = 'Auction' and (CASE WHEN ("TEST_KYLIN_FACT"."LSTG_FORMAT_NAME" IN ('Auction', 'FP-GTC')) THEN 'Auction' ELSE "TEST_KYLIN_FACT"."LSTG_FORMAT_NAME" END) = 'Auction'
+ group by lstg_format_name 

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/query/src/test/resources/query/sql/query71.sql
----------------------------------------------------------------------
diff --git a/query/src/test/resources/query/sql/query71.sql b/query/src/test/resources/query/sql/query71.sql
new file mode 100644
index 0000000..e6b69c9
--- /dev/null
+++ b/query/src/test/resources/query/sql/query71.sql
@@ -0,0 +1,6 @@
+select lstg_format_name, 
+ sum(price) as GMV, 
+ count(1) as TRANS_CNT 
+ from test_kylin_fact 
+ where "TEST_KYLIN_FACT"."LSTG_FORMAT_NAME" in ('Auction', 'ABIN') and (CASE WHEN ("TEST_KYLIN_FACT"."LSTG_FORMAT_NAME" IN ('Auction', 'FP-GTC')) THEN 'Auction' ELSE "TEST_KYLIN_FACT"."LSTG_FORMAT_NAME" END) = 'Auction'
+ group by lstg_format_name 
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/query/src/test/resources/query/sql/query72.sql
----------------------------------------------------------------------
diff --git a/query/src/test/resources/query/sql/query72.sql b/query/src/test/resources/query/sql/query72.sql
new file mode 100644
index 0000000..14ea2e9
--- /dev/null
+++ b/query/src/test/resources/query/sql/query72.sql
@@ -0,0 +1,6 @@
+select test_cal_dt.week_beg_dt, test_kylin_fact.lstg_format_name, sum(test_kylin_fact.price) as GMV, count(*) as TRANS_CNT 
+ from test_kylin_fact 
+ inner join test_cal_dt 
+ on test_kylin_fact.cal_dt = test_cal_dt.cal_dt
+ where test_kylin_fact.lstg_format_name='FP-GTC'  or extract(MONTH from test_cal_dt.week_beg_dt) = 12
+ group by test_cal_dt.week_beg_dt, test_kylin_fact.lstg_format_name
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/query/src/test/resources/query/sql/query73.sql
----------------------------------------------------------------------
diff --git a/query/src/test/resources/query/sql/query73.sql b/query/src/test/resources/query/sql/query73.sql
new file mode 100644
index 0000000..d5f1cbb
--- /dev/null
+++ b/query/src/test/resources/query/sql/query73.sql
@@ -0,0 +1 @@
+select min("t"."s") as min_pr from (select max(price) as "s" from test_kylin_fact group by lstg_format_name) "t"  having (count(1) > 0)
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/query/src/test/resources/query/sql/query74.sql
----------------------------------------------------------------------
diff --git a/query/src/test/resources/query/sql/query74.sql b/query/src/test/resources/query/sql/query74.sql
new file mode 100644
index 0000000..111804e
--- /dev/null
+++ b/query/src/test/resources/query/sql/query74.sql
@@ -0,0 +1 @@
+select min("t"."s") as min_x, max("t"."s") as max_x from (select max(price) as "s" from test_kylin_fact group by lstg_format_name) "t" having ( count(1)  > 0)
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/query/src/test/resources/query/sql/query75.sql
----------------------------------------------------------------------
diff --git a/query/src/test/resources/query/sql/query75.sql b/query/src/test/resources/query/sql/query75.sql
new file mode 100644
index 0000000..d47f65e
--- /dev/null
+++ b/query/src/test/resources/query/sql/query75.sql
@@ -0,0 +1,4 @@
+select lstg_format_name, sum(price) as GMV 
+ from test_kylin_fact 
+ where (NOT ((CASE WHEN (lstg_format_name IS NULL) THEN 1 WHEN NOT (lstg_format_name IS NULL) THEN 0 ELSE NULL END) <> 0))
+ group by lstg_format_name 

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/query/src/test/resources/query/sql/query76.sql
----------------------------------------------------------------------
diff --git a/query/src/test/resources/query/sql/query76.sql b/query/src/test/resources/query/sql/query76.sql
new file mode 100644
index 0000000..32b1582
--- /dev/null
+++ b/query/src/test/resources/query/sql/query76.sql
@@ -0,0 +1,4 @@
+select lstg_format_name, sum(price) as GMV 
+ from test_kylin_fact 
+ group by lstg_format_name, SLR_SEGMENT_CD
+  having SLR_SEGMENT_CD > 0

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/query/src/test/resources/query/sql_all_types/query00.sql
----------------------------------------------------------------------
diff --git a/query/src/test/resources/query/sql_all_types/query00.sql b/query/src/test/resources/query/sql_all_types/query00.sql
new file mode 100644
index 0000000..3e02614
--- /dev/null
+++ b/query/src/test/resources/query/sql_all_types/query00.sql
@@ -0,0 +1 @@
+select ML_tinyint, sum(price) as GMV from test_kylin_fact group by ML_tinyint order by ML_tinyint 

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/query/src/test/resources/query/sql_all_types/query01.sql
----------------------------------------------------------------------
diff --git a/query/src/test/resources/query/sql_all_types/query01.sql b/query/src/test/resources/query/sql_all_types/query01.sql
new file mode 100644
index 0000000..c3a5bef
--- /dev/null
+++ b/query/src/test/resources/query/sql_all_types/query01.sql
@@ -0,0 +1 @@
+select ML_smallint, sum(price) as GMV from test_kylin_fact group by ML_smallint order by ML_smallint 

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/query/src/test/resources/query/sql_all_types/query02.sql
----------------------------------------------------------------------
diff --git a/query/src/test/resources/query/sql_all_types/query02.sql b/query/src/test/resources/query/sql_all_types/query02.sql
new file mode 100644
index 0000000..710d991
--- /dev/null
+++ b/query/src/test/resources/query/sql_all_types/query02.sql
@@ -0,0 +1 @@
+select ML_int, sum(price) as GMV from test_kylin_fact group by ML_int order by ML_int 

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/query/src/test/resources/query/sql_all_types/query03.sql
----------------------------------------------------------------------
diff --git a/query/src/test/resources/query/sql_all_types/query03.sql b/query/src/test/resources/query/sql_all_types/query03.sql
new file mode 100644
index 0000000..34dd85e
--- /dev/null
+++ b/query/src/test/resources/query/sql_all_types/query03.sql
@@ -0,0 +1 @@
+select ML_bigint, sum(price) as GMV from test_kylin_fact group by ML_bigint order by ML_bigint 

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/query/src/test/resources/query/sql_all_types/query04.sql
----------------------------------------------------------------------
diff --git a/query/src/test/resources/query/sql_all_types/query04.sql b/query/src/test/resources/query/sql_all_types/query04.sql
new file mode 100644
index 0000000..23f9081
--- /dev/null
+++ b/query/src/test/resources/query/sql_all_types/query04.sql
@@ -0,0 +1 @@
+select ML_float, sum(price) as GMV from test_kylin_fact group by ML_float order by ML_float 

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/query/src/test/resources/query/sql_all_types/query05.sql
----------------------------------------------------------------------
diff --git a/query/src/test/resources/query/sql_all_types/query05.sql b/query/src/test/resources/query/sql_all_types/query05.sql
new file mode 100644
index 0000000..5614706
--- /dev/null
+++ b/query/src/test/resources/query/sql_all_types/query05.sql
@@ -0,0 +1 @@
+select ML_double, sum(price) as GMV from test_kylin_fact group by ML_double order by ML_double 

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/query/src/test/resources/query/sql_all_types/query06.sql
----------------------------------------------------------------------
diff --git a/query/src/test/resources/query/sql_all_types/query06.sql b/query/src/test/resources/query/sql_all_types/query06.sql
new file mode 100644
index 0000000..1f49d51
--- /dev/null
+++ b/query/src/test/resources/query/sql_all_types/query06.sql
@@ -0,0 +1 @@
+select ML_decimal, sum(price) as GMV from test_kylin_fact group by ML_decimal order by ML_decimal 

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/query/src/test/resources/query/sql_all_types/query07.sql
----------------------------------------------------------------------
diff --git a/query/src/test/resources/query/sql_all_types/query07.sql b/query/src/test/resources/query/sql_all_types/query07.sql
new file mode 100644
index 0000000..9a4b4af
--- /dev/null
+++ b/query/src/test/resources/query/sql_all_types/query07.sql
@@ -0,0 +1 @@
+select ML_boolean, sum(price) as GMV from test_kylin_fact group by ML_boolean order by ML_boolean 

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/query/src/test/resources/query/sql_all_types/query08.sql
----------------------------------------------------------------------
diff --git a/query/src/test/resources/query/sql_all_types/query08.sql b/query/src/test/resources/query/sql_all_types/query08.sql
new file mode 100644
index 0000000..dd8fef0
--- /dev/null
+++ b/query/src/test/resources/query/sql_all_types/query08.sql
@@ -0,0 +1 @@
+select ML_varchar, sum(price) as GMV from test_kylin_fact group by ML_varchar order by ML_varchar 

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/query/src/test/resources/query/sql_all_types/query09.sql
----------------------------------------------------------------------
diff --git a/query/src/test/resources/query/sql_all_types/query09.sql b/query/src/test/resources/query/sql_all_types/query09.sql
new file mode 100644
index 0000000..d2cc8d8
--- /dev/null
+++ b/query/src/test/resources/query/sql_all_types/query09.sql
@@ -0,0 +1 @@
+select ML_char, sum(price) as GMV from test_kylin_fact group by ML_char order by ML_char 

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/query/src/test/resources/query/sql_casewhen/query01.sql
----------------------------------------------------------------------
diff --git a/query/src/test/resources/query/sql_casewhen/query01.sql b/query/src/test/resources/query/sql_casewhen/query01.sql
new file mode 100644
index 0000000..825cad6
--- /dev/null
+++ b/query/src/test/resources/query/sql_casewhen/query01.sql
@@ -0,0 +1,8 @@
+SELECT lstg_format_name 
+FROM   test_kylin_fact 
+WHERE  ( NOT ( ( CASE 
+                   WHEN ( lstg_format_name IS NULL ) THEN 1
+                   WHEN NOT ( lstg_format_name IS NULL ) THEN 0 
+                   ELSE NULL 
+                 END ) <> 0 ) ) 
+GROUP  BY lstg_format_name 
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/query/src/test/resources/query/sql_casewhen/query02.sql
----------------------------------------------------------------------
diff --git a/query/src/test/resources/query/sql_casewhen/query02.sql b/query/src/test/resources/query/sql_casewhen/query02.sql
new file mode 100644
index 0000000..986fea1
--- /dev/null
+++ b/query/src/test/resources/query/sql_casewhen/query02.sql
@@ -0,0 +1,4 @@
+SELECT "TEST_KYLIN_FACT"."LSTG_FORMAT_NAME" 
+FROM   TEST_KYLIN_FACT 
+WHERE  ((CASE WHEN ("TEST_KYLIN_FACT"."LSTG_FORMAT_NAME" = 'Auction') THEN 'Auction1' ELSE "TEST_KYLIN_FACT"."LSTG_FORMAT_NAME" END) = 'Auction1')
+GROUP BY "TEST_KYLIN_FACT"."LSTG_FORMAT_NAME" 
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/query/src/test/resources/query/sql_casewhen/query03.sql
----------------------------------------------------------------------
diff --git a/query/src/test/resources/query/sql_casewhen/query03.sql b/query/src/test/resources/query/sql_casewhen/query03.sql
new file mode 100644
index 0000000..2c3881f
--- /dev/null
+++ b/query/src/test/resources/query/sql_casewhen/query03.sql
@@ -0,0 +1,7 @@
+SELECT "TEST_KYLIN_FACT"."LSTG_FORMAT_NAME" 
+FROM   TEST_KYLIN_FACT 
+WHERE  (CASE "TEST_KYLIN_FACT"."LSTG_FORMAT_NAME" 
+	WHEN 'Auction' THEN '111' 
+	WHEN 'FP-GTC' THEN '222' 
+	ELSE '999' END) = '111'
+GROUP BY "TEST_KYLIN_FACT"."LSTG_FORMAT_NAME" 
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/query/src/test/resources/query/sql_casewhen/query55.sql.disable
----------------------------------------------------------------------
diff --git a/query/src/test/resources/query/sql_casewhen/query55.sql.disable b/query/src/test/resources/query/sql_casewhen/query55.sql.disable
new file mode 100644
index 0000000..ea73cd8
--- /dev/null
+++ b/query/src/test/resources/query/sql_casewhen/query55.sql.disable
@@ -0,0 +1,4 @@
+SELECT (CASE WHEN ("TEST_KYLIN_FACT"."LSTG_FORMAT_NAME" IN ('Auction', 'FP-GTC')) THEN 'Auction' ELSE "TEST_KYLIN_FACT"."LSTG_FORMAT_NAME" END) AS "LSTG_FORMAT_NAME__group_",
+  SUM("TEST_KYLIN_FACT"."PRICE") AS "sum_PRICE_ok"
+FROM "TEST_KYLIN_FACT" "TEST_KYLIN_FACT"
+GROUP BY (CASE WHEN ("TEST_KYLIN_FACT"."LSTG_FORMAT_NAME" IN ('Auction', 'FP-GTC')) THEN 'Auction' ELSE "TEST_KYLIN_FACT"."LSTG_FORMAT_NAME" END)
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/query/src/test/resources/query/sql_casewhen/query56.sql
----------------------------------------------------------------------
diff --git a/query/src/test/resources/query/sql_casewhen/query56.sql b/query/src/test/resources/query/sql_casewhen/query56.sql
new file mode 100644
index 0000000..07f5db6
--- /dev/null
+++ b/query/src/test/resources/query/sql_casewhen/query56.sql
@@ -0,0 +1,4 @@
+ SELECT (CASE WHEN ("TEST_KYLIN_FACT"."LSTG_FORMAT_NAME" = 'Auction') THEN 'Auction2' ELSE "TEST_KYLIN_FACT"."LSTG_FORMAT_NAME" END) AS "LSTG_FORMAT_NAME__group_",
+  SUM("TEST_KYLIN_FACT"."PRICE") AS "sum_PRICE_ok"
+FROM "TEST_KYLIN_FACT" "TEST_KYLIN_FACT"
+GROUP BY (CASE WHEN ("TEST_KYLIN_FACT"."LSTG_FORMAT_NAME" =  'Auction') THEN 'Auction2' ELSE "TEST_KYLIN_FACT"."LSTG_FORMAT_NAME" END)
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/query/src/test/resources/query/sql_derived/query01.sql
----------------------------------------------------------------------
diff --git a/query/src/test/resources/query/sql_derived/query01.sql b/query/src/test/resources/query/sql_derived/query01.sql
new file mode 100644
index 0000000..c471835
--- /dev/null
+++ b/query/src/test/resources/query/sql_derived/query01.sql
@@ -0,0 +1,29 @@
+SELECT 
+ test_kylin_fact.seller_id 
+ ,test_cal_dt.week_beg_dt 
+ ,test_category_groupings.meta_categ_name 
+ ,test_category_groupings.categ_lvl2_name 
+ ,test_category_groupings.categ_lvl3_name 
+ ,test_kylin_fact.lstg_format_name 
+ ,test_sites.site_name 
+ ,test_sites.site_id 
+ ,test_sites.cre_user 
+ ,sum(test_kylin_fact.price) as GMV, count(*) as TRANS_CNT 
+ FROM test_kylin_fact 
+ inner JOIN test_cal_dt 
+ ON test_kylin_fact.cal_dt = test_cal_dt.cal_dt 
+ inner JOIN test_category_groupings 
+ ON test_kylin_fact.leaf_categ_id = test_category_groupings.leaf_categ_id AND test_kylin_fact.lstg_site_id = test_category_groupings.site_id 
+ inner JOIN test_sites 
+ ON test_kylin_fact.lstg_site_id = test_sites.site_id 
+ where test_kylin_fact.seller_id = 10000002 
+ group by 
+ test_kylin_fact.seller_id 
+ ,test_cal_dt.week_beg_dt 
+ ,test_category_groupings.meta_categ_name 
+ ,test_category_groupings.categ_lvl2_name 
+ ,test_category_groupings.categ_lvl3_name 
+ ,test_kylin_fact.lstg_format_name 
+ ,test_sites.site_name 
+ ,test_sites.site_id 
+ ,test_sites.cre_user 

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/query/src/test/resources/query/sql_derived/query02.sql
----------------------------------------------------------------------
diff --git a/query/src/test/resources/query/sql_derived/query02.sql b/query/src/test/resources/query/sql_derived/query02.sql
new file mode 100644
index 0000000..6c8b2da
--- /dev/null
+++ b/query/src/test/resources/query/sql_derived/query02.sql
@@ -0,0 +1,29 @@
+SELECT 
+ test_kylin_fact.seller_id 
+ ,test_cal_dt.week_beg_dt 
+ ,test_category_groupings.meta_categ_name 
+ ,test_category_groupings.categ_lvl2_name 
+ ,test_category_groupings.categ_lvl3_name 
+ ,test_kylin_fact.lstg_format_name 
+ ,test_sites.site_name 
+ ,test_sites.site_id 
+ ,test_sites.cre_user 
+ ,sum(test_kylin_fact.price) as GMV, count(*) as TRANS_CNT 
+ FROM test_kylin_fact 
+ inner JOIN test_cal_dt 
+ ON test_kylin_fact.cal_dt = test_cal_dt.cal_dt 
+ inner JOIN test_category_groupings 
+ ON test_kylin_fact.leaf_categ_id = test_category_groupings.leaf_categ_id AND test_kylin_fact.lstg_site_id = test_category_groupings.site_id 
+ inner JOIN test_sites 
+ ON test_kylin_fact.lstg_site_id = test_sites.site_id 
+ where test_kylin_fact.seller_id = 10000002 and test_sites.site_id=0 
+ group by 
+ test_kylin_fact.seller_id 
+ ,test_cal_dt.week_beg_dt 
+ ,test_category_groupings.meta_categ_name 
+ ,test_category_groupings.categ_lvl2_name 
+ ,test_category_groupings.categ_lvl3_name 
+ ,test_kylin_fact.lstg_format_name 
+ ,test_sites.site_name 
+ ,test_sites.site_id 
+ ,test_sites.cre_user 

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/query/src/test/resources/query/sql_derived/query03.sql
----------------------------------------------------------------------
diff --git a/query/src/test/resources/query/sql_derived/query03.sql b/query/src/test/resources/query/sql_derived/query03.sql
new file mode 100644
index 0000000..8c7938a
--- /dev/null
+++ b/query/src/test/resources/query/sql_derived/query03.sql
@@ -0,0 +1,3 @@
+SELECT distinct LSTG_FORMAT_NAME from test_kylin_fact 
+ 
+ 

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/query/src/test/resources/query/sql_derived/query04.sql
----------------------------------------------------------------------
diff --git a/query/src/test/resources/query/sql_derived/query04.sql b/query/src/test/resources/query/sql_derived/query04.sql
new file mode 100644
index 0000000..8ab1f88
--- /dev/null
+++ b/query/src/test/resources/query/sql_derived/query04.sql
@@ -0,0 +1,2 @@
+SELECT site_id, site_name, cre_user from test_sites 
+ 

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/query/src/test/resources/query/sql_derived/query05.sql
----------------------------------------------------------------------
diff --git a/query/src/test/resources/query/sql_derived/query05.sql b/query/src/test/resources/query/sql_derived/query05.sql
new file mode 100644
index 0000000..9318e05
--- /dev/null
+++ b/query/src/test/resources/query/sql_derived/query05.sql
@@ -0,0 +1,5 @@
+SELECT test_sites.site_name, test_kylin_fact.lstg_format_name, sum(test_kylin_fact.price) as GMV, count(*) as TRANS_CNT 
+ FROM test_kylin_fact 
+ inner JOIN test_sites ON test_kylin_fact.lstg_site_id = test_sites.site_id 
+ GROUP BY 
+ test_sites.site_name, test_kylin_fact.lstg_format_name 

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/query/src/test/resources/query/sql_derived/query06.sql
----------------------------------------------------------------------
diff --git a/query/src/test/resources/query/sql_derived/query06.sql b/query/src/test/resources/query/sql_derived/query06.sql
new file mode 100644
index 0000000..2d70f45
--- /dev/null
+++ b/query/src/test/resources/query/sql_derived/query06.sql
@@ -0,0 +1,2 @@
+SELECT distinct cal_dt from test_kylin_fact 
+ 

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/query/src/test/resources/query/sql_derived/query07.sql
----------------------------------------------------------------------
diff --git a/query/src/test/resources/query/sql_derived/query07.sql b/query/src/test/resources/query/sql_derived/query07.sql
new file mode 100644
index 0000000..e3ef635
--- /dev/null
+++ b/query/src/test/resources/query/sql_derived/query07.sql
@@ -0,0 +1,4 @@
+select cal_dt, lstg_format_name, sum(price) as GMV 
+ from test_kylin_fact 
+ where cal_dt=date '2013-05-06' 
+ group by cal_dt, lstg_format_name 

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/query/src/test/resources/query/sql_derived/query08.sql.disabled
----------------------------------------------------------------------
diff --git a/query/src/test/resources/query/sql_derived/query08.sql.disabled b/query/src/test/resources/query/sql_derived/query08.sql.disabled
new file mode 100644
index 0000000..17cdc4e
--- /dev/null
+++ b/query/src/test/resources/query/sql_derived/query08.sql.disabled
@@ -0,0 +1 @@
+select distinct leaf_categ_id, lstg_site_id from test_kylin_fact 

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/query/src/test/resources/query/sql_derived/query09.sql
----------------------------------------------------------------------
diff --git a/query/src/test/resources/query/sql_derived/query09.sql b/query/src/test/resources/query/sql_derived/query09.sql
new file mode 100644
index 0000000..0ecbb5b
--- /dev/null
+++ b/query/src/test/resources/query/sql_derived/query09.sql
@@ -0,0 +1 @@
+select distinct leaf_categ_id, site_id from test_category_groupings 

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/query/src/test/resources/query/sql_derived/query10.sql
----------------------------------------------------------------------
diff --git a/query/src/test/resources/query/sql_derived/query10.sql b/query/src/test/resources/query/sql_derived/query10.sql
new file mode 100644
index 0000000..5505b67
--- /dev/null
+++ b/query/src/test/resources/query/sql_derived/query10.sql
@@ -0,0 +1,26 @@
+SELECT 
+ test_cal_dt.week_beg_dt 
+ ,test_category_groupings.meta_categ_name 
+ ,test_category_groupings.upd_user 
+ ,test_category_groupings.upd_date 
+ ,test_kylin_fact.leaf_categ_id 
+ ,test_category_groupings.leaf_categ_id 
+ ,test_kylin_fact.lstg_site_id 
+ ,test_category_groupings.site_id 
+ ,sum(price) as GMV, count(*) as TRANS_CNT 
+ FROM test_kylin_fact 
+ inner JOIN test_cal_dt 
+ ON test_kylin_fact.cal_dt = test_cal_dt.cal_dt 
+ inner JOIN test_category_groupings 
+ ON test_kylin_fact.leaf_categ_id = test_category_groupings.leaf_categ_id 
+ AND test_kylin_fact.lstg_site_id = test_category_groupings.site_id 
+ where 
+ test_category_groupings.upd_date='2012-09-11 20:26:04' 
+ group by test_cal_dt.week_beg_dt 
+ ,test_category_groupings.meta_categ_name 
+ ,test_category_groupings.upd_user 
+ ,test_category_groupings.upd_date 
+ ,test_kylin_fact.leaf_categ_id 
+ ,test_category_groupings.leaf_categ_id 
+ ,test_kylin_fact.lstg_site_id 
+ ,test_category_groupings.site_id 

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/query/src/test/resources/query/sql_distinct/query00.sql
----------------------------------------------------------------------
diff --git a/query/src/test/resources/query/sql_distinct/query00.sql b/query/src/test/resources/query/sql_distinct/query00.sql
new file mode 100644
index 0000000..021caf4
--- /dev/null
+++ b/query/src/test/resources/query/sql_distinct/query00.sql
@@ -0,0 +1,6 @@
+select lstg_format_name, cal_dt,
+ sum(price) as GMV, 
+ count(1) as TRANS_CNT, 
+ count(distinct seller_id) as DIST_SELLER 
+ from test_kylin_fact 
+ group by lstg_format_name, cal_dt

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/query/src/test/resources/query/sql_distinct/query01.sql
----------------------------------------------------------------------
diff --git a/query/src/test/resources/query/sql_distinct/query01.sql b/query/src/test/resources/query/sql_distinct/query01.sql
new file mode 100644
index 0000000..d57ef11
--- /dev/null
+++ b/query/src/test/resources/query/sql_distinct/query01.sql
@@ -0,0 +1,7 @@
+select lstg_format_name, 
+ sum(price) as GMV, 
+ count(1) as TRANS_CNT, 
+ count(distinct seller_id) as DIST_SELLER 
+ from test_kylin_fact 
+ where lstg_format_name='FP-GTC' 
+ group by lstg_format_name 

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/query/src/test/resources/query/sql_distinct/query02.sql
----------------------------------------------------------------------
diff --git a/query/src/test/resources/query/sql_distinct/query02.sql b/query/src/test/resources/query/sql_distinct/query02.sql
new file mode 100644
index 0000000..3c5589c
--- /dev/null
+++ b/query/src/test/resources/query/sql_distinct/query02.sql
@@ -0,0 +1,8 @@
+select lstg_format_name, 
+ sum(price) as GMV, 
+ count(1) as TRANS_CNT, 
+ count(distinct seller_id) as DIST_SELLER 
+ from test_kylin_fact 
+ where lstg_format_name='FP-GTC' 
+ group by lstg_format_name 
+ having count(distinct seller_id) > 50 

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/query/src/test/resources/query/sql_distinct/query03.sql
----------------------------------------------------------------------
diff --git a/query/src/test/resources/query/sql_distinct/query03.sql b/query/src/test/resources/query/sql_distinct/query03.sql
new file mode 100644
index 0000000..e3c91f1
--- /dev/null
+++ b/query/src/test/resources/query/sql_distinct/query03.sql
@@ -0,0 +1,8 @@
+select test_cal_dt.week_beg_dt,sum(test_kylin_fact.price) as GMV 
+ , count(1) as TRANS_CNT, count(distinct seller_id) as DIST_SELLER 
+ from test_kylin_fact 
+ inner JOIN test_cal_dt 
+ ON test_kylin_fact.cal_dt = test_cal_dt.cal_dt 
+ where test_kylin_fact.lstg_format_name='FP-GTC' 
+ and test_cal_dt.week_beg_dt between DATE '2013-05-01' and DATE '2013-08-01' 
+ group by test_cal_dt.week_beg_dt 

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/query/src/test/resources/query/sql_distinct/query04.sql
----------------------------------------------------------------------
diff --git a/query/src/test/resources/query/sql_distinct/query04.sql b/query/src/test/resources/query/sql_distinct/query04.sql
new file mode 100644
index 0000000..3bc7c27
--- /dev/null
+++ b/query/src/test/resources/query/sql_distinct/query04.sql
@@ -0,0 +1,9 @@
+select test_cal_dt.week_beg_dt,sum(test_kylin_fact.price) as GMV 
+ , count(1) as TRANS_CNT, count(distinct seller_id) as DIST_SELLER 
+ from test_kylin_fact 
+ inner JOIN test_cal_dt 
+ ON test_kylin_fact.cal_dt = test_cal_dt.cal_dt 
+ where test_kylin_fact.lstg_format_name='FP-GTC' 
+ and test_cal_dt.week_beg_dt between DATE '2013-05-01' and DATE '2013-08-01' 
+ group by test_cal_dt.week_beg_dt 
+ having count(distinct seller_id) > 2 

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/query/src/test/resources/query/sql_distinct/query05.sql
----------------------------------------------------------------------
diff --git a/query/src/test/resources/query/sql_distinct/query05.sql b/query/src/test/resources/query/sql_distinct/query05.sql
new file mode 100644
index 0000000..4744c4b
--- /dev/null
+++ b/query/src/test/resources/query/sql_distinct/query05.sql
@@ -0,0 +1,7 @@
+select lstg_format_name, 
+ sum(price) as GMV, 
+ count(1) as TRANS_CNT, 
+ count(distinct seller_id) as DIST_SELLER 
+ from test_kylin_fact 
+ group by lstg_format_name 
+ order by lstg_format_name 

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/query/src/test/resources/query/sql_distinct/query06.sql
----------------------------------------------------------------------
diff --git a/query/src/test/resources/query/sql_distinct/query06.sql b/query/src/test/resources/query/sql_distinct/query06.sql
new file mode 100644
index 0000000..b4ca929
--- /dev/null
+++ b/query/src/test/resources/query/sql_distinct/query06.sql
@@ -0,0 +1,8 @@
+select lstg_format_name, 
+ sum(price) as GMV, 
+ count(1) as TRANS_CNT, 
+ count(distinct seller_id) as DIST_SELLER 
+ from test_kylin_fact 
+ where lstg_format_name='FP-GTC' 
+ group by lstg_format_name 
+ order by lstg_format_name 

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/query/src/test/resources/query/sql_distinct/query07.sql
----------------------------------------------------------------------
diff --git a/query/src/test/resources/query/sql_distinct/query07.sql b/query/src/test/resources/query/sql_distinct/query07.sql
new file mode 100644
index 0000000..1f48c52
--- /dev/null
+++ b/query/src/test/resources/query/sql_distinct/query07.sql
@@ -0,0 +1,6 @@
+select lstg_format_name, 
+ sum(price) as GMV, 
+ count(1) as TRANS_CNT, 
+ count(distinct seller_id) as DIST_SELLER 
+ from test_kylin_fact 
+ group by lstg_format_name 

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/query/src/test/resources/query/sql_dynamic/query01.dat
----------------------------------------------------------------------
diff --git a/query/src/test/resources/query/sql_dynamic/query01.dat b/query/src/test/resources/query/sql_dynamic/query01.dat
new file mode 100644
index 0000000..e810421
--- /dev/null
+++ b/query/src/test/resources/query/sql_dynamic/query01.dat
@@ -0,0 +1,2 @@
+FP-GTC
+Collectibles
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/query/src/test/resources/query/sql_dynamic/query01.sql
----------------------------------------------------------------------
diff --git a/query/src/test/resources/query/sql_dynamic/query01.sql b/query/src/test/resources/query/sql_dynamic/query01.sql
new file mode 100644
index 0000000..f481edf
--- /dev/null
+++ b/query/src/test/resources/query/sql_dynamic/query01.sql
@@ -0,0 +1,10 @@
+select test_cal_dt.week_beg_dt, test_kylin_fact.lstg_format_name, test_category_groupings.meta_categ_name, sum(test_kylin_fact.price) as gmv, count(*) as trans_cnt 
+ from test_kylin_fact 
+ inner JOIN test_cal_dt 
+ ON test_kylin_fact.cal_dt = test_cal_dt.cal_dt 
+ inner JOIN test_category_groupings 
+ ON test_kylin_fact.leaf_categ_id = test_category_groupings.leaf_categ_id AND test_kylin_fact.lstg_site_id = test_category_groupings.site_id 
+ where test_kylin_fact.lstg_format_name = ? 
+ and test_category_groupings.meta_categ_name = ? 
+ and test_cal_dt.week_beg_dt between DATE '2013-05-01' and DATE '2013-10-01' 
+ group by test_cal_dt.week_beg_dt, test_kylin_fact.lstg_format_name, test_category_groupings.meta_categ_name 

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/query/src/test/resources/query/sql_dynamic/query02.dat
----------------------------------------------------------------------
diff --git a/query/src/test/resources/query/sql_dynamic/query02.dat b/query/src/test/resources/query/sql_dynamic/query02.dat
new file mode 100644
index 0000000..e810421
--- /dev/null
+++ b/query/src/test/resources/query/sql_dynamic/query02.dat
@@ -0,0 +1,2 @@
+FP-GTC
+Collectibles
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/query/src/test/resources/query/sql_dynamic/query02.sql
----------------------------------------------------------------------
diff --git a/query/src/test/resources/query/sql_dynamic/query02.sql b/query/src/test/resources/query/sql_dynamic/query02.sql
new file mode 100644
index 0000000..701e6e4
--- /dev/null
+++ b/query/src/test/resources/query/sql_dynamic/query02.sql
@@ -0,0 +1,12 @@
+select sum(1) as "col" from ( 
+ select test_cal_dt.week_beg_dt, test_kylin_fact.lstg_format_name, test_category_groupings.meta_categ_name, sum(test_kylin_fact.price) as gmv, count(*) as trans_cnt 
+ from test_kylin_fact 
+ inner JOIN test_cal_dt 
+ ON test_kylin_fact.cal_dt = test_cal_dt.cal_dt 
+ inner JOIN test_category_groupings 
+ ON test_kylin_fact.leaf_categ_id = test_category_groupings.leaf_categ_id AND test_kylin_fact.lstg_site_id = test_category_groupings.site_id 
+ where test_kylin_fact.lstg_format_name = ? 
+ and test_category_groupings.meta_categ_name = ? 
+ and test_cal_dt.week_beg_dt between DATE '2013-05-01' and DATE '2013-10-01' 
+ group by test_cal_dt.week_beg_dt, test_kylin_fact.lstg_format_name, test_category_groupings.meta_categ_name 
+ ) "tableausql" having count(1)>0 

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/query/src/test/resources/query/sql_hive/query01.sql
----------------------------------------------------------------------
diff --git a/query/src/test/resources/query/sql_hive/query01.sql b/query/src/test/resources/query/sql_hive/query01.sql
new file mode 100644
index 0000000..322c260
--- /dev/null
+++ b/query/src/test/resources/query/sql_hive/query01.sql
@@ -0,0 +1,2 @@
+select cal_dt, sum(price)as GMV, count(1) as trans_cnt from test_kylin_fact 
+ group by cal_dt 

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/query/src/test/resources/query/sql_hive/query10.sql
----------------------------------------------------------------------
diff --git a/query/src/test/resources/query/sql_hive/query10.sql b/query/src/test/resources/query/sql_hive/query10.sql
new file mode 100644
index 0000000..8379174
--- /dev/null
+++ b/query/src/test/resources/query/sql_hive/query10.sql
@@ -0,0 +1,8 @@
+select test_cal_dt.QTR_BEG_DT,sum(test_kylin_fact.price) as gmv 
+ , count(*) as trans_cnt 
+ from test_kylin_fact 
+ inner JOIN test_cal_dt 
+ ON test_kylin_fact.cal_dt = test_cal_dt.cal_dt 
+ where test_kylin_fact.lstg_format_name='FP-GTC' 
+ and test_cal_dt.week_beg_dt between '2013-05-01' and '2013-08-01' 
+ group by test_cal_dt.QTR_BEG_DT 

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/query/src/test/resources/query/sql_invalid/query29_invalid_SQL.sql
----------------------------------------------------------------------
diff --git a/query/src/test/resources/query/sql_invalid/query29_invalid_SQL.sql b/query/src/test/resources/query/sql_invalid/query29_invalid_SQL.sql
new file mode 100644
index 0000000..7672a93
--- /dev/null
+++ b/query/src/test/resources/query/sql_invalid/query29_invalid_SQL.sql
@@ -0,0 +1,37 @@
+SELECT 
+ test_cal_dt.week_beg_dt_test 
+ ,test_cal_dt.retail_year 
+ ,test_cal_dt.rtl_month_of_rtl_year_id 
+ ,test_cal_dt.retail_week 
+ ,test_category_groupings.meta_categ_name 
+ ,test_category_groupings.categ_lvl2_name 
+ ,test_category_groupings.categ_lvl3_name 
+ ,test_kylin_fact.lstg_format_name 
+ ,test_sites.site_name 
+ ,test_seller_type_dim.seller_type_desc 
+ ,sum(test_kylin_fact.price) as gmv 
+ , count(*) as trans_cnt 
+ FROM test_kylin_fact 
+ inner JOIN test_cal_dt 
+ ON test_kylin_fact.cal_dt = test_cal_dt.cal_dt 
+ inner JOIN test_category_groupings 
+ ON test_kylin_fact.leaf_categ_id = test_category_groupings.leaf_categ_id AND test_kylin_fact.lstg_site_id = test_category_groupings.site_id 
+ inner JOIN test_sites 
+ ON test_kylin_fact.lstg_site_id = test_sites.site_id 
+ inner JOIN test_seller_type_dim 
+ ON test_kylin_fact.slr_segment_cd = test_seller_type_dim.seller_type_cd 
+ where test_cal_dt.retail_year='2013' 
+ and retail_week in(1,2,3,4,5,6,7,7,7) 
+ and (test_category_groupings.meta_categ_name='Collectibles' or test_category_groupings.categ_lvl3_name='Dresses') 
+ and test_sites.site_name='Ebay' 
+ and test_cal_dt.retail_year not in ('2014') 
+ group by test_cal_dt.week_beg_dt 
+ ,test_cal_dt.retail_year 
+ ,test_cal_dt.rtl_month_of_rtl_year_id 
+ ,test_cal_dt.retail_week 
+ ,test_category_groupings.meta_categ_name 
+ ,test_category_groupings.categ_lvl2_name 
+ ,test_category_groupings.categ_lvl3_name 
+ ,test_kylin_fact.lstg_format_name 
+ ,test_sites.site_name 
+ ,test_seller_type_dim.seller_type_desc 

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/query/src/test/resources/query/sql_invalid/query30_invalid_SQL.sql
----------------------------------------------------------------------
diff --git a/query/src/test/resources/query/sql_invalid/query30_invalid_SQL.sql b/query/src/test/resources/query/sql_invalid/query30_invalid_SQL.sql
new file mode 100644
index 0000000..decdc76
--- /dev/null
+++ b/query/src/test/resources/query/sql_invalid/query30_invalid_SQL.sql
@@ -0,0 +1,36 @@
+SELECT 
+ test_cal_dt.week_beg_dt_test 
+ ,test_cal_dt.retail_year 
+ ,test_cal_dt.rtl_month_of_rtl_year_id 
+ ,test_cal_dt.retail_week 
+ ,test_category_groupings.meta_categ_name 
+ ,test_category_groupings.categ_lvl2_name 
+ ,test_category_groupings.categ_lvl3_name 
+ ,test_kylin_fact.lstg_format_name 
+ ,test_sites.site_name 
+ ,test_seller_type_dim.seller_type_desc 
+ ,sum(test_kylin_fact.price) as gmv 
+ , count(*) as trans_cnt 
+ FROM test_kylin_fact 
+ inner JOIN test_cal_dt 
+ ON test_kylin_fact.cal_dt = test_cal_dt.cal_dt 
+ inner JOIN test_category_groupings 
+ ON test_kylin_fact.leaf_categ_id = test_category_groupings.leaf_categ_id AND test_kylin_fact.lstg_site_id = test_category_groupings.site_id 
+ inner JOIN test_sites 
+ ON test_kylin_fact.lstg_site_id = test_sites.site_id 
+ inner JOIN test_seller_type_dim 
+ ON test_kylin_fact.slr_segment_cd = test_seller_type_dim.seller_type_cd 
+ where test_cal_dt.retail_year='2013' 
+ and retail_week in(1,2,3,4,5,6,7,7,7) 
+ and (test_category_groupings.meta_categ_name='Collectibles' or test_category_groupings.categ_lvl3_name='Dresses') 
+ and test_sites.site_name='Ebay' 
+ and test_cal_dt.retail_year not in ('2014') 
+ group by test_cal_dt.week_beg_dt 
+ ,test_cal_dt.retail_year 
+ ,test_cal_dt.rtl_month_of_rtl_year_id 
+ ,test_cal_dt.retail_week 
+ ,test_category_groupings.meta_categ_name 
+ ,test_category_groupings.categ_lvl2_name 
+ ,test_category_groupings.categ_lvl3_name 
+ ,test_kylin_fact.lstg_format_name 
+ ,test_sites.site_name 

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/query/src/test/resources/query/sql_invalid/query31_invalid_SQL.sql
----------------------------------------------------------------------
diff --git a/query/src/test/resources/query/sql_invalid/query31_invalid_SQL.sql b/query/src/test/resources/query/sql_invalid/query31_invalid_SQL.sql
new file mode 100644
index 0000000..e479f78
--- /dev/null
+++ b/query/src/test/resources/query/sql_invalid/query31_invalid_SQL.sql
@@ -0,0 +1,37 @@
+SELECT 
+ test_cal_dt.week_beg_dt_test 
+ ,test_cal_dt.retail_year 
+ ,test_cal_dt.rtl_month_of_rtl_year_id 
+ ,test_cal_dt.retail_week 
+ ,test_category_groupings.meta_categ_name 
+ ,test_category_groupings.categ_lvl2_name 
+ ,test_category_groupings.categ_lvl3_name 
+ ,test_kylin_fact.lstg_format_name 
+ ,test_sites.site_name 
+ ,test_seller_type_dim.seller_type_desc 
+ ,sum(test_kylin_fact.price_amt) as gmv 
+ , count(*) as trans_cnt 
+ FROM test_kylin_fact 
+ inner JOIN test_cal_dt 
+ ON test_kylin_fact.cal_dt = test_cal_dt.cal_dt 
+ inner JOIN test_category_groupings 
+ ON test_kylin_fact.leaf_categ_id = test_category_groupings.leaf_categ_id AND test_kylin_fact.lstg_site_id = test_category_groupings.site_id 
+ inner JOIN test_sites 
+ ON test_kylin_fact.lstg_site_id = test_sites.site_id 
+ inner JOIN test_seller_type_dim 
+ ON test_kylin_fact.slr_segment_cd = test_seller_type_dim.seller_type_cd 
+ where test_cal_dt.retail_year='2013' 
+ and retail_week in(1,2,3,4,5,6,7,7,7) 
+ and (test_category_groupings.meta_categ_name='Collectibles' or test_category_groupings.categ_lvl3_name='Dresses') 
+ and test_sites.site_name='Ebay' 
+ and test_cal_dt.retail_year not in ('2014') 
+ group by test_cal_dt.week_beg_dt 
+ ,test_cal_dt.retail_year 
+ ,test_cal_dt.rtl_month_of_rtl_year_id 
+ ,test_cal_dt.retail_week 
+ ,test_category_groupings.meta_categ_name 
+ ,test_category_groupings.categ_lvl2_name 
+ ,test_category_groupings.categ_lvl3_name 
+ ,test_kylin_fact.lstg_format_name 
+ ,test_sites.site_name 
+ ,test_seller_type_dim.seller_type_desc 

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/query/src/test/resources/query/sql_invalid/query32_invalid_SQL.sql
----------------------------------------------------------------------
diff --git a/query/src/test/resources/query/sql_invalid/query32_invalid_SQL.sql b/query/src/test/resources/query/sql_invalid/query32_invalid_SQL.sql
new file mode 100644
index 0000000..e8cb459
--- /dev/null
+++ b/query/src/test/resources/query/sql_invalid/query32_invalid_SQL.sql
@@ -0,0 +1,38 @@
+SELECT 
+ test_cal_dt.week_beg_dt_test 
+ ,test_cal_dt.retail_year 
+ ,test_cal_dt.rtl_month_of_rtl_year_id 
+ ,test_cal_dt.retail_week 
+ ,test_category_groupings.meta_categ_name 
+ ,test_category_groupings.categ_lvl2_name 
+ ,test_category_groupings.categ_lvl3_name 
+ ,test_kylin_fact.lstg_format_name 
+ ,test_sites.site_name 
+ ,test_seller_type_dim.seller_type_desc 
+ ,sum(test_kylin_fact.price) as gmv 
+ , count(*) as trans_cnt 
+ FROM test_kylin_fact 
+ inner JOIN test_cal_dt 
+ ON test_kylin_fact.cal_dt = test_cal_dt.cal_dt 
+ inner JOIN test_category_groupings 
+ ON test_kylin_fact.leaf_categ_id = test_category_groupings.leaf_categ_id AND test_kylin_fact.lstg_site_id = test_category_groupings.site_id 
+ inner JOIN test_sites 
+ ON test_kylin_fact.lstg_site_id = test_sites.site_id 
+ inner JOIN test_seller_type_dim 
+ ON test_kylin_fact.slr_segment_cd = test_seller_type_dim.seller_type_cd 
+ where test_cal_dt.retail_year='2013' 
+ and retail_week in(1,2,3,4,5,6,7,7,7) 
+ and (test_category_groupings.meta_categ_name='Collectibles' or test_category_groupings.categ_lvl3_name='Dresses') 
+ and test_sites.site_name='Ebay' 
+ and test_cal_dt.retail_year not in ('2014') 
+ and test_kylin_fact.price<100 
+ group by test_cal_dt.week_beg_dt 
+ ,test_cal_dt.retail_year 
+ ,test_cal_dt.rtl_month_of_rtl_year_id 
+ ,test_cal_dt.retail_week 
+ ,test_category_groupings.meta_categ_name 
+ ,test_category_groupings.categ_lvl2_name 
+ ,test_category_groupings.categ_lvl3_name 
+ ,test_kylin_fact.lstg_format_name 
+ ,test_sites.site_name 
+ ,test_seller_type_dim.seller_type_desc 

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/query/src/test/resources/query/sql_invalid/query33_invalid_SQL.sql
----------------------------------------------------------------------
diff --git a/query/src/test/resources/query/sql_invalid/query33_invalid_SQL.sql b/query/src/test/resources/query/sql_invalid/query33_invalid_SQL.sql
new file mode 100644
index 0000000..0202dce
--- /dev/null
+++ b/query/src/test/resources/query/sql_invalid/query33_invalid_SQL.sql
@@ -0,0 +1,38 @@
+SELECT 
+ test_cal_dt.week_beg_dt_test 
+ ,test_cal_dt.retail_year 
+ ,test_cal_dt.rtl_month_of_rtl_year_id 
+ ,test_cal_dt.retail_week 
+ ,test_category_groupings.meta_categ_name 
+ ,test_category_groupings.categ_lvl2_name 
+ ,test_category_groupings.categ_lvl3_name 
+ ,test_kylin_fact.lstg_format_name 
+ ,test_sites.site_name 
+ ,test_seller_type_dim.seller_type_desc 
+ ,sum(test_kylin_fact.price) as gmv 
+ , count(*) as trans_cnt 
+ FROM test_kylin_fact 
+ inner JOIN test_cal_dt 
+ ON test_kylin_fact.cal_dt = test_cal_dt.cal_dt 
+ inner JOIN test_category_groupings 
+ ON test_kylin_fact.leaf_categ_id = test_category_groupings.leaf_categ_id AND test_kylin_fact.lstg_site_id = test_category_groupings.site_id 
+ inner JOIN test_sites 
+ ON test_kylin_fact.lstg_site_id = test_sites.site_id 
+ inner JOIN test_seller_type_dim 
+ ON test_kylin_fact.slr_segment_cd = test_seller_type_dim.seller_type_cd 
+ where test_cal_dt.retail_year='2013' 
+ and retail_week in(1,2,3,4,5,6,7,7,7) 
+ and (test_category_groupings.meta_categ_name='Collectibles' or test_category_groupings.categ_lvl3_name='Dresses') 
+ and test_sites.site_name='Ebay' 
+ and test_cal_dt.retail_year not in ('2014') 
+ and test_kylin_fact.trans_id=1000000001 
+ group by test_cal_dt.week_beg_dt 
+ ,test_cal_dt.retail_year 
+ ,test_cal_dt.rtl_month_of_rtl_year_id 
+ ,test_cal_dt.retail_week 
+ ,test_category_groupings.meta_categ_name 
+ ,test_category_groupings.categ_lvl2_name 
+ ,test_category_groupings.categ_lvl3_name 
+ ,test_kylin_fact.lstg_format_name 
+ ,test_sites.site_name 
+ ,test_seller_type_dim.seller_type_desc 

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/query/src/test/resources/query/sql_invalid/query34_invalid_SQL.sql
----------------------------------------------------------------------
diff --git a/query/src/test/resources/query/sql_invalid/query34_invalid_SQL.sql b/query/src/test/resources/query/sql_invalid/query34_invalid_SQL.sql
new file mode 100644
index 0000000..0bcc5b4
--- /dev/null
+++ b/query/src/test/resources/query/sql_invalid/query34_invalid_SQL.sql
@@ -0,0 +1,6 @@
+SELECT COUNT(DISTINCT "TableauSQL"."TRANS_CNT") AS "ctd_TRANS_CNT_qk", "TableauSQL"."LSTG_FORMAT_NAME" AS "none_LSTG_FORMAT_NAME_nk" 
+ FROM ( select test_kylin_fact.lstg_format_name, sum(price) as GMV, count(seller_id) as TRANS_CNT 
+ from test_kylin_fact 
+ group by test_kylin_fact.lstg_format_name 
+ ) "TableauSQL" 
+ GROUP BY "TableauSQL"."LSTG_FORMAT_NAME" 

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/query/src/test/resources/query/sql_invalid/query_count_distinct_on_dimension.sql
----------------------------------------------------------------------
diff --git a/query/src/test/resources/query/sql_invalid/query_count_distinct_on_dimension.sql b/query/src/test/resources/query/sql_invalid/query_count_distinct_on_dimension.sql
new file mode 100644
index 0000000..30bbd76
--- /dev/null
+++ b/query/src/test/resources/query/sql_invalid/query_count_distinct_on_dimension.sql
@@ -0,0 +1 @@
+select count(distinct cal_dt) from test_kylin_fact
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/query/src/test/resources/query/sql_lookup/query01.sql
----------------------------------------------------------------------
diff --git a/query/src/test/resources/query/sql_lookup/query01.sql b/query/src/test/resources/query/sql_lookup/query01.sql
new file mode 100644
index 0000000..724c4fe
--- /dev/null
+++ b/query/src/test/resources/query/sql_lookup/query01.sql
@@ -0,0 +1,6 @@
+select CAL_DT, WEEK_BEG_DT from test_cal_dt
+
+
+
+
+

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/query/src/test/resources/query/sql_lookup/query02.sql
----------------------------------------------------------------------
diff --git a/query/src/test/resources/query/sql_lookup/query02.sql b/query/src/test/resources/query/sql_lookup/query02.sql
new file mode 100644
index 0000000..bfc2beb
--- /dev/null
+++ b/query/src/test/resources/query/sql_lookup/query02.sql
@@ -0,0 +1 @@
+select CATEG_LVL3_NAME, CATEG_LVL2_NAME, SITE_ID, META_CATEG_NAME, LEAF_CATEG_ID  from test_category_groupings

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/query/src/test/resources/query/sql_lookup/query03.sql
----------------------------------------------------------------------
diff --git a/query/src/test/resources/query/sql_lookup/query03.sql b/query/src/test/resources/query/sql_lookup/query03.sql
new file mode 100644
index 0000000..3186374
--- /dev/null
+++ b/query/src/test/resources/query/sql_lookup/query03.sql
@@ -0,0 +1 @@
+select SELLER_TYPE_DESC, SELLER_TYPE_CD from test_seller_type_dim

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/query/src/test/resources/query/sql_lookup/query04.sql
----------------------------------------------------------------------
diff --git a/query/src/test/resources/query/sql_lookup/query04.sql b/query/src/test/resources/query/sql_lookup/query04.sql
new file mode 100644
index 0000000..83598c7
--- /dev/null
+++ b/query/src/test/resources/query/sql_lookup/query04.sql
@@ -0,0 +1 @@
+select SITE_NAME, SITE_ID from test_sites

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/query/src/test/resources/query/sql_optimize/enable-limit01.sql
----------------------------------------------------------------------
diff --git a/query/src/test/resources/query/sql_optimize/enable-limit01.sql b/query/src/test/resources/query/sql_optimize/enable-limit01.sql
new file mode 100644
index 0000000..a17869a
--- /dev/null
+++ b/query/src/test/resources/query/sql_optimize/enable-limit01.sql
@@ -0,0 +1 @@
+select * from test_kylin_fact limit 10
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/query/src/test/resources/query/sql_orderby/query01.sql
----------------------------------------------------------------------
diff --git a/query/src/test/resources/query/sql_orderby/query01.sql b/query/src/test/resources/query/sql_orderby/query01.sql
new file mode 100644
index 0000000..afbde8e
--- /dev/null
+++ b/query/src/test/resources/query/sql_orderby/query01.sql
@@ -0,0 +1,10 @@
+select test_cal_dt.Week_Beg_Dt, sum(price) as c1, count(1) as c2 
+ from test_kylin_fact 
+ inner JOIN test_cal_dt 
+ ON test_kylin_fact.cal_dt = test_cal_dt.cal_dt 
+ where test_kylin_fact.lstg_format_name='ABIN' 
+ and test_cal_dt.week_beg_dt >= DATE '2013-06-09' 
+ group by test_cal_dt.week_beg_dt 
+ order by test_cal_dt.week_beg_dt   
+
+ -- optiq 0.8 reports varchar instead of date on week_beg_dt and fail test case

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/query/src/test/resources/query/sql_orderby/query02.sql
----------------------------------------------------------------------
diff --git a/query/src/test/resources/query/sql_orderby/query02.sql b/query/src/test/resources/query/sql_orderby/query02.sql
new file mode 100644
index 0000000..44485c5
--- /dev/null
+++ b/query/src/test/resources/query/sql_orderby/query02.sql
@@ -0,0 +1,7 @@
+select lstg_format_name, 
+ sum(price) as GMV, 
+ count(1) as TRANS_CNT 
+ from test_kylin_fact 
+ where lstg_format_name='FP-GTC' 
+ group by lstg_format_name 
+ order by lstg_format_name 

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/query/src/test/resources/query/sql_orderby/query03.sql
----------------------------------------------------------------------
diff --git a/query/src/test/resources/query/sql_orderby/query03.sql b/query/src/test/resources/query/sql_orderby/query03.sql
new file mode 100644
index 0000000..4093650
--- /dev/null
+++ b/query/src/test/resources/query/sql_orderby/query03.sql
@@ -0,0 +1,6 @@
+select lstg_format_name, 
+ sum(price) as GMV, 
+ count(1) as TRANS_CNT 
+ from test_kylin_fact 
+ group by lstg_format_name 
+ order by sum(price)
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/query/src/test/resources/query/sql_subquery/query00.sql
----------------------------------------------------------------------
diff --git a/query/src/test/resources/query/sql_subquery/query00.sql b/query/src/test/resources/query/sql_subquery/query00.sql
new file mode 100644
index 0000000..e2737ac
--- /dev/null
+++ b/query/src/test/resources/query/sql_subquery/query00.sql
@@ -0,0 +1,4 @@
+select test_cal_dt.week_beg_dt, sum(test_kylin_fact.price)  as sum_price, count(1) as cnt_1
+ from test_kylin_fact 
+ inner join test_cal_dt ON test_kylin_fact.cal_dt = test_cal_dt.cal_dt 
+ group by test_cal_dt.week_beg_dt 
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/query/src/test/resources/query/sql_subquery/query01.sql
----------------------------------------------------------------------
diff --git a/query/src/test/resources/query/sql_subquery/query01.sql b/query/src/test/resources/query/sql_subquery/query01.sql
new file mode 100644
index 0000000..b5c93b2
--- /dev/null
+++ b/query/src/test/resources/query/sql_subquery/query01.sql
@@ -0,0 +1,8 @@
+SELECT sum(sum_price) AS "COL" 
+ FROM ( 
+ select test_cal_dt.week_beg_dt, sum(test_kylin_fact.price) as sum_price, count(1) as cnt_1
+ from test_kylin_fact 
+ inner join test_cal_dt ON test_kylin_fact.cal_dt = test_cal_dt.cal_dt 
+ group by test_cal_dt.week_beg_dt 
+ ) "TableauSQL" 
+ HAVING COUNT(1)>0 
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/query/src/test/resources/query/sql_subquery/query02.sql.disable
----------------------------------------------------------------------
diff --git a/query/src/test/resources/query/sql_subquery/query02.sql.disable b/query/src/test/resources/query/sql_subquery/query02.sql.disable
new file mode 100644
index 0000000..6e70b77
--- /dev/null
+++ b/query/src/test/resources/query/sql_subquery/query02.sql.disable
@@ -0,0 +1,7 @@
+SELECT sum(1) AS "COL" 
+ FROM ( 
+ select test_cal_dt.week_beg_dt, sum(test_kylin_fact.price) as sum_price
+ from test_kylin_fact 
+ inner join test_cal_dt ON test_kylin_fact.cal_dt = test_cal_dt.cal_dt 
+ group by test_cal_dt.week_beg_dt 
+ ) "TableauSQL" 
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/query/src/test/resources/query/sql_subquery/query03.sql
----------------------------------------------------------------------
diff --git a/query/src/test/resources/query/sql_subquery/query03.sql b/query/src/test/resources/query/sql_subquery/query03.sql
new file mode 100644
index 0000000..e74cc58
--- /dev/null
+++ b/query/src/test/resources/query/sql_subquery/query03.sql
@@ -0,0 +1,9 @@
+select fact.cal_dt, sum(fact.price) as sum_price, count(1) as cnt_1
+from test_kylin_fact fact 
+left join test_cal_dt cal on fact.cal_dt=cal.cal_dt
+inner join
+(
+	select test_kylin_fact.cal_dt, sum(test_kylin_fact.price) from test_kylin_fact left join test_cal_dt 
+	on test_kylin_fact.cal_dt=test_cal_dt.cal_dt group by test_kylin_fact.cal_dt order by 2 desc limit 7
+) cal_2 on fact.cal_dt = cal_2.cal_dt 
+group by fact.cal_dt
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/query/src/test/resources/query/sql_subquery/query04.sql
----------------------------------------------------------------------
diff --git a/query/src/test/resources/query/sql_subquery/query04.sql b/query/src/test/resources/query/sql_subquery/query04.sql
new file mode 100644
index 0000000..83d8ca6
--- /dev/null
+++ b/query/src/test/resources/query/sql_subquery/query04.sql
@@ -0,0 +1,9 @@
+select fact.cal_dt, sum(fact.price) as sum_price, count(1) as cnt_1
+from test_kylin_fact fact 
+left join test_cal_dt cal on fact.cal_dt=cal.cal_dt
+inner join
+(
+	select test_kylin_fact.cal_dt, max(test_kylin_fact.cal_dt) as mmm from test_kylin_fact left join test_cal_dt
+	on test_kylin_fact.cal_dt=test_cal_dt.cal_dt group by test_kylin_fact.cal_dt order by 2 desc limit 7
+) cal_2 on fact.cal_dt = cal_2.mmm
+group by fact.cal_dt
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/query/src/test/resources/query/sql_subquery/query05.sql
----------------------------------------------------------------------
diff --git a/query/src/test/resources/query/sql_subquery/query05.sql b/query/src/test/resources/query/sql_subquery/query05.sql
new file mode 100644
index 0000000..1ae2005
--- /dev/null
+++ b/query/src/test/resources/query/sql_subquery/query05.sql
@@ -0,0 +1,10 @@
+SELECT  SUM("TEST_KYLIN_FACT"."PRICE") AS "sum_PRICE_ok" 
+	FROM "TEST_KYLIN_FACT"
+    INNER JOIN "TEST_CAL_DT" ON ("TEST_KYLIN_FACT"."CAL_DT" = "TEST_CAL_DT"."CAL_DT")
+    INNER JOIN (
+     SELECT COUNT(1) AS "XTableau_join_flag",
+      SUM("TEST_KYLIN_FACT"."PRICE") AS "X__alias__A",
+       "TEST_KYLIN_FACT"."CAL_DT" AS "none_CAL_DT_ok"   FROM "TEST_KYLIN_FACT"
+         INNER JOIN "TEST_CAL_DT" ON ("TEST_KYLIN_FACT"."CAL_DT" = "TEST_CAL_DT"."CAL_DT")
+     GROUP BY "TEST_KYLIN_FACT"."CAL_DT"   ORDER BY 2 DESC   LIMIT 10  ) "t0" ON ("TEST_KYLIN_FACT"."CAL_DT" = "t0"."none_CAL_DT_ok") 
+    GROUP BY "TEST_KYLIN_FACT"."CAL_DT"
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/query/src/test/resources/query/sql_subquery/query06.sql
----------------------------------------------------------------------
diff --git a/query/src/test/resources/query/sql_subquery/query06.sql b/query/src/test/resources/query/sql_subquery/query06.sql
new file mode 100644
index 0000000..724eb2d
--- /dev/null
+++ b/query/src/test/resources/query/sql_subquery/query06.sql
@@ -0,0 +1,6 @@
+SELECT "TEST_KYLIN_FACT"."CAL_DT", SUM("TEST_KYLIN_FACT"."PRICE") AS "sum_PRICE_ok" FROM "TEST_KYLIN_FACT" "TEST_KYLIN_FACT"
+  INNER JOIN (
+             SELECT COUNT(1) AS "XTableau_join_flag",     SUM("TEST_KYLIN_FACT"."PRICE") AS "X__alias__A",     "TEST_KYLIN_FACT"."CAL_DT" AS "none_CAL_DT_ok"   FROM "TEST_KYLIN_FACT" "TEST_KYLIN_FACT"
+             GROUP BY "TEST_KYLIN_FACT"."CAL_DT"   ORDER BY 2 DESC   LIMIT 7  )
+
+    "t0" ON ("TEST_KYLIN_FACT"."CAL_DT" = "t0"."none_CAL_DT_ok") GROUP BY "TEST_KYLIN_FACT"."CAL_DT"

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/query/src/test/resources/query/sql_subquery/query07.sql
----------------------------------------------------------------------
diff --git a/query/src/test/resources/query/sql_subquery/query07.sql b/query/src/test/resources/query/sql_subquery/query07.sql
new file mode 100644
index 0000000..627e59b
--- /dev/null
+++ b/query/src/test/resources/query/sql_subquery/query07.sql
@@ -0,0 +1,6 @@
+select cal_dt, sum(price) as sum_price
+from test_kylin_fact fact
+inner join (
+select count(1) as cnt, min(cal_dt) as "mmm",  cal_dt as dt from test_kylin_fact group by cal_dt order by 2 desc limit 10
+) t0 on (fact.cal_dt = t0.dt) 
+group by cal_dt
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/query/src/test/resources/query/sql_tableau/query00.sql
----------------------------------------------------------------------
diff --git a/query/src/test/resources/query/sql_tableau/query00.sql b/query/src/test/resources/query/sql_tableau/query00.sql
new file mode 100644
index 0000000..790e348
--- /dev/null
+++ b/query/src/test/resources/query/sql_tableau/query00.sql
@@ -0,0 +1,5 @@
+select test_cal_dt.week_beg_dt, sum(test_kylin_fact.price) 
+ from test_kylin_fact 
+ inner join test_cal_dt ON test_kylin_fact.cal_dt = test_cal_dt.cal_dt 
+ group by test_cal_dt.week_beg_dt 
+

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/query/src/test/resources/query/sql_tableau/query01.sql
----------------------------------------------------------------------
diff --git a/query/src/test/resources/query/sql_tableau/query01.sql b/query/src/test/resources/query/sql_tableau/query01.sql
new file mode 100644
index 0000000..44e4c63
--- /dev/null
+++ b/query/src/test/resources/query/sql_tableau/query01.sql
@@ -0,0 +1,15 @@
+SELECT SUM(1) AS "COL", 
+ 2 AS "COL2" 
+ FROM ( 
+ select test_kylin_fact.lstg_format_name, test_cal_dt.week_beg_dt,sum(test_kylin_fact.price) as GMV 
+ , count(*) as TRANS_CNT 
+ from test_kylin_fact 
+ inner JOIN test_cal_dt 
+ ON test_kylin_fact.cal_dt = test_cal_dt.cal_dt 
+ where test_cal_dt.week_beg_dt between DATE '2013-05-01' and DATE '2013-08-01' 
+ group by test_kylin_fact.lstg_format_name, test_cal_dt.week_beg_dt 
+ having sum(price)>500 
+ ) "TableauSQL" 
+ GROUP BY 2 
+ HAVING COUNT(1)>0 
+ 

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/query/src/test/resources/query/sql_tableau/query02.sql
----------------------------------------------------------------------
diff --git a/query/src/test/resources/query/sql_tableau/query02.sql b/query/src/test/resources/query/sql_tableau/query02.sql
new file mode 100644
index 0000000..ca40f11
--- /dev/null
+++ b/query/src/test/resources/query/sql_tableau/query02.sql
@@ -0,0 +1,12 @@
+SELECT * 
+ FROM ( 
+ select test_kylin_fact.lstg_format_name, test_cal_dt.week_beg_dt,sum(test_kylin_fact.price) as GMV 
+ , count(*) as TRANS_CNT 
+ from test_kylin_fact 
+ inner JOIN test_cal_dt 
+ ON test_kylin_fact.cal_dt = test_cal_dt.cal_dt 
+ where test_cal_dt.week_beg_dt between DATE '2013-05-01' and DATE '2013-08-01' 
+ group by test_kylin_fact.lstg_format_name, test_cal_dt.week_beg_dt 
+ having sum(price)>500 
+ ) "TableauSQL" 
+ LIMIT 1 

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/query/src/test/resources/query/sql_tableau/query03.sql
----------------------------------------------------------------------
diff --git a/query/src/test/resources/query/sql_tableau/query03.sql b/query/src/test/resources/query/sql_tableau/query03.sql
new file mode 100644
index 0000000..b78a587
--- /dev/null
+++ b/query/src/test/resources/query/sql_tableau/query03.sql
@@ -0,0 +1,17 @@
+SELECT 
+ test_cal_dt.week_beg_dt 
+ ,test_category_groupings.meta_categ_name 
+ ,test_category_groupings.categ_lvl2_name 
+ ,test_category_groupings.categ_lvl3_name 
+ ,sum(test_kylin_fact.price) as GMV 
+ , count(*) as TRANS_CNT 
+ FROM test_kylin_fact 
+ inner JOIN test_cal_dt 
+ ON test_kylin_fact.cal_dt = test_cal_dt.cal_dt 
+ inner JOIN test_category_groupings 
+ ON test_kylin_fact.leaf_categ_id = test_category_groupings.leaf_categ_id 
+ AND test_kylin_fact.lstg_site_id = test_category_groupings.site_id 
+ group by test_cal_dt.week_beg_dt 
+ ,test_category_groupings.meta_categ_name 
+ ,test_category_groupings.categ_lvl2_name 
+ ,test_category_groupings.categ_lvl3_name 

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/query/src/test/resources/query/sql_tableau/query04.sql
----------------------------------------------------------------------
diff --git a/query/src/test/resources/query/sql_tableau/query04.sql b/query/src/test/resources/query/sql_tableau/query04.sql
new file mode 100644
index 0000000..b7c8a99
--- /dev/null
+++ b/query/src/test/resources/query/sql_tableau/query04.sql
@@ -0,0 +1,21 @@
+SELECT "TableauSQL"."META_CATEG_NAME" AS "none_META_CATEG_NAME_nk" 
+ FROM ( 
+ SELECT 
+ test_cal_dt.week_beg_dt 
+ ,test_category_groupings.meta_categ_name 
+ ,test_category_groupings.categ_lvl2_name 
+ ,test_category_groupings.categ_lvl3_name 
+ ,sum(test_kylin_fact.price) as GMV 
+ , count(*) as TRANS_CNT 
+ FROM test_kylin_fact 
+ inner JOIN test_cal_dt 
+ ON test_kylin_fact.cal_dt = test_cal_dt.cal_dt 
+ inner JOIN test_category_groupings 
+ ON test_kylin_fact.leaf_categ_id = test_category_groupings.leaf_categ_id 
+ AND test_kylin_fact.lstg_site_id = test_category_groupings.site_id 
+ group by test_cal_dt.week_beg_dt 
+ ,test_category_groupings.meta_categ_name 
+ ,test_category_groupings.categ_lvl2_name 
+ ,test_category_groupings.categ_lvl3_name 
+ ) "TableauSQL" 
+ GROUP BY "TableauSQL"."META_CATEG_NAME" 

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/query/src/test/resources/query/sql_tableau/query05.sql
----------------------------------------------------------------------
diff --git a/query/src/test/resources/query/sql_tableau/query05.sql b/query/src/test/resources/query/sql_tableau/query05.sql
new file mode 100644
index 0000000..25e8f89
--- /dev/null
+++ b/query/src/test/resources/query/sql_tableau/query05.sql
@@ -0,0 +1,22 @@
+SELECT SUM("TableauSQL"."GMV") AS "sum_GMV_ok", 
+ SUM("TableauSQL"."TRANS_CNT") AS "sum_TRANS_CNT_ok" 
+ FROM ( 
+ SELECT 
+ test_cal_dt.week_beg_dt 
+ ,test_category_groupings.meta_categ_name 
+ ,test_category_groupings.categ_lvl2_name 
+ ,test_category_groupings.categ_lvl3_name 
+ ,sum(test_kylin_fact.price) as GMV 
+ , count(*) as TRANS_CNT 
+ FROM test_kylin_fact 
+ inner JOIN test_cal_dt 
+ ON test_kylin_fact.cal_dt = test_cal_dt.cal_dt 
+ inner JOIN test_category_groupings 
+ ON test_kylin_fact.leaf_categ_id = test_category_groupings.leaf_categ_id 
+ AND test_kylin_fact.lstg_site_id = test_category_groupings.site_id 
+ group by test_cal_dt.week_beg_dt 
+ ,test_category_groupings.meta_categ_name 
+ ,test_category_groupings.categ_lvl2_name 
+ ,test_category_groupings.categ_lvl3_name 
+ ) "TableauSQL" 
+ HAVING (COUNT(1) > 0) 

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/query/src/test/resources/query/sql_tableau/query06.sql
----------------------------------------------------------------------
diff --git a/query/src/test/resources/query/sql_tableau/query06.sql b/query/src/test/resources/query/sql_tableau/query06.sql
new file mode 100644
index 0000000..25e8f89
--- /dev/null
+++ b/query/src/test/resources/query/sql_tableau/query06.sql
@@ -0,0 +1,22 @@
+SELECT SUM("TableauSQL"."GMV") AS "sum_GMV_ok", 
+ SUM("TableauSQL"."TRANS_CNT") AS "sum_TRANS_CNT_ok" 
+ FROM ( 
+ SELECT 
+ test_cal_dt.week_beg_dt 
+ ,test_category_groupings.meta_categ_name 
+ ,test_category_groupings.categ_lvl2_name 
+ ,test_category_groupings.categ_lvl3_name 
+ ,sum(test_kylin_fact.price) as GMV 
+ , count(*) as TRANS_CNT 
+ FROM test_kylin_fact 
+ inner JOIN test_cal_dt 
+ ON test_kylin_fact.cal_dt = test_cal_dt.cal_dt 
+ inner JOIN test_category_groupings 
+ ON test_kylin_fact.leaf_categ_id = test_category_groupings.leaf_categ_id 
+ AND test_kylin_fact.lstg_site_id = test_category_groupings.site_id 
+ group by test_cal_dt.week_beg_dt 
+ ,test_category_groupings.meta_categ_name 
+ ,test_category_groupings.categ_lvl2_name 
+ ,test_category_groupings.categ_lvl3_name 
+ ) "TableauSQL" 
+ HAVING (COUNT(1) > 0) 

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/query/src/test/resources/query/sql_tableau/query07.sql
----------------------------------------------------------------------
diff --git a/query/src/test/resources/query/sql_tableau/query07.sql b/query/src/test/resources/query/sql_tableau/query07.sql
new file mode 100644
index 0000000..5cf3bae
--- /dev/null
+++ b/query/src/test/resources/query/sql_tableau/query07.sql
@@ -0,0 +1,8 @@
+SELECT SUM("TableauSQL"."GMV") AS "sum_GMV_ok", SUM("TableauSQL"."TRANS_CNT") AS "sum_TRANS_CNT_ok" 
+ FROM ( 
+ SELECT test_cal_dt.week_beg_dt ,test_category_groupings.meta_categ_name ,test_category_groupings.categ_lvl2_name ,test_category_groupings.categ_lvl3_name ,sum(test_kylin_fact.price) as GMV , count(*) as TRANS_CNT 
+ FROM test_kylin_fact 
+ inner JOIN test_cal_dt ON test_kylin_fact.cal_dt = test_cal_dt.cal_dt 
+ inner JOIN test_category_groupings ON test_kylin_fact.leaf_categ_id = test_category_groupings.leaf_categ_id AND test_kylin_fact.lstg_site_id = test_category_groupings.site_id 
+ group by test_cal_dt.week_beg_dt ,test_category_groupings.meta_categ_name ,test_category_groupings.categ_lvl2_name ,test_category_groupings.categ_lvl3_name 
+ ) "TableauSQL" HAVING (COUNT(1) > 0) 

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/query/src/test/resources/query/sql_tableau/query08.sql.disabled
----------------------------------------------------------------------
diff --git a/query/src/test/resources/query/sql_tableau/query08.sql.disabled b/query/src/test/resources/query/sql_tableau/query08.sql.disabled
new file mode 100644
index 0000000..c5deb9c
--- /dev/null
+++ b/query/src/test/resources/query/sql_tableau/query08.sql.disabled
@@ -0,0 +1,6 @@
+-- LEAF_CATEG_ID is a join field on fact table, but no on cuboid. We really support this?
+
+SELECT "TEST_KYLIN_FACT"."LEAF_CATEG_ID" AS "NONE_LEAF_CATEG_ID_OK"
+FROM "olap"."TEST_KYLIN_FACT" "TEST_KYLIN_FACT"
+GROUP BY "TEST_KYLIN_FACT"."LEAF_CATEG_ID"
+

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/query/src/test/resources/query/sql_tableau/query09.sql.disabled
----------------------------------------------------------------------
diff --git a/query/src/test/resources/query/sql_tableau/query09.sql.disabled b/query/src/test/resources/query/sql_tableau/query09.sql.disabled
new file mode 100644
index 0000000..5814c81
--- /dev/null
+++ b/query/src/test/resources/query/sql_tableau/query09.sql.disabled
@@ -0,0 +1,39 @@
+SELECT "TEST_CATEGORY_GROUPINGS"."ADULT_CATEG_YN" AS "ADULT_CATEG_YN",
+  "TEST_CATEGORY_GROUPINGS"."BSNS_VRTCL_NAME" AS "BSNS_VRTCL_NAME",
+  "TEST_CATEGORY_GROUPINGS"."CATEG_BUSN_MGR" AS "CATEG_BUSN_MGR",
+  "TEST_CATEGORY_GROUPINGS"."CATEG_BUSN_UNIT" AS "CATEG_BUSN_UNIT",
+  "TEST_CATEGORY_GROUPINGS"."CATEG_FLAGS" AS "CATEG_FLAGS",
+  "TEST_CATEGORY_GROUPINGS"."CATEG_LVL2_ID" AS "CATEG_LVLC_ID",
+  "TEST_CATEGORY_GROUPINGS"."CATEG_LVL2_NAME" AS "CATEG_LVLC_NAME",
+  "TEST_CATEGORY_GROUPINGS"."CATEG_LVL3_ID" AS "CATEG_LVLD_ID",
+  "TEST_CATEGORY_GROUPINGS"."CATEG_LVL3_NAME" AS "CATEG_LVLD_NAME",
+  "TEST_CATEGORY_GROUPINGS"."CATEG_LVL4_ID" AS "CATEG_LVLE_ID",
+  "TEST_CATEGORY_GROUPINGS"."CATEG_LVL4_NAME" AS "CATEG_LVLE_NAME",
+  "TEST_CATEGORY_GROUPINGS"."CATEG_LVL5_ID" AS "CATEG_LVLF_ID",
+  "TEST_CATEGORY_GROUPINGS"."CATEG_LVL5_NAME" AS "CATEG_LVLF_NAME",
+  "TEST_CATEGORY_GROUPINGS"."CATEG_LVL6_ID" AS "CATEG_LVLG_ID",
+  "TEST_CATEGORY_GROUPINGS"."CATEG_LVL6_NAME" AS "CATEG_LVLG_NAME",
+  "TEST_CATEGORY_GROUPINGS"."CATEG_LVL7_ID" AS "CATEG_LVLH_ID",
+  "TEST_CATEGORY_GROUPINGS"."CATEG_LVL7_NAME" AS "CATEG_LVLH_NAME",
+  "TEST_CATEGORY_GROUPINGS"."CRE_DATE" AS "CRE_DATE",
+  "TEST_CATEGORY_GROUPINGS"."CRE_USER" AS "CRE_USER",
+  "TEST_CATEGORY_GROUPINGS"."DOMAIN_ID" AS "DOMAIN_ID",
+  "TEST_CATEGORY_GROUPINGS"."GCS_ID" AS "GCS_ID",
+  "TEST_CATEGORY_GROUPINGS"."LEAF_CATEG_ID" AS "LEAF_CATEG_ID",
+  "TEST_CATEGORY_GROUPINGS"."LEAF_CATEG_NAME" AS "LEAF_CATEG_NAME",
+  "TEST_CATEGORY_GROUPINGS"."META_CATEG_ID" AS "META_CATEG_ID",
+  "TEST_CATEGORY_GROUPINGS"."META_CATEG_NAME" AS "META_CATEG_NAME",
+  "TEST_CATEGORY_GROUPINGS"."MOVE_TO" AS "MOVE_TO",
+  1 AS "Number_of_Records",
+  "TEST_CATEGORY_GROUPINGS"."REGN_CATEG" AS "REGN_CATEG",
+  "TEST_CATEGORY_GROUPINGS"."SAP_CATEGORY_ID" AS "SAP_CATEGORY_ID",
+  "TEST_CATEGORY_GROUPINGS"."SITE_ID" AS "SITE_ID",
+  "TEST_CATEGORY_GROUPINGS"."SRC_ID" AS "SRC_ID",
+  "TEST_CATEGORY_GROUPINGS"."UPD_DATE" AS "UPD_DATE",
+  "TEST_CATEGORY_GROUPINGS"."UPD_USER" AS "UPD_USER",
+  "TEST_CATEGORY_GROUPINGS"."USER_DEFINED_FIELD1" AS "USER_DEFINED_FIELDB",
+  "TEST_CATEGORY_GROUPINGS"."USER_DEFINED_FIELD3" AS "USER_DEFINED_FIELDD",
+  "TEST_CATEGORY_GROUPINGS"."USER_DEFINED_FIELD5" AS "USER_DEFINED_FIELDF",
+  "TEST_CATEGORY_GROUPINGS"."VCS_ID" AS "VCS_ID"
+FROM "olap"."TEST_CATEGORY_GROUPINGS" "TEST_CATEGORY_GROUPINGS"
+LIMIT 10000

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/query/src/test/resources/query/sql_tableau/query10.sql
----------------------------------------------------------------------
diff --git a/query/src/test/resources/query/sql_tableau/query10.sql b/query/src/test/resources/query/sql_tableau/query10.sql
new file mode 100644
index 0000000..26d29a3
--- /dev/null
+++ b/query/src/test/resources/query/sql_tableau/query10.sql
@@ -0,0 +1,5 @@
+SELECT "TEST_CAL_DT"."WEEK_BEG_DT" AS "none_WEEK_BEG_DT_nk", 
+ SUM("TEST_KYLIN_FACT"."PRICE") AS "sum_PRICE_ok" 
+ FROM "TEST_KYLIN_FACT" 
+ inner JOIN "TEST_CAL_DT" ON ("TEST_KYLIN_FACT"."CAL_DT" = "TEST_CAL_DT"."CAL_DT") 
+ GROUP BY "TEST_CAL_DT"."WEEK_BEG_DT" 

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/query/src/test/resources/query/sql_tableau/query11.sql
----------------------------------------------------------------------
diff --git a/query/src/test/resources/query/sql_tableau/query11.sql b/query/src/test/resources/query/sql_tableau/query11.sql
new file mode 100644
index 0000000..d7e70c1
--- /dev/null
+++ b/query/src/test/resources/query/sql_tableau/query11.sql
@@ -0,0 +1,6 @@
+SELECT COUNT(1) AS cnt_ITEM_COUNT_ok, 
+ TEST_CAL_DT.WEEK_BEG_DT AS none_WEEK_BEG_DT_nk 
+ FROM "TEST_KYLIN_FACT" 
+ inner JOIN "TEST_CAL_DT" ON ("TEST_KYLIN_FACT"."CAL_DT" = "TEST_CAL_DT"."CAL_DT") 
+ GROUP BY "TEST_CAL_DT"."WEEK_BEG_DT" 
+ 

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/query/src/test/resources/query/sql_tableau/query12.sql
----------------------------------------------------------------------
diff --git a/query/src/test/resources/query/sql_tableau/query12.sql b/query/src/test/resources/query/sql_tableau/query12.sql
new file mode 100644
index 0000000..26d29a3
--- /dev/null
+++ b/query/src/test/resources/query/sql_tableau/query12.sql
@@ -0,0 +1,5 @@
+SELECT "TEST_CAL_DT"."WEEK_BEG_DT" AS "none_WEEK_BEG_DT_nk", 
+ SUM("TEST_KYLIN_FACT"."PRICE") AS "sum_PRICE_ok" 
+ FROM "TEST_KYLIN_FACT" 
+ inner JOIN "TEST_CAL_DT" ON ("TEST_KYLIN_FACT"."CAL_DT" = "TEST_CAL_DT"."CAL_DT") 
+ GROUP BY "TEST_CAL_DT"."WEEK_BEG_DT" 

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/query/src/test/resources/query/sql_tableau/query13.sql
----------------------------------------------------------------------
diff --git a/query/src/test/resources/query/sql_tableau/query13.sql b/query/src/test/resources/query/sql_tableau/query13.sql
new file mode 100644
index 0000000..28c0e05
--- /dev/null
+++ b/query/src/test/resources/query/sql_tableau/query13.sql
@@ -0,0 +1,4 @@
+SELECT EXTRACT(YEAR FROM TEST_CAL_DT.WEEK_BEG_DT) AS yr_WEEK_BEG_DT_ok 
+ FROM TEST_KYLIN_FACT 
+ inner JOIN TEST_CAL_DT ON (TEST_KYLIN_FACT.CAL_DT = TEST_CAL_DT.CAL_DT) 
+ GROUP BY EXTRACT(YEAR FROM TEST_CAL_DT.WEEK_BEG_DT) 

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/query/src/test/resources/query/sql_tableau/query14.sql
----------------------------------------------------------------------
diff --git a/query/src/test/resources/query/sql_tableau/query14.sql b/query/src/test/resources/query/sql_tableau/query14.sql
new file mode 100644
index 0000000..0f5d99c
--- /dev/null
+++ b/query/src/test/resources/query/sql_tableau/query14.sql
@@ -0,0 +1 @@
+SELECT QUARTER("TEST_CAL_DT"."WEEK_BEG_DT") AS "qr_WEEK_BEG_DT_ok", EXTRACT(YEAR FROM "TEST_CAL_DT"."WEEK_BEG_DT") AS "yr_WEEK_BEG_DT_ok" FROM "TEST_KYLIN_FACT" inner JOIN "TEST_CAL_DT" ON ("TEST_KYLIN_FACT"."CAL_DT" = "TEST_CAL_DT"."CAL_DT") GROUP BY QUARTER("TEST_CAL_DT"."WEEK_BEG_DT"), EXTRACT(YEAR FROM "TEST_CAL_DT"."WEEK_BEG_DT") 

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/query/src/test/resources/query/sql_tableau/query15.sql
----------------------------------------------------------------------
diff --git a/query/src/test/resources/query/sql_tableau/query15.sql b/query/src/test/resources/query/sql_tableau/query15.sql
new file mode 100644
index 0000000..6bebae6
--- /dev/null
+++ b/query/src/test/resources/query/sql_tableau/query15.sql
@@ -0,0 +1,4 @@
+SELECT QUARTER(TEST_CAL_DT.WEEK_BEG_DT) AS qr_WEEK_BEG_DT_ok 
+ FROM TEST_KYLIN_FACT 
+ inner JOIN TEST_CAL_DT ON (TEST_KYLIN_FACT.CAL_DT = TEST_CAL_DT.CAL_DT) 
+ GROUP BY QUARTER(TEST_CAL_DT.WEEK_BEG_DT) 

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/query/src/test/resources/query/sql_tableau/query16.sql
----------------------------------------------------------------------
diff --git a/query/src/test/resources/query/sql_tableau/query16.sql b/query/src/test/resources/query/sql_tableau/query16.sql
new file mode 100644
index 0000000..28d2dbb
--- /dev/null
+++ b/query/src/test/resources/query/sql_tableau/query16.sql
@@ -0,0 +1,4 @@
+SELECT EXTRACT(YEAR FROM TEST_CAL_DT.WEEK_BEG_DT) AS yr_WEEK_BEG_DT_ok, QUARTER(TEST_CAL_DT.WEEK_BEG_DT) AS qr_WEEK_BEG_DT_ok 
+ FROM TEST_KYLIN_FACT 
+ inner JOIN TEST_CAL_DT ON (TEST_KYLIN_FACT.CAL_DT = TEST_CAL_DT.CAL_DT) 
+ GROUP BY EXTRACT(YEAR FROM TEST_CAL_DT.WEEK_BEG_DT), QUARTER(TEST_CAL_DT.WEEK_BEG_DT) 

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/query/src/test/resources/query/sql_tableau/query17.sql
----------------------------------------------------------------------
diff --git a/query/src/test/resources/query/sql_tableau/query17.sql b/query/src/test/resources/query/sql_tableau/query17.sql
new file mode 100644
index 0000000..c4dfba3
--- /dev/null
+++ b/query/src/test/resources/query/sql_tableau/query17.sql
@@ -0,0 +1 @@
+SELECT EXTRACT(MONTH FROM "TEST_CAL_DT"."WEEK_BEG_DT") AS "mn_WEEK_BEG_DT_ok", (( EXTRACT(YEAR FROM "TEST_CAL_DT"."WEEK_BEG_DT") * 100) + EXTRACT(MONTH FROM "TEST_CAL_DT"."WEEK_BEG_DT")) AS "my_WEEK_BEG_DT_ok", QUARTER("TEST_CAL_DT"."WEEK_BEG_DT") AS "qr_WEEK_BEG_DT_ok", EXTRACT(YEAR FROM "TEST_CAL_DT"."WEEK_BEG_DT") AS "yr_WEEK_BEG_DT_ok" FROM "TEST_KYLIN_FACT" inner JOIN "TEST_CAL_DT" ON ("TEST_KYLIN_FACT"."CAL_DT" = "TEST_CAL_DT"."CAL_DT") GROUP BY EXTRACT(YEAR FROM "TEST_CAL_DT"."WEEK_BEG_DT"), QUARTER("TEST_CAL_DT"."WEEK_BEG_DT"), (( EXTRACT(YEAR FROM "TEST_CAL_DT"."WEEK_BEG_DT") * 100) + EXTRACT(MONTH FROM "TEST_CAL_DT"."WEEK_BEG_DT")), EXTRACT(MONTH FROM "TEST_CAL_DT"."WEEK_BEG_DT") 

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/query/src/test/resources/query/sql_tableau/query18.sql
----------------------------------------------------------------------
diff --git a/query/src/test/resources/query/sql_tableau/query18.sql b/query/src/test/resources/query/sql_tableau/query18.sql
new file mode 100644
index 0000000..c1041b4
--- /dev/null
+++ b/query/src/test/resources/query/sql_tableau/query18.sql
@@ -0,0 +1,3 @@
+select test_kylin_fact.lstg_format_name, sum(price) as GMV, count(seller_id) as TRANS_CNT 
+ from test_kylin_fact 
+ group by test_kylin_fact.lstg_format_name 

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/query/src/test/resources/query/sql_tableau/query19.sql
----------------------------------------------------------------------
diff --git a/query/src/test/resources/query/sql_tableau/query19.sql b/query/src/test/resources/query/sql_tableau/query19.sql
new file mode 100644
index 0000000..83311f1
--- /dev/null
+++ b/query/src/test/resources/query/sql_tableau/query19.sql
@@ -0,0 +1,8 @@
+
+ 
+ 
+ SELECT "TableauSQL"."LSTG_FORMAT_NAME" AS "none_LSTG_FORMAT_NAME_nk", SUM("TableauSQL"."TRANS_CNT") AS "sum_TRANS_CNT_qk" 
+ FROM ( select test_kylin_fact.lstg_format_name, sum(price) as GMV, count(seller_id) as TRANS_CNT 
+ from test_kylin_fact 
+ group by test_kylin_fact.lstg_format_name ) "TableauSQL" 
+ GROUP BY "TableauSQL"."LSTG_FORMAT_NAME" 

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/query/src/test/resources/query/sql_tableau/query20.sql
----------------------------------------------------------------------
diff --git a/query/src/test/resources/query/sql_tableau/query20.sql b/query/src/test/resources/query/sql_tableau/query20.sql
new file mode 100644
index 0000000..baa4547
--- /dev/null
+++ b/query/src/test/resources/query/sql_tableau/query20.sql
@@ -0,0 +1,8 @@
+
+ 
+ 
+ SELECT "TableauSQL"."LSTG_FORMAT_NAME" AS "none_LSTG_FORMAT_NAME_nk", SUM("TableauSQL"."TRANS_CNT") AS "sum_TRANS_CNT_qk" 
+ FROM ( select test_kylin_fact.lstg_format_name, sum(price) as GMV, count(seller_id) as TRANS_CNT 
+ from test_kylin_fact where test_kylin_fact.lstg_format_name > 'ab' 
+ group by test_kylin_fact.lstg_format_name having count(seller_id) > 2 ) "TableauSQL" 
+ GROUP BY "TableauSQL"."LSTG_FORMAT_NAME" 

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/query/src/test/resources/query/sql_tableau/query21.sql
----------------------------------------------------------------------
diff --git a/query/src/test/resources/query/sql_tableau/query21.sql b/query/src/test/resources/query/sql_tableau/query21.sql
new file mode 100644
index 0000000..a31f1a7
--- /dev/null
+++ b/query/src/test/resources/query/sql_tableau/query21.sql
@@ -0,0 +1,9 @@
+ select test_kylin_fact.lstg_format_name, test_cal_dt.week_beg_dt,sum(test_kylin_fact.price) as GMV
+ , count(*) as TRANS_CNT 
+ from test_kylin_fact 
+ inner JOIN test_cal_dt 
+ ON test_kylin_fact.cal_dt = test_cal_dt.cal_dt 
+ where test_cal_dt.week_beg_dt between DATE '2013-05-01' and DATE '2013-08-01' 
+ group by test_kylin_fact.lstg_format_name, test_cal_dt.week_beg_dt 
+ having sum(price)>500
+ limit 1


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

Posted by lu...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/examples/test_case_data/localmeta/data/TEST_SITES.csv
----------------------------------------------------------------------
diff --git a/examples/test_case_data/localmeta/data/TEST_SITES.csv b/examples/test_case_data/localmeta/data/TEST_SITES.csv
new file mode 100644
index 0000000..a9d79c6
--- /dev/null
+++ b/examples/test_case_data/localmeta/data/TEST_SITES.csv
@@ -0,0 +1,262 @@
+248,Uruguay,S,1,0,211,2013-01-30,2013-11-27 00:14:36,USER_X,USER_X
+166,Qatar,S,1,0,166,2013-01-30,2013-11-27 00:14:36,USER_X,USER_X
+84,Guam,S,1,0,84,2013-01-29,2013-11-27 00:14:36,USER_X,USER_X
+225,Peru,S,45,0,161,2005-04-27,2013-11-27 00:14:36,USER_X,USER_X
+103,Jan Mayen,S,1,0,103,2013-01-29,2013-11-27 00:14:36,USER_X,USER_X
+63,El Salvador,S,1,0,63,2013-01-29,2013-11-27 00:14:36,USER_X,USER_X
+124,Macedonia,S,1,0,124,2013-01-30,2013-11-27 00:14:36,USER_X,USER_X
+147,Netherlands Antilles,S,1,0,147,2013-01-30,2013-11-27 00:14:36,USER_X,USER_X
+21,Barbados,S,1,0,21,2013-01-29,2013-11-27 00:14:36,USER_X,USER_X
+82,Grenada,S,1,0,82,2013-01-29,2013-11-27 00:14:36,USER_X,USER_X
+980,CoBrand,D,1,0,-999,2000-06-29,,USER_G,USER_X
+2,Canada,S,2,1,2,2000-06-09,2013-11-27 00:14:36,USER_S,USER_X
+204,Indonesia,S,24,0,96,2003-10-04,2013-11-27 00:14:36,USER_X,USER_X
+246,Tuvalu,S,1,0,207,2013-01-30,2013-11-27 00:14:36,USER_X,USER_X
+25,Benin,S,1,0,25,2013-01-29,2013-11-27 00:14:36,USER_X,USER_X
+-6,StubHub US,S,1,1,1,2012-11-04,,USER_C4,USER_X
+86,Guernsey,S,1,0,86,2013-01-29,2013-11-27 00:14:36,USER_X,USER_X
+187,Sri Lanka,S,1,0,187,2013-01-30,2013-11-27 00:14:36,USER_X,USER_X
+6,Algeria,S,1,0,6,2000-07-26,2013-11-27 00:14:36,USER_G,USER_X
+984,Digit City Cobrand,D,1,0,-999,2000-06-29,,USER_G,USER_X
+145,Nepal,S,1,0,145,2013-01-30,2013-11-27 00:14:36,USER_X,USER_X
+4,Afghanistan,S,1,0,4,2000-07-26,2013-11-27 00:14:36,USER_G,USER_X
+227,Iceland,S,1,0,94,2013-01-29,2013-11-27 00:14:36,USER_X,USER_X
+229,Serbia,S,1,0,229,2013-01-30,2013-11-27 00:14:36,USER_X,USER_X
+65,Eritrea,S,1,0,65,2013-01-29,2013-11-27 00:14:36,USER_X,USER_X
+44,Chile,S,1,0,44,2012-04-27,2013-11-27 00:14:36,USER_X,USER_X
+170,Saint Helena,S,1,0,170,2013-01-30,2013-11-27 00:14:36,USER_X,USER_X
+88,Guinea-Bissau,S,1,0,88,2013-01-29,2013-11-27 00:14:36,USER_X,USER_X
+46,Colombia,S,1,0,46,2012-04-27,2013-11-27 00:14:36,USER_X,USER_X
+-4,Gmarket,S,48,0,111,2010-05-23,,USER_X,USER_X
+69,Fiji,S,1,0,69,2013-01-29,2013-11-27 00:14:36,USER_X,USER_X
+48,"Congo Democratic Republic Of The",S,1,0,48,2013-01-29,2013-11-27 00:14:36,USER_X,USER_X
+231,Turkey,S,1,0,204,2012-04-27,2013-11-27 00:14:36,USER_X,USER_X
+126,Malawi,S,1,0,126,2013-01-30,2013-11-27 00:14:36,USER_X,USER_X
+-888,Invalid,S,-999,0,-999,2004-12-03,,USER_X,USER_X
+107,Kazakhstan,S,1,0,107,2013-01-29,2013-11-27 00:14:36,USER_X,USER_X
+982,Netscape Cobrand,D,1,0,-999,2000-06-29,,USER_G,USER_X
+900,Gbh,S,1,0,1,2012-10-25,2013-11-27 00:14:36,USER_X,USER_X
+208,New Zealand,S,33,0,149,2003-10-04,2013-11-27 00:14:36,USER_X,USER_X
+71,France,S,7,1,71,2000-09-06,2013-11-27 00:14:36,USER_S,USER_X
+10,Anguilla,S,1,0,10,2013-01-29,2013-11-27 00:14:36,USER_X,USER_X
+172,Saint Lucia,S,1,0,172,2013-01-30,2013-11-27 00:14:36,USER_X,USER_X
+153,Niue,S,1,0,153,2013-01-30,2013-11-27 00:14:36,USER_X,USER_X
+67,Ethiopia,S,1,0,67,2013-01-29,2013-11-27 00:14:36,USER_X,USER_X
+128,Maldives,S,1,0,128,2013-01-30,2013-11-27 00:14:36,USER_X,USER_X
+233,United Arab Emirates,S,1,0,210,2012-04-27,2013-11-27 00:14:36,USER_X,USER_X
+143,Namibia,S,1,0,143,2013-01-30,2013-11-27 00:14:36,USER_X,USER_X
+115,Latvia,S,1,0,115,2013-01-30,2013-11-27 00:14:36,USER_X,USER_X
+206,Israel,S,26,0,100,2003-10-04,2013-11-27 00:14:36,USER_X,USER_X
+216,Singapore,S,39,0,180,2003-10-04,2013-11-27 00:14:36,USER_X,USER_X
+105,Jersey,S,1,0,105,2013-01-29,2013-11-27 00:14:36,USER_X,USER_X
+159,Papua New Guinea,S,1,0,159,2013-01-30,2013-11-27 00:14:36,USER_X,USER_X
+220,Vietnam,S,47,0,216,2003-10-04,2013-11-27 00:14:36,USER_X,USER_X
+241,Tonga,S,1,0,201,2013-01-30,2013-11-27 00:14:36,USER_X,USER_X
+201,Hong Kong,S,22,0,92,2003-10-04,2013-11-27 00:14:36,USER_X,USER_X
+117,Lesotho,S,1,0,117,2013-02-22,2013-11-27 00:14:36,USER_X,USER_X
+258,Zambia,S,1,0,223,2013-01-30,2013-11-27 00:14:36,USER_X,USER_X
+237,Reunion,S,1,0,227,2013-01-30,2013-11-27 00:14:36,USER_X,USER_X
+239,Tanzania,S,1,0,198,2013-01-30,2013-11-27 00:14:36,USER_X,USER_X
+971,Butterfields,D,1,0,-999,2000-08-30,,USER_G,USER_X
+136,Mexico,S,1,0,136,2012-04-27,2013-11-27 00:14:36,USER_X,USER_X
+50,Cook Islands,S,1,0,50,2013-01-29,2013-11-27 00:14:36,USER_X,USER_X
+193,Switzerland,S,13,0,193,2002-08-30,2013-11-27 00:14:36,USER_S,USER_X
+989,AutoTrader,D,1,0,-999,2000-06-29,,USER_G,USER_X
+16,Austria,S,7,0,16,2002-06-11,2013-11-27 00:14:36,USER_S,USER_X
+30,Botswana,S,1,0,30,2013-01-29,2013-11-27 00:14:36,USER_X,USER_X
+178,Seychelles,S,1,0,178,2013-01-30,2013-11-27 00:14:36,USER_X,USER_X
+152,Nigeria,S,1,0,152,2013-01-30,2013-11-27 00:14:36,USER_X,USER_X
+9,Angola,S,1,0,9,2013-01-29,2013-11-27 00:14:36,USER_X,USER_X
+23,Belgium (French),S,7,1,23,2001-12-27,2013-11-27 00:14:36,USER_S,USER_X
+987,Card,D,1,0,-999,2000-06-29,,USER_G,USER_X
+91,Honduras,S,1,0,91,2013-01-29,2013-11-27 00:14:36,USER_X,USER_X
+232,Ukraine,S,1,0,209,2012-04-27,2013-11-27 00:14:36,USER_X,USER_X
+89,Guyana,S,1,0,89,2013-01-29,2013-11-27 00:14:36,USER_X,USER_X
+129,Mali,S,1,0,129,2013-01-30,2013-11-27 00:14:36,USER_X,USER_X
+234,Bangladesh,S,1,0,20,2013-01-29,2013-11-27 00:14:36,USER_X,USER_X
+-5,Gitti gidiyor,S,49,0,204,2012-11-07,,USER_C4,USER_X
+26,Bermuda,S,1,0,26,2013-01-29,2013-11-27 00:14:36,USER_X,USER_X
+49,"Congo Republic Of The",S,1,0,49,2013-01-29,2013-11-27 00:14:36,USER_X,USER_X
+148,New Caledonia,S,1,0,148,2013-01-30,2013-11-27 00:14:36,USER_X,USER_X
+66,Estonia,S,1,0,66,2013-01-29,2013-11-27 00:14:36,USER_X,USER_X
+5,Albania,S,1,0,5,2000-07-26,2013-11-27 00:14:36,USER_G,USER_X
+108,Kenya,S,1,0,108,2013-01-29,2013-11-27 00:14:36,USER_X,USER_X
+983,ICQ,D,1,0,-999,2000-07-07,,USER_PFI,USER_X
+167,Romania,S,1,0,167,2012-04-27,2013-11-27 00:14:36,USER_X,USER_X
+169,Rwanda,S,1,0,169,2013-01-30,2013-11-27 00:14:36,USER_X,USER_X
+228,Montenegro,S,1,0,228,2013-01-30,2013-11-27 00:14:36,USER_X,USER_X
+106,Jordan,S,1,0,106,2013-01-29,2013-11-27 00:14:36,USER_X,USER_X
+213,Purtugal,S,7,0,164,2003-10-04,2013-11-27 00:14:36,USER_X,USER_X
+207,Malaysia,S,46,0,127,2003-10-04,2013-11-27 00:14:36,USER_X,USER_X
+68,Falkland Islands (Islas Makvinas),S,1,0,68,2013-01-29,2013-11-27 00:14:36,USER_X,USER_X
+247,Uganda,S,1,0,208,2013-01-30,2013-11-27 00:14:36,USER_X,USER_X
+125,Madagascar,S,1,0,125,2013-01-30,2013-11-27 00:14:36,USER_X,USER_X
+43,Chad,S,1,0,43,2013-01-29,2013-11-27 00:14:36,USER_X,USER_X
+219,Thailand,S,40,0,199,2003-10-04,2013-11-27 00:14:36,USER_X,USER_X
+123,Belgium (Dutch),S,7,1,23,2001-12-27,2013-11-27 00:14:36,USER_S,USER_X
+15,Australia,S,5,1,15,2000-06-09,2013-11-27 00:14:36,USER_S,USER_X
+144,Nauru,S,1,0,144,2013-01-30,2013-11-27 00:14:36,USER_X,USER_X
+81,Greenland,S,1,0,81,2013-01-29,2013-11-27 00:14:36,USER_X,USER_X
+142,Mozambique,S,1,0,142,2013-01-30,2013-11-27 00:14:36,USER_X,USER_X
+104,Japan,S,6,0,104,2000-06-09,2013-11-27 00:14:36,USER_S,USER_X
+102,Jamaica,S,1,0,102,2013-01-29,2013-11-27 00:14:36,USER_X,USER_X
+186,Spain,S,7,0,186,2001-10-20,2013-11-27 00:14:36,USER_S,USER_X
+100,Ebaymotors,S,1,0,1,2000-06-09,2013-11-27 00:14:36,USER_S,USER_X
+116,"Lebanon South",S,1,0,116,2013-01-30,2013-11-27 00:14:36,USER_X,USER_X
+76,Georgia,S,1,0,76,2013-01-29,2013-11-27 00:14:36,USER_X,USER_X
+137,Moldova,S,1,0,137,2013-01-30,2013-11-27 00:14:36,USER_X,USER_X
+79,Gibraltar,S,1,0,79,2013-01-29,2013-11-27 00:14:36,USER_X,USER_X
+87,Guinea,S,1,0,87,2013-01-29,2013-11-27 00:14:36,USER_X,USER_X
+996,Stamps,D,1,0,-999,2000-06-29,,USER_G,USER_X
+160,Paraguay,S,1,0,160,2013-01-30,2013-11-27 00:14:36,USER_X,USER_X
+240,Togo,S,1,0,200,2013-01-30,2013-11-27 00:14:36,USER_X,USER_X
+203,India,S,44,0,95,2003-10-04,2013-11-27 00:14:36,USER_X,USER_X
+3,英国,S,3,1,3,2000-06-09,2013-11-27 00:14:36,USER_S,USER_X
+198,Denmark,S,17,0,57,2003-10-04,2013-11-27 00:14:36,USER_X,USER_X
+37,Burundi,S,1,0,37,2001-09-07,2013-11-27 00:14:36,USER_DNEL,USER_X
+20,Ebay Half Integration,S,1,0,1,2002-03-21,2013-11-27 00:14:36,USER_S,USER_X
+55,Cyprus,S,1,0,55,2013-01-29,2013-11-27 00:14:36,USER_X,USER_X
+259,Zimbabwe,S,1,0,224,2013-01-30,2013-11-27 00:14:36,USER_X,USER_X
+120,Liechtenstein,S,1,0,120,2013-01-30,2013-11-27 00:14:36,USER_X,USER_X
+156,Pakistan,S,1,0,156,2013-01-30,2013-11-27 00:14:36,USER_X,USER_X
+205,Ireland,S,7,0,99,2003-10-04,2013-11-27 00:14:36,USER_X,USER_X
+998,Computers,D,1,0,-999,2000-06-29,,USER_G,USER_X
+157,Palau,S,1,0,157,2013-01-30,2013-11-27 00:14:36,USER_X,USER_X
+182,Slovenia,S,1,0,182,2013-01-30,2013-11-27 00:14:36,USER_X,USER_X
+35,Burkina Faso,S,1,0,35,2013-01-29,2013-11-27 00:14:36,USER_X,USER_X
+199,Finland,S,7,0,70,2003-10-04,2013-11-27 00:14:36,USER_X,USER_X
+197,Czech Republic,S,16,0,56,2003-10-04,2013-11-27 00:14:36,USER_X,USER_X
+138,Monaco,S,1,0,138,2013-01-30,2013-11-27 00:14:36,USER_X,USER_X
+985,AOL Cobrand,D,1,0,-999,2000-06-29,,USER_G,USER_X
+75,Gambia,S,1,0,75,2013-01-29,2013-11-27 00:14:36,USER_X,USER_X
+14,Aruba,S,1,0,14,2013-01-29,2013-11-27 00:14:36,USER_X,USER_X
+218,Sweden,S,38,0,192,2003-10-04,2013-11-27 00:14:36,USER_X,USER_X
+24,Belize,S,1,0,24,2013-01-29,2013-11-27 00:14:36,USER_X,USER_X
+217,South Africa,S,43,0,185,2003-10-04,2013-11-27 00:14:36,USER_X,USER_X
+970,Butterfields Kiss,D,1,0,-999,2000-08-30,,USER_G,USER_X
+17,Azerbaijan Republic,S,1,0,17,2013-01-29,2013-11-27 00:14:36,USER_X,USER_X
+993,Percelain,D,1,0,-999,2000-06-12,,USER_G,USER_X
+211,Philippines,S,34,0,162,2003-10-04,2013-11-27 00:14:36,USER_X,USER_X
+155,Oman,S,1,0,155,2013-01-30,2013-11-27 00:14:36,USER_X,USER_X
+73,French Polynesia,S,1,0,73,2013-01-29,2013-11-27 00:14:36,USER_X,USER_X
+53,"Croatia Democratic Republic Of The",S,1,0,53,2013-01-29,2013-11-27 00:14:36,USER_X,USER_X
+134,Mauritius,S,1,0,134,2013-01-30,2013-11-27 00:14:36,USER_X,USER_X
+113,Kyrgyzstan,S,1,0,113,2013-01-30,2013-11-27 00:14:36,USER_X,USER_X
+131,Marshall Islands,S,1,0,131,2013-01-30,2013-11-27 00:14:36,USER_X,USER_X
+992,Jewelry,D,1,0,-999,2000-06-29,,USER_G,USER_X
+174,Saint Vincent And The Grenadines,S,1,0,174,2013-01-30,2013-11-27 00:14:36,USER_X,USER_X
+22,Belarus,S,1,0,22,2013-01-29,2013-11-27 00:14:36,USER_X,USER_X
+256,Western Samoa,S,1,0,220,2013-01-30,2013-11-27 00:14:36,USER_X,USER_X
+52,Cote D Ivoire (Ivory Coast),S,1,0,52,2013-01-29,2013-11-27 00:14:36,USER_X,USER_X
+175,San Marino,S,1,0,175,2013-01-30,2013-11-27 00:14:36,USER_X,USER_X
+-3,Tradera,S,7,0,192,2009-02-11,,USER_OFF,USER_X
+200,Greece,S,7,0,80,2003-10-04,2013-11-27 00:14:36,USER_X,USER_X
+32,British Virgin Islands,S,1,0,32,2013-01-29,2013-11-27 00:14:36,USER_X,USER_X
+171,Saint Kitts-Nevis,S,1,0,171,2013-01-30,2013-11-27 00:14:36,USER_X,USER_X
+176,Saudi Arabia,S,1,0,176,2012-04-27,2013-11-27 00:14:36,USER_X,USER_X
+19,Bahrain,S,1,0,19,2013-01-29,2013-11-27 00:14:36,USER_X,USER_X
+101,Italy,S,7,1,101,2000-11-03,2013-11-27 00:14:36,USER_S,USER_X
+236,Micronesia,S,1,0,226,2013-01-30,2013-11-27 00:14:36,USER_X,USER_X
+64,Equatorial Guinea,S,1,0,64,2013-01-29,2013-11-27 00:14:36,USER_X,USER_X
+209,Norway,S,32,0,154,2003-10-04,2013-11-27 00:14:36,USER_X,USER_X
+195,Tahiti,S,1,0,195,2013-01-30,2013-11-27 00:14:36,USER_X,USER_X
+221,Us1,S,1,0,1,2003-10-04,2013-11-27 00:14:36,USER_X,USER_X
+146,Netherlands,S,7,1,146,2001-07-12,2013-11-27 00:14:36,USER_S,USER_X
+214,Puerto Rico,S,1,0,165,2003-10-04,2013-11-27 00:14:36,USER_X,USER_X
+29,Bosnia And Herzegovina,S,1,0,29,2013-01-29,2013-11-27 00:14:36,USER_X,USER_X
+215,Russia,S,84,0,168,2003-10-04,2013-11-27 00:14:36,USER_X,USER_X
+90,Haiti,S,1,0,90,2013-01-29,2013-11-27 00:14:36,USER_X,USER_X
+12,Argentina,S,1,0,12,2012-04-27,2013-11-27 00:14:36,USER_X,USER_X
+158,Panama,S,1,0,158,2013-01-30,2013-11-27 00:14:36,USER_X,USER_X
+139,Mongolia,S,1,0,139,2013-01-30,2013-11-27 00:14:36,USER_X,USER_X
+41,Cayman Islands,S,1,0,41,2013-01-29,2013-11-27 00:14:36,USER_X,USER_X
+151,Niger,S,1,0,151,2013-01-30,2013-11-27 00:14:36,USER_X,USER_X
+11,Antigua And Barbuda,S,1,0,11,2013-01-29,2013-11-27 00:14:36,USER_X,USER_X
+-1,Korea Auction.co.kr,S,48,0,111,2009-02-11,,USER_OFF,USER_X
+112,Kuwait,S,1,0,112,2013-01-29,2013-11-27 00:14:36,USER_X,USER_X
+59,Dominica,S,1,0,59,2013-01-29,2013-11-27 00:14:36,USER_X,USER_X
+179,Sierra Leone,S,1,0,179,2013-01-30,2013-11-27 00:14:36,USER_X,USER_X
+132,Martinique,S,1,0,132,2013-01-30,2013-11-27 00:14:36,USER_X,USER_X
+972,Hoianhoard,D,1,0,-999,2000-07-26,,USER_G,USER_X
+257,Yemen,S,1,0,221,2013-01-30,2013-11-27 00:14:36,USER_X,USER_X
+235,Macau,S,1,0,123,2013-01-30,2013-11-27 00:14:36,USER_X,USER_X
+8,Andorra,S,1,0,8,2000-07-26,2013-11-27 00:14:36,USER_G,USER_X
+191,Swaziland,S,1,0,191,2013-01-30,2013-11-27 00:14:36,USER_X,USER_X
+141,Morocco,S,1,0,141,2013-01-30,2013-11-27 00:14:36,USER_X,USER_X
+177,Senegal,S,1,0,177,2013-01-30,2013-11-27 00:14:36,USER_X,USER_X
+184,Somalia,S,1,0,184,2013-01-30,2013-11-27 00:14:36,USER_X,USER_X
+47,Comoros,S,1,0,47,2013-01-29,2013-11-27 00:14:36,USER_X,USER_X
+212,Poland,S,35,0,163,2003-10-04,2013-11-27 00:14:36,USER_X,USER_X
+994,Sports,D,1,0,-999,2000-07-07,,USER_PFI,USER_X
+34,Bulgaria,S,1,0,34,2013-01-29,2013-11-27 00:14:36,USER_X,USER_X
+-2,Bazee,S,44,0,95,2009-02-11,,USER_OFF,USER_X
+224,Japan Row,S,6,0,104,2003-10-04,2013-11-27 00:14:36,USER_X,USER_X
+-999,Unknown,S,-999,0,-999,2000-06-29,,USER_G,USER_X
+986,GO Cobrand,D,1,0,-999,2000-06-29,,USER_G,USER_X
+133,Mauritania,S,1,0,133,2013-01-30,2013-11-27 00:14:36,USER_X,USER_X
+196,Taiwan,S,41,0,196,2002-03-08,2013-11-27 00:14:36,USER_S,USER_X
+33,Brunei Darussalam,S,1,0,33,2013-01-29,2013-11-27 00:14:36,USER_X,USER_X
+-7,StubHub UK,S,3,1,3,2012-11-08,,USER_C4,USER_X
+140,Montserrat,S,1,0,140,2013-01-30,2013-11-27 00:14:36,USER_X,USER_X
+990,Toys,D,1,0,-999,2000-06-29,,USER_G,USER_X
+150,Nicaragua,S,1,0,150,2013-01-30,2013-11-27 00:14:36,USER_X,USER_X
+244,Turkmenistan,S,1,0,205,2013-01-30,2013-11-27 00:14:36,USER_X,USER_X
+78,Ghana,S,1,0,78,2013-01-29,2013-11-27 00:14:36,USER_X,USER_X
+973,Developer.ebay.com,D,1,0,-999,2000-10-06,2000-10-06 00:00:00,USER_G,USER_G
+202,Hungary,S,23,0,93,2003-10-04,2013-11-27 00:14:36,USER_X,USER_X
+242,Trinidad And Tobago,S,1,0,202,2013-01-30,2013-11-27 00:14:36,USER_X,USER_X
+981,Compuserve Cobrand,D,1,0,-999,2000-06-29,,USER_G,USER_X
+60,Dominican Republic,S,1,0,60,2013-01-29,2013-11-27 00:14:36,USER_X,USER_X
+13,Armenia,S,1,0,13,2013-01-29,2013-11-27 00:14:36,USER_X,USER_X
+991,Dolls,D,1,0,-999,2000-06-29,,USER_G,USER_X
+118,Liberia,S,1,0,118,2013-01-30,2013-11-27 00:14:36,USER_X,USER_X
+77,Germany,S,7,1,77,2000-06-09,2013-11-27 00:14:36,USER_S,USER_X
+51,Costa Rica,S,1,0,51,2013-01-29,2013-11-27 00:14:36,USER_X,USER_X
+1,Usa,S,1,1,1,2000-06-09,2013-11-27 00:14:36,USER_S,USER_X
+39,Cameroon,S,1,0,39,2013-01-29,2013-11-27 00:14:36,USER_X,USER_X
+997,Great Collections,D,1,0,-999,2000-06-29,,USER_G,USER_X
+181,Slovak Republic,S,1,0,181,2012-04-27,2013-11-27 00:14:36,USER_X,USER_X
+189,Suriname,S,1,0,189,2013-01-30,2013-11-27 00:14:36,USER_X,USER_X
+74,Gabon Republic,S,1,0,74,2013-01-29,2013-11-27 00:14:36,USER_X,USER_X
+31,Brazil,S,1,0,31,2012-04-27,2013-11-27 00:14:36,USER_X,USER_X
+38,Cambodia,S,1,0,38,2013-01-29,2013-11-27 00:14:36,USER_X,USER_X
+245,Turks And Caicos Islands,S,1,0,206,2013-01-30,2013-11-27 00:14:36,USER_X,USER_X
+28,Bolivia,S,1,0,28,2013-01-29,2013-11-27 00:14:36,USER_X,USER_X
+210,Canada (French),S,2,0,2,2003-10-04,2013-11-27 00:14:36,USER_X,USER_X
+238,Tajikistan,S,1,0,197,2013-01-30,2013-11-27 00:14:36,USER_X,USER_X
+58,Djibouti,S,1,0,58,2013-01-29,2013-11-27 00:14:36,USER_X,USER_X
+135,Mayotte,S,1,0,135,2013-01-30,2013-11-27 00:14:36,USER_X,USER_X
+995,Coins,D,1,0,-999,2000-06-29,,USER_G,USER_X
+114,Laos,S,1,0,114,2013-01-30,2013-11-27 00:14:36,USER_X,USER_X
+173,Saint Pierre And Miquelon,S,1,0,173,2013-01-30,2013-11-27 00:14:36,USER_X,USER_X
+83,Guadeloupe,S,1,0,83,2013-01-29,2013-11-27 00:14:36,USER_X,USER_X
+62,Egypt,S,1,0,62,2013-01-29,2013-11-27 00:14:36,USER_X,USER_X
+122,Luxembourg,S,1,0,122,2013-01-30,2013-11-27 00:14:36,USER_X,USER_X
+72,French Guiana,S,1,0,72,2013-01-29,2013-11-27 00:14:36,USER_X,USER_X
+223,China,S,14,0,45,2003-10-04,2013-11-27 00:14:36,USER_X,USER_X
+163,Unknown: Added by DW(0),S,-999,0,-999,2001-09-07,,USER_DNEL,USER_X
+40,Cape Verde Islands,S,1,0,40,2013-01-29,2013-11-27 00:14:36,USER_X,USER_X
+222,Yugoslavia,S,1,0,222,2013-03-12,2013-11-27 00:14:36,USER_X,USER_X
+61,Ecuador,S,1,0,61,2013-01-29,2013-11-27 00:14:36,USER_X,USER_X
+243,Tunisia,S,1,0,203,2013-01-30,2013-11-27 00:14:36,USER_X,USER_X
+183,Solomon Islands,S,1,0,183,2013-01-30,2013-11-27 00:14:36,USER_X,USER_X
+226,South Korea,S,1,0,111,2012-04-27,2013-11-27 00:14:36,USER_X,USER_X
+7,American Samoa,S,1,0,7,2000-07-26,2013-11-27 00:14:36,USER_G,USER_X
+121,Lithuania,S,1,0,121,2013-01-30,2013-11-27 00:14:36,USER_X,USER_X
+18,Bahamas,S,1,0,18,2013-01-29,2013-11-27 00:14:36,USER_X,USER_X
+0,Ebay,S,1,0,1,2000-06-09,2013-11-27 00:14:36,USER_S,USER_X
+-8,Wholesale Deals -- US,S,1,1,1,2013-02-19,,USER_YAJ,USER_X
+250,Vanuatu,S,1,0,213,2013-01-30,2013-11-27 00:14:36,USER_X,USER_X
+42,Central African Republic,S,1,0,42,2013-01-29,2013-11-27 00:14:36,USER_X,USER_X
+252,Venezuela,S,1,0,215,2013-01-30,2013-11-27 00:14:36,USER_X,USER_X
+109,Kiribati,S,1,0,109,2013-01-29,2013-11-27 00:14:36,USER_X,USER_X
+130,Malta,S,1,0,130,2013-01-30,2013-11-27 00:14:36,USER_X,USER_X
+251,Vatican City State,S,1,0,214,2013-01-30,2013-11-27 00:14:36,USER_X,USER_X
+253,Virgin Islands (U.S.),S,1,0,217,2013-01-30,2013-11-27 00:14:36,USER_X,USER_X
+500,Unknown: Added by DW(1),S,-999,0,-999,2000-07-26,,USER_G,USER_X
+255,Western Sahara,S,1,0,219,2013-01-30,2013-11-27 00:14:36,USER_X,USER_X
+190,Svalbard,S,1,0,190,2013-01-30,2013-11-27 00:14:36,USER_X,USER_X
+85,Guatemala,S,1,0,85,2013-01-29,2013-11-27 00:14:36,USER_X,USER_X
+254,Wallis And Futuna,S,1,0,218,2013-01-30,2013-11-27 00:14:36,USER_X,USER_X
+27,Bhutan,S,1,0,27,2013-01-29,2013-11-27 00:14:36,USER_X,USER_X
+249,Uzbekistan,S,1,0,212,2013-01-30,2013-11-27 00:14:36,USER_X,USER_X

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/examples/test_case_data/localmeta/data/data_gen_config.json
----------------------------------------------------------------------
diff --git a/examples/test_case_data/localmeta/data/data_gen_config.json b/examples/test_case_data/localmeta/data/data_gen_config.json
new file mode 100644
index 0000000..a19aeae
--- /dev/null
+++ b/examples/test_case_data/localmeta/data/data_gen_config.json
@@ -0,0 +1,29 @@
+{
+"columnConfigs":
+[
+    {
+        "columnName" : "lstg_format_name",
+        "valueSet"   : 
+            [
+                "FP-GTC",
+                "FP-non GTC",
+                "ABIN",
+                "Auction",
+                "Others"
+            ],
+        "exclusive":true
+    },
+    
+    {
+        "columnName" : "SELLER_ID",
+        "valueSet"   : 
+            [
+                "10000000" ,
+                "10001000"
+            ],
+        "asRange":true
+    }
+]
+
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/examples/test_case_data/localmeta/dict/PREDEFINED/date(yyyy-mm-dd)/64ac4f82-f2af-476e-85b9-f0805001014e.dict
----------------------------------------------------------------------
diff --git a/examples/test_case_data/localmeta/dict/PREDEFINED/date(yyyy-mm-dd)/64ac4f82-f2af-476e-85b9-f0805001014e.dict b/examples/test_case_data/localmeta/dict/PREDEFINED/date(yyyy-mm-dd)/64ac4f82-f2af-476e-85b9-f0805001014e.dict
new file mode 100644
index 0000000..36da380
Binary files /dev/null and b/examples/test_case_data/localmeta/dict/PREDEFINED/date(yyyy-mm-dd)/64ac4f82-f2af-476e-85b9-f0805001014e.dict differ

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/examples/test_case_data/localmeta/dict/PREDEFINED/date(yyyy-mm-dd)/f5e85644-db92-42b5-9ad5-240ab227d7b0.dict
----------------------------------------------------------------------
diff --git a/examples/test_case_data/localmeta/dict/PREDEFINED/date(yyyy-mm-dd)/f5e85644-db92-42b5-9ad5-240ab227d7b0.dict b/examples/test_case_data/localmeta/dict/PREDEFINED/date(yyyy-mm-dd)/f5e85644-db92-42b5-9ad5-240ab227d7b0.dict
new file mode 100644
index 0000000..5c9af56
Binary files /dev/null and b/examples/test_case_data/localmeta/dict/PREDEFINED/date(yyyy-mm-dd)/f5e85644-db92-42b5-9ad5-240ab227d7b0.dict differ

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/examples/test_case_data/localmeta/dict/TEST_CAL_DT/CAL_DT/aa634e3e-22d3-4cc2-9de5-085e9ac35c1f.dict
----------------------------------------------------------------------
diff --git a/examples/test_case_data/localmeta/dict/TEST_CAL_DT/CAL_DT/aa634e3e-22d3-4cc2-9de5-085e9ac35c1f.dict b/examples/test_case_data/localmeta/dict/TEST_CAL_DT/CAL_DT/aa634e3e-22d3-4cc2-9de5-085e9ac35c1f.dict
new file mode 100644
index 0000000..6fc3127
Binary files /dev/null and b/examples/test_case_data/localmeta/dict/TEST_CAL_DT/CAL_DT/aa634e3e-22d3-4cc2-9de5-085e9ac35c1f.dict differ

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/examples/test_case_data/localmeta/dict/TEST_CAL_DT/CAL_DT/ed0c3451-593c-494c-9019-64f63fcb0b8e.dict
----------------------------------------------------------------------
diff --git a/examples/test_case_data/localmeta/dict/TEST_CAL_DT/CAL_DT/ed0c3451-593c-494c-9019-64f63fcb0b8e.dict b/examples/test_case_data/localmeta/dict/TEST_CAL_DT/CAL_DT/ed0c3451-593c-494c-9019-64f63fcb0b8e.dict
new file mode 100644
index 0000000..944f478
Binary files /dev/null and b/examples/test_case_data/localmeta/dict/TEST_CAL_DT/CAL_DT/ed0c3451-593c-494c-9019-64f63fcb0b8e.dict differ

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/examples/test_case_data/localmeta/dict/TEST_CATEGORY_GROUPINGS/CATEG_LVL2_NAME/16d8185c-ee6b-4f8c-a919-756d9809f937.dict
----------------------------------------------------------------------
diff --git a/examples/test_case_data/localmeta/dict/TEST_CATEGORY_GROUPINGS/CATEG_LVL2_NAME/16d8185c-ee6b-4f8c-a919-756d9809f937.dict b/examples/test_case_data/localmeta/dict/TEST_CATEGORY_GROUPINGS/CATEG_LVL2_NAME/16d8185c-ee6b-4f8c-a919-756d9809f937.dict
new file mode 100644
index 0000000..a2cb760
Binary files /dev/null and b/examples/test_case_data/localmeta/dict/TEST_CATEGORY_GROUPINGS/CATEG_LVL2_NAME/16d8185c-ee6b-4f8c-a919-756d9809f937.dict differ

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/examples/test_case_data/localmeta/dict/TEST_CATEGORY_GROUPINGS/CATEG_LVL2_NAME/9bb0b83a-97b2-434f-905f-4d9e04b62018.dict
----------------------------------------------------------------------
diff --git a/examples/test_case_data/localmeta/dict/TEST_CATEGORY_GROUPINGS/CATEG_LVL2_NAME/9bb0b83a-97b2-434f-905f-4d9e04b62018.dict b/examples/test_case_data/localmeta/dict/TEST_CATEGORY_GROUPINGS/CATEG_LVL2_NAME/9bb0b83a-97b2-434f-905f-4d9e04b62018.dict
new file mode 100644
index 0000000..08dd48b
Binary files /dev/null and b/examples/test_case_data/localmeta/dict/TEST_CATEGORY_GROUPINGS/CATEG_LVL2_NAME/9bb0b83a-97b2-434f-905f-4d9e04b62018.dict differ

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/examples/test_case_data/localmeta/dict/TEST_CATEGORY_GROUPINGS/CATEG_LVL2_NAME/c12ae49d-9dbe-4a58-b169-19afac317696.dict
----------------------------------------------------------------------
diff --git a/examples/test_case_data/localmeta/dict/TEST_CATEGORY_GROUPINGS/CATEG_LVL2_NAME/c12ae49d-9dbe-4a58-b169-19afac317696.dict b/examples/test_case_data/localmeta/dict/TEST_CATEGORY_GROUPINGS/CATEG_LVL2_NAME/c12ae49d-9dbe-4a58-b169-19afac317696.dict
new file mode 100644
index 0000000..e56eefb
Binary files /dev/null and b/examples/test_case_data/localmeta/dict/TEST_CATEGORY_GROUPINGS/CATEG_LVL2_NAME/c12ae49d-9dbe-4a58-b169-19afac317696.dict differ

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/examples/test_case_data/localmeta/dict/TEST_CATEGORY_GROUPINGS/CATEG_LVL2_NAME/eaed91b0-4182-4ee5-a733-1047a622ee29.dict
----------------------------------------------------------------------
diff --git a/examples/test_case_data/localmeta/dict/TEST_CATEGORY_GROUPINGS/CATEG_LVL2_NAME/eaed91b0-4182-4ee5-a733-1047a622ee29.dict b/examples/test_case_data/localmeta/dict/TEST_CATEGORY_GROUPINGS/CATEG_LVL2_NAME/eaed91b0-4182-4ee5-a733-1047a622ee29.dict
new file mode 100644
index 0000000..efcacde
Binary files /dev/null and b/examples/test_case_data/localmeta/dict/TEST_CATEGORY_GROUPINGS/CATEG_LVL2_NAME/eaed91b0-4182-4ee5-a733-1047a622ee29.dict differ

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/examples/test_case_data/localmeta/dict/TEST_CATEGORY_GROUPINGS/CATEG_LVL2_NAME/ec83ebce-7534-4e62-ac6d-7445ee141ab4.dict
----------------------------------------------------------------------
diff --git a/examples/test_case_data/localmeta/dict/TEST_CATEGORY_GROUPINGS/CATEG_LVL2_NAME/ec83ebce-7534-4e62-ac6d-7445ee141ab4.dict b/examples/test_case_data/localmeta/dict/TEST_CATEGORY_GROUPINGS/CATEG_LVL2_NAME/ec83ebce-7534-4e62-ac6d-7445ee141ab4.dict
new file mode 100644
index 0000000..95cf51c
Binary files /dev/null and b/examples/test_case_data/localmeta/dict/TEST_CATEGORY_GROUPINGS/CATEG_LVL2_NAME/ec83ebce-7534-4e62-ac6d-7445ee141ab4.dict differ

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/examples/test_case_data/localmeta/dict/TEST_CATEGORY_GROUPINGS/CATEG_LVL3_NAME/270fbfb0-281c-4602-8413-2970a7439c47.dict
----------------------------------------------------------------------
diff --git a/examples/test_case_data/localmeta/dict/TEST_CATEGORY_GROUPINGS/CATEG_LVL3_NAME/270fbfb0-281c-4602-8413-2970a7439c47.dict b/examples/test_case_data/localmeta/dict/TEST_CATEGORY_GROUPINGS/CATEG_LVL3_NAME/270fbfb0-281c-4602-8413-2970a7439c47.dict
new file mode 100644
index 0000000..c3caf94
Binary files /dev/null and b/examples/test_case_data/localmeta/dict/TEST_CATEGORY_GROUPINGS/CATEG_LVL3_NAME/270fbfb0-281c-4602-8413-2970a7439c47.dict differ

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/examples/test_case_data/localmeta/dict/TEST_CATEGORY_GROUPINGS/CATEG_LVL3_NAME/7228fad8-a764-4bd4-8934-50e0d7cbcb19.dict
----------------------------------------------------------------------
diff --git a/examples/test_case_data/localmeta/dict/TEST_CATEGORY_GROUPINGS/CATEG_LVL3_NAME/7228fad8-a764-4bd4-8934-50e0d7cbcb19.dict b/examples/test_case_data/localmeta/dict/TEST_CATEGORY_GROUPINGS/CATEG_LVL3_NAME/7228fad8-a764-4bd4-8934-50e0d7cbcb19.dict
new file mode 100644
index 0000000..823c00d
Binary files /dev/null and b/examples/test_case_data/localmeta/dict/TEST_CATEGORY_GROUPINGS/CATEG_LVL3_NAME/7228fad8-a764-4bd4-8934-50e0d7cbcb19.dict differ

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/examples/test_case_data/localmeta/dict/TEST_CATEGORY_GROUPINGS/CATEG_LVL3_NAME/8f92faa4-7d2d-455c-8623-6e1d1b272afe.dict
----------------------------------------------------------------------
diff --git a/examples/test_case_data/localmeta/dict/TEST_CATEGORY_GROUPINGS/CATEG_LVL3_NAME/8f92faa4-7d2d-455c-8623-6e1d1b272afe.dict b/examples/test_case_data/localmeta/dict/TEST_CATEGORY_GROUPINGS/CATEG_LVL3_NAME/8f92faa4-7d2d-455c-8623-6e1d1b272afe.dict
new file mode 100644
index 0000000..5236f78
Binary files /dev/null and b/examples/test_case_data/localmeta/dict/TEST_CATEGORY_GROUPINGS/CATEG_LVL3_NAME/8f92faa4-7d2d-455c-8623-6e1d1b272afe.dict differ

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/examples/test_case_data/localmeta/dict/TEST_CATEGORY_GROUPINGS/CATEG_LVL3_NAME/ad09f2d5-054a-4e1b-a776-7cc07399a6c1.dict
----------------------------------------------------------------------
diff --git a/examples/test_case_data/localmeta/dict/TEST_CATEGORY_GROUPINGS/CATEG_LVL3_NAME/ad09f2d5-054a-4e1b-a776-7cc07399a6c1.dict b/examples/test_case_data/localmeta/dict/TEST_CATEGORY_GROUPINGS/CATEG_LVL3_NAME/ad09f2d5-054a-4e1b-a776-7cc07399a6c1.dict
new file mode 100644
index 0000000..bc1d8e8
Binary files /dev/null and b/examples/test_case_data/localmeta/dict/TEST_CATEGORY_GROUPINGS/CATEG_LVL3_NAME/ad09f2d5-054a-4e1b-a776-7cc07399a6c1.dict differ

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/examples/test_case_data/localmeta/dict/TEST_CATEGORY_GROUPINGS/CATEG_LVL3_NAME/eacccee7-d120-4f4d-97d0-c99a5b83ec32.dict
----------------------------------------------------------------------
diff --git a/examples/test_case_data/localmeta/dict/TEST_CATEGORY_GROUPINGS/CATEG_LVL3_NAME/eacccee7-d120-4f4d-97d0-c99a5b83ec32.dict b/examples/test_case_data/localmeta/dict/TEST_CATEGORY_GROUPINGS/CATEG_LVL3_NAME/eacccee7-d120-4f4d-97d0-c99a5b83ec32.dict
new file mode 100644
index 0000000..0bbe66f
Binary files /dev/null and b/examples/test_case_data/localmeta/dict/TEST_CATEGORY_GROUPINGS/CATEG_LVL3_NAME/eacccee7-d120-4f4d-97d0-c99a5b83ec32.dict differ

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/examples/test_case_data/localmeta/dict/TEST_CATEGORY_GROUPINGS/LEAF_CATEG_ID/2602386c-debb-4968-8d2f-b52b8215e385.dict
----------------------------------------------------------------------
diff --git a/examples/test_case_data/localmeta/dict/TEST_CATEGORY_GROUPINGS/LEAF_CATEG_ID/2602386c-debb-4968-8d2f-b52b8215e385.dict b/examples/test_case_data/localmeta/dict/TEST_CATEGORY_GROUPINGS/LEAF_CATEG_ID/2602386c-debb-4968-8d2f-b52b8215e385.dict
new file mode 100644
index 0000000..1d25663
Binary files /dev/null and b/examples/test_case_data/localmeta/dict/TEST_CATEGORY_GROUPINGS/LEAF_CATEG_ID/2602386c-debb-4968-8d2f-b52b8215e385.dict differ

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/examples/test_case_data/localmeta/dict/TEST_CATEGORY_GROUPINGS/LEAF_CATEG_ID/4243889f-bc81-4807-a975-7041bbbf35e7.dict
----------------------------------------------------------------------
diff --git a/examples/test_case_data/localmeta/dict/TEST_CATEGORY_GROUPINGS/LEAF_CATEG_ID/4243889f-bc81-4807-a975-7041bbbf35e7.dict b/examples/test_case_data/localmeta/dict/TEST_CATEGORY_GROUPINGS/LEAF_CATEG_ID/4243889f-bc81-4807-a975-7041bbbf35e7.dict
new file mode 100644
index 0000000..a046dbb
Binary files /dev/null and b/examples/test_case_data/localmeta/dict/TEST_CATEGORY_GROUPINGS/LEAF_CATEG_ID/4243889f-bc81-4807-a975-7041bbbf35e7.dict differ

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/examples/test_case_data/localmeta/dict/TEST_CATEGORY_GROUPINGS/LEAF_CATEG_ID/494683c1-52d1-4d9a-a3f7-8c53b30fd898.dict
----------------------------------------------------------------------
diff --git a/examples/test_case_data/localmeta/dict/TEST_CATEGORY_GROUPINGS/LEAF_CATEG_ID/494683c1-52d1-4d9a-a3f7-8c53b30fd898.dict b/examples/test_case_data/localmeta/dict/TEST_CATEGORY_GROUPINGS/LEAF_CATEG_ID/494683c1-52d1-4d9a-a3f7-8c53b30fd898.dict
new file mode 100644
index 0000000..c1a4196
Binary files /dev/null and b/examples/test_case_data/localmeta/dict/TEST_CATEGORY_GROUPINGS/LEAF_CATEG_ID/494683c1-52d1-4d9a-a3f7-8c53b30fd898.dict differ

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/examples/test_case_data/localmeta/dict/TEST_CATEGORY_GROUPINGS/LEAF_CATEG_ID/8b4b1c06-fb74-486b-a2ad-74420afebcda.dict
----------------------------------------------------------------------
diff --git a/examples/test_case_data/localmeta/dict/TEST_CATEGORY_GROUPINGS/LEAF_CATEG_ID/8b4b1c06-fb74-486b-a2ad-74420afebcda.dict b/examples/test_case_data/localmeta/dict/TEST_CATEGORY_GROUPINGS/LEAF_CATEG_ID/8b4b1c06-fb74-486b-a2ad-74420afebcda.dict
new file mode 100644
index 0000000..b60e34f
Binary files /dev/null and b/examples/test_case_data/localmeta/dict/TEST_CATEGORY_GROUPINGS/LEAF_CATEG_ID/8b4b1c06-fb74-486b-a2ad-74420afebcda.dict differ

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/examples/test_case_data/localmeta/dict/TEST_CATEGORY_GROUPINGS/LEAF_CATEG_ID/ac520edd-f9d2-419f-a7de-587bfb97dc81.dict
----------------------------------------------------------------------
diff --git a/examples/test_case_data/localmeta/dict/TEST_CATEGORY_GROUPINGS/LEAF_CATEG_ID/ac520edd-f9d2-419f-a7de-587bfb97dc81.dict b/examples/test_case_data/localmeta/dict/TEST_CATEGORY_GROUPINGS/LEAF_CATEG_ID/ac520edd-f9d2-419f-a7de-587bfb97dc81.dict
new file mode 100644
index 0000000..2c47f75
Binary files /dev/null and b/examples/test_case_data/localmeta/dict/TEST_CATEGORY_GROUPINGS/LEAF_CATEG_ID/ac520edd-f9d2-419f-a7de-587bfb97dc81.dict differ

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/examples/test_case_data/localmeta/dict/TEST_CATEGORY_GROUPINGS/META_CATEG_NAME/0410d2c4-4686-40bc-ba14-170042a2de94.dict
----------------------------------------------------------------------
diff --git a/examples/test_case_data/localmeta/dict/TEST_CATEGORY_GROUPINGS/META_CATEG_NAME/0410d2c4-4686-40bc-ba14-170042a2de94.dict b/examples/test_case_data/localmeta/dict/TEST_CATEGORY_GROUPINGS/META_CATEG_NAME/0410d2c4-4686-40bc-ba14-170042a2de94.dict
new file mode 100644
index 0000000..c72d118
Binary files /dev/null and b/examples/test_case_data/localmeta/dict/TEST_CATEGORY_GROUPINGS/META_CATEG_NAME/0410d2c4-4686-40bc-ba14-170042a2de94.dict differ

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/examples/test_case_data/localmeta/dict/TEST_CATEGORY_GROUPINGS/META_CATEG_NAME/51ff5e6c-22b3-444e-9915-d376a10f20cb.dict
----------------------------------------------------------------------
diff --git a/examples/test_case_data/localmeta/dict/TEST_CATEGORY_GROUPINGS/META_CATEG_NAME/51ff5e6c-22b3-444e-9915-d376a10f20cb.dict b/examples/test_case_data/localmeta/dict/TEST_CATEGORY_GROUPINGS/META_CATEG_NAME/51ff5e6c-22b3-444e-9915-d376a10f20cb.dict
new file mode 100644
index 0000000..4fd22f8
Binary files /dev/null and b/examples/test_case_data/localmeta/dict/TEST_CATEGORY_GROUPINGS/META_CATEG_NAME/51ff5e6c-22b3-444e-9915-d376a10f20cb.dict differ

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/examples/test_case_data/localmeta/dict/TEST_CATEGORY_GROUPINGS/META_CATEG_NAME/70c8d6ef-f55b-4cdd-8be4-9c2b236cc8e9.dict
----------------------------------------------------------------------
diff --git a/examples/test_case_data/localmeta/dict/TEST_CATEGORY_GROUPINGS/META_CATEG_NAME/70c8d6ef-f55b-4cdd-8be4-9c2b236cc8e9.dict b/examples/test_case_data/localmeta/dict/TEST_CATEGORY_GROUPINGS/META_CATEG_NAME/70c8d6ef-f55b-4cdd-8be4-9c2b236cc8e9.dict
new file mode 100644
index 0000000..4e7a8cc
Binary files /dev/null and b/examples/test_case_data/localmeta/dict/TEST_CATEGORY_GROUPINGS/META_CATEG_NAME/70c8d6ef-f55b-4cdd-8be4-9c2b236cc8e9.dict differ

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/examples/test_case_data/localmeta/dict/TEST_CATEGORY_GROUPINGS/META_CATEG_NAME/a4e57e55-48fc-4f25-a9c8-485deed25925.dict
----------------------------------------------------------------------
diff --git a/examples/test_case_data/localmeta/dict/TEST_CATEGORY_GROUPINGS/META_CATEG_NAME/a4e57e55-48fc-4f25-a9c8-485deed25925.dict b/examples/test_case_data/localmeta/dict/TEST_CATEGORY_GROUPINGS/META_CATEG_NAME/a4e57e55-48fc-4f25-a9c8-485deed25925.dict
new file mode 100644
index 0000000..67dbea4
Binary files /dev/null and b/examples/test_case_data/localmeta/dict/TEST_CATEGORY_GROUPINGS/META_CATEG_NAME/a4e57e55-48fc-4f25-a9c8-485deed25925.dict differ

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/examples/test_case_data/localmeta/dict/TEST_CATEGORY_GROUPINGS/META_CATEG_NAME/ac797050-e152-4923-a20a-02282fcf6086.dict
----------------------------------------------------------------------
diff --git a/examples/test_case_data/localmeta/dict/TEST_CATEGORY_GROUPINGS/META_CATEG_NAME/ac797050-e152-4923-a20a-02282fcf6086.dict b/examples/test_case_data/localmeta/dict/TEST_CATEGORY_GROUPINGS/META_CATEG_NAME/ac797050-e152-4923-a20a-02282fcf6086.dict
new file mode 100644
index 0000000..a786b20
Binary files /dev/null and b/examples/test_case_data/localmeta/dict/TEST_CATEGORY_GROUPINGS/META_CATEG_NAME/ac797050-e152-4923-a20a-02282fcf6086.dict differ

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/examples/test_case_data/localmeta/dict/TEST_CATEGORY_GROUPINGS/META_CATEG_NAME/aceae914-4246-4251-a0c2-692fe7a300df.dict
----------------------------------------------------------------------
diff --git a/examples/test_case_data/localmeta/dict/TEST_CATEGORY_GROUPINGS/META_CATEG_NAME/aceae914-4246-4251-a0c2-692fe7a300df.dict b/examples/test_case_data/localmeta/dict/TEST_CATEGORY_GROUPINGS/META_CATEG_NAME/aceae914-4246-4251-a0c2-692fe7a300df.dict
new file mode 100644
index 0000000..df0c657
Binary files /dev/null and b/examples/test_case_data/localmeta/dict/TEST_CATEGORY_GROUPINGS/META_CATEG_NAME/aceae914-4246-4251-a0c2-692fe7a300df.dict differ

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/examples/test_case_data/localmeta/dict/TEST_CATEGORY_GROUPINGS/META_CATEG_NAME/b298089f-9656-4693-b9b2-8fea46f06dd5.dict
----------------------------------------------------------------------
diff --git a/examples/test_case_data/localmeta/dict/TEST_CATEGORY_GROUPINGS/META_CATEG_NAME/b298089f-9656-4693-b9b2-8fea46f06dd5.dict b/examples/test_case_data/localmeta/dict/TEST_CATEGORY_GROUPINGS/META_CATEG_NAME/b298089f-9656-4693-b9b2-8fea46f06dd5.dict
new file mode 100644
index 0000000..3a28967
Binary files /dev/null and b/examples/test_case_data/localmeta/dict/TEST_CATEGORY_GROUPINGS/META_CATEG_NAME/b298089f-9656-4693-b9b2-8fea46f06dd5.dict differ

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/examples/test_case_data/localmeta/dict/TEST_CATEGORY_GROUPINGS/META_CATEG_NAME/c2af25cf-6c79-45e6-a6f6-6d2a8ecc6592.dict
----------------------------------------------------------------------
diff --git a/examples/test_case_data/localmeta/dict/TEST_CATEGORY_GROUPINGS/META_CATEG_NAME/c2af25cf-6c79-45e6-a6f6-6d2a8ecc6592.dict b/examples/test_case_data/localmeta/dict/TEST_CATEGORY_GROUPINGS/META_CATEG_NAME/c2af25cf-6c79-45e6-a6f6-6d2a8ecc6592.dict
new file mode 100644
index 0000000..e898c45
Binary files /dev/null and b/examples/test_case_data/localmeta/dict/TEST_CATEGORY_GROUPINGS/META_CATEG_NAME/c2af25cf-6c79-45e6-a6f6-6d2a8ecc6592.dict differ

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/examples/test_case_data/localmeta/dict/TEST_KYLIN_FACT/CAL_DT/48433f91-0d68-495f-b7f2-295414591275.dict
----------------------------------------------------------------------
diff --git a/examples/test_case_data/localmeta/dict/TEST_KYLIN_FACT/CAL_DT/48433f91-0d68-495f-b7f2-295414591275.dict b/examples/test_case_data/localmeta/dict/TEST_KYLIN_FACT/CAL_DT/48433f91-0d68-495f-b7f2-295414591275.dict
new file mode 100644
index 0000000..557c66a
Binary files /dev/null and b/examples/test_case_data/localmeta/dict/TEST_KYLIN_FACT/CAL_DT/48433f91-0d68-495f-b7f2-295414591275.dict differ

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/examples/test_case_data/localmeta/dict/TEST_KYLIN_FACT/ITEM_COUNT/73c9bfe1-6496-4ff8-9467-6cbee2924c16.dict
----------------------------------------------------------------------
diff --git a/examples/test_case_data/localmeta/dict/TEST_KYLIN_FACT/ITEM_COUNT/73c9bfe1-6496-4ff8-9467-6cbee2924c16.dict b/examples/test_case_data/localmeta/dict/TEST_KYLIN_FACT/ITEM_COUNT/73c9bfe1-6496-4ff8-9467-6cbee2924c16.dict
new file mode 100644
index 0000000..4892e3a
Binary files /dev/null and b/examples/test_case_data/localmeta/dict/TEST_KYLIN_FACT/ITEM_COUNT/73c9bfe1-6496-4ff8-9467-6cbee2924c16.dict differ

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/examples/test_case_data/localmeta/dict/TEST_KYLIN_FACT/LEAF_CATEG_ID/4059cab1-9b99-47ed-a494-751da52a7d58.dict
----------------------------------------------------------------------
diff --git a/examples/test_case_data/localmeta/dict/TEST_KYLIN_FACT/LEAF_CATEG_ID/4059cab1-9b99-47ed-a494-751da52a7d58.dict b/examples/test_case_data/localmeta/dict/TEST_KYLIN_FACT/LEAF_CATEG_ID/4059cab1-9b99-47ed-a494-751da52a7d58.dict
new file mode 100644
index 0000000..dc1bfe9
Binary files /dev/null and b/examples/test_case_data/localmeta/dict/TEST_KYLIN_FACT/LEAF_CATEG_ID/4059cab1-9b99-47ed-a494-751da52a7d58.dict differ

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/examples/test_case_data/localmeta/dict/TEST_KYLIN_FACT/LEAF_CATEG_ID/96b7c577-b209-45b3-a848-4d2d7af5c0cc.dict
----------------------------------------------------------------------
diff --git a/examples/test_case_data/localmeta/dict/TEST_KYLIN_FACT/LEAF_CATEG_ID/96b7c577-b209-45b3-a848-4d2d7af5c0cc.dict b/examples/test_case_data/localmeta/dict/TEST_KYLIN_FACT/LEAF_CATEG_ID/96b7c577-b209-45b3-a848-4d2d7af5c0cc.dict
new file mode 100644
index 0000000..209f3ed
Binary files /dev/null and b/examples/test_case_data/localmeta/dict/TEST_KYLIN_FACT/LEAF_CATEG_ID/96b7c577-b209-45b3-a848-4d2d7af5c0cc.dict differ

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/examples/test_case_data/localmeta/dict/TEST_KYLIN_FACT/LEAF_CATEG_ID/9fc2360e-172a-43f3-acef-be16748a9bb7.dict
----------------------------------------------------------------------
diff --git a/examples/test_case_data/localmeta/dict/TEST_KYLIN_FACT/LEAF_CATEG_ID/9fc2360e-172a-43f3-acef-be16748a9bb7.dict b/examples/test_case_data/localmeta/dict/TEST_KYLIN_FACT/LEAF_CATEG_ID/9fc2360e-172a-43f3-acef-be16748a9bb7.dict
new file mode 100644
index 0000000..35e5639
Binary files /dev/null and b/examples/test_case_data/localmeta/dict/TEST_KYLIN_FACT/LEAF_CATEG_ID/9fc2360e-172a-43f3-acef-be16748a9bb7.dict differ

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/examples/test_case_data/localmeta/dict/TEST_KYLIN_FACT/LSTG_FORMAT_NAME/bd9f6b22-36ba-4e6b-92aa-0d585faf0b39.dict
----------------------------------------------------------------------
diff --git a/examples/test_case_data/localmeta/dict/TEST_KYLIN_FACT/LSTG_FORMAT_NAME/bd9f6b22-36ba-4e6b-92aa-0d585faf0b39.dict b/examples/test_case_data/localmeta/dict/TEST_KYLIN_FACT/LSTG_FORMAT_NAME/bd9f6b22-36ba-4e6b-92aa-0d585faf0b39.dict
new file mode 100644
index 0000000..0da7747
Binary files /dev/null and b/examples/test_case_data/localmeta/dict/TEST_KYLIN_FACT/LSTG_FORMAT_NAME/bd9f6b22-36ba-4e6b-92aa-0d585faf0b39.dict differ

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/examples/test_case_data/localmeta/dict/TEST_KYLIN_FACT/LSTG_SITE_ID/7df5789b-0280-453c-b406-b75cad6770d1.dict
----------------------------------------------------------------------
diff --git a/examples/test_case_data/localmeta/dict/TEST_KYLIN_FACT/LSTG_SITE_ID/7df5789b-0280-453c-b406-b75cad6770d1.dict b/examples/test_case_data/localmeta/dict/TEST_KYLIN_FACT/LSTG_SITE_ID/7df5789b-0280-453c-b406-b75cad6770d1.dict
new file mode 100644
index 0000000..05afc58
Binary files /dev/null and b/examples/test_case_data/localmeta/dict/TEST_KYLIN_FACT/LSTG_SITE_ID/7df5789b-0280-453c-b406-b75cad6770d1.dict differ

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/examples/test_case_data/localmeta/dict/TEST_KYLIN_FACT/PRICE/927dde3f-6999-4434-b57c-adfa73160334.dict
----------------------------------------------------------------------
diff --git a/examples/test_case_data/localmeta/dict/TEST_KYLIN_FACT/PRICE/927dde3f-6999-4434-b57c-adfa73160334.dict b/examples/test_case_data/localmeta/dict/TEST_KYLIN_FACT/PRICE/927dde3f-6999-4434-b57c-adfa73160334.dict
new file mode 100644
index 0000000..90c2876
Binary files /dev/null and b/examples/test_case_data/localmeta/dict/TEST_KYLIN_FACT/PRICE/927dde3f-6999-4434-b57c-adfa73160334.dict differ

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/examples/test_case_data/localmeta/dict/TEST_KYLIN_FACT/SELLER_ID/6e285e1e-ed16-4012-9f1e-f950dd6927ce.dict
----------------------------------------------------------------------
diff --git a/examples/test_case_data/localmeta/dict/TEST_KYLIN_FACT/SELLER_ID/6e285e1e-ed16-4012-9f1e-f950dd6927ce.dict b/examples/test_case_data/localmeta/dict/TEST_KYLIN_FACT/SELLER_ID/6e285e1e-ed16-4012-9f1e-f950dd6927ce.dict
new file mode 100644
index 0000000..829c33c
Binary files /dev/null and b/examples/test_case_data/localmeta/dict/TEST_KYLIN_FACT/SELLER_ID/6e285e1e-ed16-4012-9f1e-f950dd6927ce.dict differ

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/examples/test_case_data/localmeta/dict/TEST_KYLIN_FACT/SLR_SEGMENT_CD/8300bf83-053e-48bb-8c87-88c8d483afd1.dict
----------------------------------------------------------------------
diff --git a/examples/test_case_data/localmeta/dict/TEST_KYLIN_FACT/SLR_SEGMENT_CD/8300bf83-053e-48bb-8c87-88c8d483afd1.dict b/examples/test_case_data/localmeta/dict/TEST_KYLIN_FACT/SLR_SEGMENT_CD/8300bf83-053e-48bb-8c87-88c8d483afd1.dict
new file mode 100644
index 0000000..a676d11
Binary files /dev/null and b/examples/test_case_data/localmeta/dict/TEST_KYLIN_FACT/SLR_SEGMENT_CD/8300bf83-053e-48bb-8c87-88c8d483afd1.dict differ

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/examples/test_case_data/localmeta/dict/TEST_KYLIN_FACT/TRANS_ID/c8d19f95-b6cd-4219-a114-54aaddcb2909.dict
----------------------------------------------------------------------
diff --git a/examples/test_case_data/localmeta/dict/TEST_KYLIN_FACT/TRANS_ID/c8d19f95-b6cd-4219-a114-54aaddcb2909.dict b/examples/test_case_data/localmeta/dict/TEST_KYLIN_FACT/TRANS_ID/c8d19f95-b6cd-4219-a114-54aaddcb2909.dict
new file mode 100644
index 0000000..12637ad
Binary files /dev/null and b/examples/test_case_data/localmeta/dict/TEST_KYLIN_FACT/TRANS_ID/c8d19f95-b6cd-4219-a114-54aaddcb2909.dict differ

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/examples/test_case_data/localmeta/dict/TEST_SELLER_TYPE_DIM/SELLER_TYPE_CD/0c5d77ec-316b-47e0-ba9a-0616be890ad6.dict
----------------------------------------------------------------------
diff --git a/examples/test_case_data/localmeta/dict/TEST_SELLER_TYPE_DIM/SELLER_TYPE_CD/0c5d77ec-316b-47e0-ba9a-0616be890ad6.dict b/examples/test_case_data/localmeta/dict/TEST_SELLER_TYPE_DIM/SELLER_TYPE_CD/0c5d77ec-316b-47e0-ba9a-0616be890ad6.dict
new file mode 100644
index 0000000..afe548d
Binary files /dev/null and b/examples/test_case_data/localmeta/dict/TEST_SELLER_TYPE_DIM/SELLER_TYPE_CD/0c5d77ec-316b-47e0-ba9a-0616be890ad6.dict differ

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/examples/test_case_data/localmeta/dict/TEST_SELLER_TYPE_DIM/SELLER_TYPE_CD/14fe66b3-5956-498c-bd93-40182cac5510.dict
----------------------------------------------------------------------
diff --git a/examples/test_case_data/localmeta/dict/TEST_SELLER_TYPE_DIM/SELLER_TYPE_CD/14fe66b3-5956-498c-bd93-40182cac5510.dict b/examples/test_case_data/localmeta/dict/TEST_SELLER_TYPE_DIM/SELLER_TYPE_CD/14fe66b3-5956-498c-bd93-40182cac5510.dict
new file mode 100644
index 0000000..460da2b
Binary files /dev/null and b/examples/test_case_data/localmeta/dict/TEST_SELLER_TYPE_DIM/SELLER_TYPE_CD/14fe66b3-5956-498c-bd93-40182cac5510.dict differ

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/examples/test_case_data/localmeta/dict/TEST_SELLER_TYPE_DIM/SELLER_TYPE_CD/1d383a36-81b9-4177-a822-04eab3683e5b.dict
----------------------------------------------------------------------
diff --git a/examples/test_case_data/localmeta/dict/TEST_SELLER_TYPE_DIM/SELLER_TYPE_CD/1d383a36-81b9-4177-a822-04eab3683e5b.dict b/examples/test_case_data/localmeta/dict/TEST_SELLER_TYPE_DIM/SELLER_TYPE_CD/1d383a36-81b9-4177-a822-04eab3683e5b.dict
new file mode 100644
index 0000000..7a97f71
Binary files /dev/null and b/examples/test_case_data/localmeta/dict/TEST_SELLER_TYPE_DIM/SELLER_TYPE_CD/1d383a36-81b9-4177-a822-04eab3683e5b.dict differ

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/examples/test_case_data/localmeta/dict/TEST_SELLER_TYPE_DIM/SELLER_TYPE_CD/2a44ff38-f64b-42e7-9fcf-66afccac8047.dict
----------------------------------------------------------------------
diff --git a/examples/test_case_data/localmeta/dict/TEST_SELLER_TYPE_DIM/SELLER_TYPE_CD/2a44ff38-f64b-42e7-9fcf-66afccac8047.dict b/examples/test_case_data/localmeta/dict/TEST_SELLER_TYPE_DIM/SELLER_TYPE_CD/2a44ff38-f64b-42e7-9fcf-66afccac8047.dict
new file mode 100644
index 0000000..c6a0587
Binary files /dev/null and b/examples/test_case_data/localmeta/dict/TEST_SELLER_TYPE_DIM/SELLER_TYPE_CD/2a44ff38-f64b-42e7-9fcf-66afccac8047.dict differ

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/examples/test_case_data/localmeta/dict/TEST_SELLER_TYPE_DIM/SELLER_TYPE_CD/c6bf9b51-6e90-4337-8082-4e2fdf78307f.dict
----------------------------------------------------------------------
diff --git a/examples/test_case_data/localmeta/dict/TEST_SELLER_TYPE_DIM/SELLER_TYPE_CD/c6bf9b51-6e90-4337-8082-4e2fdf78307f.dict b/examples/test_case_data/localmeta/dict/TEST_SELLER_TYPE_DIM/SELLER_TYPE_CD/c6bf9b51-6e90-4337-8082-4e2fdf78307f.dict
new file mode 100644
index 0000000..6f270fe
Binary files /dev/null and b/examples/test_case_data/localmeta/dict/TEST_SELLER_TYPE_DIM/SELLER_TYPE_CD/c6bf9b51-6e90-4337-8082-4e2fdf78307f.dict differ

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/examples/test_case_data/localmeta/dict/TEST_SITES/SITE_ID/0bec6bb3-1b0d-469c-8289-b8c4ca5d5001.dict
----------------------------------------------------------------------
diff --git a/examples/test_case_data/localmeta/dict/TEST_SITES/SITE_ID/0bec6bb3-1b0d-469c-8289-b8c4ca5d5001.dict b/examples/test_case_data/localmeta/dict/TEST_SITES/SITE_ID/0bec6bb3-1b0d-469c-8289-b8c4ca5d5001.dict
new file mode 100644
index 0000000..5be36a1
Binary files /dev/null and b/examples/test_case_data/localmeta/dict/TEST_SITES/SITE_ID/0bec6bb3-1b0d-469c-8289-b8c4ca5d5001.dict differ

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/examples/test_case_data/localmeta/dict/TEST_SITES/SITE_ID/31edf35b-ffca-4f24-8229-f87dc34e3087.dict
----------------------------------------------------------------------
diff --git a/examples/test_case_data/localmeta/dict/TEST_SITES/SITE_ID/31edf35b-ffca-4f24-8229-f87dc34e3087.dict b/examples/test_case_data/localmeta/dict/TEST_SITES/SITE_ID/31edf35b-ffca-4f24-8229-f87dc34e3087.dict
new file mode 100644
index 0000000..87057cd
Binary files /dev/null and b/examples/test_case_data/localmeta/dict/TEST_SITES/SITE_ID/31edf35b-ffca-4f24-8229-f87dc34e3087.dict differ

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/examples/test_case_data/localmeta/dict/TEST_SITES/SITE_ID/652bd393-678a-4f16-a504-fd8ce1229355.dict
----------------------------------------------------------------------
diff --git a/examples/test_case_data/localmeta/dict/TEST_SITES/SITE_ID/652bd393-678a-4f16-a504-fd8ce1229355.dict b/examples/test_case_data/localmeta/dict/TEST_SITES/SITE_ID/652bd393-678a-4f16-a504-fd8ce1229355.dict
new file mode 100644
index 0000000..194b7e2
Binary files /dev/null and b/examples/test_case_data/localmeta/dict/TEST_SITES/SITE_ID/652bd393-678a-4f16-a504-fd8ce1229355.dict differ

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/examples/test_case_data/localmeta/dict/TEST_SITES/SITE_ID/792eb972-d046-48e6-9428-f6a3aed92fad.dict
----------------------------------------------------------------------
diff --git a/examples/test_case_data/localmeta/dict/TEST_SITES/SITE_ID/792eb972-d046-48e6-9428-f6a3aed92fad.dict b/examples/test_case_data/localmeta/dict/TEST_SITES/SITE_ID/792eb972-d046-48e6-9428-f6a3aed92fad.dict
new file mode 100644
index 0000000..0a60fc3
Binary files /dev/null and b/examples/test_case_data/localmeta/dict/TEST_SITES/SITE_ID/792eb972-d046-48e6-9428-f6a3aed92fad.dict differ

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/examples/test_case_data/localmeta/dict/TEST_SITES/SITE_ID/914536f8-1e14-43e9-b8ef-525bb5bdb3b2.dict
----------------------------------------------------------------------
diff --git a/examples/test_case_data/localmeta/dict/TEST_SITES/SITE_ID/914536f8-1e14-43e9-b8ef-525bb5bdb3b2.dict b/examples/test_case_data/localmeta/dict/TEST_SITES/SITE_ID/914536f8-1e14-43e9-b8ef-525bb5bdb3b2.dict
new file mode 100644
index 0000000..eb64935
Binary files /dev/null and b/examples/test_case_data/localmeta/dict/TEST_SITES/SITE_ID/914536f8-1e14-43e9-b8ef-525bb5bdb3b2.dict differ

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/examples/test_case_data/localmeta/dict/TEST_SITES/SITE_ID/d25e554e-deac-4e4a-9289-96f0d121d8fc.dict
----------------------------------------------------------------------
diff --git a/examples/test_case_data/localmeta/dict/TEST_SITES/SITE_ID/d25e554e-deac-4e4a-9289-96f0d121d8fc.dict b/examples/test_case_data/localmeta/dict/TEST_SITES/SITE_ID/d25e554e-deac-4e4a-9289-96f0d121d8fc.dict
new file mode 100644
index 0000000..a10c74b
Binary files /dev/null and b/examples/test_case_data/localmeta/dict/TEST_SITES/SITE_ID/d25e554e-deac-4e4a-9289-96f0d121d8fc.dict differ

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/examples/test_case_data/localmeta/dict/TEST_SITES/SITE_ID/ff7e8943-ac0f-4e66-b9ed-510f6a0b875d.dict
----------------------------------------------------------------------
diff --git a/examples/test_case_data/localmeta/dict/TEST_SITES/SITE_ID/ff7e8943-ac0f-4e66-b9ed-510f6a0b875d.dict b/examples/test_case_data/localmeta/dict/TEST_SITES/SITE_ID/ff7e8943-ac0f-4e66-b9ed-510f6a0b875d.dict
new file mode 100644
index 0000000..d3530cf
Binary files /dev/null and b/examples/test_case_data/localmeta/dict/TEST_SITES/SITE_ID/ff7e8943-ac0f-4e66-b9ed-510f6a0b875d.dict differ

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/examples/test_case_data/localmeta/invertedindex_desc/test_kylin_cube_ii.json
----------------------------------------------------------------------
diff --git a/examples/test_case_data/localmeta/invertedindex_desc/test_kylin_cube_ii.json b/examples/test_case_data/localmeta/invertedindex_desc/test_kylin_cube_ii.json
new file mode 100644
index 0000000..febc397
--- /dev/null
+++ b/examples/test_case_data/localmeta/invertedindex_desc/test_kylin_cube_ii.json
@@ -0,0 +1,24 @@
+{
+  "uuid" : "74bf87b5-c7b5-4420-a12a-07f6b37b3187",
+  "last_modified" : 0,
+  "name" : "test_kylin_cube_ii",
+  "fact_table" : "TEST_KYLIN_FACT",
+  "timestamp_dimension" : "CAL_DT",
+  "bitmap_dimensions" : [
+    "LSTG_FORMAT_NAME",
+    "LSTG_SITE_ID",
+    "SLR_SEGMENT_CD"
+  ],
+  "value_dimensions" : [
+    "TRANS_ID",
+    "CAL_DT",
+    "LEAF_CATEG_ID",
+    "SELLER_ID"
+  ],
+  "metrics" : [
+    "PRICE",
+    "ITEM_COUNT"
+  ],
+  "sharding" : 4,
+  "slice_size" : 50000
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/examples/test_case_data/localmeta/kylin.properties
----------------------------------------------------------------------
diff --git a/examples/test_case_data/localmeta/kylin.properties b/examples/test_case_data/localmeta/kylin.properties
new file mode 100644
index 0000000..9b42860
--- /dev/null
+++ b/examples/test_case_data/localmeta/kylin.properties
@@ -0,0 +1,66 @@
+## Config for Kylin Engine ##
+
+# List of web servers in use, this enables one web server instance to sync up with other servers.
+kylin.rest.servers=
+
+# The metadata store in hbase
+kylin.metadata.url=
+
+# The storage for final cube file in hbase
+kylin.storage.url=hbase:sandbox.hortonworks.com:2181:/hbase-unsecure
+
+# Path to the local(relative to job engine) coprocessor jar, job engine will upload this file to hbase
+kylin.coprocessor.local.jar=/tmp/kylin/kylin-storage-latest-coprocessor.jar
+
+# Temp folder in hdfs
+kylin.hdfs.working.dir=/tmp
+
+# Path to the local(relative to job engine) job jar, job engine will use this jar
+kylin.job.jar=/tmp/kylin/kylin-job-latest-job.jar
+
+kylin.job.mapreduce.default.reduce.input.mb=500
+
+# If true, job engine will not assume that hadoop CLI reside on the same server as it self
+# you will have to specify kylin.job.remote.cli.hostname, kylin.job.remote.cli.username and kylin.job.remote.cli.password
+kylin.job.run.as.remote.cmd=false
+
+# Only necessary when kylin.job.run.as.remote.cmd=true
+kylin.job.remote.cli.hostname=
+
+# Only necessary when kylin.job.run.as.remote.cmd=true
+kylin.job.remote.cli.username=
+
+# Only necessary when kylin.job.run.as.remote.cmd=true
+kylin.job.remote.cli.password=
+
+# Used by test cases to prepare synthetic data for sample cube
+kylin.job.remote.cli.working.dir=/tmp/kylin
+
+# Use hive to flatten star schema tables
+kylin.job.hive.flatten=true
+
+# Max count of concurrent jobs running
+kylin.job.concurrent.max.limit=10
+
+# The url to check hadoop job status
+kylin.job.yarn.app.rest.check.status.url=http://sandbox:8088/ws/v1/cluster/apps/${job_id}?anonymous=true
+
+# Time interval to check hadoop job status
+kylin.job.yarn.app.rest.check.interval.seconds=10
+
+
+## Config for Restful APP ##
+# database connection settings:
+ldap.server=
+ldap.username=
+ldap.password=
+ldap.user.searchBase=
+ldap.user.searchPattern=
+ldap.user.groupSearchBase=
+ldap.service.searchBase=OU=
+ldap.service.searchPattern=
+ldap.service.groupSearchBase=
+acl.adminRole=
+acl.defaultRole=
+ganglia.group=
+ganglia.port=8664
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/examples/test_case_data/localmeta/kylin_job_conf.xml
----------------------------------------------------------------------
diff --git a/examples/test_case_data/localmeta/kylin_job_conf.xml b/examples/test_case_data/localmeta/kylin_job_conf.xml
new file mode 100644
index 0000000..f62ce55
--- /dev/null
+++ b/examples/test_case_data/localmeta/kylin_job_conf.xml
@@ -0,0 +1,60 @@
+<?xml version="1.0"?>
+<configuration>
+
+    <property>
+        <name>mapreduce.job.split.metainfo.maxsize</name>
+        <value>-1</value>
+        <description>The maximum permissible size of the split metainfo file.
+            The JobTracker won't attempt to read split metainfo files bigger than
+            the configured value. No limits if set to -1.
+        </description>
+    </property>
+
+    <!-- uncomment the following 5 properties to enable lzo compressing
+
+	<property>
+		<name>mapred.compress.map.output</name>
+		<value>true</value>
+		<description>Compress map outputs</description>
+	</property>
+
+	<property>
+		<name>mapred.map.output.compression.codec</name>
+		<value>com.hadoop.compression.lzo.LzoCodec</value>
+		<description>The compression codec to use for map outputs
+		</description>
+	</property>
+
+	<property>
+		<name>mapred.output.compress</name>
+		<value>true</value>
+		<description>Compress the output of a MapReduce job</description>
+	</property>
+
+	<property>
+		<name>mapred.output.compression.codec</name>
+		<value>com.hadoop.compression.lzo.LzoCodec</value>
+		<description>The compression codec to use for job outputs
+		</description>
+	</property>
+
+	<property>
+		<name>mapred.output.compression.type</name>
+		<value>BLOCK</value>
+		<description>The compression type to use for job outputs</description>
+	</property>
+
+	!-->
+
+    <property>
+        <name>mapreduce.job.max.split.locations</name>
+        <value>2000</value>
+        <description>No description</description>
+    </property>
+
+    <property>
+        <name>dfs.replication</name>
+        <value>2</value>
+        <description>Block replication</description>
+    </property>
+</configuration>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/examples/test_case_data/localmeta/project/default.json
----------------------------------------------------------------------
diff --git a/examples/test_case_data/localmeta/project/default.json b/examples/test_case_data/localmeta/project/default.json
new file mode 100644
index 0000000..653fcad
--- /dev/null
+++ b/examples/test_case_data/localmeta/project/default.json
@@ -0,0 +1,10 @@
+{
+"uuid" : "1eaca32a-a33e-4b69-83dd-0bb8b1f8c91b",
+  "name": "default",
+  "cubes": [
+    "test_kylin_cube_with_slr_empty",
+    "test_kylin_cube_without_slr_empty",
+    "test_kylin_cube_with_slr_left_join_empty",
+    "test_kylin_cube_without_slr_left_join_empty"
+  ]
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/examples/test_case_data/localmeta/project/onlyinner.json
----------------------------------------------------------------------
diff --git a/examples/test_case_data/localmeta/project/onlyinner.json b/examples/test_case_data/localmeta/project/onlyinner.json
new file mode 100644
index 0000000..9544895
--- /dev/null
+++ b/examples/test_case_data/localmeta/project/onlyinner.json
@@ -0,0 +1,8 @@
+{
+"uuid" : "1eaca32a-a33e-4b69-83dd-0bb8b1f8c92b",
+  "name": "onlyinner",
+  "cubes": [
+    "test_kylin_cube_with_slr_empty",
+    "test_kylin_cube_without_slr_empty"
+  ]
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/examples/test_case_data/localmeta/project/onlyleft.json
----------------------------------------------------------------------
diff --git a/examples/test_case_data/localmeta/project/onlyleft.json b/examples/test_case_data/localmeta/project/onlyleft.json
new file mode 100644
index 0000000..13e57ed
--- /dev/null
+++ b/examples/test_case_data/localmeta/project/onlyleft.json
@@ -0,0 +1,8 @@
+{
+"uuid" : "1eaca32a-a33e-4b69-83dd-0bb8b1f8c93b",
+  "name": "onlyleft",
+  "cubes": [
+    "test_kylin_cube_with_slr_left_join_empty",
+    "test_kylin_cube_without_slr_left_join_empty"
+  ]
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/examples/test_case_data/localmeta/table/TEST_CAL_DT.json
----------------------------------------------------------------------
diff --git a/examples/test_case_data/localmeta/table/TEST_CAL_DT.json b/examples/test_case_data/localmeta/table/TEST_CAL_DT.json
new file mode 100644
index 0000000..0ad7ee1
--- /dev/null
+++ b/examples/test_case_data/localmeta/table/TEST_CAL_DT.json
@@ -0,0 +1,407 @@
+{
+  "uuid" : "0ff420eb-79ad-40bd-bca9-12d8cd05c60a",
+  "name" : "TEST_CAL_DT",
+  "columns" : [ {
+    "id" : "1",
+    "name" : "CAL_DT",
+    "datatype" : "date"
+  }, {
+    "id" : "2",
+    "name" : "YEAR_BEG_DT",
+    "datatype" : "date"
+  }, {
+    "id" : "3",
+    "name" : "QTR_BEG_DT",
+    "datatype" : "date"
+  }, {
+    "id" : "4",
+    "name" : "MONTH_BEG_DT",
+    "datatype" : "date"
+  }, {
+    "id" : "5",
+    "name" : "WEEK_BEG_DT",
+    "datatype" : "date"
+  }, {
+    "id" : "6",
+    "name" : "AGE_FOR_YEAR_ID",
+    "datatype" : "smallint"
+  }, {
+    "id" : "7",
+    "name" : "AGE_FOR_QTR_ID",
+    "datatype" : "smallint"
+  }, {
+    "id" : "8",
+    "name" : "AGE_FOR_MONTH_ID",
+    "datatype" : "smallint"
+  }, {
+    "id" : "9",
+    "name" : "AGE_FOR_WEEK_ID",
+    "datatype" : "smallint"
+  }, {
+    "id" : "10",
+    "name" : "AGE_FOR_DT_ID",
+    "datatype" : "smallint"
+  }, {
+    "id" : "11",
+    "name" : "AGE_FOR_RTL_YEAR_ID",
+    "datatype" : "smallint"
+  }, {
+    "id" : "12",
+    "name" : "AGE_FOR_RTL_QTR_ID",
+    "datatype" : "smallint"
+  }, {
+    "id" : "13",
+    "name" : "AGE_FOR_RTL_MONTH_ID",
+    "datatype" : "smallint"
+  }, {
+    "id" : "14",
+    "name" : "AGE_FOR_RTL_WEEK_ID",
+    "datatype" : "smallint"
+  }, {
+    "id" : "15",
+    "name" : "AGE_FOR_CS_WEEK_ID",
+    "datatype" : "smallint"
+  }, {
+    "id" : "16",
+    "name" : "DAY_OF_CAL_ID",
+    "datatype" : "int"
+  }, {
+    "id" : "17",
+    "name" : "DAY_OF_YEAR_ID",
+    "datatype" : "smallint"
+  }, {
+    "id" : "18",
+    "name" : "DAY_OF_QTR_ID",
+    "datatype" : "smallint"
+  }, {
+    "id" : "19",
+    "name" : "DAY_OF_MONTH_ID",
+    "datatype" : "smallint"
+  }, {
+    "id" : "20",
+    "name" : "DAY_OF_WEEK_ID",
+    "datatype" : "int"
+  }, {
+    "id" : "21",
+    "name" : "WEEK_OF_YEAR_ID",
+    "datatype" : "tinyint"
+  }, {
+    "id" : "22",
+    "name" : "WEEK_OF_CAL_ID",
+    "datatype" : "int"
+  }, {
+    "id" : "23",
+    "name" : "MONTH_OF_QTR_ID",
+    "datatype" : "tinyint"
+  }, {
+    "id" : "24",
+    "name" : "MONTH_OF_YEAR_ID",
+    "datatype" : "tinyint"
+  }, {
+    "id" : "25",
+    "name" : "MONTH_OF_CAL_ID",
+    "datatype" : "smallint"
+  }, {
+    "id" : "26",
+    "name" : "QTR_OF_YEAR_ID",
+    "datatype" : "tinyint"
+  }, {
+    "id" : "27",
+    "name" : "QTR_OF_CAL_ID",
+    "datatype" : "smallint"
+  }, {
+    "id" : "28",
+    "name" : "YEAR_OF_CAL_ID",
+    "datatype" : "smallint"
+  }, {
+    "id" : "29",
+    "name" : "YEAR_END_DT",
+    "datatype" : "string"
+  }, {
+    "id" : "30",
+    "name" : "QTR_END_DT",
+    "datatype" : "string"
+  }, {
+    "id" : "31",
+    "name" : "MONTH_END_DT",
+    "datatype" : "string"
+  }, {
+    "id" : "32",
+    "name" : "WEEK_END_DT",
+    "datatype" : "string"
+  }, {
+    "id" : "33",
+    "name" : "CAL_DT_NAME",
+    "datatype" : "string"
+  }, {
+    "id" : "34",
+    "name" : "CAL_DT_DESC",
+    "datatype" : "string"
+  }, {
+    "id" : "35",
+    "name" : "CAL_DT_SHORT_NAME",
+    "datatype" : "string"
+  }, {
+    "id" : "36",
+    "name" : "YTD_YN_ID",
+    "datatype" : "tinyint"
+  }, {
+    "id" : "37",
+    "name" : "QTD_YN_ID",
+    "datatype" : "tinyint"
+  }, {
+    "id" : "38",
+    "name" : "MTD_YN_ID",
+    "datatype" : "tinyint"
+  }, {
+    "id" : "39",
+    "name" : "WTD_YN_ID",
+    "datatype" : "tinyint"
+  }, {
+    "id" : "40",
+    "name" : "SEASON_BEG_DT",
+    "datatype" : "string"
+  }, {
+    "id" : "41",
+    "name" : "DAY_IN_YEAR_COUNT",
+    "datatype" : "smallint"
+  }, {
+    "id" : "42",
+    "name" : "DAY_IN_QTR_COUNT",
+    "datatype" : "tinyint"
+  }, {
+    "id" : "43",
+    "name" : "DAY_IN_MONTH_COUNT",
+    "datatype" : "tinyint"
+  }, {
+    "id" : "44",
+    "name" : "DAY_IN_WEEK_COUNT",
+    "datatype" : "tinyint"
+  }, {
+    "id" : "45",
+    "name" : "RTL_YEAR_BEG_DT",
+    "datatype" : "string"
+  }, {
+    "id" : "46",
+    "name" : "RTL_QTR_BEG_DT",
+    "datatype" : "string"
+  }, {
+    "id" : "47",
+    "name" : "RTL_MONTH_BEG_DT",
+    "datatype" : "string"
+  }, {
+    "id" : "48",
+    "name" : "RTL_WEEK_BEG_DT",
+    "datatype" : "string"
+  }, {
+    "id" : "49",
+    "name" : "CS_WEEK_BEG_DT",
+    "datatype" : "string"
+  }, {
+    "id" : "50",
+    "name" : "CAL_DATE",
+    "datatype" : "string"
+  }, {
+    "id" : "51",
+    "name" : "DAY_OF_WEEK",
+    "datatype" : "string"
+  }, {
+    "id" : "52",
+    "name" : "MONTH_ID",
+    "datatype" : "string"
+  }, {
+    "id" : "53",
+    "name" : "PRD_DESC",
+    "datatype" : "string"
+  }, {
+    "id" : "54",
+    "name" : "PRD_FLAG",
+    "datatype" : "string"
+  }, {
+    "id" : "55",
+    "name" : "PRD_ID",
+    "datatype" : "string"
+  }, {
+    "id" : "56",
+    "name" : "PRD_IND",
+    "datatype" : "string"
+  }, {
+    "id" : "57",
+    "name" : "QTR_DESC",
+    "datatype" : "string"
+  }, {
+    "id" : "58",
+    "name" : "QTR_ID",
+    "datatype" : "string"
+  }, {
+    "id" : "59",
+    "name" : "QTR_IND",
+    "datatype" : "string"
+  }, {
+    "id" : "60",
+    "name" : "RETAIL_WEEK",
+    "datatype" : "string"
+  }, {
+    "id" : "61",
+    "name" : "RETAIL_YEAR",
+    "datatype" : "string"
+  }, {
+    "id" : "62",
+    "name" : "RETAIL_START_DATE",
+    "datatype" : "string"
+  }, {
+    "id" : "63",
+    "name" : "RETAIL_WK_END_DATE",
+    "datatype" : "string"
+  }, {
+    "id" : "64",
+    "name" : "WEEK_IND",
+    "datatype" : "string"
+  }, {
+    "id" : "65",
+    "name" : "WEEK_NUM_DESC",
+    "datatype" : "string"
+  }, {
+    "id" : "66",
+    "name" : "WEEK_BEG_DATE",
+    "datatype" : "string"
+  }, {
+    "id" : "67",
+    "name" : "WEEK_END_DATE",
+    "datatype" : "string"
+  }, {
+    "id" : "68",
+    "name" : "WEEK_IN_YEAR_ID",
+    "datatype" : "string"
+  }, {
+    "id" : "69",
+    "name" : "WEEK_ID",
+    "datatype" : "string"
+  }, {
+    "id" : "70",
+    "name" : "WEEK_BEG_END_DESC_MDY",
+    "datatype" : "string"
+  }, {
+    "id" : "71",
+    "name" : "WEEK_BEG_END_DESC_MD",
+    "datatype" : "string"
+  }, {
+    "id" : "72",
+    "name" : "YEAR_ID",
+    "datatype" : "string"
+  }, {
+    "id" : "73",
+    "name" : "YEAR_IND",
+    "datatype" : "string"
+  }, {
+    "id" : "74",
+    "name" : "CAL_DT_MNS_1YEAR_DT",
+    "datatype" : "string"
+  }, {
+    "id" : "75",
+    "name" : "CAL_DT_MNS_2YEAR_DT",
+    "datatype" : "string"
+  }, {
+    "id" : "76",
+    "name" : "CAL_DT_MNS_1QTR_DT",
+    "datatype" : "string"
+  }, {
+    "id" : "77",
+    "name" : "CAL_DT_MNS_2QTR_DT",
+    "datatype" : "string"
+  }, {
+    "id" : "78",
+    "name" : "CAL_DT_MNS_1MONTH_DT",
+    "datatype" : "string"
+  }, {
+    "id" : "79",
+    "name" : "CAL_DT_MNS_2MONTH_DT",
+    "datatype" : "string"
+  }, {
+    "id" : "80",
+    "name" : "CAL_DT_MNS_1WEEK_DT",
+    "datatype" : "string"
+  }, {
+    "id" : "81",
+    "name" : "CAL_DT_MNS_2WEEK_DT",
+    "datatype" : "string"
+  }, {
+    "id" : "82",
+    "name" : "CURR_CAL_DT_MNS_1YEAR_YN_ID",
+    "datatype" : "tinyint"
+  }, {
+    "id" : "83",
+    "name" : "CURR_CAL_DT_MNS_2YEAR_YN_ID",
+    "datatype" : "tinyint"
+  }, {
+    "id" : "84",
+    "name" : "CURR_CAL_DT_MNS_1QTR_YN_ID",
+    "datatype" : "tinyint"
+  }, {
+    "id" : "85",
+    "name" : "CURR_CAL_DT_MNS_2QTR_YN_ID",
+    "datatype" : "tinyint"
+  }, {
+    "id" : "86",
+    "name" : "CURR_CAL_DT_MNS_1MONTH_YN_ID",
+    "datatype" : "tinyint"
+  }, {
+    "id" : "87",
+    "name" : "CURR_CAL_DT_MNS_2MONTH_YN_ID",
+    "datatype" : "tinyint"
+  }, {
+    "id" : "88",
+    "name" : "CURR_CAL_DT_MNS_1WEEK_YN_IND",
+    "datatype" : "tinyint"
+  }, {
+    "id" : "89",
+    "name" : "CURR_CAL_DT_MNS_2WEEK_YN_IND",
+    "datatype" : "tinyint"
+  }, {
+    "id" : "90",
+    "name" : "RTL_MONTH_OF_RTL_YEAR_ID",
+    "datatype" : "string"
+  }, {
+    "id" : "91",
+    "name" : "RTL_QTR_OF_RTL_YEAR_ID",
+    "datatype" : "tinyint"
+  }, {
+    "id" : "92",
+    "name" : "RTL_WEEK_OF_RTL_YEAR_ID",
+    "datatype" : "tinyint"
+  }, {
+    "id" : "93",
+    "name" : "SEASON_OF_YEAR_ID",
+    "datatype" : "tinyint"
+  }, {
+    "id" : "94",
+    "name" : "YTM_YN_ID",
+    "datatype" : "tinyint"
+  }, {
+    "id" : "95",
+    "name" : "YTQ_YN_ID",
+    "datatype" : "tinyint"
+  }, {
+    "id" : "96",
+    "name" : "YTW_YN_ID",
+    "datatype" : "tinyint"
+  }, {
+    "id" : "97",
+    "name" : "CRE_DATE",
+    "datatype" : "string"
+  }, {
+    "id" : "98",
+    "name" : "CRE_USER",
+    "datatype" : "string"
+  }, {
+    "id" : "99",
+    "name" : "UPD_DATE",
+    "datatype" : "string"
+  }, {
+    "id" : "100",
+    "name" : "UPD_USER",
+    "datatype" : "string"
+  } ],
+  "database" : "edw",
+  "last_modified" : 0
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/examples/test_case_data/localmeta/table/TEST_CATEGORY_GROUPINGS.json
----------------------------------------------------------------------
diff --git a/examples/test_case_data/localmeta/table/TEST_CATEGORY_GROUPINGS.json b/examples/test_case_data/localmeta/table/TEST_CATEGORY_GROUPINGS.json
new file mode 100644
index 0000000..7bcd092
--- /dev/null
+++ b/examples/test_case_data/localmeta/table/TEST_CATEGORY_GROUPINGS.json
@@ -0,0 +1,151 @@
+{
+  "uuid" : "952d11b5-69d9-45d1-92af-227489485e3f",
+  "name" : "TEST_CATEGORY_GROUPINGS",
+  "columns" : [ {
+    "id" : "1",
+    "name" : "LEAF_CATEG_ID",
+    "datatype" : "bigint"
+  }, {
+    "id" : "2",
+    "name" : "LEAF_CATEG_NAME",
+    "datatype" : "string"
+  }, {
+    "id" : "3",
+    "name" : "SITE_ID",
+    "datatype" : "int"
+  }, {
+    "id" : "4",
+    "name" : "CATEG_BUSN_MGR",
+    "datatype" : "string"
+  }, {
+    "id" : "5",
+    "name" : "CATEG_BUSN_UNIT",
+    "datatype" : "string"
+  }, {
+    "id" : "6",
+    "name" : "REGN_CATEG",
+    "datatype" : "string"
+  }, {
+    "id" : "7",
+    "name" : "USER_DEFINED_FIELD1",
+    "datatype" : "string"
+  }, {
+    "id" : "8",
+    "name" : "USER_DEFINED_FIELD3",
+    "datatype" : "string"
+  }, {
+    "id" : "9",
+    "name" : "CRE_DATE",
+    "datatype" : "string"
+  }, {
+    "id" : "10",
+    "name" : "UPD_DATE",
+    "datatype" : "string"
+  }, {
+    "id" : "11",
+    "name" : "CRE_USER",
+    "datatype" : "string"
+  }, {
+    "id" : "12",
+    "name" : "UPD_USER",
+    "datatype" : "string"
+  }, {
+    "id" : "13",
+    "name" : "META_CATEG_ID",
+    "datatype" : "decimal"
+  }, {
+    "id" : "14",
+    "name" : "META_CATEG_NAME",
+    "datatype" : "string"
+  }, {
+    "id" : "15",
+    "name" : "CATEG_LVL2_ID",
+    "datatype" : "decimal"
+  }, {
+    "id" : "16",
+    "name" : "CATEG_LVL3_ID",
+    "datatype" : "decimal"
+  }, {
+    "id" : "17",
+    "name" : "CATEG_LVL4_ID",
+    "datatype" : "decimal"
+  }, {
+    "id" : "18",
+    "name" : "CATEG_LVL5_ID",
+    "datatype" : "decimal"
+  }, {
+    "id" : "19",
+    "name" : "CATEG_LVL6_ID",
+    "datatype" : "decimal"
+  }, {
+    "id" : "20",
+    "name" : "CATEG_LVL7_ID",
+    "datatype" : "decimal"
+  }, {
+    "id" : "21",
+    "name" : "CATEG_LVL2_NAME",
+    "datatype" : "string"
+  }, {
+    "id" : "22",
+    "name" : "CATEG_LVL3_NAME",
+    "datatype" : "string"
+  }, {
+    "id" : "23",
+    "name" : "CATEG_LVL4_NAME",
+    "datatype" : "string"
+  }, {
+    "id" : "24",
+    "name" : "CATEG_LVL5_NAME",
+    "datatype" : "string"
+  }, {
+    "id" : "25",
+    "name" : "CATEG_LVL6_NAME",
+    "datatype" : "string"
+  }, {
+    "id" : "26",
+    "name" : "CATEG_LVL7_NAME",
+    "datatype" : "string"
+  }, {
+    "id" : "27",
+    "name" : "CATEG_FLAGS",
+    "datatype" : "decimal"
+  }, {
+    "id" : "28",
+    "name" : "ADULT_CATEG_YN",
+    "datatype" : "string"
+  }, {
+    "id" : "29",
+    "name" : "DOMAIN_ID",
+    "datatype" : "decimal"
+  }, {
+    "id" : "30",
+    "name" : "USER_DEFINED_FIELD5",
+    "datatype" : "string"
+  }, {
+    "id" : "31",
+    "name" : "VCS_ID",
+    "datatype" : "decimal"
+  }, {
+    "id" : "32",
+    "name" : "GCS_ID",
+    "datatype" : "decimal"
+  }, {
+    "id" : "33",
+    "name" : "MOVE_TO",
+    "datatype" : "decimal"
+  }, {
+    "id" : "34",
+    "name" : "SAP_CATEGORY_ID",
+    "datatype" : "decimal"
+  }, {
+    "id" : "35",
+    "name" : "SRC_ID",
+    "datatype" : "tinyint"
+  }, {
+    "id" : "36",
+    "name" : "BSNS_VRTCL_NAME",
+    "datatype" : "string"
+  } ],
+  "database" : "DEFAULT",
+  "last_modified" : 0
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/examples/test_case_data/localmeta/table/TEST_KYLIN_FACT.json
----------------------------------------------------------------------
diff --git a/examples/test_case_data/localmeta/table/TEST_KYLIN_FACT.json b/examples/test_case_data/localmeta/table/TEST_KYLIN_FACT.json
new file mode 100644
index 0000000..732351a
--- /dev/null
+++ b/examples/test_case_data/localmeta/table/TEST_KYLIN_FACT.json
@@ -0,0 +1,43 @@
+{
+  "uuid" : "e286e39e-40d7-44c2-8fa2-41b365522771",
+  "name" : "TEST_KYLIN_FACT",
+  "columns" : [ {
+    "id" : "1",
+    "name" : "TRANS_ID",
+    "datatype" : "bigint"
+  }, {
+    "id" : "2",
+    "name" : "CAL_DT",
+    "datatype" : "date"
+  }, {
+    "id" : "3",
+    "name" : "LSTG_FORMAT_NAME",
+    "datatype" : "string"
+  }, {
+    "id" : "4",
+    "name" : "LEAF_CATEG_ID",
+    "datatype" : "bigint"
+  }, {
+    "id" : "5",
+    "name" : "LSTG_SITE_ID",
+    "datatype" : "int"
+  }, {
+    "id" : "6",
+    "name" : "SLR_SEGMENT_CD",
+    "datatype" : "smallint"
+  }, {
+    "id" : "7",
+    "name" : "PRICE",
+    "datatype" : "decimal(18,6)"
+  }, {
+    "id" : "8",
+    "name" : "ITEM_COUNT",
+    "datatype" : "bigint"
+  }, {
+    "id" : "9",
+    "name" : "SELLER_ID",
+    "datatype" : "bigint"
+  } ],
+  "database" : "DEFAULT",
+  "last_modified" : 0
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/examples/test_case_data/localmeta/table/TEST_SELLER_TYPE_DIM.json
----------------------------------------------------------------------
diff --git a/examples/test_case_data/localmeta/table/TEST_SELLER_TYPE_DIM.json b/examples/test_case_data/localmeta/table/TEST_SELLER_TYPE_DIM.json
new file mode 100644
index 0000000..42f4d9c
--- /dev/null
+++ b/examples/test_case_data/localmeta/table/TEST_SELLER_TYPE_DIM.json
@@ -0,0 +1,43 @@
+{
+  "uuid" : "9ecc90c4-55df-436f-8602-2fbd4bca72e1",
+  "name" : "TEST_SELLER_TYPE_DIM",
+  "columns" : [ {
+    "id" : "1",
+    "name" : "SELLER_TYPE_CD",
+    "datatype" : "smallint"
+  }, {
+    "id" : "2",
+    "name" : "SELLER_TYPE_DESC",
+    "datatype" : "string"
+  }, {
+    "id" : "3",
+    "name" : "GLBL_RPRT_SLR_SGMNT_CD",
+    "datatype" : "tinyint"
+  }, {
+    "id" : "4",
+    "name" : "SELLER_GROUP_CD",
+    "datatype" : "tinyint"
+  }, {
+    "id" : "5",
+    "name" : "SELLER_GROUP_DESC",
+    "datatype" : "string"
+  }, {
+    "id" : "6",
+    "name" : "CRE_DATE",
+    "datatype" : "string"
+  }, {
+    "id" : "7",
+    "name" : "CRE_USER",
+    "datatype" : "string"
+  }, {
+    "id" : "8",
+    "name" : "UPD_DATE",
+    "datatype" : "string"
+  }, {
+    "id" : "9",
+    "name" : "UPD_USER",
+    "datatype" : "string"
+  } ],
+  "database" : "edw",
+  "last_modified" : 0
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/examples/test_case_data/localmeta/table/TEST_SITES.json
----------------------------------------------------------------------
diff --git a/examples/test_case_data/localmeta/table/TEST_SITES.json b/examples/test_case_data/localmeta/table/TEST_SITES.json
new file mode 100644
index 0000000..d451e34
--- /dev/null
+++ b/examples/test_case_data/localmeta/table/TEST_SITES.json
@@ -0,0 +1,47 @@
+{
+  "uuid" : "338a3325-a947-46d1-9ece-e079b3b8d4a6",
+  "name" : "TEST_SITES",
+  "columns" : [ {
+    "id" : "1",
+    "name" : "SITE_ID",
+    "datatype" : "int"
+  }, {
+    "id" : "2",
+    "name" : "SITE_NAME",
+    "datatype" : "string"
+  }, {
+    "id" : "3",
+    "name" : "SITE_DOMAIN_CODE",
+    "datatype" : "string"
+  }, {
+    "id" : "4",
+    "name" : "DFAULT_LSTG_CURNCY",
+    "datatype" : "int"
+  }, {
+    "id" : "5",
+    "name" : "EOA_EMAIL_CSTMZBL_SITE_YN_ID",
+    "datatype" : "tinyint"
+  }, {
+    "id" : "6",
+    "name" : "SITE_CNTRY_ID",
+    "datatype" : "int"
+  }, {
+    "id" : "7",
+    "name" : "CRE_DATE",
+    "datatype" : "string"
+  }, {
+    "id" : "8",
+    "name" : "UPD_DATE",
+    "datatype" : "string"
+  }, {
+    "id" : "9",
+    "name" : "CRE_USER",
+    "datatype" : "string"
+  }, {
+    "id" : "10",
+    "name" : "UPD_USER",
+    "datatype" : "string"
+  } ],
+  "database" : "edw",
+  "last_modified" : 0
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/examples/test_case_data/localmeta/table_snapshot/TEST_CAL_DT.csv/16337839-a90f-46fa-a67b-37e330fe9685.snapshot
----------------------------------------------------------------------
diff --git a/examples/test_case_data/localmeta/table_snapshot/TEST_CAL_DT.csv/16337839-a90f-46fa-a67b-37e330fe9685.snapshot b/examples/test_case_data/localmeta/table_snapshot/TEST_CAL_DT.csv/16337839-a90f-46fa-a67b-37e330fe9685.snapshot
new file mode 100644
index 0000000..d6b66f0
Binary files /dev/null and b/examples/test_case_data/localmeta/table_snapshot/TEST_CAL_DT.csv/16337839-a90f-46fa-a67b-37e330fe9685.snapshot differ

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/examples/test_case_data/localmeta/table_snapshot/TEST_CAL_DT.csv/4af48c94-86de-4e22-a4fd-c49b06cbaa4f.snapshot
----------------------------------------------------------------------
diff --git a/examples/test_case_data/localmeta/table_snapshot/TEST_CAL_DT.csv/4af48c94-86de-4e22-a4fd-c49b06cbaa4f.snapshot b/examples/test_case_data/localmeta/table_snapshot/TEST_CAL_DT.csv/4af48c94-86de-4e22-a4fd-c49b06cbaa4f.snapshot
new file mode 100644
index 0000000..fab45f7
Binary files /dev/null and b/examples/test_case_data/localmeta/table_snapshot/TEST_CAL_DT.csv/4af48c94-86de-4e22-a4fd-c49b06cbaa4f.snapshot differ

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/examples/test_case_data/localmeta/table_snapshot/TEST_CAL_DT.csv/66de9855-8815-4f3b-8ac6-c9386db0b196.snapshot
----------------------------------------------------------------------
diff --git a/examples/test_case_data/localmeta/table_snapshot/TEST_CAL_DT.csv/66de9855-8815-4f3b-8ac6-c9386db0b196.snapshot b/examples/test_case_data/localmeta/table_snapshot/TEST_CAL_DT.csv/66de9855-8815-4f3b-8ac6-c9386db0b196.snapshot
new file mode 100644
index 0000000..8ce1150
Binary files /dev/null and b/examples/test_case_data/localmeta/table_snapshot/TEST_CAL_DT.csv/66de9855-8815-4f3b-8ac6-c9386db0b196.snapshot differ

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/examples/test_case_data/localmeta/table_snapshot/TEST_CAL_DT.csv/8ff1339e-f804-47f3-b42c-1d4fa4ff0cf7.snapshot
----------------------------------------------------------------------
diff --git a/examples/test_case_data/localmeta/table_snapshot/TEST_CAL_DT.csv/8ff1339e-f804-47f3-b42c-1d4fa4ff0cf7.snapshot b/examples/test_case_data/localmeta/table_snapshot/TEST_CAL_DT.csv/8ff1339e-f804-47f3-b42c-1d4fa4ff0cf7.snapshot
new file mode 100644
index 0000000..5e78425
Binary files /dev/null and b/examples/test_case_data/localmeta/table_snapshot/TEST_CAL_DT.csv/8ff1339e-f804-47f3-b42c-1d4fa4ff0cf7.snapshot differ

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/examples/test_case_data/localmeta/table_snapshot/TEST_CAL_DT.csv/9a1fbc0d-8790-4f32-99e1-7c60a9d124d4.snapshot
----------------------------------------------------------------------
diff --git a/examples/test_case_data/localmeta/table_snapshot/TEST_CAL_DT.csv/9a1fbc0d-8790-4f32-99e1-7c60a9d124d4.snapshot b/examples/test_case_data/localmeta/table_snapshot/TEST_CAL_DT.csv/9a1fbc0d-8790-4f32-99e1-7c60a9d124d4.snapshot
new file mode 100644
index 0000000..f06e2df
Binary files /dev/null and b/examples/test_case_data/localmeta/table_snapshot/TEST_CAL_DT.csv/9a1fbc0d-8790-4f32-99e1-7c60a9d124d4.snapshot differ

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/examples/test_case_data/localmeta/table_snapshot/TEST_CAL_DT.csv/e5f004b5-0be1-4ccf-8950-20f8f86c1999.snapshot
----------------------------------------------------------------------
diff --git a/examples/test_case_data/localmeta/table_snapshot/TEST_CAL_DT.csv/e5f004b5-0be1-4ccf-8950-20f8f86c1999.snapshot b/examples/test_case_data/localmeta/table_snapshot/TEST_CAL_DT.csv/e5f004b5-0be1-4ccf-8950-20f8f86c1999.snapshot
new file mode 100644
index 0000000..d3ee8c7
Binary files /dev/null and b/examples/test_case_data/localmeta/table_snapshot/TEST_CAL_DT.csv/e5f004b5-0be1-4ccf-8950-20f8f86c1999.snapshot differ

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/examples/test_case_data/localmeta/table_snapshot/TEST_CAL_DT.csv/e6cbec7f-0326-443d-b45f-54549d5609c6.snapshot
----------------------------------------------------------------------
diff --git a/examples/test_case_data/localmeta/table_snapshot/TEST_CAL_DT.csv/e6cbec7f-0326-443d-b45f-54549d5609c6.snapshot b/examples/test_case_data/localmeta/table_snapshot/TEST_CAL_DT.csv/e6cbec7f-0326-443d-b45f-54549d5609c6.snapshot
new file mode 100644
index 0000000..9b46b06
Binary files /dev/null and b/examples/test_case_data/localmeta/table_snapshot/TEST_CAL_DT.csv/e6cbec7f-0326-443d-b45f-54549d5609c6.snapshot differ

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/examples/test_case_data/localmeta/table_snapshot/TEST_CAL_DT.csv/f91cd3a0-3ae5-4306-87ab-1624d45a6cde.snapshot
----------------------------------------------------------------------
diff --git a/examples/test_case_data/localmeta/table_snapshot/TEST_CAL_DT.csv/f91cd3a0-3ae5-4306-87ab-1624d45a6cde.snapshot b/examples/test_case_data/localmeta/table_snapshot/TEST_CAL_DT.csv/f91cd3a0-3ae5-4306-87ab-1624d45a6cde.snapshot
new file mode 100644
index 0000000..2c6ba58
Binary files /dev/null and b/examples/test_case_data/localmeta/table_snapshot/TEST_CAL_DT.csv/f91cd3a0-3ae5-4306-87ab-1624d45a6cde.snapshot differ

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/examples/test_case_data/localmeta/table_snapshot/TEST_CATEGORY_GROUPINGS.csv/59a675fc-64d2-40d1-bbd7-492db2510cd1.snapshot
----------------------------------------------------------------------
diff --git a/examples/test_case_data/localmeta/table_snapshot/TEST_CATEGORY_GROUPINGS.csv/59a675fc-64d2-40d1-bbd7-492db2510cd1.snapshot b/examples/test_case_data/localmeta/table_snapshot/TEST_CATEGORY_GROUPINGS.csv/59a675fc-64d2-40d1-bbd7-492db2510cd1.snapshot
new file mode 100644
index 0000000..c3046c3
Binary files /dev/null and b/examples/test_case_data/localmeta/table_snapshot/TEST_CATEGORY_GROUPINGS.csv/59a675fc-64d2-40d1-bbd7-492db2510cd1.snapshot differ

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/examples/test_case_data/localmeta/table_snapshot/TEST_CATEGORY_GROUPINGS.csv/89715f33-15c9-4745-83f9-f2b9817d9100.snapshot
----------------------------------------------------------------------
diff --git a/examples/test_case_data/localmeta/table_snapshot/TEST_CATEGORY_GROUPINGS.csv/89715f33-15c9-4745-83f9-f2b9817d9100.snapshot b/examples/test_case_data/localmeta/table_snapshot/TEST_CATEGORY_GROUPINGS.csv/89715f33-15c9-4745-83f9-f2b9817d9100.snapshot
new file mode 100644
index 0000000..eed588c
Binary files /dev/null and b/examples/test_case_data/localmeta/table_snapshot/TEST_CATEGORY_GROUPINGS.csv/89715f33-15c9-4745-83f9-f2b9817d9100.snapshot differ

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/examples/test_case_data/localmeta/table_snapshot/TEST_CATEGORY_GROUPINGS.csv/df8a90f1-bbbe-49ff-937a-2a2a6e5f8615.snapshot
----------------------------------------------------------------------
diff --git a/examples/test_case_data/localmeta/table_snapshot/TEST_CATEGORY_GROUPINGS.csv/df8a90f1-bbbe-49ff-937a-2a2a6e5f8615.snapshot b/examples/test_case_data/localmeta/table_snapshot/TEST_CATEGORY_GROUPINGS.csv/df8a90f1-bbbe-49ff-937a-2a2a6e5f8615.snapshot
new file mode 100644
index 0000000..9a4ff3d
Binary files /dev/null and b/examples/test_case_data/localmeta/table_snapshot/TEST_CATEGORY_GROUPINGS.csv/df8a90f1-bbbe-49ff-937a-2a2a6e5f8615.snapshot differ

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/examples/test_case_data/localmeta/table_snapshot/TEST_CATEGORY_GROUPINGS.csv/e172b442-ae10-447e-9071-c7dbb2bb38cc.snapshot
----------------------------------------------------------------------
diff --git a/examples/test_case_data/localmeta/table_snapshot/TEST_CATEGORY_GROUPINGS.csv/e172b442-ae10-447e-9071-c7dbb2bb38cc.snapshot b/examples/test_case_data/localmeta/table_snapshot/TEST_CATEGORY_GROUPINGS.csv/e172b442-ae10-447e-9071-c7dbb2bb38cc.snapshot
new file mode 100644
index 0000000..bdc1afa
Binary files /dev/null and b/examples/test_case_data/localmeta/table_snapshot/TEST_CATEGORY_GROUPINGS.csv/e172b442-ae10-447e-9071-c7dbb2bb38cc.snapshot differ

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/examples/test_case_data/localmeta/table_snapshot/TEST_SELLER_TYPE_DIM.csv/4fe75ccd-9b24-4cdf-ac9d-b4038e947f89.snapshot
----------------------------------------------------------------------
diff --git a/examples/test_case_data/localmeta/table_snapshot/TEST_SELLER_TYPE_DIM.csv/4fe75ccd-9b24-4cdf-ac9d-b4038e947f89.snapshot b/examples/test_case_data/localmeta/table_snapshot/TEST_SELLER_TYPE_DIM.csv/4fe75ccd-9b24-4cdf-ac9d-b4038e947f89.snapshot
new file mode 100644
index 0000000..482c778
Binary files /dev/null and b/examples/test_case_data/localmeta/table_snapshot/TEST_SELLER_TYPE_DIM.csv/4fe75ccd-9b24-4cdf-ac9d-b4038e947f89.snapshot differ

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/examples/test_case_data/localmeta/table_snapshot/TEST_SELLER_TYPE_DIM.csv/97238b77-a45d-420f-a249-5f1572ea4d62.snapshot
----------------------------------------------------------------------
diff --git a/examples/test_case_data/localmeta/table_snapshot/TEST_SELLER_TYPE_DIM.csv/97238b77-a45d-420f-a249-5f1572ea4d62.snapshot b/examples/test_case_data/localmeta/table_snapshot/TEST_SELLER_TYPE_DIM.csv/97238b77-a45d-420f-a249-5f1572ea4d62.snapshot
new file mode 100644
index 0000000..d9a04d6
Binary files /dev/null and b/examples/test_case_data/localmeta/table_snapshot/TEST_SELLER_TYPE_DIM.csv/97238b77-a45d-420f-a249-5f1572ea4d62.snapshot differ


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

Posted by lu...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/common/.settings/org.eclipse.core.resources.prefs
----------------------------------------------------------------------
diff --git a/common/.settings/org.eclipse.core.resources.prefs b/common/.settings/org.eclipse.core.resources.prefs
new file mode 100644
index 0000000..365bbd6
--- /dev/null
+++ b/common/.settings/org.eclipse.core.resources.prefs
@@ -0,0 +1,5 @@
+eclipse.preferences.version=1
+encoding//src/main/java=UTF-8
+encoding//src/main/resources=UTF-8
+encoding//src/test/java=UTF-8
+encoding/<project>=UTF-8

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/common/.settings/org.eclipse.jdt.core.prefs
----------------------------------------------------------------------
diff --git a/common/.settings/org.eclipse.jdt.core.prefs b/common/.settings/org.eclipse.jdt.core.prefs
new file mode 100644
index 0000000..a903301
--- /dev/null
+++ b/common/.settings/org.eclipse.jdt.core.prefs
@@ -0,0 +1,379 @@
+eclipse.preferences.version=1
+org.eclipse.jdt.core.compiler.annotation.inheritNullAnnotations=disabled
+org.eclipse.jdt.core.compiler.annotation.missingNonNullByDefaultAnnotation=ignore
+org.eclipse.jdt.core.compiler.annotation.nonnull=org.eclipse.jdt.annotation.NonNull
+org.eclipse.jdt.core.compiler.annotation.nonnullbydefault=org.eclipse.jdt.annotation.NonNullByDefault
+org.eclipse.jdt.core.compiler.annotation.nullable=org.eclipse.jdt.annotation.Nullable
+org.eclipse.jdt.core.compiler.annotation.nullanalysis=disabled
+org.eclipse.jdt.core.compiler.codegen.inlineJsrBytecode=enabled
+org.eclipse.jdt.core.compiler.codegen.targetPlatform=1.7
+org.eclipse.jdt.core.compiler.codegen.unusedLocal=preserve
+org.eclipse.jdt.core.compiler.compliance=1.7
+org.eclipse.jdt.core.compiler.debug.lineNumber=generate
+org.eclipse.jdt.core.compiler.debug.localVariable=generate
+org.eclipse.jdt.core.compiler.debug.sourceFile=generate
+org.eclipse.jdt.core.compiler.problem.annotationSuperInterface=warning
+org.eclipse.jdt.core.compiler.problem.assertIdentifier=error
+org.eclipse.jdt.core.compiler.problem.autoboxing=ignore
+org.eclipse.jdt.core.compiler.problem.comparingIdentical=warning
+org.eclipse.jdt.core.compiler.problem.deadCode=warning
+org.eclipse.jdt.core.compiler.problem.deprecation=warning
+org.eclipse.jdt.core.compiler.problem.deprecationInDeprecatedCode=disabled
+org.eclipse.jdt.core.compiler.problem.deprecationWhenOverridingDeprecatedMethod=disabled
+org.eclipse.jdt.core.compiler.problem.discouragedReference=warning
+org.eclipse.jdt.core.compiler.problem.emptyStatement=ignore
+org.eclipse.jdt.core.compiler.problem.enumIdentifier=error
+org.eclipse.jdt.core.compiler.problem.explicitlyClosedAutoCloseable=ignore
+org.eclipse.jdt.core.compiler.problem.fallthroughCase=ignore
+org.eclipse.jdt.core.compiler.problem.fatalOptionalError=disabled
+org.eclipse.jdt.core.compiler.problem.fieldHiding=ignore
+org.eclipse.jdt.core.compiler.problem.finalParameterBound=warning
+org.eclipse.jdt.core.compiler.problem.finallyBlockNotCompletingNormally=warning
+org.eclipse.jdt.core.compiler.problem.forbiddenReference=warning
+org.eclipse.jdt.core.compiler.problem.hiddenCatchBlock=warning
+org.eclipse.jdt.core.compiler.problem.includeNullInfoFromAsserts=disabled
+org.eclipse.jdt.core.compiler.problem.incompatibleNonInheritedInterfaceMethod=warning
+org.eclipse.jdt.core.compiler.problem.incompleteEnumSwitch=warning
+org.eclipse.jdt.core.compiler.problem.indirectStaticAccess=ignore
+org.eclipse.jdt.core.compiler.problem.localVariableHiding=ignore
+org.eclipse.jdt.core.compiler.problem.methodWithConstructorName=warning
+org.eclipse.jdt.core.compiler.problem.missingDefaultCase=ignore
+org.eclipse.jdt.core.compiler.problem.missingDeprecatedAnnotation=ignore
+org.eclipse.jdt.core.compiler.problem.missingEnumCaseDespiteDefault=disabled
+org.eclipse.jdt.core.compiler.problem.missingHashCodeMethod=ignore
+org.eclipse.jdt.core.compiler.problem.missingOverrideAnnotation=ignore
+org.eclipse.jdt.core.compiler.problem.missingOverrideAnnotationForInterfaceMethodImplementation=enabled
+org.eclipse.jdt.core.compiler.problem.missingSerialVersion=warning
+org.eclipse.jdt.core.compiler.problem.missingSynchronizedOnInheritedMethod=ignore
+org.eclipse.jdt.core.compiler.problem.noEffectAssignment=warning
+org.eclipse.jdt.core.compiler.problem.noImplicitStringConversion=warning
+org.eclipse.jdt.core.compiler.problem.nonExternalizedStringLiteral=ignore
+org.eclipse.jdt.core.compiler.problem.nonnullParameterAnnotationDropped=warning
+org.eclipse.jdt.core.compiler.problem.nullAnnotationInferenceConflict=error
+org.eclipse.jdt.core.compiler.problem.nullReference=warning
+org.eclipse.jdt.core.compiler.problem.nullSpecViolation=error
+org.eclipse.jdt.core.compiler.problem.nullUncheckedConversion=warning
+org.eclipse.jdt.core.compiler.problem.overridingPackageDefaultMethod=warning
+org.eclipse.jdt.core.compiler.problem.parameterAssignment=ignore
+org.eclipse.jdt.core.compiler.problem.possibleAccidentalBooleanAssignment=ignore
+org.eclipse.jdt.core.compiler.problem.potentialNullReference=ignore
+org.eclipse.jdt.core.compiler.problem.potentiallyUnclosedCloseable=ignore
+org.eclipse.jdt.core.compiler.problem.rawTypeReference=warning
+org.eclipse.jdt.core.compiler.problem.redundantNullAnnotation=warning
+org.eclipse.jdt.core.compiler.problem.redundantNullCheck=ignore
+org.eclipse.jdt.core.compiler.problem.redundantSpecificationOfTypeArguments=ignore
+org.eclipse.jdt.core.compiler.problem.redundantSuperinterface=ignore
+org.eclipse.jdt.core.compiler.problem.reportMethodCanBePotentiallyStatic=ignore
+org.eclipse.jdt.core.compiler.problem.reportMethodCanBeStatic=ignore
+org.eclipse.jdt.core.compiler.problem.specialParameterHidingField=disabled
+org.eclipse.jdt.core.compiler.problem.staticAccessReceiver=warning
+org.eclipse.jdt.core.compiler.problem.suppressOptionalErrors=disabled
+org.eclipse.jdt.core.compiler.problem.suppressWarnings=enabled
+org.eclipse.jdt.core.compiler.problem.syntacticNullAnalysisForFields=disabled
+org.eclipse.jdt.core.compiler.problem.syntheticAccessEmulation=ignore
+org.eclipse.jdt.core.compiler.problem.typeParameterHiding=warning
+org.eclipse.jdt.core.compiler.problem.unavoidableGenericTypeProblems=enabled
+org.eclipse.jdt.core.compiler.problem.uncheckedTypeOperation=warning
+org.eclipse.jdt.core.compiler.problem.unclosedCloseable=warning
+org.eclipse.jdt.core.compiler.problem.undocumentedEmptyBlock=ignore
+org.eclipse.jdt.core.compiler.problem.unhandledWarningToken=warning
+org.eclipse.jdt.core.compiler.problem.unnecessaryElse=ignore
+org.eclipse.jdt.core.compiler.problem.unnecessaryTypeCheck=ignore
+org.eclipse.jdt.core.compiler.problem.unqualifiedFieldAccess=ignore
+org.eclipse.jdt.core.compiler.problem.unusedDeclaredThrownException=ignore
+org.eclipse.jdt.core.compiler.problem.unusedDeclaredThrownExceptionExemptExceptionAndThrowable=enabled
+org.eclipse.jdt.core.compiler.problem.unusedDeclaredThrownExceptionIncludeDocCommentReference=enabled
+org.eclipse.jdt.core.compiler.problem.unusedDeclaredThrownExceptionWhenOverriding=disabled
+org.eclipse.jdt.core.compiler.problem.unusedImport=warning
+org.eclipse.jdt.core.compiler.problem.unusedLabel=warning
+org.eclipse.jdt.core.compiler.problem.unusedLocal=warning
+org.eclipse.jdt.core.compiler.problem.unusedObjectAllocation=ignore
+org.eclipse.jdt.core.compiler.problem.unusedParameter=ignore
+org.eclipse.jdt.core.compiler.problem.unusedParameterIncludeDocCommentReference=enabled
+org.eclipse.jdt.core.compiler.problem.unusedParameterWhenImplementingAbstract=disabled
+org.eclipse.jdt.core.compiler.problem.unusedParameterWhenOverridingConcrete=disabled
+org.eclipse.jdt.core.compiler.problem.unusedPrivateMember=warning
+org.eclipse.jdt.core.compiler.problem.unusedTypeParameter=ignore
+org.eclipse.jdt.core.compiler.problem.unusedWarningToken=warning
+org.eclipse.jdt.core.compiler.problem.varargsArgumentNeedCast=warning
+org.eclipse.jdt.core.compiler.source=1.7
+org.eclipse.jdt.core.formatter.align_type_members_on_columns=false
+org.eclipse.jdt.core.formatter.alignment_for_arguments_in_allocation_expression=16
+org.eclipse.jdt.core.formatter.alignment_for_arguments_in_annotation=0
+org.eclipse.jdt.core.formatter.alignment_for_arguments_in_enum_constant=16
+org.eclipse.jdt.core.formatter.alignment_for_arguments_in_explicit_constructor_call=16
+org.eclipse.jdt.core.formatter.alignment_for_arguments_in_method_invocation=16
+org.eclipse.jdt.core.formatter.alignment_for_arguments_in_qualified_allocation_expression=16
+org.eclipse.jdt.core.formatter.alignment_for_assignment=0
+org.eclipse.jdt.core.formatter.alignment_for_binary_expression=16
+org.eclipse.jdt.core.formatter.alignment_for_compact_if=16
+org.eclipse.jdt.core.formatter.alignment_for_conditional_expression=80
+org.eclipse.jdt.core.formatter.alignment_for_enum_constants=0
+org.eclipse.jdt.core.formatter.alignment_for_expressions_in_array_initializer=16
+org.eclipse.jdt.core.formatter.alignment_for_method_declaration=0
+org.eclipse.jdt.core.formatter.alignment_for_multiple_fields=16
+org.eclipse.jdt.core.formatter.alignment_for_parameters_in_constructor_declaration=16
+org.eclipse.jdt.core.formatter.alignment_for_parameters_in_method_declaration=16
+org.eclipse.jdt.core.formatter.alignment_for_resources_in_try=80
+org.eclipse.jdt.core.formatter.alignment_for_selector_in_method_invocation=16
+org.eclipse.jdt.core.formatter.alignment_for_superclass_in_type_declaration=16
+org.eclipse.jdt.core.formatter.alignment_for_superinterfaces_in_enum_declaration=16
+org.eclipse.jdt.core.formatter.alignment_for_superinterfaces_in_type_declaration=16
+org.eclipse.jdt.core.formatter.alignment_for_throws_clause_in_constructor_declaration=16
+org.eclipse.jdt.core.formatter.alignment_for_throws_clause_in_method_declaration=16
+org.eclipse.jdt.core.formatter.alignment_for_union_type_in_multicatch=16
+org.eclipse.jdt.core.formatter.blank_lines_after_imports=1
+org.eclipse.jdt.core.formatter.blank_lines_after_package=1
+org.eclipse.jdt.core.formatter.blank_lines_before_field=0
+org.eclipse.jdt.core.formatter.blank_lines_before_first_class_body_declaration=0
+org.eclipse.jdt.core.formatter.blank_lines_before_imports=1
+org.eclipse.jdt.core.formatter.blank_lines_before_member_type=1
+org.eclipse.jdt.core.formatter.blank_lines_before_method=1
+org.eclipse.jdt.core.formatter.blank_lines_before_new_chunk=1
+org.eclipse.jdt.core.formatter.blank_lines_before_package=0
+org.eclipse.jdt.core.formatter.blank_lines_between_import_groups=1
+org.eclipse.jdt.core.formatter.blank_lines_between_type_declarations=1
+org.eclipse.jdt.core.formatter.brace_position_for_annotation_type_declaration=end_of_line
+org.eclipse.jdt.core.formatter.brace_position_for_anonymous_type_declaration=end_of_line
+org.eclipse.jdt.core.formatter.brace_position_for_array_initializer=end_of_line
+org.eclipse.jdt.core.formatter.brace_position_for_block=end_of_line
+org.eclipse.jdt.core.formatter.brace_position_for_block_in_case=end_of_line
+org.eclipse.jdt.core.formatter.brace_position_for_constructor_declaration=end_of_line
+org.eclipse.jdt.core.formatter.brace_position_for_enum_constant=end_of_line
+org.eclipse.jdt.core.formatter.brace_position_for_enum_declaration=end_of_line
+org.eclipse.jdt.core.formatter.brace_position_for_method_declaration=end_of_line
+org.eclipse.jdt.core.formatter.brace_position_for_switch=end_of_line
+org.eclipse.jdt.core.formatter.brace_position_for_type_declaration=end_of_line
+org.eclipse.jdt.core.formatter.comment.clear_blank_lines_in_block_comment=false
+org.eclipse.jdt.core.formatter.comment.clear_blank_lines_in_javadoc_comment=false
+org.eclipse.jdt.core.formatter.comment.format_block_comments=false
+org.eclipse.jdt.core.formatter.comment.format_header=false
+org.eclipse.jdt.core.formatter.comment.format_html=true
+org.eclipse.jdt.core.formatter.comment.format_javadoc_comments=false
+org.eclipse.jdt.core.formatter.comment.format_line_comments=false
+org.eclipse.jdt.core.formatter.comment.format_source_code=true
+org.eclipse.jdt.core.formatter.comment.indent_parameter_description=true
+org.eclipse.jdt.core.formatter.comment.indent_root_tags=true
+org.eclipse.jdt.core.formatter.comment.insert_new_line_before_root_tags=insert
+org.eclipse.jdt.core.formatter.comment.insert_new_line_for_parameter=insert
+org.eclipse.jdt.core.formatter.comment.line_length=80
+org.eclipse.jdt.core.formatter.comment.new_lines_at_block_boundaries=true
+org.eclipse.jdt.core.formatter.comment.new_lines_at_javadoc_boundaries=true
+org.eclipse.jdt.core.formatter.comment.preserve_white_space_between_code_and_line_comments=false
+org.eclipse.jdt.core.formatter.compact_else_if=true
+org.eclipse.jdt.core.formatter.continuation_indentation=2
+org.eclipse.jdt.core.formatter.continuation_indentation_for_array_initializer=2
+org.eclipse.jdt.core.formatter.disabling_tag=@formatter\:off
+org.eclipse.jdt.core.formatter.enabling_tag=@formatter\:on
+org.eclipse.jdt.core.formatter.format_guardian_clause_on_one_line=false
+org.eclipse.jdt.core.formatter.format_line_comment_starting_on_first_column=true
+org.eclipse.jdt.core.formatter.indent_body_declarations_compare_to_annotation_declaration_header=true
+org.eclipse.jdt.core.formatter.indent_body_declarations_compare_to_enum_constant_header=true
+org.eclipse.jdt.core.formatter.indent_body_declarations_compare_to_enum_declaration_header=true
+org.eclipse.jdt.core.formatter.indent_body_declarations_compare_to_type_header=true
+org.eclipse.jdt.core.formatter.indent_breaks_compare_to_cases=true
+org.eclipse.jdt.core.formatter.indent_empty_lines=false
+org.eclipse.jdt.core.formatter.indent_statements_compare_to_block=true
+org.eclipse.jdt.core.formatter.indent_statements_compare_to_body=true
+org.eclipse.jdt.core.formatter.indent_switchstatements_compare_to_cases=true
+org.eclipse.jdt.core.formatter.indent_switchstatements_compare_to_switch=false
+org.eclipse.jdt.core.formatter.indentation.size=4
+org.eclipse.jdt.core.formatter.insert_new_line_after_annotation_on_field=insert
+org.eclipse.jdt.core.formatter.insert_new_line_after_annotation_on_local_variable=insert
+org.eclipse.jdt.core.formatter.insert_new_line_after_annotation_on_method=insert
+org.eclipse.jdt.core.formatter.insert_new_line_after_annotation_on_package=insert
+org.eclipse.jdt.core.formatter.insert_new_line_after_annotation_on_parameter=do not insert
+org.eclipse.jdt.core.formatter.insert_new_line_after_annotation_on_type=insert
+org.eclipse.jdt.core.formatter.insert_new_line_after_label=do not insert
+org.eclipse.jdt.core.formatter.insert_new_line_after_opening_brace_in_array_initializer=do not insert
+org.eclipse.jdt.core.formatter.insert_new_line_at_end_of_file_if_missing=do not insert
+org.eclipse.jdt.core.formatter.insert_new_line_before_catch_in_try_statement=do not insert
+org.eclipse.jdt.core.formatter.insert_new_line_before_closing_brace_in_array_initializer=do not insert
+org.eclipse.jdt.core.formatter.insert_new_line_before_else_in_if_statement=do not insert
+org.eclipse.jdt.core.formatter.insert_new_line_before_finally_in_try_statement=do not insert
+org.eclipse.jdt.core.formatter.insert_new_line_before_while_in_do_statement=do not insert
+org.eclipse.jdt.core.formatter.insert_new_line_in_empty_annotation_declaration=insert
+org.eclipse.jdt.core.formatter.insert_new_line_in_empty_anonymous_type_declaration=insert
+org.eclipse.jdt.core.formatter.insert_new_line_in_empty_block=insert
+org.eclipse.jdt.core.formatter.insert_new_line_in_empty_enum_constant=insert
+org.eclipse.jdt.core.formatter.insert_new_line_in_empty_enum_declaration=insert
+org.eclipse.jdt.core.formatter.insert_new_line_in_empty_method_body=insert
+org.eclipse.jdt.core.formatter.insert_new_line_in_empty_type_declaration=insert
+org.eclipse.jdt.core.formatter.insert_space_after_and_in_type_parameter=insert
+org.eclipse.jdt.core.formatter.insert_space_after_assignment_operator=insert
+org.eclipse.jdt.core.formatter.insert_space_after_at_in_annotation=do not insert
+org.eclipse.jdt.core.formatter.insert_space_after_at_in_annotation_type_declaration=do not insert
+org.eclipse.jdt.core.formatter.insert_space_after_binary_operator=insert
+org.eclipse.jdt.core.formatter.insert_space_after_closing_angle_bracket_in_type_arguments=insert
+org.eclipse.jdt.core.formatter.insert_space_after_closing_angle_bracket_in_type_parameters=insert
+org.eclipse.jdt.core.formatter.insert_space_after_closing_brace_in_block=insert
+org.eclipse.jdt.core.formatter.insert_space_after_closing_paren_in_cast=insert
+org.eclipse.jdt.core.formatter.insert_space_after_colon_in_assert=insert
+org.eclipse.jdt.core.formatter.insert_space_after_colon_in_case=insert
+org.eclipse.jdt.core.formatter.insert_space_after_colon_in_conditional=insert
+org.eclipse.jdt.core.formatter.insert_space_after_colon_in_for=insert
+org.eclipse.jdt.core.formatter.insert_space_after_colon_in_labeled_statement=insert
+org.eclipse.jdt.core.formatter.insert_space_after_comma_in_allocation_expression=insert
+org.eclipse.jdt.core.formatter.insert_space_after_comma_in_annotation=insert
+org.eclipse.jdt.core.formatter.insert_space_after_comma_in_array_initializer=insert
+org.eclipse.jdt.core.formatter.insert_space_after_comma_in_constructor_declaration_parameters=insert
+org.eclipse.jdt.core.formatter.insert_space_after_comma_in_constructor_declaration_throws=insert
+org.eclipse.jdt.core.formatter.insert_space_after_comma_in_enum_constant_arguments=insert
+org.eclipse.jdt.core.formatter.insert_space_after_comma_in_enum_declarations=insert
+org.eclipse.jdt.core.formatter.insert_space_after_comma_in_explicitconstructorcall_arguments=insert
+org.eclipse.jdt.core.formatter.insert_space_after_comma_in_for_increments=insert
+org.eclipse.jdt.core.formatter.insert_space_after_comma_in_for_inits=insert
+org.eclipse.jdt.core.formatter.insert_space_after_comma_in_method_declaration_parameters=insert
+org.eclipse.jdt.core.formatter.insert_space_after_comma_in_method_declaration_throws=insert
+org.eclipse.jdt.core.formatter.insert_space_after_comma_in_method_invocation_arguments=insert
+org.eclipse.jdt.core.formatter.insert_space_after_comma_in_multiple_field_declarations=insert
+org.eclipse.jdt.core.formatter.insert_space_after_comma_in_multiple_local_declarations=insert
+org.eclipse.jdt.core.formatter.insert_space_after_comma_in_parameterized_type_reference=insert
+org.eclipse.jdt.core.formatter.insert_space_after_comma_in_superinterfaces=insert
+org.eclipse.jdt.core.formatter.insert_space_after_comma_in_type_arguments=insert
+org.eclipse.jdt.core.formatter.insert_space_after_comma_in_type_parameters=insert
+org.eclipse.jdt.core.formatter.insert_space_after_ellipsis=insert
+org.eclipse.jdt.core.formatter.insert_space_after_opening_angle_bracket_in_parameterized_type_reference=do not insert
+org.eclipse.jdt.core.formatter.insert_space_after_opening_angle_bracket_in_type_arguments=do not insert
+org.eclipse.jdt.core.formatter.insert_space_after_opening_angle_bracket_in_type_parameters=do not insert
+org.eclipse.jdt.core.formatter.insert_space_after_opening_brace_in_array_initializer=insert
+org.eclipse.jdt.core.formatter.insert_space_after_opening_bracket_in_array_allocation_expression=do not insert
+org.eclipse.jdt.core.formatter.insert_space_after_opening_bracket_in_array_reference=do not insert
+org.eclipse.jdt.core.formatter.insert_space_after_opening_paren_in_annotation=do not insert
+org.eclipse.jdt.core.formatter.insert_space_after_opening_paren_in_cast=do not insert
+org.eclipse.jdt.core.formatter.insert_space_after_opening_paren_in_catch=do not insert
+org.eclipse.jdt.core.formatter.insert_space_after_opening_paren_in_constructor_declaration=do not insert
+org.eclipse.jdt.core.formatter.insert_space_after_opening_paren_in_enum_constant=do not insert
+org.eclipse.jdt.core.formatter.insert_space_after_opening_paren_in_for=do not insert
+org.eclipse.jdt.core.formatter.insert_space_after_opening_paren_in_if=do not insert
+org.eclipse.jdt.core.formatter.insert_space_after_opening_paren_in_method_declaration=do not insert
+org.eclipse.jdt.core.formatter.insert_space_after_opening_paren_in_method_invocation=do not insert
+org.eclipse.jdt.core.formatter.insert_space_after_opening_paren_in_parenthesized_expression=do not insert
+org.eclipse.jdt.core.formatter.insert_space_after_opening_paren_in_switch=do not insert
+org.eclipse.jdt.core.formatter.insert_space_after_opening_paren_in_synchronized=do not insert
+org.eclipse.jdt.core.formatter.insert_space_after_opening_paren_in_try=do not insert
+org.eclipse.jdt.core.formatter.insert_space_after_opening_paren_in_while=do not insert
+org.eclipse.jdt.core.formatter.insert_space_after_postfix_operator=do not insert
+org.eclipse.jdt.core.formatter.insert_space_after_prefix_operator=do not insert
+org.eclipse.jdt.core.formatter.insert_space_after_question_in_conditional=insert
+org.eclipse.jdt.core.formatter.insert_space_after_question_in_wildcard=do not insert
+org.eclipse.jdt.core.formatter.insert_space_after_semicolon_in_for=insert
+org.eclipse.jdt.core.formatter.insert_space_after_semicolon_in_try_resources=insert
+org.eclipse.jdt.core.formatter.insert_space_after_unary_operator=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_and_in_type_parameter=insert
+org.eclipse.jdt.core.formatter.insert_space_before_assignment_operator=insert
+org.eclipse.jdt.core.formatter.insert_space_before_at_in_annotation_type_declaration=insert
+org.eclipse.jdt.core.formatter.insert_space_before_binary_operator=insert
+org.eclipse.jdt.core.formatter.insert_space_before_closing_angle_bracket_in_parameterized_type_reference=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_closing_angle_bracket_in_type_arguments=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_closing_angle_bracket_in_type_parameters=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_closing_brace_in_array_initializer=insert
+org.eclipse.jdt.core.formatter.insert_space_before_closing_bracket_in_array_allocation_expression=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_closing_bracket_in_array_reference=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_closing_paren_in_annotation=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_closing_paren_in_cast=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_closing_paren_in_catch=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_closing_paren_in_constructor_declaration=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_closing_paren_in_enum_constant=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_closing_paren_in_for=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_closing_paren_in_if=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_closing_paren_in_method_declaration=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_closing_paren_in_method_invocation=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_closing_paren_in_parenthesized_expression=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_closing_paren_in_switch=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_closing_paren_in_synchronized=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_closing_paren_in_try=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_closing_paren_in_while=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_colon_in_assert=insert
+org.eclipse.jdt.core.formatter.insert_space_before_colon_in_case=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_colon_in_conditional=insert
+org.eclipse.jdt.core.formatter.insert_space_before_colon_in_default=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_colon_in_for=insert
+org.eclipse.jdt.core.formatter.insert_space_before_colon_in_labeled_statement=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_comma_in_allocation_expression=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_comma_in_annotation=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_comma_in_array_initializer=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_comma_in_constructor_declaration_parameters=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_comma_in_constructor_declaration_throws=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_comma_in_enum_constant_arguments=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_comma_in_enum_declarations=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_comma_in_explicitconstructorcall_arguments=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_comma_in_for_increments=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_comma_in_for_inits=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_comma_in_method_declaration_parameters=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_comma_in_method_declaration_throws=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_comma_in_method_invocation_arguments=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_comma_in_multiple_field_declarations=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_comma_in_multiple_local_declarations=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_comma_in_parameterized_type_reference=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_comma_in_superinterfaces=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_comma_in_type_arguments=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_comma_in_type_parameters=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_ellipsis=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_opening_angle_bracket_in_parameterized_type_reference=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_opening_angle_bracket_in_type_arguments=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_opening_angle_bracket_in_type_parameters=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_opening_brace_in_annotation_type_declaration=insert
+org.eclipse.jdt.core.formatter.insert_space_before_opening_brace_in_anonymous_type_declaration=insert
+org.eclipse.jdt.core.formatter.insert_space_before_opening_brace_in_array_initializer=insert
+org.eclipse.jdt.core.formatter.insert_space_before_opening_brace_in_block=insert
+org.eclipse.jdt.core.formatter.insert_space_before_opening_brace_in_constructor_declaration=insert
+org.eclipse.jdt.core.formatter.insert_space_before_opening_brace_in_enum_constant=insert
+org.eclipse.jdt.core.formatter.insert_space_before_opening_brace_in_enum_declaration=insert
+org.eclipse.jdt.core.formatter.insert_space_before_opening_brace_in_method_declaration=insert
+org.eclipse.jdt.core.formatter.insert_space_before_opening_brace_in_switch=insert
+org.eclipse.jdt.core.formatter.insert_space_before_opening_brace_in_type_declaration=insert
+org.eclipse.jdt.core.formatter.insert_space_before_opening_bracket_in_array_allocation_expression=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_opening_bracket_in_array_reference=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_opening_bracket_in_array_type_reference=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_opening_paren_in_annotation=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_opening_paren_in_annotation_type_member_declaration=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_opening_paren_in_catch=insert
+org.eclipse.jdt.core.formatter.insert_space_before_opening_paren_in_constructor_declaration=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_opening_paren_in_enum_constant=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_opening_paren_in_for=insert
+org.eclipse.jdt.core.formatter.insert_space_before_opening_paren_in_if=insert
+org.eclipse.jdt.core.formatter.insert_space_before_opening_paren_in_method_declaration=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_opening_paren_in_method_invocation=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_opening_paren_in_parenthesized_expression=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_opening_paren_in_switch=insert
+org.eclipse.jdt.core.formatter.insert_space_before_opening_paren_in_synchronized=insert
+org.eclipse.jdt.core.formatter.insert_space_before_opening_paren_in_try=insert
+org.eclipse.jdt.core.formatter.insert_space_before_opening_paren_in_while=insert
+org.eclipse.jdt.core.formatter.insert_space_before_parenthesized_expression_in_return=insert
+org.eclipse.jdt.core.formatter.insert_space_before_parenthesized_expression_in_throw=insert
+org.eclipse.jdt.core.formatter.insert_space_before_postfix_operator=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_prefix_operator=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_question_in_conditional=insert
+org.eclipse.jdt.core.formatter.insert_space_before_question_in_wildcard=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_semicolon=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_semicolon_in_for=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_semicolon_in_try_resources=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_unary_operator=do not insert
+org.eclipse.jdt.core.formatter.insert_space_between_brackets_in_array_type_reference=do not insert
+org.eclipse.jdt.core.formatter.insert_space_between_empty_braces_in_array_initializer=do not insert
+org.eclipse.jdt.core.formatter.insert_space_between_empty_brackets_in_array_allocation_expression=do not insert
+org.eclipse.jdt.core.formatter.insert_space_between_empty_parens_in_annotation_type_member_declaration=do not insert
+org.eclipse.jdt.core.formatter.insert_space_between_empty_parens_in_constructor_declaration=do not insert
+org.eclipse.jdt.core.formatter.insert_space_between_empty_parens_in_enum_constant=do not insert
+org.eclipse.jdt.core.formatter.insert_space_between_empty_parens_in_method_declaration=do not insert
+org.eclipse.jdt.core.formatter.insert_space_between_empty_parens_in_method_invocation=do not insert
+org.eclipse.jdt.core.formatter.join_lines_in_comments=true
+org.eclipse.jdt.core.formatter.join_wrapped_lines=true
+org.eclipse.jdt.core.formatter.keep_else_statement_on_same_line=false
+org.eclipse.jdt.core.formatter.keep_empty_array_initializer_on_one_line=false
+org.eclipse.jdt.core.formatter.keep_imple_if_on_one_line=false
+org.eclipse.jdt.core.formatter.keep_then_statement_on_same_line=false
+org.eclipse.jdt.core.formatter.lineSplit=999
+org.eclipse.jdt.core.formatter.never_indent_block_comments_on_first_column=false
+org.eclipse.jdt.core.formatter.never_indent_line_comments_on_first_column=false
+org.eclipse.jdt.core.formatter.number_of_blank_lines_at_beginning_of_method_body=0
+org.eclipse.jdt.core.formatter.number_of_empty_lines_to_preserve=1
+org.eclipse.jdt.core.formatter.put_empty_statement_on_new_line=true
+org.eclipse.jdt.core.formatter.tabulation.char=space
+org.eclipse.jdt.core.formatter.tabulation.size=4
+org.eclipse.jdt.core.formatter.use_on_off_tags=false
+org.eclipse.jdt.core.formatter.use_tabs_only_for_leading_indentations=false
+org.eclipse.jdt.core.formatter.wrap_before_binary_operator=true
+org.eclipse.jdt.core.formatter.wrap_before_or_operator_multicatch=true
+org.eclipse.jdt.core.formatter.wrap_outer_expressions_when_nested=true

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/common/.settings/org.eclipse.jdt.ui.prefs
----------------------------------------------------------------------
diff --git a/common/.settings/org.eclipse.jdt.ui.prefs b/common/.settings/org.eclipse.jdt.ui.prefs
new file mode 100644
index 0000000..dece0e6
--- /dev/null
+++ b/common/.settings/org.eclipse.jdt.ui.prefs
@@ -0,0 +1,7 @@
+eclipse.preferences.version=1
+formatter_profile=_Space Indent & Long Lines
+formatter_settings_version=12
+org.eclipse.jdt.ui.ignorelowercasenames=true
+org.eclipse.jdt.ui.importorder=java;javax;org;com;
+org.eclipse.jdt.ui.ondemandthreshold=99
+org.eclipse.jdt.ui.staticondemandthreshold=1

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/common/pom.xml
----------------------------------------------------------------------
diff --git a/common/pom.xml b/common/pom.xml
new file mode 100644
index 0000000..32a46c6
--- /dev/null
+++ b/common/pom.xml
@@ -0,0 +1,88 @@
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+    <modelVersion>4.0.0</modelVersion>
+
+    <artifactId>kylin-common</artifactId>
+    <packaging>jar</packaging>
+    <name>Kylin:Common</name>
+
+    <parent>
+        <groupId>com.kylinolap</groupId>
+        <artifactId>kylin</artifactId>
+        <version>0.6.3-SNAPSHOT</version>
+    </parent>
+
+    <properties>
+    </properties>
+
+    <dependencies>
+        <dependency>
+            <groupId>com.fasterxml.jackson.core</groupId>
+            <artifactId>jackson-databind</artifactId>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.commons</groupId>
+            <artifactId>commons-lang3</artifactId>
+        </dependency>
+        <dependency>
+            <groupId>commons-io</groupId>
+            <artifactId>commons-io</artifactId>
+        </dependency>
+        <dependency>
+            <groupId>commons-configuration</groupId>
+            <artifactId>commons-configuration</artifactId>
+        </dependency>
+        <dependency>
+            <groupId>com.google.guava</groupId>
+            <artifactId>guava</artifactId>
+        </dependency>
+        <dependency>
+            <groupId>com.jcraft</groupId>
+            <artifactId>jsch</artifactId>
+        </dependency>
+        <dependency>
+            <groupId>com.ning</groupId>
+            <artifactId>compress-lzf</artifactId>
+        </dependency>
+
+        <!-- Env & Test -->
+        <dependency>
+            <groupId>org.apache.commons</groupId>
+            <artifactId>commons-email</artifactId>
+            <version>1.1</version>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.hadoop</groupId>
+            <artifactId>hadoop-common</artifactId>
+            <scope>provided</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.hadoop</groupId>
+            <artifactId>hadoop-hdfs</artifactId>
+            <scope>provided</scope>
+            <!-- protobuf version conflict with hbase -->
+            <exclusions>
+                <exclusion>
+                    <groupId>com.google.protobuf</groupId>
+                    <artifactId>protobuf-java</artifactId>
+                </exclusion>
+            </exclusions>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.hbase</groupId>
+            <artifactId>hbase-common</artifactId>
+            <scope>provided</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.hbase</groupId>
+            <artifactId>hbase-client</artifactId>
+            <scope>provided</scope>
+        </dependency>
+        <dependency>
+            <groupId>junit</groupId>
+            <artifactId>junit</artifactId>
+            <scope>test</scope>
+        </dependency>
+    </dependencies>
+
+
+</project>

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/common/src/main/java/com/kylinolap/common/KylinConfig.java
----------------------------------------------------------------------
diff --git a/common/src/main/java/com/kylinolap/common/KylinConfig.java b/common/src/main/java/com/kylinolap/common/KylinConfig.java
new file mode 100644
index 0000000..34b348d
--- /dev/null
+++ b/common/src/main/java/com/kylinolap/common/KylinConfig.java
@@ -0,0 +1,630 @@
+/*
+ * 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.common;
+
+import java.io.ByteArrayOutputStream;
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.io.InputStream;
+
+import org.apache.commons.configuration.ConfigurationException;
+import org.apache.commons.configuration.PropertiesConfiguration;
+import org.apache.commons.io.IOUtils;
+import org.apache.commons.lang.StringUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.kylinolap.common.restclient.RestClient;
+import com.kylinolap.common.util.CliCommandExecutor;
+
+/**
+ * @author yangli9
+ */
+public class KylinConfig {
+
+    public static final String KYLIN_STORAGE_URL = "kylin.storage.url";
+
+    public static final String PROP_SCAN_THRESHOLD = "PROP_SCAN_THRESHOLD";
+
+    public static final String KYLIN_METADATA_URL = "kylin.metadata.url";
+
+    public static final String KYLIN_REST_SERVERS = "kylin.rest.servers";
+
+    public static final String KYLIN_REST_TIMEZONE = "kylin.rest.timezone";
+    /**
+     * The dir containing scripts for kylin. For example: /usr/lib/kylin/bin
+     */
+    public static final String KYLIN_SCRIPT_DIR = "kylin.script.dir";
+    /**
+     * The script file name for generating table metadat from hive. For example:
+     * generateTable.sh
+     */
+    public static final String KYLIN_SCRIPT_GEN_TABLE_META = "kylin.script.genTableMeta";
+
+    public static final String KYLIN_JOB_CONCURRENT_MAX_LIMIT = "kylin.job.concurrent.max.limit";
+
+    public static final String KYLIN_JOB_YARN_APP_REST_CHECK_STATUS_URL = "kylin.job.yarn.app.rest.check.status.url";
+
+    public static final String KYLIN_JOB_YARN_APP_REST_CHECK_INTERVAL_SECONDS = "kylin.job.yarn.app.rest.check.interval.seconds";
+
+    public static final String KYLIN_TMP_HDFS_DIR = "kylin.tmp.hdfs.dir";
+
+    public static final String HIVE_TABLE_LOCATION_PREFIX = "hive.table.location.";
+
+    public static final String KYLIN_JOB_REMOTE_CLI_PASSWORD = "kylin.job.remote.cli.password";
+
+    public static final String KYLIN_JOB_REMOTE_CLI_USERNAME = "kylin.job.remote.cli.username";
+
+    public static final String KYLIN_JOB_REMOTE_CLI_HOSTNAME = "kylin.job.remote.cli.hostname";
+
+    public static final String KYLIN_JOB_REMOTE_CLI_WORKING_DIR = "kylin.job.remote.cli.working.dir";
+    
+    public static final String KYLIN_JOB_CMD_EXTRA_ARGS = "kylin.job.cmd.extra.args";
+    /**
+     * Toggle to indicate whether to use hive for table flattening. Default
+     * true.
+     */
+    public static final String KYLIN_JOB_HIVE_FLATTEN = "kylin.job.hive.flatten";
+
+    public static final String KYLIN_JOB_RUN_AS_REMOTE_CMD = "kylin.job.run.as.remote.cmd";
+
+    public static final String KYLIN_JOB_MAPREDUCE_DEFAULT_REDUCE_COUNT_RATIO = "kylin.job.mapreduce.default.reduce.count.ratio";
+
+    public static final String KYLIN_JOB_MAPREDUCE_DEFAULT_REDUCE_INPUT_MB = "kylin.job.mapreduce.default.reduce.input.mb";
+
+    public static final String KYLIN_JOB_MAPREDUCE_MAX_REDUCER_NUMBER = "kylin.job.mapreduce.max.reducer.number";
+
+    public static final String KYLIN_JOB_JAR = "kylin.job.jar";
+
+    public static final String COPROCESSOR_LOCAL_JAR = "kylin.coprocessor.local.jar";
+    public static final String COPROCESSOR_SCAN_BITS_THRESHOLD = "kylin.coprocessor.scan.bits.threshold";
+
+    public static final String KYLIN_JOB_JAR_LOCAL = "kylin.job.jar.local";
+
+    public static final String KYLIN_JOB_LOG_DIR = "kylin.job.log.dir";
+
+    public static final String KYLIN_HDFS_WORKING_DIR = "kylin.hdfs.working.dir";
+
+    public static final String HIVE_PASSWORD = "hive.password";
+
+    public static final String HIVE_USER = "hive.user";
+
+    public static final String HIVE_URL = "hive.url";
+    /**
+     * Key string to point to the kylin conf directory
+     */
+    public static final String KYLIN_CONF = "KYLIN_CONF";
+    /**
+     * Key string to specify the kylin evn: prod, dev, qa
+     */
+    public static final String KYLIN_ENV = "KYLIN_ENV";
+    /**
+     * Default Kylin conf path
+     */
+    public static final String KYLIN_CONF_DEFAULT = "/etc/kylin";
+    /**
+     * Kylin properties file
+     */
+    public static final String KYLIN_CONF_PROPERTIES_FILE = "kylin.properties";
+
+    public static final String MAIL_ENABLED = "mail.enabled";
+    
+    public static final String MAIL_HOST = "mail.host";
+    
+    public static final String MAIL_USERNAME = "mail.username";
+    
+    public static final String MAIL_PASSWORD = "mail.password";
+    
+    public static final String MAIL_SENDER = "mail.sender";
+
+    private static final Logger logger = LoggerFactory.getLogger(KylinConfig.class);
+
+    // static cached instances
+    private static KylinConfig ENV_INSTANCE = null;
+
+    public static KylinConfig getInstanceFromEnv() {
+        if (ENV_INSTANCE == null) {
+            try {
+                KylinConfig config = loadKylinConfig();
+                ENV_INSTANCE = config;
+            } catch (IllegalArgumentException e) {
+                throw new IllegalStateException("Failed to find KylinConfig ", e);
+            }
+        }
+        return ENV_INSTANCE;
+    }
+
+    public static void destoryInstance() {
+        ENV_INSTANCE = null;
+    }
+
+
+    /**
+     * This method only for test case. You can get a KylinConfig instance by
+     * path "/a/b/c", where "/a/b/c/kylin.properties" exists. By default, the
+     * getInstanceFromEnv() should be called.
+     *
+     * @param confPath
+     * @return
+     * @deprecated
+     */
+    public static KylinConfig getInstanceForTest(String confPath) {
+        File file = new File(confPath);
+        if (!file.exists() || !file.isDirectory()) {
+            throw new IllegalArgumentException(confPath + " is not a valid path");
+        }
+
+        String env = System.getProperty(KYLIN_CONF);
+        System.setProperty(KYLIN_CONF, confPath);
+        KylinConfig config = getInstanceFromEnv();
+        if (env == null) {
+            System.clearProperty(KYLIN_CONF);
+        } else {
+            System.setProperty(KYLIN_CONF, env);
+        }
+        return config;
+    }
+
+    public static enum UriType {
+        PROPERTIES_FILE, REST_ADDR, LOCAL_FOLDER
+    }
+
+    private static UriType decideUriType(String metaUri) {
+
+        try {
+            File file = new File(metaUri);
+            if (file.exists()) {
+                if (file.isDirectory()) {
+                    return UriType.LOCAL_FOLDER;
+                } else if (file.isFile()) {
+                    if (file.getName().equalsIgnoreCase(KYLIN_CONF_PROPERTIES_FILE)) {
+                        return UriType.PROPERTIES_FILE;
+                    } else {
+                        throw new IllegalStateException("Metadata uri : " + metaUri + " is a local file but not kylin.properties");
+                    }
+                }
+            } else {
+                if (RestClient.matchFullRestPattern(metaUri))
+                    return UriType.REST_ADDR;
+                else
+                    throw new IllegalStateException("Metadata uri : " + metaUri + " is not a valid REST URI address");
+            }
+        } catch (Exception e) {
+            logger.info(e.getLocalizedMessage());
+            throw new IllegalStateException("Metadata uri : " + metaUri + " is not recognized");
+        }
+
+        return null;
+    }
+
+    public static KylinConfig createInstanceFromUri(String uri) {
+        /**
+         * --hbase: 1. PROPERTIES_FILE: path to kylin.properties 2. REST_ADDR:
+         * rest service resource, format: user:password@host:port --local: 1.
+         * LOCAL_FOLDER: path to resource folder
+         */
+        UriType uriType = decideUriType(uri);
+        logger.info("The URI " + uri + " is recognized as " + uriType);
+
+        if (uriType == UriType.LOCAL_FOLDER) {
+            KylinConfig config = new KylinConfig();
+            config.setMetadataUrl(uri);
+            return config;
+        } else if (uriType == UriType.PROPERTIES_FILE) {
+            KylinConfig config;
+            try {
+                config = new KylinConfig();
+                InputStream is = new FileInputStream(uri);
+                config.reloadKylinConfig(is);
+                is.close();
+            } catch (IOException e) {
+                throw new RuntimeException(e);
+            }
+            return config;
+        } else {// rest_addr
+            try {
+                KylinConfig config = new KylinConfig();
+                RestClient client = new RestClient(uri);
+                String propertyText = client.getKylinProperties();
+                InputStream is = IOUtils.toInputStream(propertyText);
+                config.reloadKylinConfig(is);
+                is.close();
+                return config;
+            } catch (IOException e) {
+                throw new RuntimeException(e);
+            }
+        }
+    }
+
+    public static KylinConfig getKylinConfigFromInputStream(InputStream is) {
+        KylinConfig config = new KylinConfig();
+        config.reloadKylinConfig(is);
+        return config;
+    }
+
+    // ============================================================================
+
+    /**
+     * Find config from environment. The Search process: 1. Check the
+     * $KYLIN_CONF/kylin.properties 2. Check the /etc/kylin/kylin.properties 3.
+     * Check the kylin.properties in classpath
+     *
+     * @return
+     */
+    private static KylinConfig loadKylinConfig() {
+        InputStream is = getKylinPropertiesAsInputSteam();
+        if (is == null) {
+            throw new IllegalArgumentException("Failed to load kylin config");
+        }
+        KylinConfig config = new KylinConfig();
+        config.reloadKylinConfig(is);
+        return config;
+    }
+
+    private PropertiesConfiguration kylinConfig = new PropertiesConfiguration();
+
+    public CliCommandExecutor getCliCommandExecutor() throws IOException {
+        CliCommandExecutor exec = new CliCommandExecutor();
+        if (getRunAsRemoteCommand()) {
+            exec.setRunAtRemote(getRemoteHadoopCliHostname(), getRemoteHadoopCliUsername(), getRemoteHadoopCliPassword());
+        }
+        return exec;
+    }
+
+    // ============================================================================
+
+    public String getStorageUrl() {
+        return getOptional(KYLIN_STORAGE_URL);
+    }
+
+    public String getZookeeperString() {
+        // storage url format is: hbase:zookeeperHostname:port:/znode
+
+        String zookeeperString = "";
+        String storageString = this.getStorageUrl();
+        if (!storageString.startsWith("hbase:")) {
+            return "";
+        }
+
+        int cut = storageString.indexOf("/");
+        if (cut != -1) {
+            zookeeperString = storageString.substring(6, cut - 1);
+        } else {
+            zookeeperString = storageString.substring(6);
+        }
+        return zookeeperString;
+    }
+
+    public String getHiveUrl() {
+        return getOptional(HIVE_URL, "");
+    }
+
+    public String getHiveUser() {
+        return getOptional(HIVE_USER, "");
+    }
+
+    public String getHivePassword() {
+        return getOptional(HIVE_PASSWORD, "");
+    }
+
+    public String getHdfsWorkingDirectory() {
+        return getRequired(KYLIN_HDFS_WORKING_DIR);
+    }
+
+    public String getKylinJobLogDir() {
+        return getOptional(KYLIN_JOB_LOG_DIR, "/tmp/kylin/logs");
+    }
+
+    public String getKylinJobJarPath() {
+        return getRequired(KYLIN_JOB_JAR);
+    }
+    
+    public void overrideKylinJobJarPath(String path) {
+        kylinConfig.setProperty(KYLIN_JOB_JAR, path);
+    }
+
+    public String getCoprocessorLocalJar() {
+        return getRequired(COPROCESSOR_LOCAL_JAR);
+    }
+
+    public void overrideCoprocessorLocalJar(String path) {
+        kylinConfig.setProperty(COPROCESSOR_LOCAL_JAR, path);
+    }
+
+    public int getCoprocessorScanBitsThreshold() {
+        return Integer.parseInt(getOptional(COPROCESSOR_SCAN_BITS_THRESHOLD, "32"));
+    }
+
+    public double getDefaultHadoopJobReducerInputMB() {
+        return Double.parseDouble(getOptional(KYLIN_JOB_MAPREDUCE_DEFAULT_REDUCE_INPUT_MB, "500"));
+    }
+
+    public double getDefaultHadoopJobReducerCountRatio() {
+        return Double.parseDouble(getOptional(KYLIN_JOB_MAPREDUCE_DEFAULT_REDUCE_COUNT_RATIO, "1.0"));
+    }
+
+    public int getHadoopJobMaxReducerNumber() {
+        return Integer.parseInt(getOptional(KYLIN_JOB_MAPREDUCE_MAX_REDUCER_NUMBER, "5000"));
+    }
+
+    public boolean getRunAsRemoteCommand() {
+        return Boolean.parseBoolean(getOptional(KYLIN_JOB_RUN_AS_REMOTE_CMD));
+    }
+
+    public String getRemoteHadoopCliHostname() {
+        return getOptional(KYLIN_JOB_REMOTE_CLI_HOSTNAME);
+    }
+
+    public String getRemoteHadoopCliUsername() {
+        return getOptional(KYLIN_JOB_REMOTE_CLI_USERNAME);
+    }
+
+    public String getRemoteHadoopCliPassword() {
+        return getOptional(KYLIN_JOB_REMOTE_CLI_PASSWORD);
+    }
+
+    public String getCliWorkingDir() {
+        return getOptional(KYLIN_JOB_REMOTE_CLI_WORKING_DIR);
+    }
+
+    public String getMapReduceCmdExtraArgs() {
+        return getOptional(KYLIN_JOB_CMD_EXTRA_ARGS);
+    }
+
+    public boolean getFlatTableByHive() {
+        return Boolean.parseBoolean(getOptional(KYLIN_JOB_HIVE_FLATTEN, "true"));
+    }
+
+    public String getOverrideHiveTableLocation(String table) {
+        return getOptional(HIVE_TABLE_LOCATION_PREFIX + table.toUpperCase());
+    }
+
+    public String getTempHDFSDir() {
+        return getOptional(KYLIN_TMP_HDFS_DIR, "/tmp/kylin");
+    }
+
+    public String getYarnStatusServiceUrl() {
+        return getOptional(KYLIN_JOB_YARN_APP_REST_CHECK_STATUS_URL, null);
+    }
+
+    public int getYarnStatusCheckIntervalSeconds() {
+        return Integer.parseInt(getOptional(KYLIN_JOB_YARN_APP_REST_CHECK_INTERVAL_SECONDS, "60"));
+    }
+
+    /**
+     * @return
+     */
+    public int getMaxConcurrentJobLimit() {
+        return Integer.parseInt(getOptional(KYLIN_JOB_CONCURRENT_MAX_LIMIT, "10"));
+    }
+
+    /**
+     * @return
+     */
+    public String getTimeZone() {
+        return getOptional(KYLIN_REST_TIMEZONE, "PST");
+    }
+
+    public String[] getRestServers() {
+        String nodes = getOptional(KYLIN_REST_SERVERS);
+        if (nodes == null)
+            return null;
+        return nodes.split("\\s*,\\s*");
+    }
+
+    /**
+     * @return
+     */
+    public String getAdminDls() {
+        return getOptional("kylin.job.admin.dls", null);
+    }
+
+    public int getScanThreshold() {
+        return Integer.parseInt(getOptional("kylin.query.scan.threshold", "10000000"));
+    }
+
+    /**
+     * @return
+     */
+    public long getJobStepTimeout() {
+        return Long.parseLong(getOptional("kylin.job.step.timeout", String.valueOf(2 * 60 * 60)));
+    }
+
+    public String getServerMode() {
+        return this.getOptional("kylin.server.mode", "all");
+    }
+
+    public Long getQueryDurationCacheThreshold() {
+        return Long.parseLong(this.getOptional("kylin.query.cache.threshold.duration", String.valueOf(2000)));
+    }
+
+    public Long getQueryScanCountCacheThreshold() {
+        return Long.parseLong(this.getOptional("kylin.query.cache.threshold.scancount", String.valueOf(10 * 1024)));
+    }
+
+    public boolean isQuerySecureEnabled() {
+        return Boolean.parseBoolean(this.getOptional("kylin.query.security.enabled", "false"));
+    }
+
+    public int getConcurrentScanThreadCount() {
+        return Integer.parseInt(this.getOptional("kylin.query.scan.thread.count", "40"));
+    }
+
+    public boolean isQueryCacheEnabled() {
+        return Boolean.parseBoolean(this.getOptional("kylin.query.cache.enabled", "true"));
+    }
+
+    public int getHBaseKeyValueSize() {
+        return Integer.parseInt(this.getOptional("kylin.hbase.client.keyvalue.maxsize", "10485760"));
+    }
+
+    private String getOptional(String prop) {
+        return kylinConfig.getString(prop);
+    }
+
+    private String getOptional(String prop, String dft) {
+        return kylinConfig.getString(prop, dft);
+    }
+
+    private String getRequired(String prop) {
+        String r = kylinConfig.getString(prop);
+        if (StringUtils.isEmpty(r))
+            throw new IllegalArgumentException("missing '" + prop + "' in conf/kylin_instance.properties");
+        return r;
+    }
+
+    void reloadKylinConfig(InputStream is) {
+        PropertiesConfiguration config = new PropertiesConfiguration();
+        try {
+            config.load(is);
+        } catch (ConfigurationException e) {
+            throw new RuntimeException("Cannot load kylin config.", e);
+        } finally {
+            try {
+                is.close();
+            } catch (IOException e) {
+                logger.error("Failed to close inputstream.", e);
+            }
+        }
+        this.kylinConfig = config;
+    }
+
+    public void writeProperties(File file) throws IOException {
+        try {
+            kylinConfig.save(file);
+        } catch (ConfigurationException ex) {
+            throw new IOException("Error writing KylinConfig to " + file, ex);
+        }
+    }
+
+    public static InputStream getKylinPropertiesAsInputSteam() {
+        File propFile = null;
+        
+        // 1st, find conf path from env
+        String path = System.getProperty(KYLIN_CONF);
+        if (path == null) {
+            path = System.getenv(KYLIN_CONF);
+        }
+        propFile = getKylinPropertiesFile(path);
+        
+        // 2nd, find /etc/kylin
+        if (propFile == null) {
+            propFile = getKylinPropertiesFile(KYLIN_CONF_DEFAULT);
+        }
+        if (propFile != null) {
+            logger.debug("Loading property file " + propFile.getAbsolutePath());
+            try {
+                return new FileInputStream(propFile);
+            } catch (FileNotFoundException e) {
+                logger.warn("Failed to read properties " + propFile.getAbsolutePath() + " and skip");
+            }
+        }
+        
+        // 3rd, find classpath
+        logger.info("Search " + KYLIN_CONF_PROPERTIES_FILE + " from classpath ...");
+        InputStream is = KylinConfig.class.getClassLoader().getResourceAsStream("kylin.properties");
+        if (is == null) {
+            logger.info("Did not find properties file " + KYLIN_CONF_PROPERTIES_FILE + " from classpath");
+        }
+        return is;
+    }
+
+    /**
+     * Check if there is kylin.properties exist
+     *
+     *
+     * @param path
+     * @param env
+     * @return the properties file
+     */
+    private static File getKylinPropertiesFile(String path) {
+        if (path == null)
+            return null;
+        
+        File propFile = new File(path, KYLIN_CONF_PROPERTIES_FILE);
+        if (propFile.exists()) {
+            logger.info(KYLIN_CONF_PROPERTIES_FILE + " was found at " + propFile.getAbsolutePath());
+            return propFile;
+        }
+        
+        logger.info(KYLIN_CONF_PROPERTIES_FILE + " was NOT found at " + propFile.getAbsolutePath());
+        return null;
+    }
+
+    public String getMetadataUrl() {
+        return getOptional(KYLIN_METADATA_URL);
+    }
+
+    public String getMetadataUrlPrefix() {
+        String hbaseMetadataUrl = getMetadataUrl();
+        String defaultPrefix = "kylin_metadata";
+
+        if (org.apache.commons.lang3.StringUtils.containsIgnoreCase(hbaseMetadataUrl, "hbase:")) {
+            int cut = hbaseMetadataUrl.indexOf('@');
+            String tmp = cut < 0 ? defaultPrefix : hbaseMetadataUrl.substring(0, cut);
+            return tmp;
+        } else {
+            return defaultPrefix;
+        }
+    }
+
+    public void setMetadataUrl(String metadataUrl) {
+        kylinConfig.setProperty(KYLIN_METADATA_URL, metadataUrl);
+    }
+
+    /**
+     * return -1 if there is no setting
+     *
+     * @return
+     */
+    public int getPropScanThreshold() {
+        return kylinConfig.getInt(PROP_SCAN_THRESHOLD, -1);
+    }
+
+    public String getProperty(String key, String defaultValue) {
+        return kylinConfig.getString(key, defaultValue);
+    }
+
+    /**
+     * Set a new key:value into the kylin config.
+     *
+     * @param key
+     * @param value
+     */
+    public void setProperty(String key, String value) {
+        logger.info("Kylin Config was updated with " + key + " : " + value);
+        kylinConfig.setProperty(key, value);
+    }
+
+    public String getConfigAsString() throws IOException {
+        try {
+            ByteArrayOutputStream baos = new ByteArrayOutputStream();
+            kylinConfig.save(baos);
+            String content = baos.toString();
+            return content;
+        } catch (ConfigurationException ex) {
+            throw new IOException("Error writing KylinConfig to String", ex);
+        }
+    }
+
+    public String toString() {
+        return getMetadataUrl();
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/common/src/main/java/com/kylinolap/common/hll/HyperLogLogPlusCounter.java
----------------------------------------------------------------------
diff --git a/common/src/main/java/com/kylinolap/common/hll/HyperLogLogPlusCounter.java b/common/src/main/java/com/kylinolap/common/hll/HyperLogLogPlusCounter.java
new file mode 100644
index 0000000..06d2f5f
--- /dev/null
+++ b/common/src/main/java/com/kylinolap/common/hll/HyperLogLogPlusCounter.java
@@ -0,0 +1,357 @@
+/*
+ * 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.common.hll;
+
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import java.util.zip.GZIPInputStream;
+import java.util.zip.GZIPOutputStream;
+
+import org.apache.commons.compress.utils.IOUtils;
+
+import com.google.common.hash.HashFunction;
+import com.google.common.hash.Hashing;
+import com.kylinolap.common.util.BytesUtil;
+import com.ning.compress.lzf.LZFDecoder;
+import com.ning.compress.lzf.LZFEncoder;
+
+/**
+ * About compression, test on HLLC data shows
+ * 
+ * - LZF compression ratio is around 65%-80%, fast - GZIP compression ratio is
+ * around 41%-46%, very slow
+ * 
+ * @author yangli9
+ */
+public class HyperLogLogPlusCounter implements Comparable<HyperLogLogPlusCounter> {
+
+    private final int p;
+    private final int m;
+    private final HashFunction hashFunc;
+    byte[] registers;
+
+    public HyperLogLogPlusCounter() {
+        this(10);
+    }
+
+    public HyperLogLogPlusCounter(int p) {
+        this(p, Hashing.murmur3_128());
+    }
+
+    public HyperLogLogPlusCounter(HyperLogLogPlusCounter another) {
+        this(another.p, another.hashFunc);
+        merge(another);
+    }
+
+    /** The larger p is, the more storage (2^p bytes), the better accuracy */
+    private HyperLogLogPlusCounter(int p, HashFunction hashFunc) {
+        this.p = p;
+        this.m = (int) Math.pow(2, p);
+        this.hashFunc = hashFunc;
+        this.registers = new byte[m];
+    }
+
+    public void clear() {
+        for (int i = 0; i < m; i++)
+            registers[i] = 0;
+    }
+
+    public void add(String value) {
+        add(hashFunc.hashString(value).asLong());
+    }
+
+    public void add(byte[] value) {
+        add(hashFunc.hashBytes(value).asLong());
+    }
+
+    protected void add(long hash) {
+        int bucketMask = m - 1;
+        int bucket = (int) (hash & bucketMask);
+        int firstOnePos = Long.numberOfLeadingZeros(hash | bucketMask) + 1;
+
+        if (firstOnePos > registers[bucket])
+            registers[bucket] = (byte) firstOnePos;
+    }
+
+    public void merge(HyperLogLogPlusCounter another) {
+        assert this.p == another.p;
+        assert this.hashFunc == another.hashFunc;
+
+        for (int i = 0; i < m; i++) {
+            if (registers[i] < another.registers[i])
+                registers[i] = another.registers[i];
+        }
+    }
+
+    public long getCountEstimate() {
+        return new HLLCSnapshot(this).getCountEstimate();
+    }
+
+    public int getMemBytes() {
+        return 12 + m;
+    }
+
+    public double getErrorRate() {
+        return 1.04 / Math.sqrt(m);
+    }
+
+    private int size() {
+        int size = 0;
+        for (int i = 0; i < m; i++) {
+            if (registers[i] > 0)
+                size++;
+        }
+        return size;
+    }
+
+    // ============================================================================
+
+    // a memory efficient snapshot of HLL registers which can yield count
+    // estimate later
+    public static class HLLCSnapshot {
+        byte p;
+        double registerSum;
+        int zeroBuckets;
+
+        public HLLCSnapshot(HyperLogLogPlusCounter hllc) {
+            p = (byte) hllc.p;
+            registerSum = 0;
+            zeroBuckets = 0;
+
+            byte[] registers = hllc.registers;
+            for (int i = 0; i < hllc.m; i++) {
+                if (registers[i] == 0) {
+                    registerSum++;
+                    zeroBuckets++;
+                } else {
+                    registerSum += 1.0 / (1 << registers[i]);
+                }
+            }
+        }
+
+        public long getCountEstimate() {
+            int m = (int) Math.pow(2, p);
+            double alpha = 1 / (2 * Math.log(2) * (1 + (3 * Math.log(2) - 1) / m));
+            double alphaMM = alpha * m * m;
+            double estimate = alphaMM / registerSum;
+
+            // small cardinality adjustment
+            if (zeroBuckets >= m * 0.07) { // (reference presto's HLL impl)
+                estimate = m * Math.log(m * 1.0 / zeroBuckets);
+            } else if (HyperLogLogPlusTable.isBiasCorrection(m, estimate)) {
+                estimate = HyperLogLogPlusTable.biasCorrection(p, estimate);
+            }
+
+            return Math.round(estimate);
+        }
+    }
+
+    // ============================================================================
+
+    public static interface Compressor {
+
+        byte[] compress(ByteBuffer buf, int offset, int length) throws IOException;
+
+        byte[] decompress(ByteBuffer buf, int offset, int length) throws IOException;
+    }
+
+    static final Compressor GZIP_COMPRESSOR = new Compressor() {
+        @Override
+        public byte[] compress(ByteBuffer buf, int offset, int length) throws IOException {
+            ByteArrayOutputStream bout = new ByteArrayOutputStream();
+            GZIPOutputStream gzout = new GZIPOutputStream(bout);
+            gzout.write(buf.array(), offset, length);
+            gzout.close();
+            return bout.toByteArray();
+        }
+
+        @Override
+        public byte[] decompress(ByteBuffer buf, int offset, int length) throws IOException {
+            ByteArrayInputStream bin = new ByteArrayInputStream(buf.array(), offset, length);
+            GZIPInputStream gzin = new GZIPInputStream(bin);
+            ByteArrayOutputStream bout = new ByteArrayOutputStream();
+            IOUtils.copy(gzin, bout);
+            gzin.close();
+            bout.close();
+            return bout.toByteArray();
+        }
+    };
+
+    static final Compressor LZF_COMPRESSOR = new Compressor() {
+        @Override
+        public byte[] compress(ByteBuffer buf, int offset, int length) throws IOException {
+            return LZFEncoder.encode(buf.array(), offset, length);
+        }
+
+        @Override
+        public byte[] decompress(ByteBuffer buf, int offset, int length) throws IOException {
+            return LZFDecoder.decode(buf.array(), offset, length);
+        }
+    };
+
+    public static final int COMPRESSION_THRESHOLD = Integer.MAX_VALUE; // bytes,
+                                                                       // disable
+                                                                       // due to
+                                                                       // slowness
+    public static final byte COMPRESSION_FLAG = (byte) 0x02;
+    public static final Compressor DEFAULT_COMPRESSOR = GZIP_COMPRESSOR; // LZF
+                                                                         // lib
+                                                                         // has
+                                                                         // a
+                                                                         // bug
+                                                                         // at
+                                                                         // the
+                                                                         // moment
+
+    public void writeRegisters(final ByteBuffer out) throws IOException {
+        int startPos = out.position();
+
+        final int indexLen = getRegisterIndexSize();
+        int size = size();
+
+        // decide output scheme -- map (3*size bytes) or array (2^p bytes)
+        byte scheme;
+        if ((indexLen + 1) * size < m)
+            scheme = 0; // map
+        else
+            scheme = 1; // array
+        out.put(scheme);
+
+        if (scheme == 0) { // map scheme
+            BytesUtil.writeVInt(size, out);
+            for (int i = 0; i < m; i++) {
+                if (registers[i] > 0) {
+                    BytesUtil.writeUnsigned(i, indexLen, out);
+                    out.put(registers[i]);
+                }
+            }
+        } else { // array scheme
+            for (int i = 0; i < m; i++) {
+                out.put(registers[i]);
+            }
+        }
+
+        // do compression if needed
+        int len = out.position() - startPos;
+        if (len < COMPRESSION_THRESHOLD)
+            return;
+
+        scheme |= COMPRESSION_FLAG;
+        byte[] compressed = DEFAULT_COMPRESSOR.compress(out, startPos + 1, len - 1);
+        out.position(startPos);
+        out.put(scheme);
+        BytesUtil.writeVInt(compressed.length, out);
+        out.put(compressed);
+    }
+
+    public void readRegisters(ByteBuffer in) throws IOException {
+        byte scheme = in.get();
+        if ((scheme & COMPRESSION_FLAG) > 0) {
+            scheme ^= COMPRESSION_FLAG;
+            int compressedLen = BytesUtil.readVInt(in);
+            int end = in.position() + compressedLen;
+            byte[] decompressed = DEFAULT_COMPRESSOR.decompress(in, in.position(), compressedLen);
+            in.position(end);
+            in = ByteBuffer.wrap(decompressed);
+        }
+
+        if (scheme == 0) { // map scheme
+            clear();
+            int size = BytesUtil.readVInt(in);
+            if (size > m)
+                throw new IllegalArgumentException("register size (" + size + ") cannot be larger than m (" + m + ")");
+            int indexLen = getRegisterIndexSize();
+            for (int i = 0; i < size; i++) {
+                int key = BytesUtil.readUnsigned(in, indexLen);
+                registers[key] = in.get();
+            }
+        } else { // array scheme
+            for (int i = 0; i < m; i++) {
+                registers[i] = in.get();
+            }
+        }
+    }
+
+    private int getRegisterIndexSize() {
+        return (p - 1) / 8 + 1; // 2 when p=16, 3 when p=17
+    }
+
+    @Override
+    public int hashCode() {
+        final int prime = 31;
+        int result = 1;
+        result = prime * result + ((hashFunc == null) ? 0 : hashFunc.hashCode());
+        result = prime * result + p;
+        result = prime * result + Arrays.hashCode(registers);
+        return result;
+    }
+
+    @Override
+    public boolean equals(Object obj) {
+        if (this == obj)
+            return true;
+        if (obj == null)
+            return false;
+        if (getClass() != obj.getClass())
+            return false;
+        HyperLogLogPlusCounter other = (HyperLogLogPlusCounter) obj;
+        if (hashFunc == null) {
+            if (other.hashFunc != null)
+                return false;
+        } else if (!hashFunc.equals(other.hashFunc))
+            return false;
+        if (p != other.p)
+            return false;
+        if (!Arrays.equals(registers, other.registers))
+            return false;
+        return true;
+    }
+
+    @Override
+    public int compareTo(HyperLogLogPlusCounter o) {
+        if (o == null)
+            return 1;
+
+        long e1 = this.getCountEstimate();
+        long e2 = o.getCountEstimate();
+
+        if (e1 == e2)
+            return 0;
+        else if (e1 > e2)
+            return 1;
+        else
+            return -1;
+    }
+
+    public static void main(String[] args) throws IOException {
+        dumpErrorRates();
+    }
+
+    static void dumpErrorRates() {
+        for (int p = 10; p <= 18; p++) {
+            double rate = new HyperLogLogPlusCounter(p).getErrorRate();
+            double er = Math.round(rate * 10000) / 100D;
+            double er2 = Math.round(rate * 2 * 10000) / 100D;
+            double er3 = Math.round(rate * 3 * 10000) / 100D;
+            long size = Math.round(Math.pow(2, p));
+            System.out.println("HLLC" + p + ",\t" + size + " bytes,\t68% err<" + er + "%" + ",\t95% err<" + er2 + "%" + ",\t99.7% err<" + er3 + "%");
+        }
+    }
+}


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

Posted by lu...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/atopcalcite/src/main/java/org/eigenbase/sql2rel/SqlToRelConverter.java
----------------------------------------------------------------------
diff --git a/atopcalcite/src/main/java/org/eigenbase/sql2rel/SqlToRelConverter.java b/atopcalcite/src/main/java/org/eigenbase/sql2rel/SqlToRelConverter.java
new file mode 100644
index 0000000..15077e0
--- /dev/null
+++ b/atopcalcite/src/main/java/org/eigenbase/sql2rel/SqlToRelConverter.java
@@ -0,0 +1,4782 @@
+/*
+ * OVERRIDE POINTS:
+ * - getInSubqueryThreshold(), was `20`, now `Integer.MAX_VALUE`
+ * - isTrimUnusedFields(), override to false
+ */
+
+/*
+ * 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.eigenbase.sql2rel;
+
+import java.lang.reflect.Type;
+import java.math.*;
+import java.util.*;
+import java.util.logging.*;
+
+import org.eigenbase.rel.*;
+import org.eigenbase.rel.metadata.*;
+import org.eigenbase.relopt.*;
+import org.eigenbase.reltype.*;
+import org.eigenbase.rex.*;
+import org.eigenbase.sql.*;
+import org.eigenbase.sql.fun.*;
+import org.eigenbase.sql.parser.*;
+import org.eigenbase.sql.type.*;
+import org.eigenbase.sql.util.*;
+import org.eigenbase.sql.validate.*;
+import org.eigenbase.trace.*;
+import org.eigenbase.util.*;
+import org.eigenbase.util.mapping.Mappings;
+import org.eigenbase.util14.*;
+
+import net.hydromatic.linq4j.Ord;
+import net.hydromatic.optiq.ModifiableTable;
+import net.hydromatic.optiq.TranslatableTable;
+import net.hydromatic.optiq.prepare.Prepare;
+import net.hydromatic.optiq.prepare.RelOptTableImpl;
+import net.hydromatic.optiq.util.BitSets;
+
+import com.google.common.base.Function;
+import com.google.common.collect.*;
+
+import static org.eigenbase.sql.SqlUtil.stripAs;
+import static org.eigenbase.util.Static.RESOURCE;
+
+/**
+ * Converts a SQL parse tree (consisting of {@link org.eigenbase.sql.SqlNode}
+ * objects) into a relational algebra expression (consisting of
+ * {@link org.eigenbase.rel.RelNode} objects).
+ *
+ * <p>The public entry points are: {@link #convertQuery},
+ * {@link #convertExpression(SqlNode)}.
+ */
+@SuppressWarnings({"unused", "rawtypes", "unchecked", "incomplete-switch", "deprecation"})
+public class SqlToRelConverter {
+  //~ Static fields/initializers ---------------------------------------------
+
+  protected static final Logger SQL2REL_LOGGER =
+      EigenbaseTrace.getSqlToRelTracer();
+
+  private static final Function<SubQuery, SqlNode> FN =
+      new Function<SubQuery, SqlNode>() {
+        public SqlNode apply(SubQuery input) {
+          return input.node;
+        }
+      };
+
+  //~ Instance fields --------------------------------------------------------
+
+  protected final SqlValidator validator;
+  protected final RexBuilder rexBuilder;
+  protected final Prepare.CatalogReader catalogReader;
+  protected final RelOptCluster cluster;
+  private DefaultValueFactory defaultValueFactory;
+  private SubqueryConverter subqueryConverter;
+  protected final List<RelNode> leaves = new ArrayList<RelNode>();
+  private final List<SqlDynamicParam> dynamicParamSqlNodes =
+      new ArrayList<SqlDynamicParam>();
+  private final SqlOperatorTable opTab;
+  private boolean shouldConvertTableAccess;
+  protected final RelDataTypeFactory typeFactory;
+  private final SqlNodeToRexConverter exprConverter;
+  private boolean decorrelationEnabled;
+  private boolean trimUnusedFields;
+  private boolean shouldCreateValuesRel;
+  private boolean isExplain;
+  private int nDynamicParamsInExplain;
+
+  /**
+   * Fields used in name resolution for correlated subqueries.
+   */
+  private final Map<String, DeferredLookup> mapCorrelToDeferred =
+      new HashMap<String, DeferredLookup>();
+  private int nextCorrel = 0;
+
+  private static final String CORREL_PREFIX = "$cor";
+
+  /**
+   * Stack of names of datasets requested by the <code>
+   * TABLE(SAMPLE(&lt;datasetName&gt;, &lt;query&gt;))</code> construct.
+   */
+  private final Stack<String> datasetStack = new Stack<String>();
+
+  /**
+   * Mapping of non-correlated subqueries that have been converted to their
+   * equivalent constants. Used to avoid re-evaluating the subquery if it's
+   * already been evaluated.
+   */
+  private final Map<SqlNode, RexNode> mapConvertedNonCorrSubqs =
+      new HashMap<SqlNode, RexNode>();
+
+  public final RelOptTable.ViewExpander viewExpander;
+
+  //~ Constructors -----------------------------------------------------------
+  /**
+   * Creates a converter.
+   *
+   * @param viewExpander    Preparing statement
+   * @param validator       Validator
+   * @param catalogReader   Schema
+   * @param planner         Planner
+   * @param rexBuilder      Rex builder
+   * @param convertletTable Expression converter
+   */
+  public SqlToRelConverter(
+      RelOptTable.ViewExpander viewExpander,
+      SqlValidator validator,
+      Prepare.CatalogReader catalogReader,
+      RelOptPlanner planner,
+      RexBuilder rexBuilder,
+      SqlRexConvertletTable convertletTable) {
+    this.viewExpander = viewExpander;
+    this.opTab =
+        (validator
+            == null) ? SqlStdOperatorTable.instance()
+            : validator.getOperatorTable();
+    this.validator = validator;
+    this.catalogReader = catalogReader;
+    this.defaultValueFactory = new NullDefaultValueFactory();
+    this.subqueryConverter = new NoOpSubqueryConverter();
+    this.rexBuilder = rexBuilder;
+    this.typeFactory = rexBuilder.getTypeFactory();
+    RelOptQuery query = new RelOptQuery(planner);
+    this.cluster = query.createCluster(typeFactory, rexBuilder);
+    this.shouldConvertTableAccess = true;
+    this.exprConverter =
+        new SqlNodeToRexConverterImpl(convertletTable);
+    decorrelationEnabled = true;
+    trimUnusedFields = false;
+    shouldCreateValuesRel = true;
+    isExplain = false;
+    nDynamicParamsInExplain = 0;
+  }
+
+  //~ Methods ----------------------------------------------------------------
+
+  /**
+   * @return the RelOptCluster in use.
+   */
+  public RelOptCluster getCluster() {
+    return cluster;
+  }
+
+  /**
+   * Returns the row-expression builder.
+   */
+  public RexBuilder getRexBuilder() {
+    return rexBuilder;
+  }
+
+  /**
+   * Returns the number of dynamic parameters encountered during translation;
+   * this must only be called after {@link #convertQuery}.
+   *
+   * @return number of dynamic parameters
+   */
+  public int getDynamicParamCount() {
+    return dynamicParamSqlNodes.size();
+  }
+
+  /**
+   * Returns the type inferred for a dynamic parameter.
+   *
+   * @param index 0-based index of dynamic parameter
+   * @return inferred type, never null
+   */
+  public RelDataType getDynamicParamType(int index) {
+    SqlNode sqlNode = dynamicParamSqlNodes.get(index);
+    if (sqlNode == null) {
+      throw Util.needToImplement("dynamic param type inference");
+    }
+    return validator.getValidatedNodeType(sqlNode);
+  }
+
+  /**
+   * Returns the current count of the number of dynamic parameters in an
+   * EXPLAIN PLAN statement.
+   *
+   * @param increment if true, increment the count
+   * @return the current count before the optional increment
+   */
+  public int getDynamicParamCountInExplain(boolean increment) {
+    int retVal = nDynamicParamsInExplain;
+    if (increment) {
+      ++nDynamicParamsInExplain;
+    }
+    return retVal;
+  }
+
+  /**
+   * @return mapping of non-correlated subqueries that have been converted to
+   * the constants that they evaluate to
+   */
+  public Map<SqlNode, RexNode> getMapConvertedNonCorrSubqs() {
+    return mapConvertedNonCorrSubqs;
+  }
+
+  /**
+   * Adds to the current map of non-correlated converted subqueries the
+   * elements from another map that contains non-correlated subqueries that
+   * have been converted by another SqlToRelConverter.
+   *
+   * @param alreadyConvertedNonCorrSubqs the other map
+   */
+  public void addConvertedNonCorrSubqs(
+      Map<SqlNode, RexNode> alreadyConvertedNonCorrSubqs) {
+    mapConvertedNonCorrSubqs.putAll(alreadyConvertedNonCorrSubqs);
+  }
+
+  /**
+   * Set a new DefaultValueFactory. To have any effect, this must be called
+   * before any convert method.
+   *
+   * @param factory new DefaultValueFactory
+   */
+  public void setDefaultValueFactory(DefaultValueFactory factory) {
+    defaultValueFactory = factory;
+  }
+
+  /**
+   * Sets a new SubqueryConverter. To have any effect, this must be called
+   * before any convert method.
+   *
+   * @param converter new SubqueryConverter
+   */
+  public void setSubqueryConverter(SubqueryConverter converter) {
+    subqueryConverter = converter;
+  }
+
+  /**
+   * Indicates that the current statement is part of an EXPLAIN PLAN statement
+   *
+   * @param nDynamicParams number of dynamic parameters in the statement
+   */
+  public void setIsExplain(int nDynamicParams) {
+    isExplain = true;
+    nDynamicParamsInExplain = nDynamicParams;
+  }
+
+  /**
+   * Controls whether table access references are converted to physical rels
+   * immediately. The optimizer doesn't like leaf rels to have
+   * {@link Convention#NONE}. However, if we are doing further conversion
+   * passes (e.g. {@link RelStructuredTypeFlattener}), then we may need to
+   * defer conversion. To have any effect, this must be called before any
+   * convert method.
+   *
+   * @param enabled true for immediate conversion (the default); false to
+   *                generate logical TableAccessRel instances
+   */
+  public void enableTableAccessConversion(boolean enabled) {
+    shouldConvertTableAccess = enabled;
+  }
+
+  /**
+   * Controls whether instances of {@link ValuesRel} are generated. These may
+   * not be supported by all physical implementations. To have any effect,
+   * this must be called before any convert method.
+   *
+   * @param enabled true to allow ValuesRel to be generated (the default);
+   *                false to force substitution of ProjectRel+OneRowRel instead
+   */
+  public void enableValuesRelCreation(boolean enabled) {
+    shouldCreateValuesRel = enabled;
+  }
+
+  private void checkConvertedType(SqlNode query, RelNode result) {
+    if (!query.isA(SqlKind.DML)) {
+      // Verify that conversion from SQL to relational algebra did
+      // not perturb any type information.  (We can't do this if the
+      // SQL statement is something like an INSERT which has no
+      // validator type information associated with its result,
+      // hence the namespace check above.)
+      RelDataType convertedRowType = result.getRowType();
+      if (!checkConvertedRowType(query, convertedRowType)) {
+        RelDataType validatedRowType =
+            validator.getValidatedNodeType(query);
+        validatedRowType = uniquifyFields(validatedRowType);
+        throw Util.newInternal(
+            "Conversion to relational algebra failed to preserve "
+            + "datatypes:\n"
+            + "validated type:\n"
+            + validatedRowType.getFullTypeString()
+            + "\nconverted type:\n"
+            + convertedRowType.getFullTypeString()
+            + "\nrel:\n"
+            + RelOptUtil.toString(result));
+      }
+    }
+  }
+
+  public RelNode flattenTypes(
+      RelNode rootRel,
+      boolean restructure) {
+    RelStructuredTypeFlattener typeFlattener =
+        new RelStructuredTypeFlattener(rexBuilder, createToRelContext());
+    return typeFlattener.rewrite(rootRel, restructure);
+  }
+
+  /**
+   * If subquery is correlated and decorrelation is enabled, performs
+   * decorrelation.
+   *
+   * @param query   Query
+   * @param rootRel Root relational expression
+   * @return New root relational expression after decorrelation
+   */
+  public RelNode decorrelate(SqlNode query, RelNode rootRel) {
+    if (!enableDecorrelation()) {
+      return rootRel;
+    }
+    final RelNode result = decorrelateQuery(rootRel);
+    if (result != rootRel) {
+      checkConvertedType(query, result);
+    }
+    return result;
+  }
+
+  /**
+   * Walks over a tree of relational expressions, replacing each
+   * {@link RelNode} with a 'slimmed down' relational expression that projects
+   * only the fields required by its consumer.
+   *
+   * <p>This may make things easier for the optimizer, by removing crud that
+   * would expand the search space, but is difficult for the optimizer itself
+   * to do it, because optimizer rules must preserve the number and type of
+   * fields. Hence, this transform that operates on the entire tree, similar
+   * to the {@link RelStructuredTypeFlattener type-flattening transform}.
+   *
+   * <p>Currently this functionality is disabled in farrago/luciddb; the
+   * default implementation of this method does nothing.
+   *
+   * @param rootRel Relational expression that is at the root of the tree
+   * @return Trimmed relational expression
+   */
+  public RelNode trimUnusedFields(RelNode rootRel) {
+    // Trim fields that are not used by their consumer.
+    if (isTrimUnusedFields()) {
+      final RelFieldTrimmer trimmer = newFieldTrimmer();
+      rootRel = trimmer.trim(rootRel);
+      boolean dumpPlan = SQL2REL_LOGGER.isLoggable(Level.FINE);
+      if (dumpPlan) {
+        SQL2REL_LOGGER.fine(
+            RelOptUtil.dumpPlan(
+                "Plan after trimming unused fields",
+                rootRel,
+                false,
+                SqlExplainLevel.EXPPLAN_ATTRIBUTES));
+      }
+    }
+    return rootRel;
+  }
+
+  /**
+   * Creates a RelFieldTrimmer.
+   *
+   * @return Field trimmer
+   */
+  protected RelFieldTrimmer newFieldTrimmer() {
+    return new RelFieldTrimmer(validator);
+  }
+
+  /**
+   * Converts an unvalidated query's parse tree into a relational expression.
+   *
+   * @param query           Query to convert
+   * @param needsValidation Whether to validate the query before converting;
+   *                        <code>false</code> if the query has already been
+   *                        validated.
+   * @param top             Whether the query is top-level, say if its result
+   *                        will become a JDBC result set; <code>false</code> if
+   *                        the query will be part of a view.
+   */
+  public RelNode convertQuery(
+      SqlNode query,
+      final boolean needsValidation,
+      final boolean top) {
+    if (needsValidation) {
+      query = validator.validate(query);
+    }
+
+    RelNode result = convertQueryRecursive(query, top, null);
+    checkConvertedType(query, result);
+
+    boolean dumpPlan = SQL2REL_LOGGER.isLoggable(Level.FINE);
+    if (dumpPlan) {
+      SQL2REL_LOGGER.fine(
+          RelOptUtil.dumpPlan(
+              "Plan after converting SqlNode to RelNode",
+              result,
+              false,
+              SqlExplainLevel.EXPPLAN_ATTRIBUTES));
+    }
+
+    return result;
+  }
+
+  protected boolean checkConvertedRowType(
+      SqlNode query,
+      RelDataType convertedRowType) {
+    RelDataType validatedRowType = validator.getValidatedNodeType(query);
+    validatedRowType = uniquifyFields(validatedRowType);
+
+    return RelOptUtil.equal(
+        "validated row type",
+        validatedRowType,
+        "converted row type",
+        convertedRowType,
+        false);
+  }
+
+  protected RelDataType uniquifyFields(RelDataType rowType) {
+    return validator.getTypeFactory().createStructType(
+        RelOptUtil.getFieldTypeList(rowType),
+        SqlValidatorUtil.uniquify(rowType.getFieldNames()));
+  }
+
+  /**
+   * Converts a SELECT statement's parse tree into a relational expression.
+   */
+  public RelNode convertSelect(SqlSelect select) {
+    final SqlValidatorScope selectScope = validator.getWhereScope(select);
+    final Blackboard bb = createBlackboard(selectScope, null);
+    convertSelectImpl(bb, select);
+    return bb.root;
+  }
+
+  /**
+   * Factory method for creating translation workspace.
+   */
+  protected Blackboard createBlackboard(
+      SqlValidatorScope scope,
+      Map<String, RexNode> nameToNodeMap) {
+    return new Blackboard(scope, nameToNodeMap);
+  }
+
+  /**
+   * Implementation of {@link #convertSelect(SqlSelect)}; derived class may
+   * override.
+   */
+  protected void convertSelectImpl(
+      final Blackboard bb,
+      SqlSelect select) {
+    convertFrom(
+        bb,
+        select.getFrom());
+    convertWhere(
+        bb,
+        select.getWhere());
+
+    List<SqlNode> orderExprList = new ArrayList<SqlNode>();
+    List<RelFieldCollation> collationList =
+        new ArrayList<RelFieldCollation>();
+    gatherOrderExprs(
+        bb,
+        select,
+        select.getOrderList(),
+        orderExprList,
+        collationList);
+    final RelCollation collation =
+        cluster.traitSetOf().canonize(RelCollationImpl.of(collationList));
+
+    if (validator.isAggregate(select)) {
+      convertAgg(
+          bb,
+          select,
+          orderExprList);
+    } else {
+      convertSelectList(
+          bb,
+          select,
+          orderExprList);
+    }
+
+    if (select.isDistinct()) {
+      distinctify(bb, true);
+    }
+    convertOrder(
+        select, bb, collation, orderExprList, select.getOffset(),
+        select.getFetch());
+    bb.setRoot(bb.root, true);
+  }
+
+  /**
+   * Having translated 'SELECT ... FROM ... [GROUP BY ...] [HAVING ...]', adds
+   * a relational expression to make the results unique.
+   *
+   * <p>If the SELECT clause contains duplicate expressions, adds {@link
+   * ProjectRel}s so that we are grouping on the minimal set of keys. The
+   * performance gain isn't huge, but it is difficult to detect these
+   * duplicate expressions later.
+   *
+   * @param bb               Blackboard
+   * @param checkForDupExprs Check for duplicate expressions
+   */
+  private void distinctify(
+      Blackboard bb,
+      boolean checkForDupExprs) {
+    // Look for duplicate expressions in the project.
+    // Say we have 'select x, y, x, z'.
+    // Then dups will be {[2, 0]}
+    // and oldToNew will be {[0, 0], [1, 1], [2, 0], [3, 2]}
+    RelNode rel = bb.root;
+    if (checkForDupExprs && (rel instanceof ProjectRel)) {
+      ProjectRel project = (ProjectRel) rel;
+      final List<RexNode> projectExprs = project.getProjects();
+      List<Integer> origins = new ArrayList<Integer>();
+      int dupCount = 0;
+      for (int i = 0; i < projectExprs.size(); i++) {
+        int x = findExpr(projectExprs.get(i), projectExprs, i);
+        if (x >= 0) {
+          origins.add(x);
+          ++dupCount;
+        } else {
+          origins.add(i);
+        }
+      }
+      if (dupCount == 0) {
+        distinctify(bb, false);
+        return;
+      }
+
+      final Map<Integer, Integer> squished = Maps.newHashMap();
+      final List<RelDataTypeField> fields = rel.getRowType().getFieldList();
+      final List<Pair<RexNode, String>> newProjects = Lists.newArrayList();
+      for (int i = 0; i < fields.size(); i++) {
+        if (origins.get(i) == i) {
+          squished.put(i, newProjects.size());
+          newProjects.add(RexInputRef.of2(i, fields));
+        }
+      }
+      rel =
+          new ProjectRel(
+              cluster,
+              rel,
+              Pair.left(newProjects),
+              Pair.right(newProjects),
+              ProjectRel.Flags.BOXED);
+
+      bb.root = rel;
+      distinctify(bb, false);
+      rel = bb.root;
+
+      // Create the expressions to reverse the mapping.
+      // Project($0, $1, $0, $2).
+      final List<Pair<RexNode, String>> undoProjects = Lists.newArrayList();
+      for (int i = 0; i < fields.size(); i++) {
+        final int origin = origins.get(i);
+        RelDataTypeField field = fields.get(i);
+        undoProjects.add(
+            Pair.of(
+                (RexNode) new RexInputRef(
+                    squished.get(origin), field.getType()),
+                field.getName()));
+      }
+
+      rel =
+          new ProjectRel(
+              cluster,
+              rel,
+              Pair.left(undoProjects),
+              Pair.right(undoProjects),
+              ProjectRel.Flags.BOXED);
+
+      bb.setRoot(
+          rel,
+          false);
+
+      return;
+    }
+
+    // Usual case: all of the expressions in the SELECT clause are
+    // different.
+    rel =
+        createAggregate(
+            bb,
+            BitSets.range(rel.getRowType().getFieldCount()),
+            ImmutableList.<AggregateCall>of());
+
+    bb.setRoot(
+        rel,
+        false);
+  }
+
+  private int findExpr(RexNode seek, List<RexNode> exprs, int count) {
+    for (int i = 0; i < count; i++) {
+      RexNode expr = exprs.get(i);
+      if (expr.toString().equals(seek.toString())) {
+        return i;
+      }
+    }
+    return -1;
+  }
+
+  /**
+   * Converts a query's ORDER BY clause, if any.
+   *
+   * @param select        Query
+   * @param bb            Blackboard
+   * @param collation     Collation list
+   * @param orderExprList Method populates this list with orderBy expressions
+   *                      not present in selectList
+   * @param offset        Expression for number of rows to discard before
+   *                      returning first row
+   * @param fetch         Expression for number of rows to fetch
+   */
+  protected void convertOrder(
+      SqlSelect select,
+      Blackboard bb,
+      RelCollation collation,
+      List<SqlNode> orderExprList,
+      SqlNode offset,
+      SqlNode fetch) {
+    if (select.getOrderList() == null) {
+      assert collation.getFieldCollations().isEmpty();
+      if (offset == null && fetch == null) {
+        return;
+      }
+    }
+
+    // Create a sorter using the previously constructed collations.
+    bb.setRoot(
+        new SortRel(
+            cluster,
+            cluster.traitSetOf(Convention.NONE, collation),
+            bb.root,
+            collation,
+            offset == null ? null : convertExpression(offset),
+            fetch == null ? null : convertExpression(fetch)),
+        false);
+
+    // If extra expressions were added to the project list for sorting,
+    // add another project to remove them.
+    if (orderExprList.size() > 0) {
+      List<RexNode> exprs = new ArrayList<RexNode>();
+      final RelDataType rowType = bb.root.getRowType();
+      final int fieldCount =
+          rowType.getFieldCount() - orderExprList.size();
+      for (int i = 0; i < fieldCount; i++) {
+        exprs.add(rexBuilder.makeInputRef(bb.root, i));
+      }
+      bb.setRoot(
+          new ProjectRel(
+              cluster,
+              cluster.traitSetOf(RelCollationImpl.PRESERVE),
+              bb.root,
+              exprs,
+              cluster.getTypeFactory().createStructType(
+                  rowType.getFieldList().subList(0, fieldCount)),
+              ProjectRelBase.Flags.BOXED),
+          false);
+    }
+  }
+
+  /**
+   * Returns whether a given node contains a {@link SqlInOperator}.
+   *
+   * @param node a RexNode tree
+   */
+  private static boolean containsInOperator(
+      SqlNode node) {
+    try {
+      SqlVisitor<Void> visitor =
+          new SqlBasicVisitor<Void>() {
+            public Void visit(SqlCall call) {
+              if (call.getOperator() instanceof SqlInOperator) {
+                throw new Util.FoundOne(call);
+              }
+              return super.visit(call);
+            }
+          };
+      node.accept(visitor);
+      return false;
+    } catch (Util.FoundOne e) {
+      Util.swallow(e, null);
+      return true;
+    }
+  }
+
+  /**
+   * Push down all the NOT logical operators into any IN/NOT IN operators.
+   *
+   * @param sqlNode the root node from which to look for NOT operators
+   * @return the transformed SqlNode representation with NOT pushed down.
+   */
+  private static SqlNode pushDownNotForIn(SqlNode sqlNode) {
+    if ((sqlNode instanceof SqlCall) && containsInOperator(sqlNode)) {
+      SqlCall sqlCall = (SqlCall) sqlNode;
+      if ((sqlCall.getOperator() == SqlStdOperatorTable.AND)
+          || (sqlCall.getOperator() == SqlStdOperatorTable.OR)) {
+        SqlNode[] sqlOperands = ((SqlBasicCall) sqlCall).operands;
+        for (int i = 0; i < sqlOperands.length; i++) {
+          sqlOperands[i] = pushDownNotForIn(sqlOperands[i]);
+        }
+        return sqlNode;
+      } else if (sqlCall.getOperator() == SqlStdOperatorTable.NOT) {
+        SqlNode childNode = sqlCall.operand(0);
+        assert childNode instanceof SqlCall;
+        SqlBasicCall childSqlCall = (SqlBasicCall) childNode;
+        if (childSqlCall.getOperator() == SqlStdOperatorTable.AND) {
+          SqlNode[] andOperands = childSqlCall.getOperands();
+          SqlNode[] orOperands = new SqlNode[andOperands.length];
+          for (int i = 0; i < orOperands.length; i++) {
+            orOperands[i] =
+                SqlStdOperatorTable.NOT.createCall(
+                    SqlParserPos.ZERO,
+                    andOperands[i]);
+          }
+          for (int i = 0; i < orOperands.length; i++) {
+            orOperands[i] = pushDownNotForIn(orOperands[i]);
+          }
+          return SqlStdOperatorTable.OR.createCall(SqlParserPos.ZERO,
+              orOperands[0], orOperands[1]);
+        } else if (childSqlCall.getOperator() == SqlStdOperatorTable.OR) {
+          SqlNode[] orOperands = childSqlCall.getOperands();
+          SqlNode[] andOperands = new SqlNode[orOperands.length];
+          for (int i = 0; i < andOperands.length; i++) {
+            andOperands[i] =
+                SqlStdOperatorTable.NOT.createCall(
+                    SqlParserPos.ZERO,
+                    orOperands[i]);
+          }
+          for (int i = 0; i < andOperands.length; i++) {
+            andOperands[i] = pushDownNotForIn(andOperands[i]);
+          }
+          return SqlStdOperatorTable.AND.createCall(SqlParserPos.ZERO,
+              andOperands[0], andOperands[1]);
+        } else if (childSqlCall.getOperator() == SqlStdOperatorTable.NOT) {
+          SqlNode[] notOperands = childSqlCall.getOperands();
+          assert notOperands.length == 1;
+          return pushDownNotForIn(notOperands[0]);
+        } else if (childSqlCall.getOperator() instanceof SqlInOperator) {
+          SqlNode[] inOperands = childSqlCall.getOperands();
+          SqlInOperator inOp =
+              (SqlInOperator) childSqlCall.getOperator();
+          if (inOp.isNotIn()) {
+            return SqlStdOperatorTable.IN.createCall(
+                SqlParserPos.ZERO,
+                inOperands[0],
+                inOperands[1]);
+          } else {
+            return SqlStdOperatorTable.NOT_IN.createCall(
+                SqlParserPos.ZERO,
+                inOperands[0],
+                inOperands[1]);
+          }
+        } else {
+          // childSqlCall is "leaf" node in a logical expression tree
+          // (only considering AND, OR, NOT)
+          return sqlNode;
+        }
+      } else {
+        // sqlNode is "leaf" node in a logical expression tree
+        // (only considering AND, OR, NOT)
+        return sqlNode;
+      }
+    } else {
+      // tree rooted at sqlNode does not contain inOperator
+      return sqlNode;
+    }
+  }
+
+  /**
+   * Converts a WHERE clause.
+   *
+   * @param bb    Blackboard
+   * @param where WHERE clause, may be null
+   */
+  private void convertWhere(
+      final Blackboard bb,
+      final SqlNode where) {
+    if (where == null) {
+      return;
+    }
+    SqlNode newWhere = pushDownNotForIn(where);
+    replaceSubqueries(bb, newWhere, RelOptUtil.Logic.UNKNOWN_AS_FALSE);
+    final RexNode convertedWhere = bb.convertExpression(newWhere);
+
+    // only allocate filter if the condition is not TRUE
+    if (!convertedWhere.isAlwaysTrue()) {
+      bb.setRoot(
+          RelOptUtil.createFilter(bb.root, convertedWhere),
+          false);
+    }
+  }
+
+  private void replaceSubqueries(
+      final Blackboard bb,
+      final SqlNode expr,
+      RelOptUtil.Logic logic) {
+    findSubqueries(bb, expr, logic, false);
+    for (SubQuery node : bb.subqueryList) {
+      substituteSubquery(bb, node);
+    }
+  }
+
+  private void substituteSubquery(Blackboard bb, SubQuery subQuery) {
+    final RexNode expr = subQuery.expr;
+    if (expr != null) {
+      // Already done.
+      return;
+    }
+
+    final SqlBasicCall call;
+    final RelNode rel;
+    final SqlNode query;
+    final Pair<RelNode, Boolean> converted;
+    switch (subQuery.node.getKind()) {
+    case CURSOR:
+      convertCursor(bb, subQuery);
+      return;
+
+    case MULTISET_QUERY_CONSTRUCTOR:
+    case MULTISET_VALUE_CONSTRUCTOR:
+      rel = convertMultisets(ImmutableList.of(subQuery.node), bb);
+      subQuery.expr = bb.register(rel, JoinRelType.INNER);
+      return;
+
+    case IN:
+      call = (SqlBasicCall) subQuery.node;
+      final SqlNode[] operands = call.getOperands();
+
+      SqlNode leftKeyNode = operands[0];
+      query = operands[1];
+
+      final List<RexNode> leftKeys;
+      switch (leftKeyNode.getKind()) {
+      case ROW:
+        leftKeys = Lists.newArrayList();
+        for (SqlNode sqlExpr : ((SqlBasicCall) leftKeyNode).getOperandList()) {
+          leftKeys.add(bb.convertExpression(sqlExpr));
+        }
+        break;
+      default:
+        leftKeys = ImmutableList.of(bb.convertExpression(leftKeyNode));
+      }
+
+      final boolean isNotIn = ((SqlInOperator) call.getOperator()).isNotIn();
+      if (query instanceof SqlNodeList) {
+        SqlNodeList valueList = (SqlNodeList) query;
+        if (!containsNullLiteral(valueList)
+            && valueList.size() < getInSubqueryThreshold()) {
+          // We're under the threshold, so convert to OR.
+          subQuery.expr =
+              convertInToOr(
+                  bb,
+                  leftKeys,
+                  valueList,
+                  isNotIn);
+          return;
+        }
+
+        // Otherwise, let convertExists translate
+        // values list into an inline table for the
+        // reference to Q below.
+      }
+
+      // Project out the search columns from the left side
+
+      //  Q1:
+      // "select from emp where emp.deptno in (select col1 from T)"
+      //
+      // is converted to
+      //
+      // "select from
+      //   emp inner join (select distinct col1 from T)) q
+      //   on emp.deptno = q.col1
+      //
+      // Q2:
+      // "select from emp where emp.deptno not in (Q)"
+      //
+      // is converted to
+      //
+      // "select from
+      //   emp left outer join (select distinct col1, TRUE from T) q
+      //   on emp.deptno = q.col1
+      //   where emp.deptno <> null
+      //         and q.indicator <> TRUE"
+      //
+      final boolean outerJoin = bb.subqueryNeedsOuterJoin
+          || isNotIn
+          || subQuery.logic == RelOptUtil.Logic.TRUE_FALSE_UNKNOWN;
+      converted =
+          convertExists(query, RelOptUtil.SubqueryType.IN, subQuery.logic,
+              outerJoin);
+      if (converted.right) {
+        // Generate
+        //    emp CROSS JOIN (SELECT COUNT(*) AS c,
+        //                       COUNT(deptno) AS ck FROM dept)
+        final RelDataType longType =
+            typeFactory.createSqlType(SqlTypeName.BIGINT);
+        final RelNode seek = converted.left.getInput(0); // fragile
+        final int keyCount = leftKeys.size();
+        final List<Integer> args = ImmutableIntList.range(0, keyCount);
+        AggregateRel aggregate =
+            new AggregateRel(cluster, seek, BitSets.of(),
+                ImmutableList.of(
+                    new AggregateCall(SqlStdOperatorTable.COUNT, false,
+                        ImmutableList.<Integer>of(), longType, null),
+                    new AggregateCall(SqlStdOperatorTable.COUNT, false,
+                        args, longType, null)));
+        JoinRel join =
+            new JoinRel(cluster, bb.root, aggregate,
+                rexBuilder.makeLiteral(true), JoinRelType.INNER,
+                ImmutableSet.<String>of());
+        bb.setRoot(join, false);
+      }
+      RexNode rex =
+          bb.register(converted.left,
+              outerJoin ? JoinRelType.LEFT : JoinRelType.INNER, leftKeys);
+
+      subQuery.expr = translateIn(subQuery, bb.root, rex);
+      if (isNotIn) {
+        subQuery.expr =
+            rexBuilder.makeCall(SqlStdOperatorTable.NOT, subQuery.expr);
+      }
+      return;
+
+    case EXISTS:
+      // "select from emp where exists (select a from T)"
+      //
+      // is converted to the following if the subquery is correlated:
+      //
+      // "select from emp left outer join (select AGG_TRUE() as indicator
+      // from T group by corr_var) q where q.indicator is true"
+      //
+      // If there is no correlation, the expression is replaced with a
+      // boolean indicating whether the subquery returned 0 or >= 1 row.
+      call = (SqlBasicCall) subQuery.node;
+      query = call.getOperands()[0];
+      converted = convertExists(query, RelOptUtil.SubqueryType.EXISTS,
+          subQuery.logic, true);
+      assert !converted.right;
+      if (convertNonCorrelatedSubQuery(subQuery, bb, converted.left, true)) {
+        return;
+      }
+      subQuery.expr = bb.register(converted.left, JoinRelType.LEFT);
+      return;
+
+    case SCALAR_QUERY:
+      // Convert the subquery.  If it's non-correlated, convert it
+      // to a constant expression.
+      call = (SqlBasicCall) subQuery.node;
+      query = call.getOperands()[0];
+      converted = convertExists(query, RelOptUtil.SubqueryType.SCALAR,
+          subQuery.logic, true);
+      assert !converted.right;
+      if (convertNonCorrelatedSubQuery(subQuery, bb, converted.left, false)) {
+        return;
+      }
+      rel = convertToSingleValueSubq(query, converted.left);
+      subQuery.expr = bb.register(rel, JoinRelType.LEFT);
+      return;
+
+    case SELECT:
+      // This is used when converting multiset queries:
+      //
+      // select * from unnest(select multiset[deptno] from emps);
+      //
+      converted = convertExists(subQuery.node, RelOptUtil.SubqueryType.SCALAR,
+          subQuery.logic, true);
+      assert !converted.right;
+      subQuery.expr = bb.register(converted.left, JoinRelType.LEFT);
+      return;
+
+    default:
+      throw Util.newInternal("unexpected kind of subquery :" + subQuery.node);
+    }
+  }
+
+  private RexNode translateIn(SubQuery subQuery, RelNode root,
+      final RexNode rex) {
+    switch (subQuery.logic) {
+    case TRUE:
+      return rexBuilder.makeLiteral(true);
+
+    case UNKNOWN_AS_FALSE:
+      assert rex instanceof RexRangeRef;
+      final int fieldCount = rex.getType().getFieldCount();
+      RexNode rexNode = rexBuilder.makeFieldAccess(rex, fieldCount - 1);
+      rexNode = rexBuilder.makeCall(SqlStdOperatorTable.IS_TRUE, rexNode);
+
+      // Then append the IS NOT NULL(leftKeysForIn).
+      //
+      // RexRangeRef contains the following fields:
+      //   leftKeysForIn,
+      //   rightKeysForIn (the original subquery select list),
+      //   nullIndicator
+      //
+      // The first two lists contain the same number of fields.
+      final int k = (fieldCount - 1) / 2;
+      for (int i = 0; i < k; i++) {
+        rexNode =
+            rexBuilder.makeCall(
+                SqlStdOperatorTable.AND,
+                rexNode,
+                rexBuilder.makeCall(
+                    SqlStdOperatorTable.IS_NOT_NULL,
+                    rexBuilder.makeFieldAccess(rex, i)));
+      }
+      return rexNode;
+
+    case TRUE_FALSE_UNKNOWN:
+    case UNKNOWN_AS_TRUE:
+      // select e.deptno,
+      //   case
+      //   when ct.c = 0 then false
+      //   when dt.i is not null then true
+      //   when e.deptno is null then null
+      //   when ct.ck < ct.c then null
+      //   else false
+      //   end
+      // from e
+      // cross join (select count(*) as c, count(deptno) as ck from v) as ct
+      // left join (select distinct deptno, true as i from v) as dt
+      //   on e.deptno = dt.deptno
+      final JoinRelBase join = (JoinRelBase) root;
+      final ProjectRelBase left = (ProjectRelBase) join.getLeft();
+      final RelNode leftLeft = ((JoinRelBase) left.getInput(0)).getLeft();
+      final int leftLeftCount = leftLeft.getRowType().getFieldCount();
+      final RelDataType nullableBooleanType =
+          typeFactory.createTypeWithNullability(
+              typeFactory.createSqlType(SqlTypeName.BOOLEAN), true);
+      final RelDataType longType =
+          typeFactory.createSqlType(SqlTypeName.BIGINT);
+      final RexNode cRef = rexBuilder.makeInputRef(root, leftLeftCount);
+      final RexNode ckRef = rexBuilder.makeInputRef(root, leftLeftCount + 1);
+      final RexNode iRef =
+          rexBuilder.makeInputRef(root, root.getRowType().getFieldCount() - 1);
+
+      final RexLiteral zero =
+          rexBuilder.makeExactLiteral(BigDecimal.ZERO, longType);
+      final RexLiteral trueLiteral = rexBuilder.makeLiteral(true);
+      final RexLiteral falseLiteral = rexBuilder.makeLiteral(false);
+      final RexNode unknownLiteral =
+          rexBuilder.makeNullLiteral(SqlTypeName.BOOLEAN);
+
+      final ImmutableList.Builder<RexNode> args = ImmutableList.builder();
+      args.add(rexBuilder.makeCall(SqlStdOperatorTable.EQUALS, cRef, zero),
+          falseLiteral,
+          rexBuilder.makeCall(SqlStdOperatorTable.IS_NOT_NULL, iRef),
+          trueLiteral);
+      final JoinInfo joinInfo = join.analyzeCondition();
+      for (int leftKey : joinInfo.leftKeys) {
+        final RexNode kRef = rexBuilder.makeInputRef(root, leftKey);
+        args.add(rexBuilder.makeCall(SqlStdOperatorTable.IS_NULL, kRef),
+            unknownLiteral);
+      }
+      args.add(rexBuilder.makeCall(SqlStdOperatorTable.LESS_THAN, ckRef, cRef),
+          unknownLiteral,
+          falseLiteral);
+
+      return rexBuilder.makeCall(
+          nullableBooleanType,
+          SqlStdOperatorTable.CASE,
+          args.build());
+
+    default:
+      throw new AssertionError(subQuery.logic);
+    }
+  }
+
+  private static boolean containsNullLiteral(SqlNodeList valueList) {
+    for (SqlNode node : valueList.getList()) {
+      if (node instanceof SqlLiteral) {
+        SqlLiteral lit = (SqlLiteral) node;
+        if (lit.getValue() == null) {
+          return true;
+        }
+      }
+    }
+    return false;
+  }
+
+  /**
+   * Determines if a subquery is non-correlated and if so, converts it to a
+   * constant.
+   *
+   * @param subQuery  the call that references the subquery
+   * @param bb        blackboard used to convert the subquery
+   * @param converted RelNode tree corresponding to the subquery
+   * @param isExists  true if the subquery is part of an EXISTS expression
+   * @return if the subquery can be converted to a constant
+   */
+  private boolean convertNonCorrelatedSubQuery(
+      SubQuery subQuery,
+      Blackboard bb,
+      RelNode converted,
+      boolean isExists) {
+    SqlCall call = (SqlBasicCall) subQuery.node;
+    if (subqueryConverter.canConvertSubquery()
+        && isSubqNonCorrelated(converted, bb)) {
+      // First check if the subquery has already been converted
+      // because it's a nested subquery.  If so, don't re-evaluate
+      // it again.
+      RexNode constExpr = mapConvertedNonCorrSubqs.get(call);
+      if (constExpr == null) {
+        constExpr =
+            subqueryConverter.convertSubquery(
+                call,
+                this,
+                isExists,
+                isExplain);
+      }
+      if (constExpr != null) {
+        subQuery.expr = constExpr;
+        mapConvertedNonCorrSubqs.put(call, constExpr);
+        return true;
+      }
+    }
+    return false;
+  }
+
+  /**
+   * Converts the RelNode tree for a select statement to a select that
+   * produces a single value.
+   *
+   * @param query the query
+   * @param plan   the original RelNode tree corresponding to the statement
+   * @return the converted RelNode tree
+   */
+  public RelNode convertToSingleValueSubq(
+      SqlNode query,
+      RelNode plan) {
+    // Check whether query is guaranteed to produce a single value.
+    if (query instanceof SqlSelect) {
+      SqlSelect select = (SqlSelect) query;
+      SqlNodeList selectList = select.getSelectList();
+      SqlNodeList groupList = select.getGroup();
+
+      if ((selectList.size() == 1)
+          && ((groupList == null) || (groupList.size() == 0))) {
+        SqlNode selectExpr = selectList.get(0);
+        if (selectExpr instanceof SqlCall) {
+          SqlCall selectExprCall = (SqlCall) selectExpr;
+          if (selectExprCall.getOperator()
+              instanceof SqlAggFunction) {
+            return plan;
+          }
+        }
+      }
+    }
+
+    // If not, project SingleValueAgg
+    return RelOptUtil.createSingleValueAggRel(
+        cluster,
+        plan);
+  }
+
+  /**
+   * Converts "x IN (1, 2, ...)" to "x=1 OR x=2 OR ...".
+   *
+   * @param leftKeys   LHS
+   * @param valuesList RHS
+   * @param isNotIn    is this a NOT IN operator
+   * @return converted expression
+   */
+  private RexNode convertInToOr(
+      final Blackboard bb,
+      final List<RexNode> leftKeys,
+      SqlNodeList valuesList,
+      boolean isNotIn) {
+    List<RexNode> comparisons = new ArrayList<RexNode>();
+    for (SqlNode rightVals : valuesList) {
+      RexNode rexComparison;
+      if (leftKeys.size() == 1) {
+        rexComparison =
+            rexBuilder.makeCall(
+                SqlStdOperatorTable.EQUALS,
+                leftKeys.get(0),
+                bb.convertExpression(rightVals));
+      } else {
+        assert rightVals instanceof SqlCall;
+        final SqlBasicCall call = (SqlBasicCall) rightVals;
+        assert (call.getOperator() instanceof SqlRowOperator)
+            && call.getOperands().length == leftKeys.size();
+        rexComparison =
+            RexUtil.composeConjunction(
+                rexBuilder,
+                Iterables.transform(
+                    Pair.zip(leftKeys, call.getOperandList()),
+                    new Function<Pair<RexNode, SqlNode>, RexNode>() {
+                      public RexNode apply(Pair<RexNode, SqlNode> pair) {
+                        return rexBuilder.makeCall(SqlStdOperatorTable.EQUALS,
+                            pair.left, bb.convertExpression(pair.right));
+                      }
+                    }),
+                false);
+      }
+      comparisons.add(rexComparison);
+    }
+
+    RexNode result =
+        RexUtil.composeDisjunction(rexBuilder, comparisons, true);
+    assert result != null;
+
+    if (isNotIn) {
+      result =
+          rexBuilder.makeCall(
+              SqlStdOperatorTable.NOT,
+              result);
+    }
+
+    return result;
+  }
+
+  /**
+   * Gets the list size threshold under which {@link #convertInToOr} is used.
+   * Lists of this size or greater will instead be converted to use a join
+   * against an inline table ({@link ValuesRel}) rather than a predicate. A
+   * threshold of 0 forces usage of an inline table in all cases; a threshold
+   * of Integer.MAX_VALUE forces usage of OR in all cases
+   *
+   * @return threshold, default 20
+   */
+  protected int getInSubqueryThreshold() {
+    // OVERRIDE POINT
+    return Integer.MAX_VALUE; // was 20
+  }
+
+  /**
+   * Converts an EXISTS or IN predicate into a join. For EXISTS, the subquery
+   * produces an indicator variable, and the result is a relational expression
+   * which outer joins that indicator to the original query. After performing
+   * the outer join, the condition will be TRUE if the EXISTS condition holds,
+   * NULL otherwise.
+   *
+   * @param seek           A query, for example 'select * from emp' or
+   *                       'values (1,2,3)' or '('Foo', 34)'.
+   * @param subqueryType   Whether sub-query is IN, EXISTS or scalar
+   * @param logic Whether the answer needs to be in full 3-valued logic (TRUE,
+   *     FALSE, UNKNOWN) will be required, or whether we can accept an
+   *     approximation (say representing UNKNOWN as FALSE)
+   * @param needsOuterJoin Whether an outer join is needed
+   * @return join expression
+   * @pre extraExpr == null || extraName != null
+   */
+  private Pair<RelNode, Boolean> convertExists(
+      SqlNode seek,
+      RelOptUtil.SubqueryType subqueryType,
+      RelOptUtil.Logic logic,
+      boolean needsOuterJoin) {
+    final SqlValidatorScope seekScope =
+        (seek instanceof SqlSelect)
+            ? validator.getSelectScope((SqlSelect) seek)
+            : null;
+    final Blackboard seekBb = createBlackboard(seekScope, null);
+    RelNode seekRel = convertQueryOrInList(seekBb, seek);
+
+    return RelOptUtil.createExistsPlan(seekRel, subqueryType, logic,
+        needsOuterJoin);
+  }
+
+  private RelNode convertQueryOrInList(
+      Blackboard bb,
+      SqlNode seek) {
+    // NOTE: Once we start accepting single-row queries as row constructors,
+    // there will be an ambiguity here for a case like X IN ((SELECT Y FROM
+    // Z)).  The SQL standard resolves the ambiguity by saying that a lone
+    // select should be interpreted as a table expression, not a row
+    // expression.  The semantic difference is that a table expression can
+    // return multiple rows.
+    if (seek instanceof SqlNodeList) {
+      return convertRowValues(
+          bb,
+          seek,
+          ((SqlNodeList) seek).getList(),
+          false,
+          null);
+    } else {
+      return convertQueryRecursive(seek, false, null);
+    }
+  }
+
+  private RelNode convertRowValues(
+      Blackboard bb,
+      SqlNode rowList,
+      Collection<SqlNode> rows,
+      boolean allowLiteralsOnly,
+      RelDataType targetRowType) {
+    // NOTE jvs 30-Apr-2006: We combine all rows consisting entirely of
+    // literals into a single ValuesRel; this gives the optimizer a smaller
+    // input tree.  For everything else (computed expressions, row
+    // subqueries), we union each row in as a projection on top of a
+    // OneRowRel.
+
+    final List<List<RexLiteral>> tupleList =
+        new ArrayList<List<RexLiteral>>();
+    final RelDataType rowType;
+    if (targetRowType != null) {
+      rowType = targetRowType;
+    } else {
+      rowType =
+          SqlTypeUtil.promoteToRowType(
+              typeFactory,
+              validator.getValidatedNodeType(rowList),
+              null);
+    }
+
+    List<RelNode> unionInputs = new ArrayList<RelNode>();
+    for (SqlNode node : rows) {
+      SqlBasicCall call;
+      if (isRowConstructor(node)) {
+        call = (SqlBasicCall) node;
+        List<RexLiteral> tuple = new ArrayList<RexLiteral>();
+        for (SqlNode operand : call.operands) {
+          RexLiteral rexLiteral =
+              convertLiteralInValuesList(
+                  operand,
+                  bb,
+                  rowType,
+                  tuple.size());
+          if ((rexLiteral == null) && allowLiteralsOnly) {
+            return null;
+          }
+          if ((rexLiteral == null) || !shouldCreateValuesRel) {
+            // fallback to convertRowConstructor
+            tuple = null;
+            break;
+          }
+          tuple.add(rexLiteral);
+        }
+        if (tuple != null) {
+          tupleList.add(tuple);
+          continue;
+        }
+      } else {
+        RexLiteral rexLiteral =
+            convertLiteralInValuesList(
+                node,
+                bb,
+                rowType,
+                0);
+        if ((rexLiteral != null) && shouldCreateValuesRel) {
+          tupleList.add(
+              Collections.singletonList(rexLiteral));
+          continue;
+        } else {
+          if ((rexLiteral == null) && allowLiteralsOnly) {
+            return null;
+          }
+        }
+
+        // convert "1" to "row(1)"
+        call =
+            (SqlBasicCall) SqlStdOperatorTable.ROW.createCall(
+                SqlParserPos.ZERO,
+                node);
+      }
+      unionInputs.add(convertRowConstructor(bb, call));
+    }
+    ValuesRel valuesRel =
+        new ValuesRel(
+            cluster,
+            rowType,
+            tupleList);
+    RelNode resultRel;
+    if (unionInputs.isEmpty()) {
+      resultRel = valuesRel;
+    } else {
+      if (!tupleList.isEmpty()) {
+        unionInputs.add(valuesRel);
+      }
+      UnionRel unionRel =
+          new UnionRel(
+              cluster,
+              unionInputs,
+              true);
+      resultRel = unionRel;
+    }
+    leaves.add(resultRel);
+    return resultRel;
+  }
+
+  private RexLiteral convertLiteralInValuesList(
+      SqlNode sqlNode,
+      Blackboard bb,
+      RelDataType rowType,
+      int iField) {
+    if (!(sqlNode instanceof SqlLiteral)) {
+      return null;
+    }
+    RelDataTypeField field = rowType.getFieldList().get(iField);
+    RelDataType type = field.getType();
+    if (type.isStruct()) {
+      // null literals for weird stuff like UDT's need
+      // special handling during type flattening, so
+      // don't use ValuesRel for those
+      return null;
+    }
+
+    RexNode literalExpr =
+        exprConverter.convertLiteral(
+            bb,
+            (SqlLiteral) sqlNode);
+
+    if (!(literalExpr instanceof RexLiteral)) {
+      assert literalExpr.isA(SqlKind.CAST);
+      RexNode child = ((RexCall) literalExpr).getOperands().get(0);
+      assert RexLiteral.isNullLiteral(child);
+
+      // NOTE jvs 22-Nov-2006:  we preserve type info
+      // in ValuesRel digest, so it's OK to lose it here
+      return (RexLiteral) child;
+    }
+
+    RexLiteral literal = (RexLiteral) literalExpr;
+
+    Comparable value = literal.getValue();
+
+    if (SqlTypeUtil.isExactNumeric(type)) {
+      BigDecimal roundedValue =
+          NumberUtil.rescaleBigDecimal(
+              (BigDecimal) value,
+              type.getScale());
+      return rexBuilder.makeExactLiteral(
+          roundedValue,
+          type);
+    }
+
+    if ((value instanceof NlsString)
+        && (type.getSqlTypeName() == SqlTypeName.CHAR)) {
+      // pad fixed character type
+      NlsString unpadded = (NlsString) value;
+      return rexBuilder.makeCharLiteral(
+          new NlsString(
+              Util.rpad(unpadded.getValue(), type.getPrecision()),
+              unpadded.getCharsetName(),
+              unpadded.getCollation()));
+    }
+    return literal;
+  }
+
+  private boolean isRowConstructor(SqlNode node) {
+    if (!(node.getKind() == SqlKind.ROW)) {
+      return false;
+    }
+    SqlCall call = (SqlCall) node;
+    return call.getOperator().getName().equalsIgnoreCase("row");
+  }
+
+  /**
+   * Builds a list of all <code>IN</code> or <code>EXISTS</code> operators
+   * inside SQL parse tree. Does not traverse inside queries.
+   *
+   * @param bb                           blackboard
+   * @param node                         the SQL parse tree
+   * @param logic Whether the answer needs to be in full 3-valued logic (TRUE,
+   *              FALSE, UNKNOWN) will be required, or whether we can accept
+   *              an approximation (say representing UNKNOWN as FALSE)
+   * @param registerOnlyScalarSubqueries if set to true and the parse tree
+   *                                     corresponds to a variation of a select
+   *                                     node, only register it if it's a scalar
+   *                                     subquery
+   */
+private void findSubqueries(
+      Blackboard bb,
+      SqlNode node,
+      RelOptUtil.Logic logic,
+      boolean registerOnlyScalarSubqueries) {
+    final SqlKind kind = node.getKind();
+    switch (kind) {
+    case EXISTS:
+    case SELECT:
+    case MULTISET_QUERY_CONSTRUCTOR:
+    case MULTISET_VALUE_CONSTRUCTOR:
+    case CURSOR:
+    case SCALAR_QUERY:
+      if (!registerOnlyScalarSubqueries
+          || (kind == SqlKind.SCALAR_QUERY)) {
+        bb.registerSubquery(node, RelOptUtil.Logic.TRUE_FALSE);
+      }
+      return;
+    case IN:
+      if (((SqlCall) node).getOperator() == SqlStdOperatorTable.NOT_IN) {
+        logic = logic.negate();
+      }
+      break;
+    case NOT:
+      logic = logic.negate();
+      break;
+    }
+    if (node instanceof SqlCall) {
+      if (kind == SqlKind.OR
+          || kind == SqlKind.NOT) {
+        // It's always correct to outer join subquery with
+        // containing query; however, when predicates involve Or
+        // or NOT, outer join might be necessary.
+        bb.subqueryNeedsOuterJoin = true;
+      }
+      for (SqlNode operand : ((SqlCall) node).getOperandList()) {
+        if (operand != null) {
+          // In the case of an IN expression, locate scalar
+          // subqueries so we can convert them to constants
+          findSubqueries(
+              bb,
+              operand,
+              logic,
+              kind == SqlKind.IN || registerOnlyScalarSubqueries);
+        }
+      }
+    } else if (node instanceof SqlNodeList) {
+      for (SqlNode child : (SqlNodeList) node) {
+        findSubqueries(
+            bb,
+            child,
+            logic,
+            kind == SqlKind.IN || registerOnlyScalarSubqueries);
+      }
+    }
+
+    // Now that we've located any scalar subqueries inside the IN
+    // expression, register the IN expression itself.  We need to
+    // register the scalar subqueries first so they can be converted
+    // before the IN expression is converted.
+    if (kind == SqlKind.IN) {
+      if (logic == RelOptUtil.Logic.TRUE_FALSE_UNKNOWN
+          && !validator.getValidatedNodeType(node).isNullable()) {
+        logic = RelOptUtil.Logic.UNKNOWN_AS_FALSE;
+      }
+      // TODO: This conversion is only valid in the WHERE clause
+      if (logic == RelOptUtil.Logic.UNKNOWN_AS_FALSE
+          && !bb.subqueryNeedsOuterJoin) {
+        logic = RelOptUtil.Logic.TRUE;
+      }
+      bb.registerSubquery(node, logic);
+    }
+  }
+
+  /**
+   * Converts an expression from {@link SqlNode} to {@link RexNode} format.
+   *
+   * @param node Expression to translate
+   * @return Converted expression
+   */
+  public RexNode convertExpression(
+      SqlNode node) {
+    Map<String, RelDataType> nameToTypeMap = Collections.emptyMap();
+    Blackboard bb =
+        createBlackboard(
+            new ParameterScope((SqlValidatorImpl) validator, nameToTypeMap),
+            null);
+    return bb.convertExpression(node);
+  }
+
+  /**
+   * Converts an expression from {@link SqlNode} to {@link RexNode} format,
+   * mapping identifier references to predefined expressions.
+   *
+   * @param node          Expression to translate
+   * @param nameToNodeMap map from String to {@link RexNode}; when an
+   *                      {@link SqlIdentifier} is encountered, it is used as a
+   *                      key and translated to the corresponding value from
+   *                      this map
+   * @return Converted expression
+   */
+  public RexNode convertExpression(
+      SqlNode node,
+      Map<String, RexNode> nameToNodeMap) {
+    final Map<String, RelDataType> nameToTypeMap =
+        new HashMap<String, RelDataType>();
+    for (Map.Entry<String, RexNode> entry : nameToNodeMap.entrySet()) {
+      nameToTypeMap.put(entry.getKey(), entry.getValue().getType());
+    }
+    Blackboard bb =
+        createBlackboard(
+            new ParameterScope((SqlValidatorImpl) validator, nameToTypeMap),
+            nameToNodeMap);
+    return bb.convertExpression(node);
+  }
+
+  /**
+   * Converts a non-standard expression.
+   *
+   * <p>This method is an extension-point that derived classes can override. If
+   * this method returns a null result, the normal expression translation
+   * process will proceed. The default implementation always returns null.
+   *
+   * @param node Expression
+   * @param bb   Blackboard
+   * @return null to proceed with the usual expression translation process
+   */
+  protected RexNode convertExtendedExpression(
+      SqlNode node,
+      Blackboard bb) {
+    return null;
+  }
+
+  private RexNode convertOver(Blackboard bb, SqlNode node) {
+    SqlCall call = (SqlCall) node;
+    SqlCall aggCall = call.operand(0);
+    SqlNode windowOrRef = call.operand(1);
+    final SqlWindow window =
+        validator.resolveWindow(windowOrRef, bb.scope, true);
+    final SqlNodeList partitionList = window.getPartitionList();
+    final ImmutableList.Builder<RexNode> partitionKeys =
+        ImmutableList.builder();
+    for (SqlNode partition : partitionList) {
+      partitionKeys.add(bb.convertExpression(partition));
+    }
+    RexNode lowerBound = bb.convertExpression(window.getLowerBound());
+    RexNode upperBound = bb.convertExpression(window.getUpperBound());
+    SqlNodeList orderList = window.getOrderList();
+    if ((orderList.size() == 0) && !window.isRows()) {
+      // A logical range requires an ORDER BY clause. Use the implicit
+      // ordering of this relation. There must be one, otherwise it would
+      // have failed validation.
+      orderList = bb.scope.getOrderList();
+      if (orderList == null) {
+        throw new AssertionError(
+            "Relation should have sort key for implicit ORDER BY");
+      }
+    }
+    final ImmutableList.Builder<RexFieldCollation> orderKeys =
+        ImmutableList.builder();
+    final Set<SqlKind> flags = EnumSet.noneOf(SqlKind.class);
+    for (SqlNode order : orderList) {
+      flags.clear();
+      RexNode e = bb.convertSortExpression(order, flags);
+      orderKeys.add(new RexFieldCollation(e, flags));
+    }
+    try {
+      Util.permAssert(bb.window == null, "already in window agg mode");
+      bb.window = window;
+      RexNode rexAgg = exprConverter.convertCall(bb, aggCall);
+      rexAgg =
+          rexBuilder.ensureType(
+              validator.getValidatedNodeType(call), rexAgg, false);
+
+      // Walk over the tree and apply 'over' to all agg functions. This is
+      // necessary because the returned expression is not necessarily a call
+      // to an agg function. For example, AVG(x) becomes SUM(x) / COUNT(x).
+      final RexShuttle visitor =
+          new HistogramShuttle(
+              partitionKeys.build(), orderKeys.build(),
+              RexWindowBound.create(window.getLowerBound(), lowerBound),
+              RexWindowBound.create(window.getUpperBound(), upperBound),
+              window);
+      return rexAgg.accept(visitor);
+    } finally {
+      bb.window = null;
+    }
+  }
+
+  /**
+   * Converts a FROM clause into a relational expression.
+   *
+   * @param bb   Scope within which to resolve identifiers
+   * @param from FROM clause of a query. Examples include:
+   *
+   *             <ul>
+   *             <li>a single table ("SALES.EMP"),
+   *             <li>an aliased table ("EMP AS E"),
+   *             <li>a list of tables ("EMP, DEPT"),
+   *             <li>an ANSI Join expression ("EMP JOIN DEPT ON EMP.DEPTNO =
+   *             DEPT.DEPTNO"),
+   *             <li>a VALUES clause ("VALUES ('Fred', 20)"),
+   *             <li>a query ("(SELECT * FROM EMP WHERE GENDER = 'F')"),
+   *             <li>or any combination of the above.
+   *             </ul>
+   */
+  protected void convertFrom(
+      Blackboard bb,
+      SqlNode from) {
+    SqlCall call;
+    final SqlNode[] operands;
+    switch (from.getKind()) {
+    case AS:
+      operands = ((SqlBasicCall) from).getOperands();
+      convertFrom(bb, operands[0]);
+      return;
+
+    case WITH_ITEM:
+      convertFrom(bb, ((SqlWithItem) from).query);
+      return;
+
+    case WITH:
+      convertFrom(bb, ((SqlWith) from).body);
+      return;
+
+    case TABLESAMPLE:
+      operands = ((SqlBasicCall) from).getOperands();
+      SqlSampleSpec sampleSpec = SqlLiteral.sampleValue(operands[1]);
+      if (sampleSpec instanceof SqlSampleSpec.SqlSubstitutionSampleSpec) {
+        String sampleName =
+            ((SqlSampleSpec.SqlSubstitutionSampleSpec) sampleSpec)
+                .getName();
+        datasetStack.push(sampleName);
+        convertFrom(bb, operands[0]);
+        datasetStack.pop();
+      } else if (sampleSpec instanceof SqlSampleSpec.SqlTableSampleSpec) {
+        SqlSampleSpec.SqlTableSampleSpec tableSampleSpec =
+            (SqlSampleSpec.SqlTableSampleSpec) sampleSpec;
+        convertFrom(bb, operands[0]);
+        RelOptSamplingParameters params =
+            new RelOptSamplingParameters(
+                tableSampleSpec.isBernoulli(),
+                tableSampleSpec.getSamplePercentage(),
+                tableSampleSpec.isRepeatable(),
+                tableSampleSpec.getRepeatableSeed());
+        bb.setRoot(new SamplingRel(cluster, bb.root, params), false);
+      } else {
+        throw Util.newInternal(
+            "unknown TABLESAMPLE type: " + sampleSpec);
+      }
+      return;
+
+    case IDENTIFIER:
+      final SqlValidatorNamespace fromNamespace =
+          validator.getNamespace(from).resolve();
+      if (fromNamespace.getNode() != null) {
+        convertFrom(bb, fromNamespace.getNode());
+        return;
+      }
+      final String datasetName =
+          datasetStack.isEmpty() ? null : datasetStack.peek();
+      boolean[] usedDataset = {false};
+      RelOptTable table =
+          SqlValidatorUtil.getRelOptTable(
+              fromNamespace,
+              catalogReader,
+              datasetName,
+              usedDataset);
+      final RelNode tableRel;
+      if (shouldConvertTableAccess) {
+        tableRel = toRel(table);
+      } else {
+        tableRel = new TableAccessRel(cluster, table);
+      }
+      bb.setRoot(tableRel, true);
+      if (usedDataset[0]) {
+        bb.setDataset(datasetName);
+      }
+      return;
+
+    case JOIN:
+      final SqlJoin join = (SqlJoin) from;
+      final Blackboard fromBlackboard =
+          createBlackboard(validator.getJoinScope(from), null);
+      SqlNode left = join.getLeft();
+      SqlNode right = join.getRight();
+      final boolean isNatural = join.isNatural();
+      final JoinType joinType = join.getJoinType();
+      final Blackboard leftBlackboard =
+          createBlackboard(
+              Util.first(validator.getJoinScope(left),
+                  ((DelegatingScope) bb.scope).getParent()), null);
+      final Blackboard rightBlackboard =
+          createBlackboard(
+              Util.first(validator.getJoinScope(right),
+                  ((DelegatingScope) bb.scope).getParent()), null);
+      convertFrom(leftBlackboard, left);
+      RelNode leftRel = leftBlackboard.root;
+      convertFrom(rightBlackboard, right);
+      RelNode rightRel = rightBlackboard.root;
+      JoinRelType convertedJoinType = convertJoinType(joinType);
+      RexNode conditionExp;
+      if (isNatural) {
+        final List<String> columnList =
+            SqlValidatorUtil.deriveNaturalJoinColumnList(
+                validator.getNamespace(left).getRowType(),
+                validator.getNamespace(right).getRowType());
+        conditionExp = convertUsing(leftRel, rightRel, columnList);
+      } else {
+        conditionExp =
+            convertJoinCondition(
+                fromBlackboard,
+                join.getCondition(),
+                join.getConditionType(),
+                leftRel,
+                rightRel);
+      }
+
+      final RelNode joinRel =
+          createJoin(
+              fromBlackboard,
+              leftRel,
+              rightRel,
+              conditionExp,
+              convertedJoinType);
+      bb.setRoot(joinRel, false);
+      return;
+
+    case SELECT:
+    case INTERSECT:
+    case EXCEPT:
+    case UNION:
+      final RelNode rel = convertQueryRecursive(from, false, null);
+      bb.setRoot(rel, true);
+      return;
+
+    case VALUES:
+      convertValuesImpl(bb, (SqlCall) from, null);
+      return;
+
+    case UNNEST:
+      final SqlNode node = ((SqlCall) from).operand(0);
+      replaceSubqueries(bb, node, RelOptUtil.Logic.TRUE_FALSE_UNKNOWN);
+      final RelNode childRel =
+          RelOptUtil.createProject(
+              (null != bb.root) ? bb.root : new OneRowRel(cluster),
+              Collections.singletonList(bb.convertExpression(node)),
+              Collections.singletonList(validator.deriveAlias(node, 0)),
+              true);
+
+      UncollectRel uncollectRel =
+          new UncollectRel(cluster, cluster.traitSetOf(Convention.NONE),
+              childRel);
+      bb.setRoot(uncollectRel, true);
+      return;
+
+    case COLLECTION_TABLE:
+      call = (SqlCall) from;
+
+      // Dig out real call; TABLE() wrapper is just syntactic.
+      assert call.getOperandList().size() == 1;
+      call = call.operand(0);
+      convertCollectionTable(bb, call);
+      return;
+
+    default:
+      throw Util.newInternal("not a join operator " + from);
+    }
+  }
+
+  protected void convertCollectionTable(
+      Blackboard bb,
+      SqlCall call) {
+    final SqlOperator operator = call.getOperator();
+    if (operator == SqlStdOperatorTable.TABLESAMPLE) {
+      final String sampleName =
+          SqlLiteral.stringValue(call.operand(0));
+      datasetStack.push(sampleName);
+      SqlCall cursorCall = call.operand(1);
+      SqlNode query = cursorCall.operand(0);
+      RelNode converted = convertQuery(query, false, false);
+      bb.setRoot(converted, false);
+      datasetStack.pop();
+      return;
+    }
+    replaceSubqueries(bb, call, RelOptUtil.Logic.TRUE_FALSE_UNKNOWN);
+
+    // Expand table macro if possible. It's more efficient than
+    // TableFunctionRel.
+    if (operator instanceof SqlUserDefinedTableMacro) {
+      final SqlUserDefinedTableMacro udf =
+          (SqlUserDefinedTableMacro) operator;
+      final TranslatableTable table = udf.getTable(typeFactory,
+        call.getOperandList());
+      final RelDataType rowType = table.getRowType(typeFactory);
+      RelOptTable relOptTable = RelOptTableImpl.create(null, rowType, table);
+      RelNode converted = toRel(relOptTable);
+      bb.setRoot(converted, true);
+      return;
+    }
+
+    Type elementType;
+    if (operator instanceof SqlUserDefinedTableFunction) {
+      SqlUserDefinedTableFunction udtf = (SqlUserDefinedTableFunction) operator;
+      elementType = udtf.getElementType(typeFactory, call.getOperandList());
+    } else {
+      elementType = null;
+    }
+
+    RexNode rexCall = bb.convertExpression(call);
+    final List<RelNode> inputs = bb.retrieveCursors();
+    Set<RelColumnMapping> columnMappings =
+        getColumnMappings(operator);
+    TableFunctionRel callRel =
+        new TableFunctionRel(
+            cluster,
+            inputs,
+            rexCall,
+            elementType,
+            validator.getValidatedNodeType(call),
+            columnMappings);
+    bb.setRoot(callRel, true);
+    afterTableFunction(bb, call, callRel);
+  }
+
+  protected void afterTableFunction(
+      SqlToRelConverter.Blackboard bb,
+      SqlCall call,
+      TableFunctionRel callRel) {
+  }
+
+  private Set<RelColumnMapping> getColumnMappings(SqlOperator op) {
+    SqlReturnTypeInference rti = op.getReturnTypeInference();
+    if (rti == null) {
+      return null;
+    }
+    if (rti instanceof TableFunctionReturnTypeInference) {
+      TableFunctionReturnTypeInference tfrti =
+          (TableFunctionReturnTypeInference) rti;
+      return tfrti.getColumnMappings();
+    } else {
+      return null;
+    }
+  }
+
+  protected RelNode createJoin(
+      Blackboard bb,
+      RelNode leftRel,
+      RelNode rightRel,
+      RexNode joinCond,
+      JoinRelType joinType) {
+    assert joinCond != null;
+
+    Set<String> correlatedVariables = RelOptUtil.getVariablesUsed(rightRel);
+    if (correlatedVariables.size() > 0) {
+      final List<Correlation> correlations = Lists.newArrayList();
+
+      for (String correlName : correlatedVariables) {
+        DeferredLookup lookup = mapCorrelToDeferred.get(correlName);
+        RexFieldAccess fieldAccess = lookup.getFieldAccess(correlName);
+        String originalRelName = lookup.getOriginalRelName();
+        String originalFieldName = fieldAccess.getField().getName();
+
+        int[] nsIndexes = {-1};
+        final SqlValidatorScope[] ancestorScopes = {null};
+        SqlValidatorNamespace foundNs =
+            lookup.bb.scope.resolve(
+                originalRelName,
+                ancestorScopes,
+                nsIndexes);
+
+        assert foundNs != null;
+        assert nsIndexes.length == 1;
+
+        int childNamespaceIndex = nsIndexes[0];
+
+        SqlValidatorScope ancestorScope = ancestorScopes[0];
+        boolean correlInCurrentScope = ancestorScope == bb.scope;
+
+        if (correlInCurrentScope) {
+          int namespaceOffset = 0;
+          if (childNamespaceIndex > 0) {
+            // If not the first child, need to figure out the width
+            // of output types from all the preceding namespaces
+            assert ancestorScope instanceof ListScope;
+            List<SqlValidatorNamespace> children =
+                ((ListScope) ancestorScope).getChildren();
+
+            for (int i = 0; i < childNamespaceIndex; i++) {
+              SqlValidatorNamespace child = children.get(i);
+              namespaceOffset +=
+                  child.getRowType().getFieldCount();
+            }
+          }
+
+          RelDataTypeField field =
+              catalogReader.field(foundNs.getRowType(), originalFieldName);
+          int pos = namespaceOffset + field.getIndex();
+
+          assert field.getType()
+              == lookup.getFieldAccess(correlName).getField().getType();
+
+          assert pos != -1;
+
+          if (bb.mapRootRelToFieldProjection.containsKey(bb.root)) {
+            // bb.root is an aggregate and only projects group by
+            // keys.
+            Map<Integer, Integer> exprProjection =
+                bb.mapRootRelToFieldProjection.get(bb.root);
+
+            // subquery can reference group by keys projected from
+            // the root of the outer relation.
+            if (exprProjection.containsKey(pos)) {
+              pos = exprProjection.get(pos);
+            } else {
+              // correl not grouped
+              throw Util.newInternal(
+                  "Identifier '" + originalRelName + "."
+                  + originalFieldName + "' is not a group expr");
+            }
+          }
+
+          Correlation newCorVar =
+              new Correlation(
+                  getCorrelOrdinal(correlName),
+                  pos);
+
+          correlations.add(newCorVar);
+        }
+      }
+
+      if (!correlations.isEmpty()) {
+        return new CorrelatorRel(
+            rightRel.getCluster(),
+            leftRel,
+            rightRel,
+            joinCond,
+            correlations,
+            joinType);
+      }
+    }
+
+    final List<RexNode> extraLeftExprs = new ArrayList<RexNode>();
+    final List<RexNode> extraRightExprs = new ArrayList<RexNode>();
+    final int leftCount = leftRel.getRowType().getFieldCount();
+    final int rightCount = rightRel.getRowType().getFieldCount();
+    if (!containsGet(joinCond)) {
+      joinCond = pushDownJoinConditions(
+          joinCond, leftCount, rightCount, extraLeftExprs, extraRightExprs);
+    }
+    if (!extraLeftExprs.isEmpty()) {
+      final List<RelDataTypeField> fields =
+          leftRel.getRowType().getFieldList();
+      leftRel = RelOptUtil.createProject(
+          leftRel,
+          new AbstractList<Pair<RexNode, String>>() {
+            @Override
+            public int size() {
+              return leftCount + extraLeftExprs.size();
+            }
+
+            @Override
+            public Pair<RexNode, String> get(int index) {
+              if (index < leftCount) {
+                RelDataTypeField field = fields.get(index);
+                return Pair.<RexNode, String>of(
+                    new RexInputRef(index, field.getType()),
+                    field.getName());
+              } else {
+                return Pair.<RexNode, String>of(
+                    extraLeftExprs.get(index - leftCount), null);
+              }
+            }
+          },
+          true);
+    }
+    if (!extraRightExprs.isEmpty()) {
+      final List<RelDataTypeField> fields =
+          rightRel.getRowType().getFieldList();
+      final int newLeftCount = leftCount + extraLeftExprs.size();
+      rightRel = RelOptUtil.createProject(
+          rightRel,
+          new AbstractList<Pair<RexNode, String>>() {
+            @Override
+            public int size() {
+              return rightCount + extraRightExprs.size();
+            }
+
+            @Override
+            public Pair<RexNode, String> get(int index) {
+              if (index < rightCount) {
+                RelDataTypeField field = fields.get(index);
+                return Pair.<RexNode, String>of(
+                    new RexInputRef(index, field.getType()),
+                    field.getName());
+              } else {
+                return Pair.of(
+                    RexUtil.shift(
+                        extraRightExprs.get(index - rightCount),
+                        -newLeftCount),
+                    null);
+              }
+            }
+          },
+          true);
+    }
+    RelNode join = createJoin(
+        leftRel,
+        rightRel,
+        joinCond,
+        joinType,
+        ImmutableSet.<String>of());
+    if (!extraLeftExprs.isEmpty() || !extraRightExprs.isEmpty()) {
+      Mappings.TargetMapping mapping =
+          Mappings.createShiftMapping(
+              leftCount + extraLeftExprs.size()
+                  + rightCount + extraRightExprs.size(),
+              0, 0, leftCount,
+              leftCount, leftCount + extraLeftExprs.size(), rightCount);
+      return RelOptUtil.project(join, mapping);
+    }
+    return join;
+  }
+
+  private static boolean containsGet(RexNode node) {
+    try {
+      node.accept(
+          new RexVisitorImpl<Void>(true) {
+            @Override public Void visitCall(RexCall call) {
+              if (call.getOperator() == RexBuilder.GET_OPERATOR) {
+                throw Util.FoundOne.NULL;
+              }
+              return super.visitCall(call);
+            }
+          });
+      return false;
+    } catch (Util.FoundOne e) {
+      return true;
+    }
+  }
+
+  /**
+   * Pushes down parts of a join condition. For example, given
+   * "emp JOIN dept ON emp.deptno + 1 = dept.deptno", adds a project above
+   * "emp" that computes the expression
+   * "emp.deptno + 1". The resulting join condition is a simple combination
+   * of AND, equals, and input fields.
+   */
+  private RexNode pushDownJoinConditions(
+      RexNode node,
+      int leftCount,
+      int rightCount,
+      List<RexNode> extraLeftExprs,
+      List<RexNode> extraRightExprs) {
+    switch (node.getKind()) {
+    case AND:
+    case OR:
+    case EQUALS:
+      RexCall call = (RexCall) node;
+      List<RexNode> list = new ArrayList<RexNode>();
+      List<RexNode> operands = Lists.newArrayList(call.getOperands());
+      for (int i = 0; i < operands.size(); i++) {
+        RexNode operand = operands.get(i);
+        final int left2 = leftCount + extraLeftExprs.size();
+        final int right2 = rightCount + extraRightExprs.size();
+        final RexNode e =
+            pushDownJoinConditions(
+                operand,
+                leftCount,
+                rightCount,
+                extraLeftExprs,
+                extraRightExprs);
+        final List<RexNode> remainingOperands = Util.skip(operands, i + 1);
+        final int left3 = leftCount + extraLeftExprs.size();
+        final int right3 = rightCount + extraRightExprs.size();
+        fix(remainingOperands, left2, left3);
+        fix(list, left2, left3);
+        list.add(e);
+      }
+      if (!list.equals(call.getOperands())) {
+        return call.clone(call.getType(), list);
+      }
+      return call;
+    case INPUT_REF:
+    case LITERAL:
+      return node;
+    default:
+      BitSet bits = RelOptUtil.InputFinder.bits(node);
+      final int mid = leftCount + extraLeftExprs.size();
+      switch (Side.of(bits, mid)) {
+      case LEFT:
+        fix(extraRightExprs, mid, mid + 1);
+        extraLeftExprs.add(node);
+        return new RexInputRef(mid, node.getType());
+      case RIGHT:
+        final int index2 = mid + rightCount + extraRightExprs.size();
+        extraRightExprs.add(node);
+        return new RexInputRef(index2, node.getType());
+      case BOTH:
+      case EMPTY:
+      default:
+        return node;
+      }
+    }
+  }
+
+  private void fix(List<RexNode> operands, int before, int after) {
+    if (before == after) {
+      return;
+    }
+    for (int i = 0; i < operands.size(); i++) {
+      RexNode node = operands.get(i);
+      operands.set(i, RexUtil.shift(node, before, after - before));
+    }
+  }
+
+  /**
+   * Categorizes whether a bit set contains bits left and right of a
+   * line.
+   */
+  enum Side {
+    LEFT, RIGHT, BOTH, EMPTY;
+
+    static Side of(BitSet bitSet, int middle) {
+      final int firstBit = bitSet.nextSetBit(0);
+      if (firstBit < 0) {
+        return EMPTY;
+      }
+      if (firstBit >= middle) {
+        return RIGHT;
+      }
+      if (bitSet.nextSetBit(middle) < 0) {
+        return LEFT;
+      }
+      return BOTH;
+    }
+  }
+
+  /**
+   * Determines whether a subquery is non-correlated. Note that a
+   * non-correlated subquery can contain correlated references, provided those
+   * references do not reference select statements that are parents of the
+   * subquery.
+   *
+   * @param subq the subquery
+   * @param bb   blackboard used while converting the subquery, i.e., the
+   *             blackboard of the parent query of this subquery
+   * @return true if the subquery is non-correlated.
+   */
+  private boolean isSubqNonCorrelated(RelNode subq, Blackboard bb) {
+    Set<String> correlatedVariables = RelOptUtil.getVariablesUsed(subq);
+    for (String correlName : correlatedVariables) {
+      DeferredLookup lookup = mapCorrelToDeferred.get(correlName);
+      String originalRelName = lookup.getOriginalRelName();
+
+      int[] nsIndexes = {-1};
+      final SqlValidatorScope[] ancestorScopes = {null};
+      SqlValidatorNamespace foundNs =
+          lookup.bb.scope.resolve(
+              originalRelName,
+              ancestorScopes,
+              nsIndexes);
+
+      assert foundNs != null;
+      assert nsIndexes.length == 1;
+
+      SqlValidatorScope ancestorScope = ancestorScopes[0];
+
+      // If the correlated reference is in a scope that's "above" the
+      // subquery, then this is a correlated subquery.
+      SqlValidatorScope parentScope = bb.scope;
+      do {
+        if (ancestorScope == parentScope) {
+          return false;
+        }
+        if (parentScope instanceof DelegatingScope) {
+          parentScope = ((DelegatingScope) parentScope).getParent();
+        } else {
+          break;
+        }
+      } while (parentScope != null);
+    }
+    return true;
+  }
+
+  /**
+   * Returns a list of fields to be prefixed to each relational expression.
+   *
+   * @return List of system fields
+   */
+  protected List<RelDataTypeField> getSystemFields() {
+    return Collections.emptyList();
+  }
+
+  private RexNode convertJoinCondition(
+      Blackboard bb,
+      SqlNode condition,
+      JoinConditionType conditionType,
+      RelNode leftRel,
+      RelNode rightRel) {
+    if (condition == null) {
+      return rexBuilder.makeLiteral(true);
+    }
+    bb.setRoot(ImmutableList.of(leftRel, rightRel));
+    replaceSubqueries(bb, condition, RelOptUtil.Logic.UNKNOWN_AS_FALSE);
+    switch (conditionType) {
+    case ON:
+      bb.setRoot(ImmutableList.of(leftRel, rightRel));
+      return bb.convertExpression(condition);
+    case USING:
+      SqlNodeList list = (SqlNodeList) condition;
+      List<String> nameList = new ArrayList<String>();
+      for (SqlNode columnName : list) {
+        final SqlIdentifier id = (SqlIdentifier) columnName;
+        String name = id.getSimple();
+        nameList.add(name);
+      }
+      return convertUsing(leftRel, rightRel, nameList);
+    default:
+      throw Util.unexpected(conditionType);
+    }
+  }
+
+  /**
+   * Returns an expression for matching columns of a USING clause or inferred
+   * from NATURAL JOIN. "a JOIN b USING (x, y)" becomes "a.x = b.x AND a.y =
+   * b.y". Returns null if the column list is empty.
+   *
+   * @param leftRel  Left input to the join
+   * @param rightRel Right input to the join
+   * @param nameList List of column names to join on
+   * @return Expression to match columns from name list, or true if name list
+   * is empty
+   */
+  private RexNode convertUsing(
+      RelNode leftRel,
+      RelNode rightRel,
+      List<String> nameList) {
+    final List<RexNode> list = Lists.newArrayList();
+    for (String name : nameList) {
+      final RelDataType leftRowType = leftRel.getRowType();
+      RelDataTypeField leftField = catalogReader.field(leftRowType, name);
+      RexNode left =
+          rexBuilder.makeInputRef(
+              leftField.getType(),
+              leftField.getIndex());
+      final RelDataType rightRowType = rightRel.getRowType();
+      RelDataTypeField rightField =
+          catalogReader.field(rightRowType, name);
+      RexNode right =
+          rexBuilder.makeInputRef(
+              rightField.getType(),
+              leftRowType.getFieldList().size() + rightField.getIndex());
+      RexNode equalsCall =
+          rexBuilder.makeCall(
+              SqlStdOperatorTable.EQUALS,
+              left,
+              right);
+      list.add(equalsCall);
+    }
+    return RexUtil.composeConjunction(rexBuilder, list, false);
+  }
+
+  private static JoinRelType convertJoinType(JoinType joinType) {
+    switch (joinType) {
+    case COMMA:
+    case INNER:
+    case CROSS:
+      return JoinRelType.INNER;
+    case FULL:
+      return JoinRelType.FULL;
+    case LEFT:
+      return JoinRelType.LEFT;
+    case RIGHT:
+      return JoinRelType.RIGHT;
+    default:
+      throw Util.unexpected(joinType);
+    }
+  }
+
+  /**
+   * Converts the SELECT, GROUP BY and HAVING clauses of an aggregate query.
+   *
+   * <p>This method extracts SELECT, GROUP BY and HAVING clauses, and creates
+   * an {@link AggConverter}, then delegates to {@link #createAggImpl}.
+   * Derived class may override this method to change any of those clauses or
+   * specify a different {@link AggConverter}.
+   *
+   * @param bb            Scope within which to resolve identifiers
+   * @param select        Query
+   * @param orderExprList Additional expressions needed to implement ORDER BY
+   */
+  protected void convertAgg(
+      Blackboard bb,
+      SqlSelect select,
+      List<SqlNode> orderExprList) {
+    assert bb.root != null : "precondition: child != null";
+    SqlNodeList groupList = select.getGroup();
+    SqlNodeList selectList = select.getSelectList();
+    SqlNode having = select.getHaving();
+
+    final AggConverter aggConverter = new AggConverter(bb, select);
+    createAggImpl(
+        bb,
+        aggConverter,
+        selectList,
+        groupList,
+        having,
+        orderExprList);
+  }
+
+  protected final void createAggImpl(
+      Blackboard bb,
+      AggConverter aggConverter,
+      SqlNodeList selectList,
+      SqlNodeList groupList,
+      SqlNode having,
+      List<SqlNode> orderExprList) {
+    SqlNodeList aggList = new SqlNodeList(SqlParserPos.ZERO);
+
+    for (SqlNode selectNode : selectList) {
+      if (validator.isAggregate(selectNode)) {
+        aggList.add(selectNode);
+      }
+    }
+
+    // first replace the subqueries inside the aggregates
+    // because they will provide input rows to the aggregates.
+    replaceSubqueries(bb, aggList, RelOptUtil.Logic.TRUE_FALSE_UNKNOWN);
+
+    // If group-by clause is missing, pretend that it has zero elements.
+    if (groupList == null) {
+      groupList = SqlNodeList.EMPTY;
+    }
+
+    // register the group exprs
+
+    // build a map to remember the projections from the top scope to the
+    // output of the current root.
+    //
+    // Currently farrago allows expressions, not just column references in
+    // group by list. This is not SQL 2003 compliant.
+
+    Map<Integer, Integer> groupExprProjection =
+        new HashMap<Integer, Integer>();
+
+    int i = -1;
+    for (SqlNode groupExpr : groupList) {
+      ++i;
+      final SqlNode expandedGroupExpr =
+          validator.expand(groupExpr, bb.scope);
+      aggConverter.addGroupExpr(expandedGroupExpr);
+
+      if (expandedGroupExpr instanceof SqlIdentifier) {
+        // SQL 2003 does not allow expressions of column references
+        SqlIdentifier expr = (SqlIdentifier) expandedGroupExpr;
+
+        // column references should be fully qualified.
+        assert expr.names.size() == 2;
+        String originalRelName = expr.names.get(0);
+        String originalFieldName = expr.names.get(1);
+
+        int[] nsIndexes = {-1};
+        final SqlValidatorScope[] ancestorScopes = {null};
+        SqlValidatorNamespace foundNs =
+            bb.scope.resolve(
+                originalRelName,
+                ancestorScopes,
+                nsIndexes);
+
+        assert foundNs != null;
+        assert nsIndexes.length == 1;
+        int childNamespaceIndex = nsIndexes[0];
+
+        int namespaceOffset = 0;
+
+        if (childNamespaceIndex > 0) {
+          // If not the first child, need to figure out the width of
+          // output types from all the preceding namespaces
+          assert ancestorScopes[0] instanceof ListScope;
+          List<SqlValidatorNamespace> children =
+              ((ListScope) ancestorScopes[0]).getChildren();
+
+          for (int j = 0; j < childNamespaceIndex; j++) {
+            namespaceOffset +=
+                children.get(j).getRowType().getFieldCount();
+          }
+        }
+
+        RelDataTypeField field =
+            catalogReader.field(foundNs.getRowType(), originalFieldName);
+        int origPos = namespaceOffset + field.getIndex();
+
+        groupExprProjection.put(origPos, i);
+      }
+    }
+
+    RexNode havingExpr = null;
+    List<RexNode> selectExprs = new ArrayList<RexNode>();
+    List<String> selectNames = new ArrayList<String>();
+
+    try {
+      Util.permAssert(bb.agg == null, "already in agg mode");
+      bb.agg = aggConverter;
+
+      // convert the select and having expressions, so that the
+      // agg converter knows which aggregations are required
+
+      selectList.accept(aggConverter);
+      for (SqlNode expr : orderExprList) {
+        expr.accept(aggConverter);
+      }
+      if (having != null) {
+        having.accept(aggConverter);
+      }
+
+      // compute inputs to the aggregator
+      List<RexNode> preExprs = aggConverter.getPreExprs();
+      List<String> preNames = aggConverter.getPreNames();
+
+      if (preExprs.size() == 0) {
+        // Special case for COUNT(*), where we can end up with no inputs
+        // at all.  The rest of the system doesn't like 0-tuples, so we
+        // select a dummy constant here.
+        preExprs =
+            Collections.singletonList(
+                (RexNode) rexBuilder.makeExactLiteral(BigDecimal.ZERO));
+        preNames = Collections.singletonList(null);
+      }
+
+      final RelNode inputRel = bb.root;
+
+      // Project the expressions required by agg and having.
+      bb.setRoot(
+          RelOptUtil.createProject(
+              inputRel,
+              preExprs,
+              preNames,
+              true),
+          false);
+      bb.mapRootRelToFieldProjection.put(bb.root, groupExprProjection);
+
+      // REVIEW jvs 31-Oct-2007:  doesn't the declaration of
+      // monotonicity here assume sort-based aggregation at
+      // the physical level?
+
+      // Tell bb which of group columns are sorted.
+      bb.columnMonotonicities.clear();
+      for (SqlNode groupItem : groupList) {
+        bb.columnMonotonicities.add(
+            bb.scope.getMonotonicity(groupItem));
+      }
+
+      // Add the aggregator
+      bb.setRoot(
+          createAggregate(
+              bb,
+              BitSets.range(aggConverter.groupExprs.size()),
+              aggConverter.getAggCalls()),
+          false);
+
+      bb.mapRootRelToFieldProjection.put(bb.root, groupExprProjection);
+
+      // Replace subqueries in having here and modify having to use
+      // the replaced expressions
+      if (having != null) {
+        SqlNode newHav

<TRUNCATED>

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

Posted by lu...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/job/src/test/java/com/kylinolap/job/hadoop/cube/BaseCuboidJobTest.java
----------------------------------------------------------------------
diff --git a/job/src/test/java/com/kylinolap/job/hadoop/cube/BaseCuboidJobTest.java b/job/src/test/java/com/kylinolap/job/hadoop/cube/BaseCuboidJobTest.java
new file mode 100644
index 0000000..ed698c4
--- /dev/null
+++ b/job/src/test/java/com/kylinolap/job/hadoop/cube/BaseCuboidJobTest.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.job.hadoop.cube;
+
+import static org.junit.Assert.*;
+
+import java.io.File;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileUtil;
+import org.apache.hadoop.util.ToolRunner;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+import com.kylinolap.common.util.AbstractKylinTestCase;
+import com.kylinolap.common.util.LocalFileMetadataTestCase;
+
+/**
+ * @author George Song (ysong1)
+ * 
+ */
+public class BaseCuboidJobTest extends LocalFileMetadataTestCase {
+
+    private Configuration conf;
+
+    @Before
+    public void setup() throws Exception {
+        conf = new Configuration();
+        // for local runner out-of-memory issue
+        conf.set("mapreduce.task.io.sort.mb", "10");
+
+        createTestMetadata();
+    }
+
+    @After
+    public void after() throws Exception {
+        cleanupTestMetadata();
+    }
+
+    @Test
+    public void testJob() throws Exception {
+        String input = "src/test/resources/data/flat_table/";
+        String output = "target/test-output/base_cuboid/";
+        String cubeName = "test_kylin_cube_with_slr_1_new_segment";
+        String segmentName = "20130331080000_20131212080000";
+        String jobname = "base_cuboid_job";
+        String level = "0";
+        FileUtil.fullyDelete(new File(output));
+
+        String[] args = { "-input", input, "-cubename", cubeName, "-segmentname", segmentName, "-output", output, "-jobname", jobname, "-level", level };
+        assertEquals("Job failed", 0, ToolRunner.run(conf, new BaseCuboidJob(), args));
+    }
+
+    @Test
+    public void testJobWithBadParas() throws Exception {
+
+        final String input = "src/test/resources/data/flat_table/";
+        final String output = "target/test-output/base_cuboid/";
+        final String metadata = AbstractKylinTestCase.LOCALMETA_TEST_DATA;
+
+        FileUtil.fullyDelete(new File(output));
+
+        String[] args = { "-input", input, "-output", output, "-metadata", metadata };
+        assertEquals(2, ToolRunner.run(conf, new BaseCuboidJob(), args));
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/job/src/test/java/com/kylinolap/job/hadoop/cube/BaseCuboidMapperPerformanceTest.java
----------------------------------------------------------------------
diff --git a/job/src/test/java/com/kylinolap/job/hadoop/cube/BaseCuboidMapperPerformanceTest.java b/job/src/test/java/com/kylinolap/job/hadoop/cube/BaseCuboidMapperPerformanceTest.java
new file mode 100644
index 0000000..bab08ec
--- /dev/null
+++ b/job/src/test/java/com/kylinolap/job/hadoop/cube/BaseCuboidMapperPerformanceTest.java
@@ -0,0 +1,63 @@
+/*
+ * 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.job.hadoop.cube;
+
+import java.io.IOException;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.io.SequenceFile;
+import org.apache.hadoop.io.SequenceFile.Reader;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.io.Writable;
+import org.apache.hadoop.mapreduce.Mapper.Context;
+import org.apache.hadoop.util.ReflectionUtils;
+import org.junit.Ignore;
+import org.junit.Test;
+
+/**
+ * @author yangli9
+ * 
+ */
+@SuppressWarnings({ "rawtypes", "unchecked" })
+public class BaseCuboidMapperPerformanceTest {
+
+    String metadataUrl = "hbase:yadesk00:2181:/hbase-unsecure";
+    String cubeName = "test_kylin_cube_with_slr";
+    Path srcPath = new Path("/download/test_kylin_cube_with_slr_intermediate_table_64mb.seq");
+
+    @Ignore
+    @Test
+    public void test() throws IOException, InterruptedException {
+        Configuration hconf = new Configuration();
+        BaseCuboidMapper mapper = new BaseCuboidMapper();
+        Context context = MockupMapContext.create(hconf, metadataUrl, cubeName, null);
+
+        mapper.setup(context);
+
+        Reader reader = new Reader(hconf, SequenceFile.Reader.file(srcPath));
+        Writable key = (Writable) ReflectionUtils.newInstance(reader.getKeyClass(), hconf);
+        Text value = new Text();
+
+        while (reader.next(key, value)) {
+            mapper.map(key, value, context);
+        }
+
+        reader.close();
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/job/src/test/java/com/kylinolap/job/hadoop/cube/BaseCuboidMapperTest.java
----------------------------------------------------------------------
diff --git a/job/src/test/java/com/kylinolap/job/hadoop/cube/BaseCuboidMapperTest.java b/job/src/test/java/com/kylinolap/job/hadoop/cube/BaseCuboidMapperTest.java
new file mode 100644
index 0000000..95f255f
--- /dev/null
+++ b/job/src/test/java/com/kylinolap/job/hadoop/cube/BaseCuboidMapperTest.java
@@ -0,0 +1,144 @@
+/*
+ * 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.job.hadoop.cube;
+
+import static org.junit.Assert.*;
+
+import java.io.File;
+import java.math.BigDecimal;
+import java.util.List;
+
+import org.apache.commons.io.FileUtils;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.mrunit.mapreduce.MapDriver;
+import org.apache.hadoop.mrunit.types.Pair;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+import com.kylinolap.common.util.LocalFileMetadataTestCase;
+import com.kylinolap.cube.CubeInstance;
+import com.kylinolap.cube.CubeManager;
+import com.kylinolap.cube.kv.RowKeyDecoder;
+import com.kylinolap.cube.measure.MeasureCodec;
+import com.kylinolap.job.constant.BatchConstants;
+import com.kylinolap.metadata.model.cube.MeasureDesc;
+
+/**
+ * @author George Song (ysong1)
+ * 
+ */
+public class BaseCuboidMapperTest extends LocalFileMetadataTestCase {
+
+    MapDriver<Text, Text, Text, Text> mapDriver;
+    String localTempDir = System.getProperty("java.io.tmpdir") + File.separator;
+
+    @Before
+    public void setUp() throws Exception {
+        createTestMetadata();
+
+        // hack for distributed cache
+        FileUtils.deleteDirectory(new File("../job/meta"));
+        FileUtils.copyDirectory(new File(this.getTestConfig().getMetadataUrl()), new File("../job/meta"));
+
+        BaseCuboidMapper<Text> mapper = new BaseCuboidMapper<Text>();
+        mapDriver = MapDriver.newMapDriver(mapper);
+    }
+
+    @After
+    public void after() throws Exception {
+        cleanupTestMetadata();
+        FileUtils.deleteDirectory(new File("../job/meta"));
+    }
+
+    @Test
+    public void testMapperWithHeader() throws Exception {
+        String cubeName = "test_kylin_cube_with_slr_1_new_segment";
+        String segmentName = "20130331080000_20131212080000";
+        mapDriver.getConfiguration().set(BatchConstants.CFG_CUBE_NAME, cubeName);
+        mapDriver.getConfiguration().set(BatchConstants.CFG_CUBE_SEGMENT_NAME, segmentName);
+        // mapDriver.getConfiguration().set(BatchConstants.CFG_METADATA_URL,
+        // metadata);
+        mapDriver.withInput(new Text("key"), new Text("2012-12-15118480Health & BeautyFragrancesWomenAuction15123456789132.331"));
+        List<Pair<Text, Text>> result = mapDriver.run();
+
+        CubeManager cubeMgr = CubeManager.getInstance(this.getTestConfig());
+        CubeInstance cube = cubeMgr.getCube(cubeName);
+
+        assertEquals(1, result.size());
+        Text rowkey = result.get(0).getFirst();
+        byte[] key = rowkey.getBytes();
+        byte[] header = Bytes.head(key, 26);
+        byte[] sellerId = Bytes.tail(header, 18);
+        byte[] cuboidId = Bytes.head(header, 8);
+        byte[] restKey = Bytes.tail(key, rowkey.getLength() - 26);
+
+        RowKeyDecoder decoder = new RowKeyDecoder(cube.getFirstSegment());
+        decoder.decode(key);
+        assertEquals("[123456789, 2012-12-15, 11848, Health & Beauty, Fragrances, Women, Auction, 0, 15]", decoder.getValues().toString());
+
+        assertTrue(Bytes.toString(sellerId).startsWith("123456789"));
+        assertEquals(511, Bytes.toLong(cuboidId));
+        assertEquals(22, restKey.length);
+
+        verifyMeasures(cube.getDescriptor().getMeasures(), result.get(0).getSecond(), "132.33", "132.33", "132.33", 1);
+    }
+
+    private void verifyMeasures(List<MeasureDesc> measures, Text valueBytes, String m1, String m2, String m3, long m4) {
+        MeasureCodec codec = new MeasureCodec(measures);
+        Object[] values = new Object[measures.size()];
+        codec.decode(valueBytes, values);
+        assertTrue(new BigDecimal(m1).equals(values[0]));
+        assertTrue(new BigDecimal(m2).equals(values[1]));
+        assertTrue(new BigDecimal(m3).equals(values[2]));
+        assertTrue(m4 == ((LongWritable) values[3]).get());
+    }
+
+    @Test
+    public void testMapperWithNull() throws Exception {
+        String cubeName = "test_kylin_cube_with_slr_1_new_segment";
+        String segmentName = "20130331080000_20131212080000";
+        mapDriver.getConfiguration().set(BatchConstants.CFG_CUBE_NAME, cubeName);
+        mapDriver.getConfiguration().set(BatchConstants.CFG_CUBE_SEGMENT_NAME, segmentName);
+        // mapDriver.getConfiguration().set(BatchConstants.CFG_METADATA_URL,
+        // metadata);
+        mapDriver.withInput(new Text("key"), new Text("2012-12-15118480Health & BeautyFragrances\\NAuction15123456789\\N\\N"));
+        List<Pair<Text, Text>> result = mapDriver.run();
+
+        CubeManager cubeMgr = CubeManager.getInstance(this.getTestConfig());
+        CubeInstance cube = cubeMgr.getCube(cubeName);
+
+        assertEquals(1, result.size());
+        Text rowkey = result.get(0).getFirst();
+        byte[] key = rowkey.getBytes();
+        byte[] header = Bytes.head(key, 26);
+        byte[] sellerId = Bytes.tail(header, 18);
+        byte[] cuboidId = Bytes.head(header, 8);
+        byte[] restKey = Bytes.tail(key, rowkey.getLength() - 26);
+
+        RowKeyDecoder decoder = new RowKeyDecoder(cube.getFirstSegment());
+        decoder.decode(key);
+        assertEquals("[123456789, 2012-12-15, 11848, Health & Beauty, Fragrances, null, Auction, 0, 15]", decoder.getValues().toString());
+
+        assertTrue(Bytes.toString(sellerId).startsWith("123456789"));
+        assertEquals(511, Bytes.toLong(cuboidId));
+        assertEquals(22, restKey.length);
+
+        verifyMeasures(cube.getDescriptor().getMeasures(), result.get(0).getSecond(), "0", "0", "0", 1L);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/job/src/test/java/com/kylinolap/job/hadoop/cube/CopySeq.java
----------------------------------------------------------------------
diff --git a/job/src/test/java/com/kylinolap/job/hadoop/cube/CopySeq.java b/job/src/test/java/com/kylinolap/job/hadoop/cube/CopySeq.java
new file mode 100644
index 0000000..7645222
--- /dev/null
+++ b/job/src/test/java/com/kylinolap/job/hadoop/cube/CopySeq.java
@@ -0,0 +1,79 @@
+/*
+ * 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.job.hadoop.cube;
+
+import java.io.IOException;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.io.SequenceFile;
+import org.apache.hadoop.io.SequenceFile.CompressionType;
+import org.apache.hadoop.io.SequenceFile.Reader;
+import org.apache.hadoop.io.SequenceFile.Writer;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.io.Writable;
+import org.apache.hadoop.io.compress.CompressionCodec;
+import org.apache.hadoop.io.compress.CompressionCodecFactory;
+import org.apache.hadoop.util.ReflectionUtils;
+
+/**
+ * @author yangli9
+ * 
+ */
+public class CopySeq {
+
+    public static void main(String[] args) throws IOException {
+        copyTo64MB(args[0], args[1]);
+    }
+
+    public static void copyTo64MB(String src, String dst) throws IOException {
+        Configuration hconf = new Configuration();
+        Path srcPath = new Path(src);
+        Path dstPath = new Path(dst);
+
+        FileSystem fs = FileSystem.get(hconf);
+        long srcSize = fs.getFileStatus(srcPath).getLen();
+        int copyTimes = (int) (67108864 / srcSize); // 64 MB
+        System.out.println("Copy " + copyTimes + " times");
+
+        Reader reader = new Reader(hconf, SequenceFile.Reader.file(srcPath));
+        Writable key = (Writable) ReflectionUtils.newInstance(reader.getKeyClass(), hconf);
+        Text value = new Text();
+
+        Writer writer = SequenceFile.createWriter(hconf, Writer.file(dstPath), Writer.keyClass(key.getClass()), Writer.valueClass(Text.class), Writer.compression(CompressionType.BLOCK, getLZOCodec(hconf)));
+
+        int count = 0;
+        while (reader.next(key, value)) {
+            for (int i = 0; i < copyTimes; i++) {
+                writer.append(key, value);
+                count++;
+            }
+        }
+
+        System.out.println("Len: " + writer.getLength());
+        System.out.println("Rows: " + count);
+
+        reader.close();
+        writer.close();
+    }
+
+    static CompressionCodec getLZOCodec(Configuration hconf) {
+        CompressionCodecFactory factory = new CompressionCodecFactory(hconf);
+        return factory.getCodecByClassName("org.apache.hadoop.io.compress.LzoCodec");
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/job/src/test/java/com/kylinolap/job/hadoop/cube/CubeHFileMapper2Test.java
----------------------------------------------------------------------
diff --git a/job/src/test/java/com/kylinolap/job/hadoop/cube/CubeHFileMapper2Test.java b/job/src/test/java/com/kylinolap/job/hadoop/cube/CubeHFileMapper2Test.java
new file mode 100644
index 0000000..ffe2fad
--- /dev/null
+++ b/job/src/test/java/com/kylinolap/job/hadoop/cube/CubeHFileMapper2Test.java
@@ -0,0 +1,92 @@
+/*
+ * 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.job.hadoop.cube;
+
+import static org.junit.Assert.*;
+
+import java.io.File;
+import java.nio.ByteBuffer;
+
+import org.apache.commons.io.FileUtils;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.mapreduce.Mapper.Context;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+import com.kylinolap.common.util.LocalFileMetadataTestCase;
+import com.kylinolap.cube.CubeManager;
+import com.kylinolap.cube.kv.RowConstants;
+import com.kylinolap.cube.measure.MeasureCodec;
+import com.kylinolap.metadata.model.cube.CubeDesc;
+
+/**
+ * @author yangli9
+ * 
+ */
+@SuppressWarnings({ "rawtypes", "unchecked" })
+public class CubeHFileMapper2Test extends LocalFileMetadataTestCase {
+
+    String cubeName = "test_kylin_cube_with_slr_ready";
+
+    MeasureCodec codec;
+    ByteBuffer buf = ByteBuffer.allocate(RowConstants.ROWVALUE_BUFFER_SIZE);
+    Object[] outKV = new Object[2];
+
+    @Before
+    public void setup() throws Exception {
+        this.createTestMetadata();
+        // hack for distributed cache
+        FileUtils.deleteDirectory(new File("../job/meta"));
+        FileUtils.copyDirectory(new File(this.getTestConfig().getMetadataUrl()), new File("../job/meta"));
+        CubeDesc desc = CubeManager.getInstance(this.getTestConfig()).getCube(cubeName).getDescriptor();
+        codec = new MeasureCodec(desc.getMeasures());
+    }
+
+    @After
+    public void after() throws Exception {
+        cleanupTestMetadata();
+        FileUtils.deleteDirectory(new File("../job/meta"));
+    }
+
+    @Test
+    public void testBasic() throws Exception {
+
+        Configuration hconf = new Configuration();
+        Context context = MockupMapContext.create(hconf, this.getTestConfig().getMetadataUrl(), cubeName, outKV);
+
+        CubeHFileMapper mapper = new CubeHFileMapper();
+        mapper.setup(context);
+
+        Text key = new Text("not important");
+        Text value = new Text(new byte[] { 2, 2, 51, -79, 1 });
+
+        mapper.map(key, value, context);
+
+        ImmutableBytesWritable outKey = (ImmutableBytesWritable) outKV[0];
+        KeyValue outValue = (KeyValue) outKV[1];
+
+        assertTrue(Bytes.compareTo(key.getBytes(), 0, key.getLength(), outKey.get(), outKey.getOffset(), outKey.getLength()) == 0);
+
+        assertTrue(Bytes.compareTo(value.getBytes(), 0, value.getLength(), outValue.getValueArray(), outValue.getValueOffset(), outValue.getValueLength()) == 0);
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/job/src/test/java/com/kylinolap/job/hadoop/cube/CubeHFileMapperTest.java
----------------------------------------------------------------------
diff --git a/job/src/test/java/com/kylinolap/job/hadoop/cube/CubeHFileMapperTest.java b/job/src/test/java/com/kylinolap/job/hadoop/cube/CubeHFileMapperTest.java
new file mode 100644
index 0000000..0f3aff9
--- /dev/null
+++ b/job/src/test/java/com/kylinolap/job/hadoop/cube/CubeHFileMapperTest.java
@@ -0,0 +1,78 @@
+/*
+ * 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.job.hadoop.cube;
+
+import static org.junit.Assert.*;
+
+import java.io.IOException;
+import java.util.List;
+
+import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.mrunit.mapreduce.MapDriver;
+import org.apache.hadoop.mrunit.types.Pair;
+import org.junit.Before;
+import org.junit.Ignore;
+import org.junit.Test;
+
+import com.kylinolap.job.constant.BatchConstants;
+
+/**
+ * @author George Song (ysong1)
+ * 
+ */
+public class CubeHFileMapperTest {
+
+    MapDriver<Text, Text, ImmutableBytesWritable, KeyValue> mapDriver;
+
+    private String cube_name = "FLAT_ITEM_CUBE";
+
+    @Before
+    public void setUp() {
+        CubeHFileMapper mapper = new CubeHFileMapper();
+        mapDriver = MapDriver.newMapDriver(mapper);
+    }
+
+    @SuppressWarnings("deprecation")
+    @Test
+    @Ignore
+    public void testMapper2() throws IOException {
+        mapDriver.getConfiguration().set(BatchConstants.CFG_CUBE_NAME, cube_name);
+
+        mapDriver.addInput(new Text("52010tech"), new Text("35.432"));
+
+        List<Pair<ImmutableBytesWritable, KeyValue>> result = mapDriver.run();
+
+        assertEquals(2, result.size());
+
+        byte[] bytes = { 0, 0, 0, 0, 0, 0, 0, 119, 33, 0, 22, 1, 0, 121, 7 };
+        ImmutableBytesWritable key = new ImmutableBytesWritable(bytes);
+
+        Pair<ImmutableBytesWritable, KeyValue> p1 = result.get(0);
+        Pair<ImmutableBytesWritable, KeyValue> p2 = result.get(1);
+
+        assertEquals(key, p1.getFirst());
+        assertEquals("cf1", new String(p1.getSecond().getFamily()));
+        assertEquals("usd_amt", new String(p1.getSecond().getQualifier()));
+        assertEquals("35.43", new String(p1.getSecond().getValue()));
+
+        assertEquals(key, p2.getFirst());
+        assertEquals("cf1", new String(p2.getSecond().getFamily()));
+        assertEquals("item_count", new String(p2.getSecond().getQualifier()));
+        assertEquals("2", new String(p2.getSecond().getValue()));
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/job/src/test/java/com/kylinolap/job/hadoop/cube/CubeReducerTest.java
----------------------------------------------------------------------
diff --git a/job/src/test/java/com/kylinolap/job/hadoop/cube/CubeReducerTest.java b/job/src/test/java/com/kylinolap/job/hadoop/cube/CubeReducerTest.java
new file mode 100644
index 0000000..3d54122
--- /dev/null
+++ b/job/src/test/java/com/kylinolap/job/hadoop/cube/CubeReducerTest.java
@@ -0,0 +1,122 @@
+/*
+ * 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.job.hadoop.cube;
+
+import static org.junit.Assert.*;
+
+import java.io.File;
+import java.math.BigDecimal;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.commons.io.FileUtils;
+import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.mrunit.mapreduce.ReduceDriver;
+import org.apache.hadoop.mrunit.types.Pair;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+import com.kylinolap.common.util.LocalFileMetadataTestCase;
+import com.kylinolap.cube.CubeManager;
+import com.kylinolap.cube.kv.RowConstants;
+import com.kylinolap.cube.measure.MeasureCodec;
+import com.kylinolap.job.constant.BatchConstants;
+import com.kylinolap.metadata.model.cube.CubeDesc;
+
+/**
+ * @author George Song (ysong1)
+ * 
+ */
+public class CubeReducerTest extends LocalFileMetadataTestCase {
+
+    ReduceDriver<Text, Text, Text, Text> reduceDriver;
+    String localTempDir = System.getProperty("java.io.tmpdir") + File.separator;
+
+    ByteBuffer buf = ByteBuffer.allocate(RowConstants.ROWVALUE_BUFFER_SIZE);
+
+    @Before
+    public void setUp() throws Exception {
+        createTestMetadata();
+
+        // hack for distributed cache
+        FileUtils.deleteDirectory(new File("../job/meta"));
+        FileUtils.copyDirectory(new File(this.getTestConfig().getMetadataUrl()), new File("../job/meta"));
+
+        CuboidReducer reducer = new CuboidReducer();
+        reduceDriver = ReduceDriver.newReduceDriver(reducer);
+    }
+
+    @After
+    public void after() throws Exception {
+        cleanupTestMetadata();
+        FileUtils.deleteDirectory(new File("../job/meta"));
+    }
+
+    @Test
+    public void testReducer() throws Exception {
+
+        reduceDriver.getConfiguration().set(BatchConstants.CFG_CUBE_NAME, "test_kylin_cube_with_slr_ready");
+
+        CubeDesc cubeDesc = CubeManager.getInstance(this.getTestConfig()).getCube("test_kylin_cube_with_slr_ready").getDescriptor();
+        MeasureCodec codec = new MeasureCodec(cubeDesc.getMeasures());
+
+        Text key1 = new Text("72010ustech");
+        List<Text> values1 = new ArrayList<Text>();
+        values1.add(newValueText(codec, "15.09", "15.09", "15.09", 1));
+        values1.add(newValueText(codec, "20.34", "20.34", "20.34", 1));
+        values1.add(newValueText(codec, "10", "10", "10", 1));
+
+        Text key2 = new Text("1tech");
+        List<Text> values2 = new ArrayList<Text>();
+        values2.add(newValueText(codec, "15.09", "15.09", "15.09", 1));
+        values2.add(newValueText(codec, "20.34", "20.34", "20.34", 1));
+
+        Text key3 = new Text("0");
+        List<Text> values3 = new ArrayList<Text>();
+        values3.add(newValueText(codec, "146.52", "146.52", "146.52", 4));
+
+        reduceDriver.withInput(key1, values1);
+        reduceDriver.withInput(key2, values2);
+        reduceDriver.withInput(key3, values3);
+
+        List<Pair<Text, Text>> result = reduceDriver.run();
+
+        Pair<Text, Text> p1 = new Pair<Text, Text>(new Text("72010ustech"), newValueText(codec, "45.43", "10", "20.34", 3));
+        Pair<Text, Text> p2 = new Pair<Text, Text>(new Text("1tech"), newValueText(codec, "35.43", "15.09", "20.34", 2));
+        Pair<Text, Text> p3 = new Pair<Text, Text>(new Text("0"), newValueText(codec, "146.52", "146.52", "146.52", 4));
+
+        assertEquals(3, result.size());
+
+        assertTrue(result.contains(p1));
+        assertTrue(result.contains(p2));
+        assertTrue(result.contains(p3));
+    }
+
+    private Text newValueText(MeasureCodec codec, String sum, String min, String max, int count) {
+        Object[] values = new Object[] { new BigDecimal(sum), new BigDecimal(min), new BigDecimal(max), new LongWritable(count) };
+
+        buf.clear();
+        codec.encode(values, buf);
+
+        Text t = new Text();
+        t.set(buf.array(), 0, buf.position());
+        return t;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/job/src/test/java/com/kylinolap/job/hadoop/cube/KeyDistributionJobTest.java
----------------------------------------------------------------------
diff --git a/job/src/test/java/com/kylinolap/job/hadoop/cube/KeyDistributionJobTest.java b/job/src/test/java/com/kylinolap/job/hadoop/cube/KeyDistributionJobTest.java
new file mode 100644
index 0000000..14405ec
--- /dev/null
+++ b/job/src/test/java/com/kylinolap/job/hadoop/cube/KeyDistributionJobTest.java
@@ -0,0 +1,64 @@
+package com.kylinolap.job.hadoop.cube;
+
+///*
+// * 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.index.cube;
+//
+//import static org.junit.Assert.*;
+//
+//import java.io.File;
+//import java.io.IOException;
+//
+//import org.apache.hadoop.conf.Configuration;
+//import org.apache.hadoop.fs.FileUtil;
+//import org.apache.hadoop.util.ToolRunner;
+//import org.junit.Before;
+//import org.junit.Test;
+//
+//import com.kylinolap.metadata.MetadataManager;
+//
+///**
+// * @author xjiang
+// *
+// */
+//public class KeyDistributionJobTest {
+//
+//    private Configuration conf;
+//
+//    @Before
+//    public void setup() throws IOException {
+//        conf = new Configuration();
+//        conf.set("fs.default.name", "file:///");
+//        conf.set("mapred.job.tracker", "local");
+//    }
+//
+//    @Test
+//    public void testJob() throws Exception {
+//        final String input = "src/test/resources/data/base_cuboid/,src/test/resources/data/6d_cuboid/";
+//        final String output = "target/test-output/key_distribution/";
+//        final String cubeName = "test_kylin_cube_with_slr";
+//        final String metadata = MetadataManager.getMetadataUrlFromEnv();
+//
+//        FileUtil.fullyDelete(new File(output));
+//
+//        String[] args =
+//                { "-input", input, "-cubename", cubeName, "-output", output, "-metadata", metadata,
+//                        "-columnpercentage", "50", "-splitnumber", "10" };
+//        assertEquals("Job failed", 0, ToolRunner.run(new KeyDistributionJob(), args));
+//    }
+//
+// }

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/job/src/test/java/com/kylinolap/job/hadoop/cube/KeyDistributionMapperTest.java
----------------------------------------------------------------------
diff --git a/job/src/test/java/com/kylinolap/job/hadoop/cube/KeyDistributionMapperTest.java b/job/src/test/java/com/kylinolap/job/hadoop/cube/KeyDistributionMapperTest.java
new file mode 100644
index 0000000..e7d1748
--- /dev/null
+++ b/job/src/test/java/com/kylinolap/job/hadoop/cube/KeyDistributionMapperTest.java
@@ -0,0 +1,153 @@
+package com.kylinolap.job.hadoop.cube;
+
+///*
+// * 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.index.cube;
+//
+//import static org.junit.Assert.*;
+//
+//import java.io.File;
+//import java.io.IOException;
+//import java.util.List;
+//
+//import org.apache.hadoop.io.LongWritable;
+//import org.apache.hadoop.io.Text;
+//import org.apache.hadoop.mrunit.mapreduce.MapDriver;
+//import org.apache.hadoop.mrunit.types.Pair;
+//import org.junit.Before;
+//import org.junit.Test;
+//
+//import com.kylinolap.index.BatchConstants;
+//import com.kylinolap.metadata.MetadataManager;
+//
+///**
+// * @author ysong1
+// *
+// */
+//public class KeyDistributionMapperTest {
+//    @SuppressWarnings("rawtypes")
+//    MapDriver mapDriver;
+//    String localTempDir = System.getProperty("java.io.tmpdir") + File.separator;
+//
+//    @Before
+//    public void setUp() {
+//        KeyDistributionMapper mapper = new KeyDistributionMapper();
+//        mapDriver = MapDriver.newMapDriver(mapper);
+//    }
+//
+//    @SuppressWarnings("unchecked")
+//    @Test
+//    public void testMapperWithoutHeader() throws IOException {
+//        String matadata = MetadataManager.getMetadataUrlFromEnv();
+//        mapDriver.getConfiguration().set(BatchConstants.CFG_CUBE_NAME, "test_kylin_cube_without_slr");
+//        mapDriver.getConfiguration().set(BatchConstants.CFG_METADATA_URL, matadata);
+//        mapDriver.getConfiguration().set(KeyDistributionJob.KEY_COLUMN_PERCENTAGE, "7");
+//        mapDriver.getConfiguration().set(KeyDistributionJob.KEY_HEADER_LENGTH, "8");
+//
+//        Text inputKey1 =
+//                new Text(new byte[] { 0, 0, 0, 0, 0, 0, 0, 127, 11, 56, -23, 0, 22, 98, 1, 0, 121, 7 });
+//        Text inputKey2 =
+//                new Text(new byte[] { 0, 0, 0, 0, 0, 0, 0, 127, 11, 122, 1, 0, 22, 98, 1, 0, 121, 7 });
+//        Text inputKey3 =
+//                new Text(new byte[] { 2, 2, 2, 2, 2, 2, 2, 127, 11, 56, -23, 0, 22, 98, 1, 0, 121, 7 });
+//        Text inputKey4 =
+//                new Text(new byte[] { 3, 3, 3, 3, 3, 3, 3, 127, 11, 56, -23, 0, 22, 98, 1, 0, 121, 7 });
+//        Text inputKey5 =
+//                new Text(new byte[] { 4, 4, 4, 4, 4, 4, 4, 127, 11, 56, -23, 0, 22, 98, 1, 0, 121, 7 });
+//        Text inputKey6 =
+//                new Text(new byte[] { 5, 5, 5, 5, 5, 5, 5, 127, 11, 56, -23, 0, 22, 98, 1, 0, 121, 7 });
+//        Text inputKey7 =
+//                new Text(new byte[] { 6, 6, 6, 6, 6, 6, 6, 127, 11, 56, -23, 0, 22, 98, 1, 0, 121, 7 });
+//
+//        mapDriver.addInput(inputKey1, new Text("abc"));
+//        mapDriver.addInput(inputKey2, new Text("abc"));
+//        mapDriver.addInput(inputKey3, new Text("abc"));
+//        mapDriver.addInput(inputKey4, new Text("abc"));
+//        mapDriver.addInput(inputKey5, new Text("abc"));
+//        mapDriver.addInput(inputKey6, new Text("abc"));
+//        mapDriver.addInput(inputKey7, new Text("abc"));
+//
+//        List<Pair<Text, LongWritable>> result = mapDriver.run();
+//
+//        assertEquals(7, result.size());
+//
+//        byte[] key1 = result.get(0).getFirst().getBytes();
+//        LongWritable value1 = result.get(0).getSecond();
+//        assertArrayEquals(new byte[] { 0, 0, 0, 0, 0, 0, 0, 127, 11 }, key1);
+//        assertEquals(2, value1.get());
+//
+//        byte[] key7 = result.get(6).getFirst().getBytes();
+//        LongWritable value7 = result.get(6).getSecond();
+//        assertArrayEquals(new byte[] { 0 }, key7);
+//        assertEquals(7, value7.get());
+//    }
+//
+//    @SuppressWarnings("unchecked")
+//    @Test
+//    public void testMapperWithHeader() throws IOException {
+//        String matadata = MetadataManager.getMetadataUrlFromEnv();
+//        mapDriver.getConfiguration().set(BatchConstants.CFG_CUBE_NAME, "test_kylin_cube_with_slr");
+//        mapDriver.getConfiguration().set(BatchConstants.CFG_METADATA_URL, matadata);
+//        mapDriver.getConfiguration().set(KeyDistributionJob.KEY_COLUMN_PERCENTAGE, "7");
+//        mapDriver.getConfiguration().set(KeyDistributionJob.KEY_HEADER_LENGTH, "26");
+//
+//        Text inputKey1 =
+//                new Text(new byte[] { 0, 0, 0, 0, 0, 0, 0, 127, 11, 56, -23, 0, 22, 98, 1, 0, 121, 7, 0, 0,
+//                        0, 0, 0, 0, 0, 127, 11, 56, -23, 0, 22, 98, 1, 0, 121, 7 });
+//        Text inputKey2 =
+//                new Text(new byte[] { 0, 0, 0, 0, 0, 0, 0, 127, 11, 56, -23, 0, 22, 98, 1, 0, 121, 7, 0, 0,
+//                        0, 0, 0, 0, 0, 127, 11, 122, 1, 0, 22, 98, 1, 0, 121, 7 });
+//        Text inputKey3 =
+//                new Text(new byte[] { 0, 0, 0, 0, 0, 0, 0, 127, 11, 56, -23, 0, 22, 98, 1, 0, 121, 7, 2, 2,
+//                        2, 2, 2, 2, 2, 127, 11, 56, -23, 0, 22, 98, 1, 0, 121, 7 });
+//        Text inputKey4 =
+//                new Text(new byte[] { 0, 0, 0, 0, 0, 0, 0, 127, 11, 56, -23, 0, 22, 98, 1, 0, 121, 7, 3, 3,
+//                        3, 3, 3, 3, 3, 127, 11, 56, -23, 0, 22, 98, 1, 0, 121, 7 });
+//        Text inputKey5 =
+//                new Text(new byte[] { 0, 0, 0, 0, 0, 0, 0, 127, 11, 56, -23, 0, 22, 98, 1, 0, 121, 7, 4, 4,
+//                        4, 4, 4, 4, 4, 127, 11, 56, -23, 0, 22, 98, 1, 0, 121, 7 });
+//        Text inputKey6 =
+//                new Text(new byte[] { 0, 0, 0, 0, 0, 0, 0, 127, 11, 56, -23, 0, 22, 98, 1, 0, 121, 7, 5, 5,
+//                        5, 5, 5, 5, 5, 127, 11, 56, -23, 0, 22, 98, 1, 0, 121, 7 });
+//        Text inputKey7 =
+//                new Text(new byte[] { 0, 0, 0, 0, 0, 0, 0, 127, 11, 56, -23, 0, 22, 98, 1, 0, 121, 7, 6, 6,
+//                        6, 6, 6, 6, 6, 127, 11, 56, -23, 0, 22, 98, 1, 0, 121, 7 });
+//
+//        mapDriver.addInput(inputKey1, new Text("abc"));
+//        mapDriver.addInput(inputKey2, new Text("abc"));
+//        mapDriver.addInput(inputKey3, new Text("abc"));
+//        mapDriver.addInput(inputKey4, new Text("abc"));
+//        mapDriver.addInput(inputKey5, new Text("abc"));
+//        mapDriver.addInput(inputKey6, new Text("abc"));
+//        mapDriver.addInput(inputKey7, new Text("abc"));
+//
+//        List<Pair<Text, LongWritable>> result = mapDriver.run();
+//
+//        assertEquals(7, result.size());
+//
+//        byte[] key1 = result.get(0).getFirst().getBytes();
+//        LongWritable value1 = result.get(0).getSecond();
+//        assertArrayEquals(new byte[] { 0, 0, 0, 0, 0, 0, 0, 127, 11, 56, -23, 0, 22, 98, 1, 0, 121, 7, 0, 0,
+//                0, 0, 0, 0, 0, 127, 11 }, key1);
+//        assertEquals(2, value1.get());
+//
+//        byte[] key7 = result.get(6).getFirst().getBytes();
+//        LongWritable value7 = result.get(6).getSecond();
+//        assertArrayEquals(new byte[] { 0 }, key7);
+//        assertEquals(7, value7.get());
+//    }
+// }

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/job/src/test/java/com/kylinolap/job/hadoop/cube/MergeCuboidJobTest.java
----------------------------------------------------------------------
diff --git a/job/src/test/java/com/kylinolap/job/hadoop/cube/MergeCuboidJobTest.java b/job/src/test/java/com/kylinolap/job/hadoop/cube/MergeCuboidJobTest.java
new file mode 100644
index 0000000..7391191
--- /dev/null
+++ b/job/src/test/java/com/kylinolap/job/hadoop/cube/MergeCuboidJobTest.java
@@ -0,0 +1,86 @@
+/*
+ * 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.job.hadoop.cube;
+
+import static org.junit.Assert.*;
+
+import java.io.File;
+
+import org.apache.commons.io.FileUtils;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileUtil;
+import org.apache.hadoop.util.ToolRunner;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+import com.kylinolap.common.util.LocalFileMetadataTestCase;
+
+/**
+ * @author ysong1
+ */
+public class MergeCuboidJobTest extends LocalFileMetadataTestCase {
+
+    private Configuration conf;
+
+    @Before
+    public void setup() throws Exception {
+        conf = new Configuration();
+        // conf.set("fs.default.name", "file:///");
+        // conf.set("mapred.job.tracker", "local");
+
+        // for local runner out-of-memory issue
+        conf.set("mapreduce.task.io.sort.mb", "10");
+        createTestMetadata();
+    }
+
+    @After
+    public void after() throws Exception {
+        cleanupTestMetadata();
+    }
+
+    @Test
+    public void test() throws Exception {
+        // String input =
+        // "src/test/resources/data/base_cuboid,src/test/resources/data/6d_cuboid";
+        String output = "target/test-output/merged_cuboid";
+        String cubeName = "test_kylin_cube_with_slr_ready";
+        String jobname = "merge_cuboid";
+
+        File baseFolder = File.createTempFile("kylin-f24668f6-dcff-4cb6-a89b-77f1119df8fa-", "base");
+        baseFolder.delete();
+        baseFolder.mkdir();
+        FileUtils.copyDirectory(new File("src/test/resources/data/base_cuboid"), baseFolder);
+        baseFolder.deleteOnExit();
+
+        File sixDFolder = File.createTempFile("kylin-f24668f6-dcff-4cb6-a89b-77f1119df8fa-", "6d");
+        sixDFolder.delete();
+        sixDFolder.mkdir();
+        FileUtils.copyDirectory(new File("src/test/resources/data/base_cuboid"), sixDFolder);
+        sixDFolder.deleteOnExit();
+
+        FileUtil.fullyDelete(new File(output));
+
+        // CubeManager cubeManager =
+        // CubeManager.getInstanceFromEnv(this.getTestConfig());
+
+        String[] args = { "-input", baseFolder.getAbsolutePath() + "," + sixDFolder.getAbsolutePath(), "-cubename", cubeName, "-segmentname", "20130331080000_20131212080000", "-output", output, "-jobname", jobname };
+        assertEquals("Job failed", 0, ToolRunner.run(conf, new MergeCuboidJob(), args));
+
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/job/src/test/java/com/kylinolap/job/hadoop/cube/MergeCuboidMapperTest.java
----------------------------------------------------------------------
diff --git a/job/src/test/java/com/kylinolap/job/hadoop/cube/MergeCuboidMapperTest.java b/job/src/test/java/com/kylinolap/job/hadoop/cube/MergeCuboidMapperTest.java
new file mode 100644
index 0000000..ede75b0
--- /dev/null
+++ b/job/src/test/java/com/kylinolap/job/hadoop/cube/MergeCuboidMapperTest.java
@@ -0,0 +1,189 @@
+/*
+ * 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.job.hadoop.cube;
+
+import java.io.File;
+import java.io.IOException;
+import java.text.ParseException;
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.commons.io.FileUtils;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.mrunit.mapreduce.MapDriver;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.kylinolap.common.util.LocalFileMetadataTestCase;
+import com.kylinolap.cube.CubeBuildTypeEnum;
+import com.kylinolap.cube.CubeInstance;
+import com.kylinolap.cube.CubeManager;
+import com.kylinolap.cube.CubeSegment;
+import com.kylinolap.cube.exception.CubeIntegrityException;
+import com.kylinolap.cube.project.ProjectManager;
+import com.kylinolap.dict.Dictionary;
+import com.kylinolap.dict.DictionaryGenerator;
+import com.kylinolap.dict.DictionaryInfo;
+import com.kylinolap.dict.DictionaryManager;
+import com.kylinolap.dict.TrieDictionary;
+import com.kylinolap.dict.lookup.TableSignature;
+import com.kylinolap.job.constant.BatchConstants;
+import com.kylinolap.metadata.MetadataManager;
+import com.kylinolap.metadata.model.cube.TblColRef;
+
+/**
+ * @author honma
+ */
+@SuppressWarnings("rawtypes")
+public class MergeCuboidMapperTest extends LocalFileMetadataTestCase {
+
+    private static final Logger logger = LoggerFactory.getLogger(MergeCuboidMapperTest.class);
+
+    MapDriver<Text, Text, Text, Text> mapDriver;
+    CubeManager cubeManager;
+    CubeInstance cube;
+    DictionaryManager dictionaryManager;
+
+    TblColRef lfn;
+    TblColRef lsi;
+    TblColRef ssc;
+
+    private DictionaryInfo makeSharedDict() throws IOException {
+        TableSignature signature = new TableSignature();
+        signature.setSize(100);
+        signature.setLastModifiedTime(System.currentTimeMillis());
+        signature.setPath("fake_common_dict");
+
+        DictionaryInfo newDictInfo = new DictionaryInfo("", "", 0, "string", signature, "");
+
+        List<byte[]> values = new ArrayList<byte[]>();
+        values.add(new byte[] { 101, 101, 101 });
+        values.add(new byte[] { 102, 102, 102 });
+        Dictionary<?> dict = DictionaryGenerator.buildDictionaryFromValueList(newDictInfo, values);
+        dictionaryManager.trySaveNewDict(dict, newDictInfo);
+        ((TrieDictionary) dict).dump(System.out);
+
+        return newDictInfo;
+    }
+
+    @Before
+    public void setUp() throws Exception {
+
+        createTestMetadata();
+
+        logger.info("The metadataUrl is : " + this.getTestConfig());
+
+        MetadataManager.removeInstance(this.getTestConfig());
+        CubeManager.removeInstance(this.getTestConfig());
+        ProjectManager.removeInstance(this.getTestConfig());
+        DictionaryManager.removeInstance(this.getTestConfig());
+
+        // hack for distributed cache
+        // CubeManager.removeInstance(KylinConfig.createInstanceFromUri("../job/meta"));//to
+        // make sure the following mapper could get latest CubeManger
+        FileUtils.deleteDirectory(new File("../job/meta"));
+
+        MergeCuboidMapper mapper = new MergeCuboidMapper();
+        mapDriver = MapDriver.newMapDriver(mapper);
+
+        cubeManager = CubeManager.getInstance(this.getTestConfig());
+        cube = cubeManager.getCube("test_kylin_cube_without_slr_left_join_ready_2_segments");
+        dictionaryManager = DictionaryManager.getInstance(getTestConfig());
+        lfn = cube.getDescriptor().findColumnRef("TEST_KYLIN_FACT", "LSTG_FORMAT_NAME");
+        lsi = cube.getDescriptor().findColumnRef("TEST_KYLIN_FACT", "CAL_DT");
+        ssc = cube.getDescriptor().findColumnRef("TEST_CATEGORY_GROUPINGS", "META_CATEG_NAME");
+
+        DictionaryInfo sharedDict = makeSharedDict();
+
+        boolean isFirstSegment = true;
+        for (CubeSegment segment : cube.getSegments()) {
+
+            TableSignature signature = new TableSignature();
+            signature.setSize(100);
+            signature.setLastModifiedTime(System.currentTimeMillis());
+            signature.setPath("fake_dict_for" + lfn.getName() + segment.getName());
+
+            DictionaryInfo newDictInfo = new DictionaryInfo(lfn.getTable(), lfn.getColumn().getName(), lfn.getColumn().getZeroBasedIndex(), "string", signature, "");
+
+            List<byte[]> values = new ArrayList<byte[]>();
+            values.add(new byte[] { 97, 97, 97 });
+            if (isFirstSegment)
+                values.add(new byte[] { 99, 99, 99 });
+            else
+                values.add(new byte[] { 98, 98, 98 });
+            Dictionary<?> dict = DictionaryGenerator.buildDictionaryFromValueList(newDictInfo, values);
+            dictionaryManager.trySaveNewDict(dict, newDictInfo);
+            ((TrieDictionary) dict).dump(System.out);
+
+            segment.putDictResPath(lfn, newDictInfo.getResourcePath());
+            segment.putDictResPath(lsi, sharedDict.getResourcePath());
+            segment.putDictResPath(ssc, sharedDict.getResourcePath());
+
+            // cubeManager.saveResource(segment.getCubeInstance());
+            // cubeManager.afterCubeUpdated(segment.getCubeInstance());
+            cubeManager.updateCube(cube);
+
+            isFirstSegment = false;
+        }
+
+    }
+
+    @After
+    public void after() throws Exception {
+        cleanupTestMetadata();
+        FileUtils.deleteDirectory(new File("../job/meta"));
+    }
+
+    @Test
+    public void test() throws IOException, ParseException, CubeIntegrityException {
+
+        String cubeName = "test_kylin_cube_without_slr_left_join_ready_2_segments";
+
+        List<CubeSegment> newSegments = cubeManager.allocateSegments(cube, CubeBuildTypeEnum.MERGE, 1384240200000L, 1386835200000L);
+
+        logger.info("Size of new segments: " + newSegments.size());
+
+        CubeSegment newSeg = newSegments.get(0);
+        String segmentName = newSeg.getName();
+
+        ((TrieDictionary) cubeManager.getDictionary(newSeg, lfn)).dump(System.out);
+
+        // hack for distributed cache
+        File metaDir = new File("../job/meta");
+        FileUtils.copyDirectory(new File(this.getTestConfig().getMetadataUrl()), metaDir);
+
+        mapDriver.getConfiguration().set(BatchConstants.CFG_CUBE_NAME, cubeName);
+        mapDriver.getConfiguration().set(BatchConstants.CFG_CUBE_SEGMENT_NAME, segmentName);
+        // mapDriver.getConfiguration().set(KylinConfig.KYLIN_METADATA_URL,
+        // "../job/meta");
+
+        byte[] key = new byte[] { 0, 0, 0, 0, 0, 0, 0, -92, 1, 1, 1 };
+        byte[] value = new byte[] { 1, 2, 3 };
+        byte[] newkey = new byte[] { 0, 0, 0, 0, 0, 0, 0, -92, 1, 1, 2 };
+        byte[] newvalue = new byte[] { 1, 2, 3 };
+
+        mapDriver.withInput(new Text(key), new Text(value));
+        mapDriver.withOutput(new Text(newkey), new Text(newvalue));
+        mapDriver.setMapInputPath(new Path("/apps/hdmi-prod/b_kylin/prod/kylin-f24668f6-dcff-4cb6-a89b-77f1119df8fa/vac_sw_cube_v4/cuboid/15d_cuboid"));
+
+        mapDriver.runTest();
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/job/src/test/java/com/kylinolap/job/hadoop/cube/MockupMapContext.java
----------------------------------------------------------------------
diff --git a/job/src/test/java/com/kylinolap/job/hadoop/cube/MockupMapContext.java b/job/src/test/java/com/kylinolap/job/hadoop/cube/MockupMapContext.java
new file mode 100644
index 0000000..a06ddd4
--- /dev/null
+++ b/job/src/test/java/com/kylinolap/job/hadoop/cube/MockupMapContext.java
@@ -0,0 +1,312 @@
+/*
+ * 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.job.hadoop.cube;
+
+import java.io.IOException;
+import java.net.URI;
+
+import org.apache.commons.lang.NotImplementedException;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.conf.Configuration.IntegerRanges;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.io.RawComparator;
+import org.apache.hadoop.mapreduce.Counter;
+import org.apache.hadoop.mapreduce.InputFormat;
+import org.apache.hadoop.mapreduce.InputSplit;
+import org.apache.hadoop.mapreduce.JobID;
+import org.apache.hadoop.mapreduce.MapContext;
+import org.apache.hadoop.mapreduce.Mapper;
+import org.apache.hadoop.mapreduce.Mapper.Context;
+import org.apache.hadoop.mapreduce.OutputCommitter;
+import org.apache.hadoop.mapreduce.OutputFormat;
+import org.apache.hadoop.mapreduce.Partitioner;
+import org.apache.hadoop.mapreduce.Reducer;
+import org.apache.hadoop.mapreduce.TaskAttemptID;
+import org.apache.hadoop.mapreduce.lib.map.WrappedMapper;
+import org.apache.hadoop.security.Credentials;
+
+import com.kylinolap.job.constant.BatchConstants;
+
+/**
+ * @author yangli9
+ * 
+ */
+@SuppressWarnings({ "rawtypes", "unchecked" })
+public class MockupMapContext {
+
+    public static Context create(final Configuration hconf, String metadataUrl, String cubeName, final Object[] outKV) {
+
+        hconf.set(BatchConstants.CFG_CUBE_NAME, cubeName);
+
+        return new WrappedMapper().getMapContext(new MapContext() {
+
+            @Override
+            public boolean nextKeyValue() throws IOException, InterruptedException {
+                throw new NotImplementedException();
+            }
+
+            @Override
+            public Object getCurrentKey() throws IOException, InterruptedException {
+                throw new NotImplementedException();
+            }
+
+            @Override
+            public Object getCurrentValue() throws IOException, InterruptedException {
+                throw new NotImplementedException();
+            }
+
+            @Override
+            public void write(Object key, Object value) throws IOException, InterruptedException {
+                System.out.println("Write -- k:" + key + ", v:" + value);
+                if (outKV != null) {
+                    outKV[0] = key;
+                    outKV[1] = value;
+                }
+            }
+
+            @Override
+            public OutputCommitter getOutputCommitter() {
+                throw new NotImplementedException();
+            }
+
+            @Override
+            public TaskAttemptID getTaskAttemptID() {
+                throw new NotImplementedException();
+            }
+
+            @Override
+            public void setStatus(String msg) {
+                throw new NotImplementedException();
+            }
+
+            @Override
+            public String getStatus() {
+                throw new NotImplementedException();
+            }
+
+            @Override
+            public float getProgress() {
+                throw new NotImplementedException();
+            }
+
+            @Override
+            public Counter getCounter(Enum<?> counterName) {
+                throw new NotImplementedException();
+            }
+
+            @Override
+            public Counter getCounter(String groupName, String counterName) {
+                throw new NotImplementedException();
+            }
+
+            @Override
+            public Configuration getConfiguration() {
+                return hconf;
+            }
+
+            @Override
+            public Credentials getCredentials() {
+                throw new NotImplementedException();
+            }
+
+            @Override
+            public JobID getJobID() {
+                throw new NotImplementedException();
+            }
+
+            @Override
+            public int getNumReduceTasks() {
+                throw new NotImplementedException();
+            }
+
+            @Override
+            public Path getWorkingDirectory() throws IOException {
+                throw new NotImplementedException();
+            }
+
+            @Override
+            public Class<?> getOutputKeyClass() {
+                throw new NotImplementedException();
+            }
+
+            @Override
+            public Class<?> getOutputValueClass() {
+                throw new NotImplementedException();
+            }
+
+            @Override
+            public Class<?> getMapOutputKeyClass() {
+                throw new NotImplementedException();
+            }
+
+            @Override
+            public Class<?> getMapOutputValueClass() {
+                throw new NotImplementedException();
+            }
+
+            @Override
+            public String getJobName() {
+                throw new NotImplementedException();
+            }
+
+            @Override
+            public Class<? extends InputFormat<?, ?>> getInputFormatClass() throws ClassNotFoundException {
+                throw new NotImplementedException();
+            }
+
+            @Override
+            public Class<? extends Mapper<?, ?, ?, ?>> getMapperClass() throws ClassNotFoundException {
+                throw new NotImplementedException();
+            }
+
+            @Override
+            public Class<? extends Reducer<?, ?, ?, ?>> getCombinerClass() throws ClassNotFoundException {
+                throw new NotImplementedException();
+            }
+
+            @Override
+            public Class<? extends Reducer<?, ?, ?, ?>> getReducerClass() throws ClassNotFoundException {
+                throw new NotImplementedException();
+            }
+
+            @Override
+            public Class<? extends OutputFormat<?, ?>> getOutputFormatClass() throws ClassNotFoundException {
+                throw new NotImplementedException();
+            }
+
+            @Override
+            public Class<? extends Partitioner<?, ?>> getPartitionerClass() throws ClassNotFoundException {
+                throw new NotImplementedException();
+            }
+
+            @Override
+            public RawComparator<?> getSortComparator() {
+                throw new NotImplementedException();
+            }
+
+            @Override
+            public String getJar() {
+                throw new NotImplementedException();
+            }
+
+            @Override
+            public RawComparator<?> getGroupingComparator() {
+                throw new NotImplementedException();
+            }
+
+            @Override
+            public boolean getJobSetupCleanupNeeded() {
+                throw new NotImplementedException();
+            }
+
+            @Override
+            public boolean getTaskCleanupNeeded() {
+                throw new NotImplementedException();
+            }
+
+            @Override
+            public boolean getProfileEnabled() {
+                throw new NotImplementedException();
+            }
+
+            @Override
+            public String getProfileParams() {
+                throw new NotImplementedException();
+            }
+
+            @Override
+            public IntegerRanges getProfileTaskRange(boolean isMap) {
+                throw new NotImplementedException();
+            }
+
+            @Override
+            public String getUser() {
+                throw new NotImplementedException();
+            }
+
+            @Override
+            public boolean getSymlink() {
+                throw new NotImplementedException();
+            }
+
+            @Override
+            public Path[] getArchiveClassPaths() {
+                throw new NotImplementedException();
+            }
+
+            @Override
+            public URI[] getCacheArchives() throws IOException {
+                throw new NotImplementedException();
+            }
+
+            @Override
+            public URI[] getCacheFiles() throws IOException {
+                throw new NotImplementedException();
+            }
+
+            @Override
+            public Path[] getLocalCacheArchives() throws IOException {
+                throw new NotImplementedException();
+            }
+
+            @Override
+            public Path[] getLocalCacheFiles() throws IOException {
+                throw new NotImplementedException();
+            }
+
+            @Override
+            public Path[] getFileClassPaths() {
+                throw new NotImplementedException();
+            }
+
+            @Override
+            public String[] getArchiveTimestamps() {
+                throw new NotImplementedException();
+            }
+
+            @Override
+            public String[] getFileTimestamps() {
+                throw new NotImplementedException();
+            }
+
+            @Override
+            public int getMaxMapAttempts() {
+                throw new NotImplementedException();
+            }
+
+            @Override
+            public int getMaxReduceAttempts() {
+                throw new NotImplementedException();
+            }
+
+            @Override
+            public void progress() {
+                throw new NotImplementedException();
+            }
+
+            @Override
+            public InputSplit getInputSplit() {
+                throw new NotImplementedException();
+            }
+
+            @Override
+            public RawComparator<?> getCombinerKeyGroupingComparator() {
+                throw new NotImplementedException();
+            }
+        });
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/job/src/test/java/com/kylinolap/job/hadoop/cube/NDCuboidJobTest.java
----------------------------------------------------------------------
diff --git a/job/src/test/java/com/kylinolap/job/hadoop/cube/NDCuboidJobTest.java b/job/src/test/java/com/kylinolap/job/hadoop/cube/NDCuboidJobTest.java
new file mode 100644
index 0000000..66c23ba
--- /dev/null
+++ b/job/src/test/java/com/kylinolap/job/hadoop/cube/NDCuboidJobTest.java
@@ -0,0 +1,85 @@
+/*
+ * 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.job.hadoop.cube;
+
+import static org.junit.Assert.*;
+
+import java.io.File;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileUtil;
+import org.apache.hadoop.util.ToolRunner;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+import com.kylinolap.common.util.LocalFileMetadataTestCase;
+
+/**
+ * @author George Song (ysong1)
+ * 
+ */
+public class NDCuboidJobTest extends LocalFileMetadataTestCase {
+
+    private Configuration conf;
+
+    @Before
+    public void setup() throws Exception {
+        conf = new Configuration();
+        // conf.set("fs.default.name", "file:///");
+        // conf.set("mapred.job.tracker", "local");
+
+        // for local runner out-of-memory issue
+        conf.set("mapreduce.task.io.sort.mb", "10");
+
+        createTestMetadata();
+    }
+
+    @After
+    public void after() throws Exception {
+        cleanupTestMetadata();
+    }
+
+    @Test
+    public void testJob6D() throws Exception {
+        String input = "src/test/resources/data/base_cuboid/";
+        String output = "target/test-output/6d_cuboid";
+        String cubeName = "test_kylin_cube_with_slr_1_new_segment";
+        String segmentName = "20130331080000_20131212080000";
+        String jobname = "6d_cuboid";
+        String level = "1";
+
+        FileUtil.fullyDelete(new File(output));
+
+        String[] args = { "-input", input, "-cubename", cubeName, "-segmentname", segmentName, "-output", output, "-jobname", jobname, "-level", level };
+        assertEquals("Job failed", 0, ToolRunner.run(conf, new NDCuboidJob(), args));
+    }
+
+    @Test
+    public void testJob5D() throws Exception {
+        final String input = "src/test/resources/data/6d_cuboid/";
+        final String output = "target/test-output/5d_cuboid";
+        final String cubeName = "test_kylin_cube_with_slr_1_new_segment";
+        String segmentName = "20130331080000_20131212080000";
+        String jobname = "5d_cuboid";
+        String level = "2";
+
+        FileUtil.fullyDelete(new File(output));
+
+        String[] args = { "-input", input, "-cubename", cubeName, "-segmentname", segmentName, "-output", output, "-jobname", jobname, "-level", level };
+        assertEquals("Job failed", 0, ToolRunner.run(conf, new NDCuboidJob(), args));
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/job/src/test/java/com/kylinolap/job/hadoop/cube/NDCuboidMapperTest.java
----------------------------------------------------------------------
diff --git a/job/src/test/java/com/kylinolap/job/hadoop/cube/NDCuboidMapperTest.java b/job/src/test/java/com/kylinolap/job/hadoop/cube/NDCuboidMapperTest.java
new file mode 100644
index 0000000..96b4aec
--- /dev/null
+++ b/job/src/test/java/com/kylinolap/job/hadoop/cube/NDCuboidMapperTest.java
@@ -0,0 +1,101 @@
+/*
+ * 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.job.hadoop.cube;
+
+import static org.junit.Assert.*;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.List;
+
+import org.apache.commons.io.FileUtils;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.mrunit.mapreduce.MapReduceDriver;
+import org.apache.hadoop.mrunit.types.Pair;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+import com.kylinolap.common.util.LocalFileMetadataTestCase;
+import com.kylinolap.job.constant.BatchConstants;
+
+/**
+ * @author George Song (ysong1)
+ * 
+ */
+public class NDCuboidMapperTest extends LocalFileMetadataTestCase {
+    MapReduceDriver<Text, Text, Text, Text, Text, Text> mapReduceDriver;
+    String localTempDir = System.getProperty("java.io.tmpdir") + File.separator;
+
+    @Before
+    public void setUp() throws Exception {
+        createTestMetadata();
+
+        // hack for distributed cache
+        FileUtils.deleteDirectory(new File("../job/meta"));
+        FileUtils.copyDirectory(new File(this.getTestConfig().getMetadataUrl()), new File("../job/meta"));
+
+        NDCuboidMapper mapper = new NDCuboidMapper();
+        CuboidReducer reducer = new CuboidReducer();
+        mapReduceDriver = MapReduceDriver.newMapReduceDriver(mapper, reducer);
+    }
+
+    @After
+    public void after() throws Exception {
+        cleanupTestMetadata();
+        FileUtils.deleteDirectory(new File("../job/meta"));
+    }
+
+    @Test
+    public void testMapReduceWithSlr() throws IOException {
+
+        String cubeName = "test_kylin_cube_with_slr_1_new_segment";
+        String segmentName = "20130331080000_20131212080000";
+        mapReduceDriver.getConfiguration().set(BatchConstants.CFG_CUBE_NAME, cubeName);
+        mapReduceDriver.getConfiguration().set(BatchConstants.CFG_CUBE_SEGMENT_NAME, segmentName);
+
+        byte[] key = { 0, 0, 0, 0, 0, 0, 1, -1, 49, 48, 48, 48, 48, 48, 48, 48, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 11, 54, -105, 55, 13, 71, 114, 65, 66, 73, 78, 9, 9, 9, 9, 9, 9, 9, 9, 0, 10, 0 };
+        byte[] value = { 14, 7, 23, -16, 56, 92, 114, -80, 118, 14, 7, 23, -16, 56, 92, 114, -80, 118, 14, 7, 23, -16, 56, 92, 114, -80, 118, 1 };
+        Pair<Text, Text> input1 = new Pair<Text, Text>(new Text(key), new Text(value));
+
+        mapReduceDriver.addInput(input1);
+
+        List<Pair<Text, Text>> result = mapReduceDriver.run();
+
+        assertEquals(4, result.size());
+
+        byte[] resultKey = { 0, 0, 0, 0, 0, 0, 1, 127, 49, 48, 48, 48, 48, 48, 48, 48, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 55, 13, 71, 114, 65, 66, 73, 78, 9, 9, 9, 9, 9, 9, 9, 9, 0, 10, 0 };
+        byte[] resultValue = { 14, 7, 23, -16, 56, 92, 114, -80, 118, 14, 7, 23, -16, 56, 92, 114, -80, 118, 14, 7, 23, -16, 56, 92, 114, -80, 118, 1 };
+        Pair<Text, Text> output1 = new Pair<Text, Text>(new Text(resultKey), new Text(resultValue));
+
+        assertTrue(result.contains(output1));
+
+        long[] keySet = new long[result.size()];
+
+        System.out.println(Bytes.toLong(new byte[] { 0, 0, 0, 0, 0, 0, 1, -1 }));
+        for (int i = 0; i < result.size(); i++) {
+            byte[] bytes = new byte[result.get(i).getFirst().getLength()];
+            System.arraycopy(result.get(i).getFirst().getBytes(), 0, bytes, 0, result.get(i).getFirst().getLength());
+            System.out.println(Bytes.toLong(bytes));
+            keySet[i] = Bytes.toLong(bytes);
+        }
+
+        // refer to CuboidSchedulerTest.testGetSpanningCuboid()
+        assertArrayEquals(new long[] { 383, 447, 503, 504 }, keySet);
+
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/job/src/test/java/com/kylinolap/job/hadoop/cube/RandomKeyDistributionMapperTest.java
----------------------------------------------------------------------
diff --git a/job/src/test/java/com/kylinolap/job/hadoop/cube/RandomKeyDistributionMapperTest.java b/job/src/test/java/com/kylinolap/job/hadoop/cube/RandomKeyDistributionMapperTest.java
new file mode 100644
index 0000000..9cb1788
--- /dev/null
+++ b/job/src/test/java/com/kylinolap/job/hadoop/cube/RandomKeyDistributionMapperTest.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.job.hadoop.cube;
+
+import static org.junit.Assert.*;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.mrunit.mapreduce.MapDriver;
+import org.apache.hadoop.mrunit.types.Pair;
+import org.junit.Before;
+import org.junit.Test;
+
+import com.kylinolap.job.constant.BatchConstants;
+import com.kylinolap.job.hadoop.invertedindex.RandomKeyDistributionMapper;
+
+/**
+ * @author ysong1
+ * 
+ */
+public class RandomKeyDistributionMapperTest {
+
+    MapDriver<Text, Text, Text, LongWritable> mapDriver;
+
+    @Before
+    @SuppressWarnings({ "rawtypes", "unchecked" })
+    public void setUp() {
+        RandomKeyDistributionMapper mapper = new RandomKeyDistributionMapper();
+        mapDriver = MapDriver.newMapDriver(mapper);
+    }
+
+    @Test
+    public void test() throws IOException {
+        List<Text> data = new ArrayList<Text>();
+        for (int i = 0; i < 1001; i++) {
+            data.add(new Text(String.valueOf(i)));
+        }
+
+        for (Text t : data) {
+            mapDriver.addInput(t, new Text("abc"));
+        }
+
+        mapDriver.getConfiguration().set(BatchConstants.MAPPER_SAMPLE_NUMBER, "100");
+        List<Pair<Text, LongWritable>> result = mapDriver.run();
+        assertEquals(100, result.size());
+
+        for (Pair<Text, LongWritable> p : result) {
+            System.out.println(p.getFirst());
+        }
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/job/src/test/java/com/kylinolap/job/hadoop/cube/RandomKeyDistributionReducerTest.java
----------------------------------------------------------------------
diff --git a/job/src/test/java/com/kylinolap/job/hadoop/cube/RandomKeyDistributionReducerTest.java b/job/src/test/java/com/kylinolap/job/hadoop/cube/RandomKeyDistributionReducerTest.java
new file mode 100644
index 0000000..2f18834
--- /dev/null
+++ b/job/src/test/java/com/kylinolap/job/hadoop/cube/RandomKeyDistributionReducerTest.java
@@ -0,0 +1,69 @@
+/*
+ * 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.job.hadoop.cube;
+
+import static org.junit.Assert.*;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.mrunit.mapreduce.ReduceDriver;
+import org.apache.hadoop.mrunit.types.Pair;
+import org.junit.Before;
+import org.junit.Test;
+
+import com.kylinolap.job.constant.BatchConstants;
+import com.kylinolap.job.hadoop.invertedindex.RandomKeyDistributionReducer;
+
+/**
+ * @author ysong1
+ * 
+ */
+public class RandomKeyDistributionReducerTest {
+    ReduceDriver<Text, LongWritable, Text, LongWritable> reduceDriver;
+
+    @Before
+    @SuppressWarnings({ "rawtypes", "unchecked" })
+    public void setUp() {
+        RandomKeyDistributionReducer reducer = new RandomKeyDistributionReducer();
+        reduceDriver = ReduceDriver.newReduceDriver(reducer);
+    }
+
+    @Test
+    public void test() throws IOException {
+        List<Text> data = new ArrayList<Text>();
+        for (int i = 0; i < 1001; i++) {
+            data.add(new Text(String.valueOf(i)));
+        }
+        for (Text t : data) {
+            reduceDriver.addInput(t, new ArrayList<LongWritable>());
+        }
+
+        reduceDriver.getConfiguration().set(BatchConstants.REGION_NUMBER, "2");
+        List<Pair<Text, LongWritable>> result = reduceDriver.run();
+
+        assertEquals(2, result.size());
+
+        for (Pair<Text, LongWritable> p : result) {
+            System.out.println(p.getFirst());
+        }
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/job/src/test/java/com/kylinolap/job/hadoop/cube/RangeKeyDistributionJobTest.java
----------------------------------------------------------------------
diff --git a/job/src/test/java/com/kylinolap/job/hadoop/cube/RangeKeyDistributionJobTest.java b/job/src/test/java/com/kylinolap/job/hadoop/cube/RangeKeyDistributionJobTest.java
new file mode 100644
index 0000000..3ad3124
--- /dev/null
+++ b/job/src/test/java/com/kylinolap/job/hadoop/cube/RangeKeyDistributionJobTest.java
@@ -0,0 +1,69 @@
+/*
+ * 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.job.hadoop.cube;
+
+import static org.junit.Assert.*;
+
+import java.io.File;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileUtil;
+import org.apache.hadoop.util.ToolRunner;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+import com.kylinolap.common.util.LocalFileMetadataTestCase;
+
+/**
+ * @author ysong1
+ * 
+ */
+public class RangeKeyDistributionJobTest extends LocalFileMetadataTestCase {
+
+    private Configuration conf;
+
+    @Before
+    public void setup() throws Exception {
+        conf = new Configuration();
+        // conf.set("fs.default.name", "file:///");
+        // conf.set("mapred.job.tracker", "local");
+
+        // for local runner out-of-memory issue
+        conf.set("mapreduce.task.io.sort.mb", "10");
+        createTestMetadata();
+    }
+
+    @After
+    public void after() throws Exception {
+        cleanupTestMetadata();
+    }
+
+    @Test
+    public void testJob() throws Exception {
+        String input = "src/test/resources/data/base_cuboid/,src/test/resources/data/6d_cuboid/";
+        String output = "target/test-output/key_distribution_range/";
+        String jobname = "calculate_splits";
+        String cubename = "test_kylin_cube_with_slr_ready";
+
+        FileUtil.fullyDelete(new File(output));
+
+        String[] args = { "-input", input, "-output", output, "-jobname", jobname, "-cubename", cubename };
+        assertEquals("Job failed", 0, ToolRunner.run(conf, new RangeKeyDistributionJob(), args));
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/job/src/test/java/com/kylinolap/job/hadoop/cube/RangeKeyDistributionMapperTest.java
----------------------------------------------------------------------
diff --git a/job/src/test/java/com/kylinolap/job/hadoop/cube/RangeKeyDistributionMapperTest.java b/job/src/test/java/com/kylinolap/job/hadoop/cube/RangeKeyDistributionMapperTest.java
new file mode 100644
index 0000000..f8a5ff7
--- /dev/null
+++ b/job/src/test/java/com/kylinolap/job/hadoop/cube/RangeKeyDistributionMapperTest.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.job.hadoop.cube;
+
+import static org.junit.Assert.*;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.List;
+
+import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.mrunit.mapreduce.MapDriver;
+import org.apache.hadoop.mrunit.types.Pair;
+import org.junit.Before;
+import org.junit.Test;
+
+/**
+ * @author ysong1
+ * 
+ */
+public class RangeKeyDistributionMapperTest {
+
+    @SuppressWarnings("rawtypes")
+    MapDriver mapDriver;
+    String localTempDir = System.getProperty("java.io.tmpdir") + File.separator;
+
+    @Before
+    public void setUp() {
+        RangeKeyDistributionMapper mapper = new RangeKeyDistributionMapper();
+        mapDriver = MapDriver.newMapDriver(mapper);
+    }
+
+    @SuppressWarnings("unchecked")
+    @Test
+    public void testMapperWithoutHeader() throws IOException {
+
+        Text inputKey1 = new Text(new byte[] { 0, 0, 0, 0, 0, 0, 0, 127, 11, 56, -23, 0, 22, 98, 1, 0, 121, 7 });
+        Text inputKey2 = new Text(new byte[] { 0, 0, 0, 0, 0, 0, 0, 127, 11, 122, 1, 0, 22, 98, 1, 0, 121, 7 });
+        Text inputKey3 = new Text(new byte[] { 2, 2, 2, 2, 2, 2, 2, 127, 11, 56, -23, 0, 22, 98, 1, 0, 121, 7 });
+        Text inputKey4 = new Text(new byte[] { 3, 3, 3, 3, 3, 3, 3, 127, 11, 56, -23, 0, 22, 98, 1, 0, 121, 7 });
+        Text inputKey5 = new Text(new byte[] { 4, 4, 4, 4, 4, 4, 4, 127, 11, 56, -23, 0, 22, 98, 1, 0, 121, 7 });
+        Text inputKey6 = new Text(new byte[] { 5, 5, 5, 5, 5, 5, 5, 127, 11, 56, -23, 0, 22, 98, 1, 0, 121, 7 });
+        Text inputKey7 = new Text(new byte[] { 6, 6, 6, 6, 6, 6, 6, 127, 11, 56, -23, 0, 22, 98, 1, 0, 121, 7 });
+
+        mapDriver.addInput(inputKey1, new Text("abc"));
+        mapDriver.addInput(inputKey2, new Text("abc"));
+        mapDriver.addInput(inputKey3, new Text("abc"));
+        mapDriver.addInput(inputKey4, new Text("abc"));
+        mapDriver.addInput(inputKey5, new Text("abc"));
+        mapDriver.addInput(inputKey6, new Text("abc"));
+        mapDriver.addInput(inputKey7, new Text("abc"));
+
+        List<Pair<Text, LongWritable>> result = mapDriver.run();
+
+        assertEquals(1, result.size());
+
+        byte[] key1 = result.get(0).getFirst().getBytes();
+        LongWritable value1 = result.get(0).getSecond();
+        assertArrayEquals(new byte[] { 6, 6, 6, 6, 6, 6, 6, 127, 11, 56, -23, 0, 22, 98, 1, 0, 121, 7 }, key1);
+        assertEquals(147, value1.get());
+    }
+
+    @SuppressWarnings("unchecked")
+    @Test
+    public void testMapperWithHeader() throws IOException {
+
+        Text inputKey1 = new Text(new byte[] { 0, 0, 0, 0, 0, 0, 0, 127, 11, 56, -23, 0, 22, 98, 1, 0, 121, 7, 0, 0, 0, 0, 0, 0, 0, 127, 11, 56, -23, 0, 22, 98, 1, 0, 121, 7 });
+        Text inputKey2 = new Text(new byte[] { 0, 0, 0, 0, 0, 0, 0, 127, 11, 56, -23, 0, 22, 98, 1, 0, 121, 7, 0, 0, 0, 0, 0, 0, 0, 127, 11, 122, 1, 0, 22, 98, 1, 0, 121, 7 });
+        Text inputKey3 = new Text(new byte[] { 0, 0, 0, 0, 0, 0, 0, 127, 11, 56, -23, 0, 22, 98, 1, 0, 121, 7, 2, 2, 2, 2, 2, 2, 2, 127, 11, 56, -23, 0, 22, 98, 1, 0, 121, 7 });
+        Text inputKey4 = new Text(new byte[] { 0, 0, 0, 0, 0, 0, 0, 127, 11, 56, -23, 0, 22, 98, 1, 0, 121, 7, 3, 3, 3, 3, 3, 3, 3, 127, 11, 56, -23, 0, 22, 98, 1, 0, 121, 7 });
+        Text inputKey5 = new Text(new byte[] { 0, 0, 0, 0, 0, 0, 0, 127, 11, 56, -23, 0, 22, 98, 1, 0, 121, 7, 4, 4, 4, 4, 4, 4, 4, 127, 11, 56, -23, 0, 22, 98, 1, 0, 121, 7 });
+        Text inputKey6 = new Text(new byte[] { 0, 0, 0, 0, 0, 0, 0, 127, 11, 56, -23, 0, 22, 98, 1, 0, 121, 7, 5, 5, 5, 5, 5, 5, 5, 127, 11, 56, -23, 0, 22, 98, 1, 0, 121, 7 });
+        Text inputKey7 = new Text(new byte[] { 0, 0, 0, 0, 0, 0, 0, 127, 11, 56, -23, 0, 22, 98, 1, 0, 121, 7, 6, 6, 6, 6, 6, 6, 6, 127, 11, 56, -23, 0, 22, 98, 1, 0, 121, 7 });
+
+        mapDriver.addInput(inputKey1, new Text("abc"));
+        mapDriver.addInput(inputKey2, new Text("abc"));
+        mapDriver.addInput(inputKey3, new Text("abc"));
+        mapDriver.addInput(inputKey4, new Text("abc"));
+        mapDriver.addInput(inputKey5, new Text("abc"));
+        mapDriver.addInput(inputKey6, new Text("abc"));
+        mapDriver.addInput(inputKey7, new Text("abc"));
+
+        List<Pair<Text, LongWritable>> result = mapDriver.run();
+
+        assertEquals(1, result.size());
+
+        byte[] key1 = result.get(0).getFirst().getBytes();
+        LongWritable value1 = result.get(0).getSecond();
+        assertArrayEquals(new byte[] { 0, 0, 0, 0, 0, 0, 0, 127, 11, 56, -23, 0, 22, 98, 1, 0, 121, 7, 6, 6, 6, 6, 6, 6, 6, 127, 11, 56, -23, 0, 22, 98, 1, 0, 121, 7 }, key1);
+        assertEquals(273, value1.get());
+
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/job/src/test/java/com/kylinolap/job/hadoop/cube/RangeKeyDistributionReducerTest.java
----------------------------------------------------------------------
diff --git a/job/src/test/java/com/kylinolap/job/hadoop/cube/RangeKeyDistributionReducerTest.java b/job/src/test/java/com/kylinolap/job/hadoop/cube/RangeKeyDistributionReducerTest.java
new file mode 100644
index 0000000..7e2d2ed
--- /dev/null
+++ b/job/src/test/java/com/kylinolap/job/hadoop/cube/RangeKeyDistributionReducerTest.java
@@ -0,0 +1,105 @@
+/*
+ * 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.job.hadoop.cube;
+
+import static org.junit.Assert.*;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.mrunit.mapreduce.ReduceDriver;
+import org.apache.hadoop.mrunit.types.Pair;
+import org.junit.Before;
+import org.junit.Test;
+
+import com.kylinolap.job.constant.BatchConstants;
+
+/**
+ * @author ysong1
+ * 
+ */
+public class RangeKeyDistributionReducerTest {
+
+    ReduceDriver<Text, LongWritable, Text, LongWritable> reduceDriver;
+    String localTempDir = System.getProperty("java.io.tmpdir") + File.separator;
+
+    @Before
+    public void setUp() {
+        RangeKeyDistributionReducer reducer = new RangeKeyDistributionReducer();
+        reduceDriver = ReduceDriver.newReduceDriver(reducer);
+    }
+
+    @Test
+    public void testReducer() throws IOException {
+        Text key1 = new Text(new byte[] { 1 });
+        List<LongWritable> values1 = new ArrayList<LongWritable>();
+        values1.add(new LongWritable(RangeKeyDistributionReducer.TEN_GIGA_BYTES));
+        values1.add(new LongWritable(1));
+
+        Text key2 = new Text(new byte[] { 2 });
+        List<LongWritable> values2 = new ArrayList<LongWritable>();
+        values2.add(new LongWritable(123));
+
+        Text key3 = new Text(new byte[] { 3 });
+        List<LongWritable> values3 = new ArrayList<LongWritable>();
+        values3.add(new LongWritable(RangeKeyDistributionReducer.TEN_GIGA_BYTES));
+
+        Text key4 = new Text(new byte[] { 4 });
+        List<LongWritable> values4 = new ArrayList<LongWritable>();
+        values4.add(new LongWritable(RangeKeyDistributionReducer.TEN_GIGA_BYTES));
+
+        Text key5 = new Text(new byte[] { 5 });
+        List<LongWritable> values5 = new ArrayList<LongWritable>();
+        values5.add(new LongWritable(1));
+
+        reduceDriver.withInput(key1, values1);
+        reduceDriver.withInput(key2, values2);
+        reduceDriver.withInput(key3, values3);
+        reduceDriver.withInput(key4, values4);
+        reduceDriver.withInput(key5, values5);
+
+        reduceDriver.getConfiguration().set(BatchConstants.CUBE_CAPACITY, "MEDIUM");
+
+        List<Pair<Text, LongWritable>> result = reduceDriver.run();
+
+        assertEquals(4, result.size());
+
+        byte[] outputKey1 = result.get(0).getFirst().getBytes();
+        LongWritable value1 = result.get(0).getSecond();
+        assertArrayEquals(new byte[] { 1 }, outputKey1);
+        assertEquals(10737418241L, value1.get());
+
+        byte[] outputKey2 = result.get(1).getFirst().getBytes();
+        LongWritable value2 = result.get(1).getSecond();
+        assertArrayEquals(new byte[] { 3 }, outputKey2);
+        assertEquals(10737418363L, value2.get());
+
+        byte[] outputKey3 = result.get(2).getFirst().getBytes();
+        LongWritable value3 = result.get(2).getSecond();
+        assertArrayEquals(new byte[] { 4 }, outputKey3);
+        assertEquals(10737418240L, value3.get());
+
+        byte[] outputKey4 = result.get(3).getFirst().getBytes();
+        LongWritable value4 = result.get(3).getSecond();
+        assertArrayEquals(new byte[] { 5 }, outputKey4);
+        assertEquals(1L, value4.get());
+    }
+}


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

Posted by lu...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/job/src/test/resources/jarfile/SampleBadJavaProgram.jarfile
----------------------------------------------------------------------
diff --git a/job/src/test/resources/jarfile/SampleBadJavaProgram.jarfile b/job/src/test/resources/jarfile/SampleBadJavaProgram.jarfile
new file mode 100644
index 0000000..75a43d5
Binary files /dev/null and b/job/src/test/resources/jarfile/SampleBadJavaProgram.jarfile differ

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/job/src/test/resources/jarfile/SampleJavaProgram.jarfile
----------------------------------------------------------------------
diff --git a/job/src/test/resources/jarfile/SampleJavaProgram.jarfile b/job/src/test/resources/jarfile/SampleJavaProgram.jarfile
new file mode 100644
index 0000000..8ca85c4
Binary files /dev/null and b/job/src/test/resources/jarfile/SampleJavaProgram.jarfile differ

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/job/src/test/resources/json/dummy_jobinstance.json
----------------------------------------------------------------------
diff --git a/job/src/test/resources/json/dummy_jobinstance.json b/job/src/test/resources/json/dummy_jobinstance.json
new file mode 100644
index 0000000..9789b03
--- /dev/null
+++ b/job/src/test/resources/json/dummy_jobinstance.json
@@ -0,0 +1,195 @@
+{
+  "name" : "Dummy_Job",
+  "type" : "REBUILD",
+  "uuid" : "8ad83b8c-6bda-4b79-864a-9566c0f8ce2c",
+  "last_modified" : 0,
+  "related_cube" : "test_kylin_cube_with_slr_empty",
+  "related_branch" : null,
+  "related_segment" : null,
+  "input_parameters" : {
+    "storageLocationIdentifier" : "table_abc"
+  },
+  "job_status" : "PENDING",
+  "exec_start_time" : 0,
+  "exec_end_time" : 0,
+  "progress" : 0.0,
+  "duration" : 0,
+  "mr_waiting" : 0,
+  "steps" : [ {
+    "interruptCmd" : null,
+    "name" : "Build Dimension Dictionary",
+    "sequence_id" : 0,
+    "exec_cmd" : "hbase org.apache.hadoop.util.RunJar kylin-job.jar com.kylinolap.job.hadoop.dict.CreateDictionaryJob  -cubename test_kylin_cube_with_slr_empty",
+    "pre_exec_cmd" : null,
+    "post_exec_cmd" : null,
+    "interrupt_cmd" : null,
+    "exec_start_time" : 0,
+    "exec_end_time" : 0,
+    "exec_wait_time" : 0,
+    "step_status" : "PENDING",
+    "cmd_type" : "MRCLI",
+    "cmd_output" : null,
+    "info" : null,
+    "run_async" : false
+  }, {
+    "interruptCmd" : null,
+    "name" : "Create Intermediate Flat Hive Table",
+    "sequence_id" : 1,
+    "exec_cmd" : "hive -e \"DROP TABLE IF EXISTS kylin_intermediate_test_kylin_cube_with_slr_desc_8ad83b8c_6bda_4b79_864a_9566c0f8ce2c;\nCREATE EXTERNAL TABLE IF NOT EXISTS kylin_intermediate_test_kylin_cube_with_slr_desc_8ad83b8c_6bda_4b79_864a_9566c0f8ce2c\n(\nCAL_DT date\n,META_CATEG_NAME string\n,CATEG_LVL2_NAME string\n,CATEG_LVL3_NAME string\n,LSTG_FORMAT_NAME string\n,SITE_ID int\n,SELLER_TYPE_CD smallint\n,SELLER_ID bigint\n,PRICE decimal\n)\nROW FORMAT DELIMITED FIELDS TERMINATED BY '\\177'\nSTORED AS SEQUENCEFILE\nLOCATION '/tmp/8ad83b8c-6bda-4b79-864a-9566c0f8ce2c/kylin_intermediate_test_kylin_cube_with_slr_desc_8ad83b8c_6bda_4b79_864a_9566c0f8ce2c';\nSET dfs.block.size=67108864;\nSET hive.exec.compress.output=true;\nSET mapred.output.compression.type=BLOCK;\nSET mapred.output.compression.codec=com.hadoop.compression.lzo.LzoCodec;\nSET hive.auto.convert.join.noconditionaltask = true;\nSET hive.auto.convert.join.noconditionaltask.size = 300000000;\nINSERT OVERWRITE TABLE k
 ylin_intermediate_test_kylin_cube_with_slr_desc_8ad83b8c_6bda_4b79_864a_9566c0f8ce2c\nSELECT\nTEST_CAL_DT.CAL_DT\n,TEST_CATEGORY_GROUPINGS.META_CATEG_NAME\n,TEST_CATEGORY_GROUPINGS.CATEG_LVL2_NAME\n,TEST_CATEGORY_GROUPINGS.CATEG_LVL3_NAME\n,TEST_KYLIN_FACT.LSTG_FORMAT_NAME\n,TEST_SITES.SITE_ID\n,TEST_SELLER_TYPE_DIM.SELLER_TYPE_CD\n,TEST_KYLIN_FACT.SELLER_ID\n,TEST_KYLIN_FACT.PRICE\nFROM TEST_KYLIN_FACT\nINNER JOIN TEST_CAL_DT\nON TEST_KYLIN_FACT.CAL_DT = TEST_CAL_DT.CAL_DT\nINNER JOIN TEST_CATEGORY_GROUPINGS\nON TEST_KYLIN_FACT.LEAF_CATEG_ID = TEST_CATEGORY_GROUPINGS.LEAF_CATEG_ID AND TEST_KYLIN_FACT.LSTG_SITE_ID = TEST_CATEGORY_GROUPINGS.SITE_ID\nINNER JOIN TEST_SITES\nON TEST_KYLIN_FACT.LSTG_SITE_ID = TEST_SITES.SITE_ID\nINNER JOIN TEST_SELLER_TYPE_DIM\nON TEST_KYLIN_FACT.SLR_SEGMENT_CD = TEST_SELLER_TYPE_DIM.SELLER_TYPE_CD\n;\n\"",
+    "pre_exec_cmd" : null,
+    "post_exec_cmd" : null,
+    "interrupt_cmd" : null,
+    "exec_start_time" : 0,
+    "exec_end_time" : 0,
+    "exec_wait_time" : 0,
+    "step_status" : "PENDING",
+    "cmd_type" : "MRCLI",
+    "cmd_output" : null,
+    "info" : null,
+    "run_async" : false
+  }, {
+    "interruptCmd" : null,
+    "name" : "Build Base Cuboid Data",
+    "sequence_id" : 2,
+    "exec_cmd" : "hbase org.apache.hadoop.util.RunJar kylin-job.jar com.kylinolap.job.hadoop.cube.BaseCuboidJob  -D mapred.compress.map.output=true -D mapred.map.output.compression.codec=com.hadoop.compression.lzo.LzoCodec -D mapred.output.compress=true -D mapred.output.compression.codec=com.hadoop.compression.lzo.LzoCodec -D mapred.output.compression.type=BLOCK -D dfs.block.size=67108864 -D ebay.alert.email=DL-eBay-Kylin@corp.ebay.com -D ebay.job.name=Kylin_Olap_Cube_Builder -cubename test_kylin_cube_with_slr_empty -input /tmp/8ad83b8c-6bda-4b79-864a-9566c0f8ce2c/kylin_intermediate_test_kylin_cube_with_slr_desc_8ad83b8c_6bda_4b79_864a_9566c0f8ce2c -output /tmp/8ad83b8c-6bda-4b79-864a-9566c0f8ce2c/test_kylin_cube_with_slr_empty/cuboid/base_cuboid -jobname Kylin_Base_Cuboid_Builder_test_kylin_cube_with_slr_empty_Step_2 -level 0",
+    "pre_exec_cmd" : null,
+    "post_exec_cmd" : null,
+    "interrupt_cmd" : null,
+    "exec_start_time" : 0,
+    "exec_end_time" : 0,
+    "exec_wait_time" : 0,
+    "step_status" : "PENDING",
+    "cmd_type" : "MRCLI",
+    "cmd_output" : null,
+    "info" : null,
+    "run_async" : false
+  }, {
+    "interruptCmd" : null,
+    "name" : "Build N-Dimension Cuboid Data : 7-Dimension",
+    "sequence_id" : 3,
+    "exec_cmd" : "hbase org.apache.hadoop.util.RunJar kylin-job.jar com.kylinolap.job.hadoop.cube.NDCuboidJob  -D mapred.compress.map.output=true -D mapred.map.output.compression.codec=com.hadoop.compression.lzo.LzoCodec -D mapred.output.compress=true -D mapred.output.compression.codec=com.hadoop.compression.lzo.LzoCodec -D mapred.output.compression.type=BLOCK -D dfs.block.size=67108864 -D ebay.alert.email=DL-eBay-Kylin@corp.ebay.com -D ebay.job.name=Kylin_Olap_Cube_Builder -cubename test_kylin_cube_with_slr_empty -input /tmp/8ad83b8c-6bda-4b79-864a-9566c0f8ce2c/test_kylin_cube_with_slr_empty/cuboid/base_cuboid -output /tmp/8ad83b8c-6bda-4b79-864a-9566c0f8ce2c/test_kylin_cube_with_slr_empty/cuboid/7d_cuboid -jobname Kylin_ND-Cuboid_Builder_test_kylin_cube_with_slr_empty_Step_3 -level 1",
+    "pre_exec_cmd" : null,
+    "post_exec_cmd" : null,
+    "interrupt_cmd" : null,
+    "exec_start_time" : 0,
+    "exec_end_time" : 0,
+    "exec_wait_time" : 0,
+    "step_status" : "PENDING",
+    "cmd_type" : "MRCLI",
+    "cmd_output" : null,
+    "info" : null,
+    "run_async" : false
+  }, {
+    "interruptCmd" : null,
+    "name" : "Build N-Dimension Cuboid Data : 6-Dimension",
+    "sequence_id" : 4,
+    "exec_cmd" : "hbase org.apache.hadoop.util.RunJar kylin-job.jar com.kylinolap.job.hadoop.cube.NDCuboidJob  -D mapred.compress.map.output=true -D mapred.map.output.compression.codec=com.hadoop.compression.lzo.LzoCodec -D mapred.output.compress=true -D mapred.output.compression.codec=com.hadoop.compression.lzo.LzoCodec -D mapred.output.compression.type=BLOCK -D dfs.block.size=67108864 -D ebay.alert.email=DL-eBay-Kylin@corp.ebay.com -D ebay.job.name=Kylin_Olap_Cube_Builder -cubename test_kylin_cube_with_slr_empty -input /tmp/8ad83b8c-6bda-4b79-864a-9566c0f8ce2c/test_kylin_cube_with_slr_empty/cuboid/7d_cuboid -output /tmp/8ad83b8c-6bda-4b79-864a-9566c0f8ce2c/test_kylin_cube_with_slr_empty/cuboid/6d_cuboid -jobname Kylin_ND-Cuboid_Builder_test_kylin_cube_with_slr_empty_Step_4 -level 2",
+    "pre_exec_cmd" : null,
+    "post_exec_cmd" : null,
+    "interrupt_cmd" : null,
+    "exec_start_time" : 0,
+    "exec_end_time" : 0,
+    "exec_wait_time" : 0,
+    "step_status" : "PENDING",
+    "cmd_type" : "MRCLI",
+    "cmd_output" : null,
+    "info" : null,
+    "run_async" : false
+  }, {
+    "interruptCmd" : null,
+    "name" : "Build N-Dimension Cuboid Data : 5-Dimension",
+    "sequence_id" : 5,
+    "exec_cmd" : "hbase org.apache.hadoop.util.RunJar kylin-job.jar com.kylinolap.job.hadoop.cube.NDCuboidJob  -D mapred.compress.map.output=true -D mapred.map.output.compression.codec=com.hadoop.compression.lzo.LzoCodec -D mapred.output.compress=true -D mapred.output.compression.codec=com.hadoop.compression.lzo.LzoCodec -D mapred.output.compression.type=BLOCK -D dfs.block.size=67108864 -D ebay.alert.email=DL-eBay-Kylin@corp.ebay.com -D ebay.job.name=Kylin_Olap_Cube_Builder -cubename test_kylin_cube_with_slr_empty -input /tmp/8ad83b8c-6bda-4b79-864a-9566c0f8ce2c/test_kylin_cube_with_slr_empty/cuboid/6d_cuboid -output /tmp/8ad83b8c-6bda-4b79-864a-9566c0f8ce2c/test_kylin_cube_with_slr_empty/cuboid/5d_cuboid -jobname Kylin_ND-Cuboid_Builder_test_kylin_cube_with_slr_empty_Step_5 -level 3",
+    "pre_exec_cmd" : null,
+    "post_exec_cmd" : null,
+    "interrupt_cmd" : null,
+    "exec_start_time" : 0,
+    "exec_end_time" : 0,
+    "exec_wait_time" : 0,
+    "step_status" : "PENDING",
+    "cmd_type" : "MRCLI",
+    "cmd_output" : null,
+    "info" : null,
+    "run_async" : false
+  }, {
+    "interruptCmd" : null,
+    "name" : "Build N-Dimension Cuboid Data : 4-Dimension",
+    "sequence_id" : 6,
+    "exec_cmd" : "hbase org.apache.hadoop.util.RunJar kylin-job.jar com.kylinolap.job.hadoop.cube.NDCuboidJob  -D mapred.compress.map.output=true -D mapred.map.output.compression.codec=com.hadoop.compression.lzo.LzoCodec -D mapred.output.compress=true -D mapred.output.compression.codec=com.hadoop.compression.lzo.LzoCodec -D mapred.output.compression.type=BLOCK -D dfs.block.size=67108864 -D ebay.alert.email=DL-eBay-Kylin@corp.ebay.com -D ebay.job.name=Kylin_Olap_Cube_Builder -cubename test_kylin_cube_with_slr_empty -input /tmp/8ad83b8c-6bda-4b79-864a-9566c0f8ce2c/test_kylin_cube_with_slr_empty/cuboid/5d_cuboid -output /tmp/8ad83b8c-6bda-4b79-864a-9566c0f8ce2c/test_kylin_cube_with_slr_empty/cuboid/4d_cuboid -jobname Kylin_ND-Cuboid_Builder_test_kylin_cube_with_slr_empty_Step_6 -level 4",
+    "pre_exec_cmd" : null,
+    "post_exec_cmd" : null,
+    "interrupt_cmd" : null,
+    "exec_start_time" : 0,
+    "exec_end_time" : 0,
+    "exec_wait_time" : 0,
+    "step_status" : "PENDING",
+    "cmd_type" : "MRCLI",
+    "cmd_output" : null,
+    "info" : null,
+    "run_async" : false
+  }, {
+    "interruptCmd" : null,
+    "name" : "Get Cuboid Data Key Distribution",
+    "sequence_id" : 7,
+    "exec_cmd" : "hbase org.apache.hadoop.util.RunJar kylin-job.jar com.kylinolap.job.hadoop.cube.RangeKeyDistributionJob  -D mapred.compress.map.output=true -D mapred.map.output.compression.codec=com.hadoop.compression.lzo.LzoCodec -D mapred.output.compress=true -D mapred.output.compression.codec=com.hadoop.compression.lzo.LzoCodec -D mapred.output.compression.type=BLOCK -D dfs.block.size=67108864 -D ebay.alert.email=DL-eBay-Kylin@corp.ebay.com -D ebay.job.name=Kylin_Olap_Cube_Builder -input /tmp/8ad83b8c-6bda-4b79-864a-9566c0f8ce2c/test_kylin_cube_with_slr_empty/cuboid/* -output /tmp/8ad83b8c-6bda-4b79-864a-9566c0f8ce2c/test_kylin_cube_with_slr_empty/rowkey_stats -jobname Kylin_Region_Splits_Calculator_test_kylin_cube_with_slr_empty_Step_7",
+    "pre_exec_cmd" : null,
+    "post_exec_cmd" : null,
+    "interrupt_cmd" : null,
+    "exec_start_time" : 0,
+    "exec_end_time" : 0,
+    "exec_wait_time" : 0,
+    "step_status" : "PENDING",
+    "cmd_type" : "MRCLI",
+    "cmd_output" : null,
+    "info" : null,
+    "run_async" : false
+  }, {
+    "interruptCmd" : null,
+    "name" : "Create HBase Table",
+    "sequence_id" : 8,
+    "exec_cmd" : "hbase org.apache.hadoop.util.RunJar kylin-job.jar com.kylinolap.job.hadoop.hbase.CreateHTableJob  -cubename test_kylin_cube_with_slr_empty -input /tmp/8ad83b8c-6bda-4b79-864a-9566c0f8ce2c/test_kylin_cube_with_slr_empty/rowkey_stats/part-r-00000 -htablename table_abc",
+    "pre_exec_cmd" : null,
+    "post_exec_cmd" : null,
+    "interrupt_cmd" : null,
+    "exec_start_time" : 0,
+    "exec_end_time" : 0,
+    "exec_wait_time" : 0,
+    "step_status" : "PENDING",
+    "cmd_type" : "MRCLI",
+    "cmd_output" : null,
+    "info" : null,
+    "run_async" : false
+  }, {
+    "interruptCmd" : null,
+    "name" : "Convert Cuboid Data to HFile",
+    "sequence_id" : 9,
+    "exec_cmd" : "hbase org.apache.hadoop.util.RunJar kylin-job.jar com.kylinolap.job.hadoop.cube.CubeHFileJob  -D mapred.compress.map.output=true -D mapred.map.output.compression.codec=com.hadoop.compression.lzo.LzoCodec -D mapred.output.compress=true -D mapred.output.compression.codec=com.hadoop.compression.lzo.LzoCodec -D mapred.output.compression.type=BLOCK -D dfs.block.size=67108864 -D ebay.alert.email=DL-eBay-Kylin@corp.ebay.com -D ebay.job.name=Kylin_Olap_Cube_Builder -cubename test_kylin_cube_with_slr_empty -input /tmp/8ad83b8c-6bda-4b79-864a-9566c0f8ce2c/test_kylin_cube_with_slr_empty/cuboid/* -output /tmp/8ad83b8c-6bda-4b79-864a-9566c0f8ce2c/test_kylin_cube_with_slr_empty/hfile -htablename table_abc -jobname Kylin_HFile_Generator_test_kylin_cube_with_slr_empty_Step_9",
+    "pre_exec_cmd" : null,
+    "post_exec_cmd" : null,
+    "interrupt_cmd" : null,
+    "exec_start_time" : 0,
+    "exec_end_time" : 0,
+    "exec_wait_time" : 0,
+    "step_status" : "PENDING",
+    "cmd_type" : "MRCLI",
+    "cmd_output" : null,
+    "info" : null,
+    "run_async" : false
+  }, {
+    "interruptCmd" : null,
+    "name" : "Load HFile to HBase Table",
+    "sequence_id" : 10,
+    "exec_cmd" : "hadoop fs -chmod 777 /tmp/8ad83b8c-6bda-4b79-864a-9566c0f8ce2c/test_kylin_cube_with_slr_empty/hfile/*;hbase org.apache.hadoop.hbase.mapreduce.LoadIncrementalHFiles /tmp/8ad83b8c-6bda-4b79-864a-9566c0f8ce2c/test_kylin_cube_with_slr_empty/hfile/ table_abc",
+    "pre_exec_cmd" : null,
+    "post_exec_cmd" : null,
+    "interrupt_cmd" : null,
+    "exec_start_time" : 0,
+    "exec_end_time" : 0,
+    "exec_wait_time" : 0,
+    "step_status" : "PENDING",
+    "cmd_type" : "MRCLI",
+    "cmd_output" : null,
+    "info" : null,
+    "run_async" : false
+  } ]
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/job/src/test/resources/partition_list/_partition.lst
----------------------------------------------------------------------
diff --git a/job/src/test/resources/partition_list/_partition.lst b/job/src/test/resources/partition_list/_partition.lst
new file mode 100644
index 0000000..64147d4
Binary files /dev/null and b/job/src/test/resources/partition_list/_partition.lst differ

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/job/src/test/resources/partition_list/part-r-00000
----------------------------------------------------------------------
diff --git a/job/src/test/resources/partition_list/part-r-00000 b/job/src/test/resources/partition_list/part-r-00000
new file mode 100644
index 0000000..07f1110
Binary files /dev/null and b/job/src/test/resources/partition_list/part-r-00000 differ

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/kylin_architecture.png
----------------------------------------------------------------------
diff --git a/kylin_architecture.png b/kylin_architecture.png
new file mode 100644
index 0000000..404a79f
Binary files /dev/null and b/kylin_architecture.png differ

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/metadata/.settings/org.eclipse.core.resources.prefs
----------------------------------------------------------------------
diff --git a/metadata/.settings/org.eclipse.core.resources.prefs b/metadata/.settings/org.eclipse.core.resources.prefs
new file mode 100644
index 0000000..04cfa2c
--- /dev/null
+++ b/metadata/.settings/org.eclipse.core.resources.prefs
@@ -0,0 +1,6 @@
+eclipse.preferences.version=1
+encoding//src/main/java=UTF-8
+encoding//src/main/resources=UTF-8
+encoding//src/test/java=UTF-8
+encoding//src/test/resources=UTF-8
+encoding/<project>=UTF-8

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/metadata/.settings/org.eclipse.jdt.core.prefs
----------------------------------------------------------------------
diff --git a/metadata/.settings/org.eclipse.jdt.core.prefs b/metadata/.settings/org.eclipse.jdt.core.prefs
new file mode 100644
index 0000000..a903301
--- /dev/null
+++ b/metadata/.settings/org.eclipse.jdt.core.prefs
@@ -0,0 +1,379 @@
+eclipse.preferences.version=1
+org.eclipse.jdt.core.compiler.annotation.inheritNullAnnotations=disabled
+org.eclipse.jdt.core.compiler.annotation.missingNonNullByDefaultAnnotation=ignore
+org.eclipse.jdt.core.compiler.annotation.nonnull=org.eclipse.jdt.annotation.NonNull
+org.eclipse.jdt.core.compiler.annotation.nonnullbydefault=org.eclipse.jdt.annotation.NonNullByDefault
+org.eclipse.jdt.core.compiler.annotation.nullable=org.eclipse.jdt.annotation.Nullable
+org.eclipse.jdt.core.compiler.annotation.nullanalysis=disabled
+org.eclipse.jdt.core.compiler.codegen.inlineJsrBytecode=enabled
+org.eclipse.jdt.core.compiler.codegen.targetPlatform=1.7
+org.eclipse.jdt.core.compiler.codegen.unusedLocal=preserve
+org.eclipse.jdt.core.compiler.compliance=1.7
+org.eclipse.jdt.core.compiler.debug.lineNumber=generate
+org.eclipse.jdt.core.compiler.debug.localVariable=generate
+org.eclipse.jdt.core.compiler.debug.sourceFile=generate
+org.eclipse.jdt.core.compiler.problem.annotationSuperInterface=warning
+org.eclipse.jdt.core.compiler.problem.assertIdentifier=error
+org.eclipse.jdt.core.compiler.problem.autoboxing=ignore
+org.eclipse.jdt.core.compiler.problem.comparingIdentical=warning
+org.eclipse.jdt.core.compiler.problem.deadCode=warning
+org.eclipse.jdt.core.compiler.problem.deprecation=warning
+org.eclipse.jdt.core.compiler.problem.deprecationInDeprecatedCode=disabled
+org.eclipse.jdt.core.compiler.problem.deprecationWhenOverridingDeprecatedMethod=disabled
+org.eclipse.jdt.core.compiler.problem.discouragedReference=warning
+org.eclipse.jdt.core.compiler.problem.emptyStatement=ignore
+org.eclipse.jdt.core.compiler.problem.enumIdentifier=error
+org.eclipse.jdt.core.compiler.problem.explicitlyClosedAutoCloseable=ignore
+org.eclipse.jdt.core.compiler.problem.fallthroughCase=ignore
+org.eclipse.jdt.core.compiler.problem.fatalOptionalError=disabled
+org.eclipse.jdt.core.compiler.problem.fieldHiding=ignore
+org.eclipse.jdt.core.compiler.problem.finalParameterBound=warning
+org.eclipse.jdt.core.compiler.problem.finallyBlockNotCompletingNormally=warning
+org.eclipse.jdt.core.compiler.problem.forbiddenReference=warning
+org.eclipse.jdt.core.compiler.problem.hiddenCatchBlock=warning
+org.eclipse.jdt.core.compiler.problem.includeNullInfoFromAsserts=disabled
+org.eclipse.jdt.core.compiler.problem.incompatibleNonInheritedInterfaceMethod=warning
+org.eclipse.jdt.core.compiler.problem.incompleteEnumSwitch=warning
+org.eclipse.jdt.core.compiler.problem.indirectStaticAccess=ignore
+org.eclipse.jdt.core.compiler.problem.localVariableHiding=ignore
+org.eclipse.jdt.core.compiler.problem.methodWithConstructorName=warning
+org.eclipse.jdt.core.compiler.problem.missingDefaultCase=ignore
+org.eclipse.jdt.core.compiler.problem.missingDeprecatedAnnotation=ignore
+org.eclipse.jdt.core.compiler.problem.missingEnumCaseDespiteDefault=disabled
+org.eclipse.jdt.core.compiler.problem.missingHashCodeMethod=ignore
+org.eclipse.jdt.core.compiler.problem.missingOverrideAnnotation=ignore
+org.eclipse.jdt.core.compiler.problem.missingOverrideAnnotationForInterfaceMethodImplementation=enabled
+org.eclipse.jdt.core.compiler.problem.missingSerialVersion=warning
+org.eclipse.jdt.core.compiler.problem.missingSynchronizedOnInheritedMethod=ignore
+org.eclipse.jdt.core.compiler.problem.noEffectAssignment=warning
+org.eclipse.jdt.core.compiler.problem.noImplicitStringConversion=warning
+org.eclipse.jdt.core.compiler.problem.nonExternalizedStringLiteral=ignore
+org.eclipse.jdt.core.compiler.problem.nonnullParameterAnnotationDropped=warning
+org.eclipse.jdt.core.compiler.problem.nullAnnotationInferenceConflict=error
+org.eclipse.jdt.core.compiler.problem.nullReference=warning
+org.eclipse.jdt.core.compiler.problem.nullSpecViolation=error
+org.eclipse.jdt.core.compiler.problem.nullUncheckedConversion=warning
+org.eclipse.jdt.core.compiler.problem.overridingPackageDefaultMethod=warning
+org.eclipse.jdt.core.compiler.problem.parameterAssignment=ignore
+org.eclipse.jdt.core.compiler.problem.possibleAccidentalBooleanAssignment=ignore
+org.eclipse.jdt.core.compiler.problem.potentialNullReference=ignore
+org.eclipse.jdt.core.compiler.problem.potentiallyUnclosedCloseable=ignore
+org.eclipse.jdt.core.compiler.problem.rawTypeReference=warning
+org.eclipse.jdt.core.compiler.problem.redundantNullAnnotation=warning
+org.eclipse.jdt.core.compiler.problem.redundantNullCheck=ignore
+org.eclipse.jdt.core.compiler.problem.redundantSpecificationOfTypeArguments=ignore
+org.eclipse.jdt.core.compiler.problem.redundantSuperinterface=ignore
+org.eclipse.jdt.core.compiler.problem.reportMethodCanBePotentiallyStatic=ignore
+org.eclipse.jdt.core.compiler.problem.reportMethodCanBeStatic=ignore
+org.eclipse.jdt.core.compiler.problem.specialParameterHidingField=disabled
+org.eclipse.jdt.core.compiler.problem.staticAccessReceiver=warning
+org.eclipse.jdt.core.compiler.problem.suppressOptionalErrors=disabled
+org.eclipse.jdt.core.compiler.problem.suppressWarnings=enabled
+org.eclipse.jdt.core.compiler.problem.syntacticNullAnalysisForFields=disabled
+org.eclipse.jdt.core.compiler.problem.syntheticAccessEmulation=ignore
+org.eclipse.jdt.core.compiler.problem.typeParameterHiding=warning
+org.eclipse.jdt.core.compiler.problem.unavoidableGenericTypeProblems=enabled
+org.eclipse.jdt.core.compiler.problem.uncheckedTypeOperation=warning
+org.eclipse.jdt.core.compiler.problem.unclosedCloseable=warning
+org.eclipse.jdt.core.compiler.problem.undocumentedEmptyBlock=ignore
+org.eclipse.jdt.core.compiler.problem.unhandledWarningToken=warning
+org.eclipse.jdt.core.compiler.problem.unnecessaryElse=ignore
+org.eclipse.jdt.core.compiler.problem.unnecessaryTypeCheck=ignore
+org.eclipse.jdt.core.compiler.problem.unqualifiedFieldAccess=ignore
+org.eclipse.jdt.core.compiler.problem.unusedDeclaredThrownException=ignore
+org.eclipse.jdt.core.compiler.problem.unusedDeclaredThrownExceptionExemptExceptionAndThrowable=enabled
+org.eclipse.jdt.core.compiler.problem.unusedDeclaredThrownExceptionIncludeDocCommentReference=enabled
+org.eclipse.jdt.core.compiler.problem.unusedDeclaredThrownExceptionWhenOverriding=disabled
+org.eclipse.jdt.core.compiler.problem.unusedImport=warning
+org.eclipse.jdt.core.compiler.problem.unusedLabel=warning
+org.eclipse.jdt.core.compiler.problem.unusedLocal=warning
+org.eclipse.jdt.core.compiler.problem.unusedObjectAllocation=ignore
+org.eclipse.jdt.core.compiler.problem.unusedParameter=ignore
+org.eclipse.jdt.core.compiler.problem.unusedParameterIncludeDocCommentReference=enabled
+org.eclipse.jdt.core.compiler.problem.unusedParameterWhenImplementingAbstract=disabled
+org.eclipse.jdt.core.compiler.problem.unusedParameterWhenOverridingConcrete=disabled
+org.eclipse.jdt.core.compiler.problem.unusedPrivateMember=warning
+org.eclipse.jdt.core.compiler.problem.unusedTypeParameter=ignore
+org.eclipse.jdt.core.compiler.problem.unusedWarningToken=warning
+org.eclipse.jdt.core.compiler.problem.varargsArgumentNeedCast=warning
+org.eclipse.jdt.core.compiler.source=1.7
+org.eclipse.jdt.core.formatter.align_type_members_on_columns=false
+org.eclipse.jdt.core.formatter.alignment_for_arguments_in_allocation_expression=16
+org.eclipse.jdt.core.formatter.alignment_for_arguments_in_annotation=0
+org.eclipse.jdt.core.formatter.alignment_for_arguments_in_enum_constant=16
+org.eclipse.jdt.core.formatter.alignment_for_arguments_in_explicit_constructor_call=16
+org.eclipse.jdt.core.formatter.alignment_for_arguments_in_method_invocation=16
+org.eclipse.jdt.core.formatter.alignment_for_arguments_in_qualified_allocation_expression=16
+org.eclipse.jdt.core.formatter.alignment_for_assignment=0
+org.eclipse.jdt.core.formatter.alignment_for_binary_expression=16
+org.eclipse.jdt.core.formatter.alignment_for_compact_if=16
+org.eclipse.jdt.core.formatter.alignment_for_conditional_expression=80
+org.eclipse.jdt.core.formatter.alignment_for_enum_constants=0
+org.eclipse.jdt.core.formatter.alignment_for_expressions_in_array_initializer=16
+org.eclipse.jdt.core.formatter.alignment_for_method_declaration=0
+org.eclipse.jdt.core.formatter.alignment_for_multiple_fields=16
+org.eclipse.jdt.core.formatter.alignment_for_parameters_in_constructor_declaration=16
+org.eclipse.jdt.core.formatter.alignment_for_parameters_in_method_declaration=16
+org.eclipse.jdt.core.formatter.alignment_for_resources_in_try=80
+org.eclipse.jdt.core.formatter.alignment_for_selector_in_method_invocation=16
+org.eclipse.jdt.core.formatter.alignment_for_superclass_in_type_declaration=16
+org.eclipse.jdt.core.formatter.alignment_for_superinterfaces_in_enum_declaration=16
+org.eclipse.jdt.core.formatter.alignment_for_superinterfaces_in_type_declaration=16
+org.eclipse.jdt.core.formatter.alignment_for_throws_clause_in_constructor_declaration=16
+org.eclipse.jdt.core.formatter.alignment_for_throws_clause_in_method_declaration=16
+org.eclipse.jdt.core.formatter.alignment_for_union_type_in_multicatch=16
+org.eclipse.jdt.core.formatter.blank_lines_after_imports=1
+org.eclipse.jdt.core.formatter.blank_lines_after_package=1
+org.eclipse.jdt.core.formatter.blank_lines_before_field=0
+org.eclipse.jdt.core.formatter.blank_lines_before_first_class_body_declaration=0
+org.eclipse.jdt.core.formatter.blank_lines_before_imports=1
+org.eclipse.jdt.core.formatter.blank_lines_before_member_type=1
+org.eclipse.jdt.core.formatter.blank_lines_before_method=1
+org.eclipse.jdt.core.formatter.blank_lines_before_new_chunk=1
+org.eclipse.jdt.core.formatter.blank_lines_before_package=0
+org.eclipse.jdt.core.formatter.blank_lines_between_import_groups=1
+org.eclipse.jdt.core.formatter.blank_lines_between_type_declarations=1
+org.eclipse.jdt.core.formatter.brace_position_for_annotation_type_declaration=end_of_line
+org.eclipse.jdt.core.formatter.brace_position_for_anonymous_type_declaration=end_of_line
+org.eclipse.jdt.core.formatter.brace_position_for_array_initializer=end_of_line
+org.eclipse.jdt.core.formatter.brace_position_for_block=end_of_line
+org.eclipse.jdt.core.formatter.brace_position_for_block_in_case=end_of_line
+org.eclipse.jdt.core.formatter.brace_position_for_constructor_declaration=end_of_line
+org.eclipse.jdt.core.formatter.brace_position_for_enum_constant=end_of_line
+org.eclipse.jdt.core.formatter.brace_position_for_enum_declaration=end_of_line
+org.eclipse.jdt.core.formatter.brace_position_for_method_declaration=end_of_line
+org.eclipse.jdt.core.formatter.brace_position_for_switch=end_of_line
+org.eclipse.jdt.core.formatter.brace_position_for_type_declaration=end_of_line
+org.eclipse.jdt.core.formatter.comment.clear_blank_lines_in_block_comment=false
+org.eclipse.jdt.core.formatter.comment.clear_blank_lines_in_javadoc_comment=false
+org.eclipse.jdt.core.formatter.comment.format_block_comments=false
+org.eclipse.jdt.core.formatter.comment.format_header=false
+org.eclipse.jdt.core.formatter.comment.format_html=true
+org.eclipse.jdt.core.formatter.comment.format_javadoc_comments=false
+org.eclipse.jdt.core.formatter.comment.format_line_comments=false
+org.eclipse.jdt.core.formatter.comment.format_source_code=true
+org.eclipse.jdt.core.formatter.comment.indent_parameter_description=true
+org.eclipse.jdt.core.formatter.comment.indent_root_tags=true
+org.eclipse.jdt.core.formatter.comment.insert_new_line_before_root_tags=insert
+org.eclipse.jdt.core.formatter.comment.insert_new_line_for_parameter=insert
+org.eclipse.jdt.core.formatter.comment.line_length=80
+org.eclipse.jdt.core.formatter.comment.new_lines_at_block_boundaries=true
+org.eclipse.jdt.core.formatter.comment.new_lines_at_javadoc_boundaries=true
+org.eclipse.jdt.core.formatter.comment.preserve_white_space_between_code_and_line_comments=false
+org.eclipse.jdt.core.formatter.compact_else_if=true
+org.eclipse.jdt.core.formatter.continuation_indentation=2
+org.eclipse.jdt.core.formatter.continuation_indentation_for_array_initializer=2
+org.eclipse.jdt.core.formatter.disabling_tag=@formatter\:off
+org.eclipse.jdt.core.formatter.enabling_tag=@formatter\:on
+org.eclipse.jdt.core.formatter.format_guardian_clause_on_one_line=false
+org.eclipse.jdt.core.formatter.format_line_comment_starting_on_first_column=true
+org.eclipse.jdt.core.formatter.indent_body_declarations_compare_to_annotation_declaration_header=true
+org.eclipse.jdt.core.formatter.indent_body_declarations_compare_to_enum_constant_header=true
+org.eclipse.jdt.core.formatter.indent_body_declarations_compare_to_enum_declaration_header=true
+org.eclipse.jdt.core.formatter.indent_body_declarations_compare_to_type_header=true
+org.eclipse.jdt.core.formatter.indent_breaks_compare_to_cases=true
+org.eclipse.jdt.core.formatter.indent_empty_lines=false
+org.eclipse.jdt.core.formatter.indent_statements_compare_to_block=true
+org.eclipse.jdt.core.formatter.indent_statements_compare_to_body=true
+org.eclipse.jdt.core.formatter.indent_switchstatements_compare_to_cases=true
+org.eclipse.jdt.core.formatter.indent_switchstatements_compare_to_switch=false
+org.eclipse.jdt.core.formatter.indentation.size=4
+org.eclipse.jdt.core.formatter.insert_new_line_after_annotation_on_field=insert
+org.eclipse.jdt.core.formatter.insert_new_line_after_annotation_on_local_variable=insert
+org.eclipse.jdt.core.formatter.insert_new_line_after_annotation_on_method=insert
+org.eclipse.jdt.core.formatter.insert_new_line_after_annotation_on_package=insert
+org.eclipse.jdt.core.formatter.insert_new_line_after_annotation_on_parameter=do not insert
+org.eclipse.jdt.core.formatter.insert_new_line_after_annotation_on_type=insert
+org.eclipse.jdt.core.formatter.insert_new_line_after_label=do not insert
+org.eclipse.jdt.core.formatter.insert_new_line_after_opening_brace_in_array_initializer=do not insert
+org.eclipse.jdt.core.formatter.insert_new_line_at_end_of_file_if_missing=do not insert
+org.eclipse.jdt.core.formatter.insert_new_line_before_catch_in_try_statement=do not insert
+org.eclipse.jdt.core.formatter.insert_new_line_before_closing_brace_in_array_initializer=do not insert
+org.eclipse.jdt.core.formatter.insert_new_line_before_else_in_if_statement=do not insert
+org.eclipse.jdt.core.formatter.insert_new_line_before_finally_in_try_statement=do not insert
+org.eclipse.jdt.core.formatter.insert_new_line_before_while_in_do_statement=do not insert
+org.eclipse.jdt.core.formatter.insert_new_line_in_empty_annotation_declaration=insert
+org.eclipse.jdt.core.formatter.insert_new_line_in_empty_anonymous_type_declaration=insert
+org.eclipse.jdt.core.formatter.insert_new_line_in_empty_block=insert
+org.eclipse.jdt.core.formatter.insert_new_line_in_empty_enum_constant=insert
+org.eclipse.jdt.core.formatter.insert_new_line_in_empty_enum_declaration=insert
+org.eclipse.jdt.core.formatter.insert_new_line_in_empty_method_body=insert
+org.eclipse.jdt.core.formatter.insert_new_line_in_empty_type_declaration=insert
+org.eclipse.jdt.core.formatter.insert_space_after_and_in_type_parameter=insert
+org.eclipse.jdt.core.formatter.insert_space_after_assignment_operator=insert
+org.eclipse.jdt.core.formatter.insert_space_after_at_in_annotation=do not insert
+org.eclipse.jdt.core.formatter.insert_space_after_at_in_annotation_type_declaration=do not insert
+org.eclipse.jdt.core.formatter.insert_space_after_binary_operator=insert
+org.eclipse.jdt.core.formatter.insert_space_after_closing_angle_bracket_in_type_arguments=insert
+org.eclipse.jdt.core.formatter.insert_space_after_closing_angle_bracket_in_type_parameters=insert
+org.eclipse.jdt.core.formatter.insert_space_after_closing_brace_in_block=insert
+org.eclipse.jdt.core.formatter.insert_space_after_closing_paren_in_cast=insert
+org.eclipse.jdt.core.formatter.insert_space_after_colon_in_assert=insert
+org.eclipse.jdt.core.formatter.insert_space_after_colon_in_case=insert
+org.eclipse.jdt.core.formatter.insert_space_after_colon_in_conditional=insert
+org.eclipse.jdt.core.formatter.insert_space_after_colon_in_for=insert
+org.eclipse.jdt.core.formatter.insert_space_after_colon_in_labeled_statement=insert
+org.eclipse.jdt.core.formatter.insert_space_after_comma_in_allocation_expression=insert
+org.eclipse.jdt.core.formatter.insert_space_after_comma_in_annotation=insert
+org.eclipse.jdt.core.formatter.insert_space_after_comma_in_array_initializer=insert
+org.eclipse.jdt.core.formatter.insert_space_after_comma_in_constructor_declaration_parameters=insert
+org.eclipse.jdt.core.formatter.insert_space_after_comma_in_constructor_declaration_throws=insert
+org.eclipse.jdt.core.formatter.insert_space_after_comma_in_enum_constant_arguments=insert
+org.eclipse.jdt.core.formatter.insert_space_after_comma_in_enum_declarations=insert
+org.eclipse.jdt.core.formatter.insert_space_after_comma_in_explicitconstructorcall_arguments=insert
+org.eclipse.jdt.core.formatter.insert_space_after_comma_in_for_increments=insert
+org.eclipse.jdt.core.formatter.insert_space_after_comma_in_for_inits=insert
+org.eclipse.jdt.core.formatter.insert_space_after_comma_in_method_declaration_parameters=insert
+org.eclipse.jdt.core.formatter.insert_space_after_comma_in_method_declaration_throws=insert
+org.eclipse.jdt.core.formatter.insert_space_after_comma_in_method_invocation_arguments=insert
+org.eclipse.jdt.core.formatter.insert_space_after_comma_in_multiple_field_declarations=insert
+org.eclipse.jdt.core.formatter.insert_space_after_comma_in_multiple_local_declarations=insert
+org.eclipse.jdt.core.formatter.insert_space_after_comma_in_parameterized_type_reference=insert
+org.eclipse.jdt.core.formatter.insert_space_after_comma_in_superinterfaces=insert
+org.eclipse.jdt.core.formatter.insert_space_after_comma_in_type_arguments=insert
+org.eclipse.jdt.core.formatter.insert_space_after_comma_in_type_parameters=insert
+org.eclipse.jdt.core.formatter.insert_space_after_ellipsis=insert
+org.eclipse.jdt.core.formatter.insert_space_after_opening_angle_bracket_in_parameterized_type_reference=do not insert
+org.eclipse.jdt.core.formatter.insert_space_after_opening_angle_bracket_in_type_arguments=do not insert
+org.eclipse.jdt.core.formatter.insert_space_after_opening_angle_bracket_in_type_parameters=do not insert
+org.eclipse.jdt.core.formatter.insert_space_after_opening_brace_in_array_initializer=insert
+org.eclipse.jdt.core.formatter.insert_space_after_opening_bracket_in_array_allocation_expression=do not insert
+org.eclipse.jdt.core.formatter.insert_space_after_opening_bracket_in_array_reference=do not insert
+org.eclipse.jdt.core.formatter.insert_space_after_opening_paren_in_annotation=do not insert
+org.eclipse.jdt.core.formatter.insert_space_after_opening_paren_in_cast=do not insert
+org.eclipse.jdt.core.formatter.insert_space_after_opening_paren_in_catch=do not insert
+org.eclipse.jdt.core.formatter.insert_space_after_opening_paren_in_constructor_declaration=do not insert
+org.eclipse.jdt.core.formatter.insert_space_after_opening_paren_in_enum_constant=do not insert
+org.eclipse.jdt.core.formatter.insert_space_after_opening_paren_in_for=do not insert
+org.eclipse.jdt.core.formatter.insert_space_after_opening_paren_in_if=do not insert
+org.eclipse.jdt.core.formatter.insert_space_after_opening_paren_in_method_declaration=do not insert
+org.eclipse.jdt.core.formatter.insert_space_after_opening_paren_in_method_invocation=do not insert
+org.eclipse.jdt.core.formatter.insert_space_after_opening_paren_in_parenthesized_expression=do not insert
+org.eclipse.jdt.core.formatter.insert_space_after_opening_paren_in_switch=do not insert
+org.eclipse.jdt.core.formatter.insert_space_after_opening_paren_in_synchronized=do not insert
+org.eclipse.jdt.core.formatter.insert_space_after_opening_paren_in_try=do not insert
+org.eclipse.jdt.core.formatter.insert_space_after_opening_paren_in_while=do not insert
+org.eclipse.jdt.core.formatter.insert_space_after_postfix_operator=do not insert
+org.eclipse.jdt.core.formatter.insert_space_after_prefix_operator=do not insert
+org.eclipse.jdt.core.formatter.insert_space_after_question_in_conditional=insert
+org.eclipse.jdt.core.formatter.insert_space_after_question_in_wildcard=do not insert
+org.eclipse.jdt.core.formatter.insert_space_after_semicolon_in_for=insert
+org.eclipse.jdt.core.formatter.insert_space_after_semicolon_in_try_resources=insert
+org.eclipse.jdt.core.formatter.insert_space_after_unary_operator=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_and_in_type_parameter=insert
+org.eclipse.jdt.core.formatter.insert_space_before_assignment_operator=insert
+org.eclipse.jdt.core.formatter.insert_space_before_at_in_annotation_type_declaration=insert
+org.eclipse.jdt.core.formatter.insert_space_before_binary_operator=insert
+org.eclipse.jdt.core.formatter.insert_space_before_closing_angle_bracket_in_parameterized_type_reference=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_closing_angle_bracket_in_type_arguments=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_closing_angle_bracket_in_type_parameters=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_closing_brace_in_array_initializer=insert
+org.eclipse.jdt.core.formatter.insert_space_before_closing_bracket_in_array_allocation_expression=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_closing_bracket_in_array_reference=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_closing_paren_in_annotation=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_closing_paren_in_cast=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_closing_paren_in_catch=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_closing_paren_in_constructor_declaration=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_closing_paren_in_enum_constant=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_closing_paren_in_for=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_closing_paren_in_if=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_closing_paren_in_method_declaration=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_closing_paren_in_method_invocation=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_closing_paren_in_parenthesized_expression=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_closing_paren_in_switch=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_closing_paren_in_synchronized=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_closing_paren_in_try=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_closing_paren_in_while=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_colon_in_assert=insert
+org.eclipse.jdt.core.formatter.insert_space_before_colon_in_case=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_colon_in_conditional=insert
+org.eclipse.jdt.core.formatter.insert_space_before_colon_in_default=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_colon_in_for=insert
+org.eclipse.jdt.core.formatter.insert_space_before_colon_in_labeled_statement=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_comma_in_allocation_expression=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_comma_in_annotation=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_comma_in_array_initializer=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_comma_in_constructor_declaration_parameters=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_comma_in_constructor_declaration_throws=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_comma_in_enum_constant_arguments=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_comma_in_enum_declarations=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_comma_in_explicitconstructorcall_arguments=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_comma_in_for_increments=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_comma_in_for_inits=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_comma_in_method_declaration_parameters=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_comma_in_method_declaration_throws=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_comma_in_method_invocation_arguments=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_comma_in_multiple_field_declarations=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_comma_in_multiple_local_declarations=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_comma_in_parameterized_type_reference=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_comma_in_superinterfaces=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_comma_in_type_arguments=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_comma_in_type_parameters=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_ellipsis=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_opening_angle_bracket_in_parameterized_type_reference=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_opening_angle_bracket_in_type_arguments=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_opening_angle_bracket_in_type_parameters=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_opening_brace_in_annotation_type_declaration=insert
+org.eclipse.jdt.core.formatter.insert_space_before_opening_brace_in_anonymous_type_declaration=insert
+org.eclipse.jdt.core.formatter.insert_space_before_opening_brace_in_array_initializer=insert
+org.eclipse.jdt.core.formatter.insert_space_before_opening_brace_in_block=insert
+org.eclipse.jdt.core.formatter.insert_space_before_opening_brace_in_constructor_declaration=insert
+org.eclipse.jdt.core.formatter.insert_space_before_opening_brace_in_enum_constant=insert
+org.eclipse.jdt.core.formatter.insert_space_before_opening_brace_in_enum_declaration=insert
+org.eclipse.jdt.core.formatter.insert_space_before_opening_brace_in_method_declaration=insert
+org.eclipse.jdt.core.formatter.insert_space_before_opening_brace_in_switch=insert
+org.eclipse.jdt.core.formatter.insert_space_before_opening_brace_in_type_declaration=insert
+org.eclipse.jdt.core.formatter.insert_space_before_opening_bracket_in_array_allocation_expression=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_opening_bracket_in_array_reference=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_opening_bracket_in_array_type_reference=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_opening_paren_in_annotation=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_opening_paren_in_annotation_type_member_declaration=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_opening_paren_in_catch=insert
+org.eclipse.jdt.core.formatter.insert_space_before_opening_paren_in_constructor_declaration=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_opening_paren_in_enum_constant=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_opening_paren_in_for=insert
+org.eclipse.jdt.core.formatter.insert_space_before_opening_paren_in_if=insert
+org.eclipse.jdt.core.formatter.insert_space_before_opening_paren_in_method_declaration=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_opening_paren_in_method_invocation=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_opening_paren_in_parenthesized_expression=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_opening_paren_in_switch=insert
+org.eclipse.jdt.core.formatter.insert_space_before_opening_paren_in_synchronized=insert
+org.eclipse.jdt.core.formatter.insert_space_before_opening_paren_in_try=insert
+org.eclipse.jdt.core.formatter.insert_space_before_opening_paren_in_while=insert
+org.eclipse.jdt.core.formatter.insert_space_before_parenthesized_expression_in_return=insert
+org.eclipse.jdt.core.formatter.insert_space_before_parenthesized_expression_in_throw=insert
+org.eclipse.jdt.core.formatter.insert_space_before_postfix_operator=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_prefix_operator=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_question_in_conditional=insert
+org.eclipse.jdt.core.formatter.insert_space_before_question_in_wildcard=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_semicolon=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_semicolon_in_for=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_semicolon_in_try_resources=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_unary_operator=do not insert
+org.eclipse.jdt.core.formatter.insert_space_between_brackets_in_array_type_reference=do not insert
+org.eclipse.jdt.core.formatter.insert_space_between_empty_braces_in_array_initializer=do not insert
+org.eclipse.jdt.core.formatter.insert_space_between_empty_brackets_in_array_allocation_expression=do not insert
+org.eclipse.jdt.core.formatter.insert_space_between_empty_parens_in_annotation_type_member_declaration=do not insert
+org.eclipse.jdt.core.formatter.insert_space_between_empty_parens_in_constructor_declaration=do not insert
+org.eclipse.jdt.core.formatter.insert_space_between_empty_parens_in_enum_constant=do not insert
+org.eclipse.jdt.core.formatter.insert_space_between_empty_parens_in_method_declaration=do not insert
+org.eclipse.jdt.core.formatter.insert_space_between_empty_parens_in_method_invocation=do not insert
+org.eclipse.jdt.core.formatter.join_lines_in_comments=true
+org.eclipse.jdt.core.formatter.join_wrapped_lines=true
+org.eclipse.jdt.core.formatter.keep_else_statement_on_same_line=false
+org.eclipse.jdt.core.formatter.keep_empty_array_initializer_on_one_line=false
+org.eclipse.jdt.core.formatter.keep_imple_if_on_one_line=false
+org.eclipse.jdt.core.formatter.keep_then_statement_on_same_line=false
+org.eclipse.jdt.core.formatter.lineSplit=999
+org.eclipse.jdt.core.formatter.never_indent_block_comments_on_first_column=false
+org.eclipse.jdt.core.formatter.never_indent_line_comments_on_first_column=false
+org.eclipse.jdt.core.formatter.number_of_blank_lines_at_beginning_of_method_body=0
+org.eclipse.jdt.core.formatter.number_of_empty_lines_to_preserve=1
+org.eclipse.jdt.core.formatter.put_empty_statement_on_new_line=true
+org.eclipse.jdt.core.formatter.tabulation.char=space
+org.eclipse.jdt.core.formatter.tabulation.size=4
+org.eclipse.jdt.core.formatter.use_on_off_tags=false
+org.eclipse.jdt.core.formatter.use_tabs_only_for_leading_indentations=false
+org.eclipse.jdt.core.formatter.wrap_before_binary_operator=true
+org.eclipse.jdt.core.formatter.wrap_before_or_operator_multicatch=true
+org.eclipse.jdt.core.formatter.wrap_outer_expressions_when_nested=true

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/metadata/.settings/org.eclipse.jdt.ui.prefs
----------------------------------------------------------------------
diff --git a/metadata/.settings/org.eclipse.jdt.ui.prefs b/metadata/.settings/org.eclipse.jdt.ui.prefs
new file mode 100644
index 0000000..dece0e6
--- /dev/null
+++ b/metadata/.settings/org.eclipse.jdt.ui.prefs
@@ -0,0 +1,7 @@
+eclipse.preferences.version=1
+formatter_profile=_Space Indent & Long Lines
+formatter_settings_version=12
+org.eclipse.jdt.ui.ignorelowercasenames=true
+org.eclipse.jdt.ui.importorder=java;javax;org;com;
+org.eclipse.jdt.ui.ondemandthreshold=99
+org.eclipse.jdt.ui.staticondemandthreshold=1

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/metadata/pom.xml
----------------------------------------------------------------------
diff --git a/metadata/pom.xml b/metadata/pom.xml
new file mode 100644
index 0000000..3e76619
--- /dev/null
+++ b/metadata/pom.xml
@@ -0,0 +1,108 @@
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+    <modelVersion>4.0.0</modelVersion>
+
+    <artifactId>kylin-metadata</artifactId>
+    <packaging>jar</packaging>
+    <name>Kylin:Metadata</name>
+
+    <parent>
+        <groupId>com.kylinolap</groupId>
+        <artifactId>kylin</artifactId>
+        <version>0.6.3-SNAPSHOT</version>
+    </parent>
+
+    <properties>
+    </properties>
+
+    <dependencies>
+        <dependency>
+            <groupId>com.kylinolap</groupId>
+            <artifactId>kylin-common</artifactId>
+            <version>${project.parent.version}</version>
+        </dependency>
+
+        <dependency>
+            <groupId>com.fasterxml.jackson.core</groupId>
+            <artifactId>jackson-databind</artifactId>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.commons</groupId>
+            <artifactId>commons-lang3</artifactId>
+        </dependency>
+        <dependency>
+            <groupId>commons-io</groupId>
+            <artifactId>commons-io</artifactId>
+        </dependency>
+        <dependency>
+            <groupId>commons-configuration</groupId>
+            <artifactId>commons-configuration</artifactId>
+        </dependency>
+        <dependency>
+            <groupId>com.google.guava</groupId>
+            <artifactId>guava</artifactId>
+        </dependency>
+
+        <!-- Env & Test -->
+
+        <dependency>
+            <groupId>org.apache.hadoop</groupId>
+            <artifactId>hadoop-common</artifactId>
+            <scope>provided</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.hadoop</groupId>
+            <artifactId>hadoop-hdfs</artifactId>
+            <scope>provided</scope>
+            <!-- protobuf version conflict with hbase-->
+            <exclusions>
+                <exclusion>
+                    <groupId>com.google.protobuf</groupId>
+                    <artifactId>protobuf-java</artifactId>
+                </exclusion>
+            </exclusions>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.hbase</groupId>
+            <artifactId>hbase-common</artifactId>
+            <scope>provided</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.hbase</groupId>
+            <artifactId>hbase-client</artifactId>
+            <scope>provided</scope>
+        </dependency>
+        <dependency>
+            <groupId>junit</groupId>
+            <artifactId>junit</artifactId>
+            <scope>test</scope>
+        </dependency>
+    </dependencies>
+
+    <!--
+        <build>
+            <plugins>
+                <plugin>
+                    <artifactId>maven-assembly-plugin</artifactId>
+                    <configuration>
+                        <appendAssemblyId>true</appendAssemblyId>
+                        <descriptors>
+                            <descriptor>
+                                src/main/assembly/assemble.xml
+                            </descriptor>
+                        </descriptors>
+
+                    </configuration>
+                    <executions>
+                        <execution>
+                            <id>make-assembly</id>
+                            <phase>package</phase>
+                            <goals>
+                                <goal>single</goal>
+                            </goals>
+                        </execution>
+                    </executions>
+                </plugin>
+            </plugins>
+        </build>
+     -->
+</project>

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/metadata/src/main/assembly/assemble.xml
----------------------------------------------------------------------
diff --git a/metadata/src/main/assembly/assemble.xml b/metadata/src/main/assembly/assemble.xml
new file mode 100644
index 0000000..0a3c323
--- /dev/null
+++ b/metadata/src/main/assembly/assemble.xml
@@ -0,0 +1,21 @@
+<assembly xmlns="http://maven.apache.org/plugins/maven-assembly-plugin/assembly/1.1.2"
+          xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+          xsi:schemaLocation="http://maven.apache.org/plugins/maven-assembly-plugin/assembly/1.1.2 http://maven.apache.org/xsd/assembly-1.1.2.xsd">
+    <id>jar-with-dependencies</id>
+    <formats>
+        <format>jar</format>
+    </formats>
+    <includeBaseDirectory>false</includeBaseDirectory>
+    <dependencySets>
+        <dependencySet>
+            <outputDirectory>/</outputDirectory>
+            <useProjectArtifact>true</useProjectArtifact>
+            <unpack>true</unpack>
+            <scope>runtime</scope>
+            <excludes>
+                <!--<exclude>com.google.guava:guava</exclude>
+                <exclude>org.codehaus.jackson:*</exclude>      -->
+            </excludes>
+        </dependencySet>
+    </dependencySets>
+</assembly>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/metadata/src/main/java/com/kylinolap/metadata/MetadataConstances.java
----------------------------------------------------------------------
diff --git a/metadata/src/main/java/com/kylinolap/metadata/MetadataConstances.java b/metadata/src/main/java/com/kylinolap/metadata/MetadataConstances.java
new file mode 100644
index 0000000..82dc2ec
--- /dev/null
+++ b/metadata/src/main/java/com/kylinolap/metadata/MetadataConstances.java
@@ -0,0 +1,52 @@
+/*
+ * 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.metadata;
+
+/**
+ * Constances to describe metadata and it's change.
+ * 
+ * @author jianliu
+ * 
+ */
+public interface MetadataConstances {
+
+    public static final String FILE_SURFIX = ".json";
+
+    // Extended attribute keys
+    public static final String TABLE_EXD_STATUS_KEY = "EXD_STATUS";
+    public static final String TABLE_EXD_MINFS = "minFileSize";
+    public static final String TABLE_EXD_TNF = "totalNumberFiles";
+    public static final String TABLE_EXD_LOCATION = "location";
+    public static final String TABLE_EXD_LUT = "lastUpdateTime";
+    public static final String TABLE_EXD_LAT = "lastAccessTime";
+    public static final String TABLE_EXD_COLUMN = "columns";
+    public static final String TABLE_EXD_PC = "partitionColumns";
+    public static final String TABLE_EXD_MAXFS = "maxFileSize";
+    public static final String TABLE_EXD_IF = "inputformat";
+    public static final String TABLE_EXD_PARTITIONED = "partitioned";
+    public static final String TABLE_EXD_TABLENAME = "tableName";
+    public static final String TABLE_EXD_OWNER = "owner";
+    public static final String TABLE_EXD_TFS = "totalFileSize";
+    public static final String TABLE_EXD_OF = "outputformat";
+    /**
+     * The value is an array
+     */
+    public static final String TABLE_EXD_CARDINALITY = "cardinality";
+    public static final String TABLE_EXD_DELIM = "delim";
+    public static final String TABLE_EXD_DEFAULT_VALUE = "unknown";
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/metadata/src/main/java/com/kylinolap/metadata/MetadataManager.java
----------------------------------------------------------------------
diff --git a/metadata/src/main/java/com/kylinolap/metadata/MetadataManager.java b/metadata/src/main/java/com/kylinolap/metadata/MetadataManager.java
new file mode 100644
index 0000000..8ce978f
--- /dev/null
+++ b/metadata/src/main/java/com/kylinolap/metadata/MetadataManager.java
@@ -0,0 +1,482 @@
+/*
+ * 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.metadata;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.concurrent.ConcurrentHashMap;
+
+import org.apache.commons.lang.StringUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.fasterxml.jackson.core.type.TypeReference;
+import com.google.common.collect.Lists;
+import com.kylinolap.common.KylinConfig;
+import com.kylinolap.common.persistence.JsonSerializer;
+import com.kylinolap.common.persistence.ResourceStore;
+import com.kylinolap.common.persistence.Serializer;
+import com.kylinolap.common.restclient.Broadcaster;
+import com.kylinolap.common.restclient.SingleValueCache;
+import com.kylinolap.common.util.JsonUtil;
+import com.kylinolap.metadata.model.cube.CubeDesc;
+import com.kylinolap.metadata.model.invertedindex.InvertedIndexDesc;
+import com.kylinolap.metadata.model.schema.TableDesc;
+import com.kylinolap.metadata.validation.CubeMetadataValidator;
+import com.kylinolap.metadata.validation.ValidateContext;
+
+/**
+ * Serves (and caches) cube metadata for Kylin instance.
+ * <p/>
+ * Also provides a ResourceStore for general purpose data persistence. Cube
+ * metadata is serialized as JSON and stored in ResourceStore.
+ * 
+ * @author yangli9
+ */
+public class MetadataManager {
+
+    private static final Logger logger = LoggerFactory.getLogger(MetadataManager.class);
+
+    private static final Serializer<CubeDesc> CUBE_SERIALIZER = new JsonSerializer<CubeDesc>(CubeDesc.class);
+    private static final Serializer<TableDesc> TABLE_SERIALIZER = new JsonSerializer<TableDesc>(TableDesc.class);
+    private static final Serializer<InvertedIndexDesc> IIDESC_SERIALIZER = new JsonSerializer<InvertedIndexDesc>(InvertedIndexDesc.class);
+
+    TypeReference<HashMap<String, Object>> typeRef = new TypeReference<HashMap<String, Object>>() {
+    };
+
+    // static cached instances
+    private static final ConcurrentHashMap<KylinConfig, MetadataManager> CACHE = new ConcurrentHashMap<KylinConfig, MetadataManager>();
+
+    public static MetadataManager getInstance(KylinConfig config) {
+        MetadataManager r = CACHE.get(config);
+        if (r != null) {
+            return r;
+        }
+
+        synchronized (MetadataManager.class) {
+            r = CACHE.get(config);
+            if (r != null) {
+                return r;
+            }
+            try {
+                r = new MetadataManager(config);
+                CACHE.put(config, r);
+                if (CACHE.size() > 1) {
+                    logger.warn("More than one singleton exist");
+                }
+
+                return r;
+            } catch (IOException e) {
+                throw new IllegalStateException("Failed to init CubeManager from " + config, e);
+            }
+        }
+    }
+
+    public static synchronized void removeInstance(KylinConfig config) {
+        CACHE.remove(config);
+    }
+
+    public static void dropCache() {
+        CACHE.clear();
+    }
+
+    // ============================================================================
+
+    private KylinConfig config;
+    // table name ==> SourceTable
+    private SingleValueCache<String, TableDesc> srcTableMap = new SingleValueCache<String, TableDesc>(Broadcaster.TYPE.METADATA);
+    // name ==> CubeDesc
+    private SingleValueCache<String, CubeDesc> cubeDescMap = new SingleValueCache<String, CubeDesc>(Broadcaster.TYPE.METADATA);
+    // name ==> InvertedIndexDesc
+    private SingleValueCache<String, InvertedIndexDesc> iiDescMap = new SingleValueCache<String, InvertedIndexDesc>(Broadcaster.TYPE.METADATA);
+    // name => value
+    private SingleValueCache<String, Map<String, String>> srcTableExdMap = new SingleValueCache<String, Map<String, String>>(Broadcaster.TYPE.METADATA);
+
+    private MetadataManager(KylinConfig config) throws IOException {
+        init(config);
+    }
+
+    public KylinConfig getConfig() {
+        return config;
+    }
+
+    public ResourceStore getStore() {
+        return ResourceStore.getStore(this.config);
+    }
+
+    public List<TableDesc> listAllTables() {
+        return Lists.newArrayList(srcTableMap.values());
+    }
+
+    public Map<String, Map<String, String>> listAllTableExdMap() {
+        return srcTableExdMap.getMap();
+    }
+
+    /**
+     * Get Table Desc object
+     * 
+     * @param tableName
+     * @return
+     */
+    public TableDesc getTableDesc(String tableName) {
+        tableName = tableName.toUpperCase();
+        return srcTableMap.get(tableName);
+    }
+
+    /**
+     * Get table extended info. Keys are defined in {@link MetadataConstances}
+     * 
+     * @param tableName
+     * @return
+     */
+    public Map<String, String> getTableDescExd(String tableName) {
+        tableName = tableName.toUpperCase();
+        Map<String, String> result = new HashMap<String, String>();
+        if (srcTableExdMap.containsKey(tableName)) {
+            Map<String, String> tmp = srcTableExdMap.get(tableName);
+            Iterator<Entry<String, String>> it = tmp.entrySet().iterator();
+            while (it.hasNext()) {
+                Entry<String, String> entry = it.next();
+                result.put(entry.getKey(), entry.getValue());
+            }
+            result.put(MetadataConstances.TABLE_EXD_STATUS_KEY, "true");
+        } else {
+            result.put(MetadataConstances.TABLE_EXD_STATUS_KEY, "false");
+        }
+        return result;
+    }
+
+    public void createSourceTable(TableDesc srcTable) throws IOException {
+        if (srcTable.getUuid() == null || srcTable.getName() == null)
+            throw new IllegalArgumentException();
+        if (srcTableMap.containsKey(srcTable.getName()))
+            throw new IllegalArgumentException("SourceTable '" + srcTable.getName() + "' already exists");
+
+        String path = srcTable.getResourcePath();
+        getStore().putResource(path, srcTable, TABLE_SERIALIZER);
+
+        srcTableMap.put(srcTable.getName(), srcTable);
+    }
+
+    public InvertedIndexDesc getInvertedIndexDesc(String name) {
+        return iiDescMap.get(name);
+    }
+
+    public CubeDesc getCubeDesc(String name) {
+        return cubeDescMap.get(name);
+    }
+
+    /**
+     * Create a new CubeDesc
+     * 
+     * @param cubeDesc
+     * @return
+     * @throws IOException
+     */
+    public CubeDesc createCubeDesc(CubeDesc cubeDesc) throws IOException {
+        if (cubeDesc.getUuid() == null || cubeDesc.getName() == null)
+            throw new IllegalArgumentException();
+        if (cubeDescMap.containsKey(cubeDesc.getName()))
+            throw new IllegalArgumentException("CubeDesc '" + cubeDesc.getName() + "' already exists");
+
+        try {
+            cubeDesc.init(config, srcTableMap.getMap());
+        } catch (IllegalStateException e) {
+            cubeDesc.addError(e.getMessage(), true);
+        }
+        // Check base validation
+        if (!cubeDesc.getError().isEmpty()) {
+            return cubeDesc;
+        }
+        // Semantic validation
+        CubeMetadataValidator validator = new CubeMetadataValidator();
+        ValidateContext context = validator.validate(cubeDesc, true);
+        if (!context.ifPass()) {
+            return cubeDesc;
+        }
+
+        cubeDesc.setSignature(cubeDesc.calculateSignature());
+
+        String path = cubeDesc.getResourcePath();
+        getStore().putResource(path, cubeDesc, CUBE_SERIALIZER);
+        cubeDescMap.put(cubeDesc.getName(), cubeDesc);
+
+        return cubeDesc;
+    }
+    
+    // remove cubeDesc
+    public void removeCubeDesc(CubeDesc cubeDesc) throws IOException{
+        String path = cubeDesc.getResourcePath();
+        getStore().deleteResource(path);
+        cubeDescMap.remove(cubeDesc.getName());
+    }
+
+    // sync on update
+    private void init(KylinConfig config) throws IOException {
+        this.config = config;
+        reloadAllSourceTable();
+        reloadAllSourceTableExd();
+        reloadAllCubeDesc();
+        reloadAllInvertedIndexDesc();
+    }
+
+    private void reloadAllSourceTableExd() throws IOException {
+        ResourceStore store = getStore();
+        logger.debug("Reloading SourceTable exd info from folder " + store.getReadableResourcePath(ResourceStore.TABLE_EXD_RESOURCE_ROOT));
+
+        srcTableExdMap.clear();
+
+        List<String> paths = store.collectResourceRecursively(ResourceStore.TABLE_EXD_RESOURCE_ROOT, MetadataConstances.FILE_SURFIX);
+        for (String path : paths) {
+            Map<String, String> attrContainer = new HashMap<String, String>();
+            String tableName = loadSourceTableExd(getStore(), path, attrContainer);
+            srcTableExdMap.putLocal(tableName.toUpperCase(), attrContainer);
+        }
+        logger.debug("Loaded " + paths.size() + " SourceTable EXD(s)");
+    }
+
+    private void reloadAllSourceTable() throws IOException {
+        ResourceStore store = getStore();
+        logger.debug("Reloading SourceTable from folder " + store.getReadableResourcePath(ResourceStore.TABLE_RESOURCE_ROOT));
+
+        srcTableMap.clear();
+
+        List<String> paths = store.collectResourceRecursively(ResourceStore.TABLE_RESOURCE_ROOT, MetadataConstances.FILE_SURFIX);
+        for (String path : paths) {
+            loadSourceTable(path);
+        }
+
+        logger.debug("Loaded " + paths.size() + " SourceTable(s)");
+    }
+
+    @SuppressWarnings("unchecked")
+    /**
+     * return table name
+     */
+    public static String loadSourceTableExd(ResourceStore store, String path, Map<String, String> attrContainer) throws IOException {
+
+        logger.debug("Loading SourceTable exd " + path);
+        InputStream is = store.getResource(path);
+        if (is != null) {
+            attrContainer.putAll(JsonUtil.readValue(is, HashMap.class));
+            String file = path;
+            if (file.indexOf("/") > -1) {
+                file = file.substring(file.lastIndexOf("/") + 1);
+            }
+            return file.substring(0, file.length() - MetadataConstances.FILE_SURFIX.length());
+        } else {
+            logger.debug("Failed to get table exd info from " + path);
+            return null;
+        }
+    }
+
+    private TableDesc loadSourceTable(String path) throws IOException {
+        ResourceStore store = getStore();
+        logger.debug("Loading SourceTable " + store.getReadableResourcePath(path));
+
+        TableDesc t = store.getResource(path, TableDesc.class, TABLE_SERIALIZER);
+        t.init();
+
+        if (StringUtils.isBlank(t.getName()))
+            throw new IllegalStateException("SourceTable name must not be blank");
+        if (srcTableMap.containsKey(t.getName()))
+            throw new IllegalStateException("Dup SourceTable name '" + t.getName() + "'");
+
+        srcTableMap.putLocal(t.getName(), t);
+
+        return t;
+    }
+
+    private void reloadAllCubeDesc() throws IOException {
+        ResourceStore store = getStore();
+        logger.info("Reloading Cube Metadata from folder " + store.getReadableResourcePath(ResourceStore.CUBE_DESC_RESOURCE_ROOT));
+
+        cubeDescMap.clear();
+
+        List<String> paths = store.collectResourceRecursively(ResourceStore.CUBE_DESC_RESOURCE_ROOT, MetadataConstances.FILE_SURFIX);
+        for (String path : paths) {
+            CubeDesc desc;
+            try {
+                desc = loadCubeDesc(path);
+            } catch (Exception e) {
+                logger.error("Error loading cube desc " + path, e);
+                continue;
+            }
+            if (path.equals(desc.getResourcePath()) == false) {
+                logger.error("Skip suspicious desc at " + path + ", " + desc + " should be at " + desc.getResourcePath());
+                continue;
+            }
+            if (cubeDescMap.containsKey(desc.getName())) {
+                logger.error("Dup CubeDesc name '" + desc.getName() + "' on path " + path);
+                continue;
+            }
+
+            cubeDescMap.putLocal(desc.getName(), desc);
+        }
+
+        logger.debug("Loaded " + cubeDescMap.size() + " Cube(s)");
+    }
+
+    private CubeDesc loadCubeDesc(String path) throws IOException {
+        ResourceStore store = getStore();
+        logger.debug("Loading CubeDesc " + store.getReadableResourcePath(path));
+
+        CubeDesc ndesc = store.getResource(path, CubeDesc.class, CUBE_SERIALIZER);
+
+        if (StringUtils.isBlank(ndesc.getName())) {
+            throw new IllegalStateException("CubeDesc name must not be blank");
+        }
+
+        ndesc.init(config, srcTableMap.getMap());
+
+        if (ndesc.getError().isEmpty() == false) {
+            throw new IllegalStateException("Cube desc at " + path + " has issues: " + ndesc.getError());
+        }
+
+        return ndesc;
+    }
+
+    private void reloadAllInvertedIndexDesc() throws IOException {
+        ResourceStore store = getStore();
+        logger.info("Reloading Inverted Index Desc from folder " + store.getReadableResourcePath(ResourceStore.IIDESC_RESOURCE_ROOT));
+
+        iiDescMap.clear();
+
+        List<String> paths = store.collectResourceRecursively(ResourceStore.IIDESC_RESOURCE_ROOT, ".json");
+        for (String path : paths) {
+            InvertedIndexDesc desc;
+            try {
+                desc = loadInvertedIndexDesc(path);
+            } catch (Exception e) {
+                logger.error("Error loading inverted index desc " + path, e);
+                continue;
+            }
+            if (path.equals(desc.getResourcePath()) == false) {
+                logger.error("Skip suspicious desc at " + path + ", " + desc + " should be at " + desc.getResourcePath());
+                continue;
+            }
+            if (iiDescMap.containsKey(desc.getName())) {
+                logger.error("Dup InvertedIndexDesc name '" + desc.getName() + "' on path " + path);
+                continue;
+            }
+
+            iiDescMap.putLocal(desc.getName(), desc);
+        }
+
+        logger.debug("Loaded " + iiDescMap.size() + " Inverted Index Desc(s)");
+    }
+
+    private InvertedIndexDesc loadInvertedIndexDesc(String path) throws IOException {
+        ResourceStore store = getStore();
+        logger.debug("Loading InvertedIndexDesc " + store.getReadableResourcePath(path));
+
+        InvertedIndexDesc desc = store.getResource(path, InvertedIndexDesc.class, IIDESC_SERIALIZER);
+        if (StringUtils.isBlank(desc.getName())) {
+            throw new IllegalStateException("InvertedIndexDesc name must not be blank");
+        }
+
+        desc.init(this);
+
+        return desc;
+    }
+
+    /**
+     * Update CubeDesc with the input. Broadcast the event into cluster
+     * 
+     * @param desc
+     * @return
+     * @throws IOException
+     */
+    public CubeDesc updateCubeDesc(CubeDesc desc) throws IOException {
+        // Validate CubeDesc
+        if (desc.getUuid() == null || desc.getName() == null) {
+            throw new IllegalArgumentException();
+        }
+        String name = desc.getName();
+        if (!cubeDescMap.containsKey(name)) {
+            throw new IllegalArgumentException("CubeDesc '" + name + "' does not exist.");
+        }
+
+        try {
+            desc.init(config, srcTableMap.getMap());
+        } catch (IllegalStateException e) {
+            desc.addError(e.getMessage(), true);
+            return desc;
+        } catch (IllegalArgumentException e) {
+            desc.addError(e.getMessage(), true);
+            return desc;
+        }
+
+        // Semantic validation
+        CubeMetadataValidator validator = new CubeMetadataValidator();
+        ValidateContext context = validator.validate(desc, true);
+        if (!context.ifPass()) {
+            return desc;
+        }
+
+        desc.setSignature(desc.calculateSignature());
+
+        // Save Source
+        String path = desc.getResourcePath();
+        getStore().putResource(path, desc, CUBE_SERIALIZER);
+
+        // Reload the CubeDesc
+        CubeDesc ndesc = loadCubeDesc(path);
+        // Here replace the old one
+        cubeDescMap.put(ndesc.getName(), desc);
+
+        return ndesc;
+    }
+
+    /**
+     * Reload CubeDesc from resource store It will be triggered by an desc
+     * update event.
+     * 
+     * @param name
+     * @throws IOException
+     */
+    public CubeDesc reloadCubeDesc(String name) throws IOException {
+
+        // Save Source
+        String path = CubeDesc.getCubeDescResourcePath(name);
+
+        // Reload the CubeDesc
+        CubeDesc ndesc = loadCubeDesc(path);
+
+        // Here replace the old one
+        cubeDescMap.put(ndesc.getName(), ndesc);
+        return ndesc;
+    }
+
+    /**
+     * Tell CubeManager that the cube instance has changed. The cube info will
+     * be stored Reload the cube desc and source table A broadcast must be sent
+     * out
+     * 
+     * @return
+     * @throws IOException
+     */
+    public void reload() {
+        removeInstance(config);
+        getInstance(config);
+    }
+
+}


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

Posted by lu...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/common/src/main/java/com/kylinolap/common/persistence/HBaseResourceStore.java
----------------------------------------------------------------------
diff --git a/common/src/main/java/com/kylinolap/common/persistence/HBaseResourceStore.java b/common/src/main/java/com/kylinolap/common/persistence/HBaseResourceStore.java
new file mode 100644
index 0000000..3e101af
--- /dev/null
+++ b/common/src/main/java/com/kylinolap/common/persistence/HBaseResourceStore.java
@@ -0,0 +1,307 @@
+/*
+ * 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.common.persistence;
+
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.LinkedHashMap;
+import java.util.Map;
+import java.util.Map.Entry;
+
+import org.apache.commons.io.IOUtils;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.client.Delete;
+import org.apache.hadoop.hbase.client.HConnection;
+import org.apache.hadoop.hbase.client.HTableInterface;
+import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.client.Result;
+import org.apache.hadoop.hbase.client.ResultScanner;
+import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.filter.KeyOnlyFilter;
+import org.apache.hadoop.hbase.util.Bytes;
+
+import com.kylinolap.common.KylinConfig;
+import com.kylinolap.common.util.BytesUtil;
+import com.kylinolap.common.util.HadoopUtil;
+
+public class HBaseResourceStore extends ResourceStore {
+
+    private static final String DEFAULT_TABLE_NAME = "kylin_metadata";
+    private static final String FAMILY = "f";
+    private static final byte[] B_FAMILY = Bytes.toBytes(FAMILY);
+    private static final String COLUMN = "c";
+    private static final byte[] B_COLUMN = Bytes.toBytes(COLUMN);
+    private static final String COLUMN_TS = "t";
+    private static final byte[] B_COLUMN_TS = Bytes.toBytes(COLUMN_TS);
+
+    private static final Map<String, String> TABLE_SUFFIX_MAP = new LinkedHashMap<String, String>();
+
+    static {
+        TABLE_SUFFIX_MAP.put(CUBE_RESOURCE_ROOT + "/", "_cube");
+        TABLE_SUFFIX_MAP.put(DICT_RESOURCE_ROOT + "/", "_dict");
+        TABLE_SUFFIX_MAP.put("/invertedindex/", "_invertedindex");
+        TABLE_SUFFIX_MAP.put(JOB_PATH_ROOT + "/", "_job");
+        TABLE_SUFFIX_MAP.put(JOB_OUTPUT_PATH_ROOT + "/", "_job_output");
+        TABLE_SUFFIX_MAP.put(PROJECT_RESOURCE_ROOT + "/", "_proj");
+        TABLE_SUFFIX_MAP.put(SNAPSHOT_RESOURCE_ROOT + "/", "_table_snapshot");
+        TABLE_SUFFIX_MAP.put("", ""); // DEFAULT CASE
+    }
+
+    final String tableNameBase;
+    final String hbaseUrl;
+
+    final Map<String, String> tableNameMap; // path prefix ==> HBase table name
+
+    private HConnection getConnection() throws IOException {
+        return HBaseConnection.get(hbaseUrl);
+    }
+
+    public HBaseResourceStore(KylinConfig kylinConfig) throws IOException {
+        super(kylinConfig);
+
+        String metadataUrl = kylinConfig.getMetadataUrl();
+        // split TABLE@HBASE_URL
+        int cut = metadataUrl.indexOf('@');
+        tableNameBase = cut < 0 ? DEFAULT_TABLE_NAME : metadataUrl.substring(0, cut);
+        hbaseUrl = cut < 0 ? metadataUrl : metadataUrl.substring(cut + 1);
+
+        tableNameMap = new LinkedHashMap<String, String>();
+        for (Entry<String, String> entry : TABLE_SUFFIX_MAP.entrySet()) {
+            String pathPrefix = entry.getKey();
+            String tableName = tableNameBase + entry.getValue();
+            tableNameMap.put(pathPrefix, tableName);
+            createHTableIfNeeded(tableName);
+        }
+
+    }
+
+    private void createHTableIfNeeded(String tableName) throws IOException {
+        HBaseConnection.createHTableIfNeeded(getConnection(), tableName, FAMILY);
+    }
+
+    private String getTableName(String path) {
+        for (Entry<String, String> entry : tableNameMap.entrySet()) {
+            String pathPrefix = entry.getKey();
+            if (path.startsWith(pathPrefix))
+                return entry.getValue();
+        }
+        throw new IllegalStateException("failed to find HBase table for path -- " + path);
+    }
+
+    @Override
+    protected ArrayList<String> listResourcesImpl(String resPath) throws IOException {
+        assert resPath.startsWith("/");
+        String lookForPrefix = resPath.endsWith("/") ? resPath : resPath + "/";
+        byte[] startRow = Bytes.toBytes(lookForPrefix);
+        byte[] endRow = Bytes.toBytes(lookForPrefix);
+        endRow[endRow.length - 1]++;
+
+        ArrayList<String> result = new ArrayList<String>();
+
+        for (Entry<String, String> entry : tableNameMap.entrySet()) {
+            String pathPrefix = entry.getKey();
+            String tableName = entry.getValue();
+
+            if ((pathPrefix.startsWith(lookForPrefix) || lookForPrefix.startsWith(pathPrefix)) == false)
+                continue;
+
+            HTableInterface table = getConnection().getTable(tableName);
+
+            Scan scan = new Scan(startRow, endRow);
+            scan.setFilter(new KeyOnlyFilter());
+            try {
+                ResultScanner scanner = table.getScanner(scan);
+                for (Result r : scanner) {
+                    String path = Bytes.toString(r.getRow());
+                    assert path.startsWith(lookForPrefix);
+                    int cut = path.indexOf('/', lookForPrefix.length());
+                    String child = cut < 0 ? path : path.substring(0, cut);
+                    if (result.contains(child) == false)
+                        result.add(child);
+                }
+            } finally {
+                IOUtils.closeQuietly(table);
+            }
+        }
+        // return null to indicate not a folder
+        return result.isEmpty() ? null : result;
+    }
+
+    @Override
+    protected boolean existsImpl(String resPath) throws IOException {
+        Result r = getByScan(resPath, null, null);
+        return r != null;
+    }
+
+    @Override
+    protected InputStream getResourceImpl(String resPath) throws IOException {
+        Result r = getByScan(resPath, B_FAMILY, B_COLUMN);
+        if (r == null)
+            return null;
+
+        byte[] value = r.getValue(B_FAMILY, B_COLUMN);
+        if (value.length == 0) {
+            Path redirectPath = bigCellHDFSPath(resPath);
+            Configuration hconf = HadoopUtil.getDefaultConfiguration();
+            FileSystem fileSystem = FileSystem.get(hconf);
+
+            return fileSystem.open(redirectPath);
+        } else {
+            return new ByteArrayInputStream(value);
+        }
+    }
+
+    @Override
+    protected long getResourceTimestampImpl(String resPath) throws IOException {
+        Result r = getByScan(resPath, B_FAMILY, B_COLUMN_TS);
+        if (r == null)
+            return 0;
+        else
+            return Bytes.toLong(r.getValue(B_FAMILY, B_COLUMN_TS));
+    }
+
+    @Override
+    protected void putResourceImpl(String resPath, InputStream content, long ts) throws IOException {
+        ByteArrayOutputStream bout = new ByteArrayOutputStream();
+        IOUtils.copy(content, bout);
+        bout.close();
+
+        HTableInterface table = getConnection().getTable(getTableName(resPath));
+        try {
+            byte[] row = Bytes.toBytes(resPath);
+            Put put = buildPut(resPath, ts, row, bout.toByteArray(), table);
+
+            table.put(put);
+            table.flushCommits();
+        } finally {
+            IOUtils.closeQuietly(table);
+        }
+    }
+
+    @Override
+    protected long checkAndPutResourceImpl(String resPath, byte[] content, long oldTS, long newTS) throws IOException, IllegalStateException {
+        HTableInterface table = getConnection().getTable(getTableName(resPath));
+        try {
+            byte[] row = Bytes.toBytes(resPath);
+            byte[] bOldTS = oldTS == 0 ? null : Bytes.toBytes(oldTS);
+            Put put = buildPut(resPath, newTS, row, content, table);
+
+            boolean ok = table.checkAndPut(row, B_FAMILY, B_COLUMN_TS, bOldTS, put);
+            if (!ok)
+                throw new IllegalStateException("Overwriting conflict " + resPath + ", expect old TS " + oldTS + ", but it is " + getResourceTimestamp(resPath));
+
+            table.flushCommits();
+
+            return newTS;
+        } finally {
+            IOUtils.closeQuietly(table);
+        }
+    }
+
+    @Override
+    protected void deleteResourceImpl(String resPath) throws IOException {
+        HTableInterface table = getConnection().getTable(getTableName(resPath));
+        try {
+            Delete del = new Delete(Bytes.toBytes(resPath));
+            table.delete(del);
+            table.flushCommits();
+        } finally {
+            IOUtils.closeQuietly(table);
+        }
+    }
+
+    @Override
+    protected String getReadableResourcePathImpl(String resPath) {
+        return tableNameBase + "(key='" + resPath + "')@" + kylinConfig.getMetadataUrl();
+    }
+
+    private Result getByScan(String path, byte[] family, byte[] column) throws IOException {
+        byte[] startRow = Bytes.toBytes(path);
+        byte[] endRow = plusZero(startRow);
+
+        Scan scan = new Scan(startRow, endRow);
+        if (family == null || column == null) {
+            scan.setFilter(new KeyOnlyFilter());
+        } else {
+            scan.addColumn(family, column);
+        }
+
+        HTableInterface table = getConnection().getTable(getTableName(path));
+        try {
+            ResultScanner scanner = table.getScanner(scan);
+            Result result = null;
+            for (Result r : scanner) {
+                result = r;
+            }
+            return result == null || result.isEmpty() ? null : result;
+        } finally {
+            IOUtils.closeQuietly(table);
+        }
+    }
+
+    private byte[] plusZero(byte[] startRow) {
+        byte[] endRow = Arrays.copyOf(startRow, startRow.length + 1);
+        endRow[endRow.length - 1] = 0;
+        return endRow;
+    }
+
+    private Path writeLargeCellToHdfs(String resPath, byte[] largeColumn, HTableInterface table) throws IOException {
+        Path redirectPath = bigCellHDFSPath(resPath);
+        Configuration hconf = HadoopUtil.getDefaultConfiguration();
+        FileSystem fileSystem = FileSystem.get(hconf);
+
+        if (fileSystem.exists(redirectPath)) {
+            fileSystem.delete(redirectPath, true);
+        }
+
+        FSDataOutputStream out = fileSystem.create(redirectPath);
+
+        try {
+            out.write(largeColumn);
+        } finally {
+            IOUtils.closeQuietly(out);
+        }
+
+        return redirectPath;
+    }
+
+    public Path bigCellHDFSPath(String resPath) {
+        String hdfsWorkingDirectory = this.kylinConfig.getHdfsWorkingDirectory();
+        Path redirectPath = new Path(hdfsWorkingDirectory, "resources" + resPath);
+        return redirectPath;
+    }
+
+    private Put buildPut(String resPath, long ts, byte[] row, byte[] content, HTableInterface table) throws IOException {
+        int kvSizeLimit = this.kylinConfig.getHBaseKeyValueSize();
+        if (content.length > kvSizeLimit) {
+            writeLargeCellToHdfs(resPath, content, table);
+            content = BytesUtil.EMPTY_BYTE_ARRAY;
+        }
+
+        Put put = new Put(row);
+        put.add(B_FAMILY, B_COLUMN, content);
+        put.add(B_FAMILY, B_COLUMN_TS, Bytes.toBytes(ts));
+
+        return put;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/common/src/main/java/com/kylinolap/common/persistence/JsonSerializer.java
----------------------------------------------------------------------
diff --git a/common/src/main/java/com/kylinolap/common/persistence/JsonSerializer.java b/common/src/main/java/com/kylinolap/common/persistence/JsonSerializer.java
new file mode 100644
index 0000000..1471d09
--- /dev/null
+++ b/common/src/main/java/com/kylinolap/common/persistence/JsonSerializer.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.common.persistence;
+
+import java.io.DataInputStream;
+import java.io.DataOutputStream;
+import java.io.IOException;
+
+import com.kylinolap.common.util.JsonUtil;
+
+/**
+ * @author yangli9
+ */
+public class JsonSerializer<T extends RootPersistentEntity> implements Serializer<T> {
+
+    Class<T> clz;
+
+    public JsonSerializer(Class<T> clz) {
+        this.clz = clz;
+    }
+
+    @Override
+    public T deserialize(DataInputStream in) throws IOException {
+        return JsonUtil.readValue(in, clz);
+    }
+
+    @Override
+    public void serialize(T obj, DataOutputStream out) throws IOException {
+        JsonUtil.writeValueIndent(out, obj);
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/common/src/main/java/com/kylinolap/common/persistence/ResourceStore.java
----------------------------------------------------------------------
diff --git a/common/src/main/java/com/kylinolap/common/persistence/ResourceStore.java b/common/src/main/java/com/kylinolap/common/persistence/ResourceStore.java
new file mode 100644
index 0000000..36abfdd
--- /dev/null
+++ b/common/src/main/java/com/kylinolap/common/persistence/ResourceStore.java
@@ -0,0 +1,256 @@
+/*
+ * 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.common.persistence;
+
+import java.io.ByteArrayOutputStream;
+import java.io.DataInputStream;
+import java.io.DataOutputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.ConcurrentHashMap;
+
+import org.apache.commons.io.IOUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.google.common.collect.Lists;
+import com.kylinolap.common.KylinConfig;
+
+abstract public class ResourceStore {
+
+    private static final Logger logger = LoggerFactory.getLogger(ResourceStore.class);
+
+    public static final String CUBE_RESOURCE_ROOT = "/cube";
+    public static final String CUBE_DESC_RESOURCE_ROOT = "/cube_desc";
+    public static final String DICT_RESOURCE_ROOT = "/dict";
+    public static final String IIDESC_RESOURCE_ROOT = "/invertedindex_desc";
+    public static final String JOB_PATH_ROOT = "/job";
+    public static final String JOB_OUTPUT_PATH_ROOT = "/job_output";
+    public static final String PROJECT_RESOURCE_ROOT = "/project";
+    public static final String SNAPSHOT_RESOURCE_ROOT = "/table_snapshot";
+    public static final String TABLE_EXD_RESOURCE_ROOT = "/table_exd";
+    public static final String TABLE_RESOURCE_ROOT = "/table";
+
+    private static ConcurrentHashMap<KylinConfig, ResourceStore> CACHE = new ConcurrentHashMap<KylinConfig, ResourceStore>();
+
+    public static final ArrayList<Class<? extends ResourceStore>> knownImpl = new ArrayList<Class<? extends ResourceStore>>();
+
+    static {
+        knownImpl.add(HBaseResourceStore.class);
+        knownImpl.add(FileResourceStore.class);
+    }
+
+    public static ResourceStore getStore(KylinConfig kylinConfig) {
+        ResourceStore r = CACHE.get(kylinConfig);
+        List<Throwable> es = new ArrayList<Throwable>();
+        if (r == null) {
+            logger.info("Using metadata url " + kylinConfig.getMetadataUrl() + " for resource store");
+            for (Class<? extends ResourceStore> cls : knownImpl) {
+
+                try {
+                    r = cls.getConstructor(KylinConfig.class).newInstance(kylinConfig);
+                } catch (Exception e) {
+                    es.add(e);
+                } catch (NoClassDefFoundError er) {
+                    // may throw NoClassDefFoundError
+                    es.add(er);
+                }
+                if (r != null) {
+                    break;
+                }
+            }
+            if (r == null) {
+                for (Throwable exceptionOrError : es) {
+                    logger.error("Create new store instance failed ", exceptionOrError);
+                }
+                throw new IllegalArgumentException("Failed to find metadata store by url: " + kylinConfig.getMetadataUrl());
+            }
+
+            CACHE.put(kylinConfig, r);
+        }
+        return r;
+    }
+
+    // ============================================================================
+
+    KylinConfig kylinConfig;
+
+    ResourceStore(KylinConfig kylinConfig) {
+        this.kylinConfig = kylinConfig;
+    }
+
+    /**
+     * return a list of child resources & folders under given path, return null
+     * if given path is not a folder
+     */
+    final public ArrayList<String> listResources(String resPath) throws IOException {
+        resPath = norm(resPath);
+        return listResourcesImpl(resPath);
+    }
+
+    abstract protected ArrayList<String> listResourcesImpl(String resPath) throws IOException;
+
+    /**
+     * return true if a resource exists, return false in case of folder or
+     * non-exist
+     */
+    final public boolean exists(String resPath) throws IOException {
+        return existsImpl(norm(resPath));
+    }
+
+    abstract protected boolean existsImpl(String resPath) throws IOException;
+
+    /**
+     * read a resource, return null in case of not found
+     */
+    final public <T extends RootPersistentEntity> T getResource(String resPath, Class<T> clz, Serializer<T> serializer) throws IOException {
+        resPath = norm(resPath);
+        InputStream in = getResourceImpl(resPath);
+        if (in == null)
+            return null;
+
+        DataInputStream din = new DataInputStream(in);
+        try {
+            T r = serializer.deserialize(din);
+            r.setLastModified(getResourceTimestamp(resPath));
+            return r;
+        } finally {
+            IOUtils.closeQuietly(din);
+            IOUtils.closeQuietly(in);
+        }
+    }
+
+    final public InputStream getResource(String resPath) throws IOException {
+        return getResourceImpl(norm(resPath));
+    }
+
+    abstract protected InputStream getResourceImpl(String resPath) throws IOException;
+
+    final public long getResourceTimestamp(String resPath) throws IOException {
+        return getResourceTimestampImpl(norm(resPath));
+    }
+
+    abstract protected long getResourceTimestampImpl(String resPath) throws IOException;
+
+    /**
+     * overwrite a resource without write conflict check
+     */
+    final public void putResource(String resPath, InputStream content, long ts) throws IOException {
+        resPath = norm(resPath);
+        logger.debug("Saving resource " + resPath + " (Store " + kylinConfig.getMetadataUrl() + ")");
+        putResourceImpl(resPath, content, ts);
+    }
+
+    abstract protected void putResourceImpl(String resPath, InputStream content, long ts) throws IOException;
+
+    /**
+     * check & set, overwrite a resource
+     */
+    final public <T extends RootPersistentEntity> void putResource(String resPath, T obj, Serializer<T> serializer) throws IOException {
+        resPath = norm(resPath);
+        logger.debug("Saving resource " + resPath + " (Store " + kylinConfig.getMetadataUrl() + ")");
+
+        long oldTS = obj.getLastModified();
+        long newTS = System.currentTimeMillis();
+        obj.setLastModified(newTS);
+
+        try {
+            ByteArrayOutputStream buf = new ByteArrayOutputStream();
+            DataOutputStream dout = new DataOutputStream(buf);
+            serializer.serialize(obj, dout);
+            dout.close();
+            buf.close();
+
+            newTS = checkAndPutResourceImpl(resPath, buf.toByteArray(), oldTS, newTS);
+            obj.setLastModified(newTS); // update again the confirmed TS
+
+        } catch (IOException e) {
+            obj.setLastModified(oldTS); // roll back TS when write fail
+            throw e;
+        } catch (RuntimeException e) {
+            obj.setLastModified(oldTS); // roll back TS when write fail
+            throw e;
+        }
+    }
+
+    /**
+     * checks old timestamp when overwriting existing
+     */
+    abstract protected long checkAndPutResourceImpl(String resPath, byte[] content, long oldTS, long newTS) throws IOException, IllegalStateException;
+
+    /**
+     * delete a resource, does nothing on a folder
+     */
+    final public void deleteResource(String resPath) throws IOException {
+        logger.debug("Deleting resource " + resPath + " (Store " + kylinConfig.getMetadataUrl() + ")");
+        deleteResourceImpl(norm(resPath));
+    }
+
+    abstract protected void deleteResourceImpl(String resPath) throws IOException;
+
+    /**
+     * get a readable string of a resource path
+     */
+    final public String getReadableResourcePath(String resPath) {
+        return getReadableResourcePathImpl(norm(resPath));
+    }
+
+    abstract protected String getReadableResourcePathImpl(String resPath);
+
+    private String norm(String resPath) {
+        resPath = resPath.trim();
+        while (resPath.startsWith("//"))
+            resPath = resPath.substring(1);
+        while (resPath.endsWith("/"))
+            resPath = resPath.substring(0, resPath.length() - 1);
+        if (resPath.startsWith("/") == false)
+            resPath = "/" + resPath;
+        return resPath;
+    }
+
+    // ============================================================================
+
+    public static interface Visitor {
+        void visit(String path) throws IOException;
+    }
+
+    public void scanRecursively(String path, Visitor visitor) throws IOException {
+        ArrayList<String> children = listResources(path);
+        if (children != null) {
+            for (String child : children)
+                scanRecursively(child, visitor);
+            return;
+        }
+
+        if (exists(path))
+            visitor.visit(path);
+    }
+
+    public List<String> collectResourceRecursively(String root, final String suffix) throws IOException {
+        final ArrayList<String> collector = Lists.newArrayList();
+        scanRecursively(root, new Visitor() {
+            @Override
+            public void visit(String path) {
+                if (path.endsWith(suffix))
+                    collector.add(path);
+            }
+        });
+        return collector;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/common/src/main/java/com/kylinolap/common/persistence/ResourceTool.java
----------------------------------------------------------------------
diff --git a/common/src/main/java/com/kylinolap/common/persistence/ResourceTool.java b/common/src/main/java/com/kylinolap/common/persistence/ResourceTool.java
new file mode 100644
index 0000000..66783ca
--- /dev/null
+++ b/common/src/main/java/com/kylinolap/common/persistence/ResourceTool.java
@@ -0,0 +1,141 @@
+/*
+ * 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.common.persistence;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.util.ArrayList;
+
+import com.kylinolap.common.KylinConfig;
+import com.kylinolap.common.util.StringUtil;
+
+public class ResourceTool {
+
+    private static String[] excludes = null;
+
+    public static void main(String[] args) throws IOException {
+        args = StringUtil.filterSystemArgs(args);
+
+        if (args.length == 0) {
+            System.out.println("Usage: MetadataTool reset METADATA_URI");
+            System.out.println("Usage: MetadataTool copy  METADATA_URI_SRC  METADATA_URI_DST");
+            System.out.println("Usage: MetadataTool list  METADATA_URI      PATH");
+            return;
+        }
+
+        String exclude = System.getProperty("exclude");
+        if (exclude != null) {
+            excludes = exclude.split("\\s*,\\s*");
+        }
+
+        String cmd = args[0];
+        if (cmd.equals("reset"))
+            reset(args.length == 1 ? KylinConfig.getInstanceFromEnv() : KylinConfig.createInstanceFromUri(args[1]));
+        else if (cmd.equals("copy"))
+            copy(args[1], args[2]);
+        else if (cmd.equals("list"))
+            list(args[1], args[2]);
+        else if (cmd.equals("download"))
+            copy(KylinConfig.getInstanceFromEnv(), KylinConfig.createInstanceFromUri(args[1]));
+        else if (cmd.equals("upload"))
+            copy(KylinConfig.createInstanceFromUri(args[1]), KylinConfig.getInstanceFromEnv());
+        else if (cmd.equals("remove"))
+            remove(KylinConfig.getInstanceFromEnv(), args[1]);
+        else
+            System.out.println("Unknown cmd: " + cmd);
+    }
+
+    public static void list(KylinConfig config, String path) throws IOException {
+        ResourceStore store = ResourceStore.getStore(config);
+        ArrayList<String> result = store.listResources(path);
+        System.out.println("" + result);
+    }
+
+    private static void list(String metadataUri, String path) throws IOException {
+        KylinConfig config = KylinConfig.createInstanceFromUri(metadataUri);
+        list(config, path);
+    }
+
+    public static void copy(KylinConfig srcConfig, KylinConfig dstConfig) throws IOException {
+
+        ResourceStore src = ResourceStore.getStore(srcConfig);
+        ResourceStore dst = ResourceStore.getStore(dstConfig);
+        copyR(src, dst, "/");
+    }
+
+    private static void copy(String srcUri, String dstUri) throws IOException {
+
+        System.out.println("Copy from " + srcUri + " to " + dstUri);
+
+        KylinConfig srcConfig = KylinConfig.createInstanceFromUri(srcUri);
+        KylinConfig dstConfig = KylinConfig.createInstanceFromUri(dstUri);
+        copy(srcConfig, dstConfig);
+
+    }
+
+    private static void copyR(ResourceStore src, ResourceStore dst, String path) throws IOException {
+        ArrayList<String> children = src.listResources(path);
+
+        // case of resource (not a folder)
+        if (children == null) {
+            if (matchExclude(path) == false) {
+                InputStream content = src.getResource(path);
+                long ts = src.getResourceTimestamp(path);
+                if (content != null)
+                    dst.putResource(path, content, ts);
+                else
+                    System.out.println("Null inputstream for " + path);
+            }
+        }
+        // case of folder
+        else {
+            for (String child : children)
+                copyR(src, dst, child);
+        }
+    }
+
+    private static boolean matchExclude(String path) {
+        if (excludes == null)
+            return false;
+        for (String exclude : excludes) {
+            if (path.startsWith(exclude))
+                return true;
+        }
+        return false;
+    }
+
+    public static void reset(KylinConfig config) throws IOException {
+        ResourceStore store = ResourceStore.getStore(config);
+        resetR(store, "/");
+    }
+
+    private static void resetR(ResourceStore store, String path) throws IOException {
+        ArrayList<String> children = store.listResources(path);
+        if (children == null) { // path is a resource (not a folder)
+            if (matchExclude(path) == false) {
+                store.deleteResource(path);
+            }
+        } else {
+            for (String child : children)
+                resetR(store, child);
+        }
+    }
+    
+    private static void remove(KylinConfig config, String path) throws IOException {
+        ResourceStore store = ResourceStore.getStore(config);
+        resetR(store, path);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/common/src/main/java/com/kylinolap/common/persistence/RootPersistentEntity.java
----------------------------------------------------------------------
diff --git a/common/src/main/java/com/kylinolap/common/persistence/RootPersistentEntity.java b/common/src/main/java/com/kylinolap/common/persistence/RootPersistentEntity.java
new file mode 100644
index 0000000..d19d583
--- /dev/null
+++ b/common/src/main/java/com/kylinolap/common/persistence/RootPersistentEntity.java
@@ -0,0 +1,104 @@
+/*
+ * 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.common.persistence;
+
+import java.util.UUID;
+
+import org.apache.commons.lang.time.FastDateFormat;
+
+import com.fasterxml.jackson.annotation.JsonAutoDetect;
+import com.fasterxml.jackson.annotation.JsonAutoDetect.Visibility;
+import com.fasterxml.jackson.annotation.JsonProperty;
+
+/**
+ * Marks the root entity of JSON persistence. Unit of read, write, cache, and
+ * refresh.
+ * 
+ * - CubeInstance - CubeDesc - SourceTable - JobMeta - Dictionary (not JSON but
+ * also top level persistence)
+ * 
+ * @author yangli9
+ */
+@JsonAutoDetect(fieldVisibility = Visibility.NONE, getterVisibility = Visibility.NONE, isGetterVisibility = Visibility.NONE, setterVisibility = Visibility.NONE)
+abstract public class RootPersistentEntity implements AclEntity {
+
+    static FastDateFormat format = FastDateFormat.getInstance("yyyy-MM-dd HH:mm:ss z");
+
+    public static String formatTime(long millis) {
+        return format.format(millis);
+    }
+
+    // ============================================================================
+
+    @JsonProperty("uuid")
+    protected String uuid;
+
+    @JsonProperty("last_modified")
+    protected long lastModified;
+
+    public String getUuid() {
+        return uuid;
+    }
+
+    public void setUuid(String uuid) {
+        this.uuid = uuid;
+    }
+
+    public String getId() {
+        return uuid;
+    }
+
+    public long getLastModified() {
+        return lastModified;
+    }
+
+    public void setLastModified(long lastModified) {
+        this.lastModified = lastModified;
+    }
+
+    public void updateRandomUuid() {
+        setUuid(UUID.randomUUID().toString());
+    }
+
+    @Override
+    public int hashCode() {
+        final int prime = 31;
+        int result = 1;
+        result = prime * result + (int) (lastModified ^ (lastModified >>> 32));
+        result = prime * result + ((uuid == null) ? 0 : uuid.hashCode());
+        return result;
+    }
+
+    @Override
+    public boolean equals(Object obj) {
+        if (this == obj)
+            return true;
+        if (obj == null)
+            return false;
+        if (getClass() != obj.getClass())
+            return false;
+        RootPersistentEntity other = (RootPersistentEntity) obj;
+        if (lastModified != other.lastModified)
+            return false;
+        if (uuid == null) {
+            if (other.uuid != null)
+                return false;
+        } else if (!uuid.equals(other.uuid))
+            return false;
+        return true;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/common/src/main/java/com/kylinolap/common/persistence/Serializer.java
----------------------------------------------------------------------
diff --git a/common/src/main/java/com/kylinolap/common/persistence/Serializer.java b/common/src/main/java/com/kylinolap/common/persistence/Serializer.java
new file mode 100644
index 0000000..d063745
--- /dev/null
+++ b/common/src/main/java/com/kylinolap/common/persistence/Serializer.java
@@ -0,0 +1,33 @@
+/*
+ * 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.common.persistence;
+
+import java.io.DataInputStream;
+import java.io.DataOutputStream;
+import java.io.IOException;
+
+/**
+ * @author yangli9
+ * 
+ */
+public interface Serializer<T extends RootPersistentEntity> {
+
+    public void serialize(T obj, DataOutputStream out) throws IOException;
+
+    public T deserialize(DataInputStream in) throws IOException;
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/common/src/main/java/com/kylinolap/common/persistence/StorageException.java
----------------------------------------------------------------------
diff --git a/common/src/main/java/com/kylinolap/common/persistence/StorageException.java b/common/src/main/java/com/kylinolap/common/persistence/StorageException.java
new file mode 100644
index 0000000..62f12cd
--- /dev/null
+++ b/common/src/main/java/com/kylinolap/common/persistence/StorageException.java
@@ -0,0 +1,35 @@
+/*
+ * 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.common.persistence;
+
+/**
+ * 
+ * @author xjiang
+ * 
+ */
+public class StorageException extends RuntimeException {
+
+    private static final long serialVersionUID = -3748712888242406257L;
+
+    public StorageException(String msg, Throwable t) {
+        super(msg, t);
+    }
+
+    public StorageException(String msg) {
+        super(msg);
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/common/src/main/java/com/kylinolap/common/restclient/AbstractRestCache.java
----------------------------------------------------------------------
diff --git a/common/src/main/java/com/kylinolap/common/restclient/AbstractRestCache.java b/common/src/main/java/com/kylinolap/common/restclient/AbstractRestCache.java
new file mode 100644
index 0000000..8bc2468
--- /dev/null
+++ b/common/src/main/java/com/kylinolap/common/restclient/AbstractRestCache.java
@@ -0,0 +1,47 @@
+/*
+ * 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.common.restclient;
+
+/**
+ * @author xjiang
+ * 
+ */
+public abstract class AbstractRestCache<K, V> {
+
+    protected final Broadcaster.TYPE syncType;
+
+    protected AbstractRestCache(Broadcaster.TYPE syncType) {
+        this.syncType = syncType;
+    }
+
+    protected final void syncRemote(Object key, Broadcaster.EVENT syncAction) {
+        String syncKey = (syncType == Broadcaster.TYPE.METADATA) ? "metadata" : key.toString();
+        Broadcaster.queue(syncType.getType(), syncAction.getType(), syncKey);
+    }
+
+    public abstract void put(K key, V value);
+
+    public abstract void putLocal(K key, V value);
+
+    public abstract void remove(K key);
+
+    public abstract void removeLocal(K key);
+
+    public abstract void clear();
+
+    public abstract int size();
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/common/src/main/java/com/kylinolap/common/restclient/Broadcaster.java
----------------------------------------------------------------------
diff --git a/common/src/main/java/com/kylinolap/common/restclient/Broadcaster.java b/common/src/main/java/com/kylinolap/common/restclient/Broadcaster.java
new file mode 100644
index 0000000..60f0963
--- /dev/null
+++ b/common/src/main/java/com/kylinolap/common/restclient/Broadcaster.java
@@ -0,0 +1,261 @@
+/*
+ * 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.common.restclient;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Date;
+import java.util.List;
+import java.util.Timer;
+import java.util.TimerTask;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.kylinolap.common.KylinConfig;
+
+/**
+ * Broadcast kylin event out
+ * 
+ * @author jianliu
+ * 
+ */
+public class Broadcaster {
+
+    private static final Logger logger = LoggerFactory.getLogger(Broadcaster.class);
+
+    private static List<BroadcastEvent> broadcaseEvents = new ArrayList<BroadcastEvent>();
+
+    static class BroadcasterHolder {
+        static final Broadcaster INSTANCE = new Broadcaster();
+    }
+
+    private Broadcaster() {
+        Timer timer = new Timer();
+        TimerTask task = new TimerTask() {
+            public void run() {
+                Broadcaster.flush();
+            }
+        };
+
+        timer.schedule(task, new Date(), 10 * 1000);
+    }
+
+    public Broadcaster getInstance() {
+        return BroadcasterHolder.INSTANCE;
+    }
+
+    public static void queueSyncMetadata() {
+        queue(TYPE.METADATA.getType(), EVENT.CREATE.getType(), "metadata");
+    }
+
+    /**
+     * Broadcast the cubedesc event out
+     * 
+     * @param action
+     *            event action
+     */
+    public static synchronized void queue(String type, String action, String key) {
+        BroadcastEvent event = BroadcasterHolder.INSTANCE.new BroadcastEvent(type, action, key);
+
+        if (!broadcaseEvents.contains(event)) {
+            broadcaseEvents.add(event);
+        }
+    }
+
+    public static synchronized void flush() {
+        String[] nodes = KylinConfig.getInstanceFromEnv().getRestServers();
+        if (nodes == null)
+            return;
+
+        for (BroadcastEvent event : broadcaseEvents) {
+            for (String nodeUri : nodes) {
+                logger.debug("Broadcast nodeUri: " + nodeUri + ", type: " + event.getType() + ", action: " + event.getAction() + ", name: " + event.getName());
+                WipeCacheThread thread = BroadcasterHolder.INSTANCE.new WipeCacheThread(nodeUri, event.getType(), event.getAction(), event.getName());
+                thread.start();
+            }
+        }
+
+        broadcaseEvents.clear();
+    }
+
+    public static String genEventkey(String type, String action, String name) {
+        String time = String.valueOf(System.currentTimeMillis());
+        return time + "_" + type + "_" + action + "_" + name;
+    }
+
+    protected class WipeCacheThread extends Thread {
+        private String nodeUri;
+        private String type;
+        private String action;
+        private String name;
+
+        public WipeCacheThread(String nodeUri, String type, String action, String name) {
+            this.nodeUri = nodeUri;
+            this.type = type;
+            this.action = action;
+            this.name = name;
+        }
+
+        /*
+         * (non-Javadoc)
+         * 
+         * @see java.lang.Thread#run()
+         */
+        @Override
+        public void run() {
+            RestClient restClient = new RestClient(nodeUri);
+            try {
+                restClient.wipeCache(this.type, this.action, this.name);
+            } catch (IOException e) {
+                logger.warn("Thread failed during wipe cache at " + type + "." + action + "." + name + ", " + e.toString());
+            }
+        }
+    }
+
+    public enum EVENT {
+        CREATE("create"), UPDATE("update"), DROP("drop");
+        private String text;
+
+        private EVENT(String text) {
+            this.text = text;
+        }
+
+        public String getType() {
+            return text;
+        }
+
+        public static EVENT getEvent(String event) {
+            for (EVENT one : values()) {
+                if (one.getType().equalsIgnoreCase(event)) {
+                    return one;
+                }
+            }
+
+            return null;
+        }
+    }
+
+    public enum TYPE {
+        CUBE("cube"), METADATA("metadata"), PROJECT("project");
+        private String text;
+
+        private TYPE(String text) {
+            this.text = text;
+        }
+
+        public String getType() {
+            return text;
+        }
+
+        /**
+         * @param type
+         * @return
+         */
+        public static TYPE getType(String type) {
+            for (TYPE one : values()) {
+                if (one.getType().equalsIgnoreCase(type)) {
+                    return one;
+                }
+            }
+
+            return null;
+        }
+    }
+
+    public class BroadcastEvent {
+        private String type;
+        private String action;
+        private String name;
+
+        public BroadcastEvent(String type, String action, String name) {
+            super();
+            this.type = type;
+            this.action = action;
+            this.name = name;
+        }
+
+        public String getType() {
+            return type;
+        }
+
+        public void setType(String type) {
+            this.type = type;
+        }
+
+        public String getAction() {
+            return action;
+        }
+
+        public void setAction(String action) {
+            this.action = action;
+        }
+
+        public String getName() {
+            return name;
+        }
+
+        public void setName(String name) {
+            this.name = name;
+        }
+
+        @Override
+        public int hashCode() {
+            final int prime = 31;
+            int result = 1;
+            result = prime * result + getOuterType().hashCode();
+            result = prime * result + ((action == null) ? 0 : action.hashCode());
+            result = prime * result + ((name == null) ? 0 : name.hashCode());
+            result = prime * result + ((type == null) ? 0 : type.hashCode());
+            return result;
+        }
+
+        @Override
+        public boolean equals(Object obj) {
+            if (this == obj)
+                return true;
+            if (obj == null)
+                return false;
+            if (getClass() != obj.getClass())
+                return false;
+            BroadcastEvent other = (BroadcastEvent) obj;
+            if (!getOuterType().equals(other.getOuterType()))
+                return false;
+            if (action == null) {
+                if (other.action != null)
+                    return false;
+            } else if (!action.equals(other.action))
+                return false;
+            if (name == null) {
+                if (other.name != null)
+                    return false;
+            } else if (!name.equals(other.name))
+                return false;
+            if (type == null) {
+                if (other.type != null)
+                    return false;
+            } else if (!type.equals(other.type))
+                return false;
+            return true;
+        }
+
+        private Broadcaster getOuterType() {
+            return Broadcaster.this;
+        }
+
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/common/src/main/java/com/kylinolap/common/restclient/MultiValueCache.java
----------------------------------------------------------------------
diff --git a/common/src/main/java/com/kylinolap/common/restclient/MultiValueCache.java b/common/src/main/java/com/kylinolap/common/restclient/MultiValueCache.java
new file mode 100644
index 0000000..3ecaac6
--- /dev/null
+++ b/common/src/main/java/com/kylinolap/common/restclient/MultiValueCache.java
@@ -0,0 +1,86 @@
+/*
+ * 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.common.restclient;
+
+import java.util.Set;
+
+import com.google.common.collect.HashMultimap;
+
+/**
+ * @author xjiang
+ * 
+ */
+public class MultiValueCache<K, V> extends AbstractRestCache<K, V> {
+
+    private final HashMultimap<K, V> innerCache;
+
+    public MultiValueCache(Broadcaster.TYPE syncType) {
+        super(syncType);
+        innerCache = HashMultimap.create();
+    }
+
+    public void put(K key, V value) {
+        Broadcaster.EVENT eventType = innerCache.containsKey(key) ? Broadcaster.EVENT.UPDATE : Broadcaster.EVENT.CREATE;
+        synchronized (this) {
+            innerCache.put(key, value);
+        }
+        syncRemote(key, eventType);
+    }
+
+    public void putLocal(K key, V value) {
+        synchronized (this) {
+            innerCache.put(key, value);
+        }
+    }
+
+    public void remove(K key) {
+        if (innerCache.containsKey(key)) {
+            innerCache.removeAll(key);
+            syncRemote(key, Broadcaster.EVENT.DROP);
+        }
+    }
+
+    public void removeLocal(K key) {
+        if (innerCache.containsKey(key)) {
+            innerCache.removeAll(key);
+        }
+    }
+
+    public void clear() {
+        innerCache.clear();
+    }
+
+    public int size() {
+        return innerCache.size();
+    }
+
+    public Set<V> get(K key) {
+        return innerCache.get(key);
+    }
+
+    public Set<K> keySet() {
+        return innerCache.keySet();
+    }
+
+    public boolean containsKey(Object key) {
+        return innerCache.containsKey(key);
+    }
+
+    public boolean containsEntry(Object key, Object value) {
+        return innerCache.containsEntry(key, value);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/common/src/main/java/com/kylinolap/common/restclient/RestClient.java
----------------------------------------------------------------------
diff --git a/common/src/main/java/com/kylinolap/common/restclient/RestClient.java b/common/src/main/java/com/kylinolap/common/restclient/RestClient.java
new file mode 100644
index 0000000..5f1367f
--- /dev/null
+++ b/common/src/main/java/com/kylinolap/common/restclient/RestClient.java
@@ -0,0 +1,127 @@
+/*
+ * 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.common.restclient;
+
+import java.io.IOException;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+
+import org.apache.commons.httpclient.Credentials;
+import org.apache.commons.httpclient.HttpClient;
+import org.apache.commons.httpclient.HttpException;
+import org.apache.commons.httpclient.HttpMethod;
+import org.apache.commons.httpclient.UsernamePasswordCredentials;
+import org.apache.commons.httpclient.auth.AuthScope;
+import org.apache.commons.httpclient.methods.GetMethod;
+import org.apache.commons.httpclient.methods.PutMethod;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.codehaus.jettison.json.JSONException;
+import org.codehaus.jettison.json.JSONObject;
+
+/**
+ * @author yangli9
+ */
+public class RestClient {
+
+    String host;
+    int port;
+    String baseUrl;
+    String userName;
+    String password;
+    HttpClient client;
+
+    private static Pattern fullRestPattern = Pattern.compile("(?:([^:]+)[:]([^@]+)[@])?([^:]+)(?:[:](\\d+))?");
+
+    public static boolean matchFullRestPattern(String uri) {
+        Matcher m = fullRestPattern.matcher(uri);
+        return m.matches();
+    }
+
+    /**
+     * @param uri
+     *            "user:pwd@host:port"
+     */
+    public RestClient(String uri) {
+        Matcher m = fullRestPattern.matcher(uri);
+        if (!m.matches())
+            throw new IllegalArgumentException("URI: " + uri + " -- does not match pattern " + fullRestPattern);
+
+        String user = m.group(1);
+        String pwd = m.group(2);
+        String host = m.group(3);
+        String portStr = m.group(4);
+        int port = Integer.parseInt(portStr == null ? "7070" : portStr);
+
+        init(host, port, user, pwd);
+    }
+
+    private void init(String host, int port, String userName, String password) {
+        this.host = host;
+        this.port = port;
+        this.userName = userName;
+        this.password = password;
+        this.baseUrl = "http://" + host + ":" + port + "/kylin/api";
+
+        client = new HttpClient();
+
+        if (userName != null && password != null) {
+            client.getParams().setAuthenticationPreemptive(true);
+            Credentials creds = new UsernamePasswordCredentials(userName, password);
+            client.getState().setCredentials(new AuthScope(host, port, AuthScope.ANY_REALM), creds);
+        }
+    }
+
+    public void wipeCache(String type, String action, String name) throws IOException {
+        String url = baseUrl + "/cache/" + type + "/" + name + "/" + action;
+        HttpMethod get = new PutMethod(url);
+
+        try {
+            int code = client.executeMethod(get);
+            String msg = Bytes.toString(get.getResponseBody());
+
+            if (code != 200)
+                throw new IOException("Invalid response " + code + " with cache wipe url " + url + "\n" + msg);
+
+        } catch (HttpException ex) {
+            throw new IOException(ex);
+        } finally {
+            get.releaseConnection();
+        }
+    }
+
+    public String getKylinProperties() throws IOException {
+        String url = baseUrl + "/admin/config";
+        HttpMethod get = new GetMethod(url);
+        try {
+            int code = client.executeMethod(get);
+            String msg = Bytes.toString(get.getResponseBody());
+            JSONObject obj = new JSONObject(msg);
+            msg = obj.getString("config");
+
+            if (code != 200)
+                throw new IOException("Invalid response " + code + " with cache wipe url " + url + "\n" + msg);
+
+            return msg;
+
+        } catch (JSONException e) {
+            throw new IOException("Error when parsing json response from REST");
+        } finally {
+            get.releaseConnection();
+        }
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/common/src/main/java/com/kylinolap/common/restclient/SingleValueCache.java
----------------------------------------------------------------------
diff --git a/common/src/main/java/com/kylinolap/common/restclient/SingleValueCache.java b/common/src/main/java/com/kylinolap/common/restclient/SingleValueCache.java
new file mode 100644
index 0000000..394697c
--- /dev/null
+++ b/common/src/main/java/com/kylinolap/common/restclient/SingleValueCache.java
@@ -0,0 +1,86 @@
+/*
+ * 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.common.restclient;
+
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+
+/**
+ * @author xjiang
+ * 
+ */
+public class SingleValueCache<K, V> extends AbstractRestCache<K, V> {
+
+    private final Map<K, V> innerCache;
+
+    public SingleValueCache(Broadcaster.TYPE syncType) {
+        super(syncType);
+        innerCache = new ConcurrentHashMap<K, V>();
+    }
+
+    public void put(K key, V value) {
+        Broadcaster.EVENT eventType = innerCache.containsKey(key) ? Broadcaster.EVENT.UPDATE : Broadcaster.EVENT.CREATE;
+        innerCache.put(key, value);
+        syncRemote(key, eventType);
+    }
+
+    public void putLocal(K key, V value) {
+        innerCache.put(key, value);
+    }
+
+    public void remove(K key) {
+        if (innerCache.containsKey(key)) {
+            innerCache.remove(key);
+            syncRemote(key, Broadcaster.EVENT.DROP);
+        }
+    }
+
+    public void removeLocal(K key) {
+        innerCache.remove(key);
+    }
+
+    public void clear() {
+        innerCache.clear();
+    }
+
+    public int size() {
+        return innerCache.size();
+    }
+
+    public V get(K key) {
+        return innerCache.get(key);
+    }
+
+    public Collection<V> values() {
+        return innerCache.values();
+    }
+
+    public boolean containsKey(String key) {
+        return innerCache.containsKey(key);
+    }
+
+    public Map<K, V> getMap() {
+        return Collections.unmodifiableMap(innerCache);
+    }
+
+    public Set<K> keySet() {
+        return innerCache.keySet();
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/common/src/main/java/com/kylinolap/common/util/AbstractKylinTestCase.java
----------------------------------------------------------------------
diff --git a/common/src/main/java/com/kylinolap/common/util/AbstractKylinTestCase.java b/common/src/main/java/com/kylinolap/common/util/AbstractKylinTestCase.java
new file mode 100644
index 0000000..9b1005e
--- /dev/null
+++ b/common/src/main/java/com/kylinolap/common/util/AbstractKylinTestCase.java
@@ -0,0 +1,41 @@
+/*
+ * 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.common.util;
+
+import com.kylinolap.common.KylinConfig;
+
+/**
+ * @author ysong1
+ * 
+ */
+public abstract class AbstractKylinTestCase {
+
+    public static final String LOCALMETA_TEST_DATA = "../examples/test_case_data/localmeta";
+
+    public static final String MINICLUSTER_TEST_DATA = "../examples/test_case_data/minicluster";
+    
+    public static final String SANDBOX_TEST_DATA = "../examples/test_case_data/sandbox";
+    
+    public abstract void createTestMetadata();
+
+    public abstract void cleanupTestMetadata();
+
+    public KylinConfig getTestConfig() {
+        return KylinConfig.getInstanceFromEnv();
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/common/src/main/java/com/kylinolap/common/util/Array.java
----------------------------------------------------------------------
diff --git a/common/src/main/java/com/kylinolap/common/util/Array.java b/common/src/main/java/com/kylinolap/common/util/Array.java
new file mode 100644
index 0000000..0b4165f
--- /dev/null
+++ b/common/src/main/java/com/kylinolap/common/util/Array.java
@@ -0,0 +1,69 @@
+/*
+ * 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.common.util;
+
+import java.util.Arrays;
+
+/*
+ * An array with correct equals(), hashCode(), compareTo() and toString()
+ */
+public class Array<T> implements Comparable<Array<T>> {
+    public T[] data;
+
+    public Array(T[] data) {
+        this.data = data;
+    }
+
+    public String toString() {
+        return Arrays.toString(data);
+    }
+
+    @Override
+    public boolean equals(Object o) {
+        if (o != null && o instanceof Array) {
+            return Arrays.equals(this.data, ((Array<?>) o).data);
+        }
+        return false;
+    }
+
+    @Override
+    public int hashCode() {
+        return Arrays.hashCode(data);
+    }
+
+    @Override
+    public int compareTo(Array<T> other) {
+        return compare(this.data, other.data, null);
+    }
+
+    @SuppressWarnings("unchecked")
+    public static <T> int compare(T[] a, T[] b, boolean[] ascending) {
+        int r = 0;
+        int n = Math.min(a.length, b.length);
+        boolean asc = true;
+
+        for (int i = 0; i < n; i++) {
+            r = ((Comparable<T>) a[i]).compareTo(b[i]);
+            if (r != 0) {
+                asc = (ascending != null && ascending.length > i) ? ascending[i] : true;
+                return asc ? r : -r;
+            }
+        }
+        return a.length - b.length;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/common/src/main/java/com/kylinolap/common/util/ByteArray.java
----------------------------------------------------------------------
diff --git a/common/src/main/java/com/kylinolap/common/util/ByteArray.java b/common/src/main/java/com/kylinolap/common/util/ByteArray.java
new file mode 100644
index 0000000..d6aee89
--- /dev/null
+++ b/common/src/main/java/com/kylinolap/common/util/ByteArray.java
@@ -0,0 +1,57 @@
+/*
+ * 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.common.util;
+
+import java.util.Arrays;
+
+import org.apache.hadoop.hbase.util.Bytes;
+
+/**
+ * @author yangli9
+ */
+public class ByteArray implements Comparable<ByteArray> {
+
+    public byte[] data;
+
+    public ByteArray(byte[] data) {
+        this.data = data;
+    }
+
+    @Override
+    public int hashCode() {
+        return Bytes.hashCode(data);
+    }
+
+    @Override
+    public boolean equals(Object obj) {
+        if (this == obj)
+            return true;
+        if (obj == null)
+            return false;
+        if (getClass() != obj.getClass())
+            return false;
+        ByteArray other = (ByteArray) obj;
+        if (!Arrays.equals(data, other.data))
+            return false;
+        return true;
+    }
+
+    @Override
+    public int compareTo(ByteArray o) {
+        return Bytes.compareTo(this.data, o.data);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/common/src/main/java/com/kylinolap/common/util/BytesSerializer.java
----------------------------------------------------------------------
diff --git a/common/src/main/java/com/kylinolap/common/util/BytesSerializer.java b/common/src/main/java/com/kylinolap/common/util/BytesSerializer.java
new file mode 100644
index 0000000..5d46037
--- /dev/null
+++ b/common/src/main/java/com/kylinolap/common/util/BytesSerializer.java
@@ -0,0 +1,33 @@
+/*
+ * 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.common.util;
+
+import java.nio.ByteBuffer;
+
+/**
+ * @author yangli9
+ * 
+ */
+public interface BytesSerializer<T> {
+
+    public static final int SERIALIZE_BUFFER_SIZE = 65536;
+
+    abstract public void serialize(T value, ByteBuffer out);
+
+    abstract public T deserialize(ByteBuffer in);
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/common/src/main/java/com/kylinolap/common/util/BytesUtil.java
----------------------------------------------------------------------
diff --git a/common/src/main/java/com/kylinolap/common/util/BytesUtil.java b/common/src/main/java/com/kylinolap/common/util/BytesUtil.java
new file mode 100644
index 0000000..93d72a4
--- /dev/null
+++ b/common/src/main/java/com/kylinolap/common/util/BytesUtil.java
@@ -0,0 +1,348 @@
+/*
+ * 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.common.util;
+
+import java.io.ByteArrayOutputStream;
+import java.io.DataOutputStream;
+import java.io.IOException;
+import java.io.UnsupportedEncodingException;
+import java.nio.ByteBuffer;
+
+import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.io.Writable;
+
+public class BytesUtil {
+
+    public static final byte[] EMPTY_BYTE_ARRAY = new byte[0];
+
+    public static void writeLong(long num, byte[] bytes, int offset, int size) {
+        for (int i = offset + size - 1; i >= offset; i--) {
+            bytes[i] = (byte) num;
+            num >>>= 8;
+        }
+    }
+
+    public static void writeUnsigned(int num, byte[] bytes, int offset, int size) {
+        for (int i = offset + size - 1; i >= offset; i--) {
+            bytes[i] = (byte) num;
+            num >>>= 8;
+        }
+    }
+
+    public static long readLong(byte[] bytes, int offset, int size) {
+        long integer = 0;
+        for (int i = offset, n = offset + size; i < n; i++) {
+            integer <<= 8;
+            integer |= (long) bytes[i] & 0xFF;
+        }
+        return integer;
+    }
+
+    public static int readUnsigned(byte[] bytes, int offset, int size) {
+        int integer = 0;
+        for (int i = offset, n = offset + size; i < n; i++) {
+            integer <<= 8;
+            integer |= (int) bytes[i] & 0xFF;
+        }
+        return integer;
+    }
+
+    public static void writeSigned(int num, byte[] bytes, int offset, int size) {
+        writeUnsigned(num, bytes, offset, size);
+    }
+
+    public static int readSigned(byte[] bytes, int offset, int size) {
+        int integer = (bytes[offset] & 0x80) == 0 ? 0 : -1;
+        for (int i = offset, n = offset + size; i < n; i++) {
+            integer <<= 8;
+            integer |= (int) bytes[i] & 0xFF;
+        }
+        return integer;
+    }
+
+    /**
+     * No. bytes needed to store a value as big as the given
+     */
+    public static int sizeForValue(int maxValue) {
+        int size = 0;
+        while (maxValue > 0) {
+            size++;
+            maxValue >>>= 8;
+        }
+        return size;
+    }
+
+    public static int compareByteUnsigned(byte b1, byte b2) {
+        int i1 = (int) b1 & 0xFF;
+        int i2 = (int) b2 & 0xFF;
+        return i1 - i2;
+    }
+
+    public static byte[] subarray(byte[] bytes, int start, int end) {
+        byte[] r = new byte[end - start];
+        System.arraycopy(bytes, start, r, 0, r.length);
+        return r;
+    }
+
+    public static int compareBytes(byte[] src, int srcOffset, byte[] dst, int dstOffset, int length) {
+        int r = 0;
+        for (int i = 0; i < length; i++) {
+            r = src[srcOffset + i] - dst[dstOffset + i];
+            if (r != 0)
+                break;
+        }
+        return r;
+    }
+
+    // from WritableUtils
+    // ============================================================================
+
+    public static void writeVInt(int i, ByteBuffer out) {
+        writeVLong(i, out);
+    }
+
+    public static void writeVLong(long i, ByteBuffer out) {
+        if (i >= -112 && i <= 127) {
+            out.put((byte) i);
+            return;
+        }
+
+        int len = -112;
+        if (i < 0) {
+            i ^= -1L; // take one's complement'
+            len = -120;
+        }
+
+        long tmp = i;
+        while (tmp != 0) {
+            tmp = tmp >> 8;
+            len--;
+        }
+
+        out.put((byte) len);
+
+        len = (len < -120) ? -(len + 120) : -(len + 112);
+
+        for (int idx = len; idx != 0; idx--) {
+            int shiftbits = (idx - 1) * 8;
+            long mask = 0xFFL << shiftbits;
+            out.put((byte) ((i & mask) >> shiftbits));
+        }
+    }
+
+    public static long readVLong(ByteBuffer in) {
+        byte firstByte = in.get();
+        int len = decodeVIntSize(firstByte);
+        if (len == 1) {
+            return firstByte;
+        }
+        long i = 0;
+        for (int idx = 0; idx < len - 1; idx++) {
+            byte b = in.get();
+            i = i << 8;
+            i = i | (b & 0xFF);
+        }
+        return (isNegativeVInt(firstByte) ? (i ^ -1L) : i);
+    }
+
+    public static int readVInt(ByteBuffer in) {
+        long n = readVLong(in);
+        if ((n > Integer.MAX_VALUE) || (n < Integer.MIN_VALUE)) {
+            throw new IllegalArgumentException("value too long to fit in integer");
+        }
+        return (int) n;
+    }
+
+    private static boolean isNegativeVInt(byte value) {
+        return value < -120 || (value >= -112 && value < 0);
+    }
+
+    private static int decodeVIntSize(byte value) {
+        if (value >= -112) {
+            return 1;
+        } else if (value < -120) {
+            return -119 - value;
+        }
+        return -111 - value;
+    }
+
+    public static void writeUnsigned(int num, int size, ByteBuffer out) {
+        for (int i = 0; i < size; i++) {
+            out.put((byte) num);
+            num >>>= 8;
+        }
+    }
+
+    public static int readUnsigned(ByteBuffer in, int size) {
+        int integer = 0;
+        int mask = 0xff;
+        int shift = 0;
+        for (int i = 0; i < size; i++) {
+            integer |= (in.get() << shift) & mask;
+            mask = mask << 8;
+            shift += 8;
+        }
+        return integer;
+    }
+
+    public static void writeLong(long num, ByteBuffer out) {
+        for (int i = 0; i < 8; i++) {
+            out.put((byte) num);
+            num >>>= 8;
+        }
+    }
+
+    public static long readLong(ByteBuffer in) {
+        long integer = 0;
+        int mask = 0xff;
+        int shift = 0;
+        for (int i = 0; i < 8; i++) {
+            integer |= (in.get() << shift) & mask;
+            mask = mask << 8;
+            shift += 8;
+        }
+        return integer;
+    }
+
+    public static void writeUTFString(String str, ByteBuffer out) {
+        byte[] bytes = str == null ? null : Bytes.toBytes(str);
+        writeByteArray(bytes, out);
+    }
+
+    public static String readUTFString(ByteBuffer in) {
+        byte[] bytes = readByteArray(in);
+        return bytes == null ? null : Bytes.toString(bytes);
+    }
+
+    public static void writeAsciiString(String str, ByteBuffer out) {
+        if (str == null) {
+            BytesUtil.writeVInt(-1, out);
+            return;
+        }
+        int len = str.length();
+        BytesUtil.writeVInt(len, out);
+        for (int i = 0; i < len; i++) {
+            out.put((byte) str.charAt(i));
+        }
+    }
+
+    public static String readAsciiString(ByteBuffer in) {
+        int len = BytesUtil.readVInt(in);
+        if (len < 0) {
+            return null;
+        }
+        String result;
+        try {
+            if (in.hasArray()) {
+                int pos = in.position();
+                result = new String(in.array(), pos, len, "ISO-8859-1");
+                in.position(pos + len);
+            } else {
+                byte[] tmp = new byte[len];
+                in.get(tmp);
+                result = new String(tmp, "ISO-8859-1");
+            }
+        } catch (UnsupportedEncodingException e) {
+            throw new RuntimeException(e); // never happen
+        }
+        return result;
+    }
+
+    public static void writeAsciiStringArray(String[] strs, ByteBuffer out) {
+        writeVInt(strs.length, out);
+        for (int i = 0; i < strs.length; i++)
+            writeAsciiString(strs[i], out);
+    }
+
+    public static String[] readAsciiStringArray(ByteBuffer in) {
+        int len = readVInt(in);
+        String[] strs = new String[len];
+        for (int i = 0; i < len; i++)
+            strs[i] = readAsciiString(in);
+        return strs;
+    }
+
+    public static void writeIntArray(int[] array, ByteBuffer out) {
+        if (array == null) {
+            writeVInt(-1, out);
+            return;
+        }
+        writeVInt(array.length, out);
+        out.asIntBuffer().put(array);
+    }
+
+    public static int[] readIntArray(ByteBuffer in) {
+        int len = readVInt(in);
+        if (len < 0)
+            return null;
+        int[] array = new int[len];
+        in.asIntBuffer().get(array);
+        return array;
+    }
+
+    public static void writeByteArray(byte[] array, ByteBuffer out) {
+        if (array == null) {
+            writeVInt(-1, out);
+            return;
+        }
+        writeVInt(array.length, out);
+        out.put(array);
+    }
+
+    public static byte[] readByteArray(ByteBuffer in) {
+        int len = readVInt(in);
+        if (len < 0)
+            return null;
+
+        byte[] array = new byte[len];
+        in.get(array);
+        return array;
+    }
+
+    public static byte[] toBytes(Writable writable) {
+        try {
+            ByteArrayOutputStream bout = new ByteArrayOutputStream();
+            DataOutputStream out = new DataOutputStream(bout);
+            writable.write(out);
+            out.close();
+            bout.close();
+            return bout.toByteArray();
+        } catch (IOException e) {
+            throw new RuntimeException(e);
+        }
+    }
+
+    public static String toHex(byte[] array) {
+        return toHex(new ImmutableBytesWritable(array));
+    }
+
+    public static String toHex(ImmutableBytesWritable bytes) {
+        byte[] array = bytes.get();
+        int offset = bytes.getOffset();
+        int length = bytes.getLength();
+        StringBuilder sb = new StringBuilder(length * 4);
+        for (int i = 0; i < length; i++) {
+            int b = array[offset + i];
+            sb.append(String.format("\\x%02X", b & 0xFF));
+        }
+        return sb.toString();
+    }
+
+    public static void main(String[] args) throws Exception {
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/common/src/main/java/com/kylinolap/common/util/ClasspathUtil.java
----------------------------------------------------------------------
diff --git a/common/src/main/java/com/kylinolap/common/util/ClasspathUtil.java b/common/src/main/java/com/kylinolap/common/util/ClasspathUtil.java
new file mode 100644
index 0000000..6be3708
--- /dev/null
+++ b/common/src/main/java/com/kylinolap/common/util/ClasspathUtil.java
@@ -0,0 +1,43 @@
+/*
+ * 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.common.util;
+
+import java.io.File;
+import java.lang.reflect.Method;
+import java.net.URL;
+import java.net.URLClassLoader;
+
+/**
+ * @author xduo
+ * 
+ */
+public class ClasspathUtil {
+
+    public static void addClasspath(String path) throws Exception {
+        System.out.println("Adding path " + path + " to class path");
+        File file = new File(path);
+
+        if (file.exists()) {
+            URLClassLoader urlClassLoader = (URLClassLoader) ClassLoader.getSystemClassLoader();
+            Class<URLClassLoader> urlClass = URLClassLoader.class;
+            Method method = urlClass.getDeclaredMethod("addURL", new Class[] { URL.class });
+            method.setAccessible(true);
+            method.invoke(urlClassLoader, new Object[] { file.toURI().toURL() });
+        }
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/common/src/main/java/com/kylinolap/common/util/CliCommandExecutor.java
----------------------------------------------------------------------
diff --git a/common/src/main/java/com/kylinolap/common/util/CliCommandExecutor.java b/common/src/main/java/com/kylinolap/common/util/CliCommandExecutor.java
new file mode 100644
index 0000000..3f4ab7b
--- /dev/null
+++ b/common/src/main/java/com/kylinolap/common/util/CliCommandExecutor.java
@@ -0,0 +1,133 @@
+/*
+ * 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.common.util;
+
+import java.io.ByteArrayOutputStream;
+import java.io.File;
+import java.io.IOException;
+
+import org.apache.commons.io.FileUtils;
+import org.apache.commons.io.IOUtils;
+import org.apache.hadoop.hbase.util.Pair;
+
+/**
+ * @author yangli9
+ */
+public class CliCommandExecutor {
+
+    private String remoteHost;
+    private String remoteUser;
+    private String remotePwd;
+    private int remoteTimeoutSeconds = 3600;
+
+    public CliCommandExecutor() {
+    }
+
+    public void setRunAtRemote(String host, String user, String pwd) {
+        this.remoteHost = host;
+        this.remoteUser = user;
+        this.remotePwd = pwd;
+    }
+
+    public void setRunAtLocal() {
+        this.remoteHost = null;
+        this.remoteUser = null;
+        this.remotePwd = null;
+    }
+    
+    public void copyFile(String localFile, String destDir) throws IOException {
+        if (remoteHost == null)
+            copyNative(localFile, destDir);
+        else
+            copyRemote(localFile, destDir);
+    }
+
+    private void copyNative(String localFile, String destDir) throws IOException {
+        File src = new File(localFile);
+        File dest = new File(destDir, src.getName());
+        FileUtils.copyFile(src, dest);
+    }
+
+    private void copyRemote(String localFile, String destDir) throws IOException {
+        SSHClient ssh = new SSHClient(remoteHost, remoteUser, remotePwd, null);
+        try {
+            ssh.scpFileToRemote(localFile, destDir);
+        } catch (IOException e) {
+            throw e;
+        } catch (Exception e) {
+            throw new IOException(e.getMessage(), e);
+        }
+    }
+
+    public String execute(String command) throws IOException {
+        Pair<Integer, String> r;
+        if (remoteHost == null)
+            r = runNativeCommand(command);
+        else
+            r = runRemoteCommand(command);
+
+        if (r.getFirst() != 0)
+            throw new IOException("OS command error exit with " + r.getFirst() + " -- " + command + "\n" + r.getSecond());
+
+        return r.getSecond();
+    }
+
+    private Pair<Integer, String> runRemoteCommand(String command) throws IOException {
+        SSHClient ssh = new SSHClient(remoteHost, remoteUser, remotePwd, null);
+
+        SSHClientOutput sshOutput;
+        try {
+            sshOutput = ssh.execCommand(command, remoteTimeoutSeconds);
+            int exitCode = sshOutput.getExitCode();
+            String output = sshOutput.getText();
+            return new Pair<Integer, String>(exitCode, output);
+        } catch (IOException e) {
+            throw e;
+        } catch (Exception e) {
+            throw new IOException(e.getMessage(), e);
+        }
+    }
+
+    private Pair<Integer, String> runNativeCommand(String command) throws IOException {
+        String[] cmd = new String[3];
+        String osName = System.getProperty("os.name");
+        if (osName.startsWith("Windows")) {
+            cmd[0] = "cmd.exe";
+            cmd[1] = "/C";
+        } else {
+            cmd[0] = "/bin/bash";
+            cmd[1] = "-c";
+        }
+        cmd[2] = command;
+
+        ProcessBuilder builder = new ProcessBuilder(cmd);
+        builder.redirectErrorStream(true);
+        Process proc = builder.start();
+
+        ByteArrayOutputStream buf = new ByteArrayOutputStream();
+        IOUtils.copy(proc.getInputStream(), buf);
+        String output = buf.toString("UTF-8");
+
+        try {
+            int exitCode = proc.waitFor();
+            return new Pair<Integer, String>(exitCode, output);
+        } catch (InterruptedException e) {
+            throw new IOException(e);
+        }
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/common/src/main/java/com/kylinolap/common/util/HBaseMetadataTestCase.java
----------------------------------------------------------------------
diff --git a/common/src/main/java/com/kylinolap/common/util/HBaseMetadataTestCase.java b/common/src/main/java/com/kylinolap/common/util/HBaseMetadataTestCase.java
new file mode 100644
index 0000000..a1dda9a
--- /dev/null
+++ b/common/src/main/java/com/kylinolap/common/util/HBaseMetadataTestCase.java
@@ -0,0 +1,50 @@
+/*
+ * 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.common.util;
+
+import com.kylinolap.common.KylinConfig;
+
+/**
+ * @author ysong1
+ */
+public class HBaseMetadataTestCase extends AbstractKylinTestCase {
+    
+    public static void staticCreateTestMetadata() {
+
+        KylinConfig.destoryInstance();
+
+        if (System.getProperty(KylinConfig.KYLIN_CONF) == null && System.getenv(KylinConfig.KYLIN_CONF) == null)
+            System.setProperty(KylinConfig.KYLIN_CONF, SANDBOX_TEST_DATA);
+
+    }
+    
+    public static void staticCleanupTestMetadata() {
+        System.clearProperty(KylinConfig.KYLIN_CONF);
+        KylinConfig.destoryInstance();
+    }
+
+    @Override
+    public void createTestMetadata() {
+        staticCreateTestMetadata();
+    }
+
+    @Override
+    public void cleanupTestMetadata() {
+        staticCleanupTestMetadata();
+    }
+
+}


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

Posted by lu...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/common/src/main/java/com/kylinolap/common/util/HBaseRegionSizeCalculator.java
----------------------------------------------------------------------
diff --git a/common/src/main/java/com/kylinolap/common/util/HBaseRegionSizeCalculator.java b/common/src/main/java/com/kylinolap/common/util/HBaseRegionSizeCalculator.java
new file mode 100644
index 0000000..36ffc29
--- /dev/null
+++ b/common/src/main/java/com/kylinolap/common/util/HBaseRegionSizeCalculator.java
@@ -0,0 +1,125 @@
+/*
+ * 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.
+ */
+/** This class will come with HBase 2.0 in package org.apache.hadoop.hbase.util **/
+package com.kylinolap.common.util;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Map;
+import java.util.Set;
+import java.util.TreeMap;
+import java.util.TreeSet;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.ClusterStatus;
+import org.apache.hadoop.hbase.HRegionInfo;
+import org.apache.hadoop.hbase.RegionLoad;
+import org.apache.hadoop.hbase.ServerLoad;
+import org.apache.hadoop.hbase.ServerName;
+import org.apache.hadoop.hbase.client.HBaseAdmin;
+import org.apache.hadoop.hbase.client.HTable;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class HBaseRegionSizeCalculator {
+
+    private static final Logger logger = LoggerFactory.getLogger(HBaseRegionSizeCalculator.class);
+
+    /**
+     * Maps each region to its size in bytes.
+     **/
+    private final Map<byte[], Long> sizeMap = new TreeMap<byte[], Long>(Bytes.BYTES_COMPARATOR);
+
+    static final String ENABLE_REGIONSIZECALCULATOR = "hbase.regionsizecalculator.enable";
+
+    /**
+     * Computes size of each region for table and given column families.
+     * */
+    public HBaseRegionSizeCalculator(HTable table) throws IOException {
+        this(table, new HBaseAdmin(table.getConfiguration()));
+    }
+
+    /** Constructor for unit testing */
+    HBaseRegionSizeCalculator(HTable table, HBaseAdmin hBaseAdmin) throws IOException {
+
+        try {
+            if (!enabled(table.getConfiguration())) {
+                logger.info("Region size calculation disabled.");
+                return;
+            }
+
+            logger.info("Calculating region sizes for table \"" + new String(table.getTableName()) + "\".");
+
+            // Get regions for table.
+            Set<HRegionInfo> tableRegionInfos = table.getRegionLocations().keySet();
+            Set<byte[]> tableRegions = new TreeSet<byte[]>(Bytes.BYTES_COMPARATOR);
+
+            for (HRegionInfo regionInfo : tableRegionInfos) {
+                tableRegions.add(regionInfo.getRegionName());
+            }
+
+            ClusterStatus clusterStatus = hBaseAdmin.getClusterStatus();
+            Collection<ServerName> servers = clusterStatus.getServers();
+            final long megaByte = 1024L * 1024L;
+
+            // Iterate all cluster regions, filter regions from our table and
+            // compute their size.
+            for (ServerName serverName : servers) {
+                ServerLoad serverLoad = clusterStatus.getLoad(serverName);
+
+                for (RegionLoad regionLoad : serverLoad.getRegionsLoad().values()) {
+                    byte[] regionId = regionLoad.getName();
+
+                    if (tableRegions.contains(regionId)) {
+
+                        long regionSizeBytes = regionLoad.getStorefileSizeMB() * megaByte;
+                        sizeMap.put(regionId, regionSizeBytes);
+
+                        // logger.info("Region " + regionLoad.getNameAsString()
+                        // + " has size " + regionSizeBytes);
+                    }
+                }
+            }
+        } finally {
+            hBaseAdmin.close();
+        }
+
+    }
+
+    boolean enabled(Configuration configuration) {
+        return configuration.getBoolean(ENABLE_REGIONSIZECALCULATOR, true);
+    }
+
+    /**
+     * Returns size of given region in bytes. Returns 0 if region was not found.
+     **/
+    public long getRegionSize(byte[] regionId) {
+        Long size = sizeMap.get(regionId);
+        if (size == null) {
+            logger.info("Unknown region:" + Arrays.toString(regionId));
+            return 0;
+        } else {
+            return size;
+        }
+    }
+
+    public Map<byte[], Long> getRegionSizeMap() {
+        return Collections.unmodifiableMap(sizeMap);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/common/src/main/java/com/kylinolap/common/util/HadoopUtil.java
----------------------------------------------------------------------
diff --git a/common/src/main/java/com/kylinolap/common/util/HadoopUtil.java b/common/src/main/java/com/kylinolap/common/util/HadoopUtil.java
new file mode 100644
index 0000000..77767fc
--- /dev/null
+++ b/common/src/main/java/com/kylinolap/common/util/HadoopUtil.java
@@ -0,0 +1,106 @@
+/*
+ * 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.common.util;
+
+import java.io.IOException;
+import java.net.InetAddress;
+import java.net.URI;
+import java.net.URISyntaxException;
+import java.net.UnknownHostException;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+
+import org.apache.commons.lang.StringUtils;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.hbase.HBaseConfiguration;
+import org.apache.hadoop.hbase.HConstants;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class HadoopUtil {
+    private static final Logger logger = LoggerFactory.getLogger(HadoopUtil.class);
+
+    private static Configuration hadoopConfig;
+
+    public static Configuration getDefaultConfiguration() {
+        if (hadoopConfig == null) {
+            hadoopConfig = new Configuration();
+        }
+        return hadoopConfig;
+    }
+
+    public static FileSystem getFileSystem(String path) throws IOException {
+        return FileSystem.get(makeURI(path), getDefaultConfiguration());
+    }
+
+    public static URI makeURI(String filePath) {
+        try {
+            return new URI(filePath);
+        } catch (URISyntaxException e) {
+            throw new IllegalArgumentException("Cannot create FileSystem from URI: " + filePath, e);
+        }
+    }
+
+    /**
+     * e.g. "hbase:kylin-local.corp.ebay.com:2181:/hbase-unsecure"
+     */
+    public static Configuration newHBaseConfiguration(String url) {
+        Configuration conf = HBaseConfiguration.create();
+        if (StringUtils.isEmpty(url))
+            return conf;
+
+        // chop off "hbase:"
+        if (url.startsWith("hbase:") == false)
+            throw new IllegalArgumentException("hbase url must start with 'hbase:' -- " + url);
+
+        url = StringUtils.substringAfter(url, "hbase:");
+        if (StringUtils.isEmpty(url))
+            return conf;
+
+        // case of "hbase:domain.com:2181:/hbase-unsecure"
+        Pattern urlPattern = Pattern.compile("([\\w\\d\\-.]+)[:](\\d+)(?:[:](.*))?");
+        Matcher m = urlPattern.matcher(url);
+        if (m.matches() == false)
+            throw new IllegalArgumentException("HBase URL '" + url + "' is invalid, expected url is like '" + "hbase:domain.com:2181:/hbase-unsecure" + "'");
+
+        logger.debug("Creating hbase conf by parsing -- " + url);
+
+        String quorum = m.group(1);
+        try {
+            InetAddress.getByName(quorum);
+        } catch (UnknownHostException e) {
+            throw new IllegalArgumentException("Zookeeper quorum is invalid: " + quorum + "; urlString=" + url, e);
+        }
+        conf.set(HConstants.ZOOKEEPER_QUORUM, quorum);
+
+        String port = m.group(2);
+        conf.set(HConstants.ZOOKEEPER_CLIENT_PORT, port);
+
+        String znodePath = m.group(3) == null ? "" : m.group(3);
+        if (StringUtils.isEmpty(znodePath) == false)
+            conf.set(HConstants.ZOOKEEPER_ZNODE_PARENT, znodePath);
+
+        // reduce rpc retry
+        conf.set(HConstants.HBASE_CLIENT_PAUSE, "3000");
+        conf.set(HConstants.HBASE_CLIENT_RETRIES_NUMBER, "5");
+        conf.set(HConstants.HBASE_CLIENT_OPERATION_TIMEOUT, "60000");
+        // conf.set(ScannerCallable.LOG_SCANNER_ACTIVITY, "true");
+
+        return conf;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/common/src/main/java/com/kylinolap/common/util/JsonUtil.java
----------------------------------------------------------------------
diff --git a/common/src/main/java/com/kylinolap/common/util/JsonUtil.java b/common/src/main/java/com/kylinolap/common/util/JsonUtil.java
new file mode 100644
index 0000000..1ca8a90
--- /dev/null
+++ b/common/src/main/java/com/kylinolap/common/util/JsonUtil.java
@@ -0,0 +1,83 @@
+/*
+ * 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.common.util;
+
+import java.io.File;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.io.Reader;
+
+import com.fasterxml.jackson.core.JsonGenerationException;
+import com.fasterxml.jackson.core.JsonParseException;
+import com.fasterxml.jackson.core.JsonProcessingException;
+import com.fasterxml.jackson.databind.DeserializationFeature;
+import com.fasterxml.jackson.databind.JsonMappingException;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.databind.SerializationFeature;
+
+public class JsonUtil {
+
+    // reuse the object mapper to save memory footprint
+    private static final ObjectMapper mapper = new ObjectMapper();
+    private static final ObjectMapper indentMapper = new ObjectMapper();
+
+    static {
+        mapper.configure(DeserializationFeature.FAIL_ON_UNKNOWN_PROPERTIES, false);
+        indentMapper.configure(SerializationFeature.INDENT_OUTPUT, true);
+    }
+
+    public static <T> T readValue(File src, Class<T> valueType) throws IOException, JsonParseException, JsonMappingException {
+        return mapper.readValue(src, valueType);
+    }
+
+    public static <T> T readValue(String content, Class<T> valueType) throws IOException, JsonParseException, JsonMappingException {
+        return mapper.readValue(content, valueType);
+    }
+
+    public static <T> T readValue(Reader src, Class<T> valueType) throws IOException, JsonParseException, JsonMappingException {
+        return mapper.readValue(src, valueType);
+    }
+
+    public static <T> T readValue(InputStream src, Class<T> valueType) throws IOException, JsonParseException, JsonMappingException {
+        return mapper.readValue(src, valueType);
+    }
+
+    public static <T> T readValue(byte[] src, Class<T> valueType) throws IOException, JsonParseException, JsonMappingException {
+        return mapper.readValue(src, valueType);
+    }
+
+    public static void writeValueIndent(OutputStream out, Object value) throws IOException, JsonGenerationException, JsonMappingException {
+        indentMapper.writeValue(out, value);
+    }
+
+    public static void writeValue(OutputStream out, Object value) throws IOException, JsonGenerationException, JsonMappingException {
+        mapper.writeValue(out, value);
+    }
+
+    public static String writeValueAsString(Object value) throws JsonProcessingException {
+        return mapper.writeValueAsString(value);
+    }
+
+    public static byte[] writeValueAsBytes(Object value) throws JsonProcessingException {
+        return mapper.writeValueAsBytes(value);
+    }
+
+    public static String writeValueAsIndentString(Object value) throws JsonProcessingException {
+        return indentMapper.writeValueAsString(value);
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/common/src/main/java/com/kylinolap/common/util/LocalFileMetadataTestCase.java
----------------------------------------------------------------------
diff --git a/common/src/main/java/com/kylinolap/common/util/LocalFileMetadataTestCase.java b/common/src/main/java/com/kylinolap/common/util/LocalFileMetadataTestCase.java
new file mode 100644
index 0000000..f176c71
--- /dev/null
+++ b/common/src/main/java/com/kylinolap/common/util/LocalFileMetadataTestCase.java
@@ -0,0 +1,67 @@
+/*
+ * 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.common.util;
+
+import java.io.File;
+import java.io.IOException;
+
+import org.apache.commons.io.FileUtils;
+
+import com.kylinolap.common.KylinConfig;
+import com.kylinolap.common.persistence.ResourceStore;
+
+/**
+ * @author ysong1
+ */
+public class LocalFileMetadataTestCase extends AbstractKylinTestCase {
+    private String tempTestMetadataUrl = null;
+
+    @Override
+    public void createTestMetadata() {
+        KylinConfig.destoryInstance();
+
+        this.tempTestMetadataUrl = "../examples/test_metadata";
+        try {
+            FileUtils.deleteDirectory(new File(tempTestMetadataUrl));
+            FileUtils.copyDirectory(new File(LOCALMETA_TEST_DATA), new File(tempTestMetadataUrl));
+        } catch (IOException e) {
+            e.printStackTrace();
+        }
+
+        if (System.getProperty(KylinConfig.KYLIN_CONF) == null && System.getenv(KylinConfig.KYLIN_CONF) == null)
+            System.setProperty(KylinConfig.KYLIN_CONF, tempTestMetadataUrl);
+
+        KylinConfig.getInstanceFromEnv().setMetadataUrl(tempTestMetadataUrl);
+
+    }
+
+    @Override
+    public void cleanupTestMetadata() {
+        try {
+            FileUtils.deleteDirectory(new File(tempTestMetadataUrl));
+        } catch (IOException e) {
+            throw new IllegalStateException("Can't delete directory " + tempTestMetadataUrl, e);
+        }
+        System.clearProperty(KylinConfig.KYLIN_CONF);
+        KylinConfig.destoryInstance();
+        this.tempTestMetadataUrl = null;
+    }
+
+    protected ResourceStore getStore() {
+        return ResourceStore.getStore(KylinConfig.getInstanceFromEnv());
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/common/src/main/java/com/kylinolap/common/util/LongAsFloat.java
----------------------------------------------------------------------
diff --git a/common/src/main/java/com/kylinolap/common/util/LongAsFloat.java b/common/src/main/java/com/kylinolap/common/util/LongAsFloat.java
new file mode 100644
index 0000000..3d38ec2
--- /dev/null
+++ b/common/src/main/java/com/kylinolap/common/util/LongAsFloat.java
@@ -0,0 +1,5 @@
+package com.kylinolap.common.util;
+
+public class LongAsFloat {
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/common/src/main/java/com/kylinolap/common/util/MailService.java
----------------------------------------------------------------------
diff --git a/common/src/main/java/com/kylinolap/common/util/MailService.java b/common/src/main/java/com/kylinolap/common/util/MailService.java
new file mode 100644
index 0000000..608806c
--- /dev/null
+++ b/common/src/main/java/com/kylinolap/common/util/MailService.java
@@ -0,0 +1,102 @@
+/*
+ * 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.common.util;
+
+import java.io.IOException;
+import java.util.List;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.commons.mail.Email;
+import org.apache.commons.mail.EmailException;
+import org.apache.commons.mail.HtmlEmail;
+
+import com.kylinolap.common.KylinConfig;
+
+/**
+ * @author xduo
+ * 
+ */
+public class MailService {
+
+    private Boolean enabled = Boolean.TRUE;
+    private String host;
+    private String username;
+    private String password;
+    private String sender;
+
+    private static final Log logger = LogFactory.getLog(MailService.class);
+
+    public MailService() {
+        this(KylinConfig.getInstanceFromEnv());
+    }
+
+    public MailService(KylinConfig config) {
+        enabled = "true".equalsIgnoreCase(config.getProperty(KylinConfig.MAIL_ENABLED, "true"));
+        host = config.getProperty(KylinConfig.MAIL_HOST, "");
+        username = config.getProperty(KylinConfig.MAIL_USERNAME, "");
+        password = config.getProperty(KylinConfig.MAIL_PASSWORD, "");
+        sender = config.getProperty(KylinConfig.MAIL_SENDER, "");
+
+        if (enabled) {
+            assert !host.isEmpty();
+        }
+    }
+
+    /**
+     * 
+     * @param receivers
+     * @param subject
+     * @param content
+     * @return true or false indicating whether the email was delivered successfully
+     * @throws IOException
+     */
+    public boolean sendMail(List<String> receivers, String subject, String content) throws IOException {
+
+        if (!enabled) {
+            logger.info("Email service is disabled; this mail will not be delivered: " + subject);
+            logger.info("To enable mail service, set 'mail.enabled=true' in kylin.properties");
+            return false;
+        }
+
+        Email email = new HtmlEmail();
+        email.setHostName(host);
+        if (username != null && username.trim().length() > 0) {
+            email.setAuthentication(username, password);
+        }
+
+        //email.setDebug(true);
+        try {
+            for (String receiver : receivers) {
+                email.addTo(receiver);
+            }
+
+            email.setFrom(sender);
+            email.setSubject(subject);
+            email.setCharset("UTF-8");
+            ((HtmlEmail) email).setHtmlMsg(content);
+            email.send();
+            email.getMailSession();
+
+        } catch (EmailException e) {
+            logger.error(e.getLocalizedMessage(),e);
+            return false;
+        }
+        
+        return true;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/common/src/main/java/com/kylinolap/common/util/MyLogFormatter.java
----------------------------------------------------------------------
diff --git a/common/src/main/java/com/kylinolap/common/util/MyLogFormatter.java b/common/src/main/java/com/kylinolap/common/util/MyLogFormatter.java
new file mode 100644
index 0000000..b7e6180
--- /dev/null
+++ b/common/src/main/java/com/kylinolap/common/util/MyLogFormatter.java
@@ -0,0 +1,75 @@
+/*
+ * 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.common.util;
+
+import java.io.PrintWriter;
+import java.io.StringWriter;
+import java.sql.Timestamp;
+import java.util.Date;
+import java.util.logging.Formatter;
+import java.util.logging.LogRecord;
+
+public class MyLogFormatter extends Formatter {
+
+    Date dat = new Date();
+
+    // Line separator string. This is the value of the line.separator
+    // property at the moment that the SimpleFormatter was created.
+    private String lineSeparator = "\n";
+
+    /**
+     * Format the given LogRecord.
+     * 
+     * @param record
+     *            the log record to be formatted.
+     * @return a formatted log record
+     */
+    public synchronized String format(LogRecord record) {
+        StringBuffer sb = new StringBuffer();
+        // Minimize memory allocations here.
+        Timestamp ts = new Timestamp(record.getMillis());
+        String text = ts.toString();
+        sb.append("JUL ");
+        sb.append(text);
+        sb.append(" ");
+        if (record.getSourceClassName() != null) {
+            sb.append(record.getSourceClassName());
+        } else {
+            sb.append(record.getLoggerName());
+        }
+        if (record.getSourceMethodName() != null) {
+            sb.append(" ");
+            sb.append(record.getSourceMethodName());
+        }
+        sb.append(lineSeparator);
+        String message = formatMessage(record);
+        sb.append(record.getLevel().getLocalizedName());
+        sb.append(": ");
+        sb.append(message);
+        sb.append(lineSeparator);
+        if (record.getThrown() != null) {
+            try {
+                StringWriter sw = new StringWriter();
+                PrintWriter pw = new PrintWriter(sw);
+                record.getThrown().printStackTrace(pw);
+                pw.close();
+                sb.append(sw.toString());
+            } catch (Exception ex) {
+            }
+        }
+        return sb.toString();
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/common/src/main/java/com/kylinolap/common/util/RandomSampler.java
----------------------------------------------------------------------
diff --git a/common/src/main/java/com/kylinolap/common/util/RandomSampler.java b/common/src/main/java/com/kylinolap/common/util/RandomSampler.java
new file mode 100644
index 0000000..214d187
--- /dev/null
+++ b/common/src/main/java/com/kylinolap/common/util/RandomSampler.java
@@ -0,0 +1,53 @@
+/*
+ * 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.common.util;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Random;
+
+/**
+ * @author ysong1
+ * 
+ */
+public class RandomSampler<T> {
+
+    private Random rdm = new Random();
+
+    public List<T> sample(List<T> data, int sampleNumber) {
+        if (data == null) {
+            throw new IllegalArgumentException("Input list is null");
+        }
+        if (data.size() < sampleNumber) {
+            return data;
+        }
+
+        List<T> result = new ArrayList<T>(sampleNumber);
+        int n = data.size();
+        for (int i = 0; i < n; i++) {
+            if (i < sampleNumber) {
+                result.add(data.get(i));
+            } else {
+                int j = rdm.nextInt(i);
+                if (j < sampleNumber) {
+                    result.set(j, data.get(i));
+                }
+            }
+        }
+        return result;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/common/src/main/java/com/kylinolap/common/util/SSHClient.java
----------------------------------------------------------------------
diff --git a/common/src/main/java/com/kylinolap/common/util/SSHClient.java b/common/src/main/java/com/kylinolap/common/util/SSHClient.java
new file mode 100644
index 0000000..aadef76
--- /dev/null
+++ b/common/src/main/java/com/kylinolap/common/util/SSHClient.java
@@ -0,0 +1,268 @@
+/*
+ * 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.common.util;
+
+/** 
+ * @author George Song (ysong1)
+ * 
+ */
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.jcraft.jsch.Channel;
+import com.jcraft.jsch.ChannelExec;
+import com.jcraft.jsch.JSch;
+import com.jcraft.jsch.JSchException;
+import com.jcraft.jsch.Session;
+
+public class SSHClient {
+    protected static final Logger logger = LoggerFactory.getLogger(SSHClient.class);
+
+    private String hostname;
+    private String username;
+    private String password;
+    private String identityPath;
+
+    private SSHLogger sshLogger;
+
+    public SSHClient(String hostname, String username, String password, SSHLogger sshLogger) {
+        this.hostname = hostname;
+        this.username = username;
+        if (password != null && new File(password).exists()) {
+            this.identityPath = new File(password).getAbsolutePath();
+            this.password = null;
+        } else {
+            this.password = password;
+            this.identityPath = null;
+        }
+        this.sshLogger = sshLogger;
+    }
+
+    public void scpFileToRemote(String localFile, String remoteTargetDirectory) throws Exception {
+        FileInputStream fis = null;
+        try {
+            System.out.println("SCP file " + localFile + " to " + remoteTargetDirectory);
+
+            Session session = newJSchSession();
+            session.connect();
+
+            boolean ptimestamp = false;
+
+            // exec 'scp -t rfile' remotely
+            String command = "scp " + (ptimestamp ? "-p" : "") + " -t " + remoteTargetDirectory;
+            Channel channel = session.openChannel("exec");
+            ((ChannelExec) channel).setCommand(command);
+
+            // get I/O streams for remote scp
+            OutputStream out = channel.getOutputStream();
+            InputStream in = channel.getInputStream();
+
+            channel.connect();
+
+            if (checkAck(in) != 0) {
+                System.exit(0);
+            }
+
+            File _lfile = new File(localFile);
+
+            if (ptimestamp) {
+                command = "T " + (_lfile.lastModified() / 1000) + " 0";
+                // The access time should be sent here,
+                // but it is not accessible with JavaAPI ;-<
+                command += (" " + (_lfile.lastModified() / 1000) + " 0\n");
+                out.write(command.getBytes());
+                out.flush();
+                if (checkAck(in) != 0) {
+                    throw new Exception("Error in checkAck()");
+                }
+            }
+
+            // send "C0644 filesize filename", where filename should not include '/'
+            long filesize = _lfile.length();
+            command = "C0644 " + filesize + " ";
+            if (localFile.lastIndexOf("/") > 0) {
+                command += localFile.substring(localFile.lastIndexOf("/") + 1);
+            } else if (localFile.lastIndexOf(File.separator) > 0) {
+                command += localFile.substring(localFile.lastIndexOf(File.separator) + 1);
+            } else {
+                command += localFile;
+            }
+            command += "\n";
+            out.write(command.getBytes());
+            out.flush();
+            if (checkAck(in) != 0) {
+                throw new Exception("Error in checkAck()");
+            }
+
+            // send a content of lfile
+            fis = new FileInputStream(localFile);
+            byte[] buf = new byte[1024];
+            while (true) {
+                int len = fis.read(buf, 0, buf.length);
+                if (len <= 0)
+                    break;
+                out.write(buf, 0, len); // out.flush();
+            }
+            fis.close();
+            fis = null;
+            // send '\0'
+            buf[0] = 0;
+            out.write(buf, 0, 1);
+            out.flush();
+            if (checkAck(in) != 0) {
+                throw new Exception("Error in checkAck()");
+            }
+            out.close();
+
+            channel.disconnect();
+            session.disconnect();
+        } catch (Exception e) {
+            throw e;
+        } finally {
+            try {
+                if (fis != null)
+                    fis.close();
+            } catch (Exception ee) {
+            }
+        }
+    }
+
+    public SSHClientOutput execCommand(String command) throws Exception {
+        return execCommand(command, 7200);
+    }
+
+    public SSHClientOutput execCommand(String command, int timeoutSeconds) throws Exception {
+        try {
+            System.out.println("[" + username + "@" + hostname + "] Execute command: " + command);
+
+            StringBuffer text = new StringBuffer();
+            int exitCode = -1;
+
+            Session session = newJSchSession();
+            session.connect();
+
+            Channel channel = session.openChannel("exec");
+            ((ChannelExec) channel).setCommand(command);
+
+            channel.setInputStream(null);
+
+            // channel.setOutputStream(System.out);
+
+            ((ChannelExec) channel).setErrStream(System.err);
+
+            InputStream in = channel.getInputStream();
+            InputStream err = ((ChannelExec) channel).getErrStream();
+
+            channel.connect();
+
+            int timeout = timeoutSeconds;
+            byte[] tmp = new byte[1024];
+            while (true) {
+                timeout--;
+                while (in.available() > 0) {
+                    int i = in.read(tmp, 0, 1024);
+                    if (i < 0)
+                        break;
+
+                    String line = new String(tmp, 0, i);
+                    text.append(line);
+                    if (this.sshLogger != null) {
+                        this.sshLogger.log(line);
+                    }
+                }
+                while (err.available() > 0) {
+                    int i = err.read(tmp, 0, 1024);
+                    if (i < 0)
+                        break;
+
+                    String line = new String(tmp, 0, i);
+                    text.append(line);
+                    if (this.sshLogger != null) {
+                        this.sshLogger.log(line);
+                    }
+                }
+                if (channel.isClosed()) {
+                    if (in.available() > 0)
+                        continue;
+                    exitCode = channel.getExitStatus();
+                    System.out.println("[" + username + "@" + hostname + "] Command exit-status: " + exitCode);
+
+                    break;
+                }
+                try {
+                    Thread.sleep(1000);
+                } catch (Exception ee) {
+                    throw ee;
+                }
+                if (timeout < 0)
+                    throw new Exception("Remote commmand not finished within " + timeoutSeconds + " seconds.");
+            }
+            channel.disconnect();
+            session.disconnect();
+            return new SSHClientOutput(exitCode, text.toString());
+        } catch (Exception e) {
+            throw e;
+        }
+    }
+
+    private Session newJSchSession() throws JSchException {
+        JSch jsch = new JSch();
+        if (identityPath != null) {
+            jsch.addIdentity(identityPath);
+        }
+
+        Session session = jsch.getSession(username, hostname, 22);
+        if (password != null) {
+            session.setPassword(password);
+        }
+        session.setConfig("StrictHostKeyChecking", "no");
+        return session;
+    }
+
+    private int checkAck(InputStream in) throws IOException {
+        int b = in.read();
+        // b may be 0 for success,
+        // 1 for error,
+        // 2 for fatal error,
+        // -1
+        if (b == 0)
+            return b;
+        if (b == -1)
+            return b;
+
+        if (b == 1 || b == 2) {
+            StringBuffer sb = new StringBuffer();
+            int c;
+            do {
+                c = in.read();
+                sb.append((char) c);
+            } while (c != '\n');
+            if (b == 1) { // error
+                System.out.print(sb.toString());
+            }
+            if (b == 2) { // fatal error
+                System.out.print(sb.toString());
+            }
+        }
+        return b;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/common/src/main/java/com/kylinolap/common/util/SSHClientOutput.java
----------------------------------------------------------------------
diff --git a/common/src/main/java/com/kylinolap/common/util/SSHClientOutput.java b/common/src/main/java/com/kylinolap/common/util/SSHClientOutput.java
new file mode 100644
index 0000000..40f2d2d
--- /dev/null
+++ b/common/src/main/java/com/kylinolap/common/util/SSHClientOutput.java
@@ -0,0 +1,50 @@
+/*
+ * 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.common.util;
+
+/**
+ * @author George Song (ysong1)
+ * 
+ */
+
+public class SSHClientOutput {
+    private String text;
+    private int exitCode = -1;
+
+    /**
+     * @param text
+     * @param exitCode
+     */
+    public SSHClientOutput(int exitCode, String text) {
+        this.text = text;
+        this.exitCode = exitCode;
+    }
+
+    /**
+     * @return the text
+     */
+    public String getText() {
+        return text.toString();
+    }
+
+    /**
+     * @return the exitCode
+     */
+    public int getExitCode() {
+        return exitCode;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/common/src/main/java/com/kylinolap/common/util/SSHLogger.java
----------------------------------------------------------------------
diff --git a/common/src/main/java/com/kylinolap/common/util/SSHLogger.java b/common/src/main/java/com/kylinolap/common/util/SSHLogger.java
new file mode 100644
index 0000000..ebf025c
--- /dev/null
+++ b/common/src/main/java/com/kylinolap/common/util/SSHLogger.java
@@ -0,0 +1,25 @@
+/*
+ * 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.common.util;
+
+/**
+ * @author ysong1
+ * 
+ */
+public interface SSHLogger {
+    public void log(String message);
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/common/src/main/java/com/kylinolap/common/util/StringSplitter.java
----------------------------------------------------------------------
diff --git a/common/src/main/java/com/kylinolap/common/util/StringSplitter.java b/common/src/main/java/com/kylinolap/common/util/StringSplitter.java
new file mode 100644
index 0000000..fc7b6ea
--- /dev/null
+++ b/common/src/main/java/com/kylinolap/common/util/StringSplitter.java
@@ -0,0 +1,44 @@
+/*
+ * 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.common.util;
+
+import java.util.ArrayList;
+import java.util.List;
+
+/**
+ * @author George Song (ysong1)
+ * 
+ */
+public class StringSplitter {
+    public static String[] split(String str, String delimiter) {
+        // The optimized split function
+        List<String> list = new ArrayList<String>();
+        int index = 0, offset = 0;
+        int l = delimiter.length();
+        if (str.startsWith(delimiter)) {
+            // in case the first field is empty
+            list.add("");
+            offset = offset + l;
+        }
+        while ((index = str.indexOf(delimiter, index + 1)) != -1) {
+            list.add(str.substring(offset, index));
+            offset = index + l;
+        }
+        // add the last field, or the str doesn't contain delimiter at all
+        list.add(str.substring(offset));
+        return list.toArray(new String[0]);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/common/src/main/java/com/kylinolap/common/util/StringUtil.java
----------------------------------------------------------------------
diff --git a/common/src/main/java/com/kylinolap/common/util/StringUtil.java b/common/src/main/java/com/kylinolap/common/util/StringUtil.java
new file mode 100644
index 0000000..6a3eaa3
--- /dev/null
+++ b/common/src/main/java/com/kylinolap/common/util/StringUtil.java
@@ -0,0 +1,99 @@
+/*
+ * 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.common.util;
+
+import java.util.ArrayList;
+import java.util.Collection;
+
+/**
+ * Created with IntelliJ IDEA. User: lukhan Date: 12/2/13 Time: 11:43 AM To
+ * change this template use File | Settings | File Templates.
+ */
+public class StringUtil {
+
+    public static String[] filterSystemArgs(String args[]) {
+        ArrayList<String> whatsLeft = new ArrayList<String>();
+        for (String a : args) {
+            if (a.startsWith("-D")) {
+                String key;
+                String value;
+                int cut = a.indexOf('=');
+                if (cut < 0) {
+                    key = a.substring(2);
+                    value = "";
+                } else {
+                    key = a.substring(2, cut);
+                    value = a.substring(cut + 1);
+                }
+                System.setProperty(key, value);
+            } else {
+                whatsLeft.add(a);
+            }
+        }
+        return (String[]) whatsLeft.toArray(new String[whatsLeft.size()]);
+    }
+
+    public static void toUpperCaseArray(String[] source, String[] target) {
+        for (int i = 0; i < source.length; i++) {
+            if (source[i] != null) {
+                target[i] = source[i].toUpperCase();
+            }
+        }
+    }
+
+    public static String dropSuffix(String str, String suffix) {
+        if (str.endsWith(suffix))
+            return str.substring(0, str.length() - suffix.length());
+        else
+            return str;
+    }
+
+    public static String min(Collection<String> strs) {
+        String min = null;
+        for (String s : strs) {
+            if (min == null || min.compareTo(s) > 0)
+                min = s;
+        }
+        return min;
+    }
+
+    public static String max(Collection<String> strs) {
+        String max = null;
+        for (String s : strs) {
+            if (max == null || max.compareTo(s) < 0)
+                max = s;
+        }
+        return max;
+    }
+
+    public static String min(String s1, String s2) {
+        if (s1 == null)
+            return s2;
+        else if (s2 == null)
+            return s1;
+        else
+            return s1.compareTo(s2) < 0 ? s1 : s2;
+    }
+
+    public static String max(String s1, String s2) {
+        if (s1 == null)
+            return s2;
+        else if (s2 == null)
+            return s1;
+        else
+            return s1.compareTo(s2) > 0 ? s1 : s2;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/common/src/main/resources/kylinlog4j.properties
----------------------------------------------------------------------
diff --git a/common/src/main/resources/kylinlog4j.properties b/common/src/main/resources/kylinlog4j.properties
new file mode 100644
index 0000000..49fb5cc
--- /dev/null
+++ b/common/src/main/resources/kylinlog4j.properties
@@ -0,0 +1,10 @@
+# use this when conflict with hbase, enable this by -Dlog4j.configuration=kylinlog4j.properties
+
+log4j.rootLogger=INFO,stdout
+
+log4j.appender.stdout=org.apache.log4j.ConsoleAppender
+log4j.appender.stdout.layout=org.apache.log4j.PatternLayout
+log4j.appender.stdout.layout.ConversionPattern=L4J [%d{yyyy-MM-dd HH:mm:ss,SSS}][%p][%c] - %m%n
+
+#log4j.logger.org.apache.hadoop=ERROR
+log4j.logger.com.kylinolap=DEBUG

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/common/src/main/resources/log4j.properties
----------------------------------------------------------------------
diff --git a/common/src/main/resources/log4j.properties b/common/src/main/resources/log4j.properties
new file mode 100644
index 0000000..b00f355
--- /dev/null
+++ b/common/src/main/resources/log4j.properties
@@ -0,0 +1,9 @@
+
+log4j.rootLogger=INFO,stdout
+
+log4j.appender.stdout=org.apache.log4j.ConsoleAppender
+log4j.appender.stdout.layout=org.apache.log4j.PatternLayout
+log4j.appender.stdout.layout.ConversionPattern=L4J [%d{yyyy-MM-dd HH:mm:ss,SSS}][%p][%c] - %m%n
+
+#log4j.logger.org.apache.hadoop=ERROR
+log4j.logger.com.kylinolap=DEBUG

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/common/src/test/java/com/kylinolap/common/persistence/HBaseResourceStoreTest.java
----------------------------------------------------------------------
diff --git a/common/src/test/java/com/kylinolap/common/persistence/HBaseResourceStoreTest.java b/common/src/test/java/com/kylinolap/common/persistence/HBaseResourceStoreTest.java
new file mode 100644
index 0000000..86e11d0
--- /dev/null
+++ b/common/src/test/java/com/kylinolap/common/persistence/HBaseResourceStoreTest.java
@@ -0,0 +1,206 @@
+/*
+ * 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.common.persistence;
+
+import static org.junit.Assert.*;
+
+import java.io.DataInputStream;
+import java.io.DataOutputStream;
+import java.io.IOException;
+import java.util.ArrayList;
+
+import org.apache.commons.lang.StringUtils;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+import com.kylinolap.common.KylinConfig;
+import com.kylinolap.common.util.HBaseMetadataTestCase;
+import com.kylinolap.common.util.HadoopUtil;
+
+public class HBaseResourceStoreTest extends HBaseMetadataTestCase {
+
+    @Before
+    public void setup() throws Exception {
+        this.createTestMetadata();
+    }
+
+    @After
+    public void after() throws Exception {
+        this.cleanupTestMetadata();
+    }
+
+    @Test
+    public void testHBaseStore() throws Exception {
+        testAStore(ResourceStore.getStore(KylinConfig.getInstanceFromEnv()));
+    }
+
+    @Test
+    public void testHBaseStoreWithLargeCell() throws Exception {
+        String path = "/cube/_test_large_cell.json";
+        String largeContent = "THIS_IS_A_LARGE_CELL";
+        StringEntity content = new StringEntity(largeContent);
+        KylinConfig config = KylinConfig.getInstanceFromEnv();
+        int origSize = config.getHBaseKeyValueSize();
+        ResourceStore store = ResourceStore.getStore(KylinConfig.getInstanceFromEnv());
+
+        try {
+            config.setProperty("kylin.hbase.client.keyvalue.maxsize", String.valueOf(largeContent.length() - 1));
+
+            store.deleteResource(path);
+
+            store.putResource(path, content, StringEntity.serializer);
+            assertTrue(store.exists(path));
+            StringEntity t = store.getResource(path, StringEntity.class, StringEntity.serializer);
+            assertEquals(content, t);
+
+            Path redirectPath = ((HBaseResourceStore) store).bigCellHDFSPath(path);
+            Configuration hconf = HadoopUtil.getDefaultConfiguration();
+            FileSystem fileSystem = FileSystem.get(hconf);
+            assertTrue(fileSystem.exists(redirectPath));
+
+            FSDataInputStream in = fileSystem.open(redirectPath);
+            assertEquals(largeContent, in.readUTF());
+            in.close();
+
+            store.deleteResource(path);
+        } finally {
+            config.setProperty("kylin.hbase.client.keyvalue.maxsize", "" + origSize);
+            store.deleteResource(path);
+        }
+    }
+
+    void testAStore(ResourceStore store) throws IOException {
+        String dir1 = "/cube";
+        String path1 = "/cube/_test.json";
+        StringEntity content1 = new StringEntity("anything");
+        String dir2 = "/table";
+        String path2 = "/table/_test.json";
+        StringEntity content2 = new StringEntity("something");
+
+        // cleanup legacy if any
+        store.deleteResource(path1);
+        store.deleteResource(path2);
+
+        StringEntity t;
+
+        // put/get
+        store.putResource(path1, content1, StringEntity.serializer);
+        assertTrue(store.exists(path1));
+        t = store.getResource(path1, StringEntity.class, StringEntity.serializer);
+        assertEquals(content1, t);
+
+        store.putResource(path2, content2, StringEntity.serializer);
+        assertTrue(store.exists(path2));
+        t = store.getResource(path2, StringEntity.class, StringEntity.serializer);
+        assertEquals(content2, t);
+
+        // overwrite
+        t.str = "new string";
+        store.putResource(path2, t, StringEntity.serializer);
+
+        // write conflict
+        try {
+            t.setLastModified(t.lastModified - 1);
+            store.putResource(path2, t, StringEntity.serializer);
+            fail("write conflict should trigger IllegalStateException");
+        } catch (IllegalStateException e) {
+            // expected
+        }
+
+        // list
+        ArrayList<String> list;
+
+        list = store.listResources(dir1);
+        assertTrue(list.contains(path1));
+        assertTrue(list.contains(path2) == false);
+
+        list = store.listResources(dir2);
+        assertTrue(list.contains(path2));
+        assertTrue(list.contains(path1) == false);
+
+        list = store.listResources("/");
+        assertTrue(list.contains(dir1));
+        assertTrue(list.contains(dir2));
+        assertTrue(list.contains(path1) == false);
+        assertTrue(list.contains(path2) == false);
+
+        list = store.listResources(path1);
+        assertNull(list);
+        list = store.listResources(path2);
+        assertNull(list);
+
+        // delete/exist
+        store.deleteResource(path1);
+        assertTrue(store.exists(path1) == false);
+        list = store.listResources(dir1);
+        assertTrue(list == null || list.contains(path1) == false);
+
+        store.deleteResource(path2);
+        assertTrue(store.exists(path2) == false);
+        list = store.listResources(dir2);
+        assertTrue(list == null || list.contains(path2) == false);
+    }
+
+    public static class StringEntity extends RootPersistentEntity {
+
+        static final Serializer<StringEntity> serializer = new Serializer<StringEntity>() {
+            @Override
+            public void serialize(StringEntity obj, DataOutputStream out) throws IOException {
+                out.writeUTF(obj.str);
+            }
+
+            @Override
+            public StringEntity deserialize(DataInputStream in) throws IOException {
+                String str = in.readUTF();
+                return new StringEntity(str);
+            }
+        };
+
+        String str;
+
+        public StringEntity(String str) {
+            this.str = str;
+        }
+
+        @Override
+        public int hashCode() {
+            final int prime = 31;
+            int result = super.hashCode();
+            result = prime * result + ((str == null) ? 0 : str.hashCode());
+            return result;
+        }
+
+        @Override
+        public boolean equals(Object obj) {
+            if (obj == this)
+                return true;
+            if (!(obj instanceof StringEntity))
+                return false;
+            return StringUtils.equals(this.str, ((StringEntity) obj).str);
+        }
+
+        @Override
+        public String toString() {
+            return str;
+        }
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/common/src/test/java/com/kylinolap/common/persistence/LocalFileResourceStoreTest.java
----------------------------------------------------------------------
diff --git a/common/src/test/java/com/kylinolap/common/persistence/LocalFileResourceStoreTest.java b/common/src/test/java/com/kylinolap/common/persistence/LocalFileResourceStoreTest.java
new file mode 100644
index 0000000..03247e9
--- /dev/null
+++ b/common/src/test/java/com/kylinolap/common/persistence/LocalFileResourceStoreTest.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.common.persistence;
+
+import static org.junit.Assert.*;
+
+import java.io.DataInputStream;
+import java.io.DataOutputStream;
+import java.io.IOException;
+import java.util.ArrayList;
+
+import org.apache.commons.lang.StringUtils;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+import com.kylinolap.common.KylinConfig;
+import com.kylinolap.common.util.LocalFileMetadataTestCase;
+
+public class LocalFileResourceStoreTest extends LocalFileMetadataTestCase {
+
+    @Before
+    public void setup() throws Exception {
+        this.createTestMetadata();
+    }
+
+    @After
+    public void after() throws Exception {
+        this.cleanupTestMetadata();
+    }
+
+    @Test
+    public void testFileStore() throws Exception {
+        testAStore(ResourceStore.getStore(KylinConfig.getInstanceFromEnv()));
+    }
+
+    void testAStore(ResourceStore store) throws IOException {
+        String dir1 = "/cube";
+        String path1 = "/cube/_test.json";
+        StringEntity content1 = new StringEntity("anything");
+        String dir2 = "/table";
+        String path2 = "/table/_test.json";
+        StringEntity content2 = new StringEntity("something");
+
+        // cleanup legacy if any
+        store.deleteResource(path1);
+        store.deleteResource(path2);
+
+        StringEntity t;
+
+        // put/get
+        store.putResource(path1, content1, StringEntity.serializer);
+        assertTrue(store.exists(path1));
+        t = store.getResource(path1, StringEntity.class, StringEntity.serializer);
+        assertEquals(content1, t);
+
+        store.putResource(path2, content2, StringEntity.serializer);
+        assertTrue(store.exists(path2));
+        t = store.getResource(path2, StringEntity.class, StringEntity.serializer);
+        assertEquals(content2, t);
+
+        // overwrite
+        t.str = "new string";
+        store.putResource(path2, t, StringEntity.serializer);
+
+        // write conflict
+        try {
+            t.setLastModified(t.lastModified - 1);
+            store.putResource(path2, t, StringEntity.serializer);
+            fail("write conflict should trigger IllegalStateException");
+        } catch (IllegalStateException e) {
+            // expected
+        }
+
+        // list
+        ArrayList<String> list;
+
+        list = store.listResources(dir1);
+        assertTrue(list.contains(path1));
+        assertTrue(list.contains(path2) == false);
+
+        list = store.listResources(dir2);
+        assertTrue(list.contains(path2));
+        assertTrue(list.contains(path1) == false);
+
+        list = store.listResources("/");
+        assertTrue(list.contains(dir1));
+        assertTrue(list.contains(dir2));
+        assertTrue(list.contains(path1) == false);
+        assertTrue(list.contains(path2) == false);
+
+        list = store.listResources(path1);
+        assertNull(list);
+        list = store.listResources(path2);
+        assertNull(list);
+
+        // delete/exist
+        store.deleteResource(path1);
+        assertTrue(store.exists(path1) == false);
+        list = store.listResources(dir1);
+        assertTrue(list == null || list.contains(path1) == false);
+
+        store.deleteResource(path2);
+        assertTrue(store.exists(path2) == false);
+        list = store.listResources(dir2);
+        assertTrue(list == null || list.contains(path2) == false);
+    }
+
+    public static class StringEntity extends RootPersistentEntity {
+
+        static final Serializer<StringEntity> serializer = new Serializer<StringEntity>() {
+            @Override
+            public void serialize(StringEntity obj, DataOutputStream out) throws IOException {
+                out.writeUTF(obj.str);
+            }
+
+            @Override
+            public StringEntity deserialize(DataInputStream in) throws IOException {
+                String str = in.readUTF();
+                return new StringEntity(str);
+            }
+        };
+
+        String str;
+
+        public StringEntity(String str) {
+            this.str = str;
+        }
+
+        @Override
+        public int hashCode() {
+            final int prime = 31;
+            int result = super.hashCode();
+            result = prime * result + ((str == null) ? 0 : str.hashCode());
+            return result;
+        }
+
+        @Override
+        public boolean equals(Object obj) {
+            if (obj == this)
+                return true;
+            if (!(obj instanceof StringEntity))
+                return false;
+            return StringUtils.equals(this.str, ((StringEntity) obj).str);
+        }
+
+        @Override
+        public String toString() {
+            return str;
+        }
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/common/src/test/java/com/kylinolap/common/persistence/ResourceToolTest.java
----------------------------------------------------------------------
diff --git a/common/src/test/java/com/kylinolap/common/persistence/ResourceToolTest.java b/common/src/test/java/com/kylinolap/common/persistence/ResourceToolTest.java
new file mode 100644
index 0000000..5a1fca3
--- /dev/null
+++ b/common/src/test/java/com/kylinolap/common/persistence/ResourceToolTest.java
@@ -0,0 +1,28 @@
+package com.kylinolap.common.persistence;
+
+import java.io.File;
+import java.io.IOException;
+
+import org.junit.Before;
+import org.junit.Ignore;
+import org.junit.Test;
+
+import com.kylinolap.common.KylinConfig;
+import com.kylinolap.common.util.ClasspathUtil;
+
+/**
+ * Created by honma on 9/18/14.
+ */
+@Ignore
+public class ResourceToolTest {
+    @Before
+    public void setup() throws Exception {
+        ClasspathUtil.addClasspath(new File("../examples/test_case_data/hadoop-site").getAbsolutePath());
+    }
+
+    @Test
+    public void test() throws IOException {
+        ResourceTool.copy(KylinConfig.createInstanceFromUri("../examples/test_case_data"), KylinConfig.createInstanceFromUri("../examples/test_case_data/kylin.properties"));
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/common/src/test/java/com/kylinolap/common/restclient/RestClientTest.java
----------------------------------------------------------------------
diff --git a/common/src/test/java/com/kylinolap/common/restclient/RestClientTest.java b/common/src/test/java/com/kylinolap/common/restclient/RestClientTest.java
new file mode 100644
index 0000000..25968b3
--- /dev/null
+++ b/common/src/test/java/com/kylinolap/common/restclient/RestClientTest.java
@@ -0,0 +1,25 @@
+package com.kylinolap.common.restclient;
+
+
+import org.junit.Test;
+
+import java.io.IOException;
+
+public class RestClientTest {
+
+    @SuppressWarnings("unused")
+    @Test
+    public void basicTests() throws IOException {
+        RestClient a = new RestClient("prod01:80");
+        //a.wipeCache("metadata", "a", "a");
+        //String aa = a.getKylinProperties();
+        //System.out.println(aa);
+        RestClient b = new RestClient("sandbox.hortonworks.com:7070");
+        //b.wipeCache("metadata", "a", "a");
+        //String bb = b.getKylinProperties();
+        //System.out.println(bb);
+
+
+    }
+
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/common/src/test/java/com/kylinolap/common/util/BasicHadoopTest.java
----------------------------------------------------------------------
diff --git a/common/src/test/java/com/kylinolap/common/util/BasicHadoopTest.java b/common/src/test/java/com/kylinolap/common/util/BasicHadoopTest.java
new file mode 100644
index 0000000..b09173e
--- /dev/null
+++ b/common/src/test/java/com/kylinolap/common/util/BasicHadoopTest.java
@@ -0,0 +1,65 @@
+package com.kylinolap.common.util;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.HBaseConfiguration;
+import org.apache.hadoop.hbase.HColumnDescriptor;
+import org.apache.hadoop.hbase.HTableDescriptor;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.HBaseAdmin;
+import org.junit.BeforeClass;
+import org.junit.Ignore;
+import org.junit.Test;
+
+import java.io.File;
+import java.io.IOException;
+
+/**
+ * Created by honma on 11/11/14.
+ *
+ * development concept proving use
+ */
+@Ignore
+public class BasicHadoopTest {
+
+    @BeforeClass
+    public static void setup() throws Exception {
+        ClasspathUtil.addClasspath(new File("../examples/test_case_data/hadoop-site").getAbsolutePath());
+    }
+
+    @Test
+    public void testCreateHtable() throws IOException {
+        HTableDescriptor tableDesc = new HTableDescriptor(TableName.valueOf("testhbase"));
+        tableDesc.setValue("KYLIN_HOST", "dev01");
+
+        HColumnDescriptor cf = new HColumnDescriptor("f");
+        cf.setMaxVersions(1);
+
+        cf.setInMemory(true);
+        cf.setBlocksize(4 * 1024 * 1024); // set to 4MB
+        tableDesc.addFamily(cf);
+
+        Configuration conf = HBaseConfiguration.create();
+        HBaseAdmin admin = new HBaseAdmin(conf);
+        admin.createTable(tableDesc);
+        admin.close();
+    }
+
+    @Test
+    public void testRetriveHtableHost() throws IOException {
+        Configuration conf = HBaseConfiguration.create();
+        HBaseAdmin hbaseAdmin = new HBaseAdmin(conf);
+        HTableDescriptor[] tableDescriptors = hbaseAdmin.listTables();
+        for (HTableDescriptor table : tableDescriptors) {
+            String value = table.getValue("KYLIN_HOST");
+            if (value != null) {
+                System.out.println(table.getTableName());
+                System.out.println("host is " + value);
+                hbaseAdmin.disableTable(table.getTableName());
+                table.setValue("KYLIN_HOST_ANOTHER", "dev02");
+                hbaseAdmin.modifyTable(table.getTableName(), table);
+                hbaseAdmin.enableTable(table.getTableName());
+            }
+        }
+        hbaseAdmin.close();
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/common/src/test/java/com/kylinolap/common/util/BasicTest.java
----------------------------------------------------------------------
diff --git a/common/src/test/java/com/kylinolap/common/util/BasicTest.java b/common/src/test/java/com/kylinolap/common/util/BasicTest.java
new file mode 100644
index 0000000..347f951
--- /dev/null
+++ b/common/src/test/java/com/kylinolap/common/util/BasicTest.java
@@ -0,0 +1,21 @@
+package com.kylinolap.common.util;
+
+import java.io.IOException;
+
+import org.junit.Ignore;
+import org.junit.Test;
+
+/**
+ * Created by honma on 10/17/14.
+ *
+ * Keep this test case to test basic java functionality
+ * development concept proving use
+ */
+@Ignore
+public class BasicTest {
+    @Test
+    public void test() throws IOException {
+        double i2 = 3234.4324234324234;
+        System.out.println(String.format("%.2f", i2));
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/common/src/test/java/com/kylinolap/common/util/HyperLogLogCounterTest.java
----------------------------------------------------------------------
diff --git a/common/src/test/java/com/kylinolap/common/util/HyperLogLogCounterTest.java b/common/src/test/java/com/kylinolap/common/util/HyperLogLogCounterTest.java
new file mode 100644
index 0000000..c41683b
--- /dev/null
+++ b/common/src/test/java/com/kylinolap/common/util/HyperLogLogCounterTest.java
@@ -0,0 +1,209 @@
+/*
+ * 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.common.util;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.HashSet;
+import java.util.Random;
+import java.util.Set;
+
+import org.apache.hadoop.hbase.util.Bytes;
+import org.junit.Assert;
+import org.junit.Test;
+
+import com.kylinolap.common.hll.HyperLogLogPlusCounter;
+
+/**
+ * @author yangli9
+ * 
+ */
+public class HyperLogLogCounterTest {
+
+    ByteBuffer buf = ByteBuffer.allocate(1024 * 1024);
+    Random rand1 = new Random(1);
+    Random rand2 = new Random(2);
+    Random rand3 = new Random(3);
+    int errorCount1 = 0;
+    int errorCount2 = 0;
+    int errorCount3 = 0;
+
+    private Set<String> generateTestData(int n) {
+        Set<String> testData = new HashSet<String>();
+        for (int i = 0; i < n; i++) {
+            String[] samples = generateSampleData();
+            for (String sample : samples) {
+                testData.add(sample);
+            }
+        }
+        return testData;
+    }
+
+    // simulate the visit (=visitor+id)
+    private String[] generateSampleData() {
+
+        StringBuilder buf = new StringBuilder();
+        for (int i = 0; i < 19; i++) {
+            buf.append(Math.abs(rand1.nextInt()) % 10);
+        }
+        String header = buf.toString();
+
+        int size = Math.abs(rand3.nextInt()) % 9 + 1;
+        String[] samples = new String[size];
+        for (int k = 0; k < size; k++) {
+            buf = new StringBuilder(header);
+            buf.append("-");
+            for (int i = 0; i < 10; i++) {
+                buf.append(Math.abs(rand3.nextInt()) % 10);
+            }
+            samples[k] = buf.toString();
+        }
+
+        return samples;
+    }
+
+    @Test
+    public void countTest() throws IOException {
+        int n = 10;
+        for (int i = 0; i < 5; i++) {
+            count(n);
+            n *= 10;
+        }
+    }
+
+    private void count(int n) throws IOException {
+        Set<String> testSet = generateTestData(n);
+
+        HyperLogLogPlusCounter hllc = newHLLC();
+        for (String testData : testSet) {
+            hllc.add(Bytes.toBytes(testData));
+        }
+        long estimate = hllc.getCountEstimate();
+        double errorRate = hllc.getErrorRate();
+        double actualError = (double) Math.abs(testSet.size() - estimate) / testSet.size();
+        System.out.println(estimate);
+        System.out.println(testSet.size());
+        System.out.println(errorRate);
+        System.out.println("=" + actualError);
+        Assert.assertTrue(actualError < errorRate * 3.0);
+
+        checkSerialize(hllc);
+    }
+
+    private void checkSerialize(HyperLogLogPlusCounter hllc) throws IOException {
+        long estimate = hllc.getCountEstimate();
+        buf.clear();
+        hllc.writeRegisters(buf);
+        buf.flip();
+        hllc.readRegisters(buf);
+        Assert.assertEquals(estimate, hllc.getCountEstimate());
+    }
+
+    @Test
+    public void mergeTest() throws IOException {
+        double error = 0;
+        double absError = 0;
+        int n = 100;
+        for (int i = 0; i < n; i++) {
+            System.out.println("============" + i);
+            double e = merge();
+            error += e;
+            absError += Math.abs(e);
+        }
+        System.out.println("Total average error is " + error / n + " and absolute error is " + absError / n);
+
+        System.out.println("errorCount1 is " + errorCount1 + "!");
+        System.out.println("errorCount2 is " + errorCount2 + "!");
+        System.out.println("errorCount3 is " + errorCount3 + "!");
+
+        Assert.assertTrue(errorCount1 <= n * 0.40);
+        Assert.assertTrue(errorCount2 <= n * 0.08);
+        Assert.assertTrue(errorCount3 <= n * 0.02);
+    }
+
+    private double merge() throws IOException {
+
+        int ln = 50;
+        int dn = 300;
+        Set<String> testSet = new HashSet<String>();
+        HyperLogLogPlusCounter[] hllcs = new HyperLogLogPlusCounter[ln];
+        for (int i = 0; i < ln; i++) {
+            hllcs[i] = newHLLC();
+            for (int k = 0; k < dn; k++) {
+                String[] samples = generateSampleData();
+                for (String data : samples) {
+                    testSet.add(data);
+                    hllcs[i].add(Bytes.toBytes(data));
+                }
+            }
+        }
+        HyperLogLogPlusCounter mergeHllc = newHLLC();
+        for (HyperLogLogPlusCounter hllc : hllcs) {
+            mergeHllc.merge(hllc);
+            checkSerialize(mergeHllc);
+        }
+
+        double errorRate = mergeHllc.getErrorRate();
+        long estimate = mergeHllc.getCountEstimate();
+        double actualError = (double) (testSet.size() - estimate) / testSet.size();
+
+        System.out.println(testSet.size() + "-" + estimate);
+
+        System.out.println("=" + actualError);
+        if (Math.abs(actualError) > errorRate) {
+            errorCount1++;
+        }
+        if (Math.abs(actualError) > 2 * errorRate) {
+            errorCount2++;
+        }
+        if (Math.abs(actualError) > 3 * errorRate) {
+            errorCount3++;
+        }
+
+        return actualError;
+    }
+
+    @Test
+    public void testPerformance() throws IOException {
+        int N = 3; // reduce N HLLC into one
+        int M = 1000; // for M times, use 100000 for real perf test
+
+        HyperLogLogPlusCounter samples[] = new HyperLogLogPlusCounter[N];
+        for (int i = 0; i < N; i++) {
+            samples[i] = newHLLC();
+            for (String str : generateTestData(10000))
+                samples[i].add(str);
+        }
+
+        System.out.println("Perf test running ... ");
+        long start = System.currentTimeMillis();
+        HyperLogLogPlusCounter sum = newHLLC();
+        for (int i = 0; i < M; i++) {
+            sum.clear();
+            for (int j = 0; j < N; j++) {
+                sum.merge(samples[j]);
+                checkSerialize(sum);
+            }
+        }
+        long duration = System.currentTimeMillis() - start;
+        System.out.println("Perf test result: " + duration / 1000 + " seconds");
+    }
+
+    private HyperLogLogPlusCounter newHLLC() {
+        return new HyperLogLogPlusCounter(16);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/common/src/test/java/com/kylinolap/common/util/InstallJarIntoMavenTest.java
----------------------------------------------------------------------
diff --git a/common/src/test/java/com/kylinolap/common/util/InstallJarIntoMavenTest.java b/common/src/test/java/com/kylinolap/common/util/InstallJarIntoMavenTest.java
new file mode 100644
index 0000000..0bc77d5
--- /dev/null
+++ b/common/src/test/java/com/kylinolap/common/util/InstallJarIntoMavenTest.java
@@ -0,0 +1,45 @@
+package com.kylinolap.common.util;
+
+import java.io.File;
+import java.io.FileWriter;
+import java.io.IOException;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+
+import org.junit.Ignore;
+
+/**
+ * Created by honma on 6/6/14.
+ */
+public class InstallJarIntoMavenTest {
+
+    @Ignore
+    public void testInstall() throws IOException {
+        File folder = new File("/export/home/b_kylin/tmp");
+        File out = new File("/export/home/b_kylin/tmp/out.sh");
+        out.createNewFile();
+        FileWriter fw = new FileWriter(out);
+
+        for (File file : folder.listFiles()) {
+            String name = file.getName();
+
+            if (!name.endsWith(".jar"))
+                continue;
+
+            int firstSlash = name.indexOf('-');
+            int lastDot = name.lastIndexOf('.');
+            String groupId = name.substring(0, firstSlash);
+
+            Pattern pattern = Pattern.compile("-\\d");
+            Matcher match = pattern.matcher(name);
+            match.find();
+            String artifactId = name.substring(0, match.start());
+            String version = name.substring(match.start() + 1, lastDot);
+
+            fw.write(String.format("mvn install:install-file -Dfile=%s -DgroupId=%s -DartifactId=%s -Dversion=%s -Dpackaging=jar", name, "org.apache." + groupId, artifactId, version));
+            fw.write("\n");
+        }
+        fw.close();
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/common/src/test/java/com/kylinolap/common/util/MailServiceTest.java
----------------------------------------------------------------------
diff --git a/common/src/test/java/com/kylinolap/common/util/MailServiceTest.java b/common/src/test/java/com/kylinolap/common/util/MailServiceTest.java
new file mode 100644
index 0000000..044d134
--- /dev/null
+++ b/common/src/test/java/com/kylinolap/common/util/MailServiceTest.java
@@ -0,0 +1,60 @@
+/*
+ * 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.common.util;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+
+import org.junit.Test;
+
+import com.kylinolap.common.KylinConfig;
+
+public class MailServiceTest {
+
+    @Test
+    public void testSendEmail() throws IOException {
+        
+        @SuppressWarnings("deprecation")
+        KylinConfig config = KylinConfig.getInstanceForTest(AbstractKylinTestCase.SANDBOX_TEST_DATA);
+
+        MailService mailservice = new MailService(config);
+        boolean sent = sendTestEmail(mailservice);
+        assert sent;
+        
+        // set mail.enabled=false, and run again, this time should be no mail delviered
+        config.setProperty(KylinConfig.MAIL_ENABLED, "false");
+        mailservice = new MailService(config);
+        sent = sendTestEmail(mailservice);
+        assert !sent;
+        
+    }
+    
+    private boolean sendTestEmail(MailService mailservice) {
+
+        List<String> receivers = new ArrayList<String>(1);
+        receivers.add("shaoshi@ebay.com");
+        try {
+            return mailservice.sendMail(receivers, "A test email from Kylin", "Hello!");
+        } catch (IOException e) {
+            e.printStackTrace();
+        }
+
+        return false;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/common/src/test/java/com/kylinolap/common/util/RandomSamplerTest.java
----------------------------------------------------------------------
diff --git a/common/src/test/java/com/kylinolap/common/util/RandomSamplerTest.java b/common/src/test/java/com/kylinolap/common/util/RandomSamplerTest.java
new file mode 100644
index 0000000..73e8e4a
--- /dev/null
+++ b/common/src/test/java/com/kylinolap/common/util/RandomSamplerTest.java
@@ -0,0 +1,46 @@
+/*
+ * 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.common.util;
+
+import static org.junit.Assert.*;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.hadoop.io.Text;
+import org.junit.Test;
+
+/**
+ * @author ysong1
+ * 
+ */
+public class RandomSamplerTest {
+
+    @Test
+    public void test() {
+        RandomSampler<Text> s = new RandomSampler<Text>();
+        List<Text> data = new ArrayList<Text>();
+        for (int i = 0; i < 1000; i++) {
+            data.add(new Text(String.valueOf(i)));
+        }
+
+        List<Text> result = s.sample(data, 50);
+        System.out.println(result);
+        assertEquals(50, result.size());
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/common/src/test/java/com/kylinolap/common/util/SSHClientTest.java
----------------------------------------------------------------------
diff --git a/common/src/test/java/com/kylinolap/common/util/SSHClientTest.java b/common/src/test/java/com/kylinolap/common/util/SSHClientTest.java
new file mode 100644
index 0000000..4dd465b
--- /dev/null
+++ b/common/src/test/java/com/kylinolap/common/util/SSHClientTest.java
@@ -0,0 +1,83 @@
+/*
+ * 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.common.util;
+
+import static org.junit.Assert.*;
+
+import java.io.File;
+import java.io.IOException;
+
+import org.apache.hadoop.fs.FileUtil;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+import com.kylinolap.common.KylinConfig;
+
+/**
+ * @author ysong1
+ * 
+ */
+public class SSHClientTest extends LocalFileMetadataTestCase {
+
+    private boolean isRemote;
+    private String hostname;
+    private String username;
+    private String password;
+
+    private void loadPropertiesFile() throws IOException {
+
+        KylinConfig cfg = KylinConfig.getInstanceFromEnv();
+
+        this.isRemote = cfg.getRunAsRemoteCommand();
+        this.hostname = cfg.getRemoteHadoopCliHostname();
+        this.username = cfg.getRemoteHadoopCliUsername();
+        this.password = cfg.getRemoteHadoopCliPassword();
+    }
+
+    @Before
+    public void before() throws Exception {
+        this.createTestMetadata();
+        loadPropertiesFile();
+    }
+
+    @After
+    public void after() throws Exception {
+        this.cleanupTestMetadata();
+    }
+
+    @Test
+    public void testCmd() throws Exception {
+        if (isRemote == false)
+            return;
+        
+        SSHClient ssh = new SSHClient(this.hostname, this.username, this.password, null);
+        SSHClientOutput output = ssh.execCommand("echo hello");
+        assertEquals(0, output.getExitCode());
+        assertEquals("hello\n", output.getText());
+    }
+
+    @Test
+    public void testScp() throws Exception {
+        if (isRemote == false)
+            return;
+        
+        SSHClient ssh = new SSHClient(this.hostname, this.username, this.password, null);
+        File tmpFile = FileUtil.createLocalTempFile(new File("/tmp/test_scp"), "temp_", false);
+        ssh.scpFileToRemote(tmpFile.getAbsolutePath(), "/tmp");
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/cube/.settings/org.eclipse.core.resources.prefs
----------------------------------------------------------------------
diff --git a/cube/.settings/org.eclipse.core.resources.prefs b/cube/.settings/org.eclipse.core.resources.prefs
new file mode 100644
index 0000000..04cfa2c
--- /dev/null
+++ b/cube/.settings/org.eclipse.core.resources.prefs
@@ -0,0 +1,6 @@
+eclipse.preferences.version=1
+encoding//src/main/java=UTF-8
+encoding//src/main/resources=UTF-8
+encoding//src/test/java=UTF-8
+encoding//src/test/resources=UTF-8
+encoding/<project>=UTF-8


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

Posted by lu...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/cube/src/main/java/com/kylinolap/cube/project/ProjectInstance.java
----------------------------------------------------------------------
diff --git a/cube/src/main/java/com/kylinolap/cube/project/ProjectInstance.java b/cube/src/main/java/com/kylinolap/cube/project/ProjectInstance.java
new file mode 100644
index 0000000..1d4023e
--- /dev/null
+++ b/cube/src/main/java/com/kylinolap/cube/project/ProjectInstance.java
@@ -0,0 +1,234 @@
+/*
+ * 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.cube.project;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Set;
+import java.util.TreeSet;
+
+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;
+
+/**
+ * Project is a concept in Kylin similar to schema in DBMS
+ */
+@JsonAutoDetect(fieldVisibility = Visibility.NONE, getterVisibility = Visibility.NONE, isGetterVisibility = Visibility.NONE, setterVisibility = Visibility.NONE)
+public class ProjectInstance extends RootPersistentEntity {
+
+    public static final String DEFAULT_PROJECT_NAME = "DEFAULT";
+
+    @JsonProperty("name")
+    private String name;
+
+    @JsonProperty("cubes")
+    private List<String> cubes;
+
+    @JsonProperty("tables")
+    private Set<String> tables;
+
+
+    @JsonProperty("owner")
+    private String owner;
+
+    @JsonProperty("status")
+    private ProjectStatusEnum status;
+
+    @JsonProperty("create_time")
+    private String createTime;
+
+    @JsonProperty("last_update_time")
+    private String lastUpdateTime;
+
+    @JsonProperty("description")
+    private String description;
+
+    public String getResourcePath() {
+        return concatResourcePath(name);
+    }
+
+    public static String concatResourcePath(String projectName) {
+        return ResourceStore.PROJECT_RESOURCE_ROOT + "/" + projectName + ".json";
+    }
+
+    public static String getNormalizedProjectName(String project) {
+        if (project == null)
+            throw new IllegalStateException("Trying to normalized a project name which is null");
+
+        return project.toUpperCase();
+    }
+
+    // ============================================================================
+
+    public static ProjectInstance create(String name, String owner, String description, List<String> cubes) {
+        ProjectInstance projectInstance = new ProjectInstance();
+
+        projectInstance.updateRandomUuid();
+        projectInstance.setName(name);
+        projectInstance.setOwner(owner);
+        projectInstance.setDescription(description);
+        projectInstance.setStatus(ProjectStatusEnum.ENABLED);
+        projectInstance.setCreateTime(formatTime(System.currentTimeMillis()));
+        if (cubes != null)
+            projectInstance.setCubes(cubes);
+        else
+            projectInstance.setCubes(new ArrayList<String>());
+
+        return projectInstance;
+    }
+
+    public ProjectInstance() {
+
+    }
+
+    public ProjectInstance(String name, List<String> cubes, String owner) {
+        this.name = name;
+        this.cubes = cubes;
+        this.owner = owner;
+    }
+
+    public String getDescription() {
+        return description;
+    }
+
+    public void setDescription(String description) {
+        this.description = description;
+    }
+
+    public ProjectStatusEnum getStatus() {
+        return status;
+    }
+
+    public void setStatus(ProjectStatusEnum status) {
+        this.status = status;
+    }
+
+    public String getCreateTime() {
+        return createTime;
+    }
+
+    public void setCreateTime(String createTime) {
+        this.createTime = createTime;
+    }
+
+    public String getName() {
+        return this.name;
+    }
+
+    public void setName(String name) {
+        this.name = name;
+    }
+
+    public boolean containsCube(String cubeName) {
+        cubeName = cubeName.toUpperCase();
+        return cubes.contains(cubeName);
+    }
+
+    public void removeCube(String cubeName) {
+        cubeName = cubeName.toUpperCase();
+        cubes.remove(cubeName);
+    }
+
+    public int getCubesCount() {
+        return cubes.size();
+    }
+
+    public void addCube(String cubeName) {
+        cubeName = cubeName.toUpperCase();
+        this.cubes.add(cubeName);
+    }
+
+    public List<String> getCubes() {
+        return cubes;
+    }
+
+
+    public void setCubes(List<String> cubes) {
+        this.cubes = cubes;
+    }
+
+    public void setTables(Set<String> tables) {
+        this.tables = tables;
+    }
+
+    public boolean containsTable(String tableName) {
+        tableName = tableName.toUpperCase();
+        return tables.contains(tableName);
+    }
+
+    public void removeTable(String tableName) {
+        tableName = tableName.toUpperCase();
+        tables.remove(tableName);
+    }
+
+    public int getTablesCount() {
+        return this.getTables().size();
+    }
+
+    public void addTable(String tableName) {
+        tableName = tableName.toUpperCase();
+        this.getTables().add(tableName);
+    }
+
+    //will return new Set for null
+    public Set<String> getTables() {
+        tables = tables == null ? new TreeSet<String>() : tables;
+        return tables;
+    }
+
+    public String getOwner() {
+        return owner;
+    }
+
+    public void setOwner(String owner) {
+        this.owner = owner;
+    }
+
+    public String getLastUpdateTime() {
+        return lastUpdateTime;
+    }
+
+    public void setLastUpdateTime(String lastUpdateTime) {
+        this.lastUpdateTime = lastUpdateTime;
+    }
+
+    public void recordUpdateTime(long timeMillis) {
+        this.lastUpdateTime = formatTime(timeMillis);
+    }
+
+
+    public void init() {
+        if (name == null)
+            name = ProjectInstance.DEFAULT_PROJECT_NAME;
+
+        if (cubes == null) {
+            cubes = new ArrayList<String>();
+        }
+
+        for (int i = 0; i < cubes.size(); ++i) {
+            if (cubes.get(i) != null)
+                cubes.set(i, cubes.get(i).toUpperCase());
+        }
+    }
+
+    @Override
+    public String toString() {
+        return "ProjectDesc [name=" + name + "]";
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/cube/src/main/java/com/kylinolap/cube/project/ProjectManager.java
----------------------------------------------------------------------
diff --git a/cube/src/main/java/com/kylinolap/cube/project/ProjectManager.java b/cube/src/main/java/com/kylinolap/cube/project/ProjectManager.java
new file mode 100644
index 0000000..e05eae2
--- /dev/null
+++ b/cube/src/main/java/com/kylinolap/cube/project/ProjectManager.java
@@ -0,0 +1,582 @@
+/*
+ * 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.cube.project;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.concurrent.ConcurrentHashMap;
+
+import org.apache.commons.lang3.StringUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.google.common.collect.HashMultimap;
+import com.google.common.collect.Lists;
+import com.google.common.collect.Multimap;
+import com.google.common.collect.Multimaps;
+import com.kylinolap.common.KylinConfig;
+import com.kylinolap.common.persistence.JsonSerializer;
+import com.kylinolap.common.persistence.ResourceStore;
+import com.kylinolap.common.persistence.Serializer;
+import com.kylinolap.common.restclient.Broadcaster;
+import com.kylinolap.common.restclient.SingleValueCache;
+import com.kylinolap.cube.CubeInstance;
+import com.kylinolap.cube.CubeManager;
+import com.kylinolap.metadata.MetadataManager;
+import com.kylinolap.metadata.model.cube.CubeDesc;
+import com.kylinolap.metadata.model.cube.DimensionDesc;
+import com.kylinolap.metadata.model.cube.FunctionDesc;
+import com.kylinolap.metadata.model.cube.JoinDesc;
+import com.kylinolap.metadata.model.cube.MeasureDesc;
+import com.kylinolap.metadata.model.cube.TblColRef;
+import com.kylinolap.metadata.model.schema.ColumnDesc;
+import com.kylinolap.metadata.model.schema.TableDesc;
+
+/**
+ * @author xduo
+ */
+public class ProjectManager {
+    private static final Logger logger = LoggerFactory.getLogger(ProjectManager.class);
+
+    // static cached instances
+    private static final ConcurrentHashMap<KylinConfig, ProjectManager> CACHE = new ConcurrentHashMap<KylinConfig, ProjectManager>();
+    private static final Serializer<ProjectInstance> PROJECT_SERIALIZER = new JsonSerializer<ProjectInstance>(ProjectInstance.class);
+
+    private KylinConfig config;
+    // project name => ProjrectDesc
+    private SingleValueCache<String, ProjectInstance> projectMap = new SingleValueCache<String, ProjectInstance>(Broadcaster.TYPE.PROJECT);
+    // project name => tables
+    private Multimap<String, ProjectTable> projectTables = Multimaps.synchronizedMultimap(HashMultimap.<String, ProjectTable>create());
+
+    public static ProjectManager getInstance(KylinConfig config) {
+        ProjectManager r = CACHE.get(config);
+        if (r != null) {
+            return r;
+        }
+
+        synchronized (ProjectManager.class) {
+            r = CACHE.get(config);
+            if (r != null) {
+                return r;
+            }
+            try {
+                r = new ProjectManager(config);
+                CACHE.put(config, r);
+                if (CACHE.size() > 1) {
+                    logger.warn("More than one singleton exist");
+                }
+                return r;
+            } catch (IOException e) {
+                throw new IllegalStateException("Failed to init CubeManager from " + config, e);
+            }
+        }
+    }
+
+    public static synchronized void removeInstance(KylinConfig config) {
+        CACHE.remove(config);
+    }
+
+    private ProjectManager(KylinConfig config) throws IOException {
+        logger.info("Initializing CubeManager with metadata url " + config);
+        this.config = config;
+
+        loadAllProjects();
+    }
+
+    public static String getDefaultProjectName() {
+        return ProjectInstance.DEFAULT_PROJECT_NAME;
+    }
+
+    public List<ProjectInstance> listAllProjects() {
+        return new ArrayList<ProjectInstance>(projectMap.values());
+    }
+
+    public List<ProjectInstance> getProjects(String cubeName) {
+        return this.findProjects(cubeName);
+    }
+
+    public ProjectInstance dropProject(String projectName) throws IOException {
+        if (projectName == null)
+            throw new IllegalArgumentException("Project name not given");
+
+        ProjectInstance projectInstance = getProject(projectName);
+
+        if (projectInstance == null) {
+            throw new IllegalStateException("The project named " + projectName + " does not exist");
+        }
+
+        if (projectInstance.getCubes().size() != 0) {
+            throw new IllegalStateException("The project named " + projectName + " can not be deleted because there's still cubes in it. Delete all the cubes first.");
+        }
+
+        logger.info("Dropping project '" + projectInstance.getName() + "'");
+        deleteResource(projectInstance);
+
+        return projectInstance;
+    }
+
+    public ProjectInstance getProject(String projectName) {
+        if (projectName == null)
+            return null;
+        projectName = ProjectInstance.getNormalizedProjectName(projectName);
+        return projectMap.get(projectName);
+    }
+
+    public ProjectInstance createProject(String projectName, String owner, String description) throws IOException {
+
+        logger.info("Creating project '" + projectName);
+
+        ProjectInstance currentProject = getProject(projectName);
+        if (currentProject == null) {
+            currentProject = ProjectInstance.create(projectName, owner, description, null);
+        } else {
+            throw new IllegalStateException("The project named " + projectName + "already exists");
+        }
+
+        saveResource(currentProject);
+
+        return currentProject;
+    }
+
+    public ProjectInstance updateProject(ProjectInstance project, String newName, String newDesc) throws IOException {
+        if (!project.getName().equals(newName)) {
+            ProjectInstance newProject = this.createProject(newName, project.getOwner(), newDesc);
+            newProject.setCreateTime(project.getCreateTime());
+            newProject.recordUpdateTime(System.currentTimeMillis());
+            newProject.setCubes(project.getCubes());
+
+            deleteResource(project);
+            saveResource(newProject);
+
+            return newProject;
+        } else {
+            project.setName(newName);
+            project.setDescription(newDesc);
+
+            if (project.getUuid() == null)
+                project.updateRandomUuid();
+
+            saveResource(project);
+
+            return project;
+        }
+    }
+
+    public boolean isCubeInProject(String projectName, CubeInstance cube) {
+        return this.listAllCubes(projectName).contains(cube);
+    }
+
+    public ProjectInstance updateCubeToProject(String cubeName, String newProjectName, String owner) throws IOException {
+        removeCubeFromProjects(cubeName);
+
+        return addCubeToProject(cubeName, newProjectName, owner);
+    }
+
+    public ProjectInstance updateTableToProject(String[] tables, String projectName) throws IOException {
+        ProjectInstance projectInstance = getProject(projectName);
+        for (int i = 0; i < tables.length; i++) {
+            String token = tables[i].trim();
+            int cut = token.indexOf('.');
+            String tableName = cut >= 0 ? token.substring(cut + 1).trim() : token.trim();
+            if (StringUtils.isNotEmpty(tableName)) {
+                projectInstance.addTable(tableName);
+            }
+        }
+
+        List<TableDesc> exposedTables = listExposedTables(projectName);
+        for (TableDesc table : exposedTables) {
+            projectInstance.addTable(table.getName());
+        }
+
+        saveResource(projectInstance);
+        return projectInstance;
+    }
+
+
+    public void removeCubeFromProjects(String cubeName) throws IOException {
+        for (ProjectInstance projectInstance : findProjects(cubeName)) {
+            projectInstance.removeCube(cubeName);
+
+            saveResource(projectInstance);
+        }
+    }
+
+    public List<TableDesc> listExposedTables(String project) {
+        project = ProjectInstance.getNormalizedProjectName(project);
+        List<TableDesc> tables = Lists.newArrayList();
+
+        for (ProjectTable table : projectTables.get(project)) {
+            TableDesc tableDesc = getMetadataManager().getTableDesc(table.getName());
+            if (tableDesc != null) {
+                tables.add(tableDesc);
+            }
+        }
+
+        return tables;
+    }
+
+
+    public List<TableDesc> listDefinedTablesInProject(String project) throws IOException {
+        if(null==project){
+            return Collections.emptyList();
+        }
+        project = ProjectInstance.getNormalizedProjectName(project);
+        ProjectInstance projectInstance = getProject(project);
+        int originTableCount = projectInstance.getTablesCount();
+        //sync exposed table to project when list
+        List<TableDesc> exposedTables = listExposedTables(project);
+        for (TableDesc table : exposedTables) {
+            projectInstance.addTable(table.getName());
+        }
+        //only save project json if new tables are sync in
+        if (originTableCount < projectInstance.getTablesCount()) {
+            saveResource(projectInstance);
+        }
+
+        List<TableDesc> tables = Lists.newArrayList();
+        for (String table : projectInstance.getTables()) {
+            TableDesc tableDesc = getMetadataManager().getTableDesc(table);
+            if (tableDesc != null) {
+                tables.add(tableDesc);
+            }
+        }
+
+        return tables;
+    }
+
+    public List<ColumnDesc> listExposedColumns(String project, String table) {
+        project = ProjectInstance.getNormalizedProjectName(project);
+
+        MetadataManager metaMgr = getMetadataManager();
+        TableDesc tableDesc = metaMgr.getTableDesc(table);
+        List<ColumnDesc> columns = Lists.newArrayList();
+
+        for (String column : this.getProjectTable(project, table).getColumns()) {
+            columns.add(tableDesc.findColumnByName(column));
+        }
+
+        return columns;
+    }
+
+    public boolean isExposedTable(String project, String table) {
+        project = ProjectInstance.getNormalizedProjectName(project);
+
+        return projectTables.containsEntry(project, new ProjectTable(table));
+    }
+
+    public boolean isExposedColumn(String project, String table, String col) {
+        project = ProjectInstance.getNormalizedProjectName(project);
+
+        return getProjectTable(project, table).getColumns().contains(col);
+    }
+
+    public List<CubeInstance> listAllCubes(String project) {
+        project = ProjectInstance.getNormalizedProjectName(project);
+
+        HashSet<CubeInstance> ret = new HashSet<CubeInstance>();
+
+        ProjectInstance projectInstance = getProject(project);
+        if (projectInstance != null) {
+            for (String cubeName : projectInstance.getCubes()) {
+                CubeInstance cube = CubeManager.getInstance(config).getCube(cubeName);
+                if (null != cube) {
+                    ret.add(cube);
+                } else {
+                    logger.error("Failed to load cube " + cubeName);
+                }
+            }
+        }
+
+        return new ArrayList<CubeInstance>(ret);
+    }
+
+
+    public List<CubeInstance> getCubesByTable(String project, String tableName) {
+        project = ProjectInstance.getNormalizedProjectName(project);
+        tableName = tableName.toUpperCase();
+        List<CubeInstance> cubes = new ArrayList<CubeInstance>();
+
+        ProjectTable projectTable = getProjectTable(project, tableName);
+        cubes.addAll(projectTable.getCubes());
+
+        return cubes;
+    }
+
+    public List<CubeInstance> getOnlineCubesByFactTable(String project, String factTableName) {
+        project = ProjectInstance.getNormalizedProjectName(project);
+        factTableName = factTableName.toUpperCase();
+        List<CubeInstance> cubes = new ArrayList<CubeInstance>();
+        ProjectTable projectTable = this.getProjectTable(project, factTableName);
+        for (CubeInstance cube : projectTable.getCubes()) {
+            if (cube.getDescriptor().isFactTable(factTableName) && cube.isReady()) {
+                cubes.add(cube);
+            }
+        }
+
+        return cubes;
+    }
+
+    public List<MeasureDesc> listEffectiveRewriteMeasures(String project, String factTable) {
+        factTable = factTable.toUpperCase();
+
+        HashSet<CubeDesc> relatedDesc = new HashSet<CubeDesc>();
+        for (CubeInstance cube : getProjectTable(project, factTable).getCubes()) {
+            if (cube.isReady() == false)
+                continue;
+            if (cube.getDescriptor().isFactTable(factTable) == false)
+                continue;
+
+            relatedDesc.add(cube.getDescriptor());
+        }
+
+        List<MeasureDesc> result = Lists.newArrayList();
+        for (CubeDesc desc : relatedDesc) {
+            for (MeasureDesc m : desc.getMeasures()) {
+                FunctionDesc func = m.getFunction();
+                if (func.needRewrite())
+                    result.add(m);
+            }
+        }
+
+        return result;
+    }
+
+    public void loadProjectCache(ProjectInstance project, boolean triggerUpdate) throws IOException {
+        loadProject(project.getResourcePath(), triggerUpdate);
+        loadTables(project.getResourcePath());
+    }
+
+    public void removeProjectCache(ProjectInstance project) {
+        String projectName = ProjectInstance.getNormalizedProjectName(project.getName());
+        if (projectMap.containsKey(projectName)) {
+            projectMap.remove(projectName);
+            projectTables.removeAll(projectName);
+        }
+    }
+
+    private void mapTableToCube(ProjectInstance projectInstance, CubeInstance cubeInstance) {
+        // schema sanity check
+        CubeDesc cubeDesc = cubeInstance.getDescriptor();
+        if (cubeDesc == null) {
+            logger.warn("No CubeDesc found by name '" + cubeInstance.getDescName() + "'");
+            return;
+        }
+
+        // table ==> cube mapping
+        String factTable = cubeDesc.getFactTable();
+        assert this.getMetadataManager().getTableDesc(factTable) != null;
+
+        String project = ProjectInstance.getNormalizedProjectName(projectInstance.getName());
+        ProjectTable factProjTable = this.getProjectTable(project, factTable, true);
+        if (!factProjTable.getCubes().contains(cubeInstance)) {
+            factProjTable.getCubes().add(cubeInstance);
+        }
+
+        for (DimensionDesc d : cubeDesc.getDimensions()) {
+            String lookupTable = d.getTable();
+            assert this.getMetadataManager().getTableDesc(lookupTable) != null;
+
+            ProjectTable dimensionProjTable = this.getProjectTable(project, lookupTable);
+            if (!dimensionProjTable.getCubes().contains(cubeInstance)) {
+                dimensionProjTable.getCubes().add(cubeInstance);
+            }
+        }
+    }
+
+    private List<ProjectInstance> findProjects(String cubeName) {
+        List<ProjectInstance> projects = new ArrayList<ProjectInstance>();
+        for (ProjectInstance projectInstance : projectMap.values()) {
+            if (projectInstance.containsCube(cubeName)) {
+                projects.add(projectInstance);
+            }
+        }
+
+        return projects;
+    }
+
+    private synchronized ProjectInstance loadProject(String path, boolean triggerUpdate) throws IOException {
+        ResourceStore store = getStore();
+
+        ProjectInstance projectInstance = store.getResource(path, ProjectInstance.class, PROJECT_SERIALIZER);
+        projectInstance.init();
+
+        if (StringUtils.isBlank(projectInstance.getName()))
+            throw new IllegalStateException("Project name must not be blank");
+
+        if (triggerUpdate) {
+            projectMap.put(projectInstance.getName().toUpperCase(), projectInstance);
+        } else {
+            projectMap.putLocal(projectInstance.getName().toUpperCase(), projectInstance);
+        }
+
+        return projectInstance;
+    }
+
+    private synchronized void loadTables(String path) throws IOException {
+        ResourceStore store = getStore();
+
+        ProjectInstance projectInstance = store.getResource(path, ProjectInstance.class, PROJECT_SERIALIZER);
+        projectInstance.init();
+
+        String project = ProjectInstance.getNormalizedProjectName(projectInstance.getName());
+        projectTables.removeAll(project);
+
+        for (CubeInstance cubeInstance : this.listAllCubes(projectInstance.getName())) {
+            markExposedTablesAndColumns(projectInstance.getName(), cubeInstance);
+            mapTableToCube(projectInstance, cubeInstance);
+        }
+    }
+
+    private void loadAllProjects() throws IOException {
+        ResourceStore store = getStore();
+        List<String> paths = store.collectResourceRecursively(ResourceStore.PROJECT_RESOURCE_ROOT, ".json");
+
+        logger.debug("Loading Project from folder " + store.getReadableResourcePath(ResourceStore.PROJECT_RESOURCE_ROOT));
+
+        for (String path : paths) {
+            loadProject(path, false);
+            loadTables(path);
+        }
+
+        logger.debug("Loaded " + paths.size() + " Project(s)");
+    }
+
+    private ProjectInstance addCubeToProject(String cubeName, String project, String user) throws IOException {
+        String newProjectName = ProjectInstance.getNormalizedProjectName(project);
+        ProjectInstance newProject = getProject(newProjectName);
+        if (newProject == null) {
+            newProject = this.createProject(newProjectName, user, "This is a project automatically added when adding cube " + cubeName);
+        }
+        newProject.addCube(cubeName);
+        saveResource(newProject);
+
+        return newProject;
+    }
+
+    private void saveResource(ProjectInstance proj) throws IOException {
+        ResourceStore store = getStore();
+        store.putResource(proj.getResourcePath(), proj, PROJECT_SERIALIZER);
+        afterProjectUpdated(proj);
+    }
+
+    private void deleteResource(ProjectInstance proj) throws IOException {
+        ResourceStore store = getStore();
+        store.deleteResource(proj.getResourcePath());
+        this.afterProjectDropped(proj);
+    }
+
+    private void afterProjectUpdated(ProjectInstance updatedProject) {
+        try {
+            this.loadProjectCache(updatedProject, true);
+        } catch (IOException e) {
+            logger.error(e.getLocalizedMessage(), e);
+        }
+    }
+
+    private void afterProjectDropped(ProjectInstance droppedProject) {
+        this.removeProjectCache(droppedProject);
+    }
+
+    // sync on update
+    private void markExposedTablesAndColumns(String projectName, CubeInstance cubeInstance) {
+        if (!cubeInstance.isReady())
+            return;
+
+        CubeDesc cubeDesc = cubeInstance.getDescriptor();
+        String factTable = cubeDesc.getFactTable();
+        for (DimensionDesc dim : cubeDesc.getDimensions()) {
+            String lookupTable = dim.getTable();
+            JoinDesc join = dim.getJoin();
+            if (join == null)
+                continue; // for dimensions on fact table, there's no join
+
+            if (join.getForeignKeyColumns() == null) {
+                throw new IllegalStateException("Null FK for " + join);
+            }
+            for (TblColRef fkCol : join.getForeignKeyColumns()) {
+                markExposedTableAndColumn(projectName, factTable, fkCol.getName(), dim);
+            }
+            if (join.getPrimaryKeyColumns() == null) {
+                throw new IllegalStateException("Null PK for " + join);
+            }
+            for (TblColRef pkCol : join.getPrimaryKeyColumns()) {
+                markExposedTableAndColumn(projectName, lookupTable, pkCol.getName(), dim);
+            }
+        }
+        for (TblColRef col : cubeDesc.listAllColumns()) {
+            markExposedTableAndColumn(projectName, col.getTable(), col.getName(), col);
+        }
+    }
+
+    private void markExposedTableAndColumn(String project, String table, String column, Object refObj) {
+        project = ProjectInstance.getNormalizedProjectName(project);
+        TableDesc t = this.getMetadataManager().getTableDesc(table);
+        if (t == null)
+            throw new IllegalStateException("No SourceTable found by name '" + table + "', ref by " + refObj);
+        table = t.getName(); // ensures upper case
+
+        ProjectTable projTable = getProjectTable(project, table, true);
+
+        ColumnDesc srcCol = t.findColumnByName(column);
+        if (srcCol == null)
+            throw new IllegalStateException("No SourceColumn found by name '" + table + "/" + column + "', ref by " + refObj);
+
+        if (!projTable.getColumns().contains(srcCol.getName())) {
+            projTable.getColumns().add(srcCol.getName());
+        }
+    }
+
+    private ProjectTable getProjectTable(String project, final String table) {
+        return getProjectTable(project, table, false);
+    }
+
+    private ProjectTable getProjectTable(String project, final String table, boolean autoCreate) {
+        ProjectTable projectTable = null;
+        project = ProjectInstance.getNormalizedProjectName(project);
+
+        if (this.projectTables.containsEntry(project, new ProjectTable(table))) {
+            Iterator<ProjectTable> projsIter = this.projectTables.get(project).iterator();
+            while (projsIter.hasNext()) {
+                ProjectTable oneTable = projsIter.next();
+                if (oneTable.getName().equalsIgnoreCase(table)) {
+                    projectTable = oneTable;
+                    break;
+                }
+            }
+        } else {
+            projectTable = new ProjectTable(table);
+
+            if (autoCreate) {
+                this.projectTables.put(project, projectTable);
+            }
+        }
+
+        return projectTable;
+    }
+
+    private ResourceStore getStore() {
+        return ResourceStore.getStore(this.config);
+    }
+
+    private MetadataManager getMetadataManager() {
+        return MetadataManager.getInstance(config);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/cube/src/main/java/com/kylinolap/cube/project/ProjectStatusEnum.java
----------------------------------------------------------------------
diff --git a/cube/src/main/java/com/kylinolap/cube/project/ProjectStatusEnum.java b/cube/src/main/java/com/kylinolap/cube/project/ProjectStatusEnum.java
new file mode 100644
index 0000000..0c582a7
--- /dev/null
+++ b/cube/src/main/java/com/kylinolap/cube/project/ProjectStatusEnum.java
@@ -0,0 +1,22 @@
+/*
+ * 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.cube.project;
+
+public enum ProjectStatusEnum {
+
+    DISABLED, ENABLED
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/cube/src/main/java/com/kylinolap/cube/project/ProjectTable.java
----------------------------------------------------------------------
diff --git a/cube/src/main/java/com/kylinolap/cube/project/ProjectTable.java b/cube/src/main/java/com/kylinolap/cube/project/ProjectTable.java
new file mode 100644
index 0000000..0344010
--- /dev/null
+++ b/cube/src/main/java/com/kylinolap/cube/project/ProjectTable.java
@@ -0,0 +1,93 @@
+/*
+ * 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.cube.project;
+
+import com.google.common.collect.HashMultiset;
+import com.google.common.collect.Multiset;
+import com.kylinolap.cube.CubeInstance;
+
+/**
+ * @author xduo
+ * 
+ */
+public class ProjectTable {
+
+    private final String name;
+
+    private Multiset<String> columns = HashMultiset.create();
+
+    private Multiset<CubeInstance> cubes = HashMultiset.create();
+
+    /**
+     * @param name
+     */
+    public ProjectTable(String name) {
+        super();
+        this.name = name.toUpperCase();
+    }
+
+    public String getName() {
+        return name;
+    }
+
+    public Multiset<String> getColumns() {
+        return columns;
+    }
+
+    public void setColumns(Multiset<String> columns) {
+        this.columns = columns;
+    }
+
+    public Multiset<CubeInstance> getCubes() {
+        return cubes;
+    }
+
+    public void setCubes(Multiset<CubeInstance> cubes) {
+        this.cubes = cubes;
+    }
+
+    @Override
+    public int hashCode() {
+        final int prime = 31;
+        int result = 1;
+        result = prime * result + ((name == null) ? 0 : name.hashCode());
+        return result;
+    }
+
+    @Override
+    public boolean equals(Object obj) {
+        if (this == obj)
+            return true;
+        if (obj == null)
+            return false;
+        if (getClass() != obj.getClass())
+            return false;
+        ProjectTable other = (ProjectTable) obj;
+        if (name == null) {
+            if (other.name != null)
+                return false;
+        } else if (!name.equals(other.name))
+            return false;
+        return true;
+    }
+
+    @Override
+    public String toString() {
+        return "ProjectTable [name=" + name + ", columns=" + columns + "]";
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/cube/src/test/java/com/kylinolap/cube/CubeManagerCacheTest.java
----------------------------------------------------------------------
diff --git a/cube/src/test/java/com/kylinolap/cube/CubeManagerCacheTest.java b/cube/src/test/java/com/kylinolap/cube/CubeManagerCacheTest.java
new file mode 100644
index 0000000..372cc17
--- /dev/null
+++ b/cube/src/test/java/com/kylinolap/cube/CubeManagerCacheTest.java
@@ -0,0 +1,75 @@
+/*
+ * 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.cube;
+
+import static org.junit.Assert.*;
+
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+import com.kylinolap.common.persistence.ResourceStore;
+import com.kylinolap.common.util.LocalFileMetadataTestCase;
+import com.kylinolap.cube.project.ProjectManager;
+import com.kylinolap.metadata.MetadataManager;
+import com.kylinolap.metadata.model.cube.CubeDesc;
+
+/**
+ * @author yangli9
+ * 
+ */
+public class CubeManagerCacheTest extends LocalFileMetadataTestCase {
+
+    private CubeManager cubeManager;
+
+    @Before
+    public void setUp() throws Exception {
+        this.createTestMetadata();
+        MetadataManager.removeInstance(this.getTestConfig());
+        CubeManager.removeInstance(this.getTestConfig());
+        ProjectManager.removeInstance(this.getTestConfig());
+        cubeManager = CubeManager.getInstance(this.getTestConfig());
+    }
+
+    @After
+    public void after() throws Exception {
+        this.cleanupTestMetadata();
+    }
+
+    @Test
+    public void testReloadCache() throws Exception {
+        ResourceStore store = getStore();
+
+        // clean legacy in case last run failed
+        store.deleteResource("/cube/a_whole_new_cube.json");
+        MetadataManager metaMgr = getMetadataManager();
+        CubeDesc desc = metaMgr.getCubeDesc("test_kylin_cube_with_slr_desc");
+        cubeManager.createCube("a_whole_new_cube", "default", desc, null);
+
+        CubeInstance createdCube = cubeManager.getCube("a_whole_new_cube");
+        assertEquals(0, createdCube.getSegments().size());
+        assertEquals(CubeStatusEnum.DISABLED, createdCube.getStatus());
+        createdCube.setStatus(CubeStatusEnum.DESCBROKEN);
+
+        cubeManager.updateCube(createdCube);
+        assertEquals(CubeStatusEnum.DESCBROKEN, cubeManager.getCube("a_whole_new_cube").getStatus());
+    }
+
+    private MetadataManager getMetadataManager() {
+        return MetadataManager.getInstance(getTestConfig());
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/cube/src/test/java/com/kylinolap/cube/CubeManagerTest.java
----------------------------------------------------------------------
diff --git a/cube/src/test/java/com/kylinolap/cube/CubeManagerTest.java b/cube/src/test/java/com/kylinolap/cube/CubeManagerTest.java
new file mode 100644
index 0000000..82d2c0e
--- /dev/null
+++ b/cube/src/test/java/com/kylinolap/cube/CubeManagerTest.java
@@ -0,0 +1,88 @@
+/*
+ * 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.cube;
+
+import static org.junit.Assert.*;
+
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+import com.kylinolap.common.persistence.ResourceStore;
+import com.kylinolap.common.util.JsonUtil;
+import com.kylinolap.common.util.LocalFileMetadataTestCase;
+import com.kylinolap.cube.project.ProjectInstance;
+import com.kylinolap.cube.project.ProjectManager;
+import com.kylinolap.metadata.MetadataManager;
+import com.kylinolap.metadata.model.cube.CubeDesc;
+
+/**
+ * @author yangli9
+ */
+public class CubeManagerTest extends LocalFileMetadataTestCase {
+
+    @Before
+    public void setUp() throws Exception {
+        this.createTestMetadata();
+        MetadataManager.removeInstance(this.getTestConfig());
+        CubeManager.removeInstance(this.getTestConfig());
+        ProjectManager.removeInstance(this.getTestConfig());
+    }
+
+    @After
+    public void after() throws Exception {
+        this.cleanupTestMetadata();
+    }
+
+    @Test
+    public void testBasics() throws Exception {
+        CubeInstance cube = CubeManager.getInstance(this.getTestConfig()).getCube("test_kylin_cube_without_slr_ready");
+        CubeDesc desc = cube.getDescriptor();
+        System.out.println(JsonUtil.writeValueAsIndentString(desc));
+
+        String signature = desc.calculateSignature();
+        desc.getCubePartitionDesc().setPartitionDateColumn("test_column");
+        assertTrue(!signature.equals(desc.calculateSignature()));
+    }
+
+    @Test
+    public void testCreateAndDrop() throws Exception {
+
+        ResourceStore store = getStore();
+
+        // clean legacy in case last run failed
+        store.deleteResource("/cube/a_whole_new_cube.json");
+
+        MetadataManager metaMgr = getMetadataManager();
+        CubeDesc desc = metaMgr.getCubeDesc("test_kylin_cube_with_slr_desc");
+        CubeInstance createdCube = CubeManager.getInstance(this.getTestConfig()).createCube("a_whole_new_cube", ProjectInstance.DEFAULT_PROJECT_NAME, desc, null);
+        assertTrue(createdCube == CubeManager.getInstance(this.getTestConfig()).getCube("a_whole_new_cube"));
+
+        assertTrue(ProjectManager.getInstance(getTestConfig()).listAllCubes(ProjectInstance.DEFAULT_PROJECT_NAME).contains(createdCube));
+
+        CubeInstance droppedCube = CubeManager.getInstance(this.getTestConfig()).dropCube("a_whole_new_cube", true);
+        assertTrue(createdCube == droppedCube);
+
+        assertTrue(!ProjectManager.getInstance(getTestConfig()).listAllCubes(ProjectInstance.DEFAULT_PROJECT_NAME).contains(droppedCube));
+
+        assertNull(CubeManager.getInstance(this.getTestConfig()).getCube("a_whole_new_cube"));
+    }
+
+    private MetadataManager getMetadataManager() {
+        return MetadataManager.getInstance(getTestConfig());
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/cube/src/test/java/com/kylinolap/cube/CubeSizeEstimationCLITest.java
----------------------------------------------------------------------
diff --git a/cube/src/test/java/com/kylinolap/cube/CubeSizeEstimationCLITest.java b/cube/src/test/java/com/kylinolap/cube/CubeSizeEstimationCLITest.java
new file mode 100644
index 0000000..0f0a4b2
--- /dev/null
+++ b/cube/src/test/java/com/kylinolap/cube/CubeSizeEstimationCLITest.java
@@ -0,0 +1,94 @@
+package com.kylinolap.cube;
+
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+import com.kylinolap.common.util.LocalFileMetadataTestCase;
+import com.kylinolap.cube.estimation.CubeSizeEstimationCLI;
+import com.kylinolap.metadata.MetadataManager;
+import com.kylinolap.metadata.model.cube.CubeDesc;
+
+/**
+ * Created by honma on 9/1/14.
+ */
+public class CubeSizeEstimationCLITest extends LocalFileMetadataTestCase {
+
+    String cubeName = "test_kylin_cube_with_slr_ready";
+    long[] cardinality;
+    CubeDesc cubeDesc;
+
+    @Before
+    public void setUp() throws Exception {
+        this.createTestMetadata();
+        MetadataManager.removeInstance(this.getTestConfig());
+
+        String cubeName = "test_kylin_cube_with_slr_ready";
+        CubeManager cubeManager = CubeManager.getInstance(getTestConfig());
+        CubeInstance cubeInstance = cubeManager.getCube(cubeName);
+        cubeDesc = cubeInstance.getDescriptor();
+        cardinality = new long[] { 100, 100, 100, 10000, 1000, 100, 100, 100, 100 };
+
+    }
+
+    @After
+    public void after() throws Exception {
+        this.cleanupTestMetadata();
+    }
+
+    @Test
+    public void test() {
+        long[] x = new long[] { 5, 8, 5, 115, 122, 127, 137, 236, 101 };
+        CubeSizeEstimationCLI.estimatedCubeSize(cubeName, x);
+    }
+
+    @Test
+    public void baseCuboidTest() {
+        long cuboidID = getCuboidID(0, 1, 2, 3, 4, 5, 6, 7, 8);
+        long size = CubeSizeEstimationCLI.estimateCuboidSpace(cuboidID, cardinality, cubeDesc);
+
+        assert size == (10000000000000000L * (32 + 39));
+    }
+
+    @Test
+    public void cuboidTest1() {
+        long cuboidID = getCuboidID(0, 1, 2, 4, 5, 6, 7, 8);
+        long size = CubeSizeEstimationCLI.estimateCuboidSpace(cuboidID, cardinality, cubeDesc);
+
+        assert size == (1000000000000000L * (32 + 37));
+    }
+
+    @Test
+    public void cuboidTest2() {
+
+        long cuboidID = getCuboidID(0);
+        long size = CubeSizeEstimationCLI.estimateCuboidSpace(cuboidID, cardinality, cubeDesc);
+
+        assert size == (100L * (1 + 32));
+    }
+
+    @Test
+    public void cuboidTest3() {
+        long cuboidID = getCuboidID(4, 5);
+        long size = CubeSizeEstimationCLI.estimateCuboidSpace(cuboidID, cardinality, cubeDesc);
+
+        assert size == (1000L * (3 + 32));
+    }
+
+    @Test
+    public void cuboidTest4() {
+        long cuboidID = getCuboidID(4, 5, 6);
+        long size = CubeSizeEstimationCLI.estimateCuboidSpace(cuboidID, cardinality, cubeDesc);
+
+        assert size == (100000L * (4 + 32));
+    }
+
+    private long getCuboidID(int... bitIndice) {
+        long ret = 0;
+        long mask = 1L;
+        for (int index : bitIndice) {
+            ret |= mask << index;
+        }
+        return ret;
+    }
+}


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

Posted by lu...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/jdbc/src/main/java/com/kylinolap/jdbc/stub/KylinColumnMetaData.java
----------------------------------------------------------------------
diff --git a/jdbc/src/main/java/com/kylinolap/jdbc/stub/KylinColumnMetaData.java b/jdbc/src/main/java/com/kylinolap/jdbc/stub/KylinColumnMetaData.java
new file mode 100644
index 0000000..a3d5c84
--- /dev/null
+++ b/jdbc/src/main/java/com/kylinolap/jdbc/stub/KylinColumnMetaData.java
@@ -0,0 +1,36 @@
+/*
+ * 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.jdbc.stub;
+
+import java.sql.DatabaseMetaData;
+
+import net.hydromatic.avatica.ColumnMetaData;
+
+/**
+ * @author xduo
+ * 
+ */
+public class KylinColumnMetaData extends ColumnMetaData {
+
+    public KylinColumnMetaData(int ordinal, boolean autoIncrement, boolean caseSensitive, boolean searchable, boolean currency, int nullable, boolean signed, int displaySize, String label, String columnName, String schemaName, int precision, int scale, String tableName, String catalogName, AvaticaType type, boolean readOnly, boolean writable, boolean definitelyWritable, String columnClassName) {
+        super(ordinal, autoIncrement, caseSensitive, searchable, currency, nullable, signed, displaySize, label, columnName, schemaName, precision, scale, tableName, catalogName, type, readOnly, writable, definitelyWritable, columnClassName);
+    }
+
+    public static ColumnMetaData dummy(int ordinal, String label, String columnName, AvaticaType type, boolean nullable) {
+        return new ColumnMetaData(ordinal, false, true, false, false, nullable ? DatabaseMetaData.columnNullable : DatabaseMetaData.columnNoNulls, true, -1, label, columnName, null, -1, -1, null, null, type, true, false, false, null);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/jdbc/src/main/java/com/kylinolap/jdbc/stub/RemoteClient.java
----------------------------------------------------------------------
diff --git a/jdbc/src/main/java/com/kylinolap/jdbc/stub/RemoteClient.java b/jdbc/src/main/java/com/kylinolap/jdbc/stub/RemoteClient.java
new file mode 100644
index 0000000..43ded3b
--- /dev/null
+++ b/jdbc/src/main/java/com/kylinolap/jdbc/stub/RemoteClient.java
@@ -0,0 +1,57 @@
+/*
+ * 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.jdbc.stub;
+
+import java.sql.SQLException;
+
+import net.hydromatic.avatica.AvaticaStatement;
+
+import com.kylinolap.jdbc.KylinMetaImpl.MetaProject;
+
+/**
+ * Remote query stub of kylin restful service
+ * 
+ * @author xduo
+ * 
+ */
+public interface RemoteClient {
+
+    /**
+     * Connect to kylin restful service. ConnectionException will be thrown if
+     * authentication failed.
+     * 
+     * @throws ConnectionException
+     */
+    public void connect() throws ConnectionException;
+
+    /**
+     * @param project
+     * @return
+     */
+    public MetaProject getMetadata(String project) throws ConnectionException;
+
+    /**
+     * Run query
+     * 
+     * @param statement
+     * @param sql
+     * @return
+     * @throws SQLException
+     */
+    public DataSet<Object[]> query(AvaticaStatement statement, String sql) throws SQLException;
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/jdbc/src/main/java/com/kylinolap/jdbc/stub/SQLResponseStub.java
----------------------------------------------------------------------
diff --git a/jdbc/src/main/java/com/kylinolap/jdbc/stub/SQLResponseStub.java b/jdbc/src/main/java/com/kylinolap/jdbc/stub/SQLResponseStub.java
new file mode 100644
index 0000000..d980d8d
--- /dev/null
+++ b/jdbc/src/main/java/com/kylinolap/jdbc/stub/SQLResponseStub.java
@@ -0,0 +1,320 @@
+/*
+ * 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.jdbc.stub;
+
+import java.io.Serializable;
+import java.util.List;
+
+/**
+ * @author xduo
+ * 
+ */
+public class SQLResponseStub implements Serializable {
+    private static final long serialVersionUID = 1L;
+
+    // private static final Logger logger =
+    // LoggerFactory.getLogger(SQLResponse.class);
+
+    // the data type for each column
+    private List<ColumnMetaStub> columnMetas;
+
+    // the results rows, each row contains several columns
+    private List<String[]> results;
+
+    private String cube;
+
+    // if not select query, only return affected row count
+    private int affectedRowCount;
+
+    // if isException, the detailed exception message
+    private String exceptionMessage;
+
+    private boolean isException;
+
+    private long duration;
+
+    private boolean isPartial = false;
+
+    private long totalScanCount;
+
+    private boolean hitCache = false;
+
+    public SQLResponseStub() {
+    }
+
+    public List<ColumnMetaStub> getColumnMetas() {
+        return columnMetas;
+    }
+
+    public void setColumnMetas(List<ColumnMetaStub> columnMetas) {
+        this.columnMetas = columnMetas;
+    }
+
+    public List<String[]> getResults() {
+        return results;
+    }
+
+    public void setResults(List<String[]> results) {
+        this.results = results;
+    }
+
+    public String getCube() {
+        return cube;
+    }
+
+    public void setCube(String cube) {
+        this.cube = cube;
+    }
+
+    public int getAffectedRowCount() {
+        return affectedRowCount;
+    }
+
+    public void setAffectedRowCount(int affectedRowCount) {
+        this.affectedRowCount = affectedRowCount;
+    }
+
+    public boolean getIsException() {
+        return isException;
+    }
+
+    public void setIsException(boolean isException) {
+        this.isException = isException;
+    }
+
+    public String getExceptionMessage() {
+        return exceptionMessage;
+    }
+
+    public void setExceptionMessage(String exceptionMessage) {
+        this.exceptionMessage = exceptionMessage;
+    }
+
+    public long getDuration() {
+        return duration;
+    }
+
+    public void setDuration(long duration) {
+        this.duration = duration;
+    }
+
+    public boolean isPartial() {
+        return isPartial;
+    }
+
+    public void setPartial(boolean isPartial) {
+        this.isPartial = isPartial;
+    }
+
+    public long getTotalScanCount() {
+        return totalScanCount;
+    }
+
+    public void setTotalScanCount(long totalScanCount) {
+        this.totalScanCount = totalScanCount;
+    }
+
+    public boolean isHitCache() {
+        return hitCache;
+    }
+
+    public void setHitCache(boolean hitCache) {
+        this.hitCache = hitCache;
+    }
+
+    public static long getSerialversionuid() {
+        return serialVersionUID;
+    }
+
+    public static class ColumnMetaStub {
+
+        private boolean isAutoIncrement;
+        private boolean isCaseSensitive;
+        private boolean isSearchable;
+        private boolean isCurrency;
+        private int isNullable;// 0:nonull, 1:nullable, 2: nullableunknown
+        private boolean isSigned;
+        private int displaySize;
+        private String label;// AS keyword
+        private String name;
+        private String schemaName;
+        private String catelogName;
+        private String tableName;
+        private int precision;
+        private int scale;
+        private int columnType;// as defined in java.sql.Types
+        private String columnTypeName;
+        private boolean isReadOnly;
+        private boolean isWritable;
+        private boolean isDefinitelyWritable;
+
+        public ColumnMetaStub() {
+        }
+
+        public boolean isAutoIncrement() {
+            return isAutoIncrement;
+        }
+
+        public void setAutoIncrement(boolean isAutoIncrement) {
+            this.isAutoIncrement = isAutoIncrement;
+        }
+
+        public boolean isCaseSensitive() {
+            return isCaseSensitive;
+        }
+
+        public void setCaseSensitive(boolean isCaseSensitive) {
+            this.isCaseSensitive = isCaseSensitive;
+        }
+
+        public boolean isSearchable() {
+            return isSearchable;
+        }
+
+        public void setSearchable(boolean isSearchable) {
+            this.isSearchable = isSearchable;
+        }
+
+        public boolean isCurrency() {
+            return isCurrency;
+        }
+
+        public void setCurrency(boolean isCurrency) {
+            this.isCurrency = isCurrency;
+        }
+
+        public int getIsNullable() {
+            return isNullable;
+        }
+
+        public void setIsNullable(int isNullable) {
+            this.isNullable = isNullable;
+        }
+
+        public boolean isSigned() {
+            return isSigned;
+        }
+
+        public void setSigned(boolean isSigned) {
+            this.isSigned = isSigned;
+        }
+
+        public int getDisplaySize() {
+            return displaySize;
+        }
+
+        public void setDisplaySize(int displaySize) {
+            this.displaySize = displaySize;
+        }
+
+        public String getLabel() {
+            return label;
+        }
+
+        public void setLabel(String label) {
+            this.label = label;
+        }
+
+        public String getName() {
+            return name;
+        }
+
+        public void setName(String name) {
+            this.name = name;
+        }
+
+        public String getSchemaName() {
+            return schemaName;
+        }
+
+        public void setSchemaName(String schemaName) {
+            this.schemaName = schemaName;
+        }
+
+        public String getCatelogName() {
+            return catelogName;
+        }
+
+        public void setCatelogName(String catelogName) {
+            this.catelogName = catelogName;
+        }
+
+        public String getTableName() {
+            return tableName;
+        }
+
+        public void setTableName(String tableName) {
+            this.tableName = tableName;
+        }
+
+        public int getPrecision() {
+            return precision;
+        }
+
+        public void setPrecision(int precision) {
+            this.precision = precision;
+        }
+
+        public int getScale() {
+            return scale;
+        }
+
+        public void setScale(int scale) {
+            this.scale = scale;
+        }
+
+        public int getColumnType() {
+            return columnType;
+        }
+
+        public void setColumnType(int columnType) {
+            this.columnType = columnType;
+        }
+
+        public String getColumnTypeName() {
+            return columnTypeName;
+        }
+
+        public void setColumnTypeName(String columnTypeName) {
+            this.columnTypeName = columnTypeName;
+        }
+
+        public boolean isReadOnly() {
+            return isReadOnly;
+        }
+
+        public void setReadOnly(boolean isReadOnly) {
+            this.isReadOnly = isReadOnly;
+        }
+
+        public boolean isWritable() {
+            return isWritable;
+        }
+
+        public void setWritable(boolean isWritable) {
+            this.isWritable = isWritable;
+        }
+
+        public boolean isDefinitelyWritable() {
+            return isDefinitelyWritable;
+        }
+
+        public void setDefinitelyWritable(boolean isDefinitelyWritable) {
+            this.isDefinitelyWritable = isDefinitelyWritable;
+        }
+
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/jdbc/src/main/java/com/kylinolap/jdbc/stub/TableMetaStub.java
----------------------------------------------------------------------
diff --git a/jdbc/src/main/java/com/kylinolap/jdbc/stub/TableMetaStub.java b/jdbc/src/main/java/com/kylinolap/jdbc/stub/TableMetaStub.java
new file mode 100644
index 0000000..88290b4
--- /dev/null
+++ b/jdbc/src/main/java/com/kylinolap/jdbc/stub/TableMetaStub.java
@@ -0,0 +1,329 @@
+package com.kylinolap.jdbc.stub;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import com.fasterxml.jackson.annotation.JsonProperty;
+
+/**
+ * @author xduo
+ * 
+ */
+public class TableMetaStub {
+
+    private static final long serialVersionUID = 1L;
+    private String TABLE_CAT;
+    private String TABLE_SCHEM;
+    private String TABLE_NAME;
+    private String TABLE_TYPE;
+    private String REMARKS;
+    private String TYPE_CAT;
+    private String TYPE_SCHEM;
+    private String TYPE_NAME;
+    private String SELF_REFERENCING_COL_NAME;
+    private String REF_GENERATION;
+    @JsonProperty("columns")
+    private List<ColumnMetaStub> columns = new ArrayList<ColumnMetaStub>();
+
+    public String getTABLE_CAT() {
+        return TABLE_CAT;
+    }
+
+    public void setTABLE_CAT(String tABLE_CAT) {
+        TABLE_CAT = tABLE_CAT;
+    }
+
+    public String getTABLE_SCHEM() {
+        return TABLE_SCHEM;
+    }
+
+    public void setTABLE_SCHEM(String tABLE_SCHEM) {
+        TABLE_SCHEM = tABLE_SCHEM;
+    }
+
+    public String getTABLE_NAME() {
+        return TABLE_NAME;
+    }
+
+    public void setTABLE_NAME(String tABLE_NAME) {
+        TABLE_NAME = tABLE_NAME;
+    }
+
+    public String getTABLE_TYPE() {
+        return TABLE_TYPE;
+    }
+
+    public void setTABLE_TYPE(String tABLE_TYPE) {
+        TABLE_TYPE = tABLE_TYPE;
+    }
+
+    public String getREMARKS() {
+        return REMARKS;
+    }
+
+    public void setREMARKS(String rEMARKS) {
+        REMARKS = rEMARKS;
+    }
+
+    public String getTYPE_CAT() {
+        return TYPE_CAT;
+    }
+
+    public void setTYPE_CAT(String tYPE_CAT) {
+        TYPE_CAT = tYPE_CAT;
+    }
+
+    public String getTYPE_SCHEM() {
+        return TYPE_SCHEM;
+    }
+
+    public void setTYPE_SCHEM(String tYPE_SCHEM) {
+        TYPE_SCHEM = tYPE_SCHEM;
+    }
+
+    public String getTYPE_NAME() {
+        return TYPE_NAME;
+    }
+
+    public void setTYPE_NAME(String tYPE_NAME) {
+        TYPE_NAME = tYPE_NAME;
+    }
+
+    public String getSELF_REFERENCING_COL_NAME() {
+        return SELF_REFERENCING_COL_NAME;
+    }
+
+    public void setSELF_REFERENCING_COL_NAME(String sELF_REFERENCING_COL_NAME) {
+        SELF_REFERENCING_COL_NAME = sELF_REFERENCING_COL_NAME;
+    }
+
+    public String getREF_GENERATION() {
+        return REF_GENERATION;
+    }
+
+    public void setREF_GENERATION(String rEF_GENERATION) {
+        REF_GENERATION = rEF_GENERATION;
+    }
+
+    public List<ColumnMetaStub> getColumns() {
+        return columns;
+    }
+
+    public void setColumns(List<ColumnMetaStub> columns) {
+        this.columns = columns;
+    }
+
+    public static long getSerialversionuid() {
+        return serialVersionUID;
+    }
+
+    public static class ColumnMetaStub {
+        private String TABLE_CAT;
+        private String TABLE_SCHEM;
+        private String TABLE_NAME;
+        private String COLUMN_NAME;
+        private int DATA_TYPE;
+        private String TYPE_NAME;
+        private int COLUMN_SIZE;
+        private int BUFFER_LENGTH;
+        private int DECIMAL_DIGITS;
+        private int NUM_PREC_RADIX;
+        private int NULLABLE;
+        private String REMARKS;
+        private String COLUMN_DEF;
+        private int SQL_DATA_TYPE;
+        private int SQL_DATETIME_SUB;
+        private int CHAR_OCTET_LENGTH;
+        private int ORDINAL_POSITION;
+        private String IS_NULLABLE;
+        private String SCOPE_CATLOG;
+        private String SCOPE_SCHEMA;
+        private String SCOPE_TABLE;
+        private short SOURCE_DATA_TYPE;
+        private String IS_AUTOINCREMENT;
+
+        public String getTABLE_CAT() {
+            return TABLE_CAT;
+        }
+
+        public void setTABLE_CAT(String tABLE_CAT) {
+            TABLE_CAT = tABLE_CAT;
+        }
+
+        public String getTABLE_SCHEM() {
+            return TABLE_SCHEM;
+        }
+
+        public void setTABLE_SCHEM(String tABLE_SCHEM) {
+            TABLE_SCHEM = tABLE_SCHEM;
+        }
+
+        public String getTABLE_NAME() {
+            return TABLE_NAME;
+        }
+
+        public void setTABLE_NAME(String tABLE_NAME) {
+            TABLE_NAME = tABLE_NAME;
+        }
+
+        public String getCOLUMN_NAME() {
+            return COLUMN_NAME;
+        }
+
+        public void setCOLUMN_NAME(String cOLUMN_NAME) {
+            COLUMN_NAME = cOLUMN_NAME;
+        }
+
+        public int getDATA_TYPE() {
+            return DATA_TYPE;
+        }
+
+        public void setDATA_TYPE(int dATA_TYPE) {
+            DATA_TYPE = dATA_TYPE;
+        }
+
+        public String getTYPE_NAME() {
+            return TYPE_NAME;
+        }
+
+        public void setTYPE_NAME(String tYPE_NAME) {
+            TYPE_NAME = tYPE_NAME;
+        }
+
+        public int getCOLUMN_SIZE() {
+            return COLUMN_SIZE;
+        }
+
+        public void setCOLUMN_SIZE(int cOLUMN_SIZE) {
+            COLUMN_SIZE = cOLUMN_SIZE;
+        }
+
+        public int getBUFFER_LENGTH() {
+            return BUFFER_LENGTH;
+        }
+
+        public void setBUFFER_LENGTH(int bUFFER_LENGTH) {
+            BUFFER_LENGTH = bUFFER_LENGTH;
+        }
+
+        public int getDECIMAL_DIGITS() {
+            return DECIMAL_DIGITS;
+        }
+
+        public void setDECIMAL_DIGITS(int dECIMAL_DIGITS) {
+            DECIMAL_DIGITS = dECIMAL_DIGITS;
+        }
+
+        public int getNUM_PREC_RADIX() {
+            return NUM_PREC_RADIX;
+        }
+
+        public void setNUM_PREC_RADIX(int nUM_PREC_RADIX) {
+            NUM_PREC_RADIX = nUM_PREC_RADIX;
+        }
+
+        public int getNULLABLE() {
+            return NULLABLE;
+        }
+
+        public void setNULLABLE(int nULLABLE) {
+            NULLABLE = nULLABLE;
+        }
+
+        public String getREMARKS() {
+            return REMARKS;
+        }
+
+        public void setREMARKS(String rEMARKS) {
+            REMARKS = rEMARKS;
+        }
+
+        public String getCOLUMN_DEF() {
+            return COLUMN_DEF;
+        }
+
+        public void setCOLUMN_DEF(String cOLUMN_DEF) {
+            COLUMN_DEF = cOLUMN_DEF;
+        }
+
+        public int getSQL_DATA_TYPE() {
+            return SQL_DATA_TYPE;
+        }
+
+        public void setSQL_DATA_TYPE(int sQL_DATA_TYPE) {
+            SQL_DATA_TYPE = sQL_DATA_TYPE;
+        }
+
+        public int getSQL_DATETIME_SUB() {
+            return SQL_DATETIME_SUB;
+        }
+
+        public void setSQL_DATETIME_SUB(int sQL_DATETIME_SUB) {
+            SQL_DATETIME_SUB = sQL_DATETIME_SUB;
+        }
+
+        public int getCHAR_OCTET_LENGTH() {
+            return CHAR_OCTET_LENGTH;
+        }
+
+        public void setCHAR_OCTET_LENGTH(int cHAR_OCTET_LENGTH) {
+            CHAR_OCTET_LENGTH = cHAR_OCTET_LENGTH;
+        }
+
+        public int getORDINAL_POSITION() {
+            return ORDINAL_POSITION;
+        }
+
+        public void setORDINAL_POSITION(int oRDINAL_POSITION) {
+            ORDINAL_POSITION = oRDINAL_POSITION;
+        }
+
+        public String getIS_NULLABLE() {
+            return IS_NULLABLE;
+        }
+
+        public void setIS_NULLABLE(String iS_NULLABLE) {
+            IS_NULLABLE = iS_NULLABLE;
+        }
+
+        public String getSCOPE_CATLOG() {
+            return SCOPE_CATLOG;
+        }
+
+        public void setSCOPE_CATLOG(String sCOPE_CATLOG) {
+            SCOPE_CATLOG = sCOPE_CATLOG;
+        }
+
+        public String getSCOPE_SCHEMA() {
+            return SCOPE_SCHEMA;
+        }
+
+        public void setSCOPE_SCHEMA(String sCOPE_SCHEMA) {
+            SCOPE_SCHEMA = sCOPE_SCHEMA;
+        }
+
+        public String getSCOPE_TABLE() {
+            return SCOPE_TABLE;
+        }
+
+        public void setSCOPE_TABLE(String sCOPE_TABLE) {
+            SCOPE_TABLE = sCOPE_TABLE;
+        }
+
+        public short getSOURCE_DATA_TYPE() {
+            return SOURCE_DATA_TYPE;
+        }
+
+        public void setSOURCE_DATA_TYPE(short sOURCE_DATA_TYPE) {
+            SOURCE_DATA_TYPE = sOURCE_DATA_TYPE;
+        }
+
+        public String getIS_AUTOINCREMENT() {
+            return IS_AUTOINCREMENT;
+        }
+
+        public void setIS_AUTOINCREMENT(String iS_AUTOINCREMENT) {
+            this.IS_AUTOINCREMENT = iS_AUTOINCREMENT;
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/jdbc/src/main/java/com/kylinolap/jdbc/util/DefaultSslProtocolSocketFactory.java
----------------------------------------------------------------------
diff --git a/jdbc/src/main/java/com/kylinolap/jdbc/util/DefaultSslProtocolSocketFactory.java b/jdbc/src/main/java/com/kylinolap/jdbc/util/DefaultSslProtocolSocketFactory.java
new file mode 100644
index 0000000..a9fb61b
--- /dev/null
+++ b/jdbc/src/main/java/com/kylinolap/jdbc/util/DefaultSslProtocolSocketFactory.java
@@ -0,0 +1,148 @@
+/*
+ * 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.jdbc.util;
+
+import java.io.IOException;
+import java.net.InetAddress;
+import java.net.Socket;
+import java.net.UnknownHostException;
+
+import javax.net.ssl.SSLContext;
+import javax.net.ssl.TrustManager;
+
+import org.apache.commons.httpclient.ConnectTimeoutException;
+import org.apache.commons.httpclient.HttpClientError;
+import org.apache.commons.httpclient.params.HttpConnectionParams;
+import org.apache.commons.httpclient.protocol.ControllerThreadSocketFactory;
+import org.apache.commons.httpclient.protocol.SecureProtocolSocketFactory;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * @author xduo
+ * 
+ */
+public class DefaultSslProtocolSocketFactory implements SecureProtocolSocketFactory {
+    /** Log object for this class. */
+    private static Logger LOG = LoggerFactory.getLogger(DefaultSslProtocolSocketFactory.class);
+    private SSLContext sslcontext = null;
+
+    /**
+     * Constructor for DefaultSslProtocolSocketFactory.
+     */
+    public DefaultSslProtocolSocketFactory() {
+        super();
+    }
+
+    /**
+     * @see SecureProtocolSocketFactory#createSocket(java.lang.String,int,java.net.InetAddress,int)
+     */
+    public Socket createSocket(String host, int port, InetAddress clientHost, int clientPort) throws IOException, UnknownHostException {
+        return getSSLContext().getSocketFactory().createSocket(host, port, clientHost, clientPort);
+    }
+
+    /**
+     * Attempts to get a new socket connection to the given host within the
+     * given time limit.
+     * 
+     * <p>
+     * To circumvent the limitations of older JREs that do not support connect
+     * timeout a controller thread is executed. The controller thread attempts
+     * to create a new socket within the given limit of time. If socket
+     * constructor does not return until the timeout expires, the controller
+     * terminates and throws an {@link ConnectTimeoutException}
+     * </p>
+     * 
+     * @param host
+     *            the host name/IP
+     * @param port
+     *            the port on the host
+     * @param localAddress
+     *            the local host name/IP to bind the socket to
+     * @param localPort
+     *            the port on the local machine
+     * @param params
+     *            {@link HttpConnectionParams Http connection parameters}
+     * 
+     * @return Socket a new socket
+     * 
+     * @throws IOException
+     *             if an I/O error occurs while creating the socket
+     * @throws UnknownHostException
+     *             if the IP address of the host cannot be determined
+     * @throws ConnectTimeoutException
+     *             DOCUMENT ME!
+     * @throws IllegalArgumentException
+     *             DOCUMENT ME!
+     */
+    public Socket createSocket(final String host, final int port, final InetAddress localAddress, final int localPort, final HttpConnectionParams params) throws IOException, UnknownHostException, ConnectTimeoutException {
+        if (params == null) {
+            throw new IllegalArgumentException("Parameters may not be null");
+        }
+
+        int timeout = params.getConnectionTimeout();
+
+        if (timeout == 0) {
+            return createSocket(host, port, localAddress, localPort);
+        } else {
+            // To be eventually deprecated when migrated to Java 1.4 or above
+            return ControllerThreadSocketFactory.createSocket(this, host, port, localAddress, localPort, timeout);
+        }
+    }
+
+    /**
+     * @see SecureProtocolSocketFactory#createSocket(java.lang.String,int)
+     */
+    public Socket createSocket(String host, int port) throws IOException, UnknownHostException {
+        return getSSLContext().getSocketFactory().createSocket(host, port);
+    }
+
+    /**
+     * @see SecureProtocolSocketFactory#createSocket(java.net.Socket,java.lang.String,int,boolean)
+     */
+    public Socket createSocket(Socket socket, String host, int port, boolean autoClose) throws IOException, UnknownHostException {
+        return getSSLContext().getSocketFactory().createSocket(socket, host, port, autoClose);
+    }
+
+    public boolean equals(Object obj) {
+        return ((obj != null) && obj.getClass().equals(DefaultX509TrustManager.class));
+    }
+
+    public int hashCode() {
+        return DefaultX509TrustManager.class.hashCode();
+    }
+
+    private static SSLContext createEasySSLContext() {
+        try {
+            SSLContext context = SSLContext.getInstance("TLS");
+            context.init(null, new TrustManager[] { new DefaultX509TrustManager(null) }, null);
+
+            return context;
+        } catch (Exception e) {
+            LOG.error(e.getMessage(), e);
+            throw new HttpClientError(e.toString());
+        }
+    }
+
+    private SSLContext getSSLContext() {
+        if (this.sslcontext == null) {
+            this.sslcontext = createEasySSLContext();
+        }
+
+        return this.sslcontext;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/jdbc/src/main/java/com/kylinolap/jdbc/util/DefaultX509TrustManager.java
----------------------------------------------------------------------
diff --git a/jdbc/src/main/java/com/kylinolap/jdbc/util/DefaultX509TrustManager.java b/jdbc/src/main/java/com/kylinolap/jdbc/util/DefaultX509TrustManager.java
new file mode 100644
index 0000000..89b8f23
--- /dev/null
+++ b/jdbc/src/main/java/com/kylinolap/jdbc/util/DefaultX509TrustManager.java
@@ -0,0 +1,112 @@
+/*
+ * 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.jdbc.util;
+
+import java.security.KeyStore;
+import java.security.KeyStoreException;
+import java.security.NoSuchAlgorithmException;
+import java.security.cert.CertificateException;
+import java.security.cert.X509Certificate;
+
+import javax.net.ssl.KeyManagerFactory;
+import javax.net.ssl.TrustManager;
+import javax.net.ssl.TrustManagerFactory;
+import javax.net.ssl.X509TrustManager;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * @author xduo
+ * 
+ */
+public class DefaultX509TrustManager implements X509TrustManager {
+
+    /** Log object for this class. */
+    private static Logger LOG = LoggerFactory.getLogger(DefaultX509TrustManager.class);
+    private X509TrustManager standardTrustManager = null;
+
+    /**
+     * Constructor for DefaultX509TrustManager.
+     * 
+     */
+    public DefaultX509TrustManager(KeyStore keystore) throws NoSuchAlgorithmException, KeyStoreException {
+        super();
+
+        TrustManagerFactory factory = TrustManagerFactory.getInstance(KeyManagerFactory.getDefaultAlgorithm());
+        factory.init(keystore);
+
+        TrustManager[] trustmanagers = factory.getTrustManagers();
+
+        if (trustmanagers.length == 0) {
+            throw new NoSuchAlgorithmException("SunX509 trust manager not supported");
+        }
+
+        this.standardTrustManager = (X509TrustManager) trustmanagers[0];
+    }
+
+    public X509Certificate[] getAcceptedIssuers() {
+        return this.standardTrustManager.getAcceptedIssuers();
+    }
+
+    public boolean isClientTrusted(X509Certificate[] certificates) {
+        return true;
+        // return this.standardTrustManager.isClientTrusted(certificates);
+    }
+
+    public boolean isServerTrusted(X509Certificate[] certificates) {
+        if ((certificates != null) && LOG.isDebugEnabled()) {
+            LOG.debug("Server certificate chain:");
+
+            for (int i = 0; i < certificates.length; i++) {
+                if (LOG.isDebugEnabled()) {
+                    LOG.debug("X509Certificate[" + i + "]=" + certificates[i]);
+                }
+            }
+        }
+
+        if ((certificates != null) && (certificates.length == 1)) {
+            X509Certificate certificate = certificates[0];
+
+            try {
+                certificate.checkValidity();
+            } catch (CertificateException e) {
+                LOG.error(e.toString());
+
+                return false;
+            }
+
+            return true;
+        } else {
+            return true;
+            // return this.standardTrustManager.isServerTrusted(certificates);
+        }
+    }
+
+    @Override
+    public void checkClientTrusted(X509Certificate[] chain, String authType) throws CertificateException {
+        // TODO Auto-generated method stub
+
+    }
+
+    @Override
+    public void checkServerTrusted(X509Certificate[] chain, String authType) throws CertificateException {
+        // TODO Auto-generated method stub
+
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/jdbc/src/main/java/com/kylinolap/jdbc/util/SQLTypeMap.java
----------------------------------------------------------------------
diff --git a/jdbc/src/main/java/com/kylinolap/jdbc/util/SQLTypeMap.java b/jdbc/src/main/java/com/kylinolap/jdbc/util/SQLTypeMap.java
new file mode 100644
index 0000000..12eff56
--- /dev/null
+++ b/jdbc/src/main/java/com/kylinolap/jdbc/util/SQLTypeMap.java
@@ -0,0 +1,179 @@
+/*
+ * 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.jdbc.util;
+
+import java.math.BigDecimal;
+import java.sql.Date;
+import java.sql.Time;
+import java.sql.Timestamp;
+import java.sql.Types;
+import java.util.HashMap;
+import java.util.Map;
+
+import net.hydromatic.avatica.ColumnMetaData;
+import net.hydromatic.avatica.ColumnMetaData.Rep;
+
+import com.kylinolap.jdbc.stub.KylinColumnMetaData;
+
+/**
+ * Util class to handle type gap between sql types and java types.
+ * 
+ * @author xduo
+ * 
+ */
+public class SQLTypeMap {
+
+    public final static Map<String, ColumnMetaData> schemaMetaTypeMapping = new HashMap<String, ColumnMetaData>();
+
+    public final static Map<String, ColumnMetaData> columnMetaTypeMapping = new HashMap<String, ColumnMetaData>();
+
+    public final static Map<String, ColumnMetaData> tableMetaTypeMapping = new HashMap<String, ColumnMetaData>();
+
+    static {
+        schemaMetaTypeMapping.put("TABLE_CAT", KylinColumnMetaData.dummy(0, "TABLE_SCHEM", "TABLE_SCHEM", ColumnMetaData.scalar(Types.VARCHAR, "varchar", Rep.STRING), true));
+        schemaMetaTypeMapping.put("TABLE_SCHEM", KylinColumnMetaData.dummy(1, "TABLE_CATALOG", "TABLE_CATALOG", ColumnMetaData.scalar(Types.VARCHAR, "varchar", Rep.STRING), true));
+
+        tableMetaTypeMapping.put("TABLE_CAT", KylinColumnMetaData.dummy(0, "TABLE_CAT", "TABLE_CAT", ColumnMetaData.scalar(Types.VARCHAR, "varchar", Rep.STRING), true));
+        tableMetaTypeMapping.put("TABLE_SCHEM", KylinColumnMetaData.dummy(1, "TABLE_SCHEM", "TABLE_SCHEM", ColumnMetaData.scalar(Types.VARCHAR, "varchar", Rep.STRING), true));
+        tableMetaTypeMapping.put("TABLE_NAME", KylinColumnMetaData.dummy(2, "TABLE_NAME", "TABLE_NAME", ColumnMetaData.scalar(Types.VARCHAR, "varchar", Rep.STRING), true));
+        tableMetaTypeMapping.put("TABLE_TYPE", KylinColumnMetaData.dummy(3, "TABLE_TYPE", "TABLE_TYPE", ColumnMetaData.scalar(Types.VARCHAR, "varchar", Rep.STRING), true));
+        tableMetaTypeMapping.put("REMARKS", KylinColumnMetaData.dummy(4, "REMARKS", "REMARKS", ColumnMetaData.scalar(Types.VARCHAR, "varchar", Rep.STRING), true));
+        tableMetaTypeMapping.put("TYPE_CAT", KylinColumnMetaData.dummy(5, "TYPE_CAT", "TYPE_CAT", ColumnMetaData.scalar(Types.VARCHAR, "varchar", Rep.STRING), true));
+        tableMetaTypeMapping.put("TYPE_SCHEM", KylinColumnMetaData.dummy(6, "TYPE_SCHEM", "TYPE_SCHEM", ColumnMetaData.scalar(Types.VARCHAR, "varchar", Rep.STRING), true));
+        tableMetaTypeMapping.put("TYPE_NAME", KylinColumnMetaData.dummy(7, "TYPE_NAME", "TYPE_NAME", ColumnMetaData.scalar(Types.VARCHAR, "varchar", Rep.STRING), true));
+        tableMetaTypeMapping.put("SELF_REFERENCING_COL_NAME", KylinColumnMetaData.dummy(8, "SELF_REFERENCING_COL_NAME", "SELF_REFERENCING_COL_NAME", ColumnMetaData.scalar(Types.VARCHAR, "varchar", Rep.STRING), true));
+        tableMetaTypeMapping.put("REF_GENERATION", KylinColumnMetaData.dummy(9, "REF_GENERATION", "REF_GENERATION", ColumnMetaData.scalar(Types.VARCHAR, "varchar", Rep.STRING), true));
+
+        columnMetaTypeMapping.put("TABLE_CAT", KylinColumnMetaData.dummy(0, "TABLE_CAT", "TABLE_CAT", ColumnMetaData.scalar(Types.VARCHAR, "varchar", Rep.STRING), true));
+        columnMetaTypeMapping.put("TABLE_SCHEM", KylinColumnMetaData.dummy(1, "TABLE_SCHEM", "TABLE_SCHEM", ColumnMetaData.scalar(Types.VARCHAR, "varchar", Rep.STRING), true));
+        columnMetaTypeMapping.put("TABLE_NAME", KylinColumnMetaData.dummy(2, "TABLE_NAME", "TABLE_NAME", ColumnMetaData.scalar(Types.VARCHAR, "varchar", Rep.STRING), true));
+        columnMetaTypeMapping.put("COLUMN_NAME", KylinColumnMetaData.dummy(3, "COLUMN_NAME", "COLUMN_NAME", ColumnMetaData.scalar(Types.VARCHAR, "varchar", Rep.STRING), true));
+        columnMetaTypeMapping.put("DATA_TYPE", KylinColumnMetaData.dummy(4, "DATA_TYPE", "DATA_TYPE", ColumnMetaData.scalar(Types.INTEGER, "integer", Rep.INTEGER), true));
+        columnMetaTypeMapping.put("TYPE_NAME", KylinColumnMetaData.dummy(5, "TYPE_NAME", "TYPE_NAME", ColumnMetaData.scalar(Types.VARCHAR, "varchar", Rep.STRING), true));
+        columnMetaTypeMapping.put("COLUMN_SIZE", KylinColumnMetaData.dummy(6, "COLUMN_SIZE", "COLUMN_SIZE", ColumnMetaData.scalar(Types.INTEGER, "integer", Rep.INTEGER), true));
+        columnMetaTypeMapping.put("BUFFER_LENGTH", KylinColumnMetaData.dummy(7, "BUFFER_LENGTH", "BUFFER_LENGTH", ColumnMetaData.scalar(Types.INTEGER, "integer", Rep.INTEGER), true));
+        columnMetaTypeMapping.put("DECIMAL_DIGITS", KylinColumnMetaData.dummy(8, "DECIMAL_DIGITS", "DECIMAL_DIGITS", ColumnMetaData.scalar(Types.INTEGER, "integer", Rep.INTEGER), true));
+        columnMetaTypeMapping.put("NUM_PREC_RADIX", KylinColumnMetaData.dummy(9, "NUM_PREC_RADIX", "NUM_PREC_RADIX", ColumnMetaData.scalar(Types.INTEGER, "integer", Rep.INTEGER), true));
+        columnMetaTypeMapping.put("NULLABLE", KylinColumnMetaData.dummy(10, "NULLABLE", "NULLABLE", ColumnMetaData.scalar(Types.INTEGER, "integer", Rep.INTEGER), true));
+        columnMetaTypeMapping.put("REMARKS", KylinColumnMetaData.dummy(11, "REMARKS", "REMARKS", ColumnMetaData.scalar(Types.VARCHAR, "varchar", Rep.STRING), true));
+        columnMetaTypeMapping.put("COLUMN_DEF", KylinColumnMetaData.dummy(12, "COLUMN_DEF", "COLUMN_DEF", ColumnMetaData.scalar(Types.VARCHAR, "varchar", Rep.STRING), true));
+        columnMetaTypeMapping.put("SQL_DATA_TYPE", KylinColumnMetaData.dummy(13, "SQL_DATA_TYPE", "SQL_DATA_TYPE", ColumnMetaData.scalar(Types.INTEGER, "integer", Rep.INTEGER), true));
+        columnMetaTypeMapping.put("SQL_DATETIME_SUB", KylinColumnMetaData.dummy(14, "SQL_DATETIME_SUB", "SQL_DATETIME_SUB", ColumnMetaData.scalar(Types.INTEGER, "integer", Rep.INTEGER), true));
+        columnMetaTypeMapping.put("CHAR_OCTET_LENGTH", KylinColumnMetaData.dummy(15, "CHAR_OCTET_LENGTH", "CHAR_OCTET_LENGTH", ColumnMetaData.scalar(Types.INTEGER, "integer", Rep.INTEGER), true));
+        columnMetaTypeMapping.put("ORDINAL_POSITION", KylinColumnMetaData.dummy(16, "ORDINAL_POSITION", "ORDINAL_POSITION", ColumnMetaData.scalar(Types.INTEGER, "integer", Rep.INTEGER), true));
+        columnMetaTypeMapping.put("IS_NULLABLE", KylinColumnMetaData.dummy(17, "IS_NULLABLE", "IS_NULLABLE", ColumnMetaData.scalar(Types.VARCHAR, "varchar", Rep.STRING), true));
+        columnMetaTypeMapping.put("SCOPE_CATALOG", KylinColumnMetaData.dummy(18, "SCOPE_CATALOG", "SCOPE_CATALOG", ColumnMetaData.scalar(Types.VARCHAR, "varchar", Rep.STRING), true));
+        columnMetaTypeMapping.put("SCOPE_TABLE", KylinColumnMetaData.dummy(19, "SCOPE_TABLE", "SCOPE_TABLE", ColumnMetaData.scalar(Types.VARCHAR, "varchar", Rep.STRING), true));
+        columnMetaTypeMapping.put("SOURCE_DATA_TYPE", KylinColumnMetaData.dummy(20, "SOURCE_DATA_TYPE", "SOURCE_DATA_TYPE", ColumnMetaData.scalar(Types.INTEGER, "integer", Rep.INTEGER), true));
+        columnMetaTypeMapping.put("IS_AUTOINCREMENT", KylinColumnMetaData.dummy(21, "IS_AUTOINCREMENT", "IS_AUTOINCREMENT", ColumnMetaData.scalar(Types.VARCHAR, "varchar", Rep.STRING), true));
+        columnMetaTypeMapping.put("IS_GENERATEDCOLUMN", KylinColumnMetaData.dummy(22, "IS_GENERATEDCOLUMN", "IS_GENERATEDCOLUMN", ColumnMetaData.scalar(Types.VARCHAR, "varchar", Rep.STRING), true));
+    }
+
+    @SuppressWarnings("rawtypes")
+    public static Class convert(int sqlType) {
+        Class result = Object.class;
+
+        switch (sqlType) {
+        case Types.CHAR:
+        case Types.VARCHAR:
+        case Types.LONGVARCHAR:
+            result = String.class;
+            break;
+        case Types.NUMERIC:
+        case Types.DECIMAL:
+            result = BigDecimal.class;
+            break;
+        case Types.BIT:
+            result = Boolean.class;
+            break;
+        case Types.TINYINT:
+            result = Byte.class;
+            break;
+        case Types.SMALLINT:
+            result = Short.class;
+            break;
+        case Types.INTEGER:
+            result = Integer.class;
+            break;
+        case Types.BIGINT:
+            result = Long.class;
+            break;
+        case Types.REAL:
+        case Types.FLOAT:
+        case Types.DOUBLE:
+            result = Double.class;
+            break;
+        case Types.BINARY:
+        case Types.VARBINARY:
+        case Types.LONGVARBINARY:
+            result = Byte[].class;
+            break;
+        case Types.DATE:
+            result = Date.class;
+            break;
+        case Types.TIME:
+            result = Time.class;
+            break;
+        case Types.TIMESTAMP:
+            result = Timestamp.class;
+            break;
+        }
+
+        return result;
+    }
+
+    public static Object wrapObject(String value, int sqlType) {
+        if (null == value) {
+            return null;
+        }
+
+        switch (sqlType) {
+        case Types.CHAR:
+        case Types.VARCHAR:
+        case Types.LONGVARCHAR:
+            return value;
+        case Types.NUMERIC:
+        case Types.DECIMAL:
+            return new BigDecimal(value);
+        case Types.BIT:
+            return Boolean.parseBoolean(value);
+        case Types.TINYINT:
+            return Byte.valueOf(value);
+        case Types.SMALLINT:
+            return Short.valueOf(value);
+        case Types.INTEGER:
+            return Integer.parseInt(value);
+        case Types.BIGINT:
+        case Types.REAL:
+        case Types.FLOAT:
+        case Types.DOUBLE:
+            return Long.parseLong(value);
+        case Types.BINARY:
+        case Types.VARBINARY:
+        case Types.LONGVARBINARY:
+            return value.getBytes();
+        case Types.DATE:
+            return Date.valueOf(value);
+        case Types.TIME:
+            return Time.valueOf(value);
+        case Types.TIMESTAMP:
+            return Timestamp.valueOf(value);
+        }
+
+        return value;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/jdbc/src/main/resources/com-kylinolap-kylin-jdbc.properties
----------------------------------------------------------------------
diff --git a/jdbc/src/main/resources/com-kylinolap-kylin-jdbc.properties b/jdbc/src/main/resources/com-kylinolap-kylin-jdbc.properties
new file mode 100644
index 0000000..c16246b
--- /dev/null
+++ b/jdbc/src/main/resources/com-kylinolap-kylin-jdbc.properties
@@ -0,0 +1,10 @@
+driver.name=Kylin JDBC Driver
+driver.version=0.1
+product.name=Kylin
+product.version=0.1
+jdbc.compliant=true
+driver.version.major=0
+driver.version.minor=8
+database.version.major=0
+database.version.minor=8
+build.timestamp=20140918-2017
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/jdbc/src/main/resources/log4j.properties
----------------------------------------------------------------------
diff --git a/jdbc/src/main/resources/log4j.properties b/jdbc/src/main/resources/log4j.properties
new file mode 100644
index 0000000..8910f8b
--- /dev/null
+++ b/jdbc/src/main/resources/log4j.properties
@@ -0,0 +1,14 @@
+#define appenders
+log4j.appender.stdout=org.apache.log4j.ConsoleAppender
+log4j.appender.stdout.layout=org.apache.log4j.PatternLayout
+log4j.appender.stdout.layout.ConversionPattern=[%t]:[%d{yyyy-MM-dd HH:mm:ss,SSS}][%p][%l] - %m%n
+
+log4j.appender.file=org.apache.log4j.DailyRollingFileAppender
+log4j.appender.file.layout=org.apache.log4j.PatternLayout
+log4j.appender.file.File=kylin_jdbc.log
+log4j.appender.file.layout.ConversionPattern=[%t]:[%d{yyyy-MM-dd HH:mm:ss,SSS}][%p][%l] - %m%n
+log4j.appender.file.Append=true
+
+#overall config
+log4j.rootLogger=DEBUG,stdout,file
+#log4j.logger.com.kylinolap=DEBUG,stdout,file
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/jdbc/src/test/java/com/kylinolap/jdbc/DriverTest.java
----------------------------------------------------------------------
diff --git a/jdbc/src/test/java/com/kylinolap/jdbc/DriverTest.java b/jdbc/src/test/java/com/kylinolap/jdbc/DriverTest.java
new file mode 100644
index 0000000..dbc4551
--- /dev/null
+++ b/jdbc/src/test/java/com/kylinolap/jdbc/DriverTest.java
@@ -0,0 +1,158 @@
+package com.kylinolap.jdbc;
+
+import static org.junit.Assert.*;
+
+import java.sql.Connection;
+import java.sql.PreparedStatement;
+import java.sql.ResultSet;
+import java.sql.ResultSetMetaData;
+import java.sql.SQLException;
+import java.sql.Statement;
+import java.util.Properties;
+
+import org.junit.Ignore;
+import org.junit.Test;
+
+import com.kylinolap.jdbc.Driver;
+
+/**
+ * Unit test for Driver.
+ * 
+ * @author xduo
+ * 
+ */
+public class DriverTest {
+
+    @Test
+    public void testStatementWithMockData() throws SQLException {
+        Driver driver = new DummyDriver();
+
+        Connection conn = driver.connect("jdbc:kylin://test_url/test_db", null);
+
+        ResultSet tables = conn.getMetaData().getTables(null, null, null, null);
+        while (tables.next()) {
+            for (int i = 0; i < 10; i++) {
+                assertEquals("dummy", tables.getString(i + 1));
+            }
+        }
+
+        Statement state = conn.createStatement();
+        ResultSet resultSet = state.executeQuery("select * from test_table");
+
+        ResultSetMetaData metadata = resultSet.getMetaData();
+        assertEquals(12, metadata.getColumnType(1));
+        assertEquals("varchar", metadata.getColumnTypeName(1));
+        assertEquals(1, metadata.isNullable(1));
+
+        while (resultSet.next()) {
+            assertEquals("foo", resultSet.getString(1));
+            assertEquals("bar", resultSet.getString(2));
+            assertEquals("tool", resultSet.getString(3));
+        }
+    }
+
+    @Test
+    public void testPreStatementWithMockData() throws SQLException {
+        Driver driver = new DummyDriver();
+
+        Connection conn = driver.connect("jdbc:kylin://test_url/test_db", null);
+        PreparedStatement state = conn.prepareStatement("select * from test_table where id=?");
+        state.setInt(1, 10);
+        ResultSet resultSet = state.executeQuery();
+
+        ResultSetMetaData metadata = resultSet.getMetaData();
+        assertEquals(12, metadata.getColumnType(1));
+        assertEquals("varchar", metadata.getColumnTypeName(1));
+        assertEquals(1, metadata.isNullable(1));
+
+        while (resultSet.next()) {
+            assertEquals("foo", resultSet.getString(1));
+            assertEquals("bar", resultSet.getString(2));
+            assertEquals("tool", resultSet.getString(3));
+        }
+    }
+
+    @Ignore
+    @Test
+    public void testWithCubeData() throws SQLException, InstantiationException, IllegalAccessException, ClassNotFoundException {
+        Driver driver = (Driver) Class.forName("com.kylinolap.kylin.jdbc.Driver").newInstance();
+        Properties info = new Properties();
+        info.put("user", "");
+        info.put("password", "");
+        Connection conn = driver.connect("jdbc:kylin://localhost/default", info);
+
+        ResultSet catalogs = conn.getMetaData().getCatalogs();
+        while (catalogs.next()) {
+            System.out.println(catalogs.getString("TABLE_CAT"));
+        }
+
+        ResultSet schemas = conn.getMetaData().getSchemas();
+        while (schemas.next()) {
+            System.out.println(schemas.getString(1));
+            System.out.println(schemas.getString(2));
+        }
+
+        ResultSet tables = conn.getMetaData().getTables(null, null, null, null);
+        while (tables.next()) {
+            String tableName = tables.getString(3);
+            assertEquals(tables.getString("TABLE_NAME"), tableName);
+            ResultSet columns = conn.getMetaData().getColumns(null, null, tableName, null);
+
+            while (columns.next()) {
+                System.out.println(columns.getString("COLUMN_NAME"));
+                String column = "";
+                for (int i = 0; i < 23; i++) {
+                    column += columns.getString(i + 1) + ", ";
+                }
+
+                System.out.println("Column in table " + tableName + ": " + column);
+            }
+        }
+
+        for (int j = 0; j < 3; j++) {
+            Statement state = conn.createStatement();
+            ResultSet resultSet = state.executeQuery("select * from test_kylin_fact");
+
+            ResultSetMetaData metadata = resultSet.getMetaData();
+            System.out.println("Metadata:");
+
+            for (int i = 0; i < metadata.getColumnCount(); i++) {
+                String metaStr = metadata.getCatalogName(i + 1) + " " + metadata.getColumnClassName(i + 1) + " " + metadata.getColumnDisplaySize(i + 1) + " " + metadata.getColumnLabel(i + 1) + " " + metadata.getColumnName(i + 1) + " " + metadata.getColumnType(i + 1) + " " + metadata.getColumnTypeName(i + 1) + " " + metadata.getPrecision(i + 1) + " " + metadata.getScale(i + 1) + " " + metadata.getSchemaName(i + 1) + " " + metadata.getTableName(i + 1);
+                System.out.println(metaStr);
+            }
+
+            System.out.println("Data:");
+            while (resultSet.next()) {
+                String dataStr = resultSet.getFloat(1) + " " + resultSet.getInt(2) + " " + resultSet.getInt(3) + " " + resultSet.getLong(4) + " " + resultSet.getDate(5) + " " + resultSet.getString(6);
+                System.out.println(dataStr);
+            }
+        }
+    }
+
+    @Ignore
+    @Test
+    public void testPreStatementWithCubeData() throws SQLException {
+        Driver driver = new Driver();
+        Properties info = new Properties();
+        info.put("user", "");
+        info.put("password", "");
+        Connection conn = driver.connect("jdbc:kylin://localhost/default", info);
+        PreparedStatement state = conn.prepareStatement("select * from test_kylin_fact where seller_id=?");
+        state.setLong(1, 10000001);
+        ResultSet resultSet = state.executeQuery();
+
+        ResultSetMetaData metadata = resultSet.getMetaData();
+        System.out.println("Metadata:");
+
+        for (int i = 0; i < metadata.getColumnCount(); i++) {
+            String metaStr = metadata.getCatalogName(i + 1) + " " + metadata.getColumnClassName(i + 1) + " " + metadata.getColumnDisplaySize(i + 1) + " " + metadata.getColumnLabel(i + 1) + " " + metadata.getColumnName(i + 1) + " " + metadata.getColumnType(i + 1) + " " + metadata.getColumnTypeName(i + 1) + " " + metadata.getPrecision(i + 1) + " " + metadata.getScale(i + 1) + " " + metadata.getSchemaName(i + 1) + " " + metadata.getTableName(i + 1);
+            System.out.println(metaStr);
+        }
+
+        System.out.println("Data:");
+        while (resultSet.next()) {
+            String dataStr = resultSet.getFloat(1) + " " + resultSet.getInt(2) + " " + resultSet.getInt(3) + " " + resultSet.getLong(4) + " " + resultSet.getDate(5) + " " + resultSet.getString(6);
+            System.out.println(dataStr);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/jdbc/src/test/java/com/kylinolap/jdbc/DummyClient.java
----------------------------------------------------------------------
diff --git a/jdbc/src/test/java/com/kylinolap/jdbc/DummyClient.java b/jdbc/src/test/java/com/kylinolap/jdbc/DummyClient.java
new file mode 100644
index 0000000..5673eaa
--- /dev/null
+++ b/jdbc/src/test/java/com/kylinolap/jdbc/DummyClient.java
@@ -0,0 +1,69 @@
+package com.kylinolap.jdbc;
+
+import java.sql.Types;
+import java.util.ArrayList;
+import java.util.List;
+
+import net.hydromatic.avatica.AvaticaStatement;
+import net.hydromatic.avatica.ColumnMetaData;
+import net.hydromatic.avatica.ColumnMetaData.Rep;
+import net.hydromatic.linq4j.Enumerator;
+
+import com.kylinolap.jdbc.KylinConnectionImpl;
+import com.kylinolap.jdbc.KylinEnumerator;
+import com.kylinolap.jdbc.KylinMetaImpl.MetaCatalog;
+import com.kylinolap.jdbc.KylinMetaImpl.MetaColumn;
+import com.kylinolap.jdbc.KylinMetaImpl.MetaProject;
+import com.kylinolap.jdbc.KylinMetaImpl.MetaSchema;
+import com.kylinolap.jdbc.KylinMetaImpl.MetaTable;
+import com.kylinolap.jdbc.stub.ConnectionException;
+import com.kylinolap.jdbc.stub.DataSet;
+import com.kylinolap.jdbc.stub.RemoteClient;
+
+/**
+ * @author xduo
+ * 
+ */
+public class DummyClient implements RemoteClient {
+
+    public DummyClient(KylinConnectionImpl conn) {
+    }
+
+    @Override
+    public void connect() throws ConnectionException {
+    }
+
+    @Override
+    public MetaProject getMetadata(String project) throws ConnectionException {
+        List<ColumnMetaData> meta = new ArrayList<ColumnMetaData>();
+        for (int i = 0; i < 10; i++) {
+            meta.add(ColumnMetaData.dummy(ColumnMetaData.scalar(Types.VARCHAR, "varchar", Rep.STRING), true));
+        }
+
+        List<MetaTable> tables = new ArrayList<MetaTable>();
+        MetaTable table = new MetaTable("dummy", "dummy", "dummy", "dummy", "dummy", "dummy", "dummy", "dummy", "dummy", "dummy", new ArrayList<MetaColumn>());
+        tables.add(table);
+
+        List<MetaSchema> schemas = new ArrayList<MetaSchema>();
+        schemas.add(new MetaSchema("dummy", "dummy", tables));
+        List<MetaCatalog> catalogs = new ArrayList<MetaCatalog>();
+        catalogs.add(new MetaCatalog("dummy", schemas));
+
+        return new MetaProject(null, catalogs);
+    }
+
+    @Override
+    public DataSet<Object[]> query(AvaticaStatement statement, String sql) {
+        List<Object[]> data = new ArrayList<Object[]>();
+        Object[] row = new Object[] { "foo", "bar", "tool" };
+        data.add(row);
+        Enumerator<Object[]> enumerator = new KylinEnumerator<Object[]>(data);
+        List<ColumnMetaData> meta = new ArrayList<ColumnMetaData>();
+        meta.add(ColumnMetaData.dummy(ColumnMetaData.scalar(Types.VARCHAR, "varchar", Rep.STRING), true));
+        meta.add(ColumnMetaData.dummy(ColumnMetaData.scalar(Types.VARCHAR, "varchar", Rep.STRING), true));
+        meta.add(ColumnMetaData.dummy(ColumnMetaData.scalar(Types.VARCHAR, "varchar", Rep.STRING), true));
+
+        return new DataSet<Object[]>(meta, enumerator);
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/jdbc/src/test/java/com/kylinolap/jdbc/DummyDriver.java
----------------------------------------------------------------------
diff --git a/jdbc/src/test/java/com/kylinolap/jdbc/DummyDriver.java b/jdbc/src/test/java/com/kylinolap/jdbc/DummyDriver.java
new file mode 100644
index 0000000..75e6e0f
--- /dev/null
+++ b/jdbc/src/test/java/com/kylinolap/jdbc/DummyDriver.java
@@ -0,0 +1,16 @@
+package com.kylinolap.jdbc;
+
+import com.kylinolap.jdbc.Driver;
+
+/**
+ * @author xduo
+ * 
+ */
+public class DummyDriver extends Driver {
+
+    @Override
+    protected String getFactoryClassName(JdbcVersion jdbcVersion) {
+        return "com.kylinolap.jdbc.DummyJdbc41Factory";
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/jdbc/src/test/java/com/kylinolap/jdbc/DummyJdbc41Factory.java
----------------------------------------------------------------------
diff --git a/jdbc/src/test/java/com/kylinolap/jdbc/DummyJdbc41Factory.java b/jdbc/src/test/java/com/kylinolap/jdbc/DummyJdbc41Factory.java
new file mode 100644
index 0000000..e4a4b55
--- /dev/null
+++ b/jdbc/src/test/java/com/kylinolap/jdbc/DummyJdbc41Factory.java
@@ -0,0 +1,19 @@
+package com.kylinolap.jdbc;
+
+import com.kylinolap.jdbc.KylinConnectionImpl;
+import com.kylinolap.jdbc.KylinJdbc41Factory;
+import com.kylinolap.jdbc.stub.RemoteClient;
+
+/**
+ * @author xduo
+ * 
+ */
+public class DummyJdbc41Factory extends KylinJdbc41Factory {
+
+    // ~ kylin sepcified
+    @Override
+    public RemoteClient newRemoteClient(KylinConnectionImpl connection) {
+        return new DummyClient(connection);
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/job/.gitignore
----------------------------------------------------------------------
diff --git a/job/.gitignore b/job/.gitignore
new file mode 100644
index 0000000..0b42d2d
--- /dev/null
+++ b/job/.gitignore
@@ -0,0 +1 @@
+/target

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/job/.settings/org.eclipse.core.resources.prefs
----------------------------------------------------------------------
diff --git a/job/.settings/org.eclipse.core.resources.prefs b/job/.settings/org.eclipse.core.resources.prefs
new file mode 100644
index 0000000..04cfa2c
--- /dev/null
+++ b/job/.settings/org.eclipse.core.resources.prefs
@@ -0,0 +1,6 @@
+eclipse.preferences.version=1
+encoding//src/main/java=UTF-8
+encoding//src/main/resources=UTF-8
+encoding//src/test/java=UTF-8
+encoding//src/test/resources=UTF-8
+encoding/<project>=UTF-8


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

Posted by lu...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/query/src/test/java/com/kylinolap/query/test/CombinationTest.java
----------------------------------------------------------------------
diff --git a/query/src/test/java/com/kylinolap/query/test/CombinationTest.java b/query/src/test/java/com/kylinolap/query/test/CombinationTest.java
new file mode 100644
index 0000000..988ff51
--- /dev/null
+++ b/query/src/test/java/com/kylinolap/query/test/CombinationTest.java
@@ -0,0 +1,55 @@
+package com.kylinolap.query.test;
+
+import java.sql.SQLException;
+import java.util.Arrays;
+import java.util.Collection;
+
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+import com.kylinolap.storage.hbase.observer.CoprocessorEnabler;
+
+/**
+ * Created by honma on 7/2/14.
+ */
+@RunWith(Parameterized.class)
+public class CombinationTest extends KylinQueryTest {
+
+    @BeforeClass
+    public static void setUp() throws SQLException {
+    }
+
+    @AfterClass
+    public static void tearDown() {
+        clean();
+    }
+
+    /**
+     * return all config combinations, where first setting specifies join type
+     * (inner or left), and the second setting specifies whether to force using
+     * coprocessors(on, off or unset).
+     */
+    @Parameterized.Parameters
+    public static Collection<Object[]> configs() {
+        return Arrays.asList(new Object[][] { { "inner", "unset" }, { "left", "unset" }, { "inner", "off" }, { "left", "off" }, { "inner", "on" }, { "left", "on" }, });
+    }
+
+    public CombinationTest(String joinType, String coprocessorToggle) throws Exception {
+
+        KylinQueryTest.clean();
+
+        KylinQueryTest.joinType = joinType;
+        KylinQueryTest.setupAll();
+        KylinQueryTest.preferCubeOf(joinType);
+
+        if (coprocessorToggle.equals("on")) {
+            CoprocessorEnabler.forceCoprocessorOn();
+        } else if (coprocessorToggle.equals("off")) {
+            CoprocessorEnabler.forceCoprocessorOff();
+        } else if (coprocessorToggle.equals("unset")) {
+            // unset
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/query/src/test/java/com/kylinolap/query/test/H2Database.java
----------------------------------------------------------------------
diff --git a/query/src/test/java/com/kylinolap/query/test/H2Database.java b/query/src/test/java/com/kylinolap/query/test/H2Database.java
new file mode 100644
index 0000000..77996e0
--- /dev/null
+++ b/query/src/test/java/com/kylinolap/query/test/H2Database.java
@@ -0,0 +1,134 @@
+/*
+ * 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.query.test;
+
+import java.io.File;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.sql.Connection;
+import java.sql.SQLException;
+import java.sql.Statement;
+import java.util.HashMap;
+import java.util.Map;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.kylinolap.common.KylinConfig;
+import com.kylinolap.metadata.MetadataManager;
+import com.kylinolap.metadata.model.schema.ColumnDesc;
+import com.kylinolap.metadata.model.schema.TableDesc;
+
+public class H2Database {
+    private static final Logger logger = LoggerFactory.getLogger(H2Database.class);
+
+    private static final String[] ALL_TABLES = new String[] { "test_cal_dt", "test_category_groupings", "test_kylin_fact", "test_seller_type_dim", "test_sites" };
+    private static final Map<String, String> javaToH2DataTypeMapping = new HashMap<String, String>();
+
+    static {
+        javaToH2DataTypeMapping.put("short", "smallint");
+        javaToH2DataTypeMapping.put("long", "bigint");
+        javaToH2DataTypeMapping.put("byte", "tinyint");
+        javaToH2DataTypeMapping.put("string", "varchar");
+    }
+
+    private final Connection h2Connection;
+
+    private final KylinConfig config;
+
+    public H2Database(Connection h2Connection, KylinConfig config) {
+        this.h2Connection = h2Connection;
+        this.config = config;
+    }
+
+    public void loadAllTables(String joinType) throws SQLException {
+        for (String tableName : ALL_TABLES) {
+            loadH2Table(tableName, joinType);
+        }
+    }
+
+    private void loadH2Table(String tableName, String joinType) throws SQLException {
+        MetadataManager metaMgr = MetadataManager.getInstance(config);
+        TableDesc tableDesc = metaMgr.getTableDesc(tableName.toUpperCase());
+        File tempFile = null;
+
+        String fileNameSuffix = joinType.equalsIgnoreCase("default") ? "" : "." + joinType;
+
+        try {
+            tempFile = File.createTempFile("tmp_h2", ".csv");
+            FileOutputStream tempFileStream = new FileOutputStream(tempFile);
+            String normalPath = "/data/" + tableDesc.getName() + ".csv";
+
+            // If it's the fact table, there will be a facttable.csv.inner or
+            // facttable.csv.left in hbase
+            // otherwise just use lookup.csv
+            InputStream csvStream = metaMgr.getStore().getResource(normalPath + fileNameSuffix);
+            if (csvStream == null) {
+                csvStream = metaMgr.getStore().getResource(normalPath);
+            } else {
+                logger.info("H2 decides to load " + (normalPath + fileNameSuffix) + " for table " + tableDesc.getName());
+            }
+
+            org.apache.commons.io.IOUtils.copy(csvStream, tempFileStream);
+
+            csvStream.close();
+            tempFileStream.close();
+
+        } catch (IOException e) {
+            e.printStackTrace();
+        }
+
+        String cvsFilePath = tempFile.getPath();
+        Statement stmt = h2Connection.createStatement();
+        String sql = generateCreateH2TableSql(tableDesc, cvsFilePath);
+        stmt.executeUpdate(sql);
+
+        if (tempFile != null)
+            tempFile.delete();
+    }
+
+    private String generateCreateH2TableSql(TableDesc tableDesc, String csvFilePath) {
+        StringBuilder ddl = new StringBuilder();
+        StringBuilder csvColumns = new StringBuilder();
+
+        ddl.append("CREATE TABLE " + tableDesc.getName() + "\n");
+        ddl.append("(" + "\n");
+
+        for (int i = 0; i < tableDesc.getColumns().length; i++) {
+            ColumnDesc col = tableDesc.getColumns()[i];
+            if (i > 0) {
+                ddl.append(",");
+                csvColumns.append(",");
+            }
+            ddl.append(col.getName() + " " + getH2DataType((col.getDatatype())) + "\n");
+            csvColumns.append(col.getName());
+        }
+        ddl.append(")" + "\n");
+        ddl.append("AS SELECT * FROM CSVREAD('" + csvFilePath + "', '" + csvColumns + "', 'charset=UTF-8 fieldSeparator=,');");
+
+        return ddl.toString();
+    }
+
+    private static String getH2DataType(String javaDataType) {
+        String hiveDataType = javaToH2DataTypeMapping.get(javaDataType.toLowerCase());
+        if (hiveDataType == null) {
+            hiveDataType = javaDataType;
+        }
+        return hiveDataType.toLowerCase();
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/query/src/test/java/com/kylinolap/query/test/KylinQueryTest.java
----------------------------------------------------------------------
diff --git a/query/src/test/java/com/kylinolap/query/test/KylinQueryTest.java b/query/src/test/java/com/kylinolap/query/test/KylinQueryTest.java
new file mode 100644
index 0000000..237e593
--- /dev/null
+++ b/query/src/test/java/com/kylinolap/query/test/KylinQueryTest.java
@@ -0,0 +1,259 @@
+/*
+ * 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.query.test;
+
+import static org.junit.Assert.*;
+
+import java.io.File;
+import java.sql.DriverManager;
+import java.sql.SQLException;
+import java.util.List;
+import java.util.Properties;
+
+import org.apache.commons.lang3.StringUtils;
+import org.dbunit.database.DatabaseConnection;
+import org.dbunit.database.IDatabaseConnection;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Ignore;
+import org.junit.Test;
+
+import com.kylinolap.common.KylinConfig;
+import com.kylinolap.common.util.HBaseMetadataTestCase;
+import com.kylinolap.cube.CubeManager;
+import com.kylinolap.cube.project.ProjectInstance;
+import com.kylinolap.query.enumerator.OLAPQuery;
+import com.kylinolap.query.relnode.OLAPContext;
+import com.kylinolap.query.schema.OLAPSchemaFactory;
+import com.kylinolap.storage.hbase.observer.CoprocessorEnabler;
+
+public class KylinQueryTest extends KylinTestBase {
+
+    @BeforeClass
+    public static void setUp() throws Exception {
+        printInfo("setUp in KylinQueryTest");
+
+        joinType = "left";
+        setupAll();
+        preferCubeOf(joinType);
+    }
+
+    protected static void setupAll() throws SQLException {
+        setUpEnv();
+        setUpCubeConn();
+        setUpH2Conn();
+    }
+
+    private static void setUpEnv() {
+        HBaseMetadataTestCase.staticCreateTestMetadata();
+        config = KylinConfig.getInstanceFromEnv();
+    }
+
+    private static void setUpCubeConn() throws SQLException {
+        // Cube Connection
+        File olapTmp = OLAPSchemaFactory.createTempOLAPJson(ProjectInstance.DEFAULT_PROJECT_NAME, config);
+        Properties props = new Properties();
+        props.setProperty(OLAPQuery.PROP_SCAN_THRESHOLD, "10000");
+        cubeConnection = DriverManager.getConnection("jdbc:calcite:model=" + olapTmp.getAbsolutePath(), props);
+    }
+
+    private static void setUpH2Conn() throws SQLException {
+        // H2 Connection
+        h2Connection = DriverManager.getConnection("jdbc:h2:mem:db" + (h2InstanceCount++), "sa", "");
+        // Load H2 Tables (inner join)
+        H2Database h2DB = new H2Database(h2Connection, config);
+        h2DB.loadAllTables(joinType);
+    }
+
+    @AfterClass
+    public static void tearDown() throws Exception {
+        printInfo("tearDown");
+        printInfo("Closing connection...");
+        clean();
+    }
+
+    protected static void clean() {
+        if (cubeConnection != null)
+            closeConnection(cubeConnection);
+        if (h2Connection != null)
+            closeConnection(h2Connection);
+
+        CoprocessorEnabler.forceCoprocessorUnset();
+        HBaseMetadataTestCase.staticCleanupTestMetadata();
+    }
+
+    protected static void preferCubeOf(String joinType) {
+
+        CubeManager cubeManager = CubeManager.getInstance(config);
+
+        boolean cubesBuiltInBatch = cubeManager.getCube("test_kylin_cube_with_slr_empty") != null && cubeManager.getCube("test_kylin_cube_without_slr_empty") != null && cubeManager.getCube("test_kylin_cube_with_slr_left_join_empty") != null && cubeManager.getCube("test_kylin_cube_without_slr_left_join_empty") != null;
+
+        if (!cubesBuiltInBatch) {
+            printInfo("Four empty cubes built in BuildCubeWithEngineTest is not complete, preferCubeOf being ignored");
+            return;
+        }
+
+        if (joinType.equals("inner")) {
+            cubeManager.getCube("test_kylin_cube_with_slr_empty").setCost(20);
+            cubeManager.getCube("test_kylin_cube_without_slr_empty").setCost(10);
+            cubeManager.getCube("test_kylin_cube_with_slr_left_join_empty").setCost(100);
+            cubeManager.getCube("test_kylin_cube_without_slr_left_join_empty").setCost(90);
+        } else if (joinType.equals("left") || joinType.equals("default")) {
+            cubeManager.getCube("test_kylin_cube_with_slr_empty").setCost(100);
+            cubeManager.getCube("test_kylin_cube_without_slr_empty").setCost(90);
+            cubeManager.getCube("test_kylin_cube_with_slr_left_join_empty").setCost(20);
+            cubeManager.getCube("test_kylin_cube_without_slr_left_join_empty").setCost(10);
+        }
+    }
+
+    // for debug purpose
+    @Ignore
+    @Test
+    public void testTempQuery() throws Exception {
+        execAndCompQuery("src/test/resources/query/temp", null, true);
+    }
+
+    @Test
+    public void testSingleRunQuery() throws Exception {
+
+        String queryFileName = "src/test/resources/query/sql/query02.sql";
+
+        File sqlFile = new File(queryFileName);
+        runSQL(sqlFile, true, true);
+        runSQL(sqlFile, true, false);
+    }
+
+    @Test
+    public void testSingleExecuteQuery() throws Exception {
+
+        String queryFileName = "src/test/resources/query/sql/query39.sql";
+
+        File sqlFile = new File(queryFileName);
+        String sql = getTextFromFile(sqlFile);
+        IDatabaseConnection kylinConn = new DatabaseConnection(cubeConnection);
+
+        executeQuery(kylinConn, queryFileName, sql, true);
+    }
+
+    @Ignore
+    @Test
+    public void testTableauProbing() throws Exception {
+        batchExecuteQuery("src/test/resources/query/tableau_probing");
+    }
+
+    @Test
+    public void testCommonQuery() throws Exception {
+        execAndCompQuery("src/test/resources/query/sql", null, true);
+    }
+
+    @Test
+    public void testSimpleQuery() throws Exception {
+        verifyResultRowCount("src/test/resources/query/sql_verifyCount");
+    }
+
+    @Test
+    public void testOrderByQuery() throws Exception {
+        execAndCompQuery("src/test/resources/query/sql_orderby", null, true);
+        // FIXME
+        // as of optiq 0.8, we lost metadata type with "order by" clause, e.g. sql_orderby/query01.sql
+        // thus, temporarily the "order by" clause was cross out, and the needSort is set to true
+        // execAndCompQuery("src/test/resources/query/sql_orderby", null, false);
+    }
+
+    @Test
+    public void testLookupQuery() throws Exception {
+        execAndCompQuery("src/test/resources/query/sql_lookup", null, true);
+    }
+
+    @Test
+    public void testDerivedColumnQuery() throws Exception {
+        execAndCompQuery("src/test/resources/query/sql_derived", null, true);
+    }
+
+    @Test
+    public void testDistinctCountQuery() throws Exception {
+        batchExecuteQuery("src/test/resources/query/sql_distinct");
+    }
+
+    @Test
+    public void testTableauQuery() throws Exception {
+        batchExecuteQuery("src/test/resources/query/sql_tableau");
+    }
+
+    @Test
+    public void testSubQuery() throws Exception {
+        execAndCompQuery("src/test/resources/query/sql_subquery", null, true);
+    }
+
+    @Test
+    public void testCaseWhen() throws Exception {
+        execAndCompQuery("src/test/resources/query/sql_casewhen", null, true);
+    }
+
+    @Ignore
+    @Test
+    public void testHiveQuery() throws Exception {
+        execAndCompQuery("src/test/resources/query/sql_hive", null, true);
+    }
+
+    @Test
+    public void testH2Query() throws Exception {
+        this.execQueryUsingH2("src/test/resources/query/h2", false);
+    }
+
+    @Test
+    public void testInvalidQuery() throws Exception {
+
+        printInfo("-------------------- Test Invalid Query --------------------");
+        String queryFolder = "src/test/resources/query/sql_invalid";
+        List<File> sqlFiles = getFilesFromFolder(new File(queryFolder), ".sql");
+        for (File sqlFile : sqlFiles) {
+            String queryName = StringUtils.split(sqlFile.getName(), '.')[0];
+            printInfo("Testing Query " + queryName);
+            String sql = getTextFromFile(sqlFile);
+            IDatabaseConnection cubeConn = new DatabaseConnection(cubeConnection);
+            try {
+                cubeConn.createQueryTable(queryName, sql);
+            } catch (Throwable t) {
+                continue;
+            } finally {
+                cubeConn.close();
+            }
+            throw new IllegalStateException(queryName + " should be error!");
+        }
+    }
+
+    @Test
+    public void testDynamicQuery() throws Exception {
+        execAndCompDynamicQuery("src/test/resources/query/sql_dynamic", null, true);
+    }
+
+    @Test
+    public void testLimitEnabled() throws Exception {
+        runSqlFile("src/test/resources/query/sql_optimize/enable-limit01.sql");
+        assertLimitWasEnabled();
+    }
+
+    private void assertLimitWasEnabled() {
+        OLAPContext context = getFirstOLAPContext();
+        assertTrue(context.storageContext.isLimitEnabled());
+    }
+
+    private OLAPContext getFirstOLAPContext() {
+        return OLAPContext.getThreadLocalContexts().iterator().next();
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/query/src/test/java/com/kylinolap/query/test/KylinTestBase.java
----------------------------------------------------------------------
diff --git a/query/src/test/java/com/kylinolap/query/test/KylinTestBase.java b/query/src/test/java/com/kylinolap/query/test/KylinTestBase.java
new file mode 100644
index 0000000..bd8eb2e
--- /dev/null
+++ b/query/src/test/java/com/kylinolap/query/test/KylinTestBase.java
@@ -0,0 +1,471 @@
+/*
+ * 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.query.test;
+
+import java.io.BufferedReader;
+import java.io.BufferedWriter;
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.FileReader;
+import java.io.FileWriter;
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.charset.Charset;
+import java.sql.Connection;
+import java.sql.PreparedStatement;
+import java.sql.ResultSet;
+import java.sql.ResultSetMetaData;
+import java.sql.SQLException;
+import java.sql.Statement;
+import java.sql.Timestamp;
+import java.sql.Types;
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Set;
+import java.util.logging.LogManager;
+
+import org.apache.commons.lang3.StringUtils;
+import org.dbunit.Assertion;
+import org.dbunit.database.DatabaseConfig;
+import org.dbunit.database.DatabaseConnection;
+import org.dbunit.database.IDatabaseConnection;
+import org.dbunit.dataset.DataSetException;
+import org.dbunit.dataset.ITable;
+import org.dbunit.dataset.SortedTable;
+import org.dbunit.dataset.datatype.DataType;
+import org.dbunit.dataset.datatype.DataTypeException;
+import org.dbunit.ext.h2.H2Connection;
+import org.dbunit.ext.h2.H2DataTypeFactory;
+import org.junit.Assert;
+
+import com.google.common.io.Files;
+import com.kylinolap.common.KylinConfig;
+
+/**
+ * Created by hongbin on 2/18/14.
+ */
+public class KylinTestBase {
+
+    // Hack for the different constant integer type between optiq (INTEGER) and
+    // h2 (BIGINT)
+    public static class TestH2DataTypeFactory extends H2DataTypeFactory {
+        @Override
+        public DataType createDataType(int sqlType, String sqlTypeName, String tableName, String columnName) throws DataTypeException {
+
+            if ((columnName.startsWith("COL") || columnName.startsWith("col")) && sqlType == Types.BIGINT) {
+                return DataType.INTEGER;
+            }
+            return super.createDataType(sqlType, sqlTypeName);
+        }
+    }
+
+    protected static final String resultTableName = "query result of ";
+    protected static KylinConfig config = null;
+    protected static Connection cubeConnection = null;
+    protected static Connection h2Connection = null;
+    protected static String joinType = "default";
+    protected static int h2InstanceCount = 0;
+
+    protected static int compQueryCount = 0;
+    protected static ArrayList<String> zeroResultQueries = new ArrayList<String>();
+
+    protected static void closeConnection(Connection connection) {
+        if (connection != null) {
+            try {
+                connection.close();
+            } catch (SQLException e) {
+                e.printStackTrace();
+            }
+        }
+    }
+
+    /**
+     * @param folder
+     * @param fileType
+     *            specify the interested file type by file extension
+     * @return
+     */
+    protected static List<File> getFilesFromFolder(final File folder, final String fileType) {
+        List<File> files = new ArrayList<File>();
+        for (final File fileEntry : folder.listFiles()) {
+            if (fileEntry.getName().toLowerCase().endsWith(fileType.toLowerCase())) {
+                files.add(fileEntry);
+            }
+        }
+        return files;
+    }
+
+    protected static void getFilesFromFolderR(final String directoryStr, List<File> files, final String fileType) {
+        File folder = new File(directoryStr);
+        for (final File fileEntry : folder.listFiles()) {
+            if (fileEntry.isDirectory()) {
+                getFilesFromFolderR(fileEntry.getAbsolutePath(), files, fileType);
+            } else if (fileEntry.isFile()) {
+                if (fileEntry.getName().toLowerCase().endsWith(fileType.toLowerCase())) {
+                    files.add(fileEntry);
+                }
+            }
+        }
+    }
+
+    protected static void putTextTofile(File file, String sql) throws IOException {
+        BufferedWriter writer = new BufferedWriter(new FileWriter(file));
+        writer.write(sql, 0, sql.length());
+        writer.close();
+    }
+
+    protected static String getTextFromFile(File file) throws IOException {
+        BufferedReader reader = new BufferedReader(new FileReader(file));
+        String line = null;
+        StringBuilder stringBuilder = new StringBuilder();
+        String ls = System.getProperty("line.separator");
+        while ((line = reader.readLine()) != null) {
+            stringBuilder.append(line);
+            stringBuilder.append(ls);
+        }
+        reader.close();
+        return stringBuilder.toString();
+    }
+
+    protected static List<String> getParameterFromFile(File sqlFile) throws IOException {
+        String sqlFileName = sqlFile.getAbsolutePath();
+        int prefixIndex = sqlFileName.lastIndexOf(".sql");
+        String dataFielName = sqlFileName.substring(0, prefixIndex) + ".dat";
+        File dataFile = new File(dataFielName);
+        List<String> parameters = Files.readLines(dataFile, Charset.defaultCharset());
+        return parameters;
+    }
+
+    protected static void printInfo(String info) {
+        System.out.println(new Timestamp(System.currentTimeMillis()) + " - " + info);
+    }
+
+    protected static void printResult(ITable resultTable) throws DataSetException {
+        StringBuilder sb = new StringBuilder();
+
+        int columnCount = resultTable.getTableMetaData().getColumns().length;
+        String[] columns = new String[columnCount];
+
+        for (int i = 0; i < columnCount; i++) {
+            sb.append(resultTable.getTableMetaData().getColumns()[i].getColumnName());
+            sb.append("-");
+            sb.append(resultTable.getTableMetaData().getColumns()[i].getDataType());
+            sb.append("\t");
+            columns[i] = resultTable.getTableMetaData().getColumns()[i].getColumnName();
+        }
+        sb.append("\n");
+
+        for (int i = 0; i < resultTable.getRowCount(); i++) {
+            for (int j = 0; j < columns.length; j++) {
+                sb.append(resultTable.getValue(i, columns[j]));
+                sb.append("\t");
+            }
+            sb.append("\n");
+        }
+        System.out.println(sb.toString());
+    }
+
+    protected Set<String> buildExclusiveSet(String[] exclusiveQuerys) {
+        Set<String> exclusiveSet = new HashSet<String>();
+        if (exclusiveQuerys != null) {
+            for (String query : exclusiveQuerys) {
+                exclusiveSet.add(query);
+            }
+        }
+        return exclusiveSet;
+    }
+
+    // ////////////////////////////////////////////////////////////////////////////////////////
+    // execute
+
+    protected ITable executeQuery(IDatabaseConnection dbConn, String queryName, String sql, boolean needSort) throws Exception {
+
+        // change join type to match current setting
+        sql = changeJoinType(sql, joinType);
+
+        ITable queryTable = dbConn.createQueryTable(resultTableName + queryName, sql);
+        String[] columnNames = new String[queryTable.getTableMetaData().getColumns().length];
+        for (int i = 0; i < columnNames.length; i++) {
+            columnNames[i] = queryTable.getTableMetaData().getColumns()[i].getColumnName();
+        }
+        if (needSort) {
+            queryTable = new SortedTable(queryTable, columnNames);
+        }
+        printResult(queryTable);
+
+        return queryTable;
+    }
+
+    protected int executeQuery(String sql, boolean needDisplay) throws SQLException {
+
+        // change join type to match current setting
+        sql = changeJoinType(sql, joinType);
+
+        Statement statement = null;
+        ResultSet resultSet = null;
+        try {
+            printInfo("start running...");
+            statement = cubeConnection.createStatement();
+            resultSet = statement.executeQuery(sql);
+            printInfo("stop running...");
+
+            return output(resultSet, needDisplay);
+        } finally {
+            if (resultSet != null) {
+                try {
+                    resultSet.close();
+                } catch (SQLException e) {
+                    // ignore
+                }
+            }
+            if (statement != null) {
+                try {
+                    statement.close();
+                } catch (SQLException e) {
+                    // ignore
+                }
+            }
+        }
+
+    }
+
+    protected ITable executeDynamicQuery(IDatabaseConnection dbConn, String queryName, String sql, List<String> parameters, boolean needSort) throws Exception {
+
+        // change join type to match current setting
+        sql = changeJoinType(sql, joinType);
+
+        PreparedStatement prepStat = dbConn.getConnection().prepareStatement(sql);
+        for (int j = 1; j <= parameters.size(); ++j) {
+            prepStat.setString(j, parameters.get(j - 1).trim());
+        }
+
+        ITable queryTable = dbConn.createTable(resultTableName + queryName, prepStat);
+        String[] columnNames = new String[queryTable.getTableMetaData().getColumns().length];
+        for (int i = 0; i < columnNames.length; i++) {
+            columnNames[i] = queryTable.getTableMetaData().getColumns()[i].getColumnName();
+        }
+        if (needSort) {
+            queryTable = new SortedTable(queryTable, columnNames);
+        }
+        printResult(queryTable);
+        return queryTable;
+    }
+
+    // end of execute
+    // ////////////////////////////////////////////////////////////////////////////////////////
+
+    protected static String changeJoinType(String sql, String targetType) {
+
+        if (targetType.equalsIgnoreCase("default"))
+            return sql;
+
+        String specialStr = "changeJoinType_DELIMITERS";
+        sql = sql.replaceAll(System.getProperty("line.separator"), " " + specialStr + " ");
+
+        String[] tokens = StringUtils.split(sql, null);// split white spaces
+        for (int i = 0; i < tokens.length - 1; ++i) {
+            if ((tokens[i].equalsIgnoreCase("inner") || tokens[i].equalsIgnoreCase("left")) && tokens[i + 1].equalsIgnoreCase("join")) {
+                tokens[i] = targetType.toLowerCase();
+            }
+        }
+
+        String ret = StringUtils.join(tokens, " ");
+        ret = ret.replaceAll(specialStr, System.getProperty("line.separator"));
+        System.out.println("The actual sql executed is: " + ret);
+
+        return ret;
+    }
+
+    protected static void batchChangeJoinType(String targetType) throws IOException {
+        List<File> files = new LinkedList<File>();
+        getFilesFromFolderR("src/test/resources/query", files, ".sql");
+        for (File file : files) {
+            String x = changeJoinType(getTextFromFile(file), targetType);
+            putTextTofile(file, x);
+        }
+    }
+
+    protected void execQueryUsingH2(String queryFolder, boolean needSort) throws Exception {
+        printInfo("---------- Running H2 queries: " + queryFolder);
+
+        List<File> sqlFiles = getFilesFromFolder(new File(queryFolder), ".sql");
+        for (File sqlFile : sqlFiles) {
+            String queryName = StringUtils.split(sqlFile.getName(), '.')[0];
+            String sql = getTextFromFile(sqlFile);
+
+            // execute H2
+            printInfo("Query Result from H2 - " + queryName);
+            H2Connection h2Conn = new H2Connection(h2Connection, null);
+            h2Conn.getConfig().setProperty(DatabaseConfig.PROPERTY_DATATYPE_FACTORY, new TestH2DataTypeFactory());
+            executeQuery(h2Conn, queryName, sql, needSort);
+        }
+    }
+
+    protected void verifyResultRowCount(String queryFolder) throws Exception {
+        printInfo("---------- verify result count in folder: " + queryFolder);
+
+        List<File> sqlFiles = getFilesFromFolder(new File(queryFolder), ".sql");
+        for (File sqlFile : sqlFiles) {
+            String queryName = StringUtils.split(sqlFile.getName(), '.')[0];
+            String sql = getTextFromFile(sqlFile);
+
+            File expectResultFile = new File(sqlFile.getParent(), sqlFile.getName() + ".expected");
+            int expectRowCount = Integer.parseInt(Files.readFirstLine(expectResultFile, Charset.defaultCharset()));
+
+            // execute Kylin
+            printInfo("Query Result from Kylin - " + queryName + "  (" + queryFolder + ")");
+            IDatabaseConnection kylinConn = new DatabaseConnection(cubeConnection);
+            ITable kylinTable = executeQuery(kylinConn, queryName, sql, false);
+
+            // compare the result
+            Assert.assertEquals(expectRowCount, kylinTable.getRowCount());
+            // Assertion.assertEquals(expectRowCount, kylinTable.getRowCount());
+        }
+    }
+
+    protected void execAndCompQuery(String queryFolder, String[] exclusiveQuerys, boolean needSort) throws Exception {
+        printInfo("---------- test folder: " + queryFolder);
+        Set<String> exclusiveSet = buildExclusiveSet(exclusiveQuerys);
+
+        List<File> sqlFiles = getFilesFromFolder(new File(queryFolder), ".sql");
+        for (File sqlFile : sqlFiles) {
+            String queryName = StringUtils.split(sqlFile.getName(), '.')[0];
+            if (exclusiveSet.contains(queryName)) {
+                continue;
+            }
+            String sql = getTextFromFile(sqlFile);
+
+            // execute Kylin
+            printInfo("Query Result from Kylin - " + queryName + "  (" + queryFolder + ")");
+            IDatabaseConnection kylinConn = new DatabaseConnection(cubeConnection);
+            ITable kylinTable = executeQuery(kylinConn, queryName, sql, needSort);
+
+            // execute H2
+            printInfo("Query Result from H2 - " + queryName);
+            H2Connection h2Conn = new H2Connection(h2Connection, null);
+            h2Conn.getConfig().setProperty(DatabaseConfig.PROPERTY_DATATYPE_FACTORY, new TestH2DataTypeFactory());
+            ITable h2Table = executeQuery(h2Conn, queryName, sql, needSort);
+
+            // compare the result
+            Assertion.assertEquals(h2Table, kylinTable);
+
+            compQueryCount++;
+            if (kylinTable.getRowCount() == 0) {
+                zeroResultQueries.add(sql);
+            }
+        }
+    }
+
+    protected void execAndCompDynamicQuery(String queryFolder, String[] exclusiveQuerys, boolean needSort) throws Exception {
+        printInfo("---------- test folder: " + queryFolder);
+        Set<String> exclusiveSet = buildExclusiveSet(exclusiveQuerys);
+
+        List<File> sqlFiles = getFilesFromFolder(new File(queryFolder), ".sql");
+        for (File sqlFile : sqlFiles) {
+            String queryName = StringUtils.split(sqlFile.getName(), '.')[0];
+            if (exclusiveSet.contains(queryName)) {
+                continue;
+            }
+            String sql = getTextFromFile(sqlFile);
+            List<String> parameters = getParameterFromFile(sqlFile);
+
+            // execute Kylin
+            printInfo("Query Result from Kylin - " + queryName + "  (" + queryFolder + ")");
+            IDatabaseConnection kylinConn = new DatabaseConnection(cubeConnection);
+            ITable kylinTable = executeDynamicQuery(kylinConn, queryName, sql, parameters, needSort);
+
+            // execute H2
+            printInfo("Query Result from H2 - " + queryName);
+            IDatabaseConnection h2Conn = new DatabaseConnection(h2Connection);
+            h2Conn.getConfig().setProperty(DatabaseConfig.PROPERTY_DATATYPE_FACTORY, new TestH2DataTypeFactory());
+            ITable h2Table = executeDynamicQuery(h2Conn, queryName, sql, parameters, needSort);
+
+            // compare the result
+            Assertion.assertEquals(h2Table, kylinTable);
+        }
+    }
+
+    protected int runSqlFile(String file) throws Exception {
+        return runSQL(new File(file), true, false);
+    }
+    
+    protected int runSQL(File sqlFile, boolean debug, boolean explain) throws Exception {
+        if (debug) {
+            System.setProperty("calcite.debug", "true");
+            InputStream inputStream = new FileInputStream("src/test/resources/logging.properties");
+            LogManager.getLogManager().readConfiguration(inputStream);
+        }
+
+        String queryName = StringUtils.split(sqlFile.getName(), '.')[0];
+        printInfo("Testing Query " + queryName);
+        String sql = getTextFromFile(sqlFile);
+        if (explain) {
+            sql = "explain plan for " + sql;
+        }
+        int count = executeQuery(sql, true);
+
+        if (debug) {
+            System.clearProperty("optiq.debug");
+        }
+        return count;
+    }
+
+    protected void batchExecuteQuery(String queryFolder) throws Exception {
+        List<File> sqlFiles = getFilesFromFolder(new File(queryFolder), ".sql");
+        for (File sqlFile : sqlFiles) {
+            runSQL(sqlFile, false, false);
+        }
+    }
+
+    protected int output(ResultSet resultSet, boolean needDisplay) throws SQLException {
+        int count = 0;
+        ResultSetMetaData metaData = resultSet.getMetaData();
+        int columnCount = metaData.getColumnCount();
+        StringBuilder sb = new StringBuilder("\n");
+        if (needDisplay) {
+            for (int i = 1; i <= columnCount; i++) {
+                sb.append(metaData.getColumnName(i));
+                sb.append("-");
+                sb.append(metaData.getTableName(i));
+                sb.append("-");
+                sb.append(metaData.getColumnTypeName(i));
+                if (i < columnCount) {
+                    sb.append("\t");
+                } else {
+                    sb.append("\n");
+                }
+            }
+        }
+
+        while (resultSet.next()) {
+            if (needDisplay) {
+                for (int i = 1; i <= columnCount; i++) {
+                    sb.append(resultSet.getString(i));
+                    if (i < columnCount) {
+                        sb.append("\t");
+                    } else {
+                        sb.append("\n");
+                    }
+                }
+            }
+            count++;
+        }
+        printInfo(sb.toString());
+        return count;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/query/src/test/resources/logging.properties
----------------------------------------------------------------------
diff --git a/query/src/test/resources/logging.properties b/query/src/test/resources/logging.properties
new file mode 100644
index 0000000..db07b53
--- /dev/null
+++ b/query/src/test/resources/logging.properties
@@ -0,0 +1,5 @@
+handlers=java.util.logging.ConsoleHandler
+.level=INFO
+#org.eigenbase.relopt.RelOptPlanner.level=FINEST
+java.util.logging.ConsoleHandler.level=ALL
+java.util.logging.ConsoleHandler.formatter=com.kylinolap.common.util.MyLogFormatter
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/query/src/test/resources/query/h2/query07.sql
----------------------------------------------------------------------
diff --git a/query/src/test/resources/query/h2/query07.sql b/query/src/test/resources/query/h2/query07.sql
new file mode 100644
index 0000000..5841f5f
--- /dev/null
+++ b/query/src/test/resources/query/h2/query07.sql
@@ -0,0 +1,3 @@
+select count(*) from ( select test_kylin_fact.lstg_format_name from test_kylin_fact 
+ where test_kylin_fact.lstg_format_name='FP-GTC' 
+ group by test_kylin_fact.lstg_format_name ) t 

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/query/src/test/resources/query/h2/query09.sql
----------------------------------------------------------------------
diff --git a/query/src/test/resources/query/h2/query09.sql b/query/src/test/resources/query/h2/query09.sql
new file mode 100644
index 0000000..08a64d8
--- /dev/null
+++ b/query/src/test/resources/query/h2/query09.sql
@@ -0,0 +1,5 @@
+select count(*) from (select test_cal_dt.week_beg_dt 
+ from test_kylin_fact 
+ inner JOIN test_cal_dt 
+ ON test_kylin_fact.cal_dt = test_cal_dt.cal_dt 
+ group by test_cal_dt.week_beg_dt) t 

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/query/src/test/resources/query/h2/query10.sql
----------------------------------------------------------------------
diff --git a/query/src/test/resources/query/h2/query10.sql b/query/src/test/resources/query/h2/query10.sql
new file mode 100644
index 0000000..5036c63
--- /dev/null
+++ b/query/src/test/resources/query/h2/query10.sql
@@ -0,0 +1,7 @@
+select test_cal_dt.week_beg_dt 
+ from test_kylin_fact 
+ inner JOIN test_cal_dt 
+ ON test_kylin_fact.cal_dt = test_cal_dt.cal_dt 
+ where test_kylin_fact.lstg_format_name='FP-GTC' 
+ and test_cal_dt.week_beg_dt between DATE '2013-05-01' and DATE '2013-08-01' 
+ group by test_cal_dt.week_beg_dt 

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/query/src/test/resources/query/sql/.gitignore
----------------------------------------------------------------------
diff --git a/query/src/test/resources/query/sql/.gitignore b/query/src/test/resources/query/sql/.gitignore
new file mode 100644
index 0000000..0daace0
--- /dev/null
+++ b/query/src/test/resources/query/sql/.gitignore
@@ -0,0 +1,2 @@
+/sample.txt
+/0000.sql

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/query/src/test/resources/query/sql/query00.sql
----------------------------------------------------------------------
diff --git a/query/src/test/resources/query/sql/query00.sql b/query/src/test/resources/query/sql/query00.sql
new file mode 100644
index 0000000..40c247d
--- /dev/null
+++ b/query/src/test/resources/query/sql/query00.sql
@@ -0,0 +1,4 @@
+select lstg_format_name, sum(price) as GMV 
+ from test_kylin_fact 
+ where lstg_format_name='FP-GTC' 
+ group by lstg_format_name 

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/query/src/test/resources/query/sql/query01.sql
----------------------------------------------------------------------
diff --git a/query/src/test/resources/query/sql/query01.sql b/query/src/test/resources/query/sql/query01.sql
new file mode 100644
index 0000000..52a2fcf
--- /dev/null
+++ b/query/src/test/resources/query/sql/query01.sql
@@ -0,0 +1,2 @@
+select LSTG_FORMAT_NAME, sum(price) as GMV, count(1) as TRANS_CNT from test_kylin_fact 
+ group by LSTG_FORMAT_NAME 

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/query/src/test/resources/query/sql/query02.sql
----------------------------------------------------------------------
diff --git a/query/src/test/resources/query/sql/query02.sql b/query/src/test/resources/query/sql/query02.sql
new file mode 100644
index 0000000..c2baad0
--- /dev/null
+++ b/query/src/test/resources/query/sql/query02.sql
@@ -0,0 +1 @@
+select sum(price) as GMV, count(1) as TRANS_CNT from test_kylin_fact 

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/query/src/test/resources/query/sql/query03.sql
----------------------------------------------------------------------
diff --git a/query/src/test/resources/query/sql/query03.sql b/query/src/test/resources/query/sql/query03.sql
new file mode 100644
index 0000000..0db8288
--- /dev/null
+++ b/query/src/test/resources/query/sql/query03.sql
@@ -0,0 +1,2 @@
+select test_kylin_fact.lstg_format_name, sum(price) as GMV, count(*) as TRANS_CNT from test_kylin_fact 
+ group by test_kylin_fact.lstg_format_name 

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/query/src/test/resources/query/sql/query04.sql
----------------------------------------------------------------------
diff --git a/query/src/test/resources/query/sql/query04.sql b/query/src/test/resources/query/sql/query04.sql
new file mode 100644
index 0000000..d7accd9
--- /dev/null
+++ b/query/src/test/resources/query/sql/query04.sql
@@ -0,0 +1,3 @@
+select test_kylin_fact.lstg_format_name,sum(test_kylin_fact.price) as GMV 
+ , count(*) as TRANS_CNT from test_kylin_fact 
+ group by test_kylin_fact.lstg_format_name having sum(price)>5000 

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/query/src/test/resources/query/sql/query05.sql
----------------------------------------------------------------------
diff --git a/query/src/test/resources/query/sql/query05.sql b/query/src/test/resources/query/sql/query05.sql
new file mode 100644
index 0000000..711fc5e
--- /dev/null
+++ b/query/src/test/resources/query/sql/query05.sql
@@ -0,0 +1,4 @@
+select test_kylin_fact.lstg_format_name,sum(test_kylin_fact.price) as GMV 
+ , count(*) as TRANS_CNT from test_kylin_fact 
+ where test_kylin_fact.lstg_format_name is null 
+ group by test_kylin_fact.lstg_format_name having sum(price)>5000 and count(*)>72 

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/query/src/test/resources/query/sql/query06.sql
----------------------------------------------------------------------
diff --git a/query/src/test/resources/query/sql/query06.sql b/query/src/test/resources/query/sql/query06.sql
new file mode 100644
index 0000000..486cd23
--- /dev/null
+++ b/query/src/test/resources/query/sql/query06.sql
@@ -0,0 +1,5 @@
+select test_kylin_fact.lstg_format_name,sum(test_kylin_fact.price) as GMV 
+ , count(*) as TRANS_CNT from test_kylin_fact 
+ where test_kylin_fact.lstg_format_name is not null 
+ group by test_kylin_fact.lstg_format_name 
+ having sum(price)>5000 or count(*)>20 

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/query/src/test/resources/query/sql/query07.sql
----------------------------------------------------------------------
diff --git a/query/src/test/resources/query/sql/query07.sql b/query/src/test/resources/query/sql/query07.sql
new file mode 100644
index 0000000..e399a0a
--- /dev/null
+++ b/query/src/test/resources/query/sql/query07.sql
@@ -0,0 +1,4 @@
+select test_kylin_fact.lstg_format_name,sum(test_kylin_fact.price) as GMV 
+ , count(*) as TRANS_CNT from test_kylin_fact 
+ where test_kylin_fact.lstg_format_name='FP-GTC' 
+ group by test_kylin_fact.lstg_format_name having sum(price)>5000 or count(*)>20 

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/query/src/test/resources/query/sql/query08.sql
----------------------------------------------------------------------
diff --git a/query/src/test/resources/query/sql/query08.sql b/query/src/test/resources/query/sql/query08.sql
new file mode 100644
index 0000000..bc4fd53
--- /dev/null
+++ b/query/src/test/resources/query/sql/query08.sql
@@ -0,0 +1,4 @@
+select test_kylin_fact.lstg_format_name,sum(test_kylin_fact.price) as GMV 
+ , count(*) as TRANS_CNT from test_kylin_fact 
+ where test_kylin_fact.lstg_format_name='FP-GTC' 
+ group by test_kylin_fact.lstg_format_name 

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/query/src/test/resources/query/sql/query09.sql
----------------------------------------------------------------------
diff --git a/query/src/test/resources/query/sql/query09.sql b/query/src/test/resources/query/sql/query09.sql
new file mode 100644
index 0000000..d3bfa89
--- /dev/null
+++ b/query/src/test/resources/query/sql/query09.sql
@@ -0,0 +1,5 @@
+select test_cal_dt.week_beg_dt, count(*) as TRANS_CNT 
+ from test_kylin_fact 
+ inner JOIN test_cal_dt 
+ ON test_kylin_fact.cal_dt = test_cal_dt.cal_dt 
+ group by test_cal_dt.week_beg_dt 

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/query/src/test/resources/query/sql/query10.sql
----------------------------------------------------------------------
diff --git a/query/src/test/resources/query/sql/query10.sql b/query/src/test/resources/query/sql/query10.sql
new file mode 100644
index 0000000..bd92a71
--- /dev/null
+++ b/query/src/test/resources/query/sql/query10.sql
@@ -0,0 +1,8 @@
+select test_cal_dt.week_beg_dt,sum(test_kylin_fact.price) as GMV 
+ , count(*) as TRANS_CNT 
+ from test_kylin_fact 
+ inner JOIN test_cal_dt 
+ ON test_kylin_fact.cal_dt = test_cal_dt.cal_dt 
+ where test_kylin_fact.lstg_format_name='FP-GTC' 
+ and test_cal_dt.week_beg_dt between DATE '2013-05-01' and DATE '2013-08-01' 
+ group by test_cal_dt.week_beg_dt 

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/query/src/test/resources/query/sql/query11.sql
----------------------------------------------------------------------
diff --git a/query/src/test/resources/query/sql/query11.sql b/query/src/test/resources/query/sql/query11.sql
new file mode 100644
index 0000000..f225aa8
--- /dev/null
+++ b/query/src/test/resources/query/sql/query11.sql
@@ -0,0 +1,8 @@
+select test_kylin_fact.lstg_format_name, test_cal_dt.week_beg_dt,sum(test_kylin_fact.price) as GMV 
+ , count(*) as TRANS_CNT 
+ from test_kylin_fact 
+ inner JOIN test_cal_dt 
+ ON test_kylin_fact.cal_dt = test_cal_dt.cal_dt 
+ where test_cal_dt.week_beg_dt between DATE '2013-05-01' and DATE '2013-08-01' 
+ group by test_kylin_fact.lstg_format_name, test_cal_dt.week_beg_dt 
+ having sum(price)>500 

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/query/src/test/resources/query/sql/query12.sql
----------------------------------------------------------------------
diff --git a/query/src/test/resources/query/sql/query12.sql b/query/src/test/resources/query/sql/query12.sql
new file mode 100644
index 0000000..5289dde
--- /dev/null
+++ b/query/src/test/resources/query/sql/query12.sql
@@ -0,0 +1,7 @@
+select test_kylin_fact.lstg_format_name, test_cal_dt.week_beg_dt,sum(test_kylin_fact.price) as GMV 
+ , count(*) as TRANS_CNT 
+ from test_kylin_fact 
+ inner JOIN test_cal_dt 
+ ON test_kylin_fact.cal_dt = test_cal_dt.cal_dt 
+ where test_cal_dt.week_beg_dt >= DATE '2013-02-10' 
+ group by test_kylin_fact.lstg_format_name, test_cal_dt.week_beg_dt 

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/query/src/test/resources/query/sql/query13.sql
----------------------------------------------------------------------
diff --git a/query/src/test/resources/query/sql/query13.sql b/query/src/test/resources/query/sql/query13.sql
new file mode 100644
index 0000000..4e8dd76
--- /dev/null
+++ b/query/src/test/resources/query/sql/query13.sql
@@ -0,0 +1,6 @@
+select sum(test_kylin_fact.price) as GMV 
+ , count(*) as TRANS_CNT from test_kylin_fact 
+ inner JOIN test_cal_dt 
+ ON test_kylin_fact.cal_dt = test_cal_dt.cal_dt 
+ where test_kylin_fact.lstg_format_name='FP-GTC' 
+ and test_cal_dt.week_beg_dt between DATE '2013-05-01' and DATE '2013-08-01' 

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/query/src/test/resources/query/sql/query14.sql
----------------------------------------------------------------------
diff --git a/query/src/test/resources/query/sql/query14.sql b/query/src/test/resources/query/sql/query14.sql
new file mode 100644
index 0000000..9fc8660
--- /dev/null
+++ b/query/src/test/resources/query/sql/query14.sql
@@ -0,0 +1,17 @@
+SELECT 
+ test_cal_dt.week_beg_dt 
+ ,test_category_groupings.meta_categ_name 
+ ,test_category_groupings.categ_lvl2_name 
+ ,test_category_groupings.categ_lvl3_name 
+ ,sum(test_kylin_fact.price) as GMV 
+ , count(*) as trans_cnt 
+ FROM test_kylin_fact 
+ inner JOIN test_cal_dt 
+ ON test_kylin_fact.cal_dt = test_cal_dt.cal_dt 
+ inner JOIN test_category_groupings 
+ ON test_kylin_fact.leaf_categ_id = test_category_groupings.leaf_categ_id 
+ AND test_kylin_fact.lstg_site_id = test_category_groupings.site_id 
+ group by test_cal_dt.week_beg_dt 
+ ,test_category_groupings.meta_categ_name 
+ ,test_category_groupings.categ_lvl2_name 
+ ,test_category_groupings.categ_lvl3_name 

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/query/src/test/resources/query/sql/query15.sql
----------------------------------------------------------------------
diff --git a/query/src/test/resources/query/sql/query15.sql b/query/src/test/resources/query/sql/query15.sql
new file mode 100644
index 0000000..57d292b
--- /dev/null
+++ b/query/src/test/resources/query/sql/query15.sql
@@ -0,0 +1,12 @@
+SELECT 
+ test_cal_dt.week_beg_dt 
+ ,test_category_groupings.meta_categ_name 
+ ,sum(price) as GMV, count(*) as TRANS_CNT 
+ FROM test_kylin_fact 
+ inner JOIN test_cal_dt 
+ ON test_kylin_fact.cal_dt = test_cal_dt.cal_dt 
+ inner JOIN test_category_groupings 
+ ON test_kylin_fact.leaf_categ_id = test_category_groupings.leaf_categ_id AND test_kylin_fact.lstg_site_id = test_category_groupings.site_id 
+ group by test_cal_dt.week_beg_dt 
+ ,test_category_groupings.meta_categ_name 
+ 

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/query/src/test/resources/query/sql/query16.sql
----------------------------------------------------------------------
diff --git a/query/src/test/resources/query/sql/query16.sql b/query/src/test/resources/query/sql/query16.sql
new file mode 100644
index 0000000..9576f38
--- /dev/null
+++ b/query/src/test/resources/query/sql/query16.sql
@@ -0,0 +1,16 @@
+SELECT 
+ test_cal_dt.week_beg_dt 
+ ,test_category_groupings.meta_categ_name 
+ ,test_category_groupings.categ_lvl2_name 
+ ,test_category_groupings.categ_lvl3_name 
+ ,sum(price) as GMV, count(*) as TRANS_CNT 
+ FROM test_kylin_fact 
+ inner JOIN test_cal_dt 
+ ON test_kylin_fact.cal_dt = test_cal_dt.cal_dt 
+ inner JOIN test_category_groupings 
+ ON test_kylin_fact.leaf_categ_id = test_category_groupings.leaf_categ_id 
+ AND test_kylin_fact.lstg_site_id = test_category_groupings.site_id 
+ group by test_cal_dt.week_beg_dt 
+ ,test_category_groupings.meta_categ_name 
+ ,test_category_groupings.categ_lvl2_name 
+ ,test_category_groupings.categ_lvl3_name 

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/query/src/test/resources/query/sql/query17.sql
----------------------------------------------------------------------
diff --git a/query/src/test/resources/query/sql/query17.sql b/query/src/test/resources/query/sql/query17.sql
new file mode 100644
index 0000000..a262477
--- /dev/null
+++ b/query/src/test/resources/query/sql/query17.sql
@@ -0,0 +1,18 @@
+SELECT 
+ test_cal_dt.week_beg_dt 
+ ,test_category_groupings.meta_categ_name 
+ ,test_category_groupings.categ_lvl2_name 
+ ,test_category_groupings.categ_lvl3_name 
+ ,sum(price) as GMV, count(*) as TRANS_CNT 
+ FROM test_kylin_fact 
+ inner JOIN test_cal_dt 
+ ON test_kylin_fact.cal_dt = test_cal_dt.cal_dt 
+ inner JOIN test_category_groupings 
+ ON test_kylin_fact.leaf_categ_id = test_category_groupings.leaf_categ_id AND test_kylin_fact.lstg_site_id = test_category_groupings.site_id 
+ where test_cal_dt.week_beg_dt between DATE '2013-09-01' and DATE '2013-10-01' 
+ and test_category_groupings.meta_categ_name='Collectibles' 
+ group by test_cal_dt.week_beg_dt 
+ ,test_category_groupings.meta_categ_name 
+ ,test_category_groupings.categ_lvl2_name 
+ ,test_category_groupings.categ_lvl3_name 
+ 

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/query/src/test/resources/query/sql/query18.sql
----------------------------------------------------------------------
diff --git a/query/src/test/resources/query/sql/query18.sql b/query/src/test/resources/query/sql/query18.sql
new file mode 100644
index 0000000..cab2b3b
--- /dev/null
+++ b/query/src/test/resources/query/sql/query18.sql
@@ -0,0 +1,14 @@
+SELECT 
+ test_cal_dt.week_beg_dt 
+ ,test_category_groupings.meta_categ_name 
+ ,sum(price) as GMV, count(*) as TRANS_CNT 
+ FROM test_kylin_fact 
+ inner JOIN test_cal_dt 
+ ON test_kylin_fact.cal_dt = test_cal_dt.cal_dt 
+ inner JOIN test_category_groupings 
+ ON test_kylin_fact.leaf_categ_id = test_category_groupings.leaf_categ_id AND test_kylin_fact.lstg_site_id = test_category_groupings.site_id 
+ where test_cal_dt.week_beg_dt between DATE '2013-09-01' and DATE '2013-10-01' 
+ and test_category_groupings.categ_lvl2_name='Comics' 
+ group by test_cal_dt.week_beg_dt 
+ ,test_category_groupings.meta_categ_name 
+ 

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/query/src/test/resources/query/sql/query19.sql
----------------------------------------------------------------------
diff --git a/query/src/test/resources/query/sql/query19.sql b/query/src/test/resources/query/sql/query19.sql
new file mode 100644
index 0000000..a262477
--- /dev/null
+++ b/query/src/test/resources/query/sql/query19.sql
@@ -0,0 +1,18 @@
+SELECT 
+ test_cal_dt.week_beg_dt 
+ ,test_category_groupings.meta_categ_name 
+ ,test_category_groupings.categ_lvl2_name 
+ ,test_category_groupings.categ_lvl3_name 
+ ,sum(price) as GMV, count(*) as TRANS_CNT 
+ FROM test_kylin_fact 
+ inner JOIN test_cal_dt 
+ ON test_kylin_fact.cal_dt = test_cal_dt.cal_dt 
+ inner JOIN test_category_groupings 
+ ON test_kylin_fact.leaf_categ_id = test_category_groupings.leaf_categ_id AND test_kylin_fact.lstg_site_id = test_category_groupings.site_id 
+ where test_cal_dt.week_beg_dt between DATE '2013-09-01' and DATE '2013-10-01' 
+ and test_category_groupings.meta_categ_name='Collectibles' 
+ group by test_cal_dt.week_beg_dt 
+ ,test_category_groupings.meta_categ_name 
+ ,test_category_groupings.categ_lvl2_name 
+ ,test_category_groupings.categ_lvl3_name 
+ 

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/query/src/test/resources/query/sql/query20.sql
----------------------------------------------------------------------
diff --git a/query/src/test/resources/query/sql/query20.sql b/query/src/test/resources/query/sql/query20.sql
new file mode 100644
index 0000000..c0489bf
--- /dev/null
+++ b/query/src/test/resources/query/sql/query20.sql
@@ -0,0 +1,14 @@
+SELECT 
+ test_cal_dt.week_beg_dt 
+ ,test_category_groupings.meta_categ_name 
+ ,sum(price) as GMV, count(*) as TRANS_CNT 
+ FROM test_kylin_fact 
+ inner JOIN test_cal_dt 
+ ON test_kylin_fact.cal_dt = test_cal_dt.cal_dt 
+ inner JOIN test_category_groupings 
+ ON test_kylin_fact.leaf_categ_id = test_category_groupings.leaf_categ_id AND test_kylin_fact.lstg_site_id = test_category_groupings.site_id 
+ where test_cal_dt.week_beg_dt between DATE '2013-09-01' and DATE '2013-10-01' 
+ and test_category_groupings.categ_lvl3_name='Other' 
+ group by test_cal_dt.week_beg_dt 
+ ,test_category_groupings.meta_categ_name 
+ 

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/query/src/test/resources/query/sql/query21.sql
----------------------------------------------------------------------
diff --git a/query/src/test/resources/query/sql/query21.sql b/query/src/test/resources/query/sql/query21.sql
new file mode 100644
index 0000000..bd0ca84
--- /dev/null
+++ b/query/src/test/resources/query/sql/query21.sql
@@ -0,0 +1,22 @@
+SELECT 
+ test_cal_dt.week_beg_dt 
+ ,test_category_groupings.meta_categ_name 
+ ,test_category_groupings.categ_lvl2_name 
+ ,test_category_groupings.categ_lvl3_name 
+ ,test_kylin_fact.lstg_format_name 
+ ,test_sites.site_name 
+ ,sum(test_kylin_fact.price) as GMV, count(*) as TRANS_CNT 
+ FROM test_kylin_fact 
+ inner JOIN test_cal_dt 
+ ON test_kylin_fact.cal_dt = test_cal_dt.cal_dt 
+ inner JOIN test_category_groupings 
+ ON test_kylin_fact.leaf_categ_id = test_category_groupings.leaf_categ_id AND test_kylin_fact.lstg_site_id = test_category_groupings.site_id 
+ inner JOIN test_sites 
+ ON test_kylin_fact.lstg_site_id = test_sites.site_id 
+ where test_cal_dt.week_beg_dt between DATE '2013-02-01' and DATE '2013-03-01' 
+ group by test_cal_dt.week_beg_dt 
+ ,test_category_groupings.meta_categ_name 
+ ,test_category_groupings.categ_lvl2_name 
+ ,test_category_groupings.categ_lvl3_name 
+ ,test_kylin_fact.lstg_format_name 
+ ,test_sites.site_name 

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/query/src/test/resources/query/sql/query22.sql
----------------------------------------------------------------------
diff --git a/query/src/test/resources/query/sql/query22.sql b/query/src/test/resources/query/sql/query22.sql
new file mode 100644
index 0000000..03af146
--- /dev/null
+++ b/query/src/test/resources/query/sql/query22.sql
@@ -0,0 +1,24 @@
+SELECT 
+ test_cal_dt.week_beg_dt 
+ ,test_category_groupings.meta_categ_name 
+ ,test_category_groupings.categ_lvl2_name 
+ ,test_category_groupings.categ_lvl3_name 
+ ,test_kylin_fact.lstg_format_name 
+ ,test_sites.site_name 
+ ,sum(test_kylin_fact.price) as GMV 
+ , count(*) as TRANS_CNT 
+ FROM test_kylin_fact 
+ inner JOIN test_cal_dt 
+ ON test_kylin_fact.cal_dt = test_cal_dt.cal_dt 
+ inner JOIN test_category_groupings 
+ ON test_kylin_fact.leaf_categ_id = test_category_groupings.leaf_categ_id AND test_kylin_fact.lstg_site_id = test_category_groupings.site_id 
+ inner JOIN test_sites 
+ ON test_kylin_fact.lstg_site_id = test_sites.site_id 
+ where test_cal_dt.week_beg_dt between DATE '2012-02-01' and DATE '2013-10-01' 
+ and site_name='Canada' 
+ group by test_cal_dt.week_beg_dt 
+ ,test_category_groupings.meta_categ_name 
+ ,test_category_groupings.categ_lvl2_name 
+ ,test_category_groupings.categ_lvl3_name 
+ ,test_kylin_fact.lstg_format_name 
+ ,test_sites.site_name 

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/query/src/test/resources/query/sql/query23.sql
----------------------------------------------------------------------
diff --git a/query/src/test/resources/query/sql/query23.sql b/query/src/test/resources/query/sql/query23.sql
new file mode 100644
index 0000000..63de8ac
--- /dev/null
+++ b/query/src/test/resources/query/sql/query23.sql
@@ -0,0 +1,23 @@
+SELECT 
+ test_cal_dt.week_beg_dt 
+ ,test_category_groupings.meta_categ_name 
+ ,test_kylin_fact.lstg_format_name 
+ ,test_sites.site_name 
+ ,sum(test_kylin_fact.price) as GMV 
+ , count(*) as TRANS_CNT 
+ FROM test_kylin_fact 
+ inner JOIN test_cal_dt 
+ ON test_kylin_fact.cal_dt = test_cal_dt.cal_dt 
+ inner JOIN test_category_groupings 
+ ON test_kylin_fact.leaf_categ_id = test_category_groupings.leaf_categ_id AND test_kylin_fact.lstg_site_id = test_category_groupings.site_id 
+ inner JOIN test_sites 
+ ON test_kylin_fact.lstg_site_id = test_sites.site_id 
+ where test_cal_dt.week_beg_dt between DATE '2013-02-01' and DATE '2013-10-01' 
+ and site_name='Ebay' 
+ and test_category_groupings.categ_lvl3_name='Other' 
+ and test_kylin_fact.lstg_format_name='Auction' 
+ group by test_cal_dt.week_beg_dt 
+ ,test_category_groupings.meta_categ_name 
+ ,test_kylin_fact.lstg_format_name 
+ ,test_sites.site_name 
+ 

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/query/src/test/resources/query/sql/query24.sql
----------------------------------------------------------------------
diff --git a/query/src/test/resources/query/sql/query24.sql b/query/src/test/resources/query/sql/query24.sql
new file mode 100644
index 0000000..0ca7bf6
--- /dev/null
+++ b/query/src/test/resources/query/sql/query24.sql
@@ -0,0 +1,27 @@
+SELECT 
+ test_cal_dt.week_beg_dt 
+ ,test_category_groupings.meta_categ_name 
+ ,test_category_groupings.categ_lvl2_name 
+ ,test_category_groupings.categ_lvl3_name 
+ ,test_kylin_fact.lstg_format_name 
+ ,test_sites.site_name 
+ ,test_seller_type_dim.seller_type_desc 
+ ,sum(test_kylin_fact.price) as GMV 
+ , count(*) as TRANS_CNT 
+ FROM test_kylin_fact 
+ inner JOIN test_cal_dt 
+ ON test_kylin_fact.cal_dt = test_cal_dt.cal_dt 
+ inner JOIN test_category_groupings 
+ ON test_kylin_fact.leaf_categ_id = test_category_groupings.leaf_categ_id AND test_kylin_fact.lstg_site_id = test_category_groupings.site_id 
+ inner JOIN test_sites 
+ ON test_kylin_fact.lstg_site_id = test_sites.site_id 
+ inner JOIN test_seller_type_dim 
+ ON test_kylin_fact.slr_segment_cd = test_seller_type_dim.seller_type_cd 
+ group by test_cal_dt.week_beg_dt 
+ ,test_category_groupings.meta_categ_name 
+ ,test_category_groupings.categ_lvl2_name 
+ ,test_category_groupings.categ_lvl3_name 
+ ,test_kylin_fact.lstg_format_name 
+ ,test_sites.site_name 
+ ,test_seller_type_dim.seller_type_desc 
+ 

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/query/src/test/resources/query/sql/query25.sql
----------------------------------------------------------------------
diff --git a/query/src/test/resources/query/sql/query25.sql b/query/src/test/resources/query/sql/query25.sql
new file mode 100644
index 0000000..7df7680
--- /dev/null
+++ b/query/src/test/resources/query/sql/query25.sql
@@ -0,0 +1,28 @@
+SELECT 
+ test_cal_dt.week_beg_dt 
+ ,test_category_groupings.meta_categ_name 
+ ,test_category_groupings.categ_lvl2_name 
+ ,test_category_groupings.categ_lvl3_name 
+ ,test_kylin_fact.lstg_format_name 
+ ,test_sites.site_name 
+ ,test_seller_type_dim.seller_type_desc 
+ ,sum(test_kylin_fact.price) as GMV 
+ , count(*) as TRANS_CNT 
+ FROM test_kylin_fact 
+ inner JOIN test_cal_dt 
+ ON test_kylin_fact.cal_dt = test_cal_dt.cal_dt 
+ inner JOIN test_category_groupings 
+ ON test_kylin_fact.leaf_categ_id = test_category_groupings.leaf_categ_id AND test_kylin_fact.lstg_site_id = test_category_groupings.site_id 
+ inner JOIN test_sites 
+ ON test_kylin_fact.lstg_site_id = test_sites.site_id 
+ inner JOIN test_seller_type_dim 
+ ON test_kylin_fact.slr_segment_cd = test_seller_type_dim.seller_type_cd 
+ where test_cal_dt.week_beg_dt between DATE '2013-01-01' and DATE '2013-06-01' 
+ group by test_cal_dt.week_beg_dt 
+ ,test_category_groupings.meta_categ_name 
+ ,test_category_groupings.categ_lvl2_name 
+ ,test_category_groupings.categ_lvl3_name 
+ ,test_kylin_fact.lstg_format_name 
+ ,test_sites.site_name 
+ ,test_seller_type_dim.seller_type_desc 
+ 

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/query/src/test/resources/query/sql/query26.sql
----------------------------------------------------------------------
diff --git a/query/src/test/resources/query/sql/query26.sql b/query/src/test/resources/query/sql/query26.sql
new file mode 100644
index 0000000..0aecb06
--- /dev/null
+++ b/query/src/test/resources/query/sql/query26.sql
@@ -0,0 +1,30 @@
+SELECT 
+ test_cal_dt.week_beg_dt 
+ ,test_category_groupings.meta_categ_name 
+ ,test_category_groupings.categ_lvl2_name 
+ ,test_category_groupings.categ_lvl3_name 
+ ,test_kylin_fact.lstg_format_name 
+ ,test_sites.site_name 
+ ,test_seller_type_dim.seller_type_desc 
+ ,sum(test_kylin_fact.price) as GMV 
+ , count(*) as TRANS_CNT 
+ FROM test_kylin_fact 
+ inner JOIN test_cal_dt 
+ ON test_kylin_fact.cal_dt = test_cal_dt.cal_dt 
+ inner JOIN test_category_groupings 
+ ON test_kylin_fact.leaf_categ_id = test_category_groupings.leaf_categ_id AND test_kylin_fact.lstg_site_id = test_category_groupings.site_id 
+ inner JOIN test_sites 
+ ON test_kylin_fact.lstg_site_id = test_sites.site_id 
+ inner JOIN test_seller_type_dim 
+ ON test_kylin_fact.slr_segment_cd = test_seller_type_dim.seller_type_cd 
+ where test_cal_dt.week_beg_dt between DATE '2013-01-01' and DATE '2013-06-04' 
+ and (test_category_groupings.meta_categ_name='Collectibles' or test_category_groupings.meta_categ_name='Clothing, Shoes & Accessories') and 
+ test_category_groupings.categ_lvl3_name <>'Other' and test_sites.site_name='Ebay' 
+ group by test_cal_dt.week_beg_dt 
+ ,test_category_groupings.meta_categ_name 
+ ,test_category_groupings.categ_lvl2_name 
+ ,test_category_groupings.categ_lvl3_name 
+ ,test_kylin_fact.lstg_format_name 
+ ,test_sites.site_name 
+ ,test_seller_type_dim.seller_type_desc 
+ 

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/query/src/test/resources/query/sql/query27.sql
----------------------------------------------------------------------
diff --git a/query/src/test/resources/query/sql/query27.sql b/query/src/test/resources/query/sql/query27.sql
new file mode 100644
index 0000000..515aea1
--- /dev/null
+++ b/query/src/test/resources/query/sql/query27.sql
@@ -0,0 +1,29 @@
+SELECT 
+ test_cal_dt.week_beg_dt 
+ ,test_category_groupings.meta_categ_name 
+ ,test_category_groupings.categ_lvl2_name 
+ ,test_category_groupings.categ_lvl3_name 
+ ,test_kylin_fact.lstg_format_name 
+ ,test_sites.site_name 
+ ,test_seller_type_dim.seller_type_desc 
+ ,sum(test_kylin_fact.price) as GMV 
+ , count(*) as TRANS_CNT 
+ FROM test_kylin_fact 
+ inner JOIN test_cal_dt 
+ ON test_kylin_fact.cal_dt = test_cal_dt.cal_dt 
+ inner JOIN test_category_groupings 
+ ON test_kylin_fact.leaf_categ_id = test_category_groupings.leaf_categ_id AND test_kylin_fact.lstg_site_id = test_category_groupings.site_id 
+ inner JOIN test_sites 
+ ON test_kylin_fact.lstg_site_id = test_sites.site_id 
+ inner JOIN test_seller_type_dim 
+ ON test_kylin_fact.slr_segment_cd = test_seller_type_dim.seller_type_cd 
+ where 
+ (test_category_groupings.meta_categ_name='Collectibles' or test_category_groupings.categ_lvl3_name='Dresses') 
+ and test_sites.site_name='Ebay' 
+ group by test_cal_dt.week_beg_dt 
+ ,test_category_groupings.meta_categ_name 
+ ,test_category_groupings.categ_lvl2_name 
+ ,test_category_groupings.categ_lvl3_name 
+ ,test_kylin_fact.lstg_format_name 
+ ,test_sites.site_name 
+ ,test_seller_type_dim.seller_type_desc 

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/query/src/test/resources/query/sql/query28.sql
----------------------------------------------------------------------
diff --git a/query/src/test/resources/query/sql/query28.sql b/query/src/test/resources/query/sql/query28.sql
new file mode 100644
index 0000000..a3d3220
--- /dev/null
+++ b/query/src/test/resources/query/sql/query28.sql
@@ -0,0 +1,12 @@
+SELECT 
+ sum(test_kylin_fact.price) as GMV 
+ , count(*) as TRANS_CNT 
+ FROM test_kylin_fact 
+ inner JOIN test_cal_dt 
+ ON test_kylin_fact.cal_dt = test_cal_dt.cal_dt 
+ inner JOIN test_category_groupings 
+ ON test_kylin_fact.leaf_categ_id = test_category_groupings.leaf_categ_id AND test_kylin_fact.lstg_site_id = test_category_groupings.site_id 
+ inner JOIN test_sites 
+ ON test_kylin_fact.lstg_site_id = test_sites.site_id 
+ inner JOIN test_seller_type_dim 
+ ON test_kylin_fact.slr_segment_cd = test_seller_type_dim.seller_type_cd 

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/query/src/test/resources/query/sql/query29.sql
----------------------------------------------------------------------
diff --git a/query/src/test/resources/query/sql/query29.sql b/query/src/test/resources/query/sql/query29.sql
new file mode 100644
index 0000000..8d0a2f4
--- /dev/null
+++ b/query/src/test/resources/query/sql/query29.sql
@@ -0,0 +1,14 @@
+SELECT 
+ test_cal_dt.week_beg_dt 
+ ,test_category_groupings.meta_categ_name 
+ ,test_category_groupings.categ_lvl2_name 
+ ,sum(price) as GMV, count(*) as TRANS_CNT 
+ FROM test_kylin_fact 
+ inner JOIN test_cal_dt 
+ ON test_kylin_fact.cal_dt = test_cal_dt.cal_dt 
+ inner JOIN test_category_groupings 
+ ON test_kylin_fact.leaf_categ_id = test_category_groupings.leaf_categ_id 
+ AND test_kylin_fact.lstg_site_id = test_category_groupings.site_id 
+ group by test_cal_dt.week_beg_dt 
+ ,test_category_groupings.meta_categ_name 
+ ,test_category_groupings.categ_lvl2_name 

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/query/src/test/resources/query/sql/query30.sql
----------------------------------------------------------------------
diff --git a/query/src/test/resources/query/sql/query30.sql b/query/src/test/resources/query/sql/query30.sql
new file mode 100644
index 0000000..57b7f57
--- /dev/null
+++ b/query/src/test/resources/query/sql/query30.sql
@@ -0,0 +1 @@
+select sum(price) as GMV, count(1) as TRANS_CNT from test_kylin_fact limit 50 

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/query/src/test/resources/query/sql/query31.sql
----------------------------------------------------------------------
diff --git a/query/src/test/resources/query/sql/query31.sql b/query/src/test/resources/query/sql/query31.sql
new file mode 100644
index 0000000..420de7a
--- /dev/null
+++ b/query/src/test/resources/query/sql/query31.sql
@@ -0,0 +1,22 @@
+SELECT 
+ test_cal_dt.week_beg_dt 
+ ,test_category_groupings.meta_categ_name 
+ ,test_category_groupings.categ_lvl2_name 
+ ,test_category_groupings.categ_lvl3_name 
+ ,test_kylin_fact.lstg_format_name 
+ ,test_sites.site_name 
+ ,sum(test_kylin_fact.price) as GMV, count(*) as TRANS_CNT 
+ FROM test_kylin_fact 
+ inner JOIN test_cal_dt 
+ ON test_kylin_fact.cal_dt = test_cal_dt.cal_dt 
+ inner JOIN test_category_groupings 
+ ON test_kylin_fact.leaf_categ_id = test_category_groupings.leaf_categ_id AND test_kylin_fact.lstg_site_id = test_category_groupings.site_id 
+ inner JOIN test_sites 
+ ON test_kylin_fact.lstg_site_id = test_sites.site_id 
+ where test_sites.site_name = '英国' 
+ group by test_cal_dt.week_beg_dt 
+ ,test_category_groupings.meta_categ_name 
+ ,test_category_groupings.categ_lvl2_name 
+ ,test_category_groupings.categ_lvl3_name 
+ ,test_kylin_fact.lstg_format_name 
+ ,test_sites.site_name 

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/query/src/test/resources/query/sql/query32.sql
----------------------------------------------------------------------
diff --git a/query/src/test/resources/query/sql/query32.sql b/query/src/test/resources/query/sql/query32.sql
new file mode 100644
index 0000000..33210d1
--- /dev/null
+++ b/query/src/test/resources/query/sql/query32.sql
@@ -0,0 +1,21 @@
+SELECT 
+ test_cal_dt.week_beg_dt 
+ ,test_category_groupings.meta_categ_name 
+ ,test_category_groupings.categ_lvl2_name 
+ ,test_category_groupings.categ_lvl3_name 
+ ,test_kylin_fact.lstg_format_name 
+ ,test_sites.site_name 
+ ,sum(test_kylin_fact.price) as GMV, count(*) as TRANS_CNT 
+ FROM test_kylin_fact 
+ inner JOIN test_cal_dt 
+ ON test_kylin_fact.cal_dt = test_cal_dt.cal_dt 
+ inner JOIN test_category_groupings 
+ ON test_kylin_fact.leaf_categ_id = test_category_groupings.leaf_categ_id AND test_kylin_fact.lstg_site_id = test_category_groupings.site_id 
+ inner JOIN test_sites 
+ ON test_kylin_fact.lstg_site_id = test_sites.site_id 
+ group by test_cal_dt.week_beg_dt 
+ ,test_category_groupings.meta_categ_name 
+ ,test_category_groupings.categ_lvl2_name 
+ ,test_category_groupings.categ_lvl3_name 
+ ,test_kylin_fact.lstg_format_name 
+ ,test_sites.site_name 

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/query/src/test/resources/query/sql/query33.sql
----------------------------------------------------------------------
diff --git a/query/src/test/resources/query/sql/query33.sql b/query/src/test/resources/query/sql/query33.sql
new file mode 100644
index 0000000..c9116dc
--- /dev/null
+++ b/query/src/test/resources/query/sql/query33.sql
@@ -0,0 +1,25 @@
+SELECT 
+ test_kylin_fact.seller_id 
+ ,test_cal_dt.week_beg_dt 
+ ,test_category_groupings.meta_categ_name 
+ ,test_category_groupings.categ_lvl2_name 
+ ,test_category_groupings.categ_lvl3_name 
+ ,test_kylin_fact.lstg_format_name 
+ ,test_sites.site_name 
+ ,sum(test_kylin_fact.price) as GMV, count(*) as TRANS_CNT 
+ FROM test_kylin_fact 
+ inner JOIN test_cal_dt 
+ ON test_kylin_fact.cal_dt = test_cal_dt.cal_dt 
+ inner JOIN test_category_groupings 
+ ON test_kylin_fact.leaf_categ_id = test_category_groupings.leaf_categ_id AND test_kylin_fact.lstg_site_id = test_category_groupings.site_id 
+ inner JOIN test_sites 
+ ON test_kylin_fact.lstg_site_id = test_sites.site_id 
+ where test_kylin_fact.seller_id = 10000002 
+ group by 
+ test_kylin_fact.seller_id 
+ ,test_cal_dt.week_beg_dt 
+ ,test_category_groupings.meta_categ_name 
+ ,test_category_groupings.categ_lvl2_name 
+ ,test_category_groupings.categ_lvl3_name 
+ ,test_kylin_fact.lstg_format_name 
+ ,test_sites.site_name 

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/query/src/test/resources/query/sql/query34.sql
----------------------------------------------------------------------
diff --git a/query/src/test/resources/query/sql/query34.sql b/query/src/test/resources/query/sql/query34.sql
new file mode 100644
index 0000000..e9b983a
--- /dev/null
+++ b/query/src/test/resources/query/sql/query34.sql
@@ -0,0 +1,23 @@
+SELECT 
+ test_cal_dt.week_beg_dt 
+ ,test_category_groupings.meta_categ_name 
+ ,test_category_groupings.categ_lvl2_name 
+ ,test_category_groupings.categ_lvl3_name 
+ ,test_kylin_fact.lstg_format_name 
+ ,test_sites.site_name 
+ ,sum(test_kylin_fact.price) as GMV, count(*) as TRANS_CNT 
+ FROM test_kylin_fact 
+ inner JOIN test_cal_dt 
+ ON test_kylin_fact.cal_dt = test_cal_dt.cal_dt 
+ inner JOIN test_category_groupings 
+ ON test_kylin_fact.leaf_categ_id = test_category_groupings.leaf_categ_id AND test_kylin_fact.lstg_site_id = test_category_groupings.site_id 
+ inner JOIN test_sites 
+ ON test_kylin_fact.lstg_site_id = test_sites.site_id 
+ where test_kylin_fact.seller_id = 10000002 
+ group by 
+ test_cal_dt.week_beg_dt 
+ ,test_category_groupings.meta_categ_name 
+ ,test_category_groupings.categ_lvl2_name 
+ ,test_category_groupings.categ_lvl3_name 
+ ,test_kylin_fact.lstg_format_name 
+ ,test_sites.site_name 

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/query/src/test/resources/query/sql/query35.sql
----------------------------------------------------------------------
diff --git a/query/src/test/resources/query/sql/query35.sql b/query/src/test/resources/query/sql/query35.sql
new file mode 100644
index 0000000..8989c4c
--- /dev/null
+++ b/query/src/test/resources/query/sql/query35.sql
@@ -0,0 +1,23 @@
+SELECT 
+ test_cal_dt.week_beg_dt 
+ ,test_category_groupings.meta_categ_name 
+ ,test_category_groupings.categ_lvl2_name 
+ ,test_category_groupings.categ_lvl3_name 
+ ,test_kylin_fact.lstg_format_name 
+ ,test_sites.site_name 
+ ,sum(test_kylin_fact.price) as GMV, count(*) as TRANS_CNT 
+ FROM test_kylin_fact 
+ inner JOIN test_cal_dt 
+ ON test_kylin_fact.cal_dt = test_cal_dt.cal_dt 
+ inner JOIN test_category_groupings 
+ ON test_kylin_fact.leaf_categ_id = test_category_groupings.leaf_categ_id AND test_kylin_fact.lstg_site_id = test_category_groupings.site_id 
+ inner JOIN test_sites 
+ ON test_kylin_fact.lstg_site_id = test_sites.site_id 
+ where test_kylin_fact.seller_id = 10000002 or test_kylin_fact.lstg_format_name = 'FP-non GTC' 
+ group by 
+ test_cal_dt.week_beg_dt 
+ ,test_category_groupings.meta_categ_name 
+ ,test_category_groupings.categ_lvl2_name 
+ ,test_category_groupings.categ_lvl3_name 
+ ,test_kylin_fact.lstg_format_name 
+ ,test_sites.site_name 

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/query/src/test/resources/query/sql/query36.sql
----------------------------------------------------------------------
diff --git a/query/src/test/resources/query/sql/query36.sql b/query/src/test/resources/query/sql/query36.sql
new file mode 100644
index 0000000..5bf8976
--- /dev/null
+++ b/query/src/test/resources/query/sql/query36.sql
@@ -0,0 +1,16 @@
+SELECT 
+ test_category_groupings.meta_categ_name 
+ ,sum(test_kylin_fact.price) as GMV_SUM 
+ ,max(test_kylin_fact.price) as GMV_MAX 
+ ,min(test_kylin_fact.price) as GMV_MIN 
+ ,count(*) as TRANS_CNT 
+ FROM test_kylin_fact 
+ inner JOIN test_cal_dt 
+ ON test_kylin_fact.cal_dt = test_cal_dt.cal_dt 
+ inner JOIN test_category_groupings 
+ ON test_kylin_fact.leaf_categ_id = test_category_groupings.leaf_categ_id AND test_kylin_fact.lstg_site_id = test_category_groupings.site_id 
+ inner JOIN test_sites 
+ ON test_kylin_fact.lstg_site_id = test_sites.site_id 
+ where test_kylin_fact.seller_id = 10000002 or test_kylin_fact.lstg_format_name = 'FP-non GTC' 
+ group by 
+ test_category_groupings.meta_categ_name 

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/query/src/test/resources/query/sql/query37.sql
----------------------------------------------------------------------
diff --git a/query/src/test/resources/query/sql/query37.sql b/query/src/test/resources/query/sql/query37.sql
new file mode 100644
index 0000000..69ca6bf
--- /dev/null
+++ b/query/src/test/resources/query/sql/query37.sql
@@ -0,0 +1,18 @@
+select test_cal_dt.week_beg_dt, sum(test_kylin_fact.price) as GMV 
+ , count(*) as TRANS_CNT 
+ from test_kylin_fact 
+ inner JOIN test_cal_dt 
+ ON test_kylin_fact.cal_dt = test_cal_dt.cal_dt 
+ where 
+ (test_kylin_fact.lstg_format_name > '') 
+ and ( 
+ (test_kylin_fact.lstg_format_name='FP-GTC') 
+ OR 
+ (test_cal_dt.week_beg_dt between DATE '2013-05-20' and DATE '2013-05-21') 
+ ) 
+ and ( 
+ (test_kylin_fact.lstg_format_name='ABIN') 
+ OR 
+ (test_cal_dt.week_beg_dt between DATE '2013-05-20' and DATE '2013-05-21') 
+ ) 
+ group by test_cal_dt.week_beg_dt 

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/query/src/test/resources/query/sql/query38.sql
----------------------------------------------------------------------
diff --git a/query/src/test/resources/query/sql/query38.sql b/query/src/test/resources/query/sql/query38.sql
new file mode 100644
index 0000000..191e12e
--- /dev/null
+++ b/query/src/test/resources/query/sql/query38.sql
@@ -0,0 +1,4 @@
+select lstg_format_name, sum(price) as GMV 
+ from test_kylin_fact 
+ where lstg_format_name not in ('FP-GTC', 'ABIN') 
+ group by lstg_format_name 

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/query/src/test/resources/query/sql/query39.sql
----------------------------------------------------------------------
diff --git a/query/src/test/resources/query/sql/query39.sql b/query/src/test/resources/query/sql/query39.sql
new file mode 100644
index 0000000..aa37afa
--- /dev/null
+++ b/query/src/test/resources/query/sql/query39.sql
@@ -0,0 +1,2 @@
+select sum(price) as GMV 
+ from test_kylin_fact 

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/query/src/test/resources/query/sql/query40.sql
----------------------------------------------------------------------
diff --git a/query/src/test/resources/query/sql/query40.sql b/query/src/test/resources/query/sql/query40.sql
new file mode 100644
index 0000000..277e297
--- /dev/null
+++ b/query/src/test/resources/query/sql/query40.sql
@@ -0,0 +1,4 @@
+select cal_dt, lstg_format_name, sum(price) as GMV 
+ from test_kylin_fact 
+ where cal_dt between date '2013-05-06' and date '2013-07-31' 
+ group by cal_dt, lstg_format_name 

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/query/src/test/resources/query/sql/query41.sql
----------------------------------------------------------------------
diff --git a/query/src/test/resources/query/sql/query41.sql b/query/src/test/resources/query/sql/query41.sql
new file mode 100644
index 0000000..1a13939
--- /dev/null
+++ b/query/src/test/resources/query/sql/query41.sql
@@ -0,0 +1,12 @@
+SELECT 
+ test_category_groupings.meta_categ_name 
+ ,test_category_groupings.categ_lvl2_name 
+ ,sum(test_kylin_fact.price) as GMV 
+ ,count(*) as trans_cnt 
+ FROM test_kylin_fact 
+ inner JOIN test_category_groupings 
+ ON test_kylin_fact.leaf_categ_id = test_category_groupings.leaf_categ_id 
+ AND test_kylin_fact.lstg_site_id = test_category_groupings.site_id 
+ group by 
+ test_category_groupings.meta_categ_name 
+ ,test_category_groupings.categ_lvl2_name 

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/query/src/test/resources/query/sql/query42.sql
----------------------------------------------------------------------
diff --git a/query/src/test/resources/query/sql/query42.sql b/query/src/test/resources/query/sql/query42.sql
new file mode 100644
index 0000000..2777785
--- /dev/null
+++ b/query/src/test/resources/query/sql/query42.sql
@@ -0,0 +1,5 @@
+select test_cal_dt.cal_dt, count(*) as TRANS_CNT 
+ from test_kylin_fact 
+ inner JOIN test_cal_dt 
+ ON test_kylin_fact.cal_dt = test_cal_dt.cal_dt 
+ group by test_cal_dt.cal_dt 

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/query/src/test/resources/query/sql/query43.sql
----------------------------------------------------------------------
diff --git a/query/src/test/resources/query/sql/query43.sql b/query/src/test/resources/query/sql/query43.sql
new file mode 100644
index 0000000..1cdc9d8
--- /dev/null
+++ b/query/src/test/resources/query/sql/query43.sql
@@ -0,0 +1,7 @@
+select test_cal_dt.cal_dt, count(*) as CNT 
+ from test_kylin_fact 
+ inner JOIN test_cal_dt 
+ ON test_kylin_fact.cal_dt = test_cal_dt.cal_dt 
+ where test_kylin_fact.lstg_format_name='FP-GTC' 
+ and test_cal_dt.week_beg_dt between DATE '2013-05-01' and DATE '2013-08-01' 
+ group by test_cal_dt.cal_dt 

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/query/src/test/resources/query/sql/query44.sql
----------------------------------------------------------------------
diff --git a/query/src/test/resources/query/sql/query44.sql b/query/src/test/resources/query/sql/query44.sql
new file mode 100644
index 0000000..b369ec3
--- /dev/null
+++ b/query/src/test/resources/query/sql/query44.sql
@@ -0,0 +1,10 @@
+SELECT 
+ test_category_groupings.meta_categ_name 
+ ,sum(test_kylin_fact.price) as GMV 
+ ,count(*) as trans_cnt 
+ FROM test_kylin_fact 
+ inner JOIN test_category_groupings 
+ ON test_kylin_fact.leaf_categ_id = test_category_groupings.leaf_categ_id 
+ AND test_kylin_fact.lstg_site_id = test_category_groupings.site_id 
+ group by 
+ test_category_groupings.meta_categ_name 

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/query/src/test/resources/query/sql/query45.sql
----------------------------------------------------------------------
diff --git a/query/src/test/resources/query/sql/query45.sql b/query/src/test/resources/query/sql/query45.sql
new file mode 100644
index 0000000..fa752d2
--- /dev/null
+++ b/query/src/test/resources/query/sql/query45.sql
@@ -0,0 +1,6 @@
+select count(*) as CNT from test_cal_dt
+
+
+
+
+

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/query/src/test/resources/query/sql/query46.sql
----------------------------------------------------------------------
diff --git a/query/src/test/resources/query/sql/query46.sql b/query/src/test/resources/query/sql/query46.sql
new file mode 100644
index 0000000..6dba282
--- /dev/null
+++ b/query/src/test/resources/query/sql/query46.sql
@@ -0,0 +1 @@
+select count(*) as CNT  from test_category_groupings
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/query/src/test/resources/query/sql/query47.sql
----------------------------------------------------------------------
diff --git a/query/src/test/resources/query/sql/query47.sql b/query/src/test/resources/query/sql/query47.sql
new file mode 100644
index 0000000..aa124a3
--- /dev/null
+++ b/query/src/test/resources/query/sql/query47.sql
@@ -0,0 +1 @@
+select count(*) as CNT  from test_seller_type_dim
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/query/src/test/resources/query/sql/query48.sql
----------------------------------------------------------------------
diff --git a/query/src/test/resources/query/sql/query48.sql b/query/src/test/resources/query/sql/query48.sql
new file mode 100644
index 0000000..64e078b
--- /dev/null
+++ b/query/src/test/resources/query/sql/query48.sql
@@ -0,0 +1 @@
+select count(*) as CNT from test_sites
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/query/src/test/resources/query/sql/query49.sql
----------------------------------------------------------------------
diff --git a/query/src/test/resources/query/sql/query49.sql b/query/src/test/resources/query/sql/query49.sql
new file mode 100644
index 0000000..04302b3
--- /dev/null
+++ b/query/src/test/resources/query/sql/query49.sql
@@ -0,0 +1,18 @@
+SELECT 
+ test_cal_dt.week_beg_dt 
+ ,test_category_groupings.meta_categ_name 
+ ,test_category_groupings.categ_lvl2_name 
+ ,test_category_groupings.categ_lvl3_name 
+ ,sum(price) as GMV, count(*) as TRANS_CNT 
+ FROM test_kylin_fact 
+ inner JOIN test_cal_dt 
+ ON test_kylin_fact.cal_dt = test_cal_dt.cal_dt 
+ inner JOIN test_category_groupings 
+ ON test_kylin_fact.lstg_site_id = test_category_groupings.site_id  AND test_kylin_fact.leaf_categ_id = test_category_groupings.leaf_categ_id
+ where test_cal_dt.week_beg_dt between DATE '2013-09-01' and DATE '2013-10-01' 
+ and test_category_groupings.meta_categ_name='Collectibles' 
+ group by test_cal_dt.week_beg_dt 
+ ,test_category_groupings.meta_categ_name 
+ ,test_category_groupings.categ_lvl2_name 
+ ,test_category_groupings.categ_lvl3_name 
+ 

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/query/src/test/resources/query/sql/query50.sql
----------------------------------------------------------------------
diff --git a/query/src/test/resources/query/sql/query50.sql b/query/src/test/resources/query/sql/query50.sql
new file mode 100644
index 0000000..d678855
--- /dev/null
+++ b/query/src/test/resources/query/sql/query50.sql
@@ -0,0 +1,7 @@
+select test_kylin_fact.lstg_format_name, test_cal_dt.week_beg_dt,sum(test_kylin_fact.price) as GMV 
+ , count(*) as TRANS_CNT 
+ from test_kylin_fact 
+ inner JOIN test_cal_dt 
+ ON test_kylin_fact.cal_dt = test_cal_dt.cal_dt 
+ where  1 < 3
+ group by test_kylin_fact.lstg_format_name, test_cal_dt.week_beg_dt

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/query/src/test/resources/query/sql/query51.sql
----------------------------------------------------------------------
diff --git a/query/src/test/resources/query/sql/query51.sql b/query/src/test/resources/query/sql/query51.sql
new file mode 100644
index 0000000..b7bc5c4
--- /dev/null
+++ b/query/src/test/resources/query/sql/query51.sql
@@ -0,0 +1,7 @@
+select test_kylin_fact.lstg_format_name, test_cal_dt.week_beg_dt,sum(test_kylin_fact.price) as GMV 
+ , count(*) as TRANS_CNT 
+ from test_kylin_fact 
+ inner JOIN test_cal_dt 
+ ON test_kylin_fact.cal_dt = test_cal_dt.cal_dt 
+ where  DATE '2013-03-24'  <= test_cal_dt.week_beg_dt
+ group by test_kylin_fact.lstg_format_name, test_cal_dt.week_beg_dt 

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/query/src/test/resources/query/sql/query52.sql
----------------------------------------------------------------------
diff --git a/query/src/test/resources/query/sql/query52.sql b/query/src/test/resources/query/sql/query52.sql
new file mode 100644
index 0000000..1a49ce1
--- /dev/null
+++ b/query/src/test/resources/query/sql/query52.sql
@@ -0,0 +1,4 @@
+select lstg_format_name, sum(price) as GMV 
+ from test_kylin_fact 
+ where test_kylin_fact.seller_id in ( 10000002, 10000003, 10000004,10000005,10000006,10000008,10000009,10000001,10000010,10000011)
+ group by lstg_format_name 

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/query/src/test/resources/query/sql/query53.sql
----------------------------------------------------------------------
diff --git a/query/src/test/resources/query/sql/query53.sql b/query/src/test/resources/query/sql/query53.sql
new file mode 100644
index 0000000..193297d
--- /dev/null
+++ b/query/src/test/resources/query/sql/query53.sql
@@ -0,0 +1,6 @@
+select test_kylin_fact.cal_dt,test_kylin_fact.seller_id, sum(test_kylin_fact.price) as GMV
+ , count(*) as TRANS_CNT 
+from test_kylin_fact
+where DATE '2012-09-01' <= test_kylin_fact.cal_dt   and  test_kylin_fact.seller_id = 10000002
+ group by test_kylin_fact.cal_dt,
+test_kylin_fact.seller_id
\ No newline at end of file


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

Posted by lu...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/job/.settings/org.eclipse.jdt.core.prefs
----------------------------------------------------------------------
diff --git a/job/.settings/org.eclipse.jdt.core.prefs b/job/.settings/org.eclipse.jdt.core.prefs
new file mode 100644
index 0000000..a903301
--- /dev/null
+++ b/job/.settings/org.eclipse.jdt.core.prefs
@@ -0,0 +1,379 @@
+eclipse.preferences.version=1
+org.eclipse.jdt.core.compiler.annotation.inheritNullAnnotations=disabled
+org.eclipse.jdt.core.compiler.annotation.missingNonNullByDefaultAnnotation=ignore
+org.eclipse.jdt.core.compiler.annotation.nonnull=org.eclipse.jdt.annotation.NonNull
+org.eclipse.jdt.core.compiler.annotation.nonnullbydefault=org.eclipse.jdt.annotation.NonNullByDefault
+org.eclipse.jdt.core.compiler.annotation.nullable=org.eclipse.jdt.annotation.Nullable
+org.eclipse.jdt.core.compiler.annotation.nullanalysis=disabled
+org.eclipse.jdt.core.compiler.codegen.inlineJsrBytecode=enabled
+org.eclipse.jdt.core.compiler.codegen.targetPlatform=1.7
+org.eclipse.jdt.core.compiler.codegen.unusedLocal=preserve
+org.eclipse.jdt.core.compiler.compliance=1.7
+org.eclipse.jdt.core.compiler.debug.lineNumber=generate
+org.eclipse.jdt.core.compiler.debug.localVariable=generate
+org.eclipse.jdt.core.compiler.debug.sourceFile=generate
+org.eclipse.jdt.core.compiler.problem.annotationSuperInterface=warning
+org.eclipse.jdt.core.compiler.problem.assertIdentifier=error
+org.eclipse.jdt.core.compiler.problem.autoboxing=ignore
+org.eclipse.jdt.core.compiler.problem.comparingIdentical=warning
+org.eclipse.jdt.core.compiler.problem.deadCode=warning
+org.eclipse.jdt.core.compiler.problem.deprecation=warning
+org.eclipse.jdt.core.compiler.problem.deprecationInDeprecatedCode=disabled
+org.eclipse.jdt.core.compiler.problem.deprecationWhenOverridingDeprecatedMethod=disabled
+org.eclipse.jdt.core.compiler.problem.discouragedReference=warning
+org.eclipse.jdt.core.compiler.problem.emptyStatement=ignore
+org.eclipse.jdt.core.compiler.problem.enumIdentifier=error
+org.eclipse.jdt.core.compiler.problem.explicitlyClosedAutoCloseable=ignore
+org.eclipse.jdt.core.compiler.problem.fallthroughCase=ignore
+org.eclipse.jdt.core.compiler.problem.fatalOptionalError=disabled
+org.eclipse.jdt.core.compiler.problem.fieldHiding=ignore
+org.eclipse.jdt.core.compiler.problem.finalParameterBound=warning
+org.eclipse.jdt.core.compiler.problem.finallyBlockNotCompletingNormally=warning
+org.eclipse.jdt.core.compiler.problem.forbiddenReference=warning
+org.eclipse.jdt.core.compiler.problem.hiddenCatchBlock=warning
+org.eclipse.jdt.core.compiler.problem.includeNullInfoFromAsserts=disabled
+org.eclipse.jdt.core.compiler.problem.incompatibleNonInheritedInterfaceMethod=warning
+org.eclipse.jdt.core.compiler.problem.incompleteEnumSwitch=warning
+org.eclipse.jdt.core.compiler.problem.indirectStaticAccess=ignore
+org.eclipse.jdt.core.compiler.problem.localVariableHiding=ignore
+org.eclipse.jdt.core.compiler.problem.methodWithConstructorName=warning
+org.eclipse.jdt.core.compiler.problem.missingDefaultCase=ignore
+org.eclipse.jdt.core.compiler.problem.missingDeprecatedAnnotation=ignore
+org.eclipse.jdt.core.compiler.problem.missingEnumCaseDespiteDefault=disabled
+org.eclipse.jdt.core.compiler.problem.missingHashCodeMethod=ignore
+org.eclipse.jdt.core.compiler.problem.missingOverrideAnnotation=ignore
+org.eclipse.jdt.core.compiler.problem.missingOverrideAnnotationForInterfaceMethodImplementation=enabled
+org.eclipse.jdt.core.compiler.problem.missingSerialVersion=warning
+org.eclipse.jdt.core.compiler.problem.missingSynchronizedOnInheritedMethod=ignore
+org.eclipse.jdt.core.compiler.problem.noEffectAssignment=warning
+org.eclipse.jdt.core.compiler.problem.noImplicitStringConversion=warning
+org.eclipse.jdt.core.compiler.problem.nonExternalizedStringLiteral=ignore
+org.eclipse.jdt.core.compiler.problem.nonnullParameterAnnotationDropped=warning
+org.eclipse.jdt.core.compiler.problem.nullAnnotationInferenceConflict=error
+org.eclipse.jdt.core.compiler.problem.nullReference=warning
+org.eclipse.jdt.core.compiler.problem.nullSpecViolation=error
+org.eclipse.jdt.core.compiler.problem.nullUncheckedConversion=warning
+org.eclipse.jdt.core.compiler.problem.overridingPackageDefaultMethod=warning
+org.eclipse.jdt.core.compiler.problem.parameterAssignment=ignore
+org.eclipse.jdt.core.compiler.problem.possibleAccidentalBooleanAssignment=ignore
+org.eclipse.jdt.core.compiler.problem.potentialNullReference=ignore
+org.eclipse.jdt.core.compiler.problem.potentiallyUnclosedCloseable=ignore
+org.eclipse.jdt.core.compiler.problem.rawTypeReference=warning
+org.eclipse.jdt.core.compiler.problem.redundantNullAnnotation=warning
+org.eclipse.jdt.core.compiler.problem.redundantNullCheck=ignore
+org.eclipse.jdt.core.compiler.problem.redundantSpecificationOfTypeArguments=ignore
+org.eclipse.jdt.core.compiler.problem.redundantSuperinterface=ignore
+org.eclipse.jdt.core.compiler.problem.reportMethodCanBePotentiallyStatic=ignore
+org.eclipse.jdt.core.compiler.problem.reportMethodCanBeStatic=ignore
+org.eclipse.jdt.core.compiler.problem.specialParameterHidingField=disabled
+org.eclipse.jdt.core.compiler.problem.staticAccessReceiver=warning
+org.eclipse.jdt.core.compiler.problem.suppressOptionalErrors=disabled
+org.eclipse.jdt.core.compiler.problem.suppressWarnings=enabled
+org.eclipse.jdt.core.compiler.problem.syntacticNullAnalysisForFields=disabled
+org.eclipse.jdt.core.compiler.problem.syntheticAccessEmulation=ignore
+org.eclipse.jdt.core.compiler.problem.typeParameterHiding=warning
+org.eclipse.jdt.core.compiler.problem.unavoidableGenericTypeProblems=enabled
+org.eclipse.jdt.core.compiler.problem.uncheckedTypeOperation=warning
+org.eclipse.jdt.core.compiler.problem.unclosedCloseable=warning
+org.eclipse.jdt.core.compiler.problem.undocumentedEmptyBlock=ignore
+org.eclipse.jdt.core.compiler.problem.unhandledWarningToken=warning
+org.eclipse.jdt.core.compiler.problem.unnecessaryElse=ignore
+org.eclipse.jdt.core.compiler.problem.unnecessaryTypeCheck=ignore
+org.eclipse.jdt.core.compiler.problem.unqualifiedFieldAccess=ignore
+org.eclipse.jdt.core.compiler.problem.unusedDeclaredThrownException=ignore
+org.eclipse.jdt.core.compiler.problem.unusedDeclaredThrownExceptionExemptExceptionAndThrowable=enabled
+org.eclipse.jdt.core.compiler.problem.unusedDeclaredThrownExceptionIncludeDocCommentReference=enabled
+org.eclipse.jdt.core.compiler.problem.unusedDeclaredThrownExceptionWhenOverriding=disabled
+org.eclipse.jdt.core.compiler.problem.unusedImport=warning
+org.eclipse.jdt.core.compiler.problem.unusedLabel=warning
+org.eclipse.jdt.core.compiler.problem.unusedLocal=warning
+org.eclipse.jdt.core.compiler.problem.unusedObjectAllocation=ignore
+org.eclipse.jdt.core.compiler.problem.unusedParameter=ignore
+org.eclipse.jdt.core.compiler.problem.unusedParameterIncludeDocCommentReference=enabled
+org.eclipse.jdt.core.compiler.problem.unusedParameterWhenImplementingAbstract=disabled
+org.eclipse.jdt.core.compiler.problem.unusedParameterWhenOverridingConcrete=disabled
+org.eclipse.jdt.core.compiler.problem.unusedPrivateMember=warning
+org.eclipse.jdt.core.compiler.problem.unusedTypeParameter=ignore
+org.eclipse.jdt.core.compiler.problem.unusedWarningToken=warning
+org.eclipse.jdt.core.compiler.problem.varargsArgumentNeedCast=warning
+org.eclipse.jdt.core.compiler.source=1.7
+org.eclipse.jdt.core.formatter.align_type_members_on_columns=false
+org.eclipse.jdt.core.formatter.alignment_for_arguments_in_allocation_expression=16
+org.eclipse.jdt.core.formatter.alignment_for_arguments_in_annotation=0
+org.eclipse.jdt.core.formatter.alignment_for_arguments_in_enum_constant=16
+org.eclipse.jdt.core.formatter.alignment_for_arguments_in_explicit_constructor_call=16
+org.eclipse.jdt.core.formatter.alignment_for_arguments_in_method_invocation=16
+org.eclipse.jdt.core.formatter.alignment_for_arguments_in_qualified_allocation_expression=16
+org.eclipse.jdt.core.formatter.alignment_for_assignment=0
+org.eclipse.jdt.core.formatter.alignment_for_binary_expression=16
+org.eclipse.jdt.core.formatter.alignment_for_compact_if=16
+org.eclipse.jdt.core.formatter.alignment_for_conditional_expression=80
+org.eclipse.jdt.core.formatter.alignment_for_enum_constants=0
+org.eclipse.jdt.core.formatter.alignment_for_expressions_in_array_initializer=16
+org.eclipse.jdt.core.formatter.alignment_for_method_declaration=0
+org.eclipse.jdt.core.formatter.alignment_for_multiple_fields=16
+org.eclipse.jdt.core.formatter.alignment_for_parameters_in_constructor_declaration=16
+org.eclipse.jdt.core.formatter.alignment_for_parameters_in_method_declaration=16
+org.eclipse.jdt.core.formatter.alignment_for_resources_in_try=80
+org.eclipse.jdt.core.formatter.alignment_for_selector_in_method_invocation=16
+org.eclipse.jdt.core.formatter.alignment_for_superclass_in_type_declaration=16
+org.eclipse.jdt.core.formatter.alignment_for_superinterfaces_in_enum_declaration=16
+org.eclipse.jdt.core.formatter.alignment_for_superinterfaces_in_type_declaration=16
+org.eclipse.jdt.core.formatter.alignment_for_throws_clause_in_constructor_declaration=16
+org.eclipse.jdt.core.formatter.alignment_for_throws_clause_in_method_declaration=16
+org.eclipse.jdt.core.formatter.alignment_for_union_type_in_multicatch=16
+org.eclipse.jdt.core.formatter.blank_lines_after_imports=1
+org.eclipse.jdt.core.formatter.blank_lines_after_package=1
+org.eclipse.jdt.core.formatter.blank_lines_before_field=0
+org.eclipse.jdt.core.formatter.blank_lines_before_first_class_body_declaration=0
+org.eclipse.jdt.core.formatter.blank_lines_before_imports=1
+org.eclipse.jdt.core.formatter.blank_lines_before_member_type=1
+org.eclipse.jdt.core.formatter.blank_lines_before_method=1
+org.eclipse.jdt.core.formatter.blank_lines_before_new_chunk=1
+org.eclipse.jdt.core.formatter.blank_lines_before_package=0
+org.eclipse.jdt.core.formatter.blank_lines_between_import_groups=1
+org.eclipse.jdt.core.formatter.blank_lines_between_type_declarations=1
+org.eclipse.jdt.core.formatter.brace_position_for_annotation_type_declaration=end_of_line
+org.eclipse.jdt.core.formatter.brace_position_for_anonymous_type_declaration=end_of_line
+org.eclipse.jdt.core.formatter.brace_position_for_array_initializer=end_of_line
+org.eclipse.jdt.core.formatter.brace_position_for_block=end_of_line
+org.eclipse.jdt.core.formatter.brace_position_for_block_in_case=end_of_line
+org.eclipse.jdt.core.formatter.brace_position_for_constructor_declaration=end_of_line
+org.eclipse.jdt.core.formatter.brace_position_for_enum_constant=end_of_line
+org.eclipse.jdt.core.formatter.brace_position_for_enum_declaration=end_of_line
+org.eclipse.jdt.core.formatter.brace_position_for_method_declaration=end_of_line
+org.eclipse.jdt.core.formatter.brace_position_for_switch=end_of_line
+org.eclipse.jdt.core.formatter.brace_position_for_type_declaration=end_of_line
+org.eclipse.jdt.core.formatter.comment.clear_blank_lines_in_block_comment=false
+org.eclipse.jdt.core.formatter.comment.clear_blank_lines_in_javadoc_comment=false
+org.eclipse.jdt.core.formatter.comment.format_block_comments=false
+org.eclipse.jdt.core.formatter.comment.format_header=false
+org.eclipse.jdt.core.formatter.comment.format_html=true
+org.eclipse.jdt.core.formatter.comment.format_javadoc_comments=false
+org.eclipse.jdt.core.formatter.comment.format_line_comments=false
+org.eclipse.jdt.core.formatter.comment.format_source_code=true
+org.eclipse.jdt.core.formatter.comment.indent_parameter_description=true
+org.eclipse.jdt.core.formatter.comment.indent_root_tags=true
+org.eclipse.jdt.core.formatter.comment.insert_new_line_before_root_tags=insert
+org.eclipse.jdt.core.formatter.comment.insert_new_line_for_parameter=insert
+org.eclipse.jdt.core.formatter.comment.line_length=80
+org.eclipse.jdt.core.formatter.comment.new_lines_at_block_boundaries=true
+org.eclipse.jdt.core.formatter.comment.new_lines_at_javadoc_boundaries=true
+org.eclipse.jdt.core.formatter.comment.preserve_white_space_between_code_and_line_comments=false
+org.eclipse.jdt.core.formatter.compact_else_if=true
+org.eclipse.jdt.core.formatter.continuation_indentation=2
+org.eclipse.jdt.core.formatter.continuation_indentation_for_array_initializer=2
+org.eclipse.jdt.core.formatter.disabling_tag=@formatter\:off
+org.eclipse.jdt.core.formatter.enabling_tag=@formatter\:on
+org.eclipse.jdt.core.formatter.format_guardian_clause_on_one_line=false
+org.eclipse.jdt.core.formatter.format_line_comment_starting_on_first_column=true
+org.eclipse.jdt.core.formatter.indent_body_declarations_compare_to_annotation_declaration_header=true
+org.eclipse.jdt.core.formatter.indent_body_declarations_compare_to_enum_constant_header=true
+org.eclipse.jdt.core.formatter.indent_body_declarations_compare_to_enum_declaration_header=true
+org.eclipse.jdt.core.formatter.indent_body_declarations_compare_to_type_header=true
+org.eclipse.jdt.core.formatter.indent_breaks_compare_to_cases=true
+org.eclipse.jdt.core.formatter.indent_empty_lines=false
+org.eclipse.jdt.core.formatter.indent_statements_compare_to_block=true
+org.eclipse.jdt.core.formatter.indent_statements_compare_to_body=true
+org.eclipse.jdt.core.formatter.indent_switchstatements_compare_to_cases=true
+org.eclipse.jdt.core.formatter.indent_switchstatements_compare_to_switch=false
+org.eclipse.jdt.core.formatter.indentation.size=4
+org.eclipse.jdt.core.formatter.insert_new_line_after_annotation_on_field=insert
+org.eclipse.jdt.core.formatter.insert_new_line_after_annotation_on_local_variable=insert
+org.eclipse.jdt.core.formatter.insert_new_line_after_annotation_on_method=insert
+org.eclipse.jdt.core.formatter.insert_new_line_after_annotation_on_package=insert
+org.eclipse.jdt.core.formatter.insert_new_line_after_annotation_on_parameter=do not insert
+org.eclipse.jdt.core.formatter.insert_new_line_after_annotation_on_type=insert
+org.eclipse.jdt.core.formatter.insert_new_line_after_label=do not insert
+org.eclipse.jdt.core.formatter.insert_new_line_after_opening_brace_in_array_initializer=do not insert
+org.eclipse.jdt.core.formatter.insert_new_line_at_end_of_file_if_missing=do not insert
+org.eclipse.jdt.core.formatter.insert_new_line_before_catch_in_try_statement=do not insert
+org.eclipse.jdt.core.formatter.insert_new_line_before_closing_brace_in_array_initializer=do not insert
+org.eclipse.jdt.core.formatter.insert_new_line_before_else_in_if_statement=do not insert
+org.eclipse.jdt.core.formatter.insert_new_line_before_finally_in_try_statement=do not insert
+org.eclipse.jdt.core.formatter.insert_new_line_before_while_in_do_statement=do not insert
+org.eclipse.jdt.core.formatter.insert_new_line_in_empty_annotation_declaration=insert
+org.eclipse.jdt.core.formatter.insert_new_line_in_empty_anonymous_type_declaration=insert
+org.eclipse.jdt.core.formatter.insert_new_line_in_empty_block=insert
+org.eclipse.jdt.core.formatter.insert_new_line_in_empty_enum_constant=insert
+org.eclipse.jdt.core.formatter.insert_new_line_in_empty_enum_declaration=insert
+org.eclipse.jdt.core.formatter.insert_new_line_in_empty_method_body=insert
+org.eclipse.jdt.core.formatter.insert_new_line_in_empty_type_declaration=insert
+org.eclipse.jdt.core.formatter.insert_space_after_and_in_type_parameter=insert
+org.eclipse.jdt.core.formatter.insert_space_after_assignment_operator=insert
+org.eclipse.jdt.core.formatter.insert_space_after_at_in_annotation=do not insert
+org.eclipse.jdt.core.formatter.insert_space_after_at_in_annotation_type_declaration=do not insert
+org.eclipse.jdt.core.formatter.insert_space_after_binary_operator=insert
+org.eclipse.jdt.core.formatter.insert_space_after_closing_angle_bracket_in_type_arguments=insert
+org.eclipse.jdt.core.formatter.insert_space_after_closing_angle_bracket_in_type_parameters=insert
+org.eclipse.jdt.core.formatter.insert_space_after_closing_brace_in_block=insert
+org.eclipse.jdt.core.formatter.insert_space_after_closing_paren_in_cast=insert
+org.eclipse.jdt.core.formatter.insert_space_after_colon_in_assert=insert
+org.eclipse.jdt.core.formatter.insert_space_after_colon_in_case=insert
+org.eclipse.jdt.core.formatter.insert_space_after_colon_in_conditional=insert
+org.eclipse.jdt.core.formatter.insert_space_after_colon_in_for=insert
+org.eclipse.jdt.core.formatter.insert_space_after_colon_in_labeled_statement=insert
+org.eclipse.jdt.core.formatter.insert_space_after_comma_in_allocation_expression=insert
+org.eclipse.jdt.core.formatter.insert_space_after_comma_in_annotation=insert
+org.eclipse.jdt.core.formatter.insert_space_after_comma_in_array_initializer=insert
+org.eclipse.jdt.core.formatter.insert_space_after_comma_in_constructor_declaration_parameters=insert
+org.eclipse.jdt.core.formatter.insert_space_after_comma_in_constructor_declaration_throws=insert
+org.eclipse.jdt.core.formatter.insert_space_after_comma_in_enum_constant_arguments=insert
+org.eclipse.jdt.core.formatter.insert_space_after_comma_in_enum_declarations=insert
+org.eclipse.jdt.core.formatter.insert_space_after_comma_in_explicitconstructorcall_arguments=insert
+org.eclipse.jdt.core.formatter.insert_space_after_comma_in_for_increments=insert
+org.eclipse.jdt.core.formatter.insert_space_after_comma_in_for_inits=insert
+org.eclipse.jdt.core.formatter.insert_space_after_comma_in_method_declaration_parameters=insert
+org.eclipse.jdt.core.formatter.insert_space_after_comma_in_method_declaration_throws=insert
+org.eclipse.jdt.core.formatter.insert_space_after_comma_in_method_invocation_arguments=insert
+org.eclipse.jdt.core.formatter.insert_space_after_comma_in_multiple_field_declarations=insert
+org.eclipse.jdt.core.formatter.insert_space_after_comma_in_multiple_local_declarations=insert
+org.eclipse.jdt.core.formatter.insert_space_after_comma_in_parameterized_type_reference=insert
+org.eclipse.jdt.core.formatter.insert_space_after_comma_in_superinterfaces=insert
+org.eclipse.jdt.core.formatter.insert_space_after_comma_in_type_arguments=insert
+org.eclipse.jdt.core.formatter.insert_space_after_comma_in_type_parameters=insert
+org.eclipse.jdt.core.formatter.insert_space_after_ellipsis=insert
+org.eclipse.jdt.core.formatter.insert_space_after_opening_angle_bracket_in_parameterized_type_reference=do not insert
+org.eclipse.jdt.core.formatter.insert_space_after_opening_angle_bracket_in_type_arguments=do not insert
+org.eclipse.jdt.core.formatter.insert_space_after_opening_angle_bracket_in_type_parameters=do not insert
+org.eclipse.jdt.core.formatter.insert_space_after_opening_brace_in_array_initializer=insert
+org.eclipse.jdt.core.formatter.insert_space_after_opening_bracket_in_array_allocation_expression=do not insert
+org.eclipse.jdt.core.formatter.insert_space_after_opening_bracket_in_array_reference=do not insert
+org.eclipse.jdt.core.formatter.insert_space_after_opening_paren_in_annotation=do not insert
+org.eclipse.jdt.core.formatter.insert_space_after_opening_paren_in_cast=do not insert
+org.eclipse.jdt.core.formatter.insert_space_after_opening_paren_in_catch=do not insert
+org.eclipse.jdt.core.formatter.insert_space_after_opening_paren_in_constructor_declaration=do not insert
+org.eclipse.jdt.core.formatter.insert_space_after_opening_paren_in_enum_constant=do not insert
+org.eclipse.jdt.core.formatter.insert_space_after_opening_paren_in_for=do not insert
+org.eclipse.jdt.core.formatter.insert_space_after_opening_paren_in_if=do not insert
+org.eclipse.jdt.core.formatter.insert_space_after_opening_paren_in_method_declaration=do not insert
+org.eclipse.jdt.core.formatter.insert_space_after_opening_paren_in_method_invocation=do not insert
+org.eclipse.jdt.core.formatter.insert_space_after_opening_paren_in_parenthesized_expression=do not insert
+org.eclipse.jdt.core.formatter.insert_space_after_opening_paren_in_switch=do not insert
+org.eclipse.jdt.core.formatter.insert_space_after_opening_paren_in_synchronized=do not insert
+org.eclipse.jdt.core.formatter.insert_space_after_opening_paren_in_try=do not insert
+org.eclipse.jdt.core.formatter.insert_space_after_opening_paren_in_while=do not insert
+org.eclipse.jdt.core.formatter.insert_space_after_postfix_operator=do not insert
+org.eclipse.jdt.core.formatter.insert_space_after_prefix_operator=do not insert
+org.eclipse.jdt.core.formatter.insert_space_after_question_in_conditional=insert
+org.eclipse.jdt.core.formatter.insert_space_after_question_in_wildcard=do not insert
+org.eclipse.jdt.core.formatter.insert_space_after_semicolon_in_for=insert
+org.eclipse.jdt.core.formatter.insert_space_after_semicolon_in_try_resources=insert
+org.eclipse.jdt.core.formatter.insert_space_after_unary_operator=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_and_in_type_parameter=insert
+org.eclipse.jdt.core.formatter.insert_space_before_assignment_operator=insert
+org.eclipse.jdt.core.formatter.insert_space_before_at_in_annotation_type_declaration=insert
+org.eclipse.jdt.core.formatter.insert_space_before_binary_operator=insert
+org.eclipse.jdt.core.formatter.insert_space_before_closing_angle_bracket_in_parameterized_type_reference=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_closing_angle_bracket_in_type_arguments=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_closing_angle_bracket_in_type_parameters=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_closing_brace_in_array_initializer=insert
+org.eclipse.jdt.core.formatter.insert_space_before_closing_bracket_in_array_allocation_expression=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_closing_bracket_in_array_reference=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_closing_paren_in_annotation=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_closing_paren_in_cast=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_closing_paren_in_catch=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_closing_paren_in_constructor_declaration=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_closing_paren_in_enum_constant=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_closing_paren_in_for=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_closing_paren_in_if=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_closing_paren_in_method_declaration=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_closing_paren_in_method_invocation=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_closing_paren_in_parenthesized_expression=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_closing_paren_in_switch=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_closing_paren_in_synchronized=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_closing_paren_in_try=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_closing_paren_in_while=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_colon_in_assert=insert
+org.eclipse.jdt.core.formatter.insert_space_before_colon_in_case=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_colon_in_conditional=insert
+org.eclipse.jdt.core.formatter.insert_space_before_colon_in_default=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_colon_in_for=insert
+org.eclipse.jdt.core.formatter.insert_space_before_colon_in_labeled_statement=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_comma_in_allocation_expression=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_comma_in_annotation=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_comma_in_array_initializer=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_comma_in_constructor_declaration_parameters=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_comma_in_constructor_declaration_throws=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_comma_in_enum_constant_arguments=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_comma_in_enum_declarations=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_comma_in_explicitconstructorcall_arguments=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_comma_in_for_increments=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_comma_in_for_inits=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_comma_in_method_declaration_parameters=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_comma_in_method_declaration_throws=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_comma_in_method_invocation_arguments=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_comma_in_multiple_field_declarations=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_comma_in_multiple_local_declarations=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_comma_in_parameterized_type_reference=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_comma_in_superinterfaces=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_comma_in_type_arguments=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_comma_in_type_parameters=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_ellipsis=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_opening_angle_bracket_in_parameterized_type_reference=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_opening_angle_bracket_in_type_arguments=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_opening_angle_bracket_in_type_parameters=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_opening_brace_in_annotation_type_declaration=insert
+org.eclipse.jdt.core.formatter.insert_space_before_opening_brace_in_anonymous_type_declaration=insert
+org.eclipse.jdt.core.formatter.insert_space_before_opening_brace_in_array_initializer=insert
+org.eclipse.jdt.core.formatter.insert_space_before_opening_brace_in_block=insert
+org.eclipse.jdt.core.formatter.insert_space_before_opening_brace_in_constructor_declaration=insert
+org.eclipse.jdt.core.formatter.insert_space_before_opening_brace_in_enum_constant=insert
+org.eclipse.jdt.core.formatter.insert_space_before_opening_brace_in_enum_declaration=insert
+org.eclipse.jdt.core.formatter.insert_space_before_opening_brace_in_method_declaration=insert
+org.eclipse.jdt.core.formatter.insert_space_before_opening_brace_in_switch=insert
+org.eclipse.jdt.core.formatter.insert_space_before_opening_brace_in_type_declaration=insert
+org.eclipse.jdt.core.formatter.insert_space_before_opening_bracket_in_array_allocation_expression=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_opening_bracket_in_array_reference=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_opening_bracket_in_array_type_reference=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_opening_paren_in_annotation=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_opening_paren_in_annotation_type_member_declaration=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_opening_paren_in_catch=insert
+org.eclipse.jdt.core.formatter.insert_space_before_opening_paren_in_constructor_declaration=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_opening_paren_in_enum_constant=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_opening_paren_in_for=insert
+org.eclipse.jdt.core.formatter.insert_space_before_opening_paren_in_if=insert
+org.eclipse.jdt.core.formatter.insert_space_before_opening_paren_in_method_declaration=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_opening_paren_in_method_invocation=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_opening_paren_in_parenthesized_expression=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_opening_paren_in_switch=insert
+org.eclipse.jdt.core.formatter.insert_space_before_opening_paren_in_synchronized=insert
+org.eclipse.jdt.core.formatter.insert_space_before_opening_paren_in_try=insert
+org.eclipse.jdt.core.formatter.insert_space_before_opening_paren_in_while=insert
+org.eclipse.jdt.core.formatter.insert_space_before_parenthesized_expression_in_return=insert
+org.eclipse.jdt.core.formatter.insert_space_before_parenthesized_expression_in_throw=insert
+org.eclipse.jdt.core.formatter.insert_space_before_postfix_operator=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_prefix_operator=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_question_in_conditional=insert
+org.eclipse.jdt.core.formatter.insert_space_before_question_in_wildcard=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_semicolon=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_semicolon_in_for=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_semicolon_in_try_resources=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_unary_operator=do not insert
+org.eclipse.jdt.core.formatter.insert_space_between_brackets_in_array_type_reference=do not insert
+org.eclipse.jdt.core.formatter.insert_space_between_empty_braces_in_array_initializer=do not insert
+org.eclipse.jdt.core.formatter.insert_space_between_empty_brackets_in_array_allocation_expression=do not insert
+org.eclipse.jdt.core.formatter.insert_space_between_empty_parens_in_annotation_type_member_declaration=do not insert
+org.eclipse.jdt.core.formatter.insert_space_between_empty_parens_in_constructor_declaration=do not insert
+org.eclipse.jdt.core.formatter.insert_space_between_empty_parens_in_enum_constant=do not insert
+org.eclipse.jdt.core.formatter.insert_space_between_empty_parens_in_method_declaration=do not insert
+org.eclipse.jdt.core.formatter.insert_space_between_empty_parens_in_method_invocation=do not insert
+org.eclipse.jdt.core.formatter.join_lines_in_comments=true
+org.eclipse.jdt.core.formatter.join_wrapped_lines=true
+org.eclipse.jdt.core.formatter.keep_else_statement_on_same_line=false
+org.eclipse.jdt.core.formatter.keep_empty_array_initializer_on_one_line=false
+org.eclipse.jdt.core.formatter.keep_imple_if_on_one_line=false
+org.eclipse.jdt.core.formatter.keep_then_statement_on_same_line=false
+org.eclipse.jdt.core.formatter.lineSplit=999
+org.eclipse.jdt.core.formatter.never_indent_block_comments_on_first_column=false
+org.eclipse.jdt.core.formatter.never_indent_line_comments_on_first_column=false
+org.eclipse.jdt.core.formatter.number_of_blank_lines_at_beginning_of_method_body=0
+org.eclipse.jdt.core.formatter.number_of_empty_lines_to_preserve=1
+org.eclipse.jdt.core.formatter.put_empty_statement_on_new_line=true
+org.eclipse.jdt.core.formatter.tabulation.char=space
+org.eclipse.jdt.core.formatter.tabulation.size=4
+org.eclipse.jdt.core.formatter.use_on_off_tags=false
+org.eclipse.jdt.core.formatter.use_tabs_only_for_leading_indentations=false
+org.eclipse.jdt.core.formatter.wrap_before_binary_operator=true
+org.eclipse.jdt.core.formatter.wrap_before_or_operator_multicatch=true
+org.eclipse.jdt.core.formatter.wrap_outer_expressions_when_nested=true

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/job/.settings/org.eclipse.jdt.ui.prefs
----------------------------------------------------------------------
diff --git a/job/.settings/org.eclipse.jdt.ui.prefs b/job/.settings/org.eclipse.jdt.ui.prefs
new file mode 100644
index 0000000..dece0e6
--- /dev/null
+++ b/job/.settings/org.eclipse.jdt.ui.prefs
@@ -0,0 +1,7 @@
+eclipse.preferences.version=1
+formatter_profile=_Space Indent & Long Lines
+formatter_settings_version=12
+org.eclipse.jdt.ui.ignorelowercasenames=true
+org.eclipse.jdt.ui.importorder=java;javax;org;com;
+org.eclipse.jdt.ui.ondemandthreshold=99
+org.eclipse.jdt.ui.staticondemandthreshold=1

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/job/pom.xml
----------------------------------------------------------------------
diff --git a/job/pom.xml b/job/pom.xml
new file mode 100644
index 0000000..b039957
--- /dev/null
+++ b/job/pom.xml
@@ -0,0 +1,196 @@
+<?xml version="1.0"?>
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+    <modelVersion>4.0.0</modelVersion>
+    <parent>
+        <groupId>com.kylinolap</groupId>
+        <artifactId>kylin</artifactId>
+        <version>0.6.3-SNAPSHOT</version>
+    </parent>
+
+    <artifactId>kylin-job</artifactId>
+    <name>Kylin:Job</name>
+    <url>http://maven.apache.org</url>
+    <properties>
+        <project.build.sourceEncoding>UTF-8</project.build.sourceEncoding>
+    </properties>
+
+    <dependencies>
+        <!--Kylin Jar -->
+        <dependency>
+            <groupId>com.kylinolap</groupId>
+            <artifactId>kylin-cube</artifactId>
+            <version>${project.parent.version}</version>
+        </dependency>
+
+        <dependency>
+            <groupId>commons-cli</groupId>
+            <artifactId>commons-cli</artifactId>
+        </dependency>
+        <dependency>
+            <groupId>commons-lang</groupId>
+            <artifactId>commons-lang</artifactId>
+        </dependency>
+        <dependency>
+            <groupId>commons-io</groupId>
+            <artifactId>commons-io</artifactId>
+        </dependency>
+        <dependency>
+            <groupId>commons-configuration</groupId>
+            <artifactId>commons-configuration</artifactId>
+        </dependency>
+        <dependency>
+            <groupId>log4j</groupId>
+            <artifactId>log4j</artifactId>
+        </dependency>
+        <dependency>
+            <groupId>com.fasterxml.jackson.core</groupId>
+            <artifactId>jackson-databind</artifactId>
+        </dependency>
+        <dependency>
+            <groupId>commons-httpclient</groupId>
+            <artifactId>commons-httpclient</artifactId>
+        </dependency>
+        <dependency>
+            <groupId>com.google.guava</groupId>
+            <artifactId>guava</artifactId>
+        </dependency>
+
+        <dependency>
+            <groupId>org.quartz-scheduler</groupId>
+            <artifactId>quartz</artifactId>
+        </dependency>
+        <dependency>
+            <groupId>org.quartz-scheduler</groupId>
+            <artifactId>quartz-jobs</artifactId>
+        </dependency>
+
+        <dependency>
+            <groupId>commons-daemon</groupId>
+            <artifactId>commons-daemon</artifactId>
+        </dependency>
+
+        <dependency>
+            <groupId>org.apache.curator</groupId>
+            <artifactId>curator-framework</artifactId>
+            <exclusions>
+                <exclusion>
+                    <groupId>org.apache.zookeeper</groupId>
+                    <artifactId>zookeeper</artifactId>
+                </exclusion>
+            </exclusions>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.curator</groupId>
+            <artifactId>curator-recipes</artifactId>
+            <exclusions>
+                <exclusion>
+                    <groupId>org.apache.zookeeper</groupId>
+                    <artifactId>zookeeper</artifactId>
+                </exclusion>
+            </exclusions>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.commons</groupId>
+            <artifactId>commons-email</artifactId>
+            <version>1.1</version>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.hbase</groupId>
+            <artifactId>hbase-common</artifactId>
+            <!-- This is default scope to be included in the job jar, we depends on
+                hbase utils like Bytes, ImmutableBytesWritable etc. -->
+        </dependency>
+
+        <!-- Env & Test -->
+
+        <dependency>
+            <groupId>org.apache.hadoop</groupId>
+            <artifactId>hadoop-common</artifactId>
+            <scope>provided</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.hadoop</groupId>
+            <artifactId>hadoop-annotations</artifactId>
+            <scope>provided</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.hadoop</groupId>
+            <artifactId>hadoop-mapreduce-client-core</artifactId>
+            <scope>provided</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.hadoop</groupId>
+            <artifactId>hadoop-minicluster</artifactId>
+            <scope>test</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.mrunit</groupId>
+            <artifactId>mrunit</artifactId>
+            <classifier>hadoop2</classifier>
+            <scope>test</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.hbase</groupId>
+            <artifactId>hbase-hadoop2-compat</artifactId>
+            <scope>provided</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.hbase</groupId>
+            <artifactId>hbase-client</artifactId>
+            <scope>provided</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.hbase</groupId>
+            <artifactId>hbase-server</artifactId>
+            <scope>provided</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.hadoop</groupId>
+            <artifactId>hadoop-yarn-server-resourcemanager</artifactId>
+            <scope>provided</scope>
+        </dependency>
+        <dependency>
+            <groupId>junit</groupId>
+            <artifactId>junit</artifactId>
+            <scope>test</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.maven</groupId>
+            <artifactId>maven-model</artifactId>
+            <scope>test</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.hadoop</groupId>
+            <artifactId>hadoop-yarn-api</artifactId>
+            <scope>provided</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.hadoop</groupId>
+            <artifactId>hadoop-hdfs</artifactId>
+            <scope>provided</scope>
+        </dependency>
+
+    </dependencies>
+
+    <build>
+        <plugins>
+            <plugin>
+                <artifactId>maven-assembly-plugin</artifactId>
+                <executions>
+                    <execution>
+                        <id>create-job-jar</id>
+                        <phase>package</phase>
+                        <goals>
+                            <goal>single</goal>
+                        </goals>
+                        <configuration>
+                            <descriptors>
+                                <descriptor>src/main/assembly/job.xml</descriptor>
+                            </descriptors>
+                        </configuration>
+                    </execution>
+                </executions>
+            </plugin>
+        </plugins>
+    </build>
+</project>

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/job/src/main/assembly/job.xml
----------------------------------------------------------------------
diff --git a/job/src/main/assembly/job.xml b/job/src/main/assembly/job.xml
new file mode 100644
index 0000000..d0a56ee
--- /dev/null
+++ b/job/src/main/assembly/job.xml
@@ -0,0 +1,22 @@
+<assembly>
+	<id>job</id>
+	<formats>
+		<format>jar</format>
+	</formats>
+	<includeBaseDirectory>false</includeBaseDirectory>
+	<dependencySets>
+		<dependencySet>
+			<unpack>true</unpack>
+			<scope>runtime</scope>
+			<excludes>
+				<exclude>${groupId}:${artifactId}</exclude>
+			</excludes>
+		</dependencySet>
+		<dependencySet>
+			<unpack>true</unpack>
+			<includes>
+				<include>${groupId}:${artifactId}</include>
+			</includes>
+		</dependencySet>
+	</dependencySets>
+</assembly>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/job/src/main/java/com/kylinolap/job/JobDAO.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/com/kylinolap/job/JobDAO.java b/job/src/main/java/com/kylinolap/job/JobDAO.java
new file mode 100644
index 0000000..b9fb688
--- /dev/null
+++ b/job/src/main/java/com/kylinolap/job/JobDAO.java
@@ -0,0 +1,240 @@
+/*
+ * 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.job;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.concurrent.ConcurrentHashMap;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.kylinolap.common.KylinConfig;
+import com.kylinolap.common.persistence.JsonSerializer;
+import com.kylinolap.common.persistence.ResourceStore;
+import com.kylinolap.common.persistence.Serializer;
+import com.kylinolap.job.JobInstance.JobStep;
+import com.kylinolap.job.constant.JobStatusEnum;
+import com.kylinolap.job.constant.JobStepStatusEnum;
+import com.kylinolap.metadata.MetadataManager;
+
+/**
+ * @author ysong1
+ */
+public class JobDAO {
+    private static Logger log = LoggerFactory.getLogger(JobDAO.class);
+
+    private static final Serializer<JobInstance> JOB_SERIALIZER = new JsonSerializer<JobInstance>(JobInstance.class);
+    private static final Serializer<JobStepOutput> JOB_OUTPUT_SERIALIZER = new JsonSerializer<JobStepOutput>(JobStepOutput.class);
+
+    private ResourceStore store;
+
+    private static final Logger logger = LoggerFactory.getLogger(JobDAO.class);
+
+    private static final ConcurrentHashMap<KylinConfig, JobDAO> CACHE = new ConcurrentHashMap<KylinConfig, JobDAO>();
+
+    public static JobDAO getInstance(KylinConfig config) {
+        JobDAO r = CACHE.get(config);
+        if (r == null) {
+            r = new JobDAO(config);
+            CACHE.put(config, r);
+            if (CACHE.size() > 1) {
+                logger.warn("More than one singleton exist");
+            }
+
+        }
+        return r;
+    }
+
+    private JobDAO(KylinConfig config) {
+        log.info("Using metadata url: " + config);
+        this.store = MetadataManager.getInstance(config).getStore();
+    }
+
+    public List<JobInstance> listAllJobs() throws IOException {
+        ArrayList<String> jobResources = store.listResources(ResourceStore.JOB_PATH_ROOT);
+        if (jobResources == null)
+            return Collections.emptyList();
+
+        ArrayList<JobInstance> result = new ArrayList<JobInstance>(jobResources.size());
+        for (String path : jobResources) {
+            JobInstance job = readJobResource(path);
+            if (job != null) {
+                result.add(job);
+            }
+        }
+
+        return result;
+    }
+
+    public List<JobInstance> listAllJobs(String cubeName) throws IOException {
+
+        List<JobInstance> allJobs = listAllJobs();
+        if (allJobs.size() == 0) {
+            return Collections.emptyList();
+        }
+
+        if (null == cubeName || cubeName.trim().length() == 0) {
+            return allJobs;
+        }
+
+        ArrayList<JobInstance> result = new ArrayList<JobInstance>();
+        for (JobInstance job : allJobs) {
+            if (job != null) {
+                if (job.getRelatedCube().toLowerCase().contains(cubeName.toLowerCase())) {
+                    result.add(job);
+                }
+            }
+        }
+
+        return result;
+    }
+
+    public List<JobInstance> listAllJobs(JobStatusEnum status) throws IOException {
+
+        List<JobInstance> allJobs = listAllJobs();
+        if (allJobs.size() == 0) {
+            return Collections.emptyList();
+        }
+
+        ArrayList<JobInstance> result = new ArrayList<JobInstance>();
+        for (JobInstance job : allJobs) {
+            if (job != null) {
+                if (job.getStatus().equals(status)) {
+                    result.add(job);
+                }
+            }
+        }
+
+        return result;
+    }
+
+    public JobStepOutput getJobOutput(String jobUuid, int stepSequenceId) throws IOException {
+        return readJobOutputResource(ResourceStore.JOB_OUTPUT_PATH_ROOT + "/" + JobStepOutput.nameOfOutput(jobUuid, stepSequenceId));
+    }
+
+    public JobStepOutput getJobOutput(JobStep jobStep) throws IOException {
+        return getJobOutput(jobStep.getJobInstance().getUuid(), jobStep.getSequenceID());
+    }
+
+    public void saveJobOutput(String jobUuid, int stepSequenceId, String outputString) throws IOException {
+        JobStepOutput output = this.getJobOutput(jobUuid, stepSequenceId);
+
+        if (output == null) {
+            output = new JobStepOutput();
+            output.setName(JobStepOutput.nameOfOutput(jobUuid, stepSequenceId));
+        }
+
+        output.setOutput(outputString);
+        writeJobOutputResource(pathOfJobOutput(output), output);
+    }
+
+    public void saveJobOutput(JobStep jobStep, String outputString) throws IOException {
+        saveJobOutput(jobStep.getJobInstance().getUuid(), jobStep.getSequenceID(), outputString);
+    }
+
+    private void saveJob(JobInstance job) throws IOException {
+        writeJobResource(pathOfJob(job), job);
+    }
+
+    public JobInstance getJob(String uuid) throws IOException {
+        return readJobResource(ResourceStore.JOB_PATH_ROOT + "/" + uuid);
+    }
+
+    public void deleteJob(JobInstance job) throws IOException {
+        store.deleteResource(pathOfJob(job));
+    }
+
+    public void deleteJob(String uuid) throws IOException {
+        store.deleteResource(ResourceStore.JOB_PATH_ROOT + "/" + uuid);
+    }
+
+    public void updateJobInstance(JobInstance jobInstance) throws IOException {
+        try {
+            JobInstance updatedJob = getJob(jobInstance.getUuid());
+            if (updatedJob == null) {
+                saveJob(jobInstance);
+                return;
+            }
+
+            updatedJob.setExecEndTime(jobInstance.getExecEndTime());
+            updatedJob.setExecStartTime(jobInstance.getExecStartTime());
+            updatedJob.setDuration(jobInstance.getDuration());
+            updatedJob.setMrWaiting(jobInstance.getMrWaiting());
+            updatedJob.setRelatedCube(jobInstance.getRelatedCube());
+            updatedJob.setRelatedSegment(jobInstance.getRelatedSegment());
+            updatedJob.setType(jobInstance.getType());
+
+            updatedJob.clearSteps();
+            updatedJob.addSteps(jobInstance.getSteps());
+
+            saveJob(updatedJob);
+        } catch (IOException e) {
+            log.error(e.getLocalizedMessage(), e);
+            throw e;
+        }
+    }
+
+    public void updateRunningJobToError() throws IOException {
+        List<JobInstance> runningJobs = listAllJobs(JobStatusEnum.RUNNING);
+        for (JobInstance job : runningJobs) {
+            // job.setStatus(JobStatusEnum.ERROR);
+
+            // set the last running step to ERROR
+            int lastRunningStepIndex = 0;
+            for (int i = job.getSteps().size() - 1; i >= 0; i--) {
+                JobStep currentStep = job.getSteps().get(i);
+                if (currentStep.getStatus() != JobStepStatusEnum.RUNNING && currentStep.getStatus() != JobStepStatusEnum.WAITING) {
+                    continue;
+                } else {
+                    lastRunningStepIndex = i;
+                    break;
+                }
+            }
+
+            job.getSteps().get(lastRunningStepIndex).setStatus(JobStepStatusEnum.ERROR);
+            this.updateJobInstance(job);
+
+            this.saveJobOutput(job.getUuid(), lastRunningStepIndex, "ERROR state set by job engine");
+        }
+    }
+
+    private String pathOfJob(JobInstance job) {
+        return ResourceStore.JOB_PATH_ROOT + "/" + job.getUuid();
+    }
+
+    private JobInstance readJobResource(String path) throws IOException {
+        return store.getResource(path, JobInstance.class, JOB_SERIALIZER);
+    }
+
+    private void writeJobResource(String path, JobInstance job) throws IOException {
+        store.putResource(path, job, JOB_SERIALIZER);
+    }
+
+    private String pathOfJobOutput(JobStepOutput output) {
+        return ResourceStore.JOB_OUTPUT_PATH_ROOT + "/" + output.getName();
+    }
+
+    private JobStepOutput readJobOutputResource(String path) throws IOException {
+        return store.getResource(path, JobStepOutput.class, JOB_OUTPUT_SERIALIZER);
+    }
+
+    private void writeJobOutputResource(String path, JobStepOutput output) throws IOException {
+        store.putResource(path, output, JOB_OUTPUT_SERIALIZER);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/job/src/main/java/com/kylinolap/job/JobInstance.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/com/kylinolap/job/JobInstance.java b/job/src/main/java/com/kylinolap/job/JobInstance.java
new file mode 100644
index 0000000..0a7308a
--- /dev/null
+++ b/job/src/main/java/com/kylinolap/job/JobInstance.java
@@ -0,0 +1,479 @@
+/*
+ * 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.job;
+
+import java.util.Collection;
+import java.util.List;
+import java.util.concurrent.ConcurrentHashMap;
+
+import com.fasterxml.jackson.annotation.JsonAutoDetect;
+import com.fasterxml.jackson.annotation.JsonAutoDetect.Visibility;
+import com.fasterxml.jackson.annotation.JsonBackReference;
+import com.fasterxml.jackson.annotation.JsonIgnoreProperties;
+import com.fasterxml.jackson.annotation.JsonManagedReference;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.google.common.collect.Lists;
+import com.kylinolap.common.persistence.RootPersistentEntity;
+import com.kylinolap.cube.CubeBuildTypeEnum;
+import com.kylinolap.job.constant.JobStatusEnum;
+import com.kylinolap.job.constant.JobStepCmdTypeEnum;
+import com.kylinolap.job.constant.JobStepStatusEnum;
+import com.kylinolap.job.engine.JobEngineConfig;
+
+@JsonAutoDetect(fieldVisibility = Visibility.NONE, getterVisibility = Visibility.NONE, isGetterVisibility = Visibility.NONE, setterVisibility = Visibility.NONE)
+public class JobInstance extends RootPersistentEntity implements Comparable<JobInstance>{
+
+    public static final String JOB_WORKING_DIR_PREFIX = "kylin-";
+
+    public static final String YARN_APP_ID = "yarn_application_id";
+    public static final String YARN_APP_URL = "yarn_application_tracking_url";
+    public static final String MR_JOB_ID = "mr_job_id";
+    public static final String HDFS_BYTES_WRITTEN = "hdfs_bytes_written";
+    public static final String SOURCE_RECORDS_COUNT = "source_records_count";
+    public static final String SOURCE_RECORDS_SIZE = "source_records_size";
+
+    public static String getStepIdentity(JobInstance jobInstance, JobStep jobStep) {
+        return jobInstance.getRelatedCube() + "." + jobInstance.getUuid() + "." + jobStep.getSequenceID();
+    }
+
+    public static String getJobIdentity(JobInstance jobInstance) {
+        return jobInstance.getRelatedCube() + "." + jobInstance.getUuid();
+    }
+
+    public static String getJobWorkingDir(JobInstance jobInstance, JobEngineConfig engineConfig) {
+        return getJobWorkingDir(jobInstance.getUuid(), engineConfig.getHdfsWorkingDirectory());
+    }
+
+    public static String getJobWorkingDir(String jobUuid, String hdfsWorkdingDir) {
+        if (jobUuid == null || jobUuid.equals("")) {
+            throw new IllegalArgumentException("jobUuid can't be null or empty");
+        }
+        return hdfsWorkdingDir + "/" + JOB_WORKING_DIR_PREFIX + jobUuid;
+    }
+
+    @JsonProperty("name")
+    private String name;
+    @JsonProperty("type")
+    private CubeBuildTypeEnum type; // java implementation
+    @JsonProperty("duration")
+    private long duration;
+    @JsonProperty("related_cube")
+    private String relatedCube;
+    @JsonProperty("related_segment")
+    private String relatedSegment;
+    @JsonProperty("exec_start_time")
+    private long execStartTime;
+    @JsonProperty("exec_end_time")
+    private long execEndTime;
+    @JsonProperty("mr_waiting")
+    private long mrWaiting = 0;
+    @JsonManagedReference
+    @JsonProperty("steps")
+    private List<JobStep> steps;
+    @JsonProperty("submitter")
+    private String submitter;
+    
+    public JobStep getRunningStep() {
+        for (JobStep step : this.getSteps()) {
+            if (step.getStatus().equals(JobStepStatusEnum.RUNNING) || step.getStatus().equals(JobStepStatusEnum.WAITING)) {
+                return step;
+            }
+        }
+
+        return null;
+    }
+
+    @JsonProperty("progress")
+    public double getProgress() {
+        int completedStepCount = 0;
+        for (JobStep step : this.getSteps()) {
+            if (step.getStatus().equals(JobStepStatusEnum.FINISHED)) {
+                completedStepCount++;
+            }
+        }
+
+        return 100.0 * completedStepCount / steps.size();
+    }
+
+    @JsonProperty("job_status")
+    public JobStatusEnum getStatus() {
+        // JobStatusEnum finalJobStatus;
+        int compositResult = 0;
+
+        // if steps status are all NEW, then job status is NEW
+        // if steps status are all FINISHED, then job status is FINISHED
+        // if steps status are all PENDING, then job status is PENDING
+        // if steps status are FINISHED and PENDING, the job status is PENDING
+        // if one of steps status is RUNNING, then job status is RUNNING
+        // if one of steps status is ERROR, then job status is ERROR
+        // if one of steps status is KILLED, then job status is KILLED
+        // default status is RUNNING
+
+        for (JobStep step : this.getSteps()) {
+            compositResult = compositResult | step.getStatus().getCode();
+        }
+
+        if (compositResult == JobStatusEnum.FINISHED.getCode()) {
+            return JobStatusEnum.FINISHED;
+        } else if (compositResult == JobStatusEnum.NEW.getCode()) {
+            return JobStatusEnum.NEW;
+        } else if (compositResult == JobStatusEnum.PENDING.getCode()) {
+            return JobStatusEnum.PENDING;
+        } else if (compositResult == (JobStatusEnum.FINISHED.getCode() | JobStatusEnum.PENDING.getCode())) {
+            return JobStatusEnum.PENDING;
+        } else if ((compositResult & JobStatusEnum.ERROR.getCode()) == JobStatusEnum.ERROR.getCode()) {
+            return JobStatusEnum.ERROR;
+        } else if ((compositResult & JobStatusEnum.DISCARDED.getCode()) == JobStatusEnum.DISCARDED.getCode()) {
+            return JobStatusEnum.DISCARDED;
+        } else if ((compositResult & JobStatusEnum.RUNNING.getCode()) == JobStatusEnum.RUNNING.getCode()) {
+            return JobStatusEnum.RUNNING;
+        }
+
+        return JobStatusEnum.RUNNING;
+    }
+
+    public String getName() {
+        return name;
+    }
+
+    public void setName(String name) {
+        this.name = name;
+    }
+
+    public CubeBuildTypeEnum getType() {
+        return type;
+    }
+
+    public void setType(CubeBuildTypeEnum type) {
+        this.type = type;
+    }
+
+    public long getDuration() {
+        return duration;
+    }
+
+    public void setDuration(long duration) {
+        this.duration = duration;
+    }
+
+    public String getRelatedCube() {
+        return relatedCube;
+    }
+
+    public void setRelatedCube(String relatedCube) {
+        this.relatedCube = relatedCube;
+    }
+
+    public String getRelatedSegment() {
+        return relatedSegment;
+    }
+
+    public void setRelatedSegment(String relatedSegment) {
+        this.relatedSegment = relatedSegment;
+    }
+
+    /**
+     * @return the execStartTime
+     */
+    public long getExecStartTime() {
+        return execStartTime;
+    }
+
+    /**
+     * @param execStartTime
+     *            the execStartTime to set
+     */
+    public void setExecStartTime(long execStartTime) {
+        this.execStartTime = execStartTime;
+    }
+
+    /**
+     * @return the execEndTime
+     */
+    public long getExecEndTime() {
+        return execEndTime;
+    }
+
+    /**
+     * @param execEndTime
+     *            the execEndTime to set
+     */
+    public void setExecEndTime(long execEndTime) {
+        this.execEndTime = execEndTime;
+    }
+
+    public long getMrWaiting() {
+        return this.mrWaiting;
+    }
+
+    public void setMrWaiting(long mrWaiting) {
+        this.mrWaiting = mrWaiting;
+    }
+
+    public List<JobStep> getSteps() {
+        if (steps == null) {
+            steps = Lists.newArrayList();
+        }
+        return steps;
+    }
+
+    public void clearSteps() {
+        getSteps().clear();
+    }
+
+    public void addSteps(Collection<JobStep> steps) {
+        this.getSteps().addAll(steps);
+    }
+
+    public void addStep(JobStep step) {
+        getSteps().add(step);
+    }
+
+    public void addStep(int index, JobStep step) {
+        getSteps().add(index, step);
+    }
+
+    public JobStep findStep(String stepName) {
+        for (JobStep step : getSteps()) {
+            if (stepName.equals(step.getName())) {
+                return step;
+            }
+        }
+        return null;
+    }
+
+        
+    public String getSubmitter() {
+        return submitter;
+    }
+
+    public void setSubmitter(String submitter) {
+        this.submitter = submitter;
+    }
+
+
+
+
+    @JsonIgnoreProperties(ignoreUnknown = true)
+    public static class JobStep implements Comparable<JobStep> {
+
+        @JsonBackReference
+        private JobInstance jobInstance;
+
+        @JsonProperty("name")
+        private String name;
+
+        @JsonProperty("sequence_id")
+        private int sequenceID;
+
+        @JsonProperty("exec_cmd")
+        private String execCmd;
+
+        @JsonProperty("interrupt_cmd")
+        private String InterruptCmd;
+
+        @JsonProperty("exec_start_time")
+        private long execStartTime;
+        @JsonProperty("exec_end_time")
+        private long execEndTime;
+        @JsonProperty("exec_wait_time")
+        private long execWaitTime;
+
+        @JsonProperty("step_status")
+        private JobStepStatusEnum status;
+
+        @JsonProperty("cmd_type")
+        private JobStepCmdTypeEnum cmdType = JobStepCmdTypeEnum.SHELL_CMD_HADOOP;
+
+        @JsonProperty("info")
+        private ConcurrentHashMap<String, String> info;
+
+        @JsonProperty("run_async")
+        private boolean runAsync = false;
+
+        private ConcurrentHashMap<String, String> getInfo() {
+            if (info == null) {
+                info = new ConcurrentHashMap<String, String>();
+            }
+            return info;
+        }
+
+        public void putInfo(String key, String value) {
+            getInfo().put(key, value);
+        }
+
+        public String getInfo(String key) {
+            return getInfo().get(key);
+        }
+
+        public void clearInfo() {
+            getInfo().clear();
+        }
+
+        public String getName() {
+            return name;
+        }
+
+        public void setName(String name) {
+            this.name = name;
+        }
+
+        public int getSequenceID() {
+            return sequenceID;
+        }
+
+        public void setSequenceID(int sequenceID) {
+            this.sequenceID = sequenceID;
+        }
+
+        public String getExecCmd() {
+            return execCmd;
+        }
+
+        public void setExecCmd(String execCmd) {
+            this.execCmd = execCmd;
+        }
+
+        public JobStepStatusEnum getStatus() {
+            return status;
+        }
+
+        public void setStatus(JobStepStatusEnum status) {
+            this.status = status;
+        }
+
+        /**
+         * @return the execStartTime
+         */
+        public long getExecStartTime() {
+            return execStartTime;
+        }
+
+        /**
+         * @param execStartTime
+         *            the execStartTime to set
+         */
+        public void setExecStartTime(long execStartTime) {
+            this.execStartTime = execStartTime;
+        }
+
+        /**
+         * @return the execEndTime
+         */
+        public long getExecEndTime() {
+            return execEndTime;
+        }
+
+        /**
+         * @param execEndTime
+         *            the execEndTime to set
+         */
+        public void setExecEndTime(long execEndTime) {
+            this.execEndTime = execEndTime;
+        }
+
+        public long getExecWaitTime() {
+            return execWaitTime;
+        }
+
+        public void setExecWaitTime(long execWaitTime) {
+            this.execWaitTime = execWaitTime;
+        }
+
+        public String getInterruptCmd() {
+            return InterruptCmd;
+        }
+
+        public void setInterruptCmd(String interruptCmd) {
+            InterruptCmd = interruptCmd;
+        }
+
+        public JobStepCmdTypeEnum getCmdType() {
+            return cmdType;
+        }
+
+        public void setCmdType(JobStepCmdTypeEnum cmdType) {
+            this.cmdType = cmdType;
+        }
+
+        /**
+         * @return the runAsync
+         */
+        public boolean isRunAsync() {
+            return runAsync;
+        }
+
+        /**
+         * @param runAsync
+         *            the runAsync to set
+         */
+        public void setRunAsync(boolean runAsync) {
+            this.runAsync = runAsync;
+        }
+
+        /**
+         * @return the jobInstance
+         */
+        public JobInstance getJobInstance() {
+            return jobInstance;
+        }
+
+        @Override
+        public int hashCode() {
+            final int prime = 31;
+            int result = 1;
+            result = prime * result + ((name == null) ? 0 : name.hashCode());
+            result = prime * result + sequenceID;
+            return result;
+        }
+
+        @Override
+        public boolean equals(Object obj) {
+            if (this == obj)
+                return true;
+            if (obj == null)
+                return false;
+            if (getClass() != obj.getClass())
+                return false;
+            JobStep other = (JobStep) obj;
+            if (name == null) {
+                if (other.name != null)
+                    return false;
+            } else if (!name.equals(other.name))
+                return false;
+            if (sequenceID != other.sequenceID)
+                return false;
+            return true;
+        }
+
+        @Override
+        public int compareTo(JobStep o) {
+            if (this.sequenceID < o.sequenceID) {
+                return -1;
+            } else if (this.sequenceID > o.sequenceID) {
+                return 1;
+            } else {
+                return 0;
+            }
+        }
+    }
+
+    @Override
+    public int compareTo(JobInstance o) {
+        return (int) (o.lastModified - this.lastModified);
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/job/src/main/java/com/kylinolap/job/JobInstanceBuilder.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/com/kylinolap/job/JobInstanceBuilder.java b/job/src/main/java/com/kylinolap/job/JobInstanceBuilder.java
new file mode 100644
index 0000000..a17c0ff
--- /dev/null
+++ b/job/src/main/java/com/kylinolap/job/JobInstanceBuilder.java
@@ -0,0 +1,490 @@
+/*
+ * 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.job;
+
+import java.io.IOException;
+import java.util.List;
+
+import com.google.common.collect.Lists;
+import org.apache.commons.lang3.StringUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.kylinolap.common.util.JsonUtil;
+import com.kylinolap.cube.CubeInstance;
+import com.kylinolap.cube.CubeManager;
+import com.kylinolap.cube.CubeSegment;
+import com.kylinolap.cube.CubeSegmentStatusEnum;
+import com.kylinolap.dict.lookup.HiveTable;
+import com.kylinolap.job.JobInstance.JobStep;
+import com.kylinolap.job.constant.JobConstants;
+import com.kylinolap.job.constant.JobStepCmdTypeEnum;
+import com.kylinolap.job.constant.JobStepStatusEnum;
+import com.kylinolap.job.engine.JobEngineConfig;
+import com.kylinolap.job.hadoop.hive.JoinedFlatTableDesc;
+import com.kylinolap.metadata.MetadataManager;
+
+/**
+ * @author George Song (ysong1)
+ */
+public class JobInstanceBuilder {
+
+    private static Logger log = LoggerFactory.getLogger(JobInstanceBuilder.class);
+
+    private CubeInstance cube;
+    private String htablename;
+    private String cubeName;
+    private String segmentName;
+    private CubeSegment cubeSegment;
+    private String jobUUID;
+    private final JobEngineConfig engineConfig;
+
+    private String jobWorkingDir;
+
+    public JobInstanceBuilder(JobEngineConfig engineCfg) {
+        this.engineConfig = engineCfg;
+    }
+
+    public List<JobStep> buildSteps(JobInstance jobInstance) throws IOException {
+        init(jobInstance);
+        switch (jobInstance.getType()) {
+        case BUILD:
+            return createBuildCubeSegmentSteps(jobInstance);
+        case MERGE:
+            return createMergeCubeSegmentsSteps(jobInstance);
+        default:
+            throw new IllegalArgumentException("job type:" + jobInstance.getType() + " not supported");
+        }
+    }
+
+    private void init(JobInstance jobInstance) {
+        cubeName = jobInstance.getRelatedCube();
+        if (cubeName == null) {
+            throw new IllegalArgumentException("Cube name is null or empty!");
+        }
+        cube = CubeManager.getInstance(this.engineConfig.getConfig()).getCube(cubeName);
+        jobUUID = jobInstance.getUuid();
+        if (jobUUID == null || jobUUID.equals("")) {
+            throw new IllegalArgumentException("Job UUID is null or empty!");
+        }
+
+        segmentName = jobInstance.getRelatedSegment();
+        if (segmentName == null || segmentName.equals("")) {
+            throw new IllegalArgumentException("Cube segment name is null or empty!");
+        }
+
+        // only the segment which can be build
+        cubeSegment = cube.getSegment(segmentName, CubeSegmentStatusEnum.NEW);
+        htablename = cubeSegment.getStorageLocationIdentifier();
+
+        this.jobWorkingDir = JobInstance.getJobWorkingDir(jobInstance, engineConfig);
+    }
+
+    private String appendMapReduceParameters(String cmd, JobInstance jobInstance) throws IOException {
+        StringBuffer buf = new StringBuffer(cmd);
+        String jobConf = engineConfig.getHadoopJobConfFilePath(cube.getDescriptor().getCapacity());
+        if (StringUtils.isBlank(jobConf) == false) {
+            buf.append(" -conf " + jobConf);
+        }
+        
+        String extraArgs = engineConfig.getMapReduceCmdExtraArgs();
+        if (StringUtils.isBlank(extraArgs) == false) {
+            extraArgs = extraArgs.replace("${CUBE}", jobInstance.getRelatedCube());
+            extraArgs = extraArgs.replace("${TYPE}", jobInstance.getType().toString());
+            extraArgs = extraArgs.replace("${UUID}", jobInstance.getUuid());
+            buf.append(" ").append(extraArgs);
+        }
+
+        return buf.toString();
+    }
+
+    private String appendExecCmdParameters(String cmd, String paraName, String paraValue) {
+        StringBuffer buf = new StringBuffer(cmd);
+        buf.append(" -" + paraName + " " + paraValue);
+        return buf.toString();
+    }
+
+    private String getIntermediateHiveTablePath() {
+        JoinedFlatTableDesc intermediateTableDesc = new JoinedFlatTableDesc(cube.getDescriptor(), this.cubeSegment);
+        return JoinedFlatTable.getTableDir(intermediateTableDesc, jobWorkingDir, jobUUID);
+    }
+
+    private String[] getCuboidOutputPaths(String cuboidRootPath, int totalRowkeyColumnCount, int groupRowkeyColumnsCount) {
+        String[] paths = new String[groupRowkeyColumnsCount + 1];
+        for (int i = 0; i <= groupRowkeyColumnsCount; i++) {
+            int dimNum = totalRowkeyColumnCount - i;
+            if (dimNum == totalRowkeyColumnCount) {
+                paths[i] = cuboidRootPath + "base_cuboid";
+            } else {
+                paths[i] = cuboidRootPath + dimNum + "d_cuboid";
+            }
+        }
+        return paths;
+    }
+
+    private String getFactDistinctColumnsPath() {
+        return jobWorkingDir + "/" + cubeName + "/fact_distinct_columns";
+    }
+
+    private String getRowkeyDistributionOutputPath() {
+        return jobWorkingDir + "/" + cubeName + "/rowkey_stats";
+    }
+
+    private List<JobStep> createMergeCubeSegmentsSteps(JobInstance jobInstance) throws IOException {
+
+        List<CubeSegment> mergingSegments = cube.getMergingSegments();
+        if (mergingSegments == null || mergingSegments.size() < 2) {
+            throw new IllegalArgumentException("Merging segments count should be more than 2");
+        }
+
+
+        String[] cuboidPaths = new String[mergingSegments.size()];
+        for (int i = 0; i < mergingSegments.size(); i++) {
+            cuboidPaths[i] = getPathToMerge(jobInstance, mergingSegments.get(i));
+        }
+        String formattedPath = formatPaths(cuboidPaths);
+
+        // clear existing steps
+//        jobInstance.clearSteps();
+        int stepSeqNum = 0;
+        List<JobStep> result = Lists.newArrayList();
+        final String mergedCuboidPath = jobWorkingDir + "/" + cubeName + "/cuboid";
+
+        // merge cuboid data of ancestor segments
+        result.add(createMergeCuboidDataStep(jobInstance, stepSeqNum++, formattedPath, mergedCuboidPath));
+
+        // get output distribution step
+        result.add(createRangeRowkeyDistributionStep(jobInstance, stepSeqNum++, mergedCuboidPath));
+
+        // create htable step
+        result.add(createCreateHTableStep(jobInstance, stepSeqNum++));
+
+        // generate hfiles step
+        result.add(createConvertCuboidToHfileStep(jobInstance, stepSeqNum++, mergedCuboidPath));
+
+        // bulk load step
+        result.add(createBulkLoadStep(jobInstance, stepSeqNum++));
+
+        try {
+            log.debug(JsonUtil.writeValueAsIndentString(jobInstance));
+        } catch (Exception e) {
+            log.error(e.getMessage());
+        }
+        return result;
+    }
+
+    private List<JobStep> createBuildCubeSegmentSteps(JobInstance jobInstance) throws IOException {
+
+        // clear existing steps
+//        jobInstance.clearSteps();
+
+        int groupRowkeyColumnsCount = cube.getDescriptor().getRowkey().getNCuboidBuildLevels();
+        int totalRowkeyColumnsCount = cube.getDescriptor().getRowkey().getRowKeyColumns().length;
+
+
+        int stepSeqNum = 0;
+        List<JobStep> result = Lists.newArrayList();
+        if (this.engineConfig.isFlatTableByHive()) {
+            // by default in here
+
+            // flat hive table step
+            result.add(createIntermediateHiveTableStep(jobInstance, stepSeqNum++));
+        }
+
+        // fact distinct columns step
+        result.add(createFactDistinctColumnsStep(jobInstance, stepSeqNum++));
+
+        // build dictionary step
+        result.add(createBuildDictionaryStep(jobInstance, stepSeqNum++));
+
+        final String cuboidRootPath = jobWorkingDir + "/" + cubeName + "/cuboid/";
+        final String cuboidTmpRootPath = jobWorkingDir + "/" + cubeName + "/tmp_cuboid/";
+        final boolean incBuildMerge = cube.needMergeImmediatelyAfterBuild(cubeSegment);
+
+        String[] cuboidOutputTempPath = getCuboidOutputPaths(incBuildMerge?cuboidTmpRootPath:cuboidRootPath, totalRowkeyColumnsCount, groupRowkeyColumnsCount);
+        // base cuboid step
+        result.add(createBaseCuboidStep(jobInstance, stepSeqNum++, cuboidOutputTempPath));
+
+        // n dim cuboid steps
+        for (int i = 1; i <= groupRowkeyColumnsCount; i++) {
+            int dimNum = totalRowkeyColumnsCount - i;
+            result.add(createNDimensionCuboidStep(jobInstance, stepSeqNum++, cuboidOutputTempPath, dimNum, totalRowkeyColumnsCount));
+        }
+
+        if (incBuildMerge) {
+            List<String> pathToMerge = Lists.newArrayList();
+            for (CubeSegment segment: cube.getSegments(CubeSegmentStatusEnum.READY)) {
+                pathToMerge.add(getPathToMerge(jobInstance, segment));
+            }
+            pathToMerge.add(cuboidTmpRootPath + "*");
+            result.add(createMergeCuboidDataStep(jobInstance, stepSeqNum++, formatPaths(pathToMerge), cuboidRootPath));
+        }
+        String cuboidPath = incBuildMerge?cuboidRootPath:cuboidRootPath+"*";
+
+        // get output distribution step
+        result.add(createRangeRowkeyDistributionStep(jobInstance, stepSeqNum++, cuboidPath));
+
+        // create htable step
+        result.add(createCreateHTableStep(jobInstance, stepSeqNum++));
+        // generate hfiles step
+        result.add(createConvertCuboidToHfileStep(jobInstance, stepSeqNum++, cuboidPath));
+        // bulk load step
+        result.add(createBulkLoadStep(jobInstance, stepSeqNum++));
+
+        try {
+            log.debug(JsonUtil.writeValueAsIndentString(jobInstance));
+        } catch (Exception e) {
+            log.error(e.getMessage());
+        }
+        return result;
+    }
+
+    private String getPathToMerge(JobInstance jobInstance, CubeSegment segment) {
+        String uuid = segment.getUuid();
+        if (uuid == null) {
+            uuid = segment.getLastBuildJobID();
+        }
+        return JobInstance.getJobWorkingDir(uuid, engineConfig.getHdfsWorkingDirectory()) + "/" + jobInstance.getRelatedCube() + "/cuboid/*";
+    }
+
+    private String formatPaths(String[] paths) {
+        return StringUtils.join(paths, ",");
+    }
+
+    private String formatPaths(List<String> paths) {
+        return StringUtils.join(paths, ",");
+    }
+
+    private JobStep createBuildDictionaryStep(JobInstance jobInstance, int stepSeqNum) {
+        // base cuboid job
+        JobStep buildDictionaryStep = new JobStep();
+        buildDictionaryStep.setName(JobConstants.STEP_NAME_BUILD_DICTIONARY);
+        String cmd = "";
+        cmd = appendExecCmdParameters(cmd, "cubename", cubeName);
+        cmd = appendExecCmdParameters(cmd, "segmentname", segmentName);
+        cmd = appendExecCmdParameters(cmd, "input", getFactDistinctColumnsPath());
+
+        buildDictionaryStep.setExecCmd(cmd);
+        buildDictionaryStep.setSequenceID(stepSeqNum);
+        buildDictionaryStep.setStatus(JobStepStatusEnum.PENDING);
+        buildDictionaryStep.setRunAsync(false);
+        buildDictionaryStep.setCmdType(JobStepCmdTypeEnum.JAVA_CMD_HADOOP_NO_MR_DICTIONARY);
+        return buildDictionaryStep;
+    }
+
+    private JobStep createIntermediateHiveTableStep(JobInstance jobInstance, int stepSeqNum) throws IOException {
+        JoinedFlatTableDesc intermediateTableDesc = new JoinedFlatTableDesc(cube.getDescriptor(), this.cubeSegment);
+        String dropTableHql = JoinedFlatTable.generateDropTableStatement(intermediateTableDesc, jobUUID);
+        String createTableHql = JoinedFlatTable.generateCreateTableStatement(intermediateTableDesc, jobWorkingDir, jobUUID);
+        String insertDataHql = JoinedFlatTable.generateInsertDataStatement(intermediateTableDesc, jobUUID, this.engineConfig);
+
+        JobStep intermediateHiveTableStep = new JobStep();
+        intermediateHiveTableStep.setName(JobConstants.STEP_NAME_CREATE_FLAT_HIVE_TABLE);
+
+        StringBuffer buf = new StringBuffer();
+        buf.append("hive -e \"");
+        buf.append(dropTableHql + "\n");
+        buf.append(createTableHql + "\n");
+        buf.append(insertDataHql + "\n");
+        buf.append("\"");
+
+        intermediateHiveTableStep.setSequenceID(stepSeqNum);
+        intermediateHiveTableStep.setExecCmd(buf.toString());
+        intermediateHiveTableStep.setStatus(JobStepStatusEnum.PENDING);
+        intermediateHiveTableStep.setRunAsync(false);
+        intermediateHiveTableStep.setCmdType(JobStepCmdTypeEnum.SHELL_CMD_HADOOP);
+        return intermediateHiveTableStep;
+    }
+
+    private JobStep createFactDistinctColumnsStep(JobInstance jobInstance, int stepSeqNum) throws IOException {
+        // base cuboid job
+        JobStep factDistinctColumnsStep = new JobStep();
+
+        String inputLocation;
+        String cmd = "";
+
+        inputLocation = getIntermediateHiveTablePath();
+        cmd = appendMapReduceParameters(cmd, jobInstance);
+
+        factDistinctColumnsStep.setName(JobConstants.STEP_NAME_FACT_DISTINCT_COLUMNS);
+
+        cmd = appendExecCmdParameters(cmd, "cubename", cubeName);
+        cmd = appendExecCmdParameters(cmd, "input", inputLocation);
+        cmd = appendExecCmdParameters(cmd, "output", getFactDistinctColumnsPath());
+        cmd = appendExecCmdParameters(cmd, "jobname", "Kylin_Fact_Distinct_Columns_" + jobInstance.getRelatedCube() + "_Step_" + stepSeqNum);
+
+        factDistinctColumnsStep.setExecCmd(cmd);
+        factDistinctColumnsStep.setSequenceID(stepSeqNum);
+        factDistinctColumnsStep.setStatus(JobStepStatusEnum.PENDING);
+        factDistinctColumnsStep.setRunAsync(true);
+        factDistinctColumnsStep.setCmdType(JobStepCmdTypeEnum.JAVA_CMD_HADOOP_FACTDISTINCT);
+        return factDistinctColumnsStep;
+    }
+
+    private JobStep createBaseCuboidStep(JobInstance jobInstance, int stepSeqNum, String[] cuboidOutputTempPath) throws IOException {
+        // base cuboid job
+        JobStep baseCuboidStep = new JobStep();
+
+        String inputLocation;
+        String cmd = "";
+
+        if (this.engineConfig.isFlatTableByHive()) {
+            inputLocation = getIntermediateHiveTablePath();
+            cmd = appendMapReduceParameters(cmd, jobInstance);
+        } else {
+            HiveTable factTableInHive = new HiveTable(MetadataManager.getInstance(this.engineConfig.getConfig()), cube.getDescriptor().getFactTable());
+            inputLocation = factTableInHive.getHDFSLocation(false);
+            cmd = appendMapReduceParameters(cmd, jobInstance);
+            cmd = appendExecCmdParameters(cmd, "inputformat", "TextInputFormat");
+        }
+
+        baseCuboidStep.setName(JobConstants.STEP_NAME_BUILD_BASE_CUBOID);
+
+        cmd = appendExecCmdParameters(cmd, "cubename", cubeName);
+        cmd = appendExecCmdParameters(cmd, "segmentname", segmentName);
+        cmd = appendExecCmdParameters(cmd, "input", inputLocation);
+        cmd = appendExecCmdParameters(cmd, "output", cuboidOutputTempPath[0]);
+        cmd = appendExecCmdParameters(cmd, "jobname", "Kylin_Base_Cuboid_Builder_" + jobInstance.getRelatedCube() + "_Step_" + stepSeqNum);
+        cmd = appendExecCmdParameters(cmd, "level", "0");
+
+        baseCuboidStep.setExecCmd(cmd);
+        baseCuboidStep.setSequenceID(stepSeqNum);
+        baseCuboidStep.setStatus(JobStepStatusEnum.PENDING);
+        baseCuboidStep.setRunAsync(true);
+        baseCuboidStep.setCmdType(JobStepCmdTypeEnum.JAVA_CMD_HADOOP_BASECUBOID);
+        return baseCuboidStep;
+    }
+
+    private JobStep createNDimensionCuboidStep(JobInstance jobInstance, int stepSeqNum, String[] cuboidOutputTempPath, int dimNum, int totalRowkeyColumnCount) throws IOException {
+        // ND cuboid job
+        JobStep ndCuboidStep = new JobStep();
+
+        ndCuboidStep.setName(JobConstants.STEP_NAME_BUILD_N_D_CUBOID + " : " + dimNum + "-Dimension");
+        String cmd = "";
+
+        cmd = appendMapReduceParameters(cmd, jobInstance);
+        cmd = appendExecCmdParameters(cmd, "cubename", cubeName);
+        cmd = appendExecCmdParameters(cmd, "segmentname", segmentName);
+        cmd = appendExecCmdParameters(cmd, "input", cuboidOutputTempPath[totalRowkeyColumnCount - dimNum - 1]);
+        cmd = appendExecCmdParameters(cmd, "output", cuboidOutputTempPath[totalRowkeyColumnCount - dimNum]);
+        cmd = appendExecCmdParameters(cmd, "jobname", "Kylin_ND-Cuboid_Builder_" + jobInstance.getRelatedCube() + "_Step_" + stepSeqNum);
+        cmd = appendExecCmdParameters(cmd, "level", "" + (totalRowkeyColumnCount - dimNum));
+
+        ndCuboidStep.setExecCmd(cmd);
+        ndCuboidStep.setSequenceID(stepSeqNum);
+        ndCuboidStep.setStatus(JobStepStatusEnum.PENDING);
+        ndCuboidStep.setRunAsync(true);
+        ndCuboidStep.setCmdType(JobStepCmdTypeEnum.JAVA_CMD_HADOOP_NDCUBOID);
+        return ndCuboidStep;
+    }
+
+    private JobStep createRangeRowkeyDistributionStep(JobInstance jobInstance, int stepSeqNum, String inputPath) throws IOException {
+        JobStep rowkeyDistributionStep = new JobStep();
+        rowkeyDistributionStep.setName(JobConstants.STEP_NAME_GET_CUBOID_KEY_DISTRIBUTION);
+        String cmd = "";
+
+        cmd = appendMapReduceParameters(cmd, jobInstance);
+        cmd = appendExecCmdParameters(cmd, "input", inputPath);
+        cmd = appendExecCmdParameters(cmd, "output", getRowkeyDistributionOutputPath());
+        cmd = appendExecCmdParameters(cmd, "jobname", "Kylin_Region_Splits_Calculator_" + jobInstance.getRelatedCube() + "_Step_" + stepSeqNum);
+        cmd = appendExecCmdParameters(cmd, "cubename", cubeName);
+
+        rowkeyDistributionStep.setExecCmd(cmd);
+        rowkeyDistributionStep.setSequenceID(stepSeqNum);
+        rowkeyDistributionStep.setStatus(JobStepStatusEnum.PENDING);
+        rowkeyDistributionStep.setRunAsync(true);
+        rowkeyDistributionStep.setCmdType(JobStepCmdTypeEnum.JAVA_CMD_HADOOP_RANGEKEYDISTRIBUTION);
+        return rowkeyDistributionStep;
+    }
+
+    private JobStep createMergeCuboidDataStep(JobInstance jobInstance, int stepSeqNum, String inputPath, String outputPath) throws IOException {
+        JobStep mergeCuboidDataStep = new JobStep();
+        mergeCuboidDataStep.setName(JobConstants.STEP_NAME_MERGE_CUBOID);
+        String cmd = "";
+
+        cmd = appendMapReduceParameters(cmd, jobInstance);
+        cmd = appendExecCmdParameters(cmd, "cubename", cubeName);
+        cmd = appendExecCmdParameters(cmd, "segmentname", segmentName);
+        cmd = appendExecCmdParameters(cmd, "input", inputPath);
+        cmd = appendExecCmdParameters(cmd, "output", outputPath);
+        cmd = appendExecCmdParameters(cmd, "jobname", "Kylin_Merge_Cuboid_" + jobInstance.getRelatedCube() + "_Step_" + stepSeqNum);
+
+        mergeCuboidDataStep.setExecCmd(cmd);
+        mergeCuboidDataStep.setSequenceID(stepSeqNum);
+        mergeCuboidDataStep.setStatus(JobStepStatusEnum.PENDING);
+        mergeCuboidDataStep.setRunAsync(true);
+        mergeCuboidDataStep.setCmdType(JobStepCmdTypeEnum.JAVA_CMD_HADOOP_MERGECUBOID);
+        return mergeCuboidDataStep;
+    }
+
+    private JobStep createCreateHTableStep(JobInstance jobInstance, int stepSeqNum) {
+        JobStep createHtableStep = new JobStep();
+        createHtableStep.setName(JobConstants.STEP_NAME_CREATE_HBASE_TABLE);
+        String cmd = "";
+        cmd = appendExecCmdParameters(cmd, "cubename", cubeName);
+        cmd = appendExecCmdParameters(cmd, "input", getRowkeyDistributionOutputPath() + "/part-r-00000");
+        cmd = appendExecCmdParameters(cmd, "htablename", htablename);
+
+        createHtableStep.setExecCmd(cmd);
+        createHtableStep.setSequenceID(stepSeqNum);
+        createHtableStep.setStatus(JobStepStatusEnum.PENDING);
+        createHtableStep.setRunAsync(false);
+        createHtableStep.setCmdType(JobStepCmdTypeEnum.JAVA_CMD_HADDOP_NO_MR_CREATEHTABLE);
+
+        return createHtableStep;
+    }
+
+    private JobStep createConvertCuboidToHfileStep(JobInstance jobInstance, int stepSeqNum, String inputPath) throws IOException {
+        JobStep createHFilesStep = new JobStep();
+        createHFilesStep.setName(JobConstants.STEP_NAME_CONVERT_CUBOID_TO_HFILE);
+        String cmd = "";
+
+        cmd = appendMapReduceParameters(cmd, jobInstance);
+        cmd = appendExecCmdParameters(cmd, "cubename", cubeName);
+        cmd = appendExecCmdParameters(cmd, "input", inputPath);
+        cmd = appendExecCmdParameters(cmd, "output", jobWorkingDir + "/" + cubeName + "/hfile");
+        cmd = appendExecCmdParameters(cmd, "htablename", htablename);
+        cmd = appendExecCmdParameters(cmd, "jobname", "Kylin_HFile_Generator_" + jobInstance.getRelatedCube() + "_Step_" + stepSeqNum);
+
+        createHFilesStep.setExecCmd(cmd);
+        createHFilesStep.setSequenceID(stepSeqNum);
+        createHFilesStep.setStatus(JobStepStatusEnum.PENDING);
+        createHFilesStep.setRunAsync(true);
+        createHFilesStep.setCmdType(JobStepCmdTypeEnum.JAVA_CMD_HADOOP_CONVERTHFILE);
+
+        return createHFilesStep;
+
+    }
+
+    private JobStep createBulkLoadStep(JobInstance jobInstance, int stepSeqNum) {
+        JobStep bulkLoadStep = new JobStep();
+        bulkLoadStep.setName(JobConstants.STEP_NAME_BULK_LOAD_HFILE);
+
+        String cmd = "";
+        cmd = appendExecCmdParameters(cmd, "input", jobWorkingDir + "/" + cubeName + "/hfile/");
+        cmd = appendExecCmdParameters(cmd, "htablename", htablename);
+        cmd = appendExecCmdParameters(cmd, "cubename", cubeName);
+
+        bulkLoadStep.setSequenceID(stepSeqNum);
+        bulkLoadStep.setExecCmd(cmd);
+        bulkLoadStep.setStatus(JobStepStatusEnum.PENDING);
+        bulkLoadStep.setRunAsync(false);
+        bulkLoadStep.setCmdType(JobStepCmdTypeEnum.JAVA_CMD_HADOOP_NO_MR_BULKLOAD);
+
+        return bulkLoadStep;
+
+    }
+}


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

Posted by lu...@apache.org.
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


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

Posted by lu...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/deploy/web.xml
----------------------------------------------------------------------
diff --git a/deploy/web.xml b/deploy/web.xml
new file mode 100644
index 0000000..59ec414
--- /dev/null
+++ b/deploy/web.xml
@@ -0,0 +1,4618 @@
+<?xml version="1.0" encoding="ISO-8859-1"?>
+<!--
+  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.
+-->
+<web-app xmlns="http://java.sun.com/xml/ns/javaee"
+         xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+         xsi:schemaLocation="http://java.sun.com/xml/ns/javaee
+                      http://java.sun.com/xml/ns/javaee/web-app_3_0.xsd"
+         version="3.0">
+
+    <!-- ======================== Introduction ============================== -->
+    <!-- This document defines default values for *all* web applications      -->
+    <!-- loaded into this instance of Tomcat.  As each application is         -->
+    <!-- deployed, this file is processed, followed by the                    -->
+    <!-- "/WEB-INF/web.xml" deployment descriptor from your own               -->
+    <!-- applications.                                                        -->
+    <!--                                                                      -->
+    <!-- WARNING:  Do not configure application-specific resources here!      -->
+    <!-- They should go in the "/WEB-INF/web.xml" file in your application.   -->
+
+
+    <!-- ================== Built In Servlet Definitions ==================== -->
+
+
+    <!-- The default servlet for all web applications, that serves static     -->
+    <!-- resources.  It processes all requests that are not mapped to other   -->
+    <!-- servlets with servlet mappings (defined either here or in your own   -->
+    <!-- web.xml file).  This servlet supports the following initialization   -->
+    <!-- parameters (default values are in square brackets):                  -->
+    <!--                                                                      -->
+    <!--   debug               Debugging detail level for messages logged     -->
+    <!--                       by this servlet.  [0]                          -->
+    <!--                                                                      -->
+    <!--   fileEncoding        Encoding to be used to read static resources   -->
+    <!--                       [platform default]                             -->
+    <!--                                                                      -->
+    <!--   input               Input buffer size (in bytes) when reading      -->
+    <!--                       resources to be served.  [2048]                -->
+    <!--                                                                      -->
+    <!--   listings            Should directory listings be produced if there -->
+    <!--                       is no welcome file in this directory?  [false] -->
+    <!--                       WARNING: Listings for directories with many    -->
+    <!--                       entries can be slow and may consume            -->
+    <!--                       significant proportions of server resources.   -->
+    <!--                                                                      -->
+    <!--   output              Output buffer size (in bytes) when writing     -->
+    <!--                       resources to be served.  [2048]                -->
+    <!--                                                                      -->
+    <!--   readonly            Is this context "read only", so HTTP           -->
+    <!--                       commands like PUT and DELETE are               -->
+    <!--                       rejected?  [true]                              -->
+    <!--                                                                      -->
+    <!--   readmeFile          File to display together with the directory    -->
+    <!--                       contents. [null]                               -->
+    <!--                                                                      -->
+    <!--   sendfileSize        If the connector used supports sendfile, this  -->
+    <!--                       represents the minimal file size in KB for     -->
+    <!--                       which sendfile will be used. Use a negative    -->
+    <!--                       value to always disable sendfile.  [48]        -->
+    <!--                                                                      -->
+    <!--   useAcceptRanges     Should the Accept-Ranges header be included    -->
+    <!--                       in responses where appropriate? [true]         -->
+    <!--                                                                      -->
+    <!--  For directory listing customization. Checks localXsltFile, then     -->
+    <!--  globalXsltFile, then defaults to original behavior.                 -->
+    <!--                                                                      -->
+    <!--   localXsltFile       Make directory listings an XML doc and         -->
+    <!--                       pass the result to this style sheet residing   -->
+    <!--                       in that directory. This overrides              -->
+    <!--                       contextXsltFile and globalXsltFile[null]       -->
+    <!--                                                                      -->
+    <!--   contextXsltFile     Make directory listings an XML doc and         -->
+    <!--                       pass the result to this style sheet which is   -->
+    <!--                       relative to the context root. This overrides   -->
+    <!--                       globalXsltFile[null]                           -->
+    <!--                                                                      -->
+    <!--   globalXsltFile      Site wide configuration version of             -->
+    <!--                       localXsltFile. This argument must either be an -->
+    <!--                       absolute or relative (to either                -->
+    <!--                       $CATALINA_BASE/conf or $CATALINA_HOME/conf)    -->
+    <!--                       path that points to a location below either    -->
+    <!--                       $CATALINA_BASE/conf (checked first) or         -->
+    <!--                       $CATALINA_HOME/conf (checked second).[null]    -->
+    <!--                                                                      -->
+    <!--   showServerInfo      Should server information be presented in the  -->
+    <!--                       response sent to clients when directory        -->
+    <!--                       listings is enabled? [true]                    -->
+
+    <servlet>
+        <servlet-name>default</servlet-name>
+        <servlet-class>org.apache.catalina.servlets.DefaultServlet</servlet-class>
+        <init-param>
+            <param-name>debug</param-name>
+            <param-value>0</param-value>
+        </init-param>
+        <init-param>
+            <param-name>listings</param-name>
+            <param-value>false</param-value>
+        </init-param>
+        <load-on-startup>1</load-on-startup>
+    </servlet>
+
+
+    <!-- The JSP page compiler and execution servlet, which is the mechanism  -->
+    <!-- used by Tomcat to support JSP pages.  Traditionally, this servlet    -->
+    <!-- is mapped to the URL pattern "*.jsp".  This servlet supports the     -->
+    <!-- following initialization parameters (default values are in square    -->
+    <!-- brackets):                                                           -->
+    <!--                                                                      -->
+    <!--   checkInterval       If development is false and checkInterval is   -->
+    <!--                       greater than zero, background compilations are -->
+    <!--                       enabled. checkInterval is the time in seconds  -->
+    <!--                       between checks to see if a JSP page (and its   -->
+    <!--                       dependent files) needs to  be recompiled. [0]  -->
+    <!--                                                                      -->
+    <!--   classdebuginfo      Should the class file be compiled with         -->
+    <!--                       debugging information?  [true]                 -->
+    <!--                                                                      -->
+    <!--   classpath           What class path should I use while compiling   -->
+    <!--                       generated servlets?  [Created dynamically      -->
+    <!--                       based on the current web application]          -->
+    <!--                                                                      -->
+    <!--   compiler            Which compiler Ant should use to compile JSP   -->
+    <!--                       pages.  See the jasper documentation for more  -->
+    <!--                       information.                                   -->
+    <!--                                                                      -->
+    <!--   compilerSourceVM    Compiler source VM. [1.6]                      -->
+    <!--                                                                      -->
+    <!--   compilerTargetVM    Compiler target VM. [1.6]                      -->
+    <!--                                                                      -->
+    <!--   development         Is Jasper used in development mode? If true,   -->
+    <!--                       the frequency at which JSPs are checked for    -->
+    <!--                       modification may be specified via the          -->
+    <!--                       modificationTestInterval parameter. [true]     -->
+    <!--                                                                      -->
+    <!--   displaySourceFragment                                              -->
+    <!--                       Should a source fragment be included in        -->
+    <!--                       exception messages? [true]                     -->
+    <!--                                                                      -->
+    <!--   dumpSmap            Should the SMAP info for JSR45 debugging be    -->
+    <!--                       dumped to a file? [false]                      -->
+    <!--                       False if suppressSmap is true                  -->
+    <!--                                                                      -->
+    <!--   enablePooling       Determines whether tag handler pooling is      -->
+    <!--                       enabled. This is a compilation option. It will -->
+    <!--                       not alter the behaviour of JSPs that have      -->
+    <!--                       already been compiled. [true]                  -->
+    <!--                                                                      -->
+    <!--   engineOptionsClass  Allows specifying the Options class used to    -->
+    <!--                       configure Jasper. If not present, the default  -->
+    <!--                       EmbeddedServletOptions will be used.           -->
+    <!--                                                                      -->
+    <!--   errorOnUseBeanInvalidClassAttribute                                -->
+    <!--                       Should Jasper issue an error when the value of -->
+    <!--                       the class attribute in an useBean action is    -->
+    <!--                       not a valid bean class?  [true]                -->
+    <!--                                                                      -->
+    <!--   fork                Tell Ant to fork compiles of JSP pages so that -->
+    <!--                       a separate JVM is used for JSP page compiles   -->
+    <!--                       from the one Tomcat is running in. [true]      -->
+    <!--                                                                      -->
+    <!--   genStringAsCharArray                                               -->
+    <!--                       Should text strings be generated as char       -->
+    <!--                       arrays, to improve performance in some cases?  -->
+    <!--                       [false]                                        -->
+    <!--                                                                      -->
+    <!--   ieClassId           The class-id value to be sent to Internet      -->
+    <!--                       Explorer when using <jsp:plugin> tags.         -->
+    <!--                       [clsid:8AD9C840-044E-11D1-B3E9-00805F499D93]   -->
+    <!--                                                                      -->
+    <!--   javaEncoding        Java file encoding to use for generating java  -->
+    <!--                       source files. [UTF8]                           -->
+    <!--                                                                      -->
+    <!--   keepgenerated       Should we keep the generated Java source code  -->
+    <!--                       for each page instead of deleting it? [true]   -->
+    <!--                                                                      -->
+    <!--   mappedfile          Should we generate static content with one     -->
+    <!--                       print statement per input line, to ease        -->
+    <!--                       debugging?  [true]                             -->
+    <!--                                                                      -->
+    <!--   maxLoadedJsps       The maximum number of JSPs that will be loaded -->
+    <!--                       for a web application. If more than this       -->
+    <!--                       number of JSPs are loaded, the least recently  -->
+    <!--                       used JSPs will be unloaded so that the number  -->
+    <!--                       of JSPs loaded at any one time does not exceed -->
+    <!--                       this limit. A value of zero or less indicates  -->
+    <!--                       no limit. [-1]                                 -->
+    <!--                                                                      -->
+    <!--   jspIdleTimeout      The amount of time in seconds a JSP can be     -->
+    <!--                       idle before it is unloaded. A value of zero    -->
+    <!--                       or less indicates never unload. [-1]           -->
+    <!--                                                                      -->
+    <!--   modificationTestInterval                                           -->
+    <!--                       Causes a JSP (and its dependent files) to not  -->
+    <!--                       be checked for modification during the         -->
+    <!--                       specified time interval (in seconds) from the  -->
+    <!--                       last time the JSP was checked for              -->
+    <!--                       modification. A value of 0 will cause the JSP  -->
+    <!--                       to be checked on every access.                 -->
+    <!--                       Used in development mode only. [4]             -->
+    <!--                                                                      -->
+    <!--   recompileOnFail     If a JSP compilation fails should the          -->
+    <!--                       modificationTestInterval be ignored and the    -->
+    <!--                       next access trigger a re-compilation attempt?  -->
+    <!--                       Used in development mode only and is disabled  -->
+    <!--                       by default as compilation may be expensive and -->
+    <!--                       could lead to excessive resource usage.        -->
+    <!--                       [false]                                        -->
+    <!--                                                                      -->
+    <!--   scratchdir          What scratch directory should we use when      -->
+    <!--                       compiling JSP pages?  [default work directory  -->
+    <!--                       for the current web application]               -->
+    <!--                                                                      -->
+    <!--   suppressSmap        Should the generation of SMAP info for JSR45   -->
+    <!--                       debugging be suppressed?  [false]              -->
+    <!--                                                                      -->
+    <!--   trimSpaces          Should white spaces in template text between   -->
+    <!--                       actions or directives be trimmed?  [false]     -->
+    <!--                                                                      -->
+    <!--   xpoweredBy          Determines whether X-Powered-By response       -->
+    <!--                       header is added by generated servlet.  [false] -->
+
+    <servlet>
+        <servlet-name>jsp</servlet-name>
+        <servlet-class>org.apache.jasper.servlet.JspServlet</servlet-class>
+        <init-param>
+            <param-name>fork</param-name>
+            <param-value>false</param-value>
+        </init-param>
+        <init-param>
+            <param-name>xpoweredBy</param-name>
+            <param-value>false</param-value>
+        </init-param>
+        <load-on-startup>3</load-on-startup>
+    </servlet>
+
+
+    <!-- NOTE: An SSI Filter is also available as an alternative SSI          -->
+    <!-- implementation. Use either the Servlet or the Filter but NOT both.   -->
+    <!--                                                                      -->
+    <!-- Server Side Includes processing servlet, which processes SSI         -->
+    <!-- directives in HTML pages consistent with similar support in web      -->
+    <!-- servers like Apache.  Traditionally, this servlet is mapped to the   -->
+    <!-- URL pattern "*.shtml".  This servlet supports the following          -->
+    <!-- initialization parameters (default values are in square brackets):   -->
+    <!--                                                                      -->
+    <!--   buffered            Should output from this servlet be buffered?   -->
+    <!--                       (0=false, 1=true)  [0]                         -->
+    <!--                                                                      -->
+    <!--   debug               Debugging detail level for messages logged     -->
+    <!--                       by this servlet.  [0]                          -->
+    <!--                                                                      -->
+    <!--   expires             The number of seconds before a page with SSI   -->
+    <!--                       directives will expire.  [No default]          -->
+    <!--                                                                      -->
+    <!--   isVirtualWebappRelative                                            -->
+    <!--                       Should "virtual" paths be interpreted as       -->
+    <!--                       relative to the context root, instead of       -->
+    <!--                       the server root? [false]                       -->
+    <!--                                                                      -->
+    <!--   inputEncoding       The encoding to assume for SSI resources if    -->
+    <!--                       one is not available from the resource.        -->
+    <!--                       [Platform default]                             -->
+    <!--                                                                      -->
+    <!--   outputEncoding      The encoding to use for the page that results  -->
+    <!--                       from the SSI processing. [UTF-8]               -->
+    <!--                                                                      -->
+    <!--   allowExec           Is use of the exec command enabled? [false]    -->
+
+    <!--
+        <servlet>
+            <servlet-name>ssi</servlet-name>
+            <servlet-class>
+              org.apache.catalina.ssi.SSIServlet
+            </servlet-class>
+            <init-param>
+              <param-name>buffered</param-name>
+              <param-value>1</param-value>
+            </init-param>
+            <init-param>
+              <param-name>debug</param-name>
+              <param-value>0</param-value>
+            </init-param>
+            <init-param>
+              <param-name>expires</param-name>
+              <param-value>666</param-value>
+            </init-param>
+            <init-param>
+              <param-name>isVirtualWebappRelative</param-name>
+              <param-value>false</param-value>
+            </init-param>
+            <load-on-startup>4</load-on-startup>
+        </servlet>
+    -->
+
+
+    <!-- Common Gateway Includes (CGI) processing servlet, which supports     -->
+    <!-- execution of external applications that conform to the CGI spec      -->
+    <!-- requirements.  Typically, this servlet is mapped to the URL pattern  -->
+    <!-- "/cgi-bin/*", which means that any CGI applications that are         -->
+    <!-- executed must be present within the web application.  This servlet   -->
+    <!-- supports the following initialization parameters (default values     -->
+    <!-- are in square brackets):                                             -->
+    <!--                                                                      -->
+    <!--   cgiPathPrefix        The CGI search path will start at             -->
+    <!--                        webAppRootDir + File.separator + this prefix. -->
+    <!--                        [WEB-INF/cgi]                                 -->
+    <!--                                                                      -->
+    <!--   debug                Debugging detail level for messages logged    -->
+    <!--                        by this servlet.  [0]                         -->
+    <!--                                                                      -->
+    <!--   executable           Name of the executable used to run the        -->
+    <!--                        script. [perl]                                -->
+    <!--                                                                      -->
+    <!--   parameterEncoding    Name of parameter encoding to be used with    -->
+    <!--                        CGI servlet.                                  -->
+    <!--                        [System.getProperty("file.encoding","UTF-8")] -->
+    <!--                                                                      -->
+    <!--   passShellEnvironment Should the shell environment variables (if    -->
+    <!--                        any) be passed to the CGI script? [false]     -->
+    <!--                                                                      -->
+    <!--   stderrTimeout        The time (in milliseconds) to wait for the    -->
+    <!--                        reading of stderr to complete before          -->
+    <!--                        terminating the CGI process. [2000]           -->
+
+    <!--
+        <servlet>
+            <servlet-name>cgi</servlet-name>
+            <servlet-class>org.apache.catalina.servlets.CGIServlet</servlet-class>
+            <init-param>
+              <param-name>debug</param-name>
+              <param-value>0</param-value>
+            </init-param>
+            <init-param>
+              <param-name>cgiPathPrefix</param-name>
+              <param-value>WEB-INF/cgi</param-value>
+            </init-param>
+             <load-on-startup>5</load-on-startup>
+        </servlet>
+    -->
+
+
+    <!-- ================ Built In Servlet Mappings ========================= -->
+
+
+    <!-- The servlet mappings for the built in servlets defined above.  Note  -->
+    <!-- that, by default, the CGI and SSI servlets are *not* mapped.  You    -->
+    <!-- must uncomment these mappings (or add them to your application's own -->
+    <!-- web.xml deployment descriptor) to enable these services              -->
+
+    <!-- The mapping for the default servlet -->
+    <servlet-mapping>
+        <servlet-name>default</servlet-name>
+        <url-pattern>/</url-pattern>
+    </servlet-mapping>
+
+    <!-- The mappings for the JSP servlet -->
+    <servlet-mapping>
+        <servlet-name>jsp</servlet-name>
+        <url-pattern>*.jsp</url-pattern>
+        <url-pattern>*.jspx</url-pattern>
+    </servlet-mapping>
+
+    <!-- The mapping for the SSI servlet -->
+    <!--
+        <servlet-mapping>
+            <servlet-name>ssi</servlet-name>
+            <url-pattern>*.shtml</url-pattern>
+        </servlet-mapping>
+    -->
+
+    <!-- The mapping for the CGI Gateway servlet -->
+
+    <!--
+        <servlet-mapping>
+            <servlet-name>cgi</servlet-name>
+            <url-pattern>/cgi-bin/*</url-pattern>
+        </servlet-mapping>
+    -->
+
+
+    <!-- ================== Built In Filter Definitions ===================== -->
+
+    <!-- A filter that sets character encoding that is used to decode -->
+    <!-- parameters in a POST request -->
+    <!--
+        <filter>
+            <filter-name>setCharacterEncodingFilter</filter-name>
+            <filter-class>org.apache.catalina.filters.SetCharacterEncodingFilter</filter-class>
+            <init-param>
+                <param-name>encoding</param-name>
+                <param-value>UTF-8</param-value>
+            </init-param>
+            <async-supported>true</async-supported>
+        </filter>
+    -->
+
+    <!-- A filter that triggers request parameters parsing and rejects the    -->
+    <!-- request if some parameters were skipped because of parsing errors or -->
+    <!-- request size limitations.                                            -->
+    <!--
+        <filter>
+            <filter-name>failedRequestFilter</filter-name>
+            <filter-class>
+              org.apache.catalina.filters.FailedRequestFilter
+            </filter-class>
+            <async-supported>true</async-supported>
+        </filter>
+    -->
+
+
+    <!-- NOTE: An SSI Servlet is also available as an alternative SSI         -->
+    <!-- implementation. Use either the Servlet or the Filter but NOT both.   -->
+    <!--                                                                      -->
+    <!-- Server Side Includes processing filter, which processes SSI          -->
+    <!-- directives in HTML pages consistent with similar support in web      -->
+    <!-- servers like Apache.  Traditionally, this filter is mapped to the    -->
+    <!-- URL pattern "*.shtml", though it can be mapped to "*" as it will     -->
+    <!-- selectively enable/disable SSI processing based on mime types. For   -->
+    <!-- this to work you will need to uncomment the .shtml mime type         -->
+    <!-- definition towards the bottom of this file.                          -->
+    <!-- The contentType init param allows you to apply SSI processing to JSP -->
+    <!-- pages, javascript, or any other content you wish.  This filter       -->
+    <!-- supports the following initialization parameters (default values are -->
+    <!-- in square brackets):                                                 -->
+    <!--                                                                      -->
+    <!--   contentType         A regex pattern that must be matched before    -->
+    <!--                       SSI processing is applied.                     -->
+    <!--                       [text/x-server-parsed-html(;.*)?]              -->
+    <!--                                                                      -->
+    <!--   debug               Debugging detail level for messages logged     -->
+    <!--                       by this servlet.  [0]                          -->
+    <!--                                                                      -->
+    <!--   expires             The number of seconds before a page with SSI   -->
+    <!--                       directives will expire.  [No default]          -->
+    <!--                                                                      -->
+    <!--   isVirtualWebappRelative                                            -->
+    <!--                       Should "virtual" paths be interpreted as       -->
+    <!--                       relative to the context root, instead of       -->
+    <!--                       the server root? [false]                       -->
+    <!--                                                                      -->
+    <!--   allowExec           Is use of the exec command enabled? [false]    -->
+
+    <!--
+        <filter>
+            <filter-name>ssi</filter-name>
+            <filter-class>
+              org.apache.catalina.ssi.SSIFilter
+            </filter-class>
+            <init-param>
+              <param-name>contentType</param-name>
+              <param-value>text/x-server-parsed-html(;.*)?</param-value>
+            </init-param>
+            <init-param>
+              <param-name>debug</param-name>
+              <param-value>0</param-value>
+            </init-param>
+            <init-param>
+              <param-name>expires</param-name>
+              <param-value>666</param-value>
+            </init-param>
+            <init-param>
+              <param-name>isVirtualWebappRelative</param-name>
+              <param-value>false</param-value>
+            </init-param>
+        </filter>
+    -->
+
+
+    <!-- ==================== Built In Filter Mappings ====================== -->
+
+    <!-- The mapping for the Set Character Encoding Filter -->
+    <!--
+        <filter-mapping>
+            <filter-name>setCharacterEncodingFilter</filter-name>
+            <url-pattern>/*</url-pattern>
+        </filter-mapping>
+    -->
+
+    <!-- The mapping for the Failed Request Filter -->
+    <!--
+        <filter-mapping>
+            <filter-name>failedRequestFilter</filter-name>
+            <url-pattern>/*</url-pattern>
+        </filter-mapping>
+    -->
+
+    <!-- The mapping for the SSI Filter -->
+    <!--
+        <filter-mapping>
+            <filter-name>ssi</filter-name>
+            <url-pattern>*.shtml</url-pattern>
+        </filter-mapping>
+    -->
+
+
+    <!-- ==================== Default Session Configuration ================= -->
+    <!-- You can set the default session timeout (in minutes) for all newly   -->
+    <!-- created sessions by modifying the value below.                       -->
+
+    <session-config>
+        <session-timeout>30</session-timeout>
+    </session-config>
+
+
+    <!-- ===================== Default MIME Type Mappings =================== -->
+    <!-- When serving static resources, Tomcat will automatically generate    -->
+    <!-- a "Content-Type" header based on the resource's filename extension,  -->
+    <!-- based on these mappings.  Additional mappings can be added here (to  -->
+    <!-- apply to all web applications), or in your own application's web.xml -->
+    <!-- deployment descriptor.                                               -->
+
+    <mime-mapping>
+        <extension>123</extension>
+        <mime-type>application/vnd.lotus-1-2-3</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>3dml</extension>
+        <mime-type>text/vnd.in3d.3dml</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>3ds</extension>
+        <mime-type>image/x-3ds</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>3g2</extension>
+        <mime-type>video/3gpp2</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>3gp</extension>
+        <mime-type>video/3gpp</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>7z</extension>
+        <mime-type>application/x-7z-compressed</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>aab</extension>
+        <mime-type>application/x-authorware-bin</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>aac</extension>
+        <mime-type>audio/x-aac</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>aam</extension>
+        <mime-type>application/x-authorware-map</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>aas</extension>
+        <mime-type>application/x-authorware-seg</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>abs</extension>
+        <mime-type>audio/x-mpeg</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>abw</extension>
+        <mime-type>application/x-abiword</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>ac</extension>
+        <mime-type>application/pkix-attr-cert</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>acc</extension>
+        <mime-type>application/vnd.americandynamics.acc</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>ace</extension>
+        <mime-type>application/x-ace-compressed</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>acu</extension>
+        <mime-type>application/vnd.acucobol</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>acutc</extension>
+        <mime-type>application/vnd.acucorp</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>adp</extension>
+        <mime-type>audio/adpcm</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>aep</extension>
+        <mime-type>application/vnd.audiograph</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>afm</extension>
+        <mime-type>application/x-font-type1</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>afp</extension>
+        <mime-type>application/vnd.ibm.modcap</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>ahead</extension>
+        <mime-type>application/vnd.ahead.space</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>ai</extension>
+        <mime-type>application/postscript</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>aif</extension>
+        <mime-type>audio/x-aiff</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>aifc</extension>
+        <mime-type>audio/x-aiff</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>aiff</extension>
+        <mime-type>audio/x-aiff</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>aim</extension>
+        <mime-type>application/x-aim</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>air</extension>
+        <mime-type>application/vnd.adobe.air-application-installer-package+zip</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>ait</extension>
+        <mime-type>application/vnd.dvb.ait</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>ami</extension>
+        <mime-type>application/vnd.amiga.ami</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>anx</extension>
+        <mime-type>application/annodex</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>apk</extension>
+        <mime-type>application/vnd.android.package-archive</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>appcache</extension>
+        <mime-type>text/cache-manifest</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>application</extension>
+        <mime-type>application/x-ms-application</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>apr</extension>
+        <mime-type>application/vnd.lotus-approach</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>arc</extension>
+        <mime-type>application/x-freearc</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>art</extension>
+        <mime-type>image/x-jg</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>asc</extension>
+        <mime-type>application/pgp-signature</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>asf</extension>
+        <mime-type>video/x-ms-asf</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>asm</extension>
+        <mime-type>text/x-asm</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>aso</extension>
+        <mime-type>application/vnd.accpac.simply.aso</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>asx</extension>
+        <mime-type>video/x-ms-asf</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>atc</extension>
+        <mime-type>application/vnd.acucorp</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>atom</extension>
+        <mime-type>application/atom+xml</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>atomcat</extension>
+        <mime-type>application/atomcat+xml</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>atomsvc</extension>
+        <mime-type>application/atomsvc+xml</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>atx</extension>
+        <mime-type>application/vnd.antix.game-component</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>au</extension>
+        <mime-type>audio/basic</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>avi</extension>
+        <mime-type>video/x-msvideo</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>avx</extension>
+        <mime-type>video/x-rad-screenplay</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>aw</extension>
+        <mime-type>application/applixware</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>axa</extension>
+        <mime-type>audio/annodex</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>axv</extension>
+        <mime-type>video/annodex</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>azf</extension>
+        <mime-type>application/vnd.airzip.filesecure.azf</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>azs</extension>
+        <mime-type>application/vnd.airzip.filesecure.azs</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>azw</extension>
+        <mime-type>application/vnd.amazon.ebook</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>bat</extension>
+        <mime-type>application/x-msdownload</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>bcpio</extension>
+        <mime-type>application/x-bcpio</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>bdf</extension>
+        <mime-type>application/x-font-bdf</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>bdm</extension>
+        <mime-type>application/vnd.syncml.dm+wbxml</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>bed</extension>
+        <mime-type>application/vnd.realvnc.bed</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>bh2</extension>
+        <mime-type>application/vnd.fujitsu.oasysprs</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>bin</extension>
+        <mime-type>application/octet-stream</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>blb</extension>
+        <mime-type>application/x-blorb</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>blorb</extension>
+        <mime-type>application/x-blorb</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>bmi</extension>
+        <mime-type>application/vnd.bmi</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>bmp</extension>
+        <mime-type>image/bmp</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>body</extension>
+        <mime-type>text/html</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>book</extension>
+        <mime-type>application/vnd.framemaker</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>box</extension>
+        <mime-type>application/vnd.previewsystems.box</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>boz</extension>
+        <mime-type>application/x-bzip2</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>bpk</extension>
+        <mime-type>application/octet-stream</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>btif</extension>
+        <mime-type>image/prs.btif</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>bz</extension>
+        <mime-type>application/x-bzip</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>bz2</extension>
+        <mime-type>application/x-bzip2</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>c</extension>
+        <mime-type>text/x-c</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>c11amc</extension>
+        <mime-type>application/vnd.cluetrust.cartomobile-config</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>c11amz</extension>
+        <mime-type>application/vnd.cluetrust.cartomobile-config-pkg</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>c4d</extension>
+        <mime-type>application/vnd.clonk.c4group</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>c4f</extension>
+        <mime-type>application/vnd.clonk.c4group</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>c4g</extension>
+        <mime-type>application/vnd.clonk.c4group</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>c4p</extension>
+        <mime-type>application/vnd.clonk.c4group</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>c4u</extension>
+        <mime-type>application/vnd.clonk.c4group</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>cab</extension>
+        <mime-type>application/vnd.ms-cab-compressed</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>caf</extension>
+        <mime-type>audio/x-caf</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>cap</extension>
+        <mime-type>application/vnd.tcpdump.pcap</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>car</extension>
+        <mime-type>application/vnd.curl.car</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>cat</extension>
+        <mime-type>application/vnd.ms-pki.seccat</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>cb7</extension>
+        <mime-type>application/x-cbr</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>cba</extension>
+        <mime-type>application/x-cbr</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>cbr</extension>
+        <mime-type>application/x-cbr</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>cbt</extension>
+        <mime-type>application/x-cbr</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>cbz</extension>
+        <mime-type>application/x-cbr</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>cc</extension>
+        <mime-type>text/x-c</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>cct</extension>
+        <mime-type>application/x-director</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>ccxml</extension>
+        <mime-type>application/ccxml+xml</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>cdbcmsg</extension>
+        <mime-type>application/vnd.contact.cmsg</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>cdf</extension>
+        <mime-type>application/x-cdf</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>cdkey</extension>
+        <mime-type>application/vnd.mediastation.cdkey</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>cdmia</extension>
+        <mime-type>application/cdmi-capability</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>cdmic</extension>
+        <mime-type>application/cdmi-container</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>cdmid</extension>
+        <mime-type>application/cdmi-domain</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>cdmio</extension>
+        <mime-type>application/cdmi-object</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>cdmiq</extension>
+        <mime-type>application/cdmi-queue</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>cdx</extension>
+        <mime-type>chemical/x-cdx</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>cdxml</extension>
+        <mime-type>application/vnd.chemdraw+xml</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>cdy</extension>
+        <mime-type>application/vnd.cinderella</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>cer</extension>
+        <mime-type>application/pkix-cert</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>cfs</extension>
+        <mime-type>application/x-cfs-compressed</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>cgm</extension>
+        <mime-type>image/cgm</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>chat</extension>
+        <mime-type>application/x-chat</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>chm</extension>
+        <mime-type>application/vnd.ms-htmlhelp</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>chrt</extension>
+        <mime-type>application/vnd.kde.kchart</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>cif</extension>
+        <mime-type>chemical/x-cif</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>cii</extension>
+        <mime-type>application/vnd.anser-web-certificate-issue-initiation</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>cil</extension>
+        <mime-type>application/vnd.ms-artgalry</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>cla</extension>
+        <mime-type>application/vnd.claymore</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>class</extension>
+        <mime-type>application/java</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>clkk</extension>
+        <mime-type>application/vnd.crick.clicker.keyboard</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>clkp</extension>
+        <mime-type>application/vnd.crick.clicker.palette</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>clkt</extension>
+        <mime-type>application/vnd.crick.clicker.template</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>clkw</extension>
+        <mime-type>application/vnd.crick.clicker.wordbank</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>clkx</extension>
+        <mime-type>application/vnd.crick.clicker</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>clp</extension>
+        <mime-type>application/x-msclip</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>cmc</extension>
+        <mime-type>application/vnd.cosmocaller</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>cmdf</extension>
+        <mime-type>chemical/x-cmdf</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>cml</extension>
+        <mime-type>chemical/x-cml</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>cmp</extension>
+        <mime-type>application/vnd.yellowriver-custom-menu</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>cmx</extension>
+        <mime-type>image/x-cmx</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>cod</extension>
+        <mime-type>application/vnd.rim.cod</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>com</extension>
+        <mime-type>application/x-msdownload</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>conf</extension>
+        <mime-type>text/plain</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>cpio</extension>
+        <mime-type>application/x-cpio</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>cpp</extension>
+        <mime-type>text/x-c</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>cpt</extension>
+        <mime-type>application/mac-compactpro</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>crd</extension>
+        <mime-type>application/x-mscardfile</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>crl</extension>
+        <mime-type>application/pkix-crl</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>crt</extension>
+        <mime-type>application/x-x509-ca-cert</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>cryptonote</extension>
+        <mime-type>application/vnd.rig.cryptonote</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>csh</extension>
+        <mime-type>application/x-csh</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>csml</extension>
+        <mime-type>chemical/x-csml</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>csp</extension>
+        <mime-type>application/vnd.commonspace</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>css</extension>
+        <mime-type>text/css</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>cst</extension>
+        <mime-type>application/x-director</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>csv</extension>
+        <mime-type>text/csv</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>cu</extension>
+        <mime-type>application/cu-seeme</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>curl</extension>
+        <mime-type>text/vnd.curl</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>cww</extension>
+        <mime-type>application/prs.cww</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>cxt</extension>
+        <mime-type>application/x-director</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>cxx</extension>
+        <mime-type>text/x-c</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>dae</extension>
+        <mime-type>model/vnd.collada+xml</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>daf</extension>
+        <mime-type>application/vnd.mobius.daf</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>dart</extension>
+        <mime-type>application/vnd.dart</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>dataless</extension>
+        <mime-type>application/vnd.fdsn.seed</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>davmount</extension>
+        <mime-type>application/davmount+xml</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>dbk</extension>
+        <mime-type>application/docbook+xml</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>dcr</extension>
+        <mime-type>application/x-director</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>dcurl</extension>
+        <mime-type>text/vnd.curl.dcurl</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>dd2</extension>
+        <mime-type>application/vnd.oma.dd2+xml</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>ddd</extension>
+        <mime-type>application/vnd.fujixerox.ddd</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>deb</extension>
+        <mime-type>application/x-debian-package</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>def</extension>
+        <mime-type>text/plain</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>deploy</extension>
+        <mime-type>application/octet-stream</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>der</extension>
+        <mime-type>application/x-x509-ca-cert</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>dfac</extension>
+        <mime-type>application/vnd.dreamfactory</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>dgc</extension>
+        <mime-type>application/x-dgc-compressed</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>dib</extension>
+        <mime-type>image/bmp</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>dic</extension>
+        <mime-type>text/x-c</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>dir</extension>
+        <mime-type>application/x-director</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>dis</extension>
+        <mime-type>application/vnd.mobius.dis</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>dist</extension>
+        <mime-type>application/octet-stream</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>distz</extension>
+        <mime-type>application/octet-stream</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>djv</extension>
+        <mime-type>image/vnd.djvu</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>djvu</extension>
+        <mime-type>image/vnd.djvu</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>dll</extension>
+        <mime-type>application/x-msdownload</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>dmg</extension>
+        <mime-type>application/x-apple-diskimage</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>dmp</extension>
+        <mime-type>application/vnd.tcpdump.pcap</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>dms</extension>
+        <mime-type>application/octet-stream</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>dna</extension>
+        <mime-type>application/vnd.dna</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>doc</extension>
+        <mime-type>application/msword</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>docm</extension>
+        <mime-type>application/vnd.ms-word.document.macroenabled.12</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>docx</extension>
+        <mime-type>application/vnd.openxmlformats-officedocument.wordprocessingml.document</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>dot</extension>
+        <mime-type>application/msword</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>dotm</extension>
+        <mime-type>application/vnd.ms-word.template.macroenabled.12</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>dotx</extension>
+        <mime-type>application/vnd.openxmlformats-officedocument.wordprocessingml.template</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>dp</extension>
+        <mime-type>application/vnd.osgi.dp</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>dpg</extension>
+        <mime-type>application/vnd.dpgraph</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>dra</extension>
+        <mime-type>audio/vnd.dra</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>dsc</extension>
+        <mime-type>text/prs.lines.tag</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>dssc</extension>
+        <mime-type>application/dssc+der</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>dtb</extension>
+        <mime-type>application/x-dtbook+xml</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>dtd</extension>
+        <mime-type>application/xml-dtd</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>dts</extension>
+        <mime-type>audio/vnd.dts</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>dtshd</extension>
+        <mime-type>audio/vnd.dts.hd</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>dump</extension>
+        <mime-type>application/octet-stream</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>dv</extension>
+        <mime-type>video/x-dv</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>dvb</extension>
+        <mime-type>video/vnd.dvb.file</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>dvi</extension>
+        <mime-type>application/x-dvi</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>dwf</extension>
+        <mime-type>model/vnd.dwf</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>dwg</extension>
+        <mime-type>image/vnd.dwg</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>dxf</extension>
+        <mime-type>image/vnd.dxf</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>dxp</extension>
+        <mime-type>application/vnd.spotfire.dxp</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>dxr</extension>
+        <mime-type>application/x-director</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>ecelp4800</extension>
+        <mime-type>audio/vnd.nuera.ecelp4800</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>ecelp7470</extension>
+        <mime-type>audio/vnd.nuera.ecelp7470</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>ecelp9600</extension>
+        <mime-type>audio/vnd.nuera.ecelp9600</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>ecma</extension>
+        <mime-type>application/ecmascript</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>edm</extension>
+        <mime-type>application/vnd.novadigm.edm</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>edx</extension>
+        <mime-type>application/vnd.novadigm.edx</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>efif</extension>
+        <mime-type>application/vnd.picsel</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>ei6</extension>
+        <mime-type>application/vnd.pg.osasli</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>elc</extension>
+        <mime-type>application/octet-stream</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>emf</extension>
+        <mime-type>application/x-msmetafile</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>eml</extension>
+        <mime-type>message/rfc822</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>emma</extension>
+        <mime-type>application/emma+xml</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>emz</extension>
+        <mime-type>application/x-msmetafile</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>eol</extension>
+        <mime-type>audio/vnd.digital-winds</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>eot</extension>
+        <mime-type>application/vnd.ms-fontobject</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>eps</extension>
+        <mime-type>application/postscript</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>epub</extension>
+        <mime-type>application/epub+zip</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>es3</extension>
+        <mime-type>application/vnd.eszigno3+xml</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>esa</extension>
+        <mime-type>application/vnd.osgi.subsystem</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>esf</extension>
+        <mime-type>application/vnd.epson.esf</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>et3</extension>
+        <mime-type>application/vnd.eszigno3+xml</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>etx</extension>
+        <mime-type>text/x-setext</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>eva</extension>
+        <mime-type>application/x-eva</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>evy</extension>
+        <mime-type>application/x-envoy</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>exe</extension>
+        <mime-type>application/octet-stream</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>exi</extension>
+        <mime-type>application/exi</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>ext</extension>
+        <mime-type>application/vnd.novadigm.ext</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>ez</extension>
+        <mime-type>application/andrew-inset</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>ez2</extension>
+        <mime-type>application/vnd.ezpix-album</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>ez3</extension>
+        <mime-type>application/vnd.ezpix-package</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>f</extension>
+        <mime-type>text/x-fortran</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>f4v</extension>
+        <mime-type>video/x-f4v</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>f77</extension>
+        <mime-type>text/x-fortran</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>f90</extension>
+        <mime-type>text/x-fortran</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>fbs</extension>
+        <mime-type>image/vnd.fastbidsheet</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>fcdt</extension>
+        <mime-type>application/vnd.adobe.formscentral.fcdt</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>fcs</extension>
+        <mime-type>application/vnd.isac.fcs</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>fdf</extension>
+        <mime-type>application/vnd.fdf</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>fe_launch</extension>
+        <mime-type>application/vnd.denovo.fcselayout-link</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>fg5</extension>
+        <mime-type>application/vnd.fujitsu.oasysgp</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>fgd</extension>
+        <mime-type>application/x-director</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>fh</extension>
+        <mime-type>image/x-freehand</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>fh4</extension>
+        <mime-type>image/x-freehand</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>fh5</extension>
+        <mime-type>image/x-freehand</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>fh7</extension>
+        <mime-type>image/x-freehand</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>fhc</extension>
+        <mime-type>image/x-freehand</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>fig</extension>
+        <mime-type>application/x-xfig</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>flac</extension>
+        <mime-type>audio/flac</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>fli</extension>
+        <mime-type>video/x-fli</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>flo</extension>
+        <mime-type>application/vnd.micrografx.flo</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>flv</extension>
+        <mime-type>video/x-flv</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>flw</extension>
+        <mime-type>application/vnd.kde.kivio</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>flx</extension>
+        <mime-type>text/vnd.fmi.flexstor</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>fly</extension>
+        <mime-type>text/vnd.fly</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>fm</extension>
+        <mime-type>application/vnd.framemaker</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>fnc</extension>
+        <mime-type>application/vnd.frogans.fnc</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>for</extension>
+        <mime-type>text/x-fortran</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>fpx</extension>
+        <mime-type>image/vnd.fpx</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>frame</extension>
+        <mime-type>application/vnd.framemaker</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>fsc</extension>
+        <mime-type>application/vnd.fsc.weblaunch</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>fst</extension>
+        <mime-type>image/vnd.fst</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>ftc</extension>
+        <mime-type>application/vnd.fluxtime.clip</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>fti</extension>
+        <mime-type>application/vnd.anser-web-funds-transfer-initiation</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>fvt</extension>
+        <mime-type>video/vnd.fvt</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>fxp</extension>
+        <mime-type>application/vnd.adobe.fxp</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>fxpl</extension>
+        <mime-type>application/vnd.adobe.fxp</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>fzs</extension>
+        <mime-type>application/vnd.fuzzysheet</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>g2w</extension>
+        <mime-type>application/vnd.geoplan</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>g3</extension>
+        <mime-type>image/g3fax</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>g3w</extension>
+        <mime-type>application/vnd.geospace</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>gac</extension>
+        <mime-type>application/vnd.groove-account</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>gam</extension>
+        <mime-type>application/x-tads</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>gbr</extension>
+        <mime-type>application/rpki-ghostbusters</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>gca</extension>
+        <mime-type>application/x-gca-compressed</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>gdl</extension>
+        <mime-type>model/vnd.gdl</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>geo</extension>
+        <mime-type>application/vnd.dynageo</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>gex</extension>
+        <mime-type>application/vnd.geometry-explorer</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>ggb</extension>
+        <mime-type>application/vnd.geogebra.file</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>ggt</extension>
+        <mime-type>application/vnd.geogebra.tool</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>ghf</extension>
+        <mime-type>application/vnd.groove-help</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>gif</extension>
+        <mime-type>image/gif</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>gim</extension>
+        <mime-type>application/vnd.groove-identity-message</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>gml</extension>
+        <mime-type>application/gml+xml</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>gmx</extension>
+        <mime-type>application/vnd.gmx</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>gnumeric</extension>
+        <mime-type>application/x-gnumeric</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>gph</extension>
+        <mime-type>application/vnd.flographit</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>gpx</extension>
+        <mime-type>application/gpx+xml</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>gqf</extension>
+        <mime-type>application/vnd.grafeq</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>gqs</extension>
+        <mime-type>application/vnd.grafeq</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>gram</extension>
+        <mime-type>application/srgs</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>gramps</extension>
+        <mime-type>application/x-gramps-xml</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>gre</extension>
+        <mime-type>application/vnd.geometry-explorer</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>grv</extension>
+        <mime-type>application/vnd.groove-injector</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>grxml</extension>
+        <mime-type>application/srgs+xml</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>gsf</extension>
+        <mime-type>application/x-font-ghostscript</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>gtar</extension>
+        <mime-type>application/x-gtar</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>gtm</extension>
+        <mime-type>application/vnd.groove-tool-message</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>gtw</extension>
+        <mime-type>model/vnd.gtw</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>gv</extension>
+        <mime-type>text/vnd.graphviz</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>gxf</extension>
+        <mime-type>application/gxf</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>gxt</extension>
+        <mime-type>application/vnd.geonext</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>gz</extension>
+        <mime-type>application/x-gzip</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>h</extension>
+        <mime-type>text/x-c</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>h261</extension>
+        <mime-type>video/h261</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>h263</extension>
+        <mime-type>video/h263</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>h264</extension>
+        <mime-type>video/h264</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>hal</extension>
+        <mime-type>application/vnd.hal+xml</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>hbci</extension>
+        <mime-type>application/vnd.hbci</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>hdf</extension>
+        <mime-type>application/x-hdf</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>hh</extension>
+        <mime-type>text/x-c</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>hlp</extension>
+        <mime-type>application/winhlp</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>hpgl</extension>
+        <mime-type>application/vnd.hp-hpgl</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>hpid</extension>
+        <mime-type>application/vnd.hp-hpid</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>hps</extension>
+        <mime-type>application/vnd.hp-hps</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>hqx</extension>
+        <mime-type>application/mac-binhex40</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>htc</extension>
+        <mime-type>text/x-component</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>htke</extension>
+        <mime-type>application/vnd.kenameaapp</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>htm</extension>
+        <mime-type>text/html</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>html</extension>
+        <mime-type>text/html</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>hvd</extension>
+        <mime-type>application/vnd.yamaha.hv-dic</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>hvp</extension>
+        <mime-type>application/vnd.yamaha.hv-voice</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>hvs</extension>
+        <mime-type>application/vnd.yamaha.hv-script</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>i2g</extension>
+        <mime-type>application/vnd.intergeo</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>icc</extension>
+        <mime-type>application/vnd.iccprofile</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>ice</extension>
+        <mime-type>x-conference/x-cooltalk</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>icm</extension>
+        <mime-type>application/vnd.iccprofile</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>ico</extension>
+        <mime-type>image/x-icon</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>ics</extension>
+        <mime-type>text/calendar</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>ief</extension>
+        <mime-type>image/ief</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>ifb</extension>
+        <mime-type>text/calendar</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>ifm</extension>
+        <mime-type>application/vnd.shana.informed.formdata</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>iges</extension>
+        <mime-type>model/iges</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>igl</extension>
+        <mime-type>application/vnd.igloader</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>igm</extension>
+        <mime-type>application/vnd.insors.igm</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>igs</extension>
+        <mime-type>model/iges</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>igx</extension>
+        <mime-type>application/vnd.micrografx.igx</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>iif</extension>
+        <mime-type>application/vnd.shana.informed.interchange</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>imp</extension>
+        <mime-type>application/vnd.accpac.simply.imp</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>ims</extension>
+        <mime-type>application/vnd.ms-ims</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>in</extension>
+        <mime-type>text/plain</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>ink</extension>
+        <mime-type>application/inkml+xml</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>inkml</extension>
+        <mime-type>application/inkml+xml</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>install</extension>
+        <mime-type>application/x-install-instructions</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>iota</extension>
+        <mime-type>application/vnd.astraea-software.iota</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>ipfix</extension>
+        <mime-type>application/ipfix</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>ipk</extension>
+        <mime-type>application/vnd.shana.informed.package</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>irm</extension>
+        <mime-type>application/vnd.ibm.rights-management</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>irp</extension>
+        <mime-type>application/vnd.irepository.package+xml</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>iso</extension>
+        <mime-type>application/x-iso9660-image</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>itp</extension>
+        <mime-type>application/vnd.shana.informed.formtemplate</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>ivp</extension>
+        <mime-type>application/vnd.immervision-ivp</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>ivu</extension>
+        <mime-type>application/vnd.immervision-ivu</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>jad</extension>
+        <mime-type>text/vnd.sun.j2me.app-descriptor</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>jam</extension>
+        <mime-type>application/vnd.jam</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>jar</extension>
+        <mime-type>application/java-archive</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>java</extension>
+        <mime-type>text/x-java-source</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>jisp</extension>
+        <mime-type>application/vnd.jisp</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>jlt</extension>
+        <mime-type>application/vnd.hp-jlyt</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>jnlp</extension>
+        <mime-type>application/x-java-jnlp-file</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>joda</extension>
+        <mime-type>application/vnd.joost.joda-archive</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>jpe</extension>
+        <mime-type>image/jpeg</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>jpeg</extension>
+        <mime-type>image/jpeg</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>jpg</extension>
+        <mime-type>image/jpeg</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>jpgm</extension>
+        <mime-type>video/jpm</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>jpgv</extension>
+        <mime-type>video/jpeg</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>jpm</extension>
+        <mime-type>video/jpm</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>js</extension>
+        <mime-type>application/javascript</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>jsf</extension>
+        <mime-type>text/plain</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>json</extension>
+        <mime-type>application/json</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>jsonml</extension>
+        <mime-type>application/jsonml+json</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>jspf</extension>
+        <mime-type>text/plain</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>kar</extension>
+        <mime-type>audio/midi</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>karbon</extension>
+        <mime-type>application/vnd.kde.karbon</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>kfo</extension>
+        <mime-type>application/vnd.kde.kformula</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>kia</extension>
+        <mime-type>application/vnd.kidspiration</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>kml</extension>
+        <mime-type>application/vnd.google-earth.kml+xml</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>kmz</extension>
+        <mime-type>application/vnd.google-earth.kmz</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>kne</extension>
+        <mime-type>application/vnd.kinar</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>knp</extension>
+        <mime-type>application/vnd.kinar</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>kon</extension>
+        <mime-type>application/vnd.kde.kontour</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>kpr</extension>
+        <mime-type>application/vnd.kde.kpresenter</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>kpt</extension>
+        <mime-type>application/vnd.kde.kpresenter</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>kpxx</extension>
+        <mime-type>application/vnd.ds-keypoint</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>ksp</extension>
+        <mime-type>application/vnd.kde.kspread</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>ktr</extension>
+        <mime-type>application/vnd.kahootz</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>ktx</extension>
+        <mime-type>image/ktx</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>ktz</extension>
+        <mime-type>application/vnd.kahootz</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>kwd</extension>
+        <mime-type>application/vnd.kde.kword</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>kwt</extension>
+        <mime-type>application/vnd.kde.kword</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>lasxml</extension>
+        <mime-type>application/vnd.las.las+xml</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>latex</extension>
+        <mime-type>application/x-latex</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>lbd</extension>
+        <mime-type>application/vnd.llamagraphics.life-balance.desktop</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>lbe</extension>
+        <mime-type>application/vnd.llamagraphics.life-balance.exchange+xml</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>les</extension>
+        <mime-type>application/vnd.hhe.lesson-player</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>lha</extension>
+        <mime-type>application/x-lzh-compressed</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>link66</extension>
+        <mime-type>application/vnd.route66.link66+xml</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>list</extension>
+        <mime-type>text/plain</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>list3820</extension>
+        <mime-type>application/vnd.ibm.modcap</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>listafp</extension>
+        <mime-type>application/vnd.ibm.modcap</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>lnk</extension>
+        <mime-type>application/x-ms-shortcut</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>log</extension>
+        <mime-type>text/plain</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>lostxml</extension>
+        <mime-type>application/lost+xml</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>lrf</extension>
+        <mime-type>application/octet-stream</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>lrm</extension>
+        <mime-type>application/vnd.ms-lrm</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>ltf</extension>
+        <mime-type>application/vnd.frogans.ltf</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>lvp</extension>
+        <mime-type>audio/vnd.lucent.voice</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>lwp</extension>
+        <mime-type>application/vnd.lotus-wordpro</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>lzh</extension>
+        <mime-type>application/x-lzh-compressed</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>m13</extension>
+        <mime-type>application/x-msmediaview</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>m14</extension>
+        <mime-type>application/x-msmediaview</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>m1v</extension>
+        <mime-type>video/mpeg</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>m21</extension>
+        <mime-type>application/mp21</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>m2a</extension>
+        <mime-type>audio/mpeg</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>m2v</extension>
+        <mime-type>video/mpeg</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>m3a</extension>
+        <mime-type>audio/mpeg</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>m3u</extension>
+        <mime-type>audio/x-mpegurl</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>m3u8</extension>
+        <mime-type>application/vnd.apple.mpegurl</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>m4a</extension>
+        <mime-type>audio/mp4</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>m4b</extension>
+        <mime-type>audio/mp4</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>m4r</extension>
+        <mime-type>audio/mp4</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>m4u</extension>
+        <mime-type>video/vnd.mpegurl</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>m4v</extension>
+        <mime-type>video/mp4</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>ma</extension>
+        <mime-type>application/mathematica</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>mac</extension>
+        <mime-type>image/x-macpaint</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>mads</extension>
+        <mime-type>application/mads+xml</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>mag</extension>
+        <mime-type>application/vnd.ecowin.chart</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>maker</extension>
+        <mime-type>application/vnd.framemaker</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>man</extension>
+        <mime-type>text/troff</mime-type>
+    </mime-mapping>
+    <mime-mapping>
+        <extension>mar</extension>
+        <mime-type>application/octet-stream</mime-type>
+    </mime-mapping>
+ 

<TRUNCATED>

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

Posted by lu...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/dictionary/src/main/java/com/kylinolap/dict/NumberDictionary.java
----------------------------------------------------------------------
diff --git a/dictionary/src/main/java/com/kylinolap/dict/NumberDictionary.java b/dictionary/src/main/java/com/kylinolap/dict/NumberDictionary.java
new file mode 100644
index 0000000..fb3c2c1
--- /dev/null
+++ b/dictionary/src/main/java/com/kylinolap/dict/NumberDictionary.java
@@ -0,0 +1,181 @@
+/*
+ * 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;
+
+/**
+ * @author yangli9
+ * 
+ */
+public class NumberDictionary<T> extends TrieDictionary<T> {
+
+    public static final int MAX_DIGITS_BEFORE_DECIMAL_POINT = 16;
+
+    // encode a number into an order preserving byte sequence
+    // for positives -- padding '0'
+    // for negatives -- '-' sign, padding '9', invert digits, and terminate by ';'
+    static class NumberBytesCodec {
+
+        byte[] buf = new byte[MAX_DIGITS_BEFORE_DECIMAL_POINT * 2];
+        int bufOffset = 0;
+        int bufLen = 0;
+
+        void encodeNumber(byte[] value, int offset, int len) {
+            if (len == 0) {
+                bufOffset = 0;
+                bufLen = 0;
+                return;
+            }
+
+            if (len > buf.length) {
+                throw new IllegalArgumentException("Too many digits for NumberDictionary: " + Bytes.toString(value, offset, len) + ". Internal buffer is only " + buf.length + " bytes");
+            }
+
+            boolean negative = value[offset] == '-';
+
+            // terminate negative ';'
+            int start = buf.length - len;
+            int end = buf.length;
+            if (negative) {
+                start--;
+                end--;
+                buf[end] = ';';
+            }
+
+            // copy & find decimal point
+            int decimalPoint = end;
+            for (int i = start, j = offset; i < end; i++, j++) {
+                buf[i] = value[j];
+                if (buf[i] == '.' && i < decimalPoint) {
+                    decimalPoint = i;
+                }
+            }
+            // remove '-' sign
+            if (negative) {
+                start++;
+            }
+
+            // prepend '0'
+            int nZeroPadding = MAX_DIGITS_BEFORE_DECIMAL_POINT - (decimalPoint - start);
+            if (nZeroPadding < 0 || nZeroPadding + 1 > start)
+                throw new IllegalArgumentException("Too many digits for NumberDictionary: " + Bytes.toString(value, offset, len) + ". Expect " + MAX_DIGITS_BEFORE_DECIMAL_POINT + " digits before decimal point at max.");
+            for (int i = 0; i < nZeroPadding; i++) {
+                buf[--start] = '0';
+            }
+
+            // consider negative
+            if (negative) {
+                buf[--start] = '-';
+                for (int i = start + 1; i < buf.length; i++) {
+                    int c = buf[i];
+                    if (c >= '0' && c <= '9') {
+                        buf[i] = (byte) ('9' - (c - '0'));
+                    }
+                }
+            } else {
+                buf[--start] = '0';
+            }
+
+            bufOffset = start;
+            bufLen = buf.length - start;
+        }
+
+        int decodeNumber(byte[] returnValue, int offset) {
+            if (bufLen == 0) {
+                return 0;
+            }
+
+            int in = bufOffset;
+            int end = bufOffset + bufLen;
+            int out = offset;
+
+            // sign
+            boolean negative = buf[in] == '-';
+            if (negative) {
+                returnValue[out++] = '-';
+                in++;
+                end--;
+            }
+
+            // remove padding
+            byte padding = (byte) (negative ? '9' : '0');
+            for (; in < end; in++) {
+                if (buf[in] != padding)
+                    break;
+            }
+
+            // all paddings before '.', special case for '0'
+            if (in == end || !(buf[in] >= '0' && buf[in] <= '9')) {
+                returnValue[out++] = '0';
+            }
+
+            // copy the rest
+            if (negative) {
+                for (; in < end; in++, out++) {
+                    int c = buf[in];
+                    if (c >= '0' && c <= '9') {
+                        c = '9' - (c - '0');
+                    }
+                    returnValue[out] = (byte) c;
+                }
+            } else {
+                System.arraycopy(buf, in, returnValue, out, end - in);
+                out += end - in;
+            }
+
+            return out - offset;
+        }
+    }
+
+    static ThreadLocal<NumberBytesCodec> localCodec = new ThreadLocal<NumberBytesCodec>();
+
+    // ============================================================================
+
+    public NumberDictionary() { // default constructor for Writable interface
+        super();
+    }
+
+    public NumberDictionary(byte[] trieBytes) {
+        super(trieBytes);
+    }
+
+    private NumberBytesCodec getCodec() {
+        NumberBytesCodec codec = localCodec.get();
+        if (codec == null) {
+            codec = new NumberBytesCodec();
+            localCodec.set(codec);
+        }
+        return codec;
+    }
+
+    @Override
+    protected int getIdFromValueBytesImpl(byte[] value, int offset, int len, int roundingFlag) {
+        NumberBytesCodec codec = getCodec();
+        codec.encodeNumber(value, offset, len);
+        return super.getIdFromValueBytesImpl(codec.buf, codec.bufOffset, codec.bufLen, roundingFlag);
+    }
+
+    @Override
+    protected int getValueBytesFromIdImpl(int id, byte[] returnValue, int offset) {
+        NumberBytesCodec codec = getCodec();
+        codec.bufOffset = 0;
+        codec.bufLen = super.getValueBytesFromIdImpl(id, codec.buf, 0);
+        return codec.decodeNumber(returnValue, offset);
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/dictionary/src/main/java/com/kylinolap/dict/NumberDictionaryBuilder.java
----------------------------------------------------------------------
diff --git a/dictionary/src/main/java/com/kylinolap/dict/NumberDictionaryBuilder.java b/dictionary/src/main/java/com/kylinolap/dict/NumberDictionaryBuilder.java
new file mode 100644
index 0000000..9f0c931
--- /dev/null
+++ b/dictionary/src/main/java/com/kylinolap/dict/NumberDictionaryBuilder.java
@@ -0,0 +1,46 @@
+/*
+ * 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;
+
+/**
+ * @author yangli9
+ * 
+ */
+public class NumberDictionaryBuilder<T> extends TrieDictionaryBuilder<T> {
+
+    NumberDictionary.NumberBytesCodec codec = new NumberDictionary.NumberBytesCodec();
+
+    public NumberDictionaryBuilder(BytesConverter<T> bytesConverter) {
+        super(bytesConverter);
+    }
+
+    @Override
+    public void addValue(byte[] value) {
+        codec.encodeNumber(value, 0, value.length);
+        byte[] copy = Bytes.copy(codec.buf, codec.bufOffset, codec.bufLen);
+        super.addValue(copy);
+    }
+
+    public NumberDictionary<T> build(int baseId) {
+        byte[] trieBytes = buildTrieBytes(baseId);
+        NumberDictionary<T> r = new NumberDictionary<T>(trieBytes);
+        return r;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/dictionary/src/main/java/com/kylinolap/dict/StringBytesConverter.java
----------------------------------------------------------------------
diff --git a/dictionary/src/main/java/com/kylinolap/dict/StringBytesConverter.java b/dictionary/src/main/java/com/kylinolap/dict/StringBytesConverter.java
new file mode 100644
index 0000000..f99b750
--- /dev/null
+++ b/dictionary/src/main/java/com/kylinolap/dict/StringBytesConverter.java
@@ -0,0 +1,32 @@
+/*
+ * 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;
+
+public class StringBytesConverter implements BytesConverter<String> {
+
+    @Override
+    public byte[] convertToBytes(String v) {
+        return Bytes.toBytes(v);
+    }
+
+    @Override
+    public String convertFromBytes(byte[] b, int offset, int length) {
+        return Bytes.toString(b, offset, length);
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/dictionary/src/main/java/com/kylinolap/dict/TrieDictionary.java
----------------------------------------------------------------------
diff --git a/dictionary/src/main/java/com/kylinolap/dict/TrieDictionary.java b/dictionary/src/main/java/com/kylinolap/dict/TrieDictionary.java
new file mode 100644
index 0000000..c348dbb
--- /dev/null
+++ b/dictionary/src/main/java/com/kylinolap/dict/TrieDictionary.java
@@ -0,0 +1,478 @@
+/*
+ * 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 java.io.ByteArrayInputStream;
+import java.io.DataInput;
+import java.io.DataInputStream;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.io.PrintStream;
+import java.lang.ref.SoftReference;
+import java.util.Arrays;
+import java.util.HashMap;
+
+import com.kylinolap.common.util.BytesUtil;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * A dictionary based on Trie data structure that maps enumerations of byte[] to
+ * int IDs.
+ * 
+ * With Trie the memory footprint of the mapping is kinda minimized at the cost
+ * CPU, if compared to HashMap of ID Arrays. Performance test shows Trie is
+ * roughly 10 times slower, so there's a cache layer overlays on top of Trie and
+ * gracefully fall back to Trie using a weak reference.
+ * 
+ * The implementation is thread-safe.
+ * 
+ * @author yangli9
+ */
+@SuppressWarnings({ "rawtypes", "unchecked" })
+public class TrieDictionary<T> extends Dictionary<T> {
+
+    public static final byte[] HEAD_MAGIC = new byte[] { 0x54, 0x72, 0x69, 0x65, 0x44, 0x69, 0x63, 0x74 }; // "TrieDict"
+    public static final int HEAD_SIZE_I = HEAD_MAGIC.length;
+
+    public static final int BIT_IS_LAST_CHILD = 0x80;
+    public static final int BIT_IS_END_OF_VALUE = 0x40;
+
+    private static final Logger logger = LoggerFactory.getLogger(TrieDictionary.class);
+
+    private byte[] trieBytes;
+
+    // non-persistent part
+    transient private int headSize;
+    @SuppressWarnings("unused")
+    transient private int bodyLen;
+    transient private int sizeChildOffset;
+    transient private int sizeNoValuesBeneath;
+    transient private int baseId;
+    transient private int maxValueLength;
+    transient private BytesConverter<T> bytesConvert;
+
+    transient private int nValues;
+    transient private int sizeOfId;
+    transient private int childOffsetMask;
+    transient private int firstByteOffset;
+
+    transient private boolean enableCache = true;
+    transient private SoftReference<HashMap> valueToIdCache;
+    transient private SoftReference<Object[]> idToValueCache;
+
+    public TrieDictionary() { // default constructor for Writable interface
+    }
+
+    public TrieDictionary(byte[] trieBytes) {
+        init(trieBytes);
+    }
+
+    private void init(byte[] trieBytes) {
+        this.trieBytes = trieBytes;
+        if (BytesUtil.compareBytes(HEAD_MAGIC, 0, trieBytes, 0, HEAD_MAGIC.length) != 0)
+            throw new IllegalArgumentException("Wrong file type (magic does not match)");
+
+        try {
+            DataInputStream headIn = new DataInputStream( //
+                    new ByteArrayInputStream(trieBytes, HEAD_SIZE_I, trieBytes.length - HEAD_SIZE_I));
+            this.headSize = headIn.readShort();
+            this.bodyLen = headIn.readInt();
+            this.sizeChildOffset = headIn.read();
+            this.sizeNoValuesBeneath = headIn.read();
+            this.baseId = headIn.readShort();
+            this.maxValueLength = headIn.readShort();
+
+            String converterName = headIn.readUTF();
+            if (converterName.isEmpty() == false)
+                this.bytesConvert = (BytesConverter<T>) Class.forName(converterName).newInstance();
+
+            this.nValues = BytesUtil.readUnsigned(trieBytes, headSize + sizeChildOffset, sizeNoValuesBeneath);
+            this.sizeOfId = BytesUtil.sizeForValue(baseId + nValues + 1); // note
+                                                                          // baseId
+                                                                          // could
+                                                                          // raise
+                                                                          // 1
+                                                                          // byte
+                                                                          // in
+                                                                          // ID
+                                                                          // space,
+                                                                          // +1
+                                                                          // to
+                                                                          // reserve
+                                                                          // all
+                                                                          // 0xFF
+                                                                          // for
+                                                                          // NULL
+                                                                          // case
+            this.childOffsetMask = ~((BIT_IS_LAST_CHILD | BIT_IS_END_OF_VALUE) << ((sizeChildOffset - 1) * 8));
+            this.firstByteOffset = sizeChildOffset + sizeNoValuesBeneath + 1; // the
+                                                                              // offset
+                                                                              // from
+                                                                              // begin
+                                                                              // of
+                                                                              // node
+                                                                              // to
+                                                                              // its
+                                                                              // first
+                                                                              // value
+                                                                              // byte
+        } catch (Exception e) {
+            if (e instanceof RuntimeException)
+                throw (RuntimeException) e;
+            else
+                throw new RuntimeException(e);
+        }
+
+        if (enableCache) {
+            valueToIdCache = new SoftReference<HashMap>(new HashMap());
+            idToValueCache = new SoftReference<Object[]>(new Object[nValues]);
+        }
+    }
+
+    @Override
+    public int getMinId() {
+        return baseId;
+    }
+
+    @Override
+    public int getMaxId() {
+        return baseId + nValues - 1;
+    }
+
+    @Override
+    public int getSizeOfId() {
+        return sizeOfId;
+    }
+
+    @Override
+    public int getSizeOfValue() {
+        return maxValueLength;
+    }
+
+    @Override
+    final protected int getIdFromValueImpl(T value, int roundingFlag) {
+        if (enableCache && roundingFlag == 0) {
+            HashMap cache = valueToIdCache.get(); // SoftReference to skip cache
+                                                  // gracefully when short of
+                                                  // memory
+            if (cache != null) {
+                Integer id = null;
+                id = (Integer) cache.get(value);
+                if (id != null)
+                    return id.intValue();
+
+                byte[] valueBytes = bytesConvert.convertToBytes(value);
+                id = getIdFromValueBytes(valueBytes, 0, valueBytes.length, roundingFlag);
+
+                cache.put(value, id);
+                return id;
+            }
+        }
+        byte[] valueBytes = bytesConvert.convertToBytes(value);
+        return getIdFromValueBytes(valueBytes, 0, valueBytes.length, roundingFlag);
+    }
+
+    @Override
+    protected int getIdFromValueBytesImpl(byte[] value, int offset, int len, int roundingFlag) {
+        int seq = lookupSeqNoFromValue(headSize, value, offset, offset + len, roundingFlag);
+        int id = calcIdFromSeqNo(seq);
+        if (id < 0)
+            throw new IllegalArgumentException("Not a valid value: " + bytesConvert.convertFromBytes(value, offset, len));
+        return id;
+    }
+
+    /**
+     * returns a code point from [0, nValues), preserving order of value
+     * 
+     * @param n
+     *            -- the offset of current node
+     * @param inp
+     *            -- input value bytes to lookup
+     * @param o
+     *            -- offset in the input value bytes matched so far
+     * @param inpEnd
+     *            -- end of input
+     * @param roundingFlag
+     *            -- =0: return -1 if not found -- <0: return closest smaller if
+     *            not found, might be -1 -- >0: return closest bigger if not
+     *            found, might be nValues
+     */
+    private int lookupSeqNoFromValue(int n, byte[] inp, int o, int inpEnd, int roundingFlag) {
+        if (inp.length == 0) // special 'empty' value
+            return checkFlag(headSize, BIT_IS_END_OF_VALUE) ? 0 : roundSeqNo(roundingFlag, -1, -1, 0);
+
+        int seq = 0; // the sequence no under track
+
+        while (true) {
+            // match the current node, note [0] of node's value has been matched
+            // when this node is selected by its parent
+            int p = n + firstByteOffset; // start of node's value
+            int end = p + BytesUtil.readUnsigned(trieBytes, p - 1, 1); // end of
+                                                                       // node's
+                                                                       // value
+            for (p++; p < end && o < inpEnd; p++, o++) { // note matching start
+                                                         // from [1]
+                if (trieBytes[p] != inp[o]) {
+                    int comp = BytesUtil.compareByteUnsigned(trieBytes[p], inp[o]);
+                    if (comp < 0) {
+                        seq += BytesUtil.readUnsigned(trieBytes, n + sizeChildOffset, sizeNoValuesBeneath);
+                    }
+                    return roundSeqNo(roundingFlag, seq - 1, -1, seq); // mismatch
+                }
+            }
+
+            // node completely matched, is input all consumed?
+            boolean isEndOfValue = checkFlag(n, BIT_IS_END_OF_VALUE);
+            if (o == inpEnd) {
+                return p == end && isEndOfValue ? seq : roundSeqNo(roundingFlag, seq - 1, -1, seq); // input
+                                                                                                    // all
+                                                                                                    // matched
+            }
+            if (isEndOfValue)
+                seq++;
+
+            // find a child to continue
+            int c = headSize + (BytesUtil.readUnsigned(trieBytes, n, sizeChildOffset) & childOffsetMask);
+            if (c == headSize) // has no children
+                return roundSeqNo(roundingFlag, seq - 1, -1, seq); // input only
+                                                                   // partially
+                                                                   // matched
+            byte inpByte = inp[o];
+            int comp;
+            while (true) {
+                p = c + firstByteOffset;
+                comp = BytesUtil.compareByteUnsigned(trieBytes[p], inpByte);
+                if (comp == 0) { // continue in the matching child, reset n and
+                                 // loop again
+                    n = c;
+                    o++;
+                    break;
+                } else if (comp < 0) { // try next child
+                    seq += BytesUtil.readUnsigned(trieBytes, c + sizeChildOffset, sizeNoValuesBeneath);
+                    if (checkFlag(c, BIT_IS_LAST_CHILD))
+                        return roundSeqNo(roundingFlag, seq - 1, -1, seq); // no
+                                                                           // child
+                                                                           // can
+                                                                           // match
+                                                                           // the
+                                                                           // next
+                                                                           // byte
+                                                                           // of
+                                                                           // input
+                    c = p + BytesUtil.readUnsigned(trieBytes, p - 1, 1);
+                } else { // children are ordered by their first value byte
+                    return roundSeqNo(roundingFlag, seq - 1, -1, seq); // no
+                                                                       // child
+                                                                       // can
+                                                                       // match
+                                                                       // the
+                                                                       // next
+                                                                       // byte
+                                                                       // of
+                                                                       // input
+                }
+            }
+        }
+    }
+
+    private int roundSeqNo(int roundingFlag, int i, int j, int k) {
+        if (roundingFlag == 0)
+            return j;
+        else if (roundingFlag < 0)
+            return i;
+        else
+            return k;
+    }
+
+    @Override
+    final protected T getValueFromIdImpl(int id) {
+        if (enableCache) {
+            Object[] cache = idToValueCache.get(); // SoftReference to skip
+                                                   // cache gracefully when
+                                                   // short of memory
+            if (cache != null) {
+                int seq = calcSeqNoFromId(id);
+                if (seq < 0 || seq >= nValues)
+                    throw new IllegalArgumentException("Not a valid ID: " + id);
+                if (cache[seq] != null)
+                    return (T) cache[seq];
+
+                byte[] value = new byte[getSizeOfValue()];
+                int length = getValueBytesFromId(id, value, 0);
+                T result = bytesConvert.convertFromBytes(value, 0, length);
+
+                cache[seq] = result;
+                return result;
+            }
+        }
+        byte[] value = new byte[getSizeOfValue()];
+        int length = getValueBytesFromId(id, value, 0);
+        return bytesConvert.convertFromBytes(value, 0, length);
+    }
+
+    @Override
+    protected int getValueBytesFromIdImpl(int id, byte[] returnValue, int offset) {
+        if (id < baseId || id >= baseId + nValues)
+            throw new IllegalArgumentException("Not a valid ID: " + id);
+
+        int seq = calcSeqNoFromId(id);
+
+        return lookupValueFromSeqNo(headSize, seq, returnValue, offset);
+    }
+
+    /**
+     * returns a code point from [0, nValues), preserving order of value, or -1
+     * if not found
+     * 
+     * @param n
+     *            -- the offset of current node
+     * @param seq
+     *            -- the code point under track
+     * @param returnValue
+     *            -- where return value is written to
+     */
+    private int lookupValueFromSeqNo(int n, int seq, byte[] returnValue, int offset) {
+        int o = offset;
+        while (true) {
+            // write current node value
+            int p = n + firstByteOffset;
+            int len = BytesUtil.readUnsigned(trieBytes, p - 1, 1);
+            System.arraycopy(trieBytes, p, returnValue, o, len);
+            o += len;
+
+            // if the value is ended
+            boolean isEndOfValue = checkFlag(n, BIT_IS_END_OF_VALUE);
+            if (isEndOfValue) {
+                seq--;
+                if (seq < 0)
+                    return o - offset;
+            }
+
+            // find a child to continue
+            int c = headSize + (BytesUtil.readUnsigned(trieBytes, n, sizeChildOffset) & childOffsetMask);
+            if (c == headSize) // has no children
+                return -1; // no child? corrupted dictionary!
+            int nValuesBeneath;
+            while (true) {
+                nValuesBeneath = BytesUtil.readUnsigned(trieBytes, c + sizeChildOffset, sizeNoValuesBeneath);
+                if (seq - nValuesBeneath < 0) { // value is under this child,
+                                                // reset n and loop again
+                    n = c;
+                    break;
+                } else { // go to next child
+                    seq -= nValuesBeneath;
+                    if (checkFlag(c, BIT_IS_LAST_CHILD))
+                        return -1; // no more child? corrupted dictionary!
+                    p = c + firstByteOffset;
+                    c = p + BytesUtil.readUnsigned(trieBytes, p - 1, 1);
+                }
+            }
+        }
+    }
+
+    private boolean checkFlag(int offset, int bit) {
+        return (trieBytes[offset] & bit) > 0;
+    }
+
+    private int calcIdFromSeqNo(int seq) {
+        if (seq < 0 || seq >= nValues)
+            return -1;
+        else
+            return baseId + seq;
+    }
+
+    private int calcSeqNoFromId(int id) {
+        return id - baseId;
+    }
+
+    @Override
+    public void write(DataOutput out) throws IOException {
+        out.write(trieBytes);
+    }
+
+    @Override
+    public void readFields(DataInput in) throws IOException {
+        byte[] headPartial = new byte[HEAD_MAGIC.length + Short.SIZE + Integer.SIZE];
+        in.readFully(headPartial);
+
+        if (BytesUtil.compareBytes(HEAD_MAGIC, 0, headPartial, 0, HEAD_MAGIC.length) != 0)
+            throw new IllegalArgumentException("Wrong file type (magic does not match)");
+
+        DataInputStream headIn = new DataInputStream( //
+                new ByteArrayInputStream(headPartial, HEAD_SIZE_I, headPartial.length - HEAD_SIZE_I));
+        int headSize = headIn.readShort();
+        int bodyLen = headIn.readInt();
+        headIn.close();
+
+        byte[] all = new byte[headSize + bodyLen];
+        System.arraycopy(headPartial, 0, all, 0, headPartial.length);
+        in.readFully(all, headPartial.length, all.length - headPartial.length);
+
+        init(all);
+    }
+
+    @Override
+    public void dump(PrintStream out) {
+        out.println("Total " + nValues + " values");
+        for (int i = 0; i < nValues; i++) {
+            int id = calcIdFromSeqNo(i);
+            T value = getValueFromId(id);
+            out.println(id + " (" + Integer.toHexString(id) + "): " + value);
+        }
+    }
+
+    @Override
+    public int hashCode() {
+        return Arrays.hashCode(trieBytes);
+    }
+
+    @Override
+    public boolean equals(Object o) {
+        if ((o instanceof TrieDictionary) == false) {
+            logger.info("Equals return false because o is not TrieDictionary");
+            return false;
+        }
+        TrieDictionary that = (TrieDictionary) o;
+        return Arrays.equals(this.trieBytes, that.trieBytes);
+    }
+
+    public static void main(String[] args) throws Exception {
+        TrieDictionaryBuilder<String> b = new TrieDictionaryBuilder<String>(new StringBytesConverter());
+        // b.addValue("part");
+        // b.print();
+        // b.addValue("part");
+        // b.print();
+        // b.addValue("par");
+        // b.print();
+        // b.addValue("partition");
+        // b.print();
+        // b.addValue("party");
+        // b.print();
+        // b.addValue("parties");
+        // b.print();
+        // b.addValue("paint");
+        // b.print();
+        b.addValue("-000000.41");
+        b.addValue("0000101.81");
+        b.addValue("6779331");
+        String t = "0000001.6131";
+        TrieDictionary<String> dict = b.build(0);
+
+        System.out.println(dict.getIdFromValue(t, -1));
+        System.out.println(dict.getIdFromValue(t, 1));
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/dictionary/src/main/java/com/kylinolap/dict/TrieDictionaryBuilder.java
----------------------------------------------------------------------
diff --git a/dictionary/src/main/java/com/kylinolap/dict/TrieDictionaryBuilder.java b/dictionary/src/main/java/com/kylinolap/dict/TrieDictionaryBuilder.java
new file mode 100644
index 0000000..9a6e2ce
--- /dev/null
+++ b/dictionary/src/main/java/com/kylinolap/dict/TrieDictionaryBuilder.java
@@ -0,0 +1,536 @@
+/*
+ * 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 com.kylinolap.common.util.BytesUtil.*;
+import static com.kylinolap.dict.TrieDictionary.*;
+
+import java.io.ByteArrayOutputStream;
+import java.io.DataOutputStream;
+import java.io.IOException;
+import java.io.PrintStream;
+import java.io.UnsupportedEncodingException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.IdentityHashMap;
+import java.util.LinkedList;
+
+import com.kylinolap.common.util.BytesUtil;
+
+/**
+ * Builds a dictionary using Trie structure. All values are taken in byte[] form
+ * and organized in a Trie with ordering. Then numeric IDs are assigned in
+ * sequence.
+ * 
+ * @author yangli9
+ */
+public class TrieDictionaryBuilder<T> {
+
+    public static class Node {
+        public byte[] part;
+        public boolean isEndOfValue;
+        public ArrayList<Node> children;
+
+        public int nValuesBeneath; // only present after stats()
+
+        Node(byte[] value, boolean isEndOfValue) {
+            reset(value, isEndOfValue);
+        }
+
+        Node(byte[] value, boolean isEndOfValue, ArrayList<Node> children) {
+            reset(value, isEndOfValue, children);
+        }
+
+        void reset(byte[] value, boolean isEndOfValue) {
+            reset(value, isEndOfValue, new ArrayList<Node>());
+        }
+
+        void reset(byte[] value, boolean isEndOfValue, ArrayList<Node> children) {
+            this.part = value;
+            this.isEndOfValue = isEndOfValue;
+            this.children = children;
+        }
+    }
+
+    public static interface Visitor {
+        void visit(Node n, int level);
+    }
+
+    // ============================================================================
+
+    private Node root;
+    private BytesConverter<T> bytesConverter;
+
+    public TrieDictionaryBuilder(BytesConverter<T> bytesConverter) {
+        this.root = new Node(new byte[0], false);
+        this.bytesConverter = bytesConverter;
+    }
+
+    public void addValue(T value) {
+        addValue(bytesConverter.convertToBytes(value));
+    }
+
+    public void addValue(byte[] value) {
+        addValueR(root, value, 0);
+    }
+
+    private void addValueR(Node node, byte[] value, int start) {
+        // match the value part of current node
+        int i = 0, j = start;
+        int n = node.part.length, nn = value.length;
+        int comp = 0;
+        for (; i < n && j < nn; i++, j++) {
+            comp = compareByteUnsigned(node.part[i], value[j]);
+            if (comp != 0)
+                break;
+        }
+
+        // if value fully matched within the current node
+        if (j == nn) {
+            // if equals to current node, just mark end of value
+            if (i == n) {
+                node.isEndOfValue = true;
+            }
+            // otherwise, split the current node into two
+            else {
+                Node c = new Node(subarray(node.part, i, n), node.isEndOfValue, node.children);
+                node.reset(subarray(node.part, 0, i), true);
+                node.children.add(c);
+            }
+            return;
+        }
+
+        // if partially matched the current, split the current node, add the new
+        // value, make a 3-way
+        if (i < n) {
+            Node c1 = new Node(subarray(node.part, i, n), node.isEndOfValue, node.children);
+            Node c2 = new Node(subarray(value, j, nn), true);
+            node.reset(subarray(node.part, 0, i), false);
+            if (comp < 0) {
+                node.children.add(c1);
+                node.children.add(c2);
+            } else {
+                node.children.add(c2);
+                node.children.add(c1);
+            }
+            return;
+        }
+
+        // out matched the current, binary search the next byte for a child node
+        // to continue
+        byte lookfor = value[j];
+        int lo = 0;
+        int hi = node.children.size() - 1;
+        int mid = 0;
+        boolean found = false;
+        comp = 0;
+        while (!found && lo <= hi) {
+            mid = lo + (hi - lo) / 2;
+            comp = compareByteUnsigned(lookfor, node.children.get(mid).part[0]);
+            if (comp < 0)
+                hi = mid - 1;
+            else if (comp > 0)
+                lo = mid + 1;
+            else
+                found = true;
+        }
+        // found a child node matching the first byte, continue in that child
+        if (found) {
+            addValueR(node.children.get(mid), value, j);
+        }
+        // otherwise, make the value a new child
+        else {
+            Node c = new Node(subarray(value, j, nn), true);
+            node.children.add(comp <= 0 ? mid : mid + 1, c);
+        }
+    }
+
+    public void traverse(Visitor visitor) {
+        traverseR(root, visitor, 0);
+    }
+
+    private void traverseR(Node node, Visitor visitor, int level) {
+        visitor.visit(node, level);
+        for (Node c : node.children)
+            traverseR(c, visitor, level + 1);
+    }
+
+    public void traversePostOrder(Visitor visitor) {
+        traversePostOrderR(root, visitor, 0);
+    }
+
+    private void traversePostOrderR(Node node, Visitor visitor, int level) {
+        for (Node c : node.children)
+            traversePostOrderR(c, visitor, level + 1);
+        visitor.visit(node, level);
+    }
+
+    public static class Stats {
+        public int nValues; // number of values in total
+        public int nValueBytesPlain; // number of bytes for all values
+                                     // uncompressed
+        public int nValueBytesCompressed; // number of values bytes in Trie
+                                          // (compressed)
+        public int maxValueLength; // size of longest value in bytes
+
+        // the trie is multi-byte-per-node
+        public int mbpn_nNodes; // number of nodes in trie
+        public int mbpn_trieDepth; // depth of trie
+        public int mbpn_maxFanOut; // the maximum no. children
+        public int mbpn_nChildLookups; // number of child lookups during lookup
+                                       // every value once
+        public int mbpn_nTotalFanOut; // the sum of fan outs during lookup every
+                                      // value once
+        public int mbpn_sizeValueTotal; // the sum of value space in all nodes
+        public int mbpn_sizeNoValueBytes; // size of field noValueBytes
+        public int mbpn_sizeNoValueBeneath; // size of field noValuesBeneath,
+                                            // depends on cardinality
+        public int mbpn_sizeChildOffset; // size of field childOffset, points to
+                                         // first child in flattened array
+        public int mbpn_footprint; // MBPN footprint in bytes
+
+        // stats for one-byte-per-node as well, so there's comparison
+        public int obpn_sizeValue; // size of value per node, always 1
+        public int obpn_sizeNoValuesBeneath; // size of field noValuesBeneath,
+                                             // depends on cardinality
+        public int obpn_sizeChildCount; // size of field childCount, enables
+                                        // binary search among children
+        public int obpn_sizeChildOffset; // size of field childOffset, points to
+                                         // first child in flattened array
+        public int obpn_nNodes; // no. nodes in OBPN trie
+        public int obpn_footprint; // OBPN footprint in bytes
+
+        public void print() {
+            PrintStream out = System.out;
+            out.println("============================================================================");
+            out.println("No. values:             " + nValues);
+            out.println("No. bytes raw:          " + nValueBytesPlain);
+            out.println("No. bytes in trie:      " + nValueBytesCompressed);
+            out.println("Longest value length:   " + maxValueLength);
+
+            // flatten trie footprint calculation, case of One-Byte-Per-Node
+            out.println("----------------------------------------------------------------------------");
+            out.println("OBPN node size:  " + (obpn_sizeValue + obpn_sizeNoValuesBeneath + obpn_sizeChildCount + obpn_sizeChildOffset) + " = " + obpn_sizeValue + " + " + obpn_sizeNoValuesBeneath + " + " + obpn_sizeChildCount + " + " + obpn_sizeChildOffset);
+            out.println("OBPN no. nodes:  " + obpn_nNodes);
+            out.println("OBPN trie depth: " + maxValueLength);
+            out.println("OBPN footprint:  " + obpn_footprint + " in bytes");
+
+            // flatten trie footprint calculation, case of Multi-Byte-Per-Node
+            out.println("----------------------------------------------------------------------------");
+            out.println("MBPN max fan out:       " + mbpn_maxFanOut);
+            out.println("MBPN no. child lookups: " + mbpn_nChildLookups);
+            out.println("MBPN total fan out:     " + mbpn_nTotalFanOut);
+            out.println("MBPN average fan out:   " + (double) mbpn_nTotalFanOut / mbpn_nChildLookups);
+            out.println("MBPN values size total: " + mbpn_sizeValueTotal);
+            out.println("MBPN node size:         " + (mbpn_sizeNoValueBytes + mbpn_sizeNoValueBeneath + mbpn_sizeChildOffset) + " = " + mbpn_sizeNoValueBytes + " + " + mbpn_sizeNoValueBeneath + " + " + mbpn_sizeChildOffset);
+            out.println("MBPN no. nodes:         " + mbpn_nNodes);
+            out.println("MBPN trie depth:        " + mbpn_trieDepth);
+            out.println("MBPN footprint:         " + mbpn_footprint + " in bytes");
+        }
+    }
+
+    /** out print some statistics of the trie and the dictionary built from it */
+    public Stats stats() {
+        // calculate nEndValueBeneath
+        traversePostOrder(new Visitor() {
+            @Override
+            public void visit(Node n, int level) {
+                n.nValuesBeneath = n.isEndOfValue ? 1 : 0;
+                for (Node c : n.children)
+                    n.nValuesBeneath += c.nValuesBeneath;
+            }
+        });
+
+        // run stats
+        final Stats s = new Stats();
+        final ArrayList<Integer> lenAtLvl = new ArrayList<Integer>();
+        traverse(new Visitor() {
+            @Override
+            public void visit(Node n, int level) {
+                if (n.isEndOfValue)
+                    s.nValues++;
+                s.nValueBytesPlain += n.part.length * n.nValuesBeneath;
+                s.nValueBytesCompressed += n.part.length;
+                s.mbpn_nNodes++;
+                if (s.mbpn_trieDepth < level + 1)
+                    s.mbpn_trieDepth = level + 1;
+                if (n.children.size() > 0) {
+                    if (s.mbpn_maxFanOut < n.children.size())
+                        s.mbpn_maxFanOut = n.children.size();
+                    int childLookups = n.nValuesBeneath - (n.isEndOfValue ? 1 : 0);
+                    s.mbpn_nChildLookups += childLookups;
+                    s.mbpn_nTotalFanOut += childLookups * n.children.size();
+                }
+
+                if (level < lenAtLvl.size())
+                    lenAtLvl.set(level, n.part.length);
+                else
+                    lenAtLvl.add(n.part.length);
+                int lenSoFar = 0;
+                for (int i = 0; i <= level; i++)
+                    lenSoFar += lenAtLvl.get(i);
+                if (lenSoFar > s.maxValueLength)
+                    s.maxValueLength = lenSoFar;
+            }
+        });
+
+        // flatten trie footprint calculation, case of One-Byte-Per-Node
+        s.obpn_sizeValue = 1;
+        s.obpn_sizeNoValuesBeneath = BytesUtil.sizeForValue(s.nValues);
+        s.obpn_sizeChildCount = 1;
+        s.obpn_sizeChildOffset = 4; // MSB used as isEndOfValue flag
+        s.obpn_nNodes = s.nValueBytesCompressed; // no. nodes is the total
+                                                 // number of compressed
+                                                 // bytes in OBPN
+        s.obpn_footprint = s.obpn_nNodes * (s.obpn_sizeValue + s.obpn_sizeNoValuesBeneath + s.obpn_sizeChildCount + s.obpn_sizeChildOffset);
+        while (true) { // minimize the offset size to match the footprint
+            int t = s.obpn_nNodes * (s.obpn_sizeValue + s.obpn_sizeNoValuesBeneath + s.obpn_sizeChildCount + s.obpn_sizeChildOffset - 1);
+            if (BytesUtil.sizeForValue(t * 2) <= s.obpn_sizeChildOffset - 1) { // *2
+                                                                               // because
+                                                                               // MSB
+                                                                               // of
+                                                                               // offset
+                                                                               // is
+                                                                               // used
+                                                                               // for
+                                                                               // isEndOfValue
+                                                                               // flag
+                s.obpn_sizeChildOffset--;
+                s.obpn_footprint = t;
+            } else
+                break;
+        }
+
+        // flatten trie footprint calculation, case of Multi-Byte-Per-Node
+        s.mbpn_sizeValueTotal = s.nValueBytesCompressed;
+        s.mbpn_sizeNoValueBytes = 1;
+        s.mbpn_sizeNoValueBeneath = BytesUtil.sizeForValue(s.nValues);
+        s.mbpn_sizeChildOffset = 4;
+        s.mbpn_footprint = s.mbpn_sizeValueTotal + s.mbpn_nNodes * (s.mbpn_sizeNoValueBytes + s.mbpn_sizeNoValueBeneath + s.mbpn_sizeChildOffset);
+        while (true) { // minimize the offset size to match the footprint
+            int t = s.mbpn_sizeValueTotal + s.mbpn_nNodes * (s.mbpn_sizeNoValueBytes + s.mbpn_sizeNoValueBeneath + s.mbpn_sizeChildOffset - 1);
+            if (BytesUtil.sizeForValue(t * 4) <= s.mbpn_sizeChildOffset - 1) { // *4
+                                                                               // because
+                                                                               // 2
+                                                                               // MSB
+                                                                               // of
+                                                                               // offset
+                                                                               // is
+                                                                               // used
+                                                                               // for
+                                                                               // isEndOfValue
+                                                                               // &
+                                                                               // isEndChild
+                                                                               // flag
+                s.mbpn_sizeChildOffset--;
+                s.mbpn_footprint = t;
+            } else
+                break;
+        }
+
+        return s;
+    }
+
+    /** out print trie for debug */
+    public void print() {
+        print(System.out);
+    }
+
+    public void print(final PrintStream out) {
+        traverse(new Visitor() {
+            @Override
+            public void visit(Node n, int level) {
+                try {
+                    for (int i = 0; i < level; i++)
+                        out.print("  ");
+                    out.print(new String(n.part, "UTF-8"));
+                    out.print(" - ");
+                    if (n.nValuesBeneath > 0)
+                        out.print(n.nValuesBeneath);
+                    if (n.isEndOfValue)
+                        out.print("*");
+                    out.print("\n");
+                } catch (UnsupportedEncodingException e) {
+                    e.printStackTrace();
+                }
+            }
+        });
+    }
+
+    private CompleteParts completeParts = new CompleteParts();
+
+    private class CompleteParts {
+        byte[] data = new byte[4096];
+        int current = 0;
+
+        public void append(byte[] part) {
+            while (current + part.length > data.length)
+                expand();
+
+            System.arraycopy(part, 0, data, current, part.length);
+            current += part.length;
+        }
+
+        public void withdraw(int size) {
+            current -= size;
+        }
+
+        public byte[] retrieve() {
+            return Arrays.copyOf(data, current);
+        }
+
+        private void expand() {
+            byte[] temp = new byte[2 * data.length];
+            System.arraycopy(data, 0, temp, 0, data.length);
+            data = temp;
+        }
+    }
+
+    // there is a 255 limitation of length for each node's part.
+    // we interpolate nodes to satisfy this when a node's part becomes
+    // too long(overflow)
+    private void checkOverflowParts(Node node) {
+        LinkedList<Node> childrenCopy = new LinkedList<Node>(node.children);
+        for (Node child : childrenCopy) {
+            if (child.part.length > 255) {
+                byte[] first255 = Arrays.copyOf(child.part, 255);
+
+                completeParts.append(node.part);
+                completeParts.append(first255);
+                byte[] visited = completeParts.retrieve();
+                this.addValue(visited);
+                completeParts.withdraw(255);
+                completeParts.withdraw(node.part.length);
+            }
+        }
+
+        completeParts.append(node.part);// by here the node.children may have
+                                        // been changed
+        for (Node child : node.children) {
+            checkOverflowParts(child);
+        }
+        completeParts.withdraw(node.part.length);
+    }
+
+    /**
+     * Flatten the trie into a byte array for a minimized memory footprint.
+     * Lookup remains fast. Cost is inflexibility to modify (becomes immutable).
+     * 
+     * Flattened node structure is HEAD + NODEs, for each node: - o byte, offset
+     * to child node, o = stats.mbpn_sizeChildOffset - 1 bit, isLastChild flag,
+     * the 1st MSB of o - 1 bit, isEndOfValue flag, the 2nd MSB of o - c byte,
+     * number of values beneath, c = stats.mbpn_sizeNoValueBeneath - 1 byte,
+     * number of value bytes - n byte, value bytes
+     */
+    public TrieDictionary<T> build(int baseId) {
+        byte[] trieBytes = buildTrieBytes(baseId);
+        TrieDictionary<T> r = new TrieDictionary<T>(trieBytes);
+        return r;
+    }
+
+    protected byte[] buildTrieBytes(int baseId) {
+        checkOverflowParts(this.root);
+
+        Stats stats = stats();
+        int sizeNoValuesBeneath = stats.mbpn_sizeNoValueBeneath;
+        int sizeChildOffset = stats.mbpn_sizeChildOffset;
+
+        // write head
+        byte[] head;
+        try {
+            ByteArrayOutputStream byteBuf = new ByteArrayOutputStream();
+            DataOutputStream headOut = new DataOutputStream(byteBuf);
+            headOut.write(HEAD_MAGIC);
+            headOut.writeShort(0); // head size, will back fill
+            headOut.writeInt(stats.mbpn_footprint); // body size
+            headOut.write(sizeChildOffset);
+            headOut.write(sizeNoValuesBeneath);
+            headOut.writeShort(baseId);
+            headOut.writeShort(stats.maxValueLength);
+            headOut.writeUTF(bytesConverter == null ? "" : bytesConverter.getClass().getName());
+            headOut.close();
+            head = byteBuf.toByteArray();
+            BytesUtil.writeUnsigned(head.length, head, HEAD_SIZE_I, 2);
+        } catch (IOException e) {
+            throw new RuntimeException(e); // shall not happen, as we are
+                                           // writing in memory
+        }
+
+        byte[] trieBytes = new byte[stats.mbpn_footprint + head.length];
+        System.arraycopy(head, 0, trieBytes, 0, head.length);
+
+        LinkedList<Node> open = new LinkedList<Node>();
+        IdentityHashMap<Node, Integer> offsetMap = new IdentityHashMap<Node, Integer>();
+
+        // write body
+        int o = head.length;
+        offsetMap.put(root, o);
+        o = build_writeNode(root, o, true, sizeNoValuesBeneath, sizeChildOffset, trieBytes);
+        if (root.children.isEmpty() == false)
+            open.addLast(root);
+
+        while (open.isEmpty() == false) {
+            Node parent = open.removeFirst();
+            build_overwriteChildOffset(offsetMap.get(parent), o - head.length, sizeChildOffset, trieBytes);
+            for (int i = 0; i < parent.children.size(); i++) {
+                Node c = parent.children.get(i);
+                boolean isLastChild = (i == parent.children.size() - 1);
+                offsetMap.put(c, o);
+                o = build_writeNode(c, o, isLastChild, sizeNoValuesBeneath, sizeChildOffset, trieBytes);
+                if (c.children.isEmpty() == false)
+                    open.addLast(c);
+            }
+        }
+
+        if (o != trieBytes.length)
+            throw new RuntimeException();
+        return trieBytes;
+    }
+
+    private void build_overwriteChildOffset(int parentOffset, int childOffset, int sizeChildOffset, byte[] trieBytes) {
+        int flags = (int) trieBytes[parentOffset] & (BIT_IS_LAST_CHILD | BIT_IS_END_OF_VALUE);
+        BytesUtil.writeUnsigned(childOffset, trieBytes, parentOffset, sizeChildOffset);
+        trieBytes[parentOffset] |= flags;
+    }
+
+    private int build_writeNode(Node n, int offset, boolean isLastChild, int sizeNoValuesBeneath, int sizeChildOffset, byte[] trieBytes) {
+        int o = offset;
+
+        // childOffset
+        if (isLastChild)
+            trieBytes[o] |= BIT_IS_LAST_CHILD;
+        if (n.isEndOfValue)
+            trieBytes[o] |= BIT_IS_END_OF_VALUE;
+        o += sizeChildOffset;
+
+        // nValuesBeneath
+        BytesUtil.writeUnsigned(n.nValuesBeneath, trieBytes, o, sizeNoValuesBeneath);
+        o += sizeNoValuesBeneath;
+
+        // nValueBytes
+        if (n.part.length > 255)
+            throw new RuntimeException();
+        BytesUtil.writeUnsigned(n.part.length, trieBytes, o, 1);
+        o++;
+
+        // valueBytes
+        System.arraycopy(n.part, 0, trieBytes, o, n.part.length);
+        o += n.part.length;
+
+        return o;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/dictionary/src/main/java/com/kylinolap/dict/lookup/DictionaryLookupTable.java
----------------------------------------------------------------------
diff --git a/dictionary/src/main/java/com/kylinolap/dict/lookup/DictionaryLookupTable.java b/dictionary/src/main/java/com/kylinolap/dict/lookup/DictionaryLookupTable.java
new file mode 100644
index 0000000..4d7686f
--- /dev/null
+++ b/dictionary/src/main/java/com/kylinolap/dict/lookup/DictionaryLookupTable.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 java.io.IOException;
+
+import com.kylinolap.dict.Dictionary;
+import com.kylinolap.metadata.model.schema.TableDesc;
+
+/**
+ * An in-memory lookup table indexed by a dictionary column. The column value
+ * must be unique within the table.
+ * 
+ * @author yangli9
+ */
+public class DictionaryLookupTable {
+
+    private static final int MAX_CARDINALITY = 1000000;
+
+    private TableDesc tableDesc;
+    private String keyCol; // whose value must be unique across table
+    private Dictionary<String> dict;
+    private String tablePath;
+
+    private int keyColIndex;
+    private String[][] table;
+
+    public DictionaryLookupTable(TableDesc tableDesc, String keyCol, Dictionary<String> dict, String tablePath) throws IOException {
+        this.tableDesc = tableDesc;
+        this.keyCol = keyCol;
+        this.dict = dict;
+        this.tablePath = tablePath;
+        init();
+    }
+
+    private void init() throws IOException {
+        keyColIndex = tableDesc.findColumnByName(keyCol).getZeroBasedIndex();
+        table = new String[dict.getMaxId() - dict.getMinId() + 1][];
+
+        if (table.length > MAX_CARDINALITY) // 1 million
+            throw new IllegalStateException("Too high cardinality of table " + tableDesc + " as an in-mem lookup: " + table.length);
+
+        TableReader reader = new FileTable(tablePath, tableDesc.getColumnCount()).getReader();
+        try {
+            while (reader.next()) {
+                String[] cols = reader.getRow();
+                String key = cols[keyColIndex];
+                int rowNo = getRowNoByValue(key);
+
+                if (table[rowNo] != null) // dup key
+                    throw new IllegalStateException("Dup key found, key=" + key + ", value1=" + toString(table[rowNo]) + ", value2=" + toString(cols));
+
+                table[rowNo] = cols;
+            }
+        } finally {
+            reader.close();
+        }
+    }
+
+    public String[] getRow(int id) {
+        return table[getRowNoByID(id)];
+    }
+
+    public String[] getRow(String key) {
+        return table[getRowNoByValue(key)];
+    }
+
+    private int getRowNoByValue(String key) {
+        return getRowNoByID(dict.getIdFromValue(key));
+    }
+
+    private int getRowNoByID(int id) {
+        int rowNo = id - dict.getMinId();
+        return rowNo;
+    }
+
+    public void dump() {
+        for (int i = 0; i < table.length; i++) {
+            String key = dict.getValueFromId(i + dict.getMinId());
+            System.out.println(key + " => " + toString(table[i]));
+        }
+    }
+
+    private String toString(String[] cols) {
+        StringBuilder b = new StringBuilder();
+        b.append("[");
+        for (int i = 0; i < cols.length; i++) {
+            if (i > 0)
+                b.append(",");
+            b.append(cols[i]);
+        }
+        b.append("]");
+        return b.toString();
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/dictionary/src/main/java/com/kylinolap/dict/lookup/FileTable.java
----------------------------------------------------------------------
diff --git a/dictionary/src/main/java/com/kylinolap/dict/lookup/FileTable.java b/dictionary/src/main/java/com/kylinolap/dict/lookup/FileTable.java
new file mode 100644
index 0000000..f502adf
--- /dev/null
+++ b/dictionary/src/main/java/com/kylinolap/dict/lookup/FileTable.java
@@ -0,0 +1,68 @@
+/*
+ * 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 org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+
+import com.kylinolap.common.util.HadoopUtil;
+
+/**
+ * @author yangli9
+ * 
+ */
+public class FileTable implements ReadableTable {
+
+    String path;
+    String delim;
+    int nColumns;
+
+    public FileTable(String path, int nColumns) {
+        this(path, ReadableTable.DELIM_AUTO, nColumns);
+    }
+
+    public FileTable(String path, String delim, int nColumns) {
+        this.path = path;
+        this.delim = delim;
+        this.nColumns = nColumns;
+    }
+
+    @Override
+    public String getColumnDelimeter() {
+        return delim;
+    }
+
+    @Override
+    public TableReader getReader() throws IOException {
+        return new FileTableReader(path, delim, nColumns);
+    }
+
+    @Override
+    public TableSignature getSignature() throws IOException {
+        FileSystem fs = HadoopUtil.getFileSystem(path);
+        FileStatus status = fs.getFileStatus(new Path(path));
+        return new TableSignature(path, status.getLen(), status.getModificationTime());
+    }
+
+    @Override
+    public String toString() {
+        return path;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/dictionary/src/main/java/com/kylinolap/dict/lookup/FileTableReader.java
----------------------------------------------------------------------
diff --git a/dictionary/src/main/java/com/kylinolap/dict/lookup/FileTableReader.java b/dictionary/src/main/java/com/kylinolap/dict/lookup/FileTableReader.java
new file mode 100644
index 0000000..e113bc5
--- /dev/null
+++ b/dictionary/src/main/java/com/kylinolap/dict/lookup/FileTableReader.java
@@ -0,0 +1,218 @@
+/*
+ * 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.BufferedReader;
+import java.io.Closeable;
+import java.io.EOFException;
+import java.io.IOException;
+import java.io.InputStreamReader;
+
+import org.apache.commons.lang.StringEscapeUtils;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.io.SequenceFile;
+import org.apache.hadoop.io.SequenceFile.Reader;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.io.Writable;
+import org.apache.hadoop.util.ReflectionUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.kylinolap.common.util.HadoopUtil;
+import com.kylinolap.common.util.StringSplitter;
+
+/**
+ * Tables are typically CSV or SEQ file.
+ * 
+ * @author yangli9
+ */
+public class FileTableReader implements TableReader {
+
+    private static final Logger logger = LoggerFactory.getLogger(FileTableReader.class);
+    private static final char CSV_QUOTE = '"';
+    private static final String[] DETECT_DELIMS = new String[] { "\177", "|", "\t", "," };
+
+    private String filePath;
+    private String delim;
+    private RowReader reader;
+
+    private String curLine;
+    private String[] curColumns;
+    private int expectedColumnNumber = -1; // helps delimiter detection
+
+    public FileTableReader(String filePath, String delim, int expectedColumnNumber) throws IOException {
+        this.filePath = filePath;
+        this.delim = delim;
+        this.expectedColumnNumber = expectedColumnNumber;
+
+        FileSystem fs = HadoopUtil.getFileSystem(filePath);
+
+        try {
+            this.reader = new SeqRowReader(HadoopUtil.getDefaultConfiguration(), fs, filePath);
+
+        } catch (IOException e) {
+            if (isExceptionSayingNotSeqFile(e) == false)
+                throw e;
+
+            this.reader = new CsvRowReader(fs, filePath);
+        }
+    }
+
+    private boolean isExceptionSayingNotSeqFile(IOException e) {
+        if (e.getMessage() != null && e.getMessage().contains("not a SequenceFile"))
+            return true;
+
+        if (e instanceof EOFException) // in case the file is very very small
+            return true;
+
+        return false;
+    }
+
+    @Override
+    public void setExpectedColumnNumber(int expectedColumnNumber) {
+        this.expectedColumnNumber = expectedColumnNumber;
+    }
+
+    @Override
+    public boolean next() throws IOException {
+        curLine = reader.nextLine();
+        curColumns = null;
+        return curLine != null;
+    }
+
+    public String getLine() {
+        return curLine;
+    }
+
+    @Override
+    public String[] getRow() {
+        if (curColumns == null) {
+            if (ReadableTable.DELIM_AUTO.equals(delim))
+                delim = autoDetectDelim(curLine);
+
+            if (delim == null)
+                curColumns = new String[] { curLine };
+            else
+                curColumns = split(curLine, delim);
+        }
+        return curColumns;
+    }
+
+    private String[] split(String line, String delim) {
+        // FIXME CVS line should be parsed considering escapes
+        String str[] = StringSplitter.split(line, delim);
+
+        // un-escape CSV
+        if (ReadableTable.DELIM_COMMA.equals(delim)) {
+            for (int i = 0; i < str.length; i++) {
+                str[i] = unescapeCsv(str[i]);
+            }
+        }
+
+        return str;
+    }
+
+    private String unescapeCsv(String str) {
+        if (str == null || str.length() < 2)
+            return str;
+
+        str = StringEscapeUtils.unescapeCsv(str);
+
+        // unescapeCsv may not remove the outer most quotes
+        if (str.charAt(0) == CSV_QUOTE && str.charAt(str.length() - 1) == CSV_QUOTE)
+            str = str.substring(1, str.length() - 1);
+
+        return str;
+    }
+
+    @Override
+    public void close() throws IOException {
+        if (reader != null)
+            reader.close();
+    }
+
+    private String autoDetectDelim(String line) {
+        if (expectedColumnNumber > 0) {
+            for (String delim : DETECT_DELIMS) {
+                if (StringSplitter.split(line, delim).length == expectedColumnNumber) {
+                    logger.info("Auto detect delim to be '" + delim + "', split line to " + expectedColumnNumber + " columns -- " + line);
+                    return delim;
+                }
+            }
+        }
+
+        logger.info("Auto detect delim to be null, will take THE-WHOLE-LINE as a single value, for " + filePath);
+        return null;
+    }
+
+    // ============================================================================
+
+    private interface RowReader extends Closeable {
+        String nextLine() throws IOException; // return null on EOF
+    }
+
+    private class SeqRowReader implements RowReader {
+        Reader reader;
+        Writable key;
+        Text value;
+
+        SeqRowReader(Configuration hconf, FileSystem fs, String path) throws IOException {
+            reader = new Reader(hconf, SequenceFile.Reader.file(new Path(path)));
+            key = (Writable) ReflectionUtils.newInstance(reader.getKeyClass(), hconf);
+            value = new Text();
+        }
+
+        @Override
+        public String nextLine() throws IOException {
+            boolean hasNext = reader.next(key, value);
+            if (hasNext)
+                return Bytes.toString(value.getBytes(), 0, value.getLength());
+            else
+                return null;
+        }
+
+        @Override
+        public void close() throws IOException {
+            reader.close();
+        }
+    }
+
+    private class CsvRowReader implements RowReader {
+        BufferedReader reader;
+
+        CsvRowReader(FileSystem fs, String path) throws IOException {
+            FSDataInputStream in = fs.open(new Path(path));
+            reader = new BufferedReader(new InputStreamReader(in, "UTF-8"));
+        }
+
+        @Override
+        public String nextLine() throws IOException {
+            return reader.readLine();
+        }
+
+        @Override
+        public void close() throws IOException {
+            reader.close();
+        }
+
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/dictionary/src/main/java/com/kylinolap/dict/lookup/HiveTable.java
----------------------------------------------------------------------
diff --git a/dictionary/src/main/java/com/kylinolap/dict/lookup/HiveTable.java b/dictionary/src/main/java/com/kylinolap/dict/lookup/HiveTable.java
new file mode 100644
index 0000000..25d2a87
--- /dev/null
+++ b/dictionary/src/main/java/com/kylinolap/dict/lookup/HiveTable.java
@@ -0,0 +1,129 @@
+/*
+ * 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.FileNotFoundException;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.google.common.collect.Lists;
+import com.kylinolap.common.KylinConfig;
+import com.kylinolap.common.util.HadoopUtil;
+import com.kylinolap.common.util.CliCommandExecutor;
+import com.kylinolap.metadata.MetadataManager;
+
+/**
+ * @author yangli9
+ * 
+ */
+public class HiveTable implements ReadableTable {
+
+    private static final Logger logger = LoggerFactory.getLogger(HiveTable.class);
+
+    private String hiveTable;
+    private int nColumns;
+    private String hdfsLocation;
+    private FileTable fileTable;
+
+    public HiveTable(MetadataManager metaMgr, String table) {
+        this.hiveTable = table;
+        this.nColumns = metaMgr.getTableDesc(table).getColumnCount();
+    }
+
+    @Override
+    public String getColumnDelimeter() throws IOException {
+        return getFileTable().getColumnDelimeter();
+    }
+
+    @Override
+    public TableReader getReader() throws IOException {
+        return getFileTable().getReader();
+    }
+
+    @Override
+    public TableSignature getSignature() throws IOException {
+        return getFileTable().getSignature();
+    }
+
+    private FileTable getFileTable() throws IOException {
+        if (fileTable == null) {
+            fileTable = new FileTable(getHDFSLocation(true), nColumns);
+        }
+        return fileTable;
+    }
+
+    public String getHDFSLocation(boolean needFilePath) throws IOException {
+        if (hdfsLocation == null) {
+            hdfsLocation = computeHDFSLocation(needFilePath);
+        }
+        return hdfsLocation;
+    }
+
+    private String computeHDFSLocation(boolean needFilePath) throws IOException {
+
+        String override = KylinConfig.getInstanceFromEnv().getOverrideHiveTableLocation(hiveTable);
+        if (override != null) {
+            logger.debug("Override hive table location " + hiveTable + " -- " + override);
+            return override;
+        }
+
+        String cmd = "hive -e \"describe extended " + hiveTable + ";\"";
+        CliCommandExecutor exec = KylinConfig.getInstanceFromEnv().getCliCommandExecutor();
+        String output = exec.execute(cmd);
+
+        Pattern ptn = Pattern.compile("location:(.*?),");
+        Matcher m = ptn.matcher(output);
+        if (m.find() == false)
+            throw new IOException("Failed to find HDFS location for hive table " + hiveTable + " from output -- " + output);
+
+        String hdfsDir = m.group(1);
+
+        if (needFilePath) {
+            FileSystem fs = HadoopUtil.getFileSystem(hdfsDir);
+            FileStatus file = findOnlyFile(hdfsDir, fs);
+            return file.getPath().toString();
+        } else {
+            return hdfsDir;
+        }
+    }
+
+    private FileStatus findOnlyFile(String hdfsDir, FileSystem fs) throws FileNotFoundException, IOException {
+        FileStatus[] files = fs.listStatus(new Path(hdfsDir));
+        ArrayList<FileStatus> nonZeroFiles = Lists.newArrayList();
+        for (FileStatus f : files) {
+            if (f.getLen() > 0)
+                nonZeroFiles.add(f);
+        }
+        if (nonZeroFiles.size() != 1)
+            throw new IllegalStateException("Expect 1 and only 1 non-zero file under " + hdfsDir + ", but find " + nonZeroFiles.size());
+        return nonZeroFiles.get(0);
+    }
+
+    @Override
+    public String toString() {
+        return "hive:" + hiveTable;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/dictionary/src/main/java/com/kylinolap/dict/lookup/LookupBytesTable.java
----------------------------------------------------------------------
diff --git a/dictionary/src/main/java/com/kylinolap/dict/lookup/LookupBytesTable.java b/dictionary/src/main/java/com/kylinolap/dict/lookup/LookupBytesTable.java
new file mode 100644
index 0000000..25f771f
--- /dev/null
+++ b/dictionary/src/main/java/com/kylinolap/dict/lookup/LookupBytesTable.java
@@ -0,0 +1,50 @@
+/*
+ * 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 org.apache.hadoop.hbase.util.Bytes;
+
+import com.kylinolap.common.util.ByteArray;
+import com.kylinolap.metadata.model.schema.TableDesc;
+
+/**
+ * @author yangli9
+ * 
+ */
+public class LookupBytesTable extends LookupTable<ByteArray> {
+
+    public LookupBytesTable(TableDesc tableDesc, String[] keyColumns, ReadableTable table) throws IOException {
+        super(tableDesc, keyColumns, table);
+    }
+
+    @Override
+    protected ByteArray[] convertRow(String[] cols) {
+        ByteArray[] r = new ByteArray[cols.length];
+        for (int i = 0; i < cols.length; i++) {
+            r[i] = cols[i] == null ? null : new ByteArray(Bytes.toBytes(cols[i]));
+        }
+        return r;
+    }
+
+    @Override
+    protected String toString(ByteArray cell) {
+        return Bytes.toString(cell.data);
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/dictionary/src/main/java/com/kylinolap/dict/lookup/LookupStringTable.java
----------------------------------------------------------------------
diff --git a/dictionary/src/main/java/com/kylinolap/dict/lookup/LookupStringTable.java b/dictionary/src/main/java/com/kylinolap/dict/lookup/LookupStringTable.java
new file mode 100644
index 0000000..0b6cc86
--- /dev/null
+++ b/dictionary/src/main/java/com/kylinolap/dict/lookup/LookupStringTable.java
@@ -0,0 +1,43 @@
+/*
+ * 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 com.kylinolap.metadata.model.schema.TableDesc;
+
+/**
+ * @author yangli9
+ * 
+ */
+public class LookupStringTable extends LookupTable<String> {
+
+    public LookupStringTable(TableDesc tableDesc, String[] keyColumns, ReadableTable table) throws IOException {
+        super(tableDesc, keyColumns, table);
+    }
+
+    @Override
+    protected String[] convertRow(String[] cols) {
+        return cols;
+    }
+
+    @Override
+    protected String toString(String cell) {
+        return cell;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/dictionary/src/main/java/com/kylinolap/dict/lookup/LookupTable.java
----------------------------------------------------------------------
diff --git a/dictionary/src/main/java/com/kylinolap/dict/lookup/LookupTable.java b/dictionary/src/main/java/com/kylinolap/dict/lookup/LookupTable.java
new file mode 100644
index 0000000..19d40ca
--- /dev/null
+++ b/dictionary/src/main/java/com/kylinolap/dict/lookup/LookupTable.java
@@ -0,0 +1,167 @@
+/*
+ * 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.Collection;
+import java.util.List;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+
+import org.apache.hadoop.hbase.util.Pair;
+
+import com.google.common.collect.Sets;
+import com.kylinolap.common.util.Array;
+import com.kylinolap.metadata.model.schema.TableDesc;
+
+/**
+ * An in-memory lookup table, in which each cell is an object of type T. The
+ * table is indexed by specified PK for fast lookup.
+ * 
+ * @author yangli9
+ */
+abstract public class LookupTable<T extends Comparable<T>> {
+
+    protected TableDesc tableDesc;
+    protected String[] keyColumns;
+    protected ReadableTable table;
+    protected ConcurrentHashMap<Array<T>, T[]> data;
+
+    public LookupTable(TableDesc tableDesc, String[] keyColumns, ReadableTable table) throws IOException {
+        this.tableDesc = tableDesc;
+        this.keyColumns = keyColumns;
+        this.table = table;
+        this.data = new ConcurrentHashMap<Array<T>, T[]>();
+        init();
+    }
+
+    protected void init() throws IOException {
+        int[] keyIndex = new int[keyColumns.length];
+        for (int i = 0; i < keyColumns.length; i++) {
+            keyIndex[i] = tableDesc.findColumnByName(keyColumns[i]).getZeroBasedIndex();
+        }
+
+        TableReader reader = table.getReader();
+        try {
+            while (reader.next()) {
+                initRow(reader.getRow(), keyIndex);
+            }
+        } finally {
+            reader.close();
+        }
+    }
+
+    @SuppressWarnings("unchecked")
+    private void initRow(String[] cols, int[] keyIndex) {
+        T[] value = convertRow(cols);
+        T[] keyCols = (T[]) java.lang.reflect.Array.newInstance(value[0].getClass(), keyIndex.length);
+        for (int i = 0; i < keyCols.length; i++)
+            keyCols[i] = value[keyIndex[i]];
+
+        Array<T> key = new Array<T>(keyCols);
+
+        if (data.containsKey(key))
+            throw new IllegalStateException("Dup key found, key=" + toString(keyCols) + ", value1=" + toString(data.get(key)) + ", value2=" + toString(value));
+
+        data.put(key, value);
+    }
+
+    abstract protected T[] convertRow(String[] cols);
+
+    public T[] getRow(Array<T> key) {
+        return data.get(key);
+    }
+
+    public Collection<T[]> getAllRows() {
+        return data.values();
+    }
+
+    public List<T> scan(String col, List<T> values, String returnCol) {
+        ArrayList<T> result = new ArrayList<T>();
+        int colIdx = tableDesc.findColumnByName(col).getZeroBasedIndex();
+        int returnIdx = tableDesc.findColumnByName(returnCol).getZeroBasedIndex();
+        for (T[] row : data.values()) {
+            if (values.contains(row[colIdx]))
+                result.add(row[returnIdx]);
+        }
+        return result;
+    }
+
+    public Pair<T, T> mapRange(String col, T beginValue, T endValue, String returnCol) {
+        int colIdx = tableDesc.findColumnByName(col).getZeroBasedIndex();
+        int returnIdx = tableDesc.findColumnByName(returnCol).getZeroBasedIndex();
+        T returnBegin = null;
+        T returnEnd = null;
+        for (T[] row : data.values()) {
+            if (between(beginValue, row[colIdx], endValue)) {
+                T returnValue = row[returnIdx];
+                if (returnBegin == null || returnValue.compareTo(returnBegin) < 0) {
+                    returnBegin = returnValue;
+                }
+                if (returnEnd == null || returnValue.compareTo(returnEnd) > 0) {
+                    returnEnd = returnValue;
+                }
+            }
+        }
+        if (returnBegin == null && returnEnd == null)
+            return null;
+        else
+            return new Pair<T, T>(returnBegin, returnEnd);
+    }
+
+    public Set<T> mapValues(String col, Set<T> values, String returnCol) {
+        int colIdx = tableDesc.findColumnByName(col).getZeroBasedIndex();
+        int returnIdx = tableDesc.findColumnByName(returnCol).getZeroBasedIndex();
+        Set<T> result = Sets.newHashSetWithExpectedSize(values.size());
+        for (T[] row : data.values()) {
+            if (values.contains(row[colIdx])) {
+                result.add(row[returnIdx]);
+            }
+        }
+        return result;
+    }
+
+    private boolean between(T beginValue, T v, T endValue) {
+        return (beginValue == null || beginValue.compareTo(v) <= 0) && (endValue == null || v.compareTo(endValue) <= 0);
+    }
+
+    public String toString() {
+        return "LookupTable [path=" + table + "]";
+    }
+
+    protected String toString(T[] cols) {
+        StringBuilder b = new StringBuilder();
+        b.append("[");
+        for (int i = 0; i < cols.length; i++) {
+            if (i > 0)
+                b.append(",");
+            b.append(toString(cols[i]));
+        }
+        b.append("]");
+        return b.toString();
+    }
+
+    abstract protected String toString(T cell);
+
+    public void dump() {
+        for (Array<T> key : data.keySet()) {
+            System.out.println(toString(key.data) + " => " + toString(data.get(key)));
+        }
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/dictionary/src/main/java/com/kylinolap/dict/lookup/ReadableTable.java
----------------------------------------------------------------------
diff --git a/dictionary/src/main/java/com/kylinolap/dict/lookup/ReadableTable.java b/dictionary/src/main/java/com/kylinolap/dict/lookup/ReadableTable.java
new file mode 100644
index 0000000..4f5c22f
--- /dev/null
+++ b/dictionary/src/main/java/com/kylinolap/dict/lookup/ReadableTable.java
@@ -0,0 +1,35 @@
+/*
+ * 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;
+
+/**
+ * @author yangli9
+ */
+public interface ReadableTable {
+
+    public static final String DELIM_AUTO = "auto";
+    public static final String DELIM_COMMA = ",";
+
+    public TableReader getReader() throws IOException;
+
+    public TableSignature getSignature() throws IOException;
+
+    public String getColumnDelimeter() throws IOException;
+
+}


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

Posted by lu...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/job/src/test/java/com/kylinolap/job/hadoop/hbase/CreateHTableTest.java
----------------------------------------------------------------------
diff --git a/job/src/test/java/com/kylinolap/job/hadoop/hbase/CreateHTableTest.java b/job/src/test/java/com/kylinolap/job/hadoop/hbase/CreateHTableTest.java
new file mode 100644
index 0000000..9c0745e
--- /dev/null
+++ b/job/src/test/java/com/kylinolap/job/hadoop/hbase/CreateHTableTest.java
@@ -0,0 +1,63 @@
+/*
+ * 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.job.hadoop.hbase;
+
+import static org.junit.Assert.*;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+import com.kylinolap.common.util.LocalFileMetadataTestCase;
+
+/**
+ * @author George Song (ysong1)
+ * 
+ */
+public class CreateHTableTest extends LocalFileMetadataTestCase {
+
+    private Configuration conf;
+
+    @Before
+    public void setup() throws Exception {
+        conf = new Configuration();
+        conf.set("fs.default.name", "file:///");
+        conf.set("mapred.job.tracker", "local");
+        this.createTestMetadata();
+
+    }
+
+    @After
+    public void after() throws Exception {
+        this.cleanupTestMetadata();
+    }
+
+    @Test
+    public void testGetSplits() throws IllegalArgumentException, Exception {
+        CreateHTableJob c = new CreateHTableJob();
+
+        String input = "src/test/resources/partition_list/part-r-00000";
+
+        byte[][] splits = c.getSplits(conf, new Path(input));
+
+        assertEquals(497, splits.length);
+        assertArrayEquals(new byte[] { 0, 0, 0, 0, 0, 0, 15, -1, 11, 51, -45, 2 }, splits[0]);
+        assertArrayEquals(new byte[] { 0, 0, 0, 0, 0, 3, -1, -1, -54, -61, 109, -44, 1 }, splits[496]);
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/job/src/test/java/com/kylinolap/job/hadoop/hbase/TestHbaseClient.java
----------------------------------------------------------------------
diff --git a/job/src/test/java/com/kylinolap/job/hadoop/hbase/TestHbaseClient.java b/job/src/test/java/com/kylinolap/job/hadoop/hbase/TestHbaseClient.java
new file mode 100644
index 0000000..416d44f
--- /dev/null
+++ b/job/src/test/java/com/kylinolap/job/hadoop/hbase/TestHbaseClient.java
@@ -0,0 +1,84 @@
+package com.kylinolap.job.hadoop.hbase;
+
+import java.io.IOException;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.HBaseConfiguration;
+import org.apache.hadoop.hbase.client.HTable;
+import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.util.Bytes;
+
+/**
+ * Created by hongbin on 5/15/14.
+ */
+public class TestHbaseClient {
+
+    private static boolean reverse = false;
+
+    public static void foo(int n, int k) {
+        int t = k;
+        if (n - k < k) {
+            t = n - k;
+            reverse = true;
+        }
+        boolean[] flags = new boolean[n];
+        inner(flags, 0, t);
+    }
+
+    private static void print(boolean[] flags) {
+        for (int i = 0; i < flags.length; i++) {
+            if (!reverse) {
+                if (flags[i])
+                    System.out.print("0");
+                else
+                    System.out.print("1");
+            } else {
+                if (flags[i])
+                    System.out.print("1");
+                else
+                    System.out.print("0");
+
+            }
+        }
+        System.out.println();
+
+    }
+
+    private static void inner(boolean[] flags, int start, int remaining) {
+        if (remaining <= 0) {
+            print(flags);
+            return;
+        }
+
+        if (flags.length - start < remaining) {
+            return;
+        }
+
+        // write at flags[start]
+        flags[start] = true;
+        inner(flags, start + 1, remaining - 1);
+
+        // not write at flags[start]
+        flags[start] = false;
+        inner(flags, start + 1, remaining);
+    }
+
+    public static void main(String[] args) throws IOException {
+        foo(6, 5);
+        foo(5, 2);
+        foo(3, 0);
+
+        Configuration conf = HBaseConfiguration.create();
+        conf.set("hbase.zookeeper.quorum", "yadesk00.corp.ebay.com");
+        conf.set("zookeeper.znode.parent", "/hbase-unsecure");
+
+        HTable table = new HTable(conf, "test1");
+        Put put = new Put(Bytes.toBytes("row1"));
+
+        put.add(Bytes.toBytes("colfam1"), Bytes.toBytes("qual1"), Bytes.toBytes("val1"));
+        put.add(Bytes.toBytes("colfam1"), Bytes.toBytes("qual2"), Bytes.toBytes("val2"));
+
+        table.put(put);
+        table.close();
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/job/src/test/java/com/kylinolap/job/hadoop/hdfs/HdfsOpsTest.java
----------------------------------------------------------------------
diff --git a/job/src/test/java/com/kylinolap/job/hadoop/hdfs/HdfsOpsTest.java b/job/src/test/java/com/kylinolap/job/hadoop/hdfs/HdfsOpsTest.java
new file mode 100644
index 0000000..030a91e
--- /dev/null
+++ b/job/src/test/java/com/kylinolap/job/hadoop/hdfs/HdfsOpsTest.java
@@ -0,0 +1,50 @@
+package com.kylinolap.job.hadoop.hdfs;
+
+import java.io.IOException;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+import com.kylinolap.common.KylinConfig;
+import com.kylinolap.common.util.LocalFileMetadataTestCase;
+
+/**
+ * Created by honma on 8/20/14.
+ */
+public class HdfsOpsTest extends LocalFileMetadataTestCase {
+
+    FileSystem fileSystem;
+
+    @Before
+    public void setup() throws Exception {
+
+        this.createTestMetadata();
+
+        Configuration hconf = new Configuration();
+
+        fileSystem = FileSystem.get(hconf);
+    }
+
+    @Test
+    public void TestPath() throws IOException {
+        String hdfsWorkingDirectory = KylinConfig.getInstanceFromEnv().getHdfsWorkingDirectory();
+        Path coprocessorDir = new Path(hdfsWorkingDirectory, "test");
+        fileSystem.mkdirs(coprocessorDir);
+
+        Path newFile = new Path(coprocessorDir, "test_file");
+        newFile = newFile.makeQualified(fileSystem.getUri(), null);
+        FSDataOutputStream stream = fileSystem.create(newFile);
+        stream.write(new byte[] { 0, 1, 2 });
+        stream.close();
+    }
+
+    @After
+    public void after() throws Exception {
+        this.cleanupTestMetadata();
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/job/src/test/java/com/kylinolap/job/hadoop/hive/JoinedFlatTableTest.java
----------------------------------------------------------------------
diff --git a/job/src/test/java/com/kylinolap/job/hadoop/hive/JoinedFlatTableTest.java b/job/src/test/java/com/kylinolap/job/hadoop/hive/JoinedFlatTableTest.java
new file mode 100644
index 0000000..2b44aed
--- /dev/null
+++ b/job/src/test/java/com/kylinolap/job/hadoop/hive/JoinedFlatTableTest.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.job.hadoop.hive;
+
+import static org.junit.Assert.*;
+
+import java.io.IOException;
+
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+import com.kylinolap.common.KylinConfig;
+import com.kylinolap.common.util.LocalFileMetadataTestCase;
+import com.kylinolap.cube.CubeInstance;
+import com.kylinolap.cube.CubeManager;
+import com.kylinolap.cube.CubeSegment;
+import com.kylinolap.job.JoinedFlatTable;
+import com.kylinolap.job.engine.JobEngineConfig;
+
+/**
+ * @author George Song (ysong1)
+ * 
+ */
+public class JoinedFlatTableTest extends LocalFileMetadataTestCase {
+
+    CubeInstance cube = null;
+    JoinedFlatTableDesc intermediateTableDesc = null;
+    String fakeJobUUID = "abc-def";
+    CubeSegment cubeSegment = null;
+
+    @Before
+    public void setUp() throws Exception {
+        this.createTestMetadata();
+        cube = CubeManager.getInstance(this.getTestConfig()).getCube("test_kylin_cube_with_slr_ready");
+        cubeSegment = cube.getSegments().get(0);
+        intermediateTableDesc = new JoinedFlatTableDesc(cube.getDescriptor(), cubeSegment);
+    }
+
+    @After
+    public void after() throws Exception {
+        this.cleanupTestMetadata();
+    }
+
+    @Test
+    public void testGenCreateTableDDL() {
+        String ddl = JoinedFlatTable.generateCreateTableStatement(intermediateTableDesc, "/tmp", fakeJobUUID);
+        System.out.println(ddl);
+        assertEquals(513, ddl.length());
+    }
+
+    @Test
+    public void testGenDropTableDDL() {
+        String ddl = JoinedFlatTable.generateDropTableStatement(intermediateTableDesc, fakeJobUUID);
+        System.out.println(ddl);
+        assertEquals(108, ddl.length());
+    }
+
+    @Test
+    public void testGenerateInsertSql() throws IOException {
+        String sql = JoinedFlatTable.generateInsertDataStatement(intermediateTableDesc, fakeJobUUID, new JobEngineConfig(KylinConfig.getInstanceFromEnv()));
+        System.out.println(sql);
+
+        assertEquals(1239, sql.length());
+    }
+
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/job/src/test/java/com/kylinolap/job/tools/ColumnCardinalityJobTest.java
----------------------------------------------------------------------
diff --git a/job/src/test/java/com/kylinolap/job/tools/ColumnCardinalityJobTest.java b/job/src/test/java/com/kylinolap/job/tools/ColumnCardinalityJobTest.java
new file mode 100644
index 0000000..258870b
--- /dev/null
+++ b/job/src/test/java/com/kylinolap/job/tools/ColumnCardinalityJobTest.java
@@ -0,0 +1,60 @@
+/*
+ * 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.job.tools;
+
+import static org.junit.Assert.*;
+
+import java.io.File;
+import java.io.IOException;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileUtil;
+import org.apache.hadoop.util.ToolRunner;
+import org.junit.Before;
+import org.junit.Ignore;
+import org.junit.Test;
+
+import com.kylinolap.job.hadoop.cardinality.HiveColumnCardinalityJob;
+
+/**
+ * @author ysong1
+ * 
+ */
+public class ColumnCardinalityJobTest {
+
+    private Configuration conf;
+
+    @Before
+    public void setup() throws IOException {
+        conf = new Configuration();
+        conf.set("fs.default.name", "file:///");
+        conf.set("mapred.job.tracker", "local");
+    }
+
+    @Test
+    @Ignore
+    public void testJob() throws Exception {
+        final String input = "src/test/resources/data/test_cal_dt/";
+        final String output = "target/test-output/column-cardinality/";
+
+        FileUtil.fullyDelete(new File(output));
+
+        String[] args = { "-input", input, "-output", output, "-cols", "1,2,3,4,5,6,9,0" };
+        assertEquals("Job failed", 0, ToolRunner.run(new HiveColumnCardinalityJob(), args));
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/job/src/test/java/com/kylinolap/job/tools/ColumnCardinalityMapperTest.java
----------------------------------------------------------------------
diff --git a/job/src/test/java/com/kylinolap/job/tools/ColumnCardinalityMapperTest.java b/job/src/test/java/com/kylinolap/job/tools/ColumnCardinalityMapperTest.java
new file mode 100644
index 0000000..728c3e4
--- /dev/null
+++ b/job/src/test/java/com/kylinolap/job/tools/ColumnCardinalityMapperTest.java
@@ -0,0 +1,125 @@
+/*
+ * 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.job.tools;
+
+import static org.junit.Assert.*;
+
+import java.io.BufferedReader;
+import java.io.File;
+import java.io.FileReader;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.List;
+
+import org.apache.hadoop.io.BytesWritable;
+import org.apache.hadoop.io.IntWritable;
+import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.mrunit.mapreduce.MapDriver;
+import org.apache.hadoop.mrunit.types.Pair;
+import org.junit.Before;
+import org.junit.Ignore;
+import org.junit.Test;
+
+import com.kylinolap.common.hll.HyperLogLogPlusCounter;
+import com.kylinolap.job.hadoop.cardinality.ColumnCardinalityMapper;
+import com.kylinolap.job.hadoop.cardinality.HiveColumnCardinalityJob;
+
+/**
+ * @author ysong1
+ * 
+ */
+public class ColumnCardinalityMapperTest {
+
+    @SuppressWarnings("rawtypes")
+    MapDriver mapDriver;
+    String localTempDir = System.getProperty("java.io.tmpdir") + File.separator;
+
+    @SuppressWarnings({ "rawtypes", "unchecked" })
+    @Before
+    public void setUp() {
+        ColumnCardinalityMapper mapper = new ColumnCardinalityMapper();
+        mapDriver = MapDriver.newMapDriver(mapper);
+    }
+
+    public final static String strArr = "abc,tests,test,test,as,sts,test,tss,sets";
+
+    @SuppressWarnings({ "unchecked" })
+    @Test
+    @Ignore
+    public void testMapperOn177() throws IOException {
+        mapDriver.clearInput();
+        File file = new File("src/test/resources/data/test_cal_dt/part-r-00000");
+        FileReader reader = new FileReader(file);
+        BufferedReader breader = new BufferedReader(reader);
+        String s = breader.readLine();
+        int i = 0;
+        while (s != null) {
+            LongWritable inputKey = new LongWritable(i++);
+            mapDriver.addInput(inputKey, new Text(s));
+            s = breader.readLine();
+        }
+        // breader.close();
+        mapDriver.getConfiguration().set(HiveColumnCardinalityJob.KEY_INPUT_DELIM, "\20");
+        List<Pair<IntWritable, BytesWritable>> result = mapDriver.run();
+        breader.close();
+        assertEquals(9, result.size());
+
+        int key1 = result.get(0).getFirst().get();
+        BytesWritable value1 = result.get(0).getSecond();
+        byte[] bytes = value1.getBytes();
+        HyperLogLogPlusCounter hllc = new HyperLogLogPlusCounter();
+        hllc.readRegisters(ByteBuffer.wrap(bytes));
+        assertTrue(key1 > 0);
+        assertEquals(8, hllc.getCountEstimate());
+    }
+
+    @SuppressWarnings("unchecked")
+    @Test
+    public void testMapperOnComma() throws IOException {
+        mapDriver.clearInput();
+        LongWritable inputKey1 = new LongWritable(1);
+        LongWritable inputKey2 = new LongWritable(2);
+        LongWritable inputKey3 = new LongWritable(3);
+        LongWritable inputKey4 = new LongWritable(4);
+        LongWritable inputKey5 = new LongWritable(5);
+        LongWritable inputKey6 = new LongWritable(6);
+        LongWritable inputKey7 = new LongWritable(7);
+
+        mapDriver.addInput(inputKey1, new Text());
+        mapDriver.addInput(inputKey2, new Text(strArr));
+        mapDriver.addInput(inputKey3, new Text(strArr));
+        mapDriver.addInput(inputKey4, new Text(strArr));
+        mapDriver.addInput(inputKey5, new Text(strArr));
+        mapDriver.addInput(inputKey6, new Text(strArr));
+        mapDriver.addInput(inputKey7, new Text(strArr));
+
+        List<Pair<IntWritable, BytesWritable>> result = mapDriver.run();
+
+        assertEquals(9, result.size());
+
+        int key1 = result.get(0).getFirst().get();
+        BytesWritable value1 = result.get(0).getSecond();
+        byte[] bytes = value1.getBytes();
+        HyperLogLogPlusCounter hllc = new HyperLogLogPlusCounter();
+        hllc.readRegisters(ByteBuffer.wrap(bytes));
+        System.out.println("ab\177ab".length());
+        assertTrue(key1 > 0);
+        assertEquals(1, hllc.getCountEstimate());
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/job/src/test/java/com/kylinolap/job/tools/ColumnCardinalityReducerTest.java
----------------------------------------------------------------------
diff --git a/job/src/test/java/com/kylinolap/job/tools/ColumnCardinalityReducerTest.java b/job/src/test/java/com/kylinolap/job/tools/ColumnCardinalityReducerTest.java
new file mode 100644
index 0000000..e6efa0c
--- /dev/null
+++ b/job/src/test/java/com/kylinolap/job/tools/ColumnCardinalityReducerTest.java
@@ -0,0 +1,111 @@
+/*
+ * 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.job.tools;
+
+import static org.junit.Assert.*;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.StringTokenizer;
+
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.io.BytesWritable;
+import org.apache.hadoop.io.IntWritable;
+import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.mrunit.mapreduce.ReduceDriver;
+import org.apache.hadoop.mrunit.types.Pair;
+import org.junit.Before;
+import org.junit.Test;
+
+import com.kylinolap.common.hll.HyperLogLogPlusCounter;
+import com.kylinolap.cube.kv.RowConstants;
+import com.kylinolap.job.hadoop.cardinality.ColumnCardinalityMapper;
+import com.kylinolap.job.hadoop.cardinality.ColumnCardinalityReducer;
+
+/**
+ * @author ysong1
+ * 
+ */
+public class ColumnCardinalityReducerTest {
+
+    ReduceDriver<IntWritable, BytesWritable, IntWritable, LongWritable> reduceDriver;
+    String localTempDir = System.getProperty("java.io.tmpdir") + File.separator;
+
+    @Before
+    public void setUp() {
+        ColumnCardinalityReducer reducer = new ColumnCardinalityReducer();
+        reduceDriver = ReduceDriver.newReduceDriver(reducer);
+    }
+
+    private byte[] getBytes(String str) throws IOException {
+        HyperLogLogPlusCounter hllc = new HyperLogLogPlusCounter();
+        StringTokenizer tokenizer = new StringTokenizer(str, ColumnCardinalityMapper.DEFAULT_DELIM);
+        int i = 0;
+        while (tokenizer.hasMoreTokens()) {
+            String temp = i + "_" + tokenizer.nextToken();
+            i++;
+            hllc.add(Bytes.toBytes(temp));
+        }
+        ByteBuffer buf = ByteBuffer.allocate(RowConstants.ROWVALUE_BUFFER_SIZE);
+        buf.clear();
+        hllc.writeRegisters(buf);
+        buf.flip();
+        return buf.array();
+    }
+
+    @Test
+    public void testReducer() throws IOException {
+        IntWritable key1 = new IntWritable(1);
+        List<BytesWritable> values1 = new ArrayList<BytesWritable>();
+        values1.add(new BytesWritable(getBytes(ColumnCardinalityMapperTest.strArr)));
+
+        IntWritable key2 = new IntWritable(2);
+        List<BytesWritable> values2 = new ArrayList<BytesWritable>();
+        values2.add(new BytesWritable(getBytes(ColumnCardinalityMapperTest.strArr + " x")));
+
+        IntWritable key3 = new IntWritable(3);
+        List<BytesWritable> values3 = new ArrayList<BytesWritable>();
+        values3.add(new BytesWritable(getBytes(ColumnCardinalityMapperTest.strArr + " xx")));
+
+        IntWritable key4 = new IntWritable(4);
+        List<BytesWritable> values4 = new ArrayList<BytesWritable>();
+        values4.add(new BytesWritable(getBytes(ColumnCardinalityMapperTest.strArr + " xxx")));
+
+        IntWritable key5 = new IntWritable(5);
+        List<BytesWritable> values5 = new ArrayList<BytesWritable>();
+        values5.add(new BytesWritable(getBytes(ColumnCardinalityMapperTest.strArr + " xxxx")));
+
+        reduceDriver.withInput(key1, values1);
+        reduceDriver.withInput(key2, values2);
+        reduceDriver.withInput(key3, values3);
+        reduceDriver.withInput(key4, values4);
+        reduceDriver.withInput(key5, values5);
+
+        List<Pair<IntWritable, LongWritable>> result = reduceDriver.run();
+
+        assertEquals(5, result.size());
+
+        int outputKey1 = result.get(0).getFirst().get();
+        LongWritable value1 = result.get(0).getSecond();
+        assertTrue(outputKey1 == 1);
+        assertTrue((10 == value1.get()) || (9 == value1.get()));
+
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/job/src/test/java/com/kylinolap/job/tools/CubeMigrationTests.java
----------------------------------------------------------------------
diff --git a/job/src/test/java/com/kylinolap/job/tools/CubeMigrationTests.java b/job/src/test/java/com/kylinolap/job/tools/CubeMigrationTests.java
new file mode 100644
index 0000000..d3c804e
--- /dev/null
+++ b/job/src/test/java/com/kylinolap/job/tools/CubeMigrationTests.java
@@ -0,0 +1,40 @@
+package com.kylinolap.job.tools;
+
+import java.io.File;
+import java.io.IOException;
+
+import org.codehaus.jettison.json.JSONException;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Ignore;
+import org.junit.Test;
+
+import com.kylinolap.common.util.AbstractKylinTestCase;
+import com.kylinolap.common.util.ClasspathUtil;
+import com.kylinolap.common.util.LocalFileMetadataTestCase;
+
+/**
+ * Created by honma on 9/17/14.
+ */
+@Ignore
+public class CubeMigrationTests extends LocalFileMetadataTestCase {
+    @Before
+    public void setup() throws Exception {
+        super.createTestMetadata();
+        ClasspathUtil.addClasspath(new File(AbstractKylinTestCase.SANDBOX_TEST_DATA).getAbsolutePath());
+    }
+
+    @After
+    public void clean() {
+        this.cleanupTestMetadata();
+    }
+
+    @Test
+    public void testMigrate() throws IOException, JSONException, InterruptedException {
+
+        // CubeMigrationCLI.moveCube(KylinConfig.getInstanceFromEnv(),
+        // KylinConfig.getInstanceFromEnv(),
+        // "test_kylin_cube_with_slr_empty", "migration", "true", "false");
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/job/src/test/resources/data/6d_cuboid/part-r-00000
----------------------------------------------------------------------
diff --git a/job/src/test/resources/data/6d_cuboid/part-r-00000 b/job/src/test/resources/data/6d_cuboid/part-r-00000
new file mode 100644
index 0000000..b41ba18
Binary files /dev/null and b/job/src/test/resources/data/6d_cuboid/part-r-00000 differ

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/job/src/test/resources/data/base_cuboid/part-r-00000
----------------------------------------------------------------------
diff --git a/job/src/test/resources/data/base_cuboid/part-r-00000 b/job/src/test/resources/data/base_cuboid/part-r-00000
new file mode 100644
index 0000000..1a011a4
Binary files /dev/null and b/job/src/test/resources/data/base_cuboid/part-r-00000 differ

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/job/src/test/resources/data/flat_table/000000_0
----------------------------------------------------------------------
diff --git a/job/src/test/resources/data/flat_table/000000_0 b/job/src/test/resources/data/flat_table/000000_0
new file mode 100644
index 0000000..058c92a
Binary files /dev/null and b/job/src/test/resources/data/flat_table/000000_0 differ


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

Posted by lu...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/cube/src/test/java/com/kylinolap/cube/SegmentManagementTest.java
----------------------------------------------------------------------
diff --git a/cube/src/test/java/com/kylinolap/cube/SegmentManagementTest.java b/cube/src/test/java/com/kylinolap/cube/SegmentManagementTest.java
new file mode 100644
index 0000000..f6f81c1
--- /dev/null
+++ b/cube/src/test/java/com/kylinolap/cube/SegmentManagementTest.java
@@ -0,0 +1,710 @@
+/*
+ * 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.cube;
+
+import static org.junit.Assert.*;
+
+import java.io.IOException;
+import java.text.ParseException;
+import java.text.SimpleDateFormat;
+import java.util.List;
+import java.util.TimeZone;
+
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+import com.kylinolap.common.persistence.ResourceStore;
+import com.kylinolap.common.util.JsonUtil;
+import com.kylinolap.common.util.LocalFileMetadataTestCase;
+import com.kylinolap.cube.exception.CubeIntegrityException;
+import com.kylinolap.cube.project.ProjectInstance;
+import com.kylinolap.cube.project.ProjectManager;
+import com.kylinolap.metadata.MetadataManager;
+import com.kylinolap.metadata.model.cube.CubeDesc;
+
+/**
+ * @author ysong1
+ * 
+ */
+public class SegmentManagementTest extends LocalFileMetadataTestCase {
+
+    CubeManager cubeMgr = null;
+
+    @Before
+    public void setUp() throws Exception {
+        this.createTestMetadata();
+        MetadataManager.removeInstance(this.getTestConfig());
+        CubeManager.removeInstance(this.getTestConfig());
+        ProjectManager.removeInstance(this.getTestConfig());
+        cubeMgr = CubeManager.getInstance(this.getTestConfig());
+    }
+
+    @After
+    public void after() throws Exception {
+        this.cleanupTestMetadata();
+    }
+
+    private void createNewCube(CubeDesc cubeDesc) throws IOException {
+        ResourceStore store = getStore();
+        // clean legacy in case last run failed
+        store.deleteResource("/cube/a_whole_new_cube.json");
+
+        CubeInstance createdCube = cubeMgr.createCube("a_whole_new_cube", ProjectInstance.DEFAULT_PROJECT_NAME, cubeDesc, "username");
+        assertTrue(createdCube == cubeMgr.getCube("a_whole_new_cube"));
+
+        System.out.println(JsonUtil.writeValueAsIndentString(createdCube));
+    }
+
+    @Test
+    public void testInitialAndAppend() throws ParseException, IOException, CubeIntegrityException {
+        // create a new cube
+        MetadataManager metaMgr = getMetadataManager();
+        CubeDesc desc = metaMgr.getCubeDesc("test_kylin_cube_with_slr_desc");
+        createNewCube(desc);
+
+        SimpleDateFormat f = new SimpleDateFormat("yyyy-MM-dd");
+        f.setTimeZone(TimeZone.getTimeZone("GMT"));
+
+        long dateEnd = f.parse("2013-11-12").getTime();
+
+        CubeInstance cubeInstance = cubeMgr.getCube("a_whole_new_cube");
+        assertEquals(CubeStatusEnum.DISABLED, cubeInstance.getStatus());
+        assertEquals(0, cubeInstance.getSegments().size());
+        assertEquals(0, cubeInstance.getBuildingSegments().size());
+        assertEquals(0, cubeInstance.getAllocatedEndDate());
+
+        // initial build
+        System.out.println("Initial Build");
+        CubeSegment initialSegment = cubeMgr.allocateSegments(cubeInstance, CubeBuildTypeEnum.BUILD, desc.getCubePartitionDesc().getPartitionDateStart(), dateEnd).get(0);
+        System.out.println(JsonUtil.writeValueAsIndentString(cubeInstance));
+        assertEquals(CubeStatusEnum.DISABLED, cubeInstance.getStatus());
+        assertEquals(CubeSegmentStatusEnum.NEW, cubeInstance.getBuildingSegments().get(0).getStatus());
+        assertEquals(1, cubeInstance.getSegments().size());
+        assertEquals(1, cubeInstance.getBuildingSegments().size());
+        assertEquals(0, cubeInstance.getRebuildingSegments().size());
+        assertTrue("".equals(initialSegment.getStorageLocationIdentifier()) == false);
+        assertEquals(desc.getCubePartitionDesc().getPartitionDateStart(), cubeInstance.getAllocatedStartDate());
+        assertEquals(dateEnd, cubeInstance.getAllocatedEndDate());
+
+        // initial build success
+        System.out.println("Initial Build Success");
+        cubeMgr.updateSegmentOnJobSucceed(cubeInstance, CubeBuildTypeEnum.BUILD, initialSegment.getName(), "job_1", System.currentTimeMillis(), 111L, 222L, 333L);
+        System.out.println(JsonUtil.writeValueAsIndentString(cubeInstance));
+        assertEquals(CubeStatusEnum.READY, cubeInstance.getStatus());
+        assertEquals(1, cubeInstance.getSegments(CubeSegmentStatusEnum.READY).size());
+        assertEquals(0, cubeInstance.getBuildingSegments().size());
+        assertEquals(0, cubeInstance.getRebuildingSegments().size());
+        assertEquals(desc.getCubePartitionDesc().getPartitionDateStart(), cubeInstance.getAllocatedStartDate());
+        assertEquals(dateEnd, cubeInstance.getAllocatedEndDate());
+
+        // incremental build
+        System.out.println("Incremental Build");
+        long dateEnd2 = f.parse("2013-12-12").getTime();
+        CubeSegment incrementalSegment = cubeMgr.allocateSegments(cubeInstance, CubeBuildTypeEnum.BUILD, dateEnd, dateEnd2).get(0);
+        System.out.println(JsonUtil.writeValueAsIndentString(cubeInstance));
+        assertEquals(CubeStatusEnum.READY, cubeInstance.getStatus());
+        assertEquals(2, cubeInstance.getSegments().size());
+        assertEquals(1, cubeInstance.getSegments(CubeSegmentStatusEnum.NEW).size());
+        assertEquals(1, cubeInstance.getBuildingSegments().size());
+        assertEquals(desc.getCubePartitionDesc().getPartitionDateStart(), cubeInstance.getAllocatedStartDate());
+        assertEquals(dateEnd2, cubeInstance.getAllocatedEndDate());
+        assertEquals(dateEnd, cubeInstance.getBuildingSegments().get(0).getDateRangeStart());
+        assertEquals(dateEnd2, cubeInstance.getBuildingSegments().get(0).getDateRangeEnd());
+
+        // incremental build success
+        System.out.println("Incremental Build Success");
+        cubeMgr.updateSegmentOnJobSucceed(cubeInstance, CubeBuildTypeEnum.BUILD, incrementalSegment.getName(), "job_2", System.currentTimeMillis(), 111L, 222L, 333L);
+        System.out.println(JsonUtil.writeValueAsIndentString(cubeInstance));
+        assertEquals(CubeStatusEnum.READY, cubeInstance.getStatus());
+        assertEquals(2, cubeInstance.getSegments().size());
+        assertEquals(2, cubeInstance.getSegments(CubeSegmentStatusEnum.READY).size());
+        assertEquals(0, cubeInstance.getBuildingSegments().size());
+        assertEquals(desc.getCubePartitionDesc().getPartitionDateStart(), cubeInstance.getAllocatedStartDate());
+        assertEquals(dateEnd2, cubeInstance.getAllocatedEndDate());
+    }
+
+    @Test
+    public void testRebuildSegment() throws IOException, CubeIntegrityException {
+        CubeInstance cubeInstance = cubeMgr.getCube("test_kylin_cube_with_slr_ready");
+
+        // rebuild segment
+        System.out.println("Rebuild Segment");
+        CubeSegment rebuildSegment = cubeMgr.allocateSegments(cubeInstance, CubeBuildTypeEnum.BUILD, 1364688000000L, 1386806400000L).get(0);
+        System.out.println(JsonUtil.writeValueAsIndentString(cubeInstance));
+        assertEquals(CubeStatusEnum.READY, cubeInstance.getStatus());
+        assertEquals(1, cubeInstance.getBuildingSegments().size());
+        assertEquals(CubeSegmentStatusEnum.NEW, cubeInstance.getBuildingSegments().get(0).getStatus());
+        assertEquals(1, cubeInstance.getSegments(CubeSegmentStatusEnum.READY).size());
+        assertEquals(1, cubeInstance.getSegments(CubeSegmentStatusEnum.NEW).size());
+        assertEquals(1, cubeInstance.getRebuildingSegments().size());
+        assertEquals(1364688000000L, cubeInstance.getAllocatedStartDate());
+        assertEquals(1386806400000L, cubeInstance.getAllocatedEndDate());
+
+        // rebuild success
+        System.out.println("Rebuild Success");
+        cubeMgr.updateSegmentOnJobSucceed(cubeInstance, CubeBuildTypeEnum.BUILD, rebuildSegment.getName(), "job_3", System.currentTimeMillis(), 111, 222, 333);
+        assertEquals(CubeStatusEnum.READY, cubeInstance.getStatus());
+        assertEquals(1, cubeInstance.getSegments().size());
+        assertEquals(1, cubeInstance.getSegments(CubeSegmentStatusEnum.READY).size());
+        assertEquals(0, cubeInstance.getBuildingSegments().size());
+        assertEquals(0, cubeInstance.getRebuildingSegments().size());
+        assertEquals(1364688000000L, cubeInstance.getAllocatedStartDate());
+        assertEquals(1386806400000L, cubeInstance.getAllocatedEndDate());
+        assertEquals("job_3", cubeInstance.getSegments().get(0).getLastBuildJobID());
+        System.out.println(JsonUtil.writeValueAsIndentString(cubeInstance));
+    }
+
+    @Test(expected = CubeIntegrityException.class)
+    public void testInvalidRebuild() throws IOException, CubeIntegrityException {
+        CubeInstance cubeInstance = cubeMgr.getCube("test_kylin_cube_with_slr_ready");
+
+        // rebuild segment
+        System.out.println("Rebuild Segment");
+        cubeMgr.allocateSegments(cubeInstance, CubeBuildTypeEnum.BUILD, 1364688000000L + 1000L, 1386806400000L).get(0);
+    }
+
+    @Test
+    public void testMergeSegments() throws IOException, CubeIntegrityException {
+        CubeInstance cubeInstance = cubeMgr.getCube("test_kylin_cube_with_slr_ready_2_segments");
+
+        // merge segments
+        System.out.println("Merge Segment");
+        CubeSegment mergedSegment = cubeMgr.allocateSegments(cubeInstance, CubeBuildTypeEnum.MERGE, 1384240200000L, 1386835200000L).get(0);
+        System.out.println(JsonUtil.writeValueAsIndentString(cubeInstance));
+        assertEquals(CubeStatusEnum.READY, cubeInstance.getStatus());
+        assertEquals(CubeSegmentStatusEnum.NEW, cubeInstance.getBuildingSegments().get(0).getStatus());
+        assertEquals(2, cubeInstance.getSegments(CubeSegmentStatusEnum.READY).size());
+        assertEquals(1, cubeInstance.getSegments(CubeSegmentStatusEnum.NEW).size());
+        assertEquals(2, cubeInstance.getMergingSegments().size());
+        assertEquals(1384240200000L, cubeInstance.getAllocatedStartDate());
+        assertEquals(1386835200000L, cubeInstance.getAllocatedEndDate());
+
+        // build success
+        System.out.println("Build Success");
+        cubeMgr.updateSegmentOnJobSucceed(cubeInstance, CubeBuildTypeEnum.MERGE, mergedSegment.getName(), "job_4", System.currentTimeMillis(), 123, 20000L, 1216024L);
+        assertEquals(CubeStatusEnum.READY, cubeInstance.getStatus());
+        assertEquals(1, cubeInstance.getSegments(CubeSegmentStatusEnum.READY).size());
+        assertEquals(0, cubeInstance.getBuildingSegments().size());
+        assertEquals(0, cubeInstance.getMergingSegments().size());
+        assertEquals(1384240200000L, cubeInstance.getAllocatedStartDate());
+        assertEquals(1386835200000L, cubeInstance.getAllocatedEndDate());
+        assertEquals("job_4", cubeInstance.getSegments().get(0).getLastBuildJobID());
+        assertEquals(20000L, cubeInstance.getSegments().get(0).getSourceRecords());
+        assertEquals(1216024L, cubeInstance.getSegments().get(0).getSourceRecordsSize());
+        System.out.println(JsonUtil.writeValueAsIndentString(cubeInstance));
+    }
+
+    @Test
+    public void testNonPartitionedCube() throws ParseException, IOException, CubeIntegrityException {
+        // create a new cube
+        MetadataManager metaMgr = getMetadataManager();
+        CubeDesc desc = metaMgr.getCubeDesc("test_kylin_cube_without_slr_desc");
+        createNewCube(desc);
+
+        SimpleDateFormat f = new SimpleDateFormat("yyyy-MM-dd");
+        f.setTimeZone(TimeZone.getTimeZone("GMT"));
+
+        long dateEnd = f.parse("2013-11-12").getTime();
+
+        CubeInstance cubeInstance = cubeMgr.getCube("a_whole_new_cube");
+        assertEquals(CubeStatusEnum.DISABLED, cubeInstance.getStatus());
+        assertEquals(0, cubeInstance.getSegments().size());
+        assertEquals(0, cubeInstance.getBuildingSegments().size());
+        assertEquals(0, cubeInstance.getAllocatedEndDate());
+
+        // initial build
+        System.out.println("Initial Build");
+        CubeSegment initialSegment = cubeMgr.allocateSegments(cubeInstance, CubeBuildTypeEnum.BUILD, desc.getCubePartitionDesc().getPartitionDateStart(), dateEnd).get(0);
+        System.out.println(JsonUtil.writeValueAsIndentString(cubeInstance));
+        assertEquals(CubeStatusEnum.DISABLED, cubeInstance.getStatus());
+        assertEquals(CubeSegmentStatusEnum.NEW, cubeInstance.getBuildingSegments().get(0).getStatus());
+        assertEquals(1, cubeInstance.getSegments().size());
+        assertEquals(0, cubeInstance.getSegments(CubeSegmentStatusEnum.READY).size());
+        assertEquals(1, cubeInstance.getBuildingSegments().size());
+        assertEquals(0, cubeInstance.getRebuildingSegments().size());
+        assertTrue("".equals(initialSegment.getStorageLocationIdentifier()) == false);
+        assertEquals("FULL_BUILD", initialSegment.getName());
+        assertEquals(desc.getCubePartitionDesc().getPartitionDateStart(), cubeInstance.getAllocatedStartDate());
+        assertEquals(0, cubeInstance.getAllocatedEndDate());
+
+        // initial build success
+        System.out.println("Initial Build Success");
+        cubeMgr.updateSegmentOnJobSucceed(cubeInstance, CubeBuildTypeEnum.BUILD, initialSegment.getName(), "job_5", System.currentTimeMillis(), 111L, 222L, 333L);
+        assertEquals(CubeStatusEnum.READY, cubeInstance.getStatus());
+        assertEquals(1, cubeInstance.getSegments(CubeSegmentStatusEnum.READY).size());
+        assertEquals(0, cubeInstance.getBuildingSegments().size());
+        assertEquals(0, cubeInstance.getRebuildingSegments().size());
+        assertEquals(0, cubeInstance.getAllocatedStartDate());
+        assertEquals(0, cubeInstance.getAllocatedEndDate());
+        System.out.println(JsonUtil.writeValueAsIndentString(cubeInstance));
+
+        // rebuild segment
+        System.out.println("Rebuild Segment");
+        CubeSegment rebuildSegment = cubeMgr.allocateSegments(cubeInstance, CubeBuildTypeEnum.BUILD, 1364688000000L, 1386806400000L).get(0);
+        System.out.println(JsonUtil.writeValueAsIndentString(cubeInstance));
+        assertEquals(CubeStatusEnum.READY, cubeInstance.getStatus());
+        assertEquals(CubeSegmentStatusEnum.NEW, cubeInstance.getBuildingSegments().get(0).getStatus());
+        assertEquals(1, cubeInstance.getSegments(CubeSegmentStatusEnum.READY).size());
+        assertEquals(1, cubeInstance.getSegments(CubeSegmentStatusEnum.NEW).size());
+        assertEquals(1, cubeInstance.getBuildingSegments().size());
+        assertEquals(1, cubeInstance.getRebuildingSegments().size());
+        assertEquals(0, cubeInstance.getAllocatedStartDate());
+        assertEquals(0, cubeInstance.getAllocatedEndDate());
+
+        // rebuild success
+        System.out.println("Rebuild Success");
+        cubeMgr.updateSegmentOnJobSucceed(cubeInstance, CubeBuildTypeEnum.BUILD, rebuildSegment.getName(), "job_6", System.currentTimeMillis(), 111, 222, 333);
+        assertEquals(CubeStatusEnum.READY, cubeInstance.getStatus());
+        assertEquals(1, cubeInstance.getSegments().size());
+        assertEquals(1, cubeInstance.getSegments(CubeSegmentStatusEnum.READY).size());
+        assertEquals(0, cubeInstance.getBuildingSegments().size());
+        assertEquals(0, cubeInstance.getRebuildingSegments().size());
+        assertEquals(0, cubeInstance.getAllocatedStartDate());
+        assertEquals(0, cubeInstance.getAllocatedEndDate());
+        assertEquals("job_6", cubeInstance.getSegments().get(0).getLastBuildJobID());
+        System.out.println(JsonUtil.writeValueAsIndentString(cubeInstance));
+    }
+
+    @Test(expected = CubeIntegrityException.class)
+    public void testInvalidAppend() throws ParseException, IOException, CubeIntegrityException {
+        // create a new cube
+        MetadataManager metaMgr = getMetadataManager();
+        CubeDesc desc = metaMgr.getCubeDesc("test_kylin_cube_with_slr_desc");
+        createNewCube(desc);
+
+        SimpleDateFormat f = new SimpleDateFormat("yyyy-MM-dd");
+        f.setTimeZone(TimeZone.getTimeZone("GMT"));
+
+        long dateEnd = f.parse("2013-11-12").getTime();
+
+        CubeInstance cubeInstance = cubeMgr.getCube("a_whole_new_cube");
+        assertEquals(CubeStatusEnum.DISABLED, cubeInstance.getStatus());
+        assertEquals(0, cubeInstance.getSegments().size());
+        assertEquals(0, cubeInstance.getBuildingSegments().size());
+        assertEquals(0, cubeInstance.getAllocatedEndDate());
+
+        // initial build
+        System.out.println("Initial Build");
+        CubeSegment initialSegment = cubeMgr.allocateSegments(cubeInstance, CubeBuildTypeEnum.BUILD, desc.getCubePartitionDesc().getPartitionDateStart(), dateEnd).get(0);
+        System.out.println(JsonUtil.writeValueAsIndentString(cubeInstance));
+        assertEquals(CubeStatusEnum.DISABLED, cubeInstance.getStatus());
+        assertEquals(CubeSegmentStatusEnum.NEW, cubeInstance.getBuildingSegments().get(0).getStatus());
+        assertEquals(1, cubeInstance.getSegments().size());
+        assertEquals(1, cubeInstance.getBuildingSegments().size());
+        assertEquals(0, cubeInstance.getRebuildingSegments().size());
+        assertTrue("".equals(initialSegment.getStorageLocationIdentifier()) == false);
+        assertEquals(desc.getCubePartitionDesc().getPartitionDateStart(), cubeInstance.getAllocatedStartDate());
+        assertEquals(dateEnd, cubeInstance.getAllocatedEndDate());
+
+        // initial build success
+        System.out.println("Initial Build Success");
+        cubeMgr.updateSegmentOnJobSucceed(cubeInstance, CubeBuildTypeEnum.BUILD, initialSegment.getName(), "job_1", System.currentTimeMillis(), 111L, 222L, 333L);
+        System.out.println(JsonUtil.writeValueAsIndentString(cubeInstance));
+        assertEquals(CubeStatusEnum.READY, cubeInstance.getStatus());
+        assertEquals(1, cubeInstance.getSegments(CubeSegmentStatusEnum.READY).size());
+        assertEquals(0, cubeInstance.getBuildingSegments().size());
+        assertEquals(0, cubeInstance.getRebuildingSegments().size());
+        assertEquals(desc.getCubePartitionDesc().getPartitionDateStart(), cubeInstance.getAllocatedStartDate());
+        assertEquals(dateEnd, cubeInstance.getAllocatedEndDate());
+
+        // incremental build
+        System.out.println("Invalid Incremental Build");
+        long dateEnd2 = f.parse("2013-12-12").getTime();
+        cubeMgr.allocateSegments(cubeInstance, CubeBuildTypeEnum.BUILD, dateEnd + 1000, dateEnd2);
+
+    }
+
+    @Test
+    public void testInitialAndUpsert() throws ParseException, IOException, CubeIntegrityException {
+        // create a new cube
+        MetadataManager metaMgr = getMetadataManager();
+        CubeDesc desc = metaMgr.getCubeDesc("test_kylin_cube_without_slr_left_join_desc");
+        createNewCube(desc);
+
+        SimpleDateFormat f = new SimpleDateFormat("yyyy-MM-dd");
+        f.setTimeZone(TimeZone.getTimeZone("GMT"));
+
+        long dateEnd = f.parse("2013-11-12").getTime();
+
+        CubeInstance cubeInstance = cubeMgr.getCube("a_whole_new_cube");
+        assertEquals(CubeStatusEnum.DISABLED, cubeInstance.getStatus());
+        assertEquals(0, cubeInstance.getSegments().size());
+        assertEquals(0, cubeInstance.getBuildingSegments().size());
+        assertEquals(0, cubeInstance.getAllocatedEndDate());
+
+        // initial build
+        System.out.println("Initial Build");
+        CubeSegment initialSegment = cubeMgr.allocateSegments(cubeInstance, CubeBuildTypeEnum.BUILD, desc.getCubePartitionDesc().getPartitionDateStart(), dateEnd).get(0);
+        System.out.println(JsonUtil.writeValueAsIndentString(cubeInstance));
+        assertEquals(CubeStatusEnum.DISABLED, cubeInstance.getStatus());
+        for (CubeSegment cubeSegment : cubeInstance.getBuildingSegments()) {
+            assertEquals(CubeSegmentStatusEnum.NEW, cubeSegment.getStatus());
+        }
+        assertEquals(1, cubeInstance.getSegments().size());
+        assertEquals(1, cubeInstance.getBuildingSegments().size());
+        assertEquals(0, cubeInstance.getRebuildingSegments().size());
+        assertTrue("".equals(initialSegment.getStorageLocationIdentifier()) == false);
+        assertEquals(desc.getCubePartitionDesc().getPartitionDateStart(), cubeInstance.getAllocatedStartDate());
+        assertEquals(dateEnd, cubeInstance.getAllocatedEndDate());
+
+        // initial build success
+        System.out.println("Initial Build Success");
+        cubeMgr.updateSegmentOnJobSucceed(cubeInstance, CubeBuildTypeEnum.BUILD, initialSegment.getName(), "job_1", System.currentTimeMillis(), 111L, 222L, 333L);
+        System.out.println(JsonUtil.writeValueAsIndentString(cubeInstance));
+        assertEquals(CubeStatusEnum.READY, cubeInstance.getStatus());
+        assertEquals(1, cubeInstance.getSegments(CubeSegmentStatusEnum.READY).size());
+        assertEquals(0, cubeInstance.getBuildingSegments().size());
+        assertEquals(desc.getCubePartitionDesc().getPartitionDateStart(), cubeInstance.getAllocatedStartDate());
+        assertEquals(dateEnd, cubeInstance.getAllocatedEndDate());
+
+        // upsert build
+        System.out.println("Upsert Build");
+        long start = f.parse("2013-11-01").getTime();
+        long dateEnd2 = f.parse("2013-12-12").getTime();
+        List<CubeSegment> upsertSegments = cubeMgr.allocateSegments(cubeInstance, CubeBuildTypeEnum.BUILD, start, dateEnd2);
+        System.out.println(JsonUtil.writeValueAsIndentString(cubeInstance));
+        assertEquals(2, upsertSegments.size());
+        assertEquals(CubeStatusEnum.READY, cubeInstance.getStatus());
+        assertEquals(3, cubeInstance.getSegments().size());
+        assertEquals(2, cubeInstance.getSegments(CubeSegmentStatusEnum.NEW).size());
+        assertEquals(2, cubeInstance.getBuildingSegments().size());
+        assertEquals(1, cubeInstance.getRebuildingSegments().size());
+        assertEquals(0, cubeInstance.getMergingSegments().size());
+        assertEquals(desc.getCubePartitionDesc().getPartitionDateStart(), cubeInstance.getAllocatedStartDate());
+        assertEquals(dateEnd2, cubeInstance.getAllocatedEndDate());
+        assertEquals(0L, cubeInstance.getBuildingSegments().get(0).getDateRangeStart());
+        assertEquals(dateEnd2, cubeInstance.getBuildingSegments().get(1).getDateRangeEnd());
+
+        // upsert build success
+        System.out.println("Upsert Build Success");
+        cubeMgr.updateSegmentOnJobSucceed(cubeInstance, CubeBuildTypeEnum.BUILD, upsertSegments.get(0).getName(), "job_2", System.currentTimeMillis(), 111L, 222L, 333L);
+        System.out.println(JsonUtil.writeValueAsIndentString(cubeInstance));
+        cubeMgr.updateSegmentOnJobSucceed(cubeInstance, CubeBuildTypeEnum.BUILD, upsertSegments.get(1).getName(), "job_3", System.currentTimeMillis(), 111L, 222L, 333L);
+        System.out.println(JsonUtil.writeValueAsIndentString(cubeInstance));
+        assertEquals(CubeStatusEnum.READY, cubeInstance.getStatus());
+        assertEquals(2, cubeInstance.getSegments().size());
+        assertEquals(2, cubeInstance.getSegments(CubeSegmentStatusEnum.READY).size());
+        assertEquals(0, cubeInstance.getBuildingSegments().size());
+        assertEquals(desc.getCubePartitionDesc().getPartitionDateStart(), cubeInstance.getAllocatedStartDate());
+        assertEquals(dateEnd2, cubeInstance.getAllocatedEndDate());
+
+        // upsert build again
+        System.out.println("Upsert Build");
+        long start2 = f.parse("2013-12-01").getTime();
+        long dateEnd3 = f.parse("2013-12-31").getTime();
+        List<CubeSegment> upsertSegments2 = cubeMgr.allocateSegments(cubeInstance, CubeBuildTypeEnum.BUILD, start2, dateEnd3);
+        System.out.println(JsonUtil.writeValueAsIndentString(cubeInstance));
+        assertEquals(2, upsertSegments2.size());
+        assertEquals(CubeStatusEnum.READY, cubeInstance.getStatus());
+        assertEquals(4, cubeInstance.getSegments().size());
+        assertEquals(2, cubeInstance.getSegments(CubeSegmentStatusEnum.NEW).size());
+        assertEquals(2, cubeInstance.getBuildingSegments().size());
+        assertEquals(1, cubeInstance.getRebuildingSegments().size());
+        assertEquals(0, cubeInstance.getMergingSegments().size());
+        assertEquals(dateEnd3, cubeInstance.getAllocatedEndDate());
+        // building segment 1 from 2013-11-01 to 2013-12-01
+        assertEquals(f.parse("2013-11-01").getTime(), cubeInstance.getBuildingSegments().get(0).getDateRangeStart());
+        assertEquals(f.parse("2013-12-01").getTime(), cubeInstance.getBuildingSegments().get(0).getDateRangeEnd());
+        // building segment 2 from 2013-12-01 to 2013-12-31
+        assertEquals(f.parse("2013-12-01").getTime(), cubeInstance.getBuildingSegments().get(1).getDateRangeStart());
+        assertEquals(f.parse("2013-12-31").getTime(), cubeInstance.getBuildingSegments().get(1).getDateRangeEnd());
+
+        // upsert build success
+        System.out.println("Upsert Build Success");
+        cubeMgr.updateSegmentOnJobSucceed(cubeInstance, CubeBuildTypeEnum.BUILD, upsertSegments2.get(1).getName(), "job_5", System.currentTimeMillis(), 111L, 222L, 333L);
+        cubeMgr.updateSegmentOnJobSucceed(cubeInstance, CubeBuildTypeEnum.BUILD, upsertSegments2.get(0).getName(), "job_4", System.currentTimeMillis(), 111L, 222L, 333L);
+        System.out.println(JsonUtil.writeValueAsIndentString(cubeInstance));
+        assertEquals(CubeStatusEnum.READY, cubeInstance.getStatus());
+        assertEquals(3, cubeInstance.getSegments().size());
+        assertEquals(3, cubeInstance.getSegments(CubeSegmentStatusEnum.READY).size());
+        assertEquals(0, cubeInstance.getBuildingSegments().size());
+        assertEquals(0, cubeInstance.getMergingSegments().size());
+        assertEquals(cubeInstance.getDescriptor().getCubePartitionDesc().getPartitionDateStart(), cubeInstance.getAllocatedStartDate());
+        assertEquals(dateEnd3, cubeInstance.getAllocatedEndDate());
+        // segment 1 from 1970-01-01 to 2013-11-01
+        assertEquals(cubeInstance.getDescriptor().getCubePartitionDesc().getPartitionDateStart(), cubeInstance.getSegments().get(0).getDateRangeStart());
+        assertEquals(f.parse("2013-11-01").getTime(), cubeInstance.getSegments().get(0).getDateRangeEnd());
+        // segment 2 from 2013-11-01 to 2013-12-01
+        assertEquals(f.parse("2013-11-01").getTime(), cubeInstance.getSegments().get(1).getDateRangeStart());
+        assertEquals(f.parse("2013-12-01").getTime(), cubeInstance.getSegments().get(1).getDateRangeEnd());
+        // segment 3 from 2013-12-01 to 2013-12-31
+        assertEquals(f.parse("2013-12-01").getTime(), cubeInstance.getSegments().get(2).getDateRangeStart());
+        assertEquals(f.parse("2013-12-31").getTime(), cubeInstance.getSegments().get(2).getDateRangeEnd());
+
+        // upsert build again
+        System.out.println("Upsert Build");
+        List<CubeSegment> upsertSegments3 = cubeMgr.allocateSegments(cubeInstance, CubeBuildTypeEnum.BUILD, f.parse("2013-10-01").getTime(), f.parse("2014-02-01").getTime());
+        System.out.println(JsonUtil.writeValueAsIndentString(cubeInstance));
+        assertEquals(2, upsertSegments3.size());
+        assertEquals(CubeStatusEnum.READY, cubeInstance.getStatus());
+        assertEquals(5, cubeInstance.getSegments().size());
+        assertEquals(2, cubeInstance.getSegments(CubeSegmentStatusEnum.NEW).size());
+        assertEquals(2, cubeInstance.getBuildingSegments().size());
+        assertEquals(3, cubeInstance.getRebuildingSegments().size());
+        assertEquals(0, cubeInstance.getMergingSegments().size());
+        assertEquals(f.parse("2014-02-01").getTime(), cubeInstance.getAllocatedEndDate());
+        // building segment 1 from 2013-11-01 to 2013-10-01
+        assertEquals(cubeInstance.getDescriptor().getCubePartitionDesc().getPartitionDateStart(), cubeInstance.getBuildingSegments().get(0).getDateRangeStart());
+        assertEquals(f.parse("2013-10-01").getTime(), cubeInstance.getBuildingSegments().get(0).getDateRangeEnd());
+        // building segment 2 from 2013-10-01 to 2014-02-01
+        assertEquals(f.parse("2013-10-01").getTime(), cubeInstance.getBuildingSegments().get(1).getDateRangeStart());
+        assertEquals(f.parse("2014-02-01").getTime(), cubeInstance.getBuildingSegments().get(1).getDateRangeEnd());
+
+        // upsert build success
+        System.out.println("Upsert Build Success");
+        cubeMgr.updateSegmentOnJobSucceed(cubeInstance, CubeBuildTypeEnum.BUILD, upsertSegments3.get(1).getName(), "job_7", System.currentTimeMillis(), 111L, 222L, 333L);
+        cubeMgr.updateSegmentOnJobSucceed(cubeInstance, CubeBuildTypeEnum.BUILD, upsertSegments3.get(0).getName(), "job_6", System.currentTimeMillis(), 111L, 222L, 333L);
+        System.out.println(JsonUtil.writeValueAsIndentString(cubeInstance));
+        assertEquals(CubeStatusEnum.READY, cubeInstance.getStatus());
+        assertEquals(2, cubeInstance.getSegments().size());
+        assertEquals(2, cubeInstance.getSegments(CubeSegmentStatusEnum.READY).size());
+        assertEquals(0, cubeInstance.getBuildingSegments().size());
+        assertEquals(0, cubeInstance.getMergingSegments().size());
+        assertEquals(cubeInstance.getDescriptor().getCubePartitionDesc().getPartitionDateStart(), cubeInstance.getAllocatedStartDate());
+        assertEquals(f.parse("2014-02-01").getTime(), cubeInstance.getAllocatedEndDate());
+        // segment 1 from 1970-01-01 to 2013-10-01
+        assertEquals(cubeInstance.getDescriptor().getCubePartitionDesc().getPartitionDateStart(), cubeInstance.getSegments().get(0).getDateRangeStart());
+        assertEquals(f.parse("2013-10-01").getTime(), cubeInstance.getSegments().get(0).getDateRangeEnd());
+        // segment 2 from 2013-10-01 to 2014-02-01
+        assertEquals(f.parse("2013-10-01").getTime(), cubeInstance.getSegments().get(1).getDateRangeStart());
+        assertEquals(f.parse("2014-02-01").getTime(), cubeInstance.getSegments().get(1).getDateRangeEnd());
+    }
+
+    @Test
+    public void testInitialAndUpsert2() throws ParseException, IOException, CubeIntegrityException {
+        // create a new cube
+        MetadataManager metaMgr = getMetadataManager();
+        CubeDesc desc = metaMgr.getCubeDesc("test_kylin_cube_without_slr_left_join_desc");
+        createNewCube(desc);
+
+        SimpleDateFormat f = new SimpleDateFormat("yyyy-MM-dd");
+        f.setTimeZone(TimeZone.getTimeZone("GMT"));
+
+        long dateEnd = f.parse("2013-01-01").getTime();
+
+        CubeInstance cubeInstance = cubeMgr.getCube("a_whole_new_cube");
+        assertEquals(CubeStatusEnum.DISABLED, cubeInstance.getStatus());
+        assertEquals(0, cubeInstance.getSegments().size());
+        assertEquals(0, cubeInstance.getBuildingSegments().size());
+        assertEquals(0, cubeInstance.getAllocatedEndDate());
+
+        // initial build
+        System.out.println("Initial Build");
+        CubeSegment initialSegment = cubeMgr.allocateSegments(cubeInstance, CubeBuildTypeEnum.BUILD, desc.getCubePartitionDesc().getPartitionDateStart(), dateEnd).get(0);
+        System.out.println(JsonUtil.writeValueAsIndentString(cubeInstance));
+        assertEquals(CubeStatusEnum.DISABLED, cubeInstance.getStatus());
+        for (CubeSegment cubeSegment : cubeInstance.getBuildingSegments()) {
+            assertEquals(CubeSegmentStatusEnum.NEW, cubeSegment.getStatus());
+        }
+        assertEquals(1, cubeInstance.getSegments().size());
+        assertEquals(1, cubeInstance.getBuildingSegments().size());
+        assertEquals(0, cubeInstance.getRebuildingSegments().size());
+        assertTrue("".equals(initialSegment.getStorageLocationIdentifier()) == false);
+        assertEquals(desc.getCubePartitionDesc().getPartitionDateStart(), cubeInstance.getAllocatedStartDate());
+        assertEquals(dateEnd, cubeInstance.getAllocatedEndDate());
+
+        // initial build success
+        System.out.println("Initial Build Success");
+        cubeMgr.updateSegmentOnJobSucceed(cubeInstance, CubeBuildTypeEnum.BUILD, initialSegment.getName(), "job_1", System.currentTimeMillis(), 111L, 222L, 333L);
+        System.out.println(JsonUtil.writeValueAsIndentString(cubeInstance));
+        assertEquals(CubeStatusEnum.READY, cubeInstance.getStatus());
+        assertEquals(1, cubeInstance.getSegments(CubeSegmentStatusEnum.READY).size());
+        assertEquals(0, cubeInstance.getBuildingSegments().size());
+        assertEquals(desc.getCubePartitionDesc().getPartitionDateStart(), cubeInstance.getAllocatedStartDate());
+        assertEquals(dateEnd, cubeInstance.getAllocatedEndDate());
+
+        // upsert build
+        System.out.println("Upsert Build");
+        long start = f.parse("2013-01-01").getTime();
+        long dateEnd2 = f.parse("2014-01-01").getTime();
+        List<CubeSegment> upsertSegments = cubeMgr.allocateSegments(cubeInstance, CubeBuildTypeEnum.BUILD, start, dateEnd2);
+        System.out.println(JsonUtil.writeValueAsIndentString(cubeInstance));
+        assertEquals(1, upsertSegments.size());
+        assertEquals(CubeStatusEnum.READY, cubeInstance.getStatus());
+        assertEquals(2, cubeInstance.getSegments().size());
+        assertEquals(1, cubeInstance.getSegments(CubeSegmentStatusEnum.NEW).size());
+        assertEquals(1, cubeInstance.getBuildingSegments().size());
+        assertEquals(0, cubeInstance.getRebuildingSegments().size());
+        assertEquals(0, cubeInstance.getMergingSegments().size());
+        assertEquals(desc.getCubePartitionDesc().getPartitionDateStart(), cubeInstance.getAllocatedStartDate());
+        assertEquals(dateEnd2, cubeInstance.getAllocatedEndDate());
+        assertEquals(start, cubeInstance.getBuildingSegments().get(0).getDateRangeStart());
+        assertEquals(dateEnd2, cubeInstance.getBuildingSegments().get(0).getDateRangeEnd());
+
+        // upsert build success
+        System.out.println("Upsert Build Success");
+        cubeMgr.updateSegmentOnJobSucceed(cubeInstance, CubeBuildTypeEnum.BUILD, upsertSegments.get(0).getName(), "job_2", System.currentTimeMillis(), 111L, 222L, 333L);
+        System.out.println(JsonUtil.writeValueAsIndentString(cubeInstance));
+
+        assertEquals(CubeStatusEnum.READY, cubeInstance.getStatus());
+        assertEquals(2, cubeInstance.getSegments().size());
+        assertEquals(2, cubeInstance.getSegments(CubeSegmentStatusEnum.READY).size());
+        assertEquals(0, cubeInstance.getBuildingSegments().size());
+        assertEquals(desc.getCubePartitionDesc().getPartitionDateStart(), cubeInstance.getAllocatedStartDate());
+        assertEquals(dateEnd2, cubeInstance.getAllocatedEndDate());
+
+        // upsert build again
+        System.out.println("Upsert Build");
+        List<CubeSegment> upsertSegments3 = cubeMgr.allocateSegments(cubeInstance, CubeBuildTypeEnum.BUILD, f.parse("2013-10-01").getTime(), f.parse("2014-02-01").getTime());
+        System.out.println(JsonUtil.writeValueAsIndentString(cubeInstance));
+        assertEquals(2, upsertSegments3.size());
+        assertEquals(CubeStatusEnum.READY, cubeInstance.getStatus());
+        assertEquals(4, cubeInstance.getSegments().size());
+        assertEquals(2, cubeInstance.getSegments(CubeSegmentStatusEnum.NEW).size());
+        assertEquals(2, cubeInstance.getBuildingSegments().size());
+        assertEquals(1, cubeInstance.getRebuildingSegments().size());
+        assertEquals(0, cubeInstance.getMergingSegments().size());
+        assertEquals(f.parse("2014-02-01").getTime(), cubeInstance.getAllocatedEndDate());
+        // building segment 1 from 2013-11-01 to 2013-10-01
+        assertEquals(f.parse("2013-01-01").getTime(), cubeInstance.getBuildingSegments().get(0).getDateRangeStart());
+        assertEquals(f.parse("2013-10-01").getTime(), cubeInstance.getBuildingSegments().get(0).getDateRangeEnd());
+        // building segment 2 from 2013-10-01 to 2014-02-01
+        assertEquals(f.parse("2013-10-01").getTime(), cubeInstance.getBuildingSegments().get(1).getDateRangeStart());
+        assertEquals(f.parse("2014-02-01").getTime(), cubeInstance.getBuildingSegments().get(1).getDateRangeEnd());
+
+        // upsert build success
+        System.out.println("Upsert Build Success");
+        cubeMgr.updateSegmentOnJobSucceed(cubeInstance, CubeBuildTypeEnum.BUILD, upsertSegments3.get(1).getName(), "job_7", System.currentTimeMillis(), 111L, 222L, 333L);
+        cubeMgr.updateSegmentOnJobSucceed(cubeInstance, CubeBuildTypeEnum.BUILD, upsertSegments3.get(0).getName(), "job_6", System.currentTimeMillis(), 111L, 222L, 333L);
+        System.out.println(JsonUtil.writeValueAsIndentString(cubeInstance));
+        assertEquals(CubeStatusEnum.READY, cubeInstance.getStatus());
+        assertEquals(3, cubeInstance.getSegments().size());
+        assertEquals(3, cubeInstance.getSegments(CubeSegmentStatusEnum.READY).size());
+        assertEquals(0, cubeInstance.getBuildingSegments().size());
+        assertEquals(0, cubeInstance.getMergingSegments().size());
+        assertEquals(cubeInstance.getDescriptor().getCubePartitionDesc().getPartitionDateStart(), cubeInstance.getAllocatedStartDate());
+        assertEquals(f.parse("2014-02-01").getTime(), cubeInstance.getAllocatedEndDate());
+        // segment 1 from 1970-01-01 to 2013-10-01
+        assertEquals(cubeInstance.getDescriptor().getCubePartitionDesc().getPartitionDateStart(), cubeInstance.getSegments().get(0).getDateRangeStart());
+        assertEquals(f.parse("2013-01-01").getTime(), cubeInstance.getSegments().get(0).getDateRangeEnd());
+        // segment 2 from 2013-10-01 to 2014-02-01
+        assertEquals(f.parse("2013-01-01").getTime(), cubeInstance.getSegments().get(1).getDateRangeStart());
+        assertEquals(f.parse("2013-10-01").getTime(), cubeInstance.getSegments().get(1).getDateRangeEnd());
+        // segment 3 from 2013-10-01 to 2014-02-01
+        assertEquals(f.parse("2013-10-01").getTime(), cubeInstance.getSegments().get(2).getDateRangeStart());
+        assertEquals(f.parse("2014-02-01").getTime(), cubeInstance.getSegments().get(2).getDateRangeEnd());
+    }
+
+    @Test(expected = CubeIntegrityException.class)
+    public void testInvalidUpsert() throws IOException, CubeIntegrityException, ParseException {
+        // create a new cube
+        MetadataManager metaMgr = getMetadataManager();
+        CubeDesc desc = metaMgr.getCubeDesc("test_kylin_cube_without_slr_left_join_desc");
+        createNewCube(desc);
+
+        SimpleDateFormat f = new SimpleDateFormat("yyyy-MM-dd");
+        f.setTimeZone(TimeZone.getTimeZone("GMT"));
+
+        long dateEnd = f.parse("2013-11-12").getTime();
+
+        CubeInstance cubeInstance = cubeMgr.getCube("a_whole_new_cube");
+        assertEquals(CubeStatusEnum.DISABLED, cubeInstance.getStatus());
+        assertEquals(0, cubeInstance.getSegments().size());
+        assertEquals(0, cubeInstance.getBuildingSegments().size());
+        assertEquals(0, cubeInstance.getAllocatedEndDate());
+
+        // initial build
+        System.out.println("Initial Build");
+        CubeSegment initialSegment = cubeMgr.allocateSegments(cubeInstance, CubeBuildTypeEnum.BUILD, desc.getCubePartitionDesc().getPartitionDateStart(), dateEnd).get(0);
+        System.out.println(JsonUtil.writeValueAsIndentString(cubeInstance));
+        assertEquals(CubeStatusEnum.DISABLED, cubeInstance.getStatus());
+        for (CubeSegment cubeSegment : cubeInstance.getBuildingSegments()) {
+            assertEquals(CubeSegmentStatusEnum.NEW, cubeSegment.getStatus());
+        }
+        assertEquals(1, cubeInstance.getSegments().size());
+        assertEquals(1, cubeInstance.getBuildingSegments().size());
+        assertEquals(0, cubeInstance.getRebuildingSegments().size());
+        assertTrue("".equals(initialSegment.getStorageLocationIdentifier()) == false);
+        assertEquals(desc.getCubePartitionDesc().getPartitionDateStart(), cubeInstance.getAllocatedStartDate());
+        assertEquals(dateEnd, cubeInstance.getAllocatedEndDate());
+
+        // initial build success
+        System.out.println("Initial Build Success");
+        cubeMgr.updateSegmentOnJobSucceed(cubeInstance, CubeBuildTypeEnum.BUILD, initialSegment.getName(), "job_1", System.currentTimeMillis(), 111L, 222L, 333L);
+        System.out.println(JsonUtil.writeValueAsIndentString(cubeInstance));
+        assertEquals(CubeStatusEnum.READY, cubeInstance.getStatus());
+        assertEquals(1, cubeInstance.getSegments(CubeSegmentStatusEnum.READY).size());
+        assertEquals(0, cubeInstance.getBuildingSegments().size());
+        assertEquals(desc.getCubePartitionDesc().getPartitionDateStart(), cubeInstance.getAllocatedStartDate());
+        assertEquals(dateEnd, cubeInstance.getAllocatedEndDate());
+
+        // upsert build
+        // time gap in new segment
+        System.out.println("Upsert Build");
+        long start = f.parse("2013-11-13").getTime();
+        long dateEnd2 = f.parse("2013-12-12").getTime();
+        cubeMgr.allocateSegments(cubeInstance, CubeBuildTypeEnum.BUILD, start, dateEnd2);
+    }
+
+    @Test(expected = CubeIntegrityException.class)
+    public void testInvalidUpsert2() throws IOException, CubeIntegrityException, ParseException {
+        // create a new cube
+        MetadataManager metaMgr = getMetadataManager();
+        CubeDesc desc = metaMgr.getCubeDesc("test_kylin_cube_without_slr_left_join_desc");
+        createNewCube(desc);
+
+        SimpleDateFormat f = new SimpleDateFormat("yyyy-MM-dd");
+        f.setTimeZone(TimeZone.getTimeZone("GMT"));
+
+        long dateEnd = f.parse("2013-11-12").getTime();
+
+        CubeInstance cubeInstance = cubeMgr.getCube("a_whole_new_cube");
+        assertEquals(CubeStatusEnum.DISABLED, cubeInstance.getStatus());
+        assertEquals(0, cubeInstance.getSegments().size());
+        assertEquals(0, cubeInstance.getBuildingSegments().size());
+        assertEquals(0, cubeInstance.getAllocatedEndDate());
+
+        // initial build
+        System.out.println("Initial Build");
+        CubeSegment initialSegment = cubeMgr.allocateSegments(cubeInstance, CubeBuildTypeEnum.BUILD, desc.getCubePartitionDesc().getPartitionDateStart(), dateEnd).get(0);
+        System.out.println(JsonUtil.writeValueAsIndentString(cubeInstance));
+        assertEquals(CubeStatusEnum.DISABLED, cubeInstance.getStatus());
+        for (CubeSegment cubeSegment : cubeInstance.getBuildingSegments()) {
+            assertEquals(CubeSegmentStatusEnum.NEW, cubeSegment.getStatus());
+        }
+        assertEquals(1, cubeInstance.getSegments().size());
+        assertEquals(1, cubeInstance.getBuildingSegments().size());
+        assertEquals(0, cubeInstance.getRebuildingSegments().size());
+        assertTrue("".equals(initialSegment.getStorageLocationIdentifier()) == false);
+        assertEquals(desc.getCubePartitionDesc().getPartitionDateStart(), cubeInstance.getAllocatedStartDate());
+        assertEquals(dateEnd, cubeInstance.getAllocatedEndDate());
+
+        // initial build success
+        System.out.println("Initial Build Success");
+        cubeMgr.updateSegmentOnJobSucceed(cubeInstance, CubeBuildTypeEnum.BUILD, initialSegment.getName(), "job_1", System.currentTimeMillis(), 111L, 222L, 333L);
+        System.out.println(JsonUtil.writeValueAsIndentString(cubeInstance));
+        assertEquals(CubeStatusEnum.READY, cubeInstance.getStatus());
+        assertEquals(1, cubeInstance.getSegments(CubeSegmentStatusEnum.READY).size());
+        assertEquals(0, cubeInstance.getBuildingSegments().size());
+        assertEquals(desc.getCubePartitionDesc().getPartitionDateStart(), cubeInstance.getAllocatedStartDate());
+        assertEquals(dateEnd, cubeInstance.getAllocatedEndDate());
+
+        // upsert build
+        // time gap in new segment
+        System.out.println("Upsert Build");
+        long start = f.parse("2013-11-01").getTime();
+        long dateEnd2 = f.parse("2013-11-02").getTime();
+
+        cubeMgr.allocateSegments(cubeInstance, CubeBuildTypeEnum.BUILD, start, dateEnd2);
+        System.out.println(JsonUtil.writeValueAsIndentString(cubeInstance));
+    }
+
+    private MetadataManager getMetadataManager() {
+        return MetadataManager.getInstance(getTestConfig());
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/cube/src/test/java/com/kylinolap/cube/common/BytesSplitterTest.java
----------------------------------------------------------------------
diff --git a/cube/src/test/java/com/kylinolap/cube/common/BytesSplitterTest.java b/cube/src/test/java/com/kylinolap/cube/common/BytesSplitterTest.java
new file mode 100644
index 0000000..6d378db
--- /dev/null
+++ b/cube/src/test/java/com/kylinolap/cube/common/BytesSplitterTest.java
@@ -0,0 +1,50 @@
+/*
+ * 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.cube.common;
+
+import static org.junit.Assert.*;
+
+import org.junit.Test;
+
+/**
+ * @author George Song (ysong1)
+ * 
+ */
+public class BytesSplitterTest {
+
+    @Test
+    public void test() {
+        BytesSplitter bytesSplitter = new BytesSplitter(10, 15);
+        byte[] input = "2013-02-17Collectibles".getBytes();
+        bytesSplitter.split(input, input.length, (byte) 127);
+
+        assertEquals(2, bytesSplitter.getBufferSize());
+        assertEquals("2013-02-17", new String(bytesSplitter.getSplitBuffers()[0].value, 0, bytesSplitter.getSplitBuffers()[0].length));
+        assertEquals("Collectibles", new String(bytesSplitter.getSplitBuffers()[1].value, 0, bytesSplitter.getSplitBuffers()[1].length));
+    }
+
+    @Test
+    public void testNullValue() {
+        BytesSplitter bytesSplitter = new BytesSplitter(10, 15);
+        byte[] input = "2013-02-17Collectibles".getBytes();
+        bytesSplitter.split(input, input.length, (byte) 127);
+
+        assertEquals(3, bytesSplitter.getBufferSize());
+        assertEquals("2013-02-17", new String(bytesSplitter.getSplitBuffers()[0].value, 0, bytesSplitter.getSplitBuffers()[0].length));
+        assertEquals("", new String(bytesSplitter.getSplitBuffers()[1].value, 0, bytesSplitter.getSplitBuffers()[1].length));
+        assertEquals("Collectibles", new String(bytesSplitter.getSplitBuffers()[2].value, 0, bytesSplitter.getSplitBuffers()[2].length));
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/cube/src/test/java/com/kylinolap/cube/common/RowKeySplitterTest.java
----------------------------------------------------------------------
diff --git a/cube/src/test/java/com/kylinolap/cube/common/RowKeySplitterTest.java b/cube/src/test/java/com/kylinolap/cube/common/RowKeySplitterTest.java
new file mode 100644
index 0000000..5d5d377
--- /dev/null
+++ b/cube/src/test/java/com/kylinolap/cube/common/RowKeySplitterTest.java
@@ -0,0 +1,69 @@
+/*
+ * 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.cube.common;
+
+import static org.junit.Assert.*;
+
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+import com.kylinolap.common.util.LocalFileMetadataTestCase;
+import com.kylinolap.cube.CubeInstance;
+import com.kylinolap.cube.CubeManager;
+import com.kylinolap.metadata.MetadataManager;
+
+/**
+ * @author George Song (ysong1)
+ * 
+ */
+public class RowKeySplitterTest extends LocalFileMetadataTestCase {
+
+    @Before
+    public void setUp() throws Exception {
+        this.createTestMetadata();
+        MetadataManager.removeInstance(this.getTestConfig());
+    }
+
+    @After
+    public void after() throws Exception {
+        this.cleanupTestMetadata();
+    }
+
+    @Test
+    public void testWithSlr() throws Exception {
+        CubeInstance cube = CubeManager.getInstance(this.getTestConfig()).getCube("TEST_KYLIN_CUBE_WITH_SLR_READY");
+
+        RowKeySplitter rowKeySplitter = new RowKeySplitter(cube.getFirstSegment(), 10, 20);
+        // base cuboid rowkey
+        byte[] input = { 0, 0, 0, 0, 0, 0, 1, -1, 49, 48, 48, 48, 48, 48, 48, 48, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 11, 54, -105, 55, 13, 71, 114, 65, 66, 73, 78, 9, 9, 9, 9, 9, 9, 9, 9, 0, 10, 0 };
+        rowKeySplitter.split(input, input.length);
+
+        assertEquals(10, rowKeySplitter.getBufferSize());
+    }
+
+    @Test
+    public void testWithoutSlr() throws Exception {
+        CubeInstance cube = CubeManager.getInstance(this.getTestConfig()).getCube("TEST_KYLIN_CUBE_WITHOUT_SLR_READY");
+
+        RowKeySplitter rowKeySplitter = new RowKeySplitter(cube.getFirstSegment(), 10, 20);
+        // base cuboid rowkey
+        byte[] input = { 0, 0, 0, 0, 0, 0, 0, -1, 11, 55, -13, 13, 22, 34, 121, 70, 80, 45, 71, 84, 67, 9, 9, 9, 9, 9, 9, 0, 10, 5 };
+        rowKeySplitter.split(input, input.length);
+
+        assertEquals(9, rowKeySplitter.getBufferSize());
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/cube/src/test/java/com/kylinolap/cube/cuboid/CombinationTest.java
----------------------------------------------------------------------
diff --git a/cube/src/test/java/com/kylinolap/cube/cuboid/CombinationTest.java b/cube/src/test/java/com/kylinolap/cube/cuboid/CombinationTest.java
new file mode 100644
index 0000000..1cd48bd
--- /dev/null
+++ b/cube/src/test/java/com/kylinolap/cube/cuboid/CombinationTest.java
@@ -0,0 +1,92 @@
+/*
+ * 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.cube.cuboid;
+
+import static org.junit.Assert.*;
+
+import org.junit.Test;
+
+/**
+ * @author xjiang
+ * 
+ */
+public class CombinationTest {
+
+    public int findSmallerSibling(long valueBits, long valueMask) {
+        if ((valueBits | valueMask) != valueMask) {
+            throw new IllegalArgumentException("Dismatch " + Long.toBinaryString(valueBits) + " from " + Long.toBinaryString(valueMask));
+        }
+
+        int n = Long.bitCount(valueMask);
+        int k = Long.bitCount(valueBits);
+        long[] bitMasks = new long[n];
+        long leftBits = valueMask;
+        for (int i = 0; i < n; i++) {
+            long lowestBit = Long.lowestOneBit(leftBits);
+            bitMasks[i] = lowestBit;
+            leftBits &= ~lowestBit;
+        }
+        return combination(valueBits, bitMasks, 0, 0L, k);
+    }
+
+    private int combination(long valueBits, long[] bitMasks, int offset, long prefix, int k) {
+        if (k == 0) {
+            if (prefix < valueBits) {
+                System.out.println(Long.toBinaryString(prefix));
+                return 1;
+            } else {
+                return 0;
+            }
+        } else {
+            int count = 0;
+            for (int i = offset; i < bitMasks.length; i++) {
+                long newPrefix = prefix | bitMasks[i];
+                if (newPrefix < valueBits) {
+                    count += combination(valueBits, bitMasks, i + 1, newPrefix, k - 1);
+                }
+            }
+            return count;
+        }
+    }
+
+    private long calculateCombination(int n, int k) {
+        if (n < k) {
+            throw new IllegalArgumentException("N < K");
+        }
+        long res = 1;
+        for (int i = n - k + 1; i <= n; i++) {
+            res *= i;
+        }
+        for (int i = 1; i <= k; i++) {
+            res /= i;
+        }
+        return res;
+    }
+
+    @Test
+    public void testComb3() {
+        long valueBits = 1 << 4 | 1 << 6 | 1 << 8;
+        System.out.println("value = " + Long.toBinaryString(valueBits) + ", count = " + Long.bitCount(valueBits));
+        long valueMask = (long) Math.pow(2, 10) - 1;
+        System.out.println("mask = " + Long.toBinaryString(valueMask) + ", count = " + Long.bitCount(valueMask));
+        System.out.println("************");
+        int count = findSmallerSibling(valueBits, valueMask);
+        System.out.println("smaller sibling count = " + count);
+        int cnk = (int) calculateCombination(Long.bitCount(valueMask), Long.bitCount(valueBits));
+        assertTrue(cnk > count);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/cube/src/test/java/com/kylinolap/cube/cuboid/CuboidSchedulerTest.java
----------------------------------------------------------------------
diff --git a/cube/src/test/java/com/kylinolap/cube/cuboid/CuboidSchedulerTest.java b/cube/src/test/java/com/kylinolap/cube/cuboid/CuboidSchedulerTest.java
new file mode 100644
index 0000000..c3e4540
--- /dev/null
+++ b/cube/src/test/java/com/kylinolap/cube/cuboid/CuboidSchedulerTest.java
@@ -0,0 +1,309 @@
+/*
+ * 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.cube.cuboid;
+
+import static org.junit.Assert.*;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Ignore;
+import org.junit.Test;
+
+import com.kylinolap.common.util.LocalFileMetadataTestCase;
+import com.kylinolap.metadata.MetadataManager;
+import com.kylinolap.metadata.model.cube.CubeDesc;
+
+/**
+ * @author George Song (ysong1)
+ * 
+ */
+public class CuboidSchedulerTest extends LocalFileMetadataTestCase {
+    @Before
+    public void setUp() throws Exception {
+        this.createTestMetadata();
+        MetadataManager.removeInstance(this.getTestConfig());
+    }
+
+    @After
+    public void after() throws Exception {
+        this.cleanupTestMetadata();
+    }
+
+    static long toLong(String bin) {
+        return Long.parseLong(bin, 2);
+    }
+
+    static String toString(long l) {
+        return Long.toBinaryString(l);
+    }
+
+    static String toString(Collection<Long> cuboids) {
+        StringBuilder buf = new StringBuilder();
+        buf.append("[");
+        for (Long l : cuboids) {
+            if (buf.length() > 1)
+                buf.append(",");
+            buf.append(l).append("(").append(Long.toBinaryString(l)).append(")");
+        }
+        buf.append("]");
+        return buf.toString();
+    }
+
+    private CubeDesc getTestKylinCubeWithoutSeller() {
+        return getMetadataManager().getCubeDesc("test_kylin_cube_without_slr_desc");
+    }
+
+    private CubeDesc getTestKylinCubeWithSeller() {
+        return getMetadataManager().getCubeDesc("test_kylin_cube_with_slr_desc");
+    }
+
+    private CubeDesc getTestKylinCubeWithoutSellerLeftJoin() {
+        return getMetadataManager().getCubeDesc("test_kylin_cube_without_slr_left_join_desc");
+    }
+
+    @Test
+    public void testFindSmallerSibling1() {
+        CubeDesc cube = getTestKylinCubeWithoutSeller();
+        CuboidScheduler scheduler = new CuboidScheduler(cube);
+
+        Collection<Long> siblings;
+
+        siblings = scheduler.findSmallerSibling(255);
+        assertEquals("[]", siblings.toString());
+
+        siblings = scheduler.findSmallerSibling(133);
+        assertEquals("[131]", siblings.toString());
+
+        siblings = scheduler.findSmallerSibling(127);
+        assertEquals("[]", siblings.toString());
+
+        siblings = scheduler.findSmallerSibling(134);
+        assertEquals("[131, 133]", sortToString(siblings));
+
+        siblings = scheduler.findSmallerSibling(130);
+        assertEquals("[129]", siblings.toString());
+
+        siblings = scheduler.findSmallerSibling(5);
+        assertEquals("[]", siblings.toString());
+
+        siblings = scheduler.findSmallerSibling(135);
+        assertEquals("[]", siblings.toString());
+    }
+
+    private void testSpanningAndGetParent(CuboidScheduler scheduler, CubeDesc cube, long[] cuboidIds) {
+        for (long cuboidId : cuboidIds) {
+            Collection<Long> spannings = scheduler.getSpanningCuboid(cuboidId);
+            System.out.println("Spanning result for " + cuboidId + "(" + Long.toBinaryString(cuboidId) + "): " + toString(spannings));
+
+            for (long child : spannings) {
+                assertTrue(Cuboid.isValid(cube, child));
+            }
+        }
+    }
+
+    @Test
+    public void testFindSmallerSibling2() {
+        CubeDesc cube = getTestKylinCubeWithSeller();
+        CuboidScheduler scheduler = new CuboidScheduler(cube);
+
+        Collection<Long> siblings;
+
+        siblings = scheduler.findSmallerSibling(511);
+        assertEquals("[]", siblings.toString());
+
+        siblings = scheduler.findSmallerSibling(toLong("110111111"));
+        assertEquals("[383]", siblings.toString());
+
+        siblings = scheduler.findSmallerSibling(toLong("101110111"));
+        assertEquals("[319]", siblings.toString());
+
+        siblings = scheduler.findSmallerSibling(toLong("111111000"));
+        assertEquals("[]", siblings.toString());
+
+        siblings = scheduler.findSmallerSibling(toLong("111111000"));
+        assertEquals("[]", siblings.toString());
+
+        siblings = scheduler.findSmallerSibling(toLong("110000000"));
+        assertEquals("[288, 320]", sortToString(siblings));
+    }
+
+    @Test
+    public void testGetSpanningCuboid2() {
+        CubeDesc cube = getTestKylinCubeWithSeller();
+        CuboidScheduler scheduler = new CuboidScheduler(cube);
+
+        // generate 8d
+        System.out.println("Spanning for 8D Cuboids");
+        testSpanningAndGetParent(scheduler, cube, new long[] { 511 });
+        // generate 7d
+        System.out.println("Spanning for 7D Cuboids");
+        testSpanningAndGetParent(scheduler, cube, new long[] { 504, 447, 503, 383 });
+        // generate 6d
+        System.out.println("Spanning for 6D Cuboids");
+        testSpanningAndGetParent(scheduler, cube, new long[] { 440, 496, 376, 439, 487, 319, 375 });
+        // generate 5d
+        System.out.println("Spanning for 5D Cuboids");
+        testSpanningAndGetParent(scheduler, cube, new long[] { 432, 480, 312, 368, 423, 455, 311, 359 });
+        // generate 4d
+        System.out.println("Spanning for 4D Cuboids");
+        testSpanningAndGetParent(scheduler, cube, new long[] { 416, 448, 304, 352, 391, 295, 327 });
+        // generate 3d
+        System.out.println("Spanning for 3D Cuboids");
+        testSpanningAndGetParent(scheduler, cube, new long[] { 384, 288, 320, 263 });
+        // generate 2d
+        // generate 1d
+        // generate 0d
+    }
+
+    @Test
+    public void testGetSpanningCuboid1() {
+        CubeDesc cube = getTestKylinCubeWithoutSeller();
+        CuboidScheduler scheduler = new CuboidScheduler(cube);
+
+        long quiz = toLong("01100111");
+        testSpanningAndGetParent(scheduler, cube, new long[] { quiz });
+
+        // generate 7d
+        System.out.println("Spanning for 7D Cuboids");
+        testSpanningAndGetParent(scheduler, cube, new long[] { 255 });
+        // generate 6d
+        System.out.println("Spanning for 6D Cuboids");
+        testSpanningAndGetParent(scheduler, cube, new long[] { 135, 251, 253, 254 });
+        // generate 5d
+        System.out.println("Spanning for 5D Cuboids");
+        testSpanningAndGetParent(scheduler, cube, new long[] { 131, 133, 134, 249, 250, 252 });
+        // generate 4d
+        System.out.println("Spanning for 4D Cuboids");
+        testSpanningAndGetParent(scheduler, cube, new long[] { 129, 130, 132, 248 });
+        // generate 3d
+        System.out.println("Spanning for 3D Cuboids");
+        testSpanningAndGetParent(scheduler, cube, new long[] { 184, 240 });
+        // generate 2d
+        System.out.println("Spanning for 2D Cuboids");
+        testSpanningAndGetParent(scheduler, cube, new long[] { 176, 224 });
+        // generate 1d
+        System.out.println("Spanning for 1D Cuboids");
+        testSpanningAndGetParent(scheduler, cube, new long[] { 160, 192 });
+        // generate 0d
+    }
+
+    @Test
+    public void testGetSpanningCuboid() {
+        CubeDesc cube = getTestKylinCubeWithoutSeller();
+        CuboidScheduler scheduler = new CuboidScheduler(cube);
+
+        Collection<Long> spnanningCuboids = scheduler.getSpanningCuboid(248);
+
+        assertEquals("[]", spnanningCuboids.toString());
+    }
+
+    @Test
+    public void testGetCardinality() {
+        CubeDesc cube = getTestKylinCubeWithSeller();
+        CuboidScheduler scheduler = new CuboidScheduler(cube);
+
+        assertEquals(0, scheduler.getCardinality(0));
+        assertEquals(7, scheduler.getCardinality(127));
+        assertEquals(1, scheduler.getCardinality(1));
+        assertEquals(1, scheduler.getCardinality(8));
+        assertEquals(6, scheduler.getCardinality(126));
+    }
+
+    @Test
+    public void testCuboidGeneration1() {
+        CubeDesc cube = getTestKylinCubeWithoutSeller();
+        CuboidCLI.simulateCuboidGeneration(cube);
+    }
+
+    @Test
+    public void testCuboidGeneration2() {
+        CubeDesc cube = getTestKylinCubeWithSeller();
+        CuboidCLI.simulateCuboidGeneration(cube);
+    }
+
+    @Test
+    public void testCuboidGeneration3() {
+        CubeDesc cube = getTestKylinCubeWithoutSellerLeftJoin();
+        CuboidCLI.simulateCuboidGeneration(cube);
+    }
+
+    @Test
+    @Ignore
+    public void testCuboidGeneration4() {
+        CubeDesc cube = getMetadataManager().getCubeDesc("geox_trans_mtrc_sd_cube_desc");
+        CuboidCLI.simulateCuboidGeneration(cube);
+    }
+
+    @Test
+    @Ignore
+    public void testCuboidGeneration5() {
+        CubeDesc cube = getMetadataManager().getCubeDesc("clsfd_ga_dayweek");
+        CuboidCLI.simulateCuboidGeneration(cube);
+    }
+
+    @Test
+    @Ignore
+    public void testCuboidGeneration6() {
+        CubeDesc cube = getMetadataManager().getCubeDesc("clsfd_ga_day");
+        CuboidCLI.simulateCuboidGeneration(cube);
+    }
+
+    @Test
+    @Ignore
+    public void testCuboidGeneration7() {
+        CubeDesc cube = getMetadataManager().getCubeDesc("clsfd_ga_week");
+        CuboidCLI.simulateCuboidGeneration(cube);
+    }
+
+    @Test
+    public void testCuboidCounts1() {
+        CubeDesc cube = getTestKylinCubeWithoutSeller();
+        int[] counts = CuboidCLI.calculateAllLevelCount(cube);
+        printCount(counts);
+        assertArrayEquals(new int[] { 1, 4, 6, 6, 4, 4, 2, 0 }, counts);
+    }
+
+    @Test
+    public void testCuboidCounts2() {
+        CubeDesc cube = getTestKylinCubeWithSeller();
+        CuboidCLI.calculateAllLevelCount(cube);
+        int[] counts = CuboidCLI.calculateAllLevelCount(cube);
+        printCount(counts);
+        assertArrayEquals(new int[] { 1, 4, 7, 8, 7, 4 }, counts);
+    }
+
+    private String sortToString(Collection<Long> longs) {
+        ArrayList<Long> copy = new ArrayList<Long>(longs);
+        Collections.sort(copy);
+        return copy.toString();
+    }
+
+    private MetadataManager getMetadataManager() {
+        return MetadataManager.getInstance(getTestConfig());
+    }
+
+    private void printCount(int[] counts) {
+        int sum = 0;
+        for (int c : counts)
+            sum += c;
+        System.out.println(sum + " = " + Arrays.toString(counts));
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/cube/src/test/java/com/kylinolap/cube/cuboid/CuboidTest.java
----------------------------------------------------------------------
diff --git a/cube/src/test/java/com/kylinolap/cube/cuboid/CuboidTest.java b/cube/src/test/java/com/kylinolap/cube/cuboid/CuboidTest.java
new file mode 100644
index 0000000..6682c9d
--- /dev/null
+++ b/cube/src/test/java/com/kylinolap/cube/cuboid/CuboidTest.java
@@ -0,0 +1,181 @@
+/*
+ * 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.cube.cuboid;
+
+import static org.junit.Assert.*;
+
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+import com.kylinolap.common.util.LocalFileMetadataTestCase;
+import com.kylinolap.metadata.MetadataManager;
+import com.kylinolap.metadata.model.cube.CubeDesc;
+
+/**
+ * @author yangli9
+ */
+public class CuboidTest extends LocalFileMetadataTestCase {
+
+    private long toLong(String bin) {
+        return Long.parseLong(bin, 2);
+    }
+
+    private MetadataManager getMetadataManager() {
+        return MetadataManager.getInstance(getTestConfig());
+    }
+
+    private CubeDesc getTestKylinCubeII() {
+        return getMetadataManager().getCubeDesc("test_kylin_cube_ii");
+    }
+
+    private CubeDesc getTestKylinCubeWithoutSeller() {
+        return getMetadataManager().getCubeDesc("test_kylin_cube_without_slr_desc");
+    }
+
+    private CubeDesc getTestKylinCubeWithSeller() {
+        return getMetadataManager().getCubeDesc("test_kylin_cube_with_slr_desc");
+    }
+
+    @Before
+    public void setUp() throws Exception {
+        this.createTestMetadata();
+        MetadataManager.removeInstance(this.getTestConfig());
+    }
+
+    @After
+    public void after() throws Exception {
+        this.cleanupTestMetadata();
+    }
+
+    @Test
+    public void testIsValid() {
+
+        CubeDesc cube = getTestKylinCubeWithSeller();
+
+        // base
+        assertEquals(false, Cuboid.isValid(cube, 0));
+        assertEquals(true, Cuboid.isValid(cube, toLong("111111111")));
+
+        // mandatory column
+        assertEquals(false, Cuboid.isValid(cube, toLong("011111110")));
+        assertEquals(false, Cuboid.isValid(cube, toLong("100000000")));
+
+        // zero tail
+        assertEquals(true, Cuboid.isValid(cube, toLong("111111000")));
+
+        // aggregation group & zero tail
+        assertEquals(true, Cuboid.isValid(cube, toLong("110000111")));
+        assertEquals(true, Cuboid.isValid(cube, toLong("110111000")));
+        assertEquals(true, Cuboid.isValid(cube, toLong("111110111")));
+        assertEquals(false, Cuboid.isValid(cube, toLong("111110001")));
+        assertEquals(false, Cuboid.isValid(cube, toLong("111110100")));
+        assertEquals(false, Cuboid.isValid(cube, toLong("110000100")));
+    }
+
+    @Test
+    public void testCuboid1() {
+        CubeDesc cube = getTestKylinCubeWithoutSeller();
+        Cuboid cuboid;
+
+        cuboid = Cuboid.findById(cube, 0);
+        assertEquals(toLong("10000001"), cuboid.getId());
+
+        cuboid = Cuboid.findById(cube, 1);
+        assertEquals(toLong("10000001"), cuboid.getId());
+
+        cuboid = Cuboid.findById(cube, toLong("00000010"));
+        assertEquals(toLong("10000010"), cuboid.getId());
+
+        cuboid = Cuboid.findById(cube, toLong("00100000"));
+        assertEquals(toLong("10100000"), cuboid.getId());
+
+        cuboid = Cuboid.findById(cube, toLong("01001000"));
+        assertEquals(toLong("11111000"), cuboid.getId());
+
+        cuboid = Cuboid.findById(cube, toLong("01000111"));
+        assertEquals(toLong("11111111"), cuboid.getId());
+
+        cuboid = Cuboid.findById(cube, toLong("11111111"));
+        assertEquals(toLong("11111111"), cuboid.getId());
+    }
+
+    @Test
+    public void testIsValid2() {
+        CubeDesc cube = getTestKylinCubeWithoutSeller();
+        try {
+            assertEquals(false, Cuboid.isValid(cube, toLong("111111111")));
+            fail();
+        } catch (IllegalArgumentException ex) {
+            // expected
+        }
+
+        // base
+        assertEquals(false, Cuboid.isValid(cube, 0));
+        assertEquals(true, Cuboid.isValid(cube, toLong("11111111")));
+
+        // aggregation group & zero tail
+        assertEquals(true, Cuboid.isValid(cube, toLong("10000111")));
+        assertEquals(false, Cuboid.isValid(cube, toLong("10001111")));
+        assertEquals(false, Cuboid.isValid(cube, toLong("11001111")));
+        assertEquals(true, Cuboid.isValid(cube, toLong("10000001")));
+        assertEquals(true, Cuboid.isValid(cube, toLong("10000101")));
+
+        // hierarchy
+        assertEquals(true, Cuboid.isValid(cube, toLong("10100000")));
+        assertEquals(true, Cuboid.isValid(cube, toLong("10110000")));
+        assertEquals(true, Cuboid.isValid(cube, toLong("10111000")));
+        assertEquals(false, Cuboid.isValid(cube, toLong("10001000")));
+        assertEquals(false, Cuboid.isValid(cube, toLong("10011000")));
+    }
+
+    @Test
+    public void testCuboid2() {
+        CubeDesc cube = getTestKylinCubeWithSeller();
+        Cuboid cuboid;
+
+        cuboid = Cuboid.findById(cube, 0);
+        assertEquals(toLong("100100000"), cuboid.getId());
+
+        cuboid = Cuboid.findById(cube, 1);
+        assertEquals(toLong("100000111"), cuboid.getId());
+
+        cuboid = Cuboid.findById(cube, toLong("010"));
+        assertEquals(toLong("100000111"), cuboid.getId());
+
+        cuboid = Cuboid.findById(cube, toLong("0100000"));
+        assertEquals(toLong("100100000"), cuboid.getId());
+
+        cuboid = Cuboid.findById(cube, toLong("1001000"));
+        assertEquals(toLong("101111000"), cuboid.getId());
+
+        cuboid = Cuboid.findById(cube, toLong("1000111"));
+        assertEquals(toLong("101000111"), cuboid.getId());
+
+        cuboid = Cuboid.findById(cube, toLong("111111111"));
+        assertEquals(toLong("111111111"), cuboid.getId());
+    }
+
+    @Test
+    public void testII() {
+        CubeDesc cube = getTestKylinCubeII();
+        assertEquals(toLong("111111111"), Cuboid.getBaseCuboidId(cube));
+        assertEquals(true, Cuboid.isValid(cube, toLong("111111111")));
+        assertEquals(false, Cuboid.isValid(cube, toLong("111111011")));
+        assertEquals(false, Cuboid.isValid(cube, toLong("101011011")));
+        assertEquals(false, Cuboid.isValid(cube, toLong("000000000")));
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/cube/src/test/java/com/kylinolap/cube/dataGen/DataGenTest.java
----------------------------------------------------------------------
diff --git a/cube/src/test/java/com/kylinolap/cube/dataGen/DataGenTest.java b/cube/src/test/java/com/kylinolap/cube/dataGen/DataGenTest.java
new file mode 100644
index 0000000..8cd0532
--- /dev/null
+++ b/cube/src/test/java/com/kylinolap/cube/dataGen/DataGenTest.java
@@ -0,0 +1,32 @@
+package com.kylinolap.cube.dataGen;
+
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+import com.kylinolap.common.util.LocalFileMetadataTestCase;
+import com.kylinolap.metadata.MetadataManager;
+
+/**
+ * Created by honma on 6/19/14.
+ */
+public class DataGenTest extends LocalFileMetadataTestCase {
+
+    @Before
+    public void before() throws Exception {
+        this.createTestMetadata();
+        MetadataManager.removeInstance(this.getTestConfig());
+    }
+
+    @After
+    public void after() throws Exception {
+        this.cleanupTestMetadata();
+    }
+
+    @Test
+    public void testBasics() throws Exception {
+        FactTableGenerator.generate("test_kylin_cube_with_slr_ready", "10000", "0.6", null, "left");// default
+                                                                                                    // settings
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/cube/src/test/java/com/kylinolap/cube/invertedindex/InvertedIndexLocalTest.java
----------------------------------------------------------------------
diff --git a/cube/src/test/java/com/kylinolap/cube/invertedindex/InvertedIndexLocalTest.java b/cube/src/test/java/com/kylinolap/cube/invertedindex/InvertedIndexLocalTest.java
new file mode 100644
index 0000000..2c42c5b
--- /dev/null
+++ b/cube/src/test/java/com/kylinolap/cube/invertedindex/InvertedIndexLocalTest.java
@@ -0,0 +1,198 @@
+package com.kylinolap.cube.invertedindex;
+
+import static org.junit.Assert.*;
+
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.IOException;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.HashSet;
+import java.util.List;
+
+import org.apache.commons.io.IOUtils;
+import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
+import org.apache.hadoop.hbase.util.Pair;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+import com.google.common.collect.Lists;
+import com.kylinolap.common.util.BytesUtil;
+import com.kylinolap.common.util.LocalFileMetadataTestCase;
+import com.kylinolap.cube.CubeInstance;
+import com.kylinolap.cube.CubeManager;
+import com.kylinolap.dict.Dictionary;
+
+public class InvertedIndexLocalTest extends LocalFileMetadataTestCase {
+
+    CubeInstance cube;
+    TableRecordInfo info;
+
+    @Before
+    public void setUp() throws Exception {
+        this.createTestMetadata();
+        this.cube = CubeManager.getInstance(getTestConfig()).getCube("test_kylin_cube_ii");
+        this.info = new TableRecordInfo(cube.getFirstSegment());
+    }
+
+    @After
+    public void after() throws Exception {
+        this.cleanupTestMetadata();
+    }
+    
+    @Test
+    public void testBitMapContainer() {
+        // create container
+        BitMapContainer container = new BitMapContainer(info, 0);
+        Dictionary<String> dict = info.dict(0);
+        for (int v = dict.getMinId(); v <= dict.getMaxId(); v++) {
+            container.append(v);
+        }
+        container.append(Dictionary.NULL_ID[dict.getSizeOfId()]);
+        container.closeForChange();
+
+        // copy by serialization
+        List<ImmutableBytesWritable> bytes = container.toBytes();
+        BitMapContainer container2 = new BitMapContainer(info, 0);
+        container2.fromBytes(bytes);
+
+        // check the copy
+        int i = 0;
+        for (int v = dict.getMinId(); v <= dict.getMaxId(); v++) {
+            int value = container2.getValueIntAt(i++);
+            assertEquals(v, value);
+        }
+        assertEquals(Dictionary.NULL_ID[dict.getSizeOfId()], container2.getValueIntAt(i++));
+        assertEquals(container, container2);
+    }
+
+    @Test
+    public void testCompressedValueContainer() {
+        // create container
+        CompressedValueContainer container = new CompressedValueContainer(info, 0, 500);
+        Dictionary<String> dict = info.dict(0);
+        
+        byte[] buf = new byte[dict.getSizeOfId()];
+        ImmutableBytesWritable bytes = new ImmutableBytesWritable(buf);
+        
+        for (int v = dict.getMinId(); v <= dict.getMaxId(); v++) {
+            BytesUtil.writeUnsigned(v, buf, 0, dict.getSizeOfId());
+            container.append(bytes);
+        }
+        BytesUtil.writeUnsigned(Dictionary.NULL_ID[dict.getSizeOfId()], buf, 0, dict.getSizeOfId());
+        container.append(bytes);
+        container.closeForChange();
+
+        // copy by serialization
+        ImmutableBytesWritable copy = container.toBytes();
+        CompressedValueContainer container2 = new CompressedValueContainer(info, 0, 500);
+        container2.fromBytes(copy);
+
+        // check the copy
+        int i = 0;
+        for (int v = dict.getMinId(); v <= dict.getMaxId(); v++) {
+            container2.getValueAt(i++, bytes);
+            int value = BytesUtil.readUnsigned(bytes.get(), bytes.getOffset(), bytes.getLength());
+            assertEquals(v, value);
+        }
+        container2.getValueAt(i++, bytes);
+        int value = BytesUtil.readUnsigned(bytes.get(), bytes.getOffset(), bytes.getLength());
+        assertEquals(Dictionary.NULL_ID[dict.getSizeOfId()], value);
+        assertEquals(container, container2);
+    }
+
+    @Test
+    public void testCodec() throws IOException {
+        List<TableRecord> records = loadRecordsSorted();
+        System.out.println(records.size() + " records");
+        List<Slice> slices = buildTimeSlices(records);
+        System.out.println(slices.size() + " slices");
+
+        IIKeyValueCodec codec = new IIKeyValueCodec(info);
+        List<Pair<ImmutableBytesWritable, ImmutableBytesWritable>> kvs = encodeKVs(codec, slices);
+        System.out.println(kvs.size() + " KV pairs");
+
+        List<Slice> slicesCopy = decodeKVs(codec, kvs);
+        assertEquals(slices, slicesCopy);
+
+        List<TableRecord> recordsCopy = iterateRecords(slicesCopy);
+        assertEquals(new HashSet<TableRecord>(records), new HashSet<TableRecord>(recordsCopy));
+        dump(recordsCopy);
+    }
+
+    private List<TableRecord> loadRecordsSorted() throws IOException {
+        File file = new File(LOCALMETA_TEST_DATA, "data/TEST_KYLIN_FACT.csv");
+        FileInputStream in = new FileInputStream(file);
+        List<String> lines = IOUtils.readLines(in, "UTF-8");
+        in.close();
+
+        List<TableRecord> records = Lists.newArrayList();
+        for (String line : lines) {
+            String[] fields = line.split(",");
+            TableRecord rec = new TableRecord(info);
+            for (int col = 0; col < fields.length; col++) {
+                rec.setValueString(col, fields[col]);
+            }
+            records.add(rec);
+        }
+
+        Collections.sort(records, new Comparator<TableRecord>() {
+            @Override
+            public int compare(TableRecord a, TableRecord b) {
+                return (int) (a.getTimestamp() - b.getTimestamp());
+            }
+        });
+
+        return records;
+    }
+
+    private List<Slice> buildTimeSlices(List<TableRecord> records) throws IOException {
+        ShardingSliceBuilder builder = new ShardingSliceBuilder(info);
+        List<Slice> slices = Lists.newArrayList();
+        for (TableRecord rec : records) {
+            Slice slice = builder.append(rec);
+            if (slice != null)
+                slices.add(slice);
+        }
+        List<Slice> finals = builder.close();
+        slices.addAll(finals);
+        
+        Collections.sort(slices);
+        return slices;
+    }
+
+    private List<Pair<ImmutableBytesWritable, ImmutableBytesWritable>> encodeKVs(IIKeyValueCodec codec, List<Slice> slices) {
+
+        List<Pair<ImmutableBytesWritable, ImmutableBytesWritable>> kvs = Lists.newArrayList();
+        for (Slice slice : slices) {
+            kvs.addAll(codec.encodeKeyValue(slice));
+        }
+        return kvs;
+    }
+
+    private List<Slice> decodeKVs(IIKeyValueCodec codec, List<Pair<ImmutableBytesWritable, ImmutableBytesWritable>> kvs) {
+        List<Slice> slices = Lists.newArrayList();
+        for (Slice slice : codec.decodeKeyValue(kvs)) {
+            slices.add(slice);
+        }
+        return slices;
+    }
+
+    private List<TableRecord> iterateRecords(List<Slice> slices) {
+        List<TableRecord> records = Lists.newArrayList();
+        for (Slice slice : slices) {
+            for (TableRecordBytes rec : slice) {
+                records.add((TableRecord) rec.clone());
+            }
+        }
+        return records;
+    }
+
+    private void dump(Iterable<TableRecord> records) {
+        for (TableRecord rec : records) {
+            System.out.println(rec.toString());
+        }
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/cube/src/test/java/com/kylinolap/cube/kv/RowKeyDecoderTest.java
----------------------------------------------------------------------
diff --git a/cube/src/test/java/com/kylinolap/cube/kv/RowKeyDecoderTest.java b/cube/src/test/java/com/kylinolap/cube/kv/RowKeyDecoderTest.java
new file mode 100644
index 0000000..b78b2fb
--- /dev/null
+++ b/cube/src/test/java/com/kylinolap/cube/kv/RowKeyDecoderTest.java
@@ -0,0 +1,115 @@
+/*
+ * 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.cube.kv;
+
+import static org.junit.Assert.*;
+
+import java.io.IOException;
+import java.util.List;
+
+import org.apache.hadoop.hbase.util.Bytes;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+import com.kylinolap.common.util.LocalFileMetadataTestCase;
+import com.kylinolap.cube.CubeInstance;
+import com.kylinolap.cube.CubeManager;
+import com.kylinolap.cube.cuboid.Cuboid;
+import com.kylinolap.metadata.MetadataManager;
+import com.kylinolap.metadata.model.cube.CubeDesc;
+
+/**
+ * @author George Song (ysong1)
+ * 
+ */
+public class RowKeyDecoderTest extends LocalFileMetadataTestCase {
+
+    @Before
+    public void setUp() throws Exception {
+        this.createTestMetadata();
+        MetadataManager.removeInstance(this.getTestConfig());
+    }
+
+    @After
+    public void after() throws Exception {
+        this.cleanupTestMetadata();
+    }
+
+    @Test
+    public void testDecodeWithoutSlr() throws Exception {
+        CubeInstance cube = CubeManager.getInstance(this.getTestConfig()).getCube("TEST_KYLIN_CUBE_WITHOUT_SLR_READY");
+
+        RowKeyDecoder rowKeyDecoder = new RowKeyDecoder(cube.getFirstSegment());
+
+        byte[] key = { 0, 0, 0, 0, 0, 0, 0, -1, 11, 55, -13, 13, 22, 34, 121, 70, 80, 45, 71, 84, 67, 9, 9, 9, 9, 9, 9, 0, 10, 5 };
+
+        rowKeyDecoder.decode(key);
+        List<String> names = rowKeyDecoder.getNames(null);
+        List<String> values = rowKeyDecoder.getValues();
+
+        assertEquals("[CAL_DT, LEAF_CATEG_ID, META_CATEG_NAME, CATEG_LVL2_NAME, CATEG_LVL3_NAME, LSTG_FORMAT_NAME, LSTG_SITE_ID, SLR_SEGMENT_CD]", names.toString());
+        assertEquals("[2012-12-15, 11848, Health & Beauty, Fragrances, Women, FP-GTC, 0, 15]", values.toString());
+
+    }
+
+    @Test
+    public void testDecodeWithSlr() throws Exception {
+        CubeInstance cube = CubeManager.getInstance(this.getTestConfig()).getCube("TEST_KYLIN_CUBE_WITH_SLR_READY");
+
+        RowKeyDecoder rowKeyDecoder = new RowKeyDecoder(cube.getFirstSegment());
+
+        byte[] key = { 0, 0, 0, 0, 0, 0, 1, -1, 49, 48, 48, 48, 48, 48, 48, 48, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 11, 54, -105, 55, 13, 71, 114, 65, 66, 73, 78, 9, 9, 9, 9, 9, 9, 9, 9, 0, 10, 0 };
+
+        rowKeyDecoder.decode(key);
+        List<String> names = rowKeyDecoder.getNames(null);
+        List<String> values = rowKeyDecoder.getValues();
+
+        assertEquals("[SELLER_ID, CAL_DT, LEAF_CATEG_ID, META_CATEG_NAME, CATEG_LVL2_NAME, CATEG_LVL3_NAME, LSTG_FORMAT_NAME, LSTG_SITE_ID, SLR_SEGMENT_CD]", names.toString());
+        assertEquals("[10000000, 2012-01-02, 20213, Collectibles, Postcards, US StateCities & Towns, ABIN, 0, -99]", values.toString());
+
+    }
+
+    @Test
+    public void testEncodeAndDecodeWithUtf8() throws IOException {
+        CubeInstance cube = CubeManager.getInstance(this.getTestConfig()).getCube("TEST_KYLIN_CUBE_WITHOUT_SLR_READY");
+        CubeDesc cubeDesc = cube.getDescriptor();
+
+        byte[][] data = new byte[8][];
+        data[0] = Bytes.toBytes("2012-12-15");
+        data[1] = Bytes.toBytes("11848");
+        data[2] = Bytes.toBytes("Health & Beauty");
+        data[3] = Bytes.toBytes("Fragrances");
+        data[4] = Bytes.toBytes("Women");
+        data[5] = Bytes.toBytes("刊登格式测试");// UTF-8
+        data[6] = Bytes.toBytes("0");
+        data[7] = Bytes.toBytes("15");
+
+        long baseCuboidId = Cuboid.getBaseCuboidId(cubeDesc);
+        Cuboid baseCuboid = Cuboid.findById(cubeDesc, baseCuboidId);
+        AbstractRowKeyEncoder rowKeyEncoder = AbstractRowKeyEncoder.createInstance(cube.getFirstSegment(), baseCuboid);
+
+        byte[] encodedKey = rowKeyEncoder.encode(data);
+        assertEquals(30, encodedKey.length);
+
+        RowKeyDecoder rowKeyDecoder = new RowKeyDecoder(cube.getFirstSegment());
+        rowKeyDecoder.decode(encodedKey);
+        List<String> names = rowKeyDecoder.getNames(null);
+        List<String> values = rowKeyDecoder.getValues();
+        assertEquals("[CAL_DT, LEAF_CATEG_ID, META_CATEG_NAME, CATEG_LVL2_NAME, CATEG_LVL3_NAME, LSTG_FORMAT_NAME, LSTG_SITE_ID, SLR_SEGMENT_CD]", names.toString());
+        assertEquals("[2012-12-15, 11848, Health & Beauty, Fragrances, Women, 刊登格式, 0, 15]", values.toString());
+    }
+}


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

Posted by lu...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/query/src/test/resources/query/sql_tableau/query22.sql.disabled
----------------------------------------------------------------------
diff --git a/query/src/test/resources/query/sql_tableau/query22.sql.disabled b/query/src/test/resources/query/sql_tableau/query22.sql.disabled
new file mode 100644
index 0000000..df40a9c
--- /dev/null
+++ b/query/src/test/resources/query/sql_tableau/query22.sql.disabled
@@ -0,0 +1,10 @@
+select fact.cal_dt, sum(fact.price) from test_kylin_fact fact 
+left join test_cal_dt cal 
+	on fact.cal_dt=cal.cal_dt
+inner join
+(
+	select test_kylin_fact.cal_dt, count(1) from test_kylin_fact left join test_cal_dt 
+	on test_kylin_fact.cal_dt=test_cal_dt.cal_dt group by test_kylin_fact.cal_dt order by 2 desc limit 2
+) cal_2 
+	on fact.cal_dt = cal_2.cal_dt 
+group by fact.cal_dt
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/query/src/test/resources/query/sql_tableau/query23.sql
----------------------------------------------------------------------
diff --git a/query/src/test/resources/query/sql_tableau/query23.sql b/query/src/test/resources/query/sql_tableau/query23.sql
new file mode 100644
index 0000000..50df99e
--- /dev/null
+++ b/query/src/test/resources/query/sql_tableau/query23.sql
@@ -0,0 +1,5 @@
+
+select fact.cal_dt, sum(fact.price) from test_kylin_fact fact 
+left join test_cal_dt cal on fact.cal_dt=cal.cal_dt
+where cal.cal_dt  = date '2012-05-17' or cal.cal_dt  = date '2013-05-17'
+group by fact.cal_dt
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/query/src/test/resources/query/sql_tableau/query24.sql
----------------------------------------------------------------------
diff --git a/query/src/test/resources/query/sql_tableau/query24.sql b/query/src/test/resources/query/sql_tableau/query24.sql
new file mode 100644
index 0000000..6a975b1
--- /dev/null
+++ b/query/src/test/resources/query/sql_tableau/query24.sql
@@ -0,0 +1,6 @@
+select test_kylin_fact.cal_dt, count(1) as cnt_1
+from test_kylin_fact 
+left join test_cal_dt on test_kylin_fact.cal_dt=test_cal_dt.cal_dt 
+group by test_kylin_fact.cal_dt 
+order by 2 desc 
+limit 3
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/query/src/test/resources/query/sql_tableau/query25.sql
----------------------------------------------------------------------
diff --git a/query/src/test/resources/query/sql_tableau/query25.sql b/query/src/test/resources/query/sql_tableau/query25.sql
new file mode 100644
index 0000000..4b6fb88
--- /dev/null
+++ b/query/src/test/resources/query/sql_tableau/query25.sql
@@ -0,0 +1,10 @@
+SELECT "TEST_KYLIN_FACT"."CAL_DT", SUM("TEST_KYLIN_FACT"."PRICE") AS "sum_PRICE_ok" 
+	FROM "TEST_KYLIN_FACT"
+    INNER JOIN "TEST_CAL_DT" ON ("TEST_KYLIN_FACT"."CAL_DT" = "TEST_CAL_DT"."CAL_DT")
+    INNER JOIN (
+     SELECT COUNT(1) AS "XTableau_join_flag",
+      SUM("TEST_KYLIN_FACT"."PRICE") AS "X__alias__A",
+       "TEST_KYLIN_FACT"."CAL_DT" AS "none_CAL_DT_ok"   FROM "TEST_KYLIN_FACT"
+         INNER JOIN "TEST_CAL_DT" ON ("TEST_KYLIN_FACT"."CAL_DT" = "TEST_CAL_DT"."CAL_DT")
+     GROUP BY "TEST_KYLIN_FACT"."CAL_DT"   ORDER BY 2 DESC   LIMIT 10  ) "t0" ON ("TEST_KYLIN_FACT"."CAL_DT" = "t0"."none_CAL_DT_ok") 
+    GROUP BY "TEST_KYLIN_FACT"."CAL_DT"
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/query/src/test/resources/query/sql_tableau/query27.sql
----------------------------------------------------------------------
diff --git a/query/src/test/resources/query/sql_tableau/query27.sql b/query/src/test/resources/query/sql_tableau/query27.sql
new file mode 100644
index 0000000..c118f34
--- /dev/null
+++ b/query/src/test/resources/query/sql_tableau/query27.sql
@@ -0,0 +1,6 @@
+SELECT "TEST_KYLIN_FACT"."CAL_DT", SUM("TEST_KYLIN_FACT"."PRICE") AS "sum_PRICE_ok" FROM "EDW"."TEST_KYLIN_FACT" "TEST_KYLIN_FACT"
+  INNER JOIN (
+             SELECT COUNT(1) AS "XTableau_join_flag",     SUM("TEST_KYLIN_FACT"."PRICE") AS "X__alias__A",     "TEST_KYLIN_FACT"."CAL_DT" AS "none_CAL_DT_ok"   FROM "EDW"."TEST_KYLIN_FACT" "TEST_KYLIN_FACT"
+             GROUP BY "TEST_KYLIN_FACT"."CAL_DT"   ORDER BY 2 DESC   LIMIT 7  )
+
+    "t0" ON ("TEST_KYLIN_FACT"."CAL_DT" = "t0"."none_CAL_DT_ok") GROUP BY "TEST_KYLIN_FACT"."CAL_DT"

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/query/src/test/resources/query/sql_tableau/query67.sql
----------------------------------------------------------------------
diff --git a/query/src/test/resources/query/sql_tableau/query67.sql b/query/src/test/resources/query/sql_tableau/query67.sql
new file mode 100644
index 0000000..4cfa60d
--- /dev/null
+++ b/query/src/test/resources/query/sql_tableau/query67.sql
@@ -0,0 +1,5 @@
+SELECT (CASE "TEST_KYLIN_FACT"."LSTG_FORMAT_NAME" WHEN 'Auction' THEN '111' ELSE '222' END) AS "LSTG_FORMAT_NAME__group_",
+  SUM("TEST_KYLIN_FACT"."PRICE") AS "sum_PRICE_ok"
+FROM "TEST_KYLIN_FACT" "TEST_KYLIN_FACT"
+--group by (CASE "TEST_KYLIN_FACT"."LSTG_FORMAT_NAME" WHEN 'Auction' THEN '111' ELSE '222' END)  ORDER BY 1 ASC
+GROUP BY "TEST_KYLIN_FACT"."LSTG_FORMAT_NAME"  ORDER BY 1 ASC
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/query/src/test/resources/query/sql_verifyCount/query04.sql
----------------------------------------------------------------------
diff --git a/query/src/test/resources/query/sql_verifyCount/query04.sql b/query/src/test/resources/query/sql_verifyCount/query04.sql
new file mode 100644
index 0000000..08a0354
--- /dev/null
+++ b/query/src/test/resources/query/sql_verifyCount/query04.sql
@@ -0,0 +1 @@
+select * from test_kylin_fact limit 100

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/query/src/test/resources/query/sql_verifyCount/query04.sql.expected
----------------------------------------------------------------------
diff --git a/query/src/test/resources/query/sql_verifyCount/query04.sql.expected b/query/src/test/resources/query/sql_verifyCount/query04.sql.expected
new file mode 100644
index 0000000..105d7d9
--- /dev/null
+++ b/query/src/test/resources/query/sql_verifyCount/query04.sql.expected
@@ -0,0 +1 @@
+100
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/query/src/test/resources/query/sql_verifyCount/query05.sql
----------------------------------------------------------------------
diff --git a/query/src/test/resources/query/sql_verifyCount/query05.sql b/query/src/test/resources/query/sql_verifyCount/query05.sql
new file mode 100644
index 0000000..37fe636
--- /dev/null
+++ b/query/src/test/resources/query/sql_verifyCount/query05.sql
@@ -0,0 +1 @@
+select price from test_kylin_fact limit 100

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/query/src/test/resources/query/sql_verifyCount/query05.sql.expected
----------------------------------------------------------------------
diff --git a/query/src/test/resources/query/sql_verifyCount/query05.sql.expected b/query/src/test/resources/query/sql_verifyCount/query05.sql.expected
new file mode 100644
index 0000000..105d7d9
--- /dev/null
+++ b/query/src/test/resources/query/sql_verifyCount/query05.sql.expected
@@ -0,0 +1 @@
+100
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/query/src/test/resources/query/sql_verifyCount/query06.sql
----------------------------------------------------------------------
diff --git a/query/src/test/resources/query/sql_verifyCount/query06.sql b/query/src/test/resources/query/sql_verifyCount/query06.sql
new file mode 100644
index 0000000..1fc00c4
--- /dev/null
+++ b/query/src/test/resources/query/sql_verifyCount/query06.sql
@@ -0,0 +1 @@
+select lstg_format_name from test_kylin_fact limit 100

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/query/src/test/resources/query/sql_verifyCount/query06.sql.expected
----------------------------------------------------------------------
diff --git a/query/src/test/resources/query/sql_verifyCount/query06.sql.expected b/query/src/test/resources/query/sql_verifyCount/query06.sql.expected
new file mode 100644
index 0000000..105d7d9
--- /dev/null
+++ b/query/src/test/resources/query/sql_verifyCount/query06.sql.expected
@@ -0,0 +1 @@
+100
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/query/src/test/resources/query/sql_verifyCount/query07.sql
----------------------------------------------------------------------
diff --git a/query/src/test/resources/query/sql_verifyCount/query07.sql b/query/src/test/resources/query/sql_verifyCount/query07.sql
new file mode 100644
index 0000000..7309d3d
--- /dev/null
+++ b/query/src/test/resources/query/sql_verifyCount/query07.sql
@@ -0,0 +1 @@
+select price,lstg_format_name from test_kylin_fact limit 100

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/query/src/test/resources/query/sql_verifyCount/query07.sql.expected
----------------------------------------------------------------------
diff --git a/query/src/test/resources/query/sql_verifyCount/query07.sql.expected b/query/src/test/resources/query/sql_verifyCount/query07.sql.expected
new file mode 100644
index 0000000..105d7d9
--- /dev/null
+++ b/query/src/test/resources/query/sql_verifyCount/query07.sql.expected
@@ -0,0 +1 @@
+100
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/query/src/test/resources/query/tableau_probing/query01.sql
----------------------------------------------------------------------
diff --git a/query/src/test/resources/query/tableau_probing/query01.sql b/query/src/test/resources/query/tableau_probing/query01.sql
new file mode 100644
index 0000000..f54fbd7
--- /dev/null
+++ b/query/src/test/resources/query/tableau_probing/query01.sql
@@ -0,0 +1 @@
+SELECT 1 

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/query/src/test/resources/query/tableau_probing/query02.sql
----------------------------------------------------------------------
diff --git a/query/src/test/resources/query/tableau_probing/query02.sql b/query/src/test/resources/query/tableau_probing/query02.sql
new file mode 100644
index 0000000..a9c6c3c
--- /dev/null
+++ b/query/src/test/resources/query/tableau_probing/query02.sql
@@ -0,0 +1 @@
+SELECT 1 AS "COL" 

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/query/src/test/resources/query/tableau_probing/query03.sql
----------------------------------------------------------------------
diff --git a/query/src/test/resources/query/tableau_probing/query03.sql b/query/src/test/resources/query/tableau_probing/query03.sql
new file mode 100644
index 0000000..1fc4bed
--- /dev/null
+++ b/query/src/test/resources/query/tableau_probing/query03.sql
@@ -0,0 +1 @@
+SELECT "COL" FROM (SELECT 1 AS "COL") AS "SUBQUERY" 

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/query/src/test/resources/query/tableau_probing/query04.sql
----------------------------------------------------------------------
diff --git a/query/src/test/resources/query/tableau_probing/query04.sql b/query/src/test/resources/query/tableau_probing/query04.sql
new file mode 100644
index 0000000..e732d24
--- /dev/null
+++ b/query/src/test/resources/query/tableau_probing/query04.sql
@@ -0,0 +1 @@
+SELECT TOP 1 "COL" FROM (SELECT 1 AS "COL") AS "CHECKTOP" 

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/query/src/test/resources/query/tableau_probing/query05.sql
----------------------------------------------------------------------
diff --git a/query/src/test/resources/query/tableau_probing/query05.sql b/query/src/test/resources/query/tableau_probing/query05.sql
new file mode 100644
index 0000000..fc92170
--- /dev/null
+++ b/query/src/test/resources/query/tableau_probing/query05.sql
@@ -0,0 +1 @@
+SELECT "COL" FROM (SELECT 1 AS "COL") AS "CHECKTOP" LIMIT 1 

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/query/src/test/resources/query/tableau_probing/query06.sql
----------------------------------------------------------------------
diff --git a/query/src/test/resources/query/tableau_probing/query06.sql b/query/src/test/resources/query/tableau_probing/query06.sql
new file mode 100644
index 0000000..41d714e
--- /dev/null
+++ b/query/src/test/resources/query/tableau_probing/query06.sql
@@ -0,0 +1 @@
+SELECT "SUBCOL" AS "COL" FROM ( SELECT 1 AS "SUBCOL" ) "SUBQUERY" GROUP BY 1 

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/query/src/test/resources/query/tableau_probing/query07.sql
----------------------------------------------------------------------
diff --git a/query/src/test/resources/query/tableau_probing/query07.sql b/query/src/test/resources/query/tableau_probing/query07.sql
new file mode 100644
index 0000000..3c0856d
--- /dev/null
+++ b/query/src/test/resources/query/tableau_probing/query07.sql
@@ -0,0 +1 @@
+SELECT "SUBCOL" AS "COL" FROM ( SELECT 1 AS "SUBCOL" ) "SUBQUERY" GROUP BY 2 

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/server/.settings/org.eclipse.core.resources.prefs
----------------------------------------------------------------------
diff --git a/server/.settings/org.eclipse.core.resources.prefs b/server/.settings/org.eclipse.core.resources.prefs
new file mode 100644
index 0000000..365bbd6
--- /dev/null
+++ b/server/.settings/org.eclipse.core.resources.prefs
@@ -0,0 +1,5 @@
+eclipse.preferences.version=1
+encoding//src/main/java=UTF-8
+encoding//src/main/resources=UTF-8
+encoding//src/test/java=UTF-8
+encoding/<project>=UTF-8

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/server/.settings/org.eclipse.jdt.core.prefs
----------------------------------------------------------------------
diff --git a/server/.settings/org.eclipse.jdt.core.prefs b/server/.settings/org.eclipse.jdt.core.prefs
new file mode 100644
index 0000000..a903301
--- /dev/null
+++ b/server/.settings/org.eclipse.jdt.core.prefs
@@ -0,0 +1,379 @@
+eclipse.preferences.version=1
+org.eclipse.jdt.core.compiler.annotation.inheritNullAnnotations=disabled
+org.eclipse.jdt.core.compiler.annotation.missingNonNullByDefaultAnnotation=ignore
+org.eclipse.jdt.core.compiler.annotation.nonnull=org.eclipse.jdt.annotation.NonNull
+org.eclipse.jdt.core.compiler.annotation.nonnullbydefault=org.eclipse.jdt.annotation.NonNullByDefault
+org.eclipse.jdt.core.compiler.annotation.nullable=org.eclipse.jdt.annotation.Nullable
+org.eclipse.jdt.core.compiler.annotation.nullanalysis=disabled
+org.eclipse.jdt.core.compiler.codegen.inlineJsrBytecode=enabled
+org.eclipse.jdt.core.compiler.codegen.targetPlatform=1.7
+org.eclipse.jdt.core.compiler.codegen.unusedLocal=preserve
+org.eclipse.jdt.core.compiler.compliance=1.7
+org.eclipse.jdt.core.compiler.debug.lineNumber=generate
+org.eclipse.jdt.core.compiler.debug.localVariable=generate
+org.eclipse.jdt.core.compiler.debug.sourceFile=generate
+org.eclipse.jdt.core.compiler.problem.annotationSuperInterface=warning
+org.eclipse.jdt.core.compiler.problem.assertIdentifier=error
+org.eclipse.jdt.core.compiler.problem.autoboxing=ignore
+org.eclipse.jdt.core.compiler.problem.comparingIdentical=warning
+org.eclipse.jdt.core.compiler.problem.deadCode=warning
+org.eclipse.jdt.core.compiler.problem.deprecation=warning
+org.eclipse.jdt.core.compiler.problem.deprecationInDeprecatedCode=disabled
+org.eclipse.jdt.core.compiler.problem.deprecationWhenOverridingDeprecatedMethod=disabled
+org.eclipse.jdt.core.compiler.problem.discouragedReference=warning
+org.eclipse.jdt.core.compiler.problem.emptyStatement=ignore
+org.eclipse.jdt.core.compiler.problem.enumIdentifier=error
+org.eclipse.jdt.core.compiler.problem.explicitlyClosedAutoCloseable=ignore
+org.eclipse.jdt.core.compiler.problem.fallthroughCase=ignore
+org.eclipse.jdt.core.compiler.problem.fatalOptionalError=disabled
+org.eclipse.jdt.core.compiler.problem.fieldHiding=ignore
+org.eclipse.jdt.core.compiler.problem.finalParameterBound=warning
+org.eclipse.jdt.core.compiler.problem.finallyBlockNotCompletingNormally=warning
+org.eclipse.jdt.core.compiler.problem.forbiddenReference=warning
+org.eclipse.jdt.core.compiler.problem.hiddenCatchBlock=warning
+org.eclipse.jdt.core.compiler.problem.includeNullInfoFromAsserts=disabled
+org.eclipse.jdt.core.compiler.problem.incompatibleNonInheritedInterfaceMethod=warning
+org.eclipse.jdt.core.compiler.problem.incompleteEnumSwitch=warning
+org.eclipse.jdt.core.compiler.problem.indirectStaticAccess=ignore
+org.eclipse.jdt.core.compiler.problem.localVariableHiding=ignore
+org.eclipse.jdt.core.compiler.problem.methodWithConstructorName=warning
+org.eclipse.jdt.core.compiler.problem.missingDefaultCase=ignore
+org.eclipse.jdt.core.compiler.problem.missingDeprecatedAnnotation=ignore
+org.eclipse.jdt.core.compiler.problem.missingEnumCaseDespiteDefault=disabled
+org.eclipse.jdt.core.compiler.problem.missingHashCodeMethod=ignore
+org.eclipse.jdt.core.compiler.problem.missingOverrideAnnotation=ignore
+org.eclipse.jdt.core.compiler.problem.missingOverrideAnnotationForInterfaceMethodImplementation=enabled
+org.eclipse.jdt.core.compiler.problem.missingSerialVersion=warning
+org.eclipse.jdt.core.compiler.problem.missingSynchronizedOnInheritedMethod=ignore
+org.eclipse.jdt.core.compiler.problem.noEffectAssignment=warning
+org.eclipse.jdt.core.compiler.problem.noImplicitStringConversion=warning
+org.eclipse.jdt.core.compiler.problem.nonExternalizedStringLiteral=ignore
+org.eclipse.jdt.core.compiler.problem.nonnullParameterAnnotationDropped=warning
+org.eclipse.jdt.core.compiler.problem.nullAnnotationInferenceConflict=error
+org.eclipse.jdt.core.compiler.problem.nullReference=warning
+org.eclipse.jdt.core.compiler.problem.nullSpecViolation=error
+org.eclipse.jdt.core.compiler.problem.nullUncheckedConversion=warning
+org.eclipse.jdt.core.compiler.problem.overridingPackageDefaultMethod=warning
+org.eclipse.jdt.core.compiler.problem.parameterAssignment=ignore
+org.eclipse.jdt.core.compiler.problem.possibleAccidentalBooleanAssignment=ignore
+org.eclipse.jdt.core.compiler.problem.potentialNullReference=ignore
+org.eclipse.jdt.core.compiler.problem.potentiallyUnclosedCloseable=ignore
+org.eclipse.jdt.core.compiler.problem.rawTypeReference=warning
+org.eclipse.jdt.core.compiler.problem.redundantNullAnnotation=warning
+org.eclipse.jdt.core.compiler.problem.redundantNullCheck=ignore
+org.eclipse.jdt.core.compiler.problem.redundantSpecificationOfTypeArguments=ignore
+org.eclipse.jdt.core.compiler.problem.redundantSuperinterface=ignore
+org.eclipse.jdt.core.compiler.problem.reportMethodCanBePotentiallyStatic=ignore
+org.eclipse.jdt.core.compiler.problem.reportMethodCanBeStatic=ignore
+org.eclipse.jdt.core.compiler.problem.specialParameterHidingField=disabled
+org.eclipse.jdt.core.compiler.problem.staticAccessReceiver=warning
+org.eclipse.jdt.core.compiler.problem.suppressOptionalErrors=disabled
+org.eclipse.jdt.core.compiler.problem.suppressWarnings=enabled
+org.eclipse.jdt.core.compiler.problem.syntacticNullAnalysisForFields=disabled
+org.eclipse.jdt.core.compiler.problem.syntheticAccessEmulation=ignore
+org.eclipse.jdt.core.compiler.problem.typeParameterHiding=warning
+org.eclipse.jdt.core.compiler.problem.unavoidableGenericTypeProblems=enabled
+org.eclipse.jdt.core.compiler.problem.uncheckedTypeOperation=warning
+org.eclipse.jdt.core.compiler.problem.unclosedCloseable=warning
+org.eclipse.jdt.core.compiler.problem.undocumentedEmptyBlock=ignore
+org.eclipse.jdt.core.compiler.problem.unhandledWarningToken=warning
+org.eclipse.jdt.core.compiler.problem.unnecessaryElse=ignore
+org.eclipse.jdt.core.compiler.problem.unnecessaryTypeCheck=ignore
+org.eclipse.jdt.core.compiler.problem.unqualifiedFieldAccess=ignore
+org.eclipse.jdt.core.compiler.problem.unusedDeclaredThrownException=ignore
+org.eclipse.jdt.core.compiler.problem.unusedDeclaredThrownExceptionExemptExceptionAndThrowable=enabled
+org.eclipse.jdt.core.compiler.problem.unusedDeclaredThrownExceptionIncludeDocCommentReference=enabled
+org.eclipse.jdt.core.compiler.problem.unusedDeclaredThrownExceptionWhenOverriding=disabled
+org.eclipse.jdt.core.compiler.problem.unusedImport=warning
+org.eclipse.jdt.core.compiler.problem.unusedLabel=warning
+org.eclipse.jdt.core.compiler.problem.unusedLocal=warning
+org.eclipse.jdt.core.compiler.problem.unusedObjectAllocation=ignore
+org.eclipse.jdt.core.compiler.problem.unusedParameter=ignore
+org.eclipse.jdt.core.compiler.problem.unusedParameterIncludeDocCommentReference=enabled
+org.eclipse.jdt.core.compiler.problem.unusedParameterWhenImplementingAbstract=disabled
+org.eclipse.jdt.core.compiler.problem.unusedParameterWhenOverridingConcrete=disabled
+org.eclipse.jdt.core.compiler.problem.unusedPrivateMember=warning
+org.eclipse.jdt.core.compiler.problem.unusedTypeParameter=ignore
+org.eclipse.jdt.core.compiler.problem.unusedWarningToken=warning
+org.eclipse.jdt.core.compiler.problem.varargsArgumentNeedCast=warning
+org.eclipse.jdt.core.compiler.source=1.7
+org.eclipse.jdt.core.formatter.align_type_members_on_columns=false
+org.eclipse.jdt.core.formatter.alignment_for_arguments_in_allocation_expression=16
+org.eclipse.jdt.core.formatter.alignment_for_arguments_in_annotation=0
+org.eclipse.jdt.core.formatter.alignment_for_arguments_in_enum_constant=16
+org.eclipse.jdt.core.formatter.alignment_for_arguments_in_explicit_constructor_call=16
+org.eclipse.jdt.core.formatter.alignment_for_arguments_in_method_invocation=16
+org.eclipse.jdt.core.formatter.alignment_for_arguments_in_qualified_allocation_expression=16
+org.eclipse.jdt.core.formatter.alignment_for_assignment=0
+org.eclipse.jdt.core.formatter.alignment_for_binary_expression=16
+org.eclipse.jdt.core.formatter.alignment_for_compact_if=16
+org.eclipse.jdt.core.formatter.alignment_for_conditional_expression=80
+org.eclipse.jdt.core.formatter.alignment_for_enum_constants=0
+org.eclipse.jdt.core.formatter.alignment_for_expressions_in_array_initializer=16
+org.eclipse.jdt.core.formatter.alignment_for_method_declaration=0
+org.eclipse.jdt.core.formatter.alignment_for_multiple_fields=16
+org.eclipse.jdt.core.formatter.alignment_for_parameters_in_constructor_declaration=16
+org.eclipse.jdt.core.formatter.alignment_for_parameters_in_method_declaration=16
+org.eclipse.jdt.core.formatter.alignment_for_resources_in_try=80
+org.eclipse.jdt.core.formatter.alignment_for_selector_in_method_invocation=16
+org.eclipse.jdt.core.formatter.alignment_for_superclass_in_type_declaration=16
+org.eclipse.jdt.core.formatter.alignment_for_superinterfaces_in_enum_declaration=16
+org.eclipse.jdt.core.formatter.alignment_for_superinterfaces_in_type_declaration=16
+org.eclipse.jdt.core.formatter.alignment_for_throws_clause_in_constructor_declaration=16
+org.eclipse.jdt.core.formatter.alignment_for_throws_clause_in_method_declaration=16
+org.eclipse.jdt.core.formatter.alignment_for_union_type_in_multicatch=16
+org.eclipse.jdt.core.formatter.blank_lines_after_imports=1
+org.eclipse.jdt.core.formatter.blank_lines_after_package=1
+org.eclipse.jdt.core.formatter.blank_lines_before_field=0
+org.eclipse.jdt.core.formatter.blank_lines_before_first_class_body_declaration=0
+org.eclipse.jdt.core.formatter.blank_lines_before_imports=1
+org.eclipse.jdt.core.formatter.blank_lines_before_member_type=1
+org.eclipse.jdt.core.formatter.blank_lines_before_method=1
+org.eclipse.jdt.core.formatter.blank_lines_before_new_chunk=1
+org.eclipse.jdt.core.formatter.blank_lines_before_package=0
+org.eclipse.jdt.core.formatter.blank_lines_between_import_groups=1
+org.eclipse.jdt.core.formatter.blank_lines_between_type_declarations=1
+org.eclipse.jdt.core.formatter.brace_position_for_annotation_type_declaration=end_of_line
+org.eclipse.jdt.core.formatter.brace_position_for_anonymous_type_declaration=end_of_line
+org.eclipse.jdt.core.formatter.brace_position_for_array_initializer=end_of_line
+org.eclipse.jdt.core.formatter.brace_position_for_block=end_of_line
+org.eclipse.jdt.core.formatter.brace_position_for_block_in_case=end_of_line
+org.eclipse.jdt.core.formatter.brace_position_for_constructor_declaration=end_of_line
+org.eclipse.jdt.core.formatter.brace_position_for_enum_constant=end_of_line
+org.eclipse.jdt.core.formatter.brace_position_for_enum_declaration=end_of_line
+org.eclipse.jdt.core.formatter.brace_position_for_method_declaration=end_of_line
+org.eclipse.jdt.core.formatter.brace_position_for_switch=end_of_line
+org.eclipse.jdt.core.formatter.brace_position_for_type_declaration=end_of_line
+org.eclipse.jdt.core.formatter.comment.clear_blank_lines_in_block_comment=false
+org.eclipse.jdt.core.formatter.comment.clear_blank_lines_in_javadoc_comment=false
+org.eclipse.jdt.core.formatter.comment.format_block_comments=false
+org.eclipse.jdt.core.formatter.comment.format_header=false
+org.eclipse.jdt.core.formatter.comment.format_html=true
+org.eclipse.jdt.core.formatter.comment.format_javadoc_comments=false
+org.eclipse.jdt.core.formatter.comment.format_line_comments=false
+org.eclipse.jdt.core.formatter.comment.format_source_code=true
+org.eclipse.jdt.core.formatter.comment.indent_parameter_description=true
+org.eclipse.jdt.core.formatter.comment.indent_root_tags=true
+org.eclipse.jdt.core.formatter.comment.insert_new_line_before_root_tags=insert
+org.eclipse.jdt.core.formatter.comment.insert_new_line_for_parameter=insert
+org.eclipse.jdt.core.formatter.comment.line_length=80
+org.eclipse.jdt.core.formatter.comment.new_lines_at_block_boundaries=true
+org.eclipse.jdt.core.formatter.comment.new_lines_at_javadoc_boundaries=true
+org.eclipse.jdt.core.formatter.comment.preserve_white_space_between_code_and_line_comments=false
+org.eclipse.jdt.core.formatter.compact_else_if=true
+org.eclipse.jdt.core.formatter.continuation_indentation=2
+org.eclipse.jdt.core.formatter.continuation_indentation_for_array_initializer=2
+org.eclipse.jdt.core.formatter.disabling_tag=@formatter\:off
+org.eclipse.jdt.core.formatter.enabling_tag=@formatter\:on
+org.eclipse.jdt.core.formatter.format_guardian_clause_on_one_line=false
+org.eclipse.jdt.core.formatter.format_line_comment_starting_on_first_column=true
+org.eclipse.jdt.core.formatter.indent_body_declarations_compare_to_annotation_declaration_header=true
+org.eclipse.jdt.core.formatter.indent_body_declarations_compare_to_enum_constant_header=true
+org.eclipse.jdt.core.formatter.indent_body_declarations_compare_to_enum_declaration_header=true
+org.eclipse.jdt.core.formatter.indent_body_declarations_compare_to_type_header=true
+org.eclipse.jdt.core.formatter.indent_breaks_compare_to_cases=true
+org.eclipse.jdt.core.formatter.indent_empty_lines=false
+org.eclipse.jdt.core.formatter.indent_statements_compare_to_block=true
+org.eclipse.jdt.core.formatter.indent_statements_compare_to_body=true
+org.eclipse.jdt.core.formatter.indent_switchstatements_compare_to_cases=true
+org.eclipse.jdt.core.formatter.indent_switchstatements_compare_to_switch=false
+org.eclipse.jdt.core.formatter.indentation.size=4
+org.eclipse.jdt.core.formatter.insert_new_line_after_annotation_on_field=insert
+org.eclipse.jdt.core.formatter.insert_new_line_after_annotation_on_local_variable=insert
+org.eclipse.jdt.core.formatter.insert_new_line_after_annotation_on_method=insert
+org.eclipse.jdt.core.formatter.insert_new_line_after_annotation_on_package=insert
+org.eclipse.jdt.core.formatter.insert_new_line_after_annotation_on_parameter=do not insert
+org.eclipse.jdt.core.formatter.insert_new_line_after_annotation_on_type=insert
+org.eclipse.jdt.core.formatter.insert_new_line_after_label=do not insert
+org.eclipse.jdt.core.formatter.insert_new_line_after_opening_brace_in_array_initializer=do not insert
+org.eclipse.jdt.core.formatter.insert_new_line_at_end_of_file_if_missing=do not insert
+org.eclipse.jdt.core.formatter.insert_new_line_before_catch_in_try_statement=do not insert
+org.eclipse.jdt.core.formatter.insert_new_line_before_closing_brace_in_array_initializer=do not insert
+org.eclipse.jdt.core.formatter.insert_new_line_before_else_in_if_statement=do not insert
+org.eclipse.jdt.core.formatter.insert_new_line_before_finally_in_try_statement=do not insert
+org.eclipse.jdt.core.formatter.insert_new_line_before_while_in_do_statement=do not insert
+org.eclipse.jdt.core.formatter.insert_new_line_in_empty_annotation_declaration=insert
+org.eclipse.jdt.core.formatter.insert_new_line_in_empty_anonymous_type_declaration=insert
+org.eclipse.jdt.core.formatter.insert_new_line_in_empty_block=insert
+org.eclipse.jdt.core.formatter.insert_new_line_in_empty_enum_constant=insert
+org.eclipse.jdt.core.formatter.insert_new_line_in_empty_enum_declaration=insert
+org.eclipse.jdt.core.formatter.insert_new_line_in_empty_method_body=insert
+org.eclipse.jdt.core.formatter.insert_new_line_in_empty_type_declaration=insert
+org.eclipse.jdt.core.formatter.insert_space_after_and_in_type_parameter=insert
+org.eclipse.jdt.core.formatter.insert_space_after_assignment_operator=insert
+org.eclipse.jdt.core.formatter.insert_space_after_at_in_annotation=do not insert
+org.eclipse.jdt.core.formatter.insert_space_after_at_in_annotation_type_declaration=do not insert
+org.eclipse.jdt.core.formatter.insert_space_after_binary_operator=insert
+org.eclipse.jdt.core.formatter.insert_space_after_closing_angle_bracket_in_type_arguments=insert
+org.eclipse.jdt.core.formatter.insert_space_after_closing_angle_bracket_in_type_parameters=insert
+org.eclipse.jdt.core.formatter.insert_space_after_closing_brace_in_block=insert
+org.eclipse.jdt.core.formatter.insert_space_after_closing_paren_in_cast=insert
+org.eclipse.jdt.core.formatter.insert_space_after_colon_in_assert=insert
+org.eclipse.jdt.core.formatter.insert_space_after_colon_in_case=insert
+org.eclipse.jdt.core.formatter.insert_space_after_colon_in_conditional=insert
+org.eclipse.jdt.core.formatter.insert_space_after_colon_in_for=insert
+org.eclipse.jdt.core.formatter.insert_space_after_colon_in_labeled_statement=insert
+org.eclipse.jdt.core.formatter.insert_space_after_comma_in_allocation_expression=insert
+org.eclipse.jdt.core.formatter.insert_space_after_comma_in_annotation=insert
+org.eclipse.jdt.core.formatter.insert_space_after_comma_in_array_initializer=insert
+org.eclipse.jdt.core.formatter.insert_space_after_comma_in_constructor_declaration_parameters=insert
+org.eclipse.jdt.core.formatter.insert_space_after_comma_in_constructor_declaration_throws=insert
+org.eclipse.jdt.core.formatter.insert_space_after_comma_in_enum_constant_arguments=insert
+org.eclipse.jdt.core.formatter.insert_space_after_comma_in_enum_declarations=insert
+org.eclipse.jdt.core.formatter.insert_space_after_comma_in_explicitconstructorcall_arguments=insert
+org.eclipse.jdt.core.formatter.insert_space_after_comma_in_for_increments=insert
+org.eclipse.jdt.core.formatter.insert_space_after_comma_in_for_inits=insert
+org.eclipse.jdt.core.formatter.insert_space_after_comma_in_method_declaration_parameters=insert
+org.eclipse.jdt.core.formatter.insert_space_after_comma_in_method_declaration_throws=insert
+org.eclipse.jdt.core.formatter.insert_space_after_comma_in_method_invocation_arguments=insert
+org.eclipse.jdt.core.formatter.insert_space_after_comma_in_multiple_field_declarations=insert
+org.eclipse.jdt.core.formatter.insert_space_after_comma_in_multiple_local_declarations=insert
+org.eclipse.jdt.core.formatter.insert_space_after_comma_in_parameterized_type_reference=insert
+org.eclipse.jdt.core.formatter.insert_space_after_comma_in_superinterfaces=insert
+org.eclipse.jdt.core.formatter.insert_space_after_comma_in_type_arguments=insert
+org.eclipse.jdt.core.formatter.insert_space_after_comma_in_type_parameters=insert
+org.eclipse.jdt.core.formatter.insert_space_after_ellipsis=insert
+org.eclipse.jdt.core.formatter.insert_space_after_opening_angle_bracket_in_parameterized_type_reference=do not insert
+org.eclipse.jdt.core.formatter.insert_space_after_opening_angle_bracket_in_type_arguments=do not insert
+org.eclipse.jdt.core.formatter.insert_space_after_opening_angle_bracket_in_type_parameters=do not insert
+org.eclipse.jdt.core.formatter.insert_space_after_opening_brace_in_array_initializer=insert
+org.eclipse.jdt.core.formatter.insert_space_after_opening_bracket_in_array_allocation_expression=do not insert
+org.eclipse.jdt.core.formatter.insert_space_after_opening_bracket_in_array_reference=do not insert
+org.eclipse.jdt.core.formatter.insert_space_after_opening_paren_in_annotation=do not insert
+org.eclipse.jdt.core.formatter.insert_space_after_opening_paren_in_cast=do not insert
+org.eclipse.jdt.core.formatter.insert_space_after_opening_paren_in_catch=do not insert
+org.eclipse.jdt.core.formatter.insert_space_after_opening_paren_in_constructor_declaration=do not insert
+org.eclipse.jdt.core.formatter.insert_space_after_opening_paren_in_enum_constant=do not insert
+org.eclipse.jdt.core.formatter.insert_space_after_opening_paren_in_for=do not insert
+org.eclipse.jdt.core.formatter.insert_space_after_opening_paren_in_if=do not insert
+org.eclipse.jdt.core.formatter.insert_space_after_opening_paren_in_method_declaration=do not insert
+org.eclipse.jdt.core.formatter.insert_space_after_opening_paren_in_method_invocation=do not insert
+org.eclipse.jdt.core.formatter.insert_space_after_opening_paren_in_parenthesized_expression=do not insert
+org.eclipse.jdt.core.formatter.insert_space_after_opening_paren_in_switch=do not insert
+org.eclipse.jdt.core.formatter.insert_space_after_opening_paren_in_synchronized=do not insert
+org.eclipse.jdt.core.formatter.insert_space_after_opening_paren_in_try=do not insert
+org.eclipse.jdt.core.formatter.insert_space_after_opening_paren_in_while=do not insert
+org.eclipse.jdt.core.formatter.insert_space_after_postfix_operator=do not insert
+org.eclipse.jdt.core.formatter.insert_space_after_prefix_operator=do not insert
+org.eclipse.jdt.core.formatter.insert_space_after_question_in_conditional=insert
+org.eclipse.jdt.core.formatter.insert_space_after_question_in_wildcard=do not insert
+org.eclipse.jdt.core.formatter.insert_space_after_semicolon_in_for=insert
+org.eclipse.jdt.core.formatter.insert_space_after_semicolon_in_try_resources=insert
+org.eclipse.jdt.core.formatter.insert_space_after_unary_operator=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_and_in_type_parameter=insert
+org.eclipse.jdt.core.formatter.insert_space_before_assignment_operator=insert
+org.eclipse.jdt.core.formatter.insert_space_before_at_in_annotation_type_declaration=insert
+org.eclipse.jdt.core.formatter.insert_space_before_binary_operator=insert
+org.eclipse.jdt.core.formatter.insert_space_before_closing_angle_bracket_in_parameterized_type_reference=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_closing_angle_bracket_in_type_arguments=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_closing_angle_bracket_in_type_parameters=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_closing_brace_in_array_initializer=insert
+org.eclipse.jdt.core.formatter.insert_space_before_closing_bracket_in_array_allocation_expression=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_closing_bracket_in_array_reference=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_closing_paren_in_annotation=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_closing_paren_in_cast=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_closing_paren_in_catch=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_closing_paren_in_constructor_declaration=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_closing_paren_in_enum_constant=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_closing_paren_in_for=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_closing_paren_in_if=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_closing_paren_in_method_declaration=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_closing_paren_in_method_invocation=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_closing_paren_in_parenthesized_expression=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_closing_paren_in_switch=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_closing_paren_in_synchronized=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_closing_paren_in_try=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_closing_paren_in_while=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_colon_in_assert=insert
+org.eclipse.jdt.core.formatter.insert_space_before_colon_in_case=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_colon_in_conditional=insert
+org.eclipse.jdt.core.formatter.insert_space_before_colon_in_default=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_colon_in_for=insert
+org.eclipse.jdt.core.formatter.insert_space_before_colon_in_labeled_statement=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_comma_in_allocation_expression=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_comma_in_annotation=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_comma_in_array_initializer=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_comma_in_constructor_declaration_parameters=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_comma_in_constructor_declaration_throws=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_comma_in_enum_constant_arguments=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_comma_in_enum_declarations=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_comma_in_explicitconstructorcall_arguments=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_comma_in_for_increments=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_comma_in_for_inits=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_comma_in_method_declaration_parameters=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_comma_in_method_declaration_throws=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_comma_in_method_invocation_arguments=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_comma_in_multiple_field_declarations=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_comma_in_multiple_local_declarations=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_comma_in_parameterized_type_reference=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_comma_in_superinterfaces=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_comma_in_type_arguments=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_comma_in_type_parameters=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_ellipsis=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_opening_angle_bracket_in_parameterized_type_reference=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_opening_angle_bracket_in_type_arguments=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_opening_angle_bracket_in_type_parameters=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_opening_brace_in_annotation_type_declaration=insert
+org.eclipse.jdt.core.formatter.insert_space_before_opening_brace_in_anonymous_type_declaration=insert
+org.eclipse.jdt.core.formatter.insert_space_before_opening_brace_in_array_initializer=insert
+org.eclipse.jdt.core.formatter.insert_space_before_opening_brace_in_block=insert
+org.eclipse.jdt.core.formatter.insert_space_before_opening_brace_in_constructor_declaration=insert
+org.eclipse.jdt.core.formatter.insert_space_before_opening_brace_in_enum_constant=insert
+org.eclipse.jdt.core.formatter.insert_space_before_opening_brace_in_enum_declaration=insert
+org.eclipse.jdt.core.formatter.insert_space_before_opening_brace_in_method_declaration=insert
+org.eclipse.jdt.core.formatter.insert_space_before_opening_brace_in_switch=insert
+org.eclipse.jdt.core.formatter.insert_space_before_opening_brace_in_type_declaration=insert
+org.eclipse.jdt.core.formatter.insert_space_before_opening_bracket_in_array_allocation_expression=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_opening_bracket_in_array_reference=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_opening_bracket_in_array_type_reference=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_opening_paren_in_annotation=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_opening_paren_in_annotation_type_member_declaration=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_opening_paren_in_catch=insert
+org.eclipse.jdt.core.formatter.insert_space_before_opening_paren_in_constructor_declaration=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_opening_paren_in_enum_constant=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_opening_paren_in_for=insert
+org.eclipse.jdt.core.formatter.insert_space_before_opening_paren_in_if=insert
+org.eclipse.jdt.core.formatter.insert_space_before_opening_paren_in_method_declaration=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_opening_paren_in_method_invocation=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_opening_paren_in_parenthesized_expression=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_opening_paren_in_switch=insert
+org.eclipse.jdt.core.formatter.insert_space_before_opening_paren_in_synchronized=insert
+org.eclipse.jdt.core.formatter.insert_space_before_opening_paren_in_try=insert
+org.eclipse.jdt.core.formatter.insert_space_before_opening_paren_in_while=insert
+org.eclipse.jdt.core.formatter.insert_space_before_parenthesized_expression_in_return=insert
+org.eclipse.jdt.core.formatter.insert_space_before_parenthesized_expression_in_throw=insert
+org.eclipse.jdt.core.formatter.insert_space_before_postfix_operator=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_prefix_operator=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_question_in_conditional=insert
+org.eclipse.jdt.core.formatter.insert_space_before_question_in_wildcard=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_semicolon=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_semicolon_in_for=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_semicolon_in_try_resources=do not insert
+org.eclipse.jdt.core.formatter.insert_space_before_unary_operator=do not insert
+org.eclipse.jdt.core.formatter.insert_space_between_brackets_in_array_type_reference=do not insert
+org.eclipse.jdt.core.formatter.insert_space_between_empty_braces_in_array_initializer=do not insert
+org.eclipse.jdt.core.formatter.insert_space_between_empty_brackets_in_array_allocation_expression=do not insert
+org.eclipse.jdt.core.formatter.insert_space_between_empty_parens_in_annotation_type_member_declaration=do not insert
+org.eclipse.jdt.core.formatter.insert_space_between_empty_parens_in_constructor_declaration=do not insert
+org.eclipse.jdt.core.formatter.insert_space_between_empty_parens_in_enum_constant=do not insert
+org.eclipse.jdt.core.formatter.insert_space_between_empty_parens_in_method_declaration=do not insert
+org.eclipse.jdt.core.formatter.insert_space_between_empty_parens_in_method_invocation=do not insert
+org.eclipse.jdt.core.formatter.join_lines_in_comments=true
+org.eclipse.jdt.core.formatter.join_wrapped_lines=true
+org.eclipse.jdt.core.formatter.keep_else_statement_on_same_line=false
+org.eclipse.jdt.core.formatter.keep_empty_array_initializer_on_one_line=false
+org.eclipse.jdt.core.formatter.keep_imple_if_on_one_line=false
+org.eclipse.jdt.core.formatter.keep_then_statement_on_same_line=false
+org.eclipse.jdt.core.formatter.lineSplit=999
+org.eclipse.jdt.core.formatter.never_indent_block_comments_on_first_column=false
+org.eclipse.jdt.core.formatter.never_indent_line_comments_on_first_column=false
+org.eclipse.jdt.core.formatter.number_of_blank_lines_at_beginning_of_method_body=0
+org.eclipse.jdt.core.formatter.number_of_empty_lines_to_preserve=1
+org.eclipse.jdt.core.formatter.put_empty_statement_on_new_line=true
+org.eclipse.jdt.core.formatter.tabulation.char=space
+org.eclipse.jdt.core.formatter.tabulation.size=4
+org.eclipse.jdt.core.formatter.use_on_off_tags=false
+org.eclipse.jdt.core.formatter.use_tabs_only_for_leading_indentations=false
+org.eclipse.jdt.core.formatter.wrap_before_binary_operator=true
+org.eclipse.jdt.core.formatter.wrap_before_or_operator_multicatch=true
+org.eclipse.jdt.core.formatter.wrap_outer_expressions_when_nested=true

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/server/.settings/org.eclipse.jdt.ui.prefs
----------------------------------------------------------------------
diff --git a/server/.settings/org.eclipse.jdt.ui.prefs b/server/.settings/org.eclipse.jdt.ui.prefs
new file mode 100644
index 0000000..dece0e6
--- /dev/null
+++ b/server/.settings/org.eclipse.jdt.ui.prefs
@@ -0,0 +1,7 @@
+eclipse.preferences.version=1
+formatter_profile=_Space Indent & Long Lines
+formatter_settings_version=12
+org.eclipse.jdt.ui.ignorelowercasenames=true
+org.eclipse.jdt.ui.importorder=java;javax;org;com;
+org.eclipse.jdt.ui.ondemandthreshold=99
+org.eclipse.jdt.ui.staticondemandthreshold=1

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/server/ServiceConfig.md
----------------------------------------------------------------------
diff --git a/server/ServiceConfig.md b/server/ServiceConfig.md
new file mode 100644
index 0000000..723557e
--- /dev/null
+++ b/server/ServiceConfig.md
@@ -0,0 +1,42 @@
+### Service
+##### kylin.rest.servers 
+##### kylin.rest.timezone
+##### kylin.server.mode Valid value: 'query', 'job' and 'all'
+
+### Storage
+##### kylin.tmp.hdfs.dir
+##### kylin.metadata.url
+##### kylin.storage.url 
+##### kylin.hdfs.working.dir
+##### kylin.hbase.coprocessor.jar 
+##### kylin.hbase.default.table.region.count 
+
+### Query
+##### kylin.query.scan.threshold
+##### kylin.query.security.enabled 
+##### kylin.query.cache.threshold.duration
+##### kylin.query.cache.threshold.scancount
+##### kylin.query.scan.thread.count
+##### kylin.query.cache.enabled
+
+### Job
+##### kylin.job.jar 
+##### kylin.job.log.dir
+##### kylin.job.hdfs.working.dir
+##### kylin.job.mapreduce.default.reduce.input.mb
+##### kylin.job.mapreduce.default.reduce.count.ratio
+##### kylin.job.mapreduce.default.compress.output
+##### kylin.job.mapreduce.default.compress.codec
+##### kylin.job.run.as.remote.cmd
+##### kylin.job.remote.cli.hostname
+##### kylin.job.remote.cli.username
+##### kylin.job.remote.cli.password
+##### kylin.job.remote.cli.working.dir
+##### kylin.job.concurrent.max.limit
+##### kylin.job.yarn.app.rest.check.status.url
+##### kylin.job.command.exe.type
+##### kylin.job.admin.dls
+##### kylin.job.step.timeout
+##### kylin.job.yarn.app.rest.check.status.url
+##### kylin.job.yarn.app.rest.check.interval.seconds
+##### kylin.job.concurrent.max.limit
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/server/pom.xml
----------------------------------------------------------------------
diff --git a/server/pom.xml b/server/pom.xml
new file mode 100644
index 0000000..fea3397
--- /dev/null
+++ b/server/pom.xml
@@ -0,0 +1,390 @@
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+    <modelVersion>4.0.0</modelVersion>
+
+    <artifactId>kylin-server</artifactId>
+    <packaging>war</packaging>
+    <name>Kylin:RESTServer</name>
+    <description>Kylin REST Service and RPC Server.</description>
+
+    <parent>
+        <groupId>com.kylinolap</groupId>
+        <artifactId>kylin</artifactId>
+        <version>0.6.3-SNAPSHOT</version>
+    </parent>
+
+    <properties>
+        <start-class>com.kylinolap.odbc.Application</start-class>
+    </properties>
+    <dependencies>
+
+        <dependency>
+            <groupId>com.kylinolap</groupId>
+            <artifactId>kylin-query</artifactId>
+            <version>${project.parent.version}</version>
+            <exclusions>
+                <exclusion>
+                    <groupId>javax.servlet</groupId>
+                    <artifactId>servlet-api</artifactId>
+                </exclusion>
+                <exclusion>
+                    <groupId>javax.servlet.jsp</groupId>
+                    <artifactId>jsp-api</artifactId>
+                </exclusion>
+            </exclusions>
+        </dependency>
+        <dependency>
+            <groupId>com.kylinolap</groupId>
+            <artifactId>kylin-job</artifactId>
+            <version>${project.parent.version}</version>
+            <exclusions>
+                <exclusion>
+                    <groupId>org.apache.hbase</groupId>
+                    <artifactId>hbase-common</artifactId>
+                </exclusion>
+            </exclusions>
+        </dependency>
+
+        <dependency>
+            <groupId>xerces</groupId>
+            <artifactId>xercesImpl</artifactId>
+        </dependency>
+        <dependency>
+            <groupId>xalan</groupId>
+            <artifactId>xalan</artifactId>
+        </dependency>
+
+        <dependency>
+            <groupId>org.springframework.boot</groupId>
+            <artifactId>spring-boot-starter-web</artifactId>
+            <version>0.5.0.M6</version>
+            <exclusions>
+                <exclusion>
+                    <groupId>org.springframework.boot</groupId>
+                    <artifactId>spring-boot-starter-tomcat</artifactId>
+                </exclusion>
+                <exclusion>
+                    <groupId>ch.qos.logback</groupId>
+                    <artifactId>logback-classic</artifactId>
+                </exclusion>
+            </exclusions>
+        </dependency>
+        <dependency>
+            <groupId>com.fasterxml.jackson.core</groupId>
+            <artifactId>jackson-databind</artifactId>
+        </dependency>
+
+        <dependency>
+            <groupId>log4j</groupId>
+            <artifactId>log4j</artifactId>
+        </dependency>
+        <dependency>
+            <groupId>org.slf4j</groupId>
+            <artifactId>jcl-over-slf4j</artifactId>
+        </dependency>
+        <dependency>
+            <groupId>org.slf4j</groupId>
+            <artifactId>slf4j-api</artifactId>
+        </dependency>
+        <dependency>
+            <groupId>org.slf4j</groupId>
+            <artifactId>slf4j-log4j12</artifactId>
+        </dependency>
+        <dependency>
+            <groupId>org.springframework</groupId>
+            <artifactId>spring-beans</artifactId>
+            <version>${spring.framework.version}</version>
+        </dependency>
+        <dependency>
+            <groupId>org.springframework</groupId>
+            <artifactId>spring-webmvc</artifactId>
+            <version>${spring.framework.version}</version>
+        </dependency>
+        <dependency>
+            <groupId>org.springframework</groupId>
+            <artifactId>spring-jdbc</artifactId>
+            <version>${spring.framework.version}</version>
+        </dependency>
+        <dependency>
+            <groupId>org.springframework</groupId>
+            <artifactId>spring-web</artifactId>
+            <version>${spring.framework.version}</version>
+            <type>jar</type>
+        </dependency>
+        <dependency>
+            <groupId>org.springframework</groupId>
+            <artifactId>spring-core</artifactId>
+            <version>${spring.framework.version}</version>
+            <type>jar</type>
+        </dependency>
+        <dependency>
+            <groupId>org.springframework</groupId>
+            <artifactId>spring-aop</artifactId>
+            <version>${spring.framework.version}</version>
+            <type>jar</type>
+        </dependency>
+        <dependency>
+            <groupId>org.springframework</groupId>
+            <artifactId>spring-test</artifactId>
+            <version>${spring.framework.version}</version>
+        </dependency>
+
+        <!-- Spring Security -->
+        <dependency>
+            <groupId>org.springframework.security</groupId>
+            <artifactId>spring-security-core</artifactId>
+            <version>${spring.framework.version}</version>
+        </dependency>
+        <dependency>
+            <groupId>org.springframework.security</groupId>
+            <artifactId>spring-security-web</artifactId>
+            <version>${spring.framework.version}</version>
+        </dependency>
+        <dependency>
+            <groupId>org.springframework.security</groupId>
+            <artifactId>spring-security-config</artifactId>
+            <version>${spring.framework.version}</version>
+        </dependency>
+        <dependency>
+            <groupId>org.springframework.security</groupId>
+            <artifactId>spring-security-acl</artifactId>
+            <version>${spring.framework.version}</version>
+        </dependency>
+        <dependency>
+            <groupId>org.springframework.security</groupId>
+            <artifactId>spring-security-ldap</artifactId>
+            <version>${spring.framework.version}</version>
+        </dependency>
+        <dependency>
+            <groupId>net.sf.ehcache</groupId>
+            <artifactId>ehcache</artifactId>
+            <version>2.8.1</version>
+        </dependency>
+        <dependency>
+            <groupId>com.codahale.metrics</groupId>
+            <artifactId>metrics-core</artifactId>
+            <version>${metrics.version}</version>
+        </dependency>
+        <dependency>
+            <groupId>com.codahale.metrics</groupId>
+            <artifactId>metrics-servlets</artifactId>
+            <version>${metrics.version}</version>
+        </dependency>
+        <dependency>
+            <groupId>com.codahale.metrics</groupId>
+            <artifactId>metrics-servlet</artifactId>
+            <version>${metrics.version}</version>
+        </dependency>
+        <dependency>
+            <groupId>com.codahale.metrics</groupId>
+            <artifactId>metrics-ganglia</artifactId>
+            <version>${metrics.version}</version>
+        </dependency>
+        <dependency>
+            <groupId>com.ryantenney.metrics</groupId>
+            <artifactId>metrics-spring</artifactId>
+            <version>3.0.0</version>
+        </dependency>
+        <dependency>
+            <groupId>cglib</groupId>
+            <artifactId>cglib</artifactId>
+            <version>2.2.2</version>
+        </dependency>
+        <dependency>
+            <groupId>net.sf.supercsv</groupId>
+            <artifactId>super-csv</artifactId>
+            <version>2.1.0</version>
+        </dependency>
+        <dependency>
+            <groupId>com.h2database</groupId>
+            <artifactId>h2</artifactId>
+        </dependency>
+		
+        <!-- spring aop -->
+        <dependency>
+            <groupId>org.aspectj</groupId>
+            <artifactId>aspectjrt</artifactId>
+            <version>1.6.11</version>
+        </dependency>
+        <dependency>
+            <groupId>org.aspectj</groupId>
+            <artifactId>aspectjweaver</artifactId>
+            <version>1.6.11</version>
+        </dependency>
+
+        <!-- Env & Test -->
+        <dependency>
+            <groupId>org.apache.hadoop</groupId>
+            <artifactId>hadoop-yarn-server-resourcemanager</artifactId>
+            <scope>provided</scope>
+            <exclusions>
+                <exclusion>
+                    <groupId>javax.servlet</groupId>
+                    <artifactId>servlet-api</artifactId>
+                </exclusion>
+                <exclusion>
+                    <groupId>javax.servlet.jsp</groupId>
+                    <artifactId>jsp-api</artifactId>
+                </exclusion>
+            </exclusions>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.hadoop</groupId>
+            <artifactId>hadoop-common</artifactId>
+            <scope>provided</scope>
+            <exclusions>
+                <exclusion>
+                    <groupId>javax.servlet</groupId>
+                    <artifactId>servlet-api</artifactId>
+                </exclusion>
+                <exclusion>
+                    <groupId>javax.servlet.jsp</groupId>
+                    <artifactId>jsp-api</artifactId>
+                </exclusion>
+            </exclusions>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.hadoop</groupId>
+            <artifactId>hadoop-mapreduce-client-core</artifactId>
+            <scope>provided</scope>
+            <exclusions>
+                <exclusion>
+                    <groupId>javax.servlet</groupId>
+                    <artifactId>servlet-api</artifactId>
+                </exclusion>
+                <exclusion>
+                    <groupId>javax.servlet.jsp</groupId>
+                    <artifactId>jsp-api</artifactId>
+                </exclusion>
+            </exclusions>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.hadoop</groupId>
+            <artifactId>hadoop-hdfs</artifactId>
+            <scope>provided</scope>
+            <exclusions>
+                <exclusion>
+                    <groupId>javax.servlet</groupId>
+                    <artifactId>servlet-api</artifactId>
+                </exclusion>
+                <exclusion>
+                    <groupId>javax.servlet.jsp</groupId>
+                    <artifactId>jsp-api</artifactId>
+                </exclusion>
+            </exclusions>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.hbase</groupId>
+            <artifactId>hbase-client</artifactId>
+            <scope>provided</scope>
+            <exclusions>
+                <exclusion>
+                    <groupId>javax.servlet</groupId>
+                    <artifactId>servlet-api</artifactId>
+                </exclusion>
+                <exclusion>
+                    <groupId>javax.servlet.jsp</groupId>
+                    <artifactId>jsp-api</artifactId>
+                </exclusion>
+            </exclusions>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.hbase</groupId>
+            <artifactId>hbase-server</artifactId>
+            <scope>provided</scope>
+            <exclusions>
+                <exclusion>
+                    <groupId>javax.servlet</groupId>
+                    <artifactId>servlet-api</artifactId>
+                </exclusion>
+                <exclusion>
+                    <groupId>javax.servlet.jsp</groupId>
+                    <artifactId>jsp-api</artifactId>
+                </exclusion>
+                <exclusion>
+                    <groupId>org.mortbay.jetty</groupId>
+                    <artifactId>servlet-api-2.5</artifactId>
+                </exclusion>
+                <exclusion>
+                    <groupId>javax.servlet.jsp</groupId>
+                    <artifactId>jsp-api</artifactId>
+                </exclusion>
+            </exclusions>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.hadoop</groupId>
+            <artifactId>hadoop-mapreduce-client-jobclient</artifactId>
+            <scope>provided</scope>
+            <exclusions>
+                <exclusion>
+                    <groupId>javax.servlet</groupId>
+                    <artifactId>servlet-api</artifactId>
+                </exclusion>
+                <exclusion>
+                    <groupId>javax.servlet.jsp</groupId>
+                    <artifactId>jsp-api</artifactId>
+                </exclusion>
+            </exclusions>
+        </dependency>
+
+        <!-- Tomcat Env -->
+        <dependency>
+            <groupId>org.apache.tomcat</groupId>
+            <artifactId>tomcat-catalina</artifactId>
+            <version>7.0.52</version>
+            <scope>provided</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.tomcat</groupId>
+            <artifactId>tomcat-jasper</artifactId>
+            <version>7.0.52</version>
+            <scope>provided</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.tomcat.embed</groupId>
+            <artifactId>tomcat-embed-core</artifactId>
+            <version>7.0.52</version>
+            <scope>provided</scope>
+        </dependency>
+        <dependency>
+            <groupId>junit</groupId>
+            <artifactId>junit</artifactId>
+            <scope>test</scope>
+        </dependency>
+    </dependencies>
+
+    <build>
+        <plugins>
+            <plugin>
+                <artifactId>maven-compiler-plugin</artifactId>
+            </plugin>
+            <plugin>
+                <groupId>org.springframework.boot</groupId>
+                <artifactId>spring-boot-maven-plugin</artifactId>
+            </plugin>
+        </plugins>
+    </build>
+
+    <repositories>
+        <repository>
+            <id>spring-snapshots</id>
+            <url>http://repo.spring.io/libs-snapshot</url>
+            <snapshots>
+                <enabled>true</enabled>
+            </snapshots>
+        </repository>
+        <repository>
+            <id>mvn-adamgent</id>
+            <url>http://mvn-adamgent.googlecode.com/svn/maven/release</url>
+            <name>Adam Gent Maven Repository</name>
+        </repository>
+    </repositories>
+    <pluginRepositories>
+        <pluginRepository>
+            <id>spring-snapshots</id>
+            <url>http://repo.spring.io/libs-snapshot</url>
+            <snapshots>
+                <enabled>true</enabled>
+            </snapshots>
+        </pluginRepository>
+    </pluginRepositories>
+</project>

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/server/src/main/java/com/kylinolap/rest/DebugTomcat.java
----------------------------------------------------------------------
diff --git a/server/src/main/java/com/kylinolap/rest/DebugTomcat.java b/server/src/main/java/com/kylinolap/rest/DebugTomcat.java
new file mode 100644
index 0000000..ea5813d
--- /dev/null
+++ b/server/src/main/java/com/kylinolap/rest/DebugTomcat.java
@@ -0,0 +1,65 @@
+/*
+ * 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.rest;
+
+import java.io.File;
+
+import org.apache.catalina.Context;
+import org.apache.catalina.core.AprLifecycleListener;
+import org.apache.catalina.core.StandardServer;
+import org.apache.catalina.deploy.ErrorPage;
+import org.apache.catalina.startup.Tomcat;
+
+import com.kylinolap.rest.util.ClasspathUtil;
+
+public class DebugTomcat {
+
+    public static void main(String[] args) throws Exception {
+        if (args.length >= 1) {
+            System.setProperty("kylin.metadata.url", args[0]);
+        }
+        int port = 7070;
+        if (args.length >= 2) {
+            port = Integer.parseInt(args[1]);
+        }
+
+        ClasspathUtil.addClasspath(new File("../examples/test_case_data/sandbox").getAbsolutePath());
+        String webBase = new File("../webapp/app").getAbsolutePath();
+        String apiBase = new File("src/main/webapp").getAbsolutePath();
+
+        Tomcat tomcat = new Tomcat();
+        tomcat.setPort(port);
+        tomcat.setBaseDir(".");
+
+        // Add AprLifecycleListener
+        StandardServer server = (StandardServer) tomcat.getServer();
+        AprLifecycleListener listener = new AprLifecycleListener();
+        server.addLifecycleListener(listener);
+
+        tomcat.addWebapp("/kylin", apiBase);
+        Context webContext = tomcat.addWebapp("/", webBase);
+        ErrorPage notFound = new ErrorPage();
+        notFound.setErrorCode(404);
+        notFound.setLocation("/index.html");
+        webContext.addErrorPage(notFound);
+        webContext.addWelcomeFile("index.html");
+
+        // tomcat start
+        tomcat.start();
+        tomcat.getServer().await();
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/server/src/main/java/com/kylinolap/rest/constant/Constant.java
----------------------------------------------------------------------
diff --git a/server/src/main/java/com/kylinolap/rest/constant/Constant.java b/server/src/main/java/com/kylinolap/rest/constant/Constant.java
new file mode 100644
index 0000000..6868f2d
--- /dev/null
+++ b/server/src/main/java/com/kylinolap/rest/constant/Constant.java
@@ -0,0 +1,50 @@
+/*
+ * 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.rest.constant;
+
+/**
+ * @author xduo
+ * 
+ */
+public class Constant {
+
+    // @hardcode
+    public final static String FakeSchemaName = "defaultSchema";
+
+    // @hardcode
+    public final static String FakeCatalogName = "defaultCatalog";
+
+    public final static String IDENTITY_USER = "user";
+
+    public final static String IDENTITY_ROLE = "role";
+
+    public final static String ROLE_ADMIN = "ROLE_ADMIN";
+    public static final String ROLE_MODELER = "ROLE_MODELER";
+    public static final String ROLE_ANALYST = "ROLE_ANALYST";
+    
+    public final static String ACCESS_HAS_ROLE_ADMIN = "hasRole('ROLE_ADMIN')";
+    public final static String ACCESS_HAS_ROLE_MODELER = "hasRole('ROLE_MODELER')";
+    
+    public final static String ACCESS_POST_FILTER_READ = "hasRole('ROLE_ADMIN') or hasPermission(filterObject, 'READ') or hasPermission(filterObject, 'MANAGEMENT') " + "or hasPermission(filterObject, 'OPERATION') or hasPermission(filterObject, 'ADMINISTRATION')";
+
+    public final static String SERVER_MODE_QUERY = "query";
+    public final static String SERVER_MODE_JOB = "job";
+    public final static String SERVER_MODE_ALL = "all";
+
+
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/server/src/main/java/com/kylinolap/rest/controller/AccessController.java
----------------------------------------------------------------------
diff --git a/server/src/main/java/com/kylinolap/rest/controller/AccessController.java b/server/src/main/java/com/kylinolap/rest/controller/AccessController.java
new file mode 100644
index 0000000..997b65e
--- /dev/null
+++ b/server/src/main/java/com/kylinolap/rest/controller/AccessController.java
@@ -0,0 +1,117 @@
+/*
+ * 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.rest.controller;
+
+import java.io.IOException;
+import java.util.List;
+
+import org.springframework.beans.factory.annotation.Autowired;
+import org.springframework.security.acls.model.Acl;
+import org.springframework.security.acls.model.Permission;
+import org.springframework.security.acls.model.Sid;
+import org.springframework.stereotype.Controller;
+import org.springframework.web.bind.annotation.PathVariable;
+import org.springframework.web.bind.annotation.RequestBody;
+import org.springframework.web.bind.annotation.RequestMapping;
+import org.springframework.web.bind.annotation.RequestMethod;
+import org.springframework.web.bind.annotation.ResponseBody;
+
+import com.kylinolap.common.persistence.AclEntity;
+import com.kylinolap.rest.request.AccessRequest;
+import com.kylinolap.rest.response.AccessEntryResponse;
+import com.kylinolap.rest.security.AclPermissionFactory;
+import com.kylinolap.rest.service.AccessService;
+
+/**
+ * @author xduo
+ * 
+ */
+@Controller
+@RequestMapping(value = "/access")
+public class AccessController extends BasicController {
+
+    @Autowired
+    private AccessService accessService;
+
+    /**
+     * Get access entry list of a domain object
+     * 
+     * @param uuid
+     * @return
+     * @throws IOException
+     */
+    @RequestMapping(value = "/{type}/{uuid}", method = { RequestMethod.GET })
+    @ResponseBody
+    public List<AccessEntryResponse> getAccessEntities(@PathVariable String type, @PathVariable String uuid) {
+        AclEntity ae = accessService.getAclEntity(type, uuid);
+        Acl acl = accessService.getAcl(ae);
+
+        return accessService.generateAceResponses(acl);
+    }
+
+    /**
+     * Grant a new access on a domain object to a user/role
+     * 
+     * @param accessRequest
+     */
+    @RequestMapping(value = "/{type}/{uuid}", method = { RequestMethod.POST })
+    @ResponseBody
+    public List<AccessEntryResponse> grant(@PathVariable String type, @PathVariable String uuid, @RequestBody AccessRequest accessRequest) {
+        AclEntity ae = accessService.getAclEntity(type, uuid);
+        Sid sid = accessService.getSid(accessRequest.getSid(), accessRequest.isPrincipal());
+        Permission permission = AclPermissionFactory.getPermission(accessRequest.getPermission());
+        Acl acl = accessService.grant(ae, permission, sid);
+
+        return accessService.generateAceResponses(acl);
+    }
+
+    /**
+     * Update a access on a domain object
+     * 
+     * @param accessRequest
+     */
+    @RequestMapping(value = "/{type}/{uuid}", method = { RequestMethod.PUT })
+    @ResponseBody
+    public List<AccessEntryResponse> update(@PathVariable String type, @PathVariable String uuid, @RequestBody AccessRequest accessRequest) {
+        AclEntity ae = accessService.getAclEntity(type, uuid);
+        Permission permission = AclPermissionFactory.getPermission(accessRequest.getPermission());
+        Acl acl = accessService.update(ae, accessRequest.getAccessEntryId(), permission);
+
+        return accessService.generateAceResponses(acl);
+    }
+
+    /**
+     * Revoke access on a domain object from a user/role
+     * 
+     * @param AccessRequest
+     */
+    @RequestMapping(value = "/{type}/{uuid}", method = { RequestMethod.DELETE })
+    public List<AccessEntryResponse> revoke(@PathVariable String type, @PathVariable String uuid, AccessRequest accessRequest) {
+        AclEntity ae = accessService.getAclEntity(type, uuid);
+        Acl acl = accessService.revoke(ae, accessRequest.getAccessEntryId());
+
+        return accessService.generateAceResponses(acl);
+    }
+
+    /**
+     * @param accessService
+     */
+    public void setAccessService(AccessService accessService) {
+        this.accessService = accessService;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/server/src/main/java/com/kylinolap/rest/controller/AdminController.java
----------------------------------------------------------------------
diff --git a/server/src/main/java/com/kylinolap/rest/controller/AdminController.java b/server/src/main/java/com/kylinolap/rest/controller/AdminController.java
new file mode 100644
index 0000000..c3ddbed
--- /dev/null
+++ b/server/src/main/java/com/kylinolap/rest/controller/AdminController.java
@@ -0,0 +1,95 @@
+/*
+ * 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.rest.controller;
+
+import org.springframework.beans.factory.annotation.Autowired;
+import org.springframework.stereotype.Controller;
+import org.springframework.web.bind.annotation.RequestBody;
+import org.springframework.web.bind.annotation.RequestMapping;
+import org.springframework.web.bind.annotation.RequestMethod;
+import org.springframework.web.bind.annotation.ResponseBody;
+
+import com.kylinolap.common.KylinConfig;
+import com.kylinolap.rest.request.MetricsRequest;
+import com.kylinolap.rest.request.UpdateConfigRequest;
+import com.kylinolap.rest.response.GeneralResponse;
+import com.kylinolap.rest.response.MetricsResponse;
+import com.kylinolap.rest.service.AdminService;
+import com.kylinolap.rest.service.CubeService;
+
+/**
+ * Admin Controller is defined as Restful API entrance for UI.
+ * 
+ * @author jianliu
+ * 
+ */
+@Controller
+@RequestMapping(value = "/admin")
+public class AdminController extends BasicController {
+
+    @Autowired
+    private AdminService adminService;
+    @Autowired
+    private CubeService cubeMgmtService;
+
+    @RequestMapping(value = "/env", method = { RequestMethod.GET })
+    @ResponseBody
+    public GeneralResponse getEnv() {
+        String env = adminService.getEnv();
+
+        GeneralResponse envRes = new GeneralResponse();
+        envRes.put("env", env);
+
+        return envRes;
+    }
+
+    @RequestMapping(value = "/config", method = { RequestMethod.GET })
+    @ResponseBody
+    public GeneralResponse getConfig() {
+        String config = adminService.getConfigAsString();
+
+        GeneralResponse configRes = new GeneralResponse();
+        configRes.put("config", config);
+
+        return configRes;
+    }
+
+    @RequestMapping(value = "/metrics/cubes", method = { RequestMethod.GET })
+    @ResponseBody
+    public MetricsResponse cubeMetrics(MetricsRequest request) {
+        return cubeMgmtService.calculateMetrics(request);
+    }
+
+    @RequestMapping(value = "/storage", method = { RequestMethod.DELETE })
+    @ResponseBody
+    public void cleanupStorage() {
+        adminService.cleanupStorage();
+    }
+
+    @RequestMapping(value = "/config", method = { RequestMethod.PUT })
+    public void updateKylinConfig(@RequestBody UpdateConfigRequest updateConfigRequest) {
+        KylinConfig.getInstanceFromEnv().setProperty(updateConfigRequest.getKey(), updateConfigRequest.getValue());
+    }
+
+    public void setAdminService(AdminService adminService) {
+        this.adminService = adminService;
+    }
+
+    public void setCubeMgmtService(CubeService cubeMgmtService) {
+        this.cubeMgmtService = cubeMgmtService;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/server/src/main/java/com/kylinolap/rest/controller/BasicController.java
----------------------------------------------------------------------
diff --git a/server/src/main/java/com/kylinolap/rest/controller/BasicController.java b/server/src/main/java/com/kylinolap/rest/controller/BasicController.java
new file mode 100644
index 0000000..e69691a
--- /dev/null
+++ b/server/src/main/java/com/kylinolap/rest/controller/BasicController.java
@@ -0,0 +1,75 @@
+/*
+ * 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.rest.controller;
+
+import javax.servlet.http.HttpServletRequest;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.springframework.beans.factory.annotation.Autowired;
+import org.springframework.http.HttpStatus;
+import org.springframework.web.bind.annotation.ExceptionHandler;
+import org.springframework.web.bind.annotation.ResponseBody;
+import org.springframework.web.bind.annotation.ResponseStatus;
+
+import com.kylinolap.rest.exception.BadRequestException;
+import com.kylinolap.rest.exception.ForbiddenException;
+import com.kylinolap.rest.exception.InternalErrorException;
+import com.kylinolap.rest.exception.NotFoundException;
+import com.kylinolap.rest.response.ErrorResponse;
+import com.kylinolap.rest.service.MetricsService;
+
+/**
+ * Created with IntelliJ IDEA. User: lukhan Date: 9/1/13 Time: 10:54 PM To
+ * change this template use File | Settings | File Templates.
+ */
+public class BasicController {
+
+    private static final Logger logger = LoggerFactory.getLogger(BasicController.class);
+
+    @Autowired
+    protected MetricsService metricsService;
+
+    // ~ general handler ~
+    @ResponseStatus(HttpStatus.INTERNAL_SERVER_ERROR)
+    @ExceptionHandler(Exception.class)
+    @ResponseBody
+    ErrorResponse handleError(HttpServletRequest req, Exception ex) {
+        return new ErrorResponse(req.getRequestURL().toString(), ex);
+    }
+    
+
+    @ResponseStatus(HttpStatus.FORBIDDEN)
+    @ExceptionHandler(ForbiddenException.class)
+    @ResponseBody
+    ErrorResponse handleForbidden(HttpServletRequest req, Exception ex) {
+        return new ErrorResponse(req.getRequestURL().toString(), ex);
+    }
+
+    @ResponseStatus(HttpStatus.NOT_FOUND)
+    @ExceptionHandler(NotFoundException.class)
+    @ResponseBody
+    ErrorResponse handleNotFound(HttpServletRequest req, Exception ex) {
+        return new ErrorResponse(req.getRequestURL().toString(), ex);
+    }
+
+    @ResponseStatus(HttpStatus.BAD_REQUEST)
+    @ExceptionHandler(BadRequestException.class)
+    @ResponseBody
+    ErrorResponse handleBadRequest(HttpServletRequest req, Exception ex) {
+        return new ErrorResponse(req.getRequestURL().toString(), ex);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/server/src/main/java/com/kylinolap/rest/controller/CacheController.java
----------------------------------------------------------------------
diff --git a/server/src/main/java/com/kylinolap/rest/controller/CacheController.java b/server/src/main/java/com/kylinolap/rest/controller/CacheController.java
new file mode 100644
index 0000000..9a870ca
--- /dev/null
+++ b/server/src/main/java/com/kylinolap/rest/controller/CacheController.java
@@ -0,0 +1,113 @@
+/*
+ * 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.rest.controller;
+
+import java.io.IOException;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.springframework.beans.factory.annotation.Autowired;
+import org.springframework.stereotype.Controller;
+import org.springframework.web.bind.annotation.PathVariable;
+import org.springframework.web.bind.annotation.RequestMapping;
+import org.springframework.web.bind.annotation.RequestMethod;
+import org.springframework.web.bind.annotation.ResponseBody;
+
+import com.kylinolap.common.restclient.Broadcaster;
+import com.kylinolap.common.restclient.Broadcaster.EVENT;
+import com.kylinolap.metadata.MetadataConstances;
+import com.kylinolap.rest.service.CubeService;
+import com.kylinolap.rest.service.ProjectService;
+
+/**
+ * CubeController is defined as Restful API entrance for UI.
+ * 
+ * @author jianliu
+ * 
+ */
+@Controller
+@RequestMapping(value = "/cache")
+public class CacheController extends BasicController {
+    private static final Logger logger = LoggerFactory.getLogger(CacheController.class);
+
+    @Autowired
+    private CubeService cubeMgmtService;
+
+    @Autowired
+    private ProjectService projectService;
+
+    /**
+     * Wipe system cache
+     * 
+     * @param type
+     *            {@link MetadataConstances.TYPE}
+     * @param event
+     *            {@link MetadataConstances.EVENT}
+     * @param name
+     * @return if the action success
+     * @throws IOException
+     */
+    @RequestMapping(value = "/{type}/{name}/{event}", method = { RequestMethod.PUT })
+    @ResponseBody
+    public void wipeCache(@PathVariable String type, @PathVariable String event, @PathVariable String name) throws IOException {
+        Broadcaster.TYPE wipeType = Broadcaster.TYPE.getType(type);
+        EVENT wipeEvent = Broadcaster.EVENT.getEvent(event);
+        switch (wipeType) {
+        case METADATA:
+            logger.debug("Reload all metadata");
+            cubeMgmtService.reloadMetadataCache();
+            projectService.cleanDataCache();
+            cubeMgmtService.cleanDataCache();
+            break;
+        case CUBE:
+            logger.debug("Reload cube " + name + " with type:" + type + ", event type " + event);
+            cubeMgmtService.reloadMetadataCache();
+            if ("ALL".equalsIgnoreCase(name.toUpperCase())) {
+                cubeMgmtService.cleanDataCache();
+                break;
+            }
+
+            switch (wipeEvent) {
+            case CREATE:
+            case UPDATE:
+                cubeMgmtService.reloadCubeCache(name);
+                break;
+            case DROP:
+                cubeMgmtService.removeCubeCache(name);
+                break;
+            }
+            break;
+        case PROJECT:
+            logger.debug("Reload project " + name + " with type:" + type + ", event type " + event);
+            cubeMgmtService.reloadMetadataCache();
+            if ("ALL".equalsIgnoreCase(name.toUpperCase())) {
+                projectService.cleanDataCache();
+                break;
+            }
+
+            switch (wipeEvent) {
+            case CREATE:
+            case UPDATE:
+                projectService.reloadProjectCache(name);
+                break;
+            case DROP:
+                projectService.removeProjectCache(name);
+                break;
+            }
+            break;
+        }
+    }
+}


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

Posted by lu...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/job/src/main/java/com/kylinolap/job/engine/JobEngine.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/com/kylinolap/job/engine/JobEngine.java b/job/src/main/java/com/kylinolap/job/engine/JobEngine.java
new file mode 100644
index 0000000..b86fe36
--- /dev/null
+++ b/job/src/main/java/com/kylinolap/job/engine/JobEngine.java
@@ -0,0 +1,227 @@
+/*
+ * 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.job.engine;
+
+/** 
+ * @author George Song (ysong1), xduo
+ * 
+ */
+import java.io.File;
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.concurrent.ConcurrentHashMap;
+
+import com.kylinolap.common.KylinConfig;
+import org.apache.commons.lang.ArrayUtils;
+import org.apache.commons.math.stat.descriptive.rank.Percentile;
+import org.apache.curator.RetryPolicy;
+import org.apache.curator.framework.CuratorFramework;
+import org.apache.curator.framework.CuratorFrameworkFactory;
+import org.apache.curator.framework.imps.CuratorFrameworkState;
+import org.apache.curator.framework.recipes.locks.InterProcessMutex;
+import org.apache.curator.framework.state.ConnectionState;
+import org.apache.curator.framework.state.ConnectionStateListener;
+import org.apache.curator.retry.ExponentialBackoffRetry;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.quartz.Scheduler;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.kylinolap.job.JobInstance;
+import com.kylinolap.job.JobInstance.JobStep;
+import com.kylinolap.job.constant.JobConstants;
+import com.kylinolap.job.exception.JobException;
+
+public class JobEngine implements ConnectionStateListener {
+
+    private static Logger log = LoggerFactory.getLogger(JobEngine.class);
+
+    private final String engineID;
+
+    private final JobEngineConfig engineConfig;
+    private final QuatzScheduler scheduler;
+    private InterProcessMutex sharedLock;
+    private CuratorFramework zkClient;
+
+    private static final String ZOOKEEPER_LOCK_PATH = "/kylin/job_engine/lock";
+    private static final ConcurrentHashMap<JobEngineConfig, JobEngine> CACHE = new ConcurrentHashMap<JobEngineConfig, JobEngine>();
+    private int daemonJobIntervalInSeconds;
+
+    public static JobEngine getInstance(String engineID, JobEngineConfig engineCfg) throws JobException {
+        JobEngine r = CACHE.get(engineCfg);
+        if (r == null) {
+            r = new JobEngine(engineID, engineCfg);
+            CACHE.putIfAbsent(engineCfg, r);
+        }
+        return r;
+    }
+
+    private JobEngine(String engineID, JobEngineConfig context) throws JobException {
+        if (context.getZookeeperString() == null || context.getZookeeperString().equals("")) {
+            throw new IllegalArgumentException("Zookeeper connection string is null or empty");
+        }
+        log.info("Using metadata url: " + context.getConfig());
+
+        this.engineID = engineID;
+        this.engineConfig = context;
+        this.scheduler = new QuatzScheduler();
+
+        RetryPolicy retryPolicy = new ExponentialBackoffRetry(1000, 3);
+        this.zkClient = CuratorFrameworkFactory.newClient(context.getZookeeperString(), retryPolicy);
+        this.zkClient.start();
+
+        Runtime.getRuntime().addShutdownHook(new Thread() {
+            public void run() {
+                log.debug("Closing HBASE connection");
+                releaseLock();
+            }
+        });
+    }
+
+    private void releaseLock() {
+        try {
+            if (sharedLock != null && sharedLock.isAcquiredInThisProcess()) {
+                sharedLock.release();
+            }
+            if (zkClient.getState().equals(CuratorFrameworkState.STARTED)) {
+                // client.setData().forPath(ZOOKEEPER_LOCK_PATH, null);
+                if (zkClient.checkExists().forPath(ZOOKEEPER_LOCK_PATH + "/" + this.engineID) != null) {
+                    zkClient.delete().guaranteed().deletingChildrenIfNeeded().forPath(ZOOKEEPER_LOCK_PATH + "/" + this.engineID);
+                }
+            }
+        } catch (Exception e) {
+            throw new RuntimeException(e);
+        }
+    }
+
+    public void start(int daemonJobIntervalInSeconds) throws Exception {
+        this.daemonJobIntervalInSeconds = daemonJobIntervalInSeconds;
+
+        sharedLock = new InterProcessMutex(zkClient, ZOOKEEPER_LOCK_PATH + "/" + this.engineID);
+        log.info("Trying to obtain the shared lock...");
+        // current thread will be blocked until the lock is got
+        sharedLock.acquire();
+
+        log.info("Obtained the shared lock. Starting job scheduler...");
+        zkClient.setData().forPath(ZOOKEEPER_LOCK_PATH + "/" + this.engineID, Bytes.toBytes(this.engineID));
+        startScheduler();
+    }
+
+    private void startScheduler() throws JobException, IOException {
+        String logDir = KylinConfig.getInstanceFromEnv().getKylinJobLogDir();
+        new File(logDir).mkdirs();
+
+        log.info("Starting scheduler.");
+        this.scheduler.start();
+        this.scheduler.scheduleFetcher(this.daemonJobIntervalInSeconds, this.engineConfig);
+    }
+
+    public void start() throws Exception {
+        start(JobConstants.DEFAULT_SCHEDULER_INTERVAL_SECONDS);
+    }
+
+    public void stop() throws JobException {
+        releaseLock();
+        this.scheduler.stop();
+    }
+
+    @Override
+    public void stateChanged(CuratorFramework client, ConnectionState newState) {
+        if ((newState == ConnectionState.SUSPENDED) || (newState == ConnectionState.LOST)) {
+            releaseLock();
+        }
+    }
+
+    public void interruptJob(JobInstance jobInstance, JobStep jobStep) throws IOException, JobException {
+        // kill the running step
+        this.scheduler.interrupt(jobInstance, jobStep);
+    }
+
+    public Scheduler getScheduler() {
+        return this.scheduler.getScheduler();
+    }
+
+    // Job engine metrics related methods
+
+    // <StepID, Duration Seconds>
+    public static ConcurrentHashMap<String, Double> JOB_DURATION = new ConcurrentHashMap<String, Double>();
+
+    public int getNumberOfJobStepsExecuted() {
+        return JOB_DURATION.values().size();
+    }
+
+    public String getPrimaryEngineID() throws Exception {
+        byte[] data = zkClient.getData().forPath(ZOOKEEPER_LOCK_PATH + "/" + this.engineID);
+        if (data == null) {
+            return "";
+        } else {
+            return Bytes.toString(data);
+        }
+    }
+
+    public double getMinJobStepDuration() {
+        double[] all = getJobStepDuration();
+        Arrays.sort(all);
+
+        if (all.length > 0) {
+            return all[0];
+        } else {
+            return 0;
+        }
+    }
+
+    private double[] getJobStepDuration() {
+        Collection<Double> values = JOB_DURATION.values();
+        Double[] all = (Double[]) values.toArray(new Double[values.size()]);
+        return ArrayUtils.toPrimitive(all);
+    }
+
+    public double getMaxJobStepDuration() {
+        double[] all = getJobStepDuration();
+        Arrays.sort(all);
+
+        if (all.length > 1) {
+            return all[all.length - 1];
+        } else {
+            return 0;
+        }
+    }
+
+    public double getPercentileJobStepDuration(double percentile) {
+        Collection<Double> values = JOB_DURATION.values();
+        Double[] all = (Double[]) values.toArray(new Double[values.size()]);
+        Percentile p = new Percentile(percentile);
+        return p.evaluate(ArrayUtils.toPrimitive(all));
+    }
+
+    public Integer getScheduledJobsSzie() {
+        return scheduler.getScheduledJobs();
+    }
+
+    public int getEngineThreadPoolSize() {
+        return scheduler.getThreadPoolSize();
+    }
+
+    public int getNumberOfIdleSlots() {
+        return scheduler.getIdleSlots();
+    }
+
+    public int getNumberOfJobStepsRunning() {
+        return scheduler.getRunningJobs();
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/job/src/main/java/com/kylinolap/job/engine/JobEngineConfig.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/com/kylinolap/job/engine/JobEngineConfig.java b/job/src/main/java/com/kylinolap/job/engine/JobEngineConfig.java
new file mode 100644
index 0000000..5527cf6
--- /dev/null
+++ b/job/src/main/java/com/kylinolap/job/engine/JobEngineConfig.java
@@ -0,0 +1,254 @@
+/*
+ * 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.job.engine;
+
+import java.io.File;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+
+import org.apache.commons.lang.StringUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.kylinolap.common.KylinConfig;
+import com.kylinolap.job.tools.OptionsHelper;
+import com.kylinolap.metadata.model.cube.CubeDesc.CubeCapacity;
+
+/**
+ * @author ysong1
+ */
+public class JobEngineConfig {
+    private static final Logger logger = LoggerFactory.getLogger(JobEngineConfig.class);
+    public static String HADOOP_JOB_CONF_FILENAME = "kylin_job_conf";
+
+    private String getHadoopJobConfFilePath(CubeCapacity capaticy, boolean appendSuffix) throws IOException {
+        String hadoopJobConfFile;
+        if (appendSuffix)
+            hadoopJobConfFile = (HADOOP_JOB_CONF_FILENAME + "_" + capaticy.toString().toLowerCase() + ".xml");
+        else
+            hadoopJobConfFile = (HADOOP_JOB_CONF_FILENAME + ".xml");
+
+        String path = System.getProperty(KylinConfig.KYLIN_CONF);
+
+        if (path == null) {
+            path = System.getenv(KylinConfig.KYLIN_CONF);
+        }
+
+        if (path != null) {
+            path = path + File.separator + hadoopJobConfFile;
+        }
+
+        if (null == path || !new File(path).exists()) {
+            File defaultFilePath = new File("/etc/kylin/" + hadoopJobConfFile);
+
+            if (defaultFilePath.exists()) {
+                path = defaultFilePath.getAbsolutePath();
+            } else {
+                logger.debug("Search conf file " + hadoopJobConfFile + "  from classpath ...");
+                InputStream is = JobEngineConfig.class.getClassLoader().getResourceAsStream(hadoopJobConfFile);
+                if (is == null) {
+                    logger.debug("Can't get " + hadoopJobConfFile + " from classpath");
+                    logger.debug("No " + hadoopJobConfFile + " file were found");
+                } else {
+                    File tmp = File.createTempFile(HADOOP_JOB_CONF_FILENAME, ".xml");
+                    inputStreamToFile(is, tmp);
+                    path = tmp.getAbsolutePath();
+                }
+            }
+        }
+
+        if (null == path || !new File(path).exists()) {
+            return "";
+        }
+
+        return OptionsHelper.convertToFileURL(path);
+    }
+
+    public String getHadoopJobConfFilePath(CubeCapacity capaticy) throws IOException {
+        String path = getHadoopJobConfFilePath(capaticy, true);
+        if (!StringUtils.isEmpty(path)) {
+            logger.info("Chosen job conf is : " + path);
+            return path;
+        } else {
+            path = getHadoopJobConfFilePath(capaticy, false);
+            if (!StringUtils.isEmpty(path)) {
+                logger.info("Chosen job conf is : " + path);
+                return path;
+            }
+        }
+        return "";
+    }
+
+    private void inputStreamToFile(InputStream ins, File file) throws IOException {
+        OutputStream os = new FileOutputStream(file);
+        int bytesRead = 0;
+        byte[] buffer = new byte[8192];
+        while ((bytesRead = ins.read(buffer, 0, 8192)) != -1) {
+            os.write(buffer, 0, bytesRead);
+        }
+        os.close();
+        ins.close();
+    }
+
+    // there should be no setters
+    private final KylinConfig config;
+
+    public JobEngineConfig(KylinConfig config) {
+        this.config = config;
+    }
+
+    public KylinConfig getConfig() {
+        return config;
+    }
+
+    public String getHdfsWorkingDirectory() {
+        return config.getHdfsWorkingDirectory();
+    }
+
+    /**
+     * @return the kylinJobJarPath
+     */
+    public String getKylinJobJarPath() {
+        return config.getKylinJobJarPath();
+    }
+
+    /**
+     * @return the runAsRemoteCommand
+     */
+    public boolean isRunAsRemoteCommand() {
+        return config.getRunAsRemoteCommand();
+    }
+
+    /**
+     * @return the zookeeperString
+     */
+    public String getZookeeperString() {
+        return config.getZookeeperString();
+    }
+
+    /**
+     * @return the remoteHadoopCliHostname
+     */
+    public String getRemoteHadoopCliHostname() {
+        return config.getRemoteHadoopCliHostname();
+    }
+
+    /**
+     * @return the remoteHadoopCliUsername
+     */
+    public String getRemoteHadoopCliUsername() {
+        return config.getRemoteHadoopCliUsername();
+    }
+
+    /**
+     * @return the remoteHadoopCliPassword
+     */
+    public String getRemoteHadoopCliPassword() {
+        return config.getRemoteHadoopCliPassword();
+    }
+    
+    public String getMapReduceCmdExtraArgs() {
+        return config.getMapReduceCmdExtraArgs();
+    }
+
+    /**
+     * @return the yarnStatusServiceUrl
+     */
+    public String getYarnStatusServiceUrl() {
+        return config.getYarnStatusServiceUrl();
+    }
+
+    /**
+     * @return the maxConcurrentJobLimit
+     */
+    public int getMaxConcurrentJobLimit() {
+        return config.getMaxConcurrentJobLimit();
+    }
+
+    /**
+     * @return the timeZone
+     */
+    public String getTimeZone() {
+        return config.getTimeZone();
+    }
+
+    /**
+     * @return the adminDls
+     */
+    public String getAdminDls() {
+        return config.getAdminDls();
+    }
+
+    /**
+     * @return the jobStepTimeout
+     */
+    public long getJobStepTimeout() {
+        return config.getJobStepTimeout();
+    }
+
+    /**
+     * @return the asyncJobCheckInterval
+     */
+    public int getAsyncJobCheckInterval() {
+        return config.getYarnStatusCheckIntervalSeconds();
+    }
+
+    /**
+     * @return the flatTableByHive
+     */
+    public boolean isFlatTableByHive() {
+        return config.getFlatTableByHive();
+    }
+
+    /*
+     * (non-Javadoc)
+     * 
+     * @see java.lang.Object#hashCode()
+     */
+    @Override
+    public int hashCode() {
+        final int prime = 31;
+        int result = 1;
+        result = prime * result + ((config == null) ? 0 : config.hashCode());
+        return result;
+    }
+
+    /*
+     * (non-Javadoc)
+     * 
+     * @see java.lang.Object#equals(java.lang.Object)
+     */
+    @Override
+    public boolean equals(Object obj) {
+        if (this == obj)
+            return true;
+        if (obj == null)
+            return false;
+        if (getClass() != obj.getClass())
+            return false;
+        JobEngineConfig other = (JobEngineConfig) obj;
+        if (config == null) {
+            if (other.config != null)
+                return false;
+        } else if (!config.equals(other.config))
+            return false;
+        return true;
+    }
+
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/job/src/main/java/com/kylinolap/job/engine/JobFetcher.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/com/kylinolap/job/engine/JobFetcher.java b/job/src/main/java/com/kylinolap/job/engine/JobFetcher.java
new file mode 100644
index 0000000..2389f27
--- /dev/null
+++ b/job/src/main/java/com/kylinolap/job/engine/JobFetcher.java
@@ -0,0 +1,117 @@
+/*
+ * 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.job.engine;
+
+import java.util.List;
+import java.util.Random;
+import java.util.concurrent.ConcurrentHashMap;
+
+import org.quartz.Job;
+import org.quartz.JobDetail;
+import org.quartz.JobExecutionContext;
+import org.quartz.JobExecutionException;
+import org.quartz.Trigger;
+import org.quartz.TriggerBuilder;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.kylinolap.common.util.StringSplitter;
+import com.kylinolap.job.JobDAO;
+import com.kylinolap.job.JobInstance;
+import com.kylinolap.job.constant.JobConstants;
+import com.kylinolap.job.constant.JobStatusEnum;
+import com.kylinolap.job.flow.JobFlow;
+
+/**
+ * @author ysong1, xduo
+ * 
+ */
+public class JobFetcher implements Job {
+
+    private static final Logger log = LoggerFactory.getLogger(JobFetcher.class);
+
+    public static final int JOB_THRESHOLD = 10;
+
+    @Override
+    public void execute(JobExecutionContext context) throws JobExecutionException {
+
+        JobEngineConfig engineConfig = (JobEngineConfig) context.getJobDetail().getJobDataMap().get(JobConstants.PROP_ENGINE_CONTEXT);
+
+        JobDAO jobDAO = JobDAO.getInstance(engineConfig.getConfig());
+
+        try {
+            // get all pending jobs
+            List<JobInstance> pendingJobList = jobDAO.listAllJobs(JobStatusEnum.PENDING);
+
+            log.debug(pendingJobList.size() + " pending jobs");
+            int leftJobs = JOB_THRESHOLD;
+            Random rand = new Random();
+            int maxConcurrentJobCount = engineConfig.getMaxConcurrentJobLimit();
+
+            for (JobInstance jobInstance : pendingJobList) {
+                @SuppressWarnings("unchecked")
+                ConcurrentHashMap<String, JobFlow> jobFlows = (ConcurrentHashMap<String, JobFlow>) context.getScheduler().getContext().get(JobConstants.PROP_JOB_RUNTIME_FLOWS);
+
+                if (jobFlows.size() >= maxConcurrentJobCount) {
+                    // If too many job instances in current job context, just
+                    // wait.
+                    break;
+                }
+
+                try {
+                    // there should be only 1 job for a certain job running
+                    boolean cubeHasRunningJob = false;
+                    for (String s : jobFlows.keySet()) {
+                        String[] tmp = StringSplitter.split(s, ".");
+                        String cubename = tmp[0];
+                        String jobid = tmp[1];
+                        if (cubename.equals(jobInstance.getRelatedCube())) {
+                            log.info("There is already a job of cube " + jobInstance.getRelatedCube() + " running, job uuid is " + jobid);
+                            cubeHasRunningJob = true;
+                            break;
+                        }
+                    }
+
+                    if (cubeHasRunningJob == false && jobFlows.containsKey(JobInstance.getJobIdentity(jobInstance)) == false) {
+                        // create job flow
+                        JobFlow jobFlow = new JobFlow(jobInstance, engineConfig);
+                        jobFlows.put(JobInstance.getJobIdentity(jobInstance), jobFlow);
+
+                        // schedule the 1st step
+                        Trigger trigger = TriggerBuilder.newTrigger().startNow().build();
+                        JobDetail firstStep = jobFlow.getFirst();
+                        context.getScheduler().scheduleJob(firstStep, trigger);
+
+                        log.info("Job " + jobInstance.getUuid() + " has been scheduled with the first step " + firstStep.getKey().toString());
+                    }
+                } catch (Exception e) {
+                    log.error("Failed to trigger the job detail", e);
+                }
+
+                if (--leftJobs < 0) {
+                    log.info("Too many pending jobs!");
+                    break;
+                }
+                long ms = Math.abs(rand.nextLong() % 10L);
+                Thread.sleep(ms * 1000L);
+            }
+        } catch (Throwable t) {
+            log.error(t.getMessage());
+            throw new JobExecutionException(t);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/job/src/main/java/com/kylinolap/job/engine/QuatzScheduler.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/com/kylinolap/job/engine/QuatzScheduler.java b/job/src/main/java/com/kylinolap/job/engine/QuatzScheduler.java
new file mode 100644
index 0000000..f080c37
--- /dev/null
+++ b/job/src/main/java/com/kylinolap/job/engine/QuatzScheduler.java
@@ -0,0 +1,193 @@
+/*
+ * 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.job.engine;
+
+import java.io.IOException;
+import java.util.Properties;
+import java.util.concurrent.ConcurrentHashMap;
+
+import org.quartz.JobBuilder;
+import org.quartz.JobDetail;
+import org.quartz.JobKey;
+import org.quartz.Scheduler;
+import org.quartz.SchedulerException;
+import org.quartz.SimpleScheduleBuilder;
+import org.quartz.Trigger;
+import org.quartz.TriggerBuilder;
+import org.quartz.UnableToInterruptJobException;
+import org.quartz.impl.StdSchedulerFactory;
+import org.quartz.impl.matchers.GroupMatcher;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.kylinolap.job.JobInstance;
+import com.kylinolap.job.JobInstance.JobStep;
+import com.kylinolap.job.cmd.IJobCommand;
+import com.kylinolap.job.constant.JobConstants;
+import com.kylinolap.job.exception.JobException;
+import com.kylinolap.job.flow.JobFlow;
+import com.kylinolap.job.flow.JobFlowListener;
+
+/**
+ * @author xduo
+ * 
+ */
+public class QuatzScheduler {
+
+    private static Logger log = LoggerFactory.getLogger(QuatzScheduler.class);
+
+    private Scheduler scheduler;
+    private JobFlowListener globalJobListener;
+
+    // public static void scheduleJobFlow(Scheduler scheduler, JobFlow jobFlow)
+    // throws JobException {
+    // // schedule the 1st step
+    // Trigger trigger = TriggerBuilder.newTrigger().startNow().build();
+    // JobDetail firstStep = jobFlow.getFirst();
+    // try {
+    // scheduler.scheduleJob(firstStep, trigger);
+    // } catch (SchedulerException e) {
+    // throw new JobException(e);
+    // }
+    // }
+
+    public QuatzScheduler() throws JobException {
+        this.globalJobListener = new JobFlowListener(JobConstants.GLOBAL_LISTENER_NAME);
+        StdSchedulerFactory sf = new StdSchedulerFactory();
+        Properties schedulerProperties = new Properties();
+        int numberOfProcessors = Runtime.getRuntime().availableProcessors();
+        schedulerProperties.setProperty("org.quartz.threadPool.threadCount", String.valueOf(numberOfProcessors));
+        schedulerProperties.setProperty("org.quartz.scheduler.skipUpdateCheck", "true");
+
+        try {
+            sf.initialize(schedulerProperties);
+            this.scheduler = sf.getScheduler();
+            this.scheduler.getListenerManager().addJobListener(this.globalJobListener, GroupMatcher.jobGroupEquals(JobConstants.CUBE_JOB_GROUP_NAME));
+
+            // cubename.jobUuid -> job flow
+            this.scheduler.getContext().put(JobConstants.PROP_JOB_RUNTIME_FLOWS, new ConcurrentHashMap<String, JobFlow>());
+
+            // put the scheduler in standby mode first
+            this.scheduler.standby();
+        } catch (SchedulerException e) {
+            throw new JobException(e);
+        }
+    }
+
+    public void start() throws JobException {
+        try {
+            this.scheduler.start();
+        } catch (SchedulerException e) {
+            throw new JobException(e);
+        }
+    }
+
+    public void scheduleFetcher(int intervalInSeconds, JobEngineConfig engineConfig) throws JobException {
+        JobDetail job = JobBuilder.newJob(JobFetcher.class).withIdentity(JobFetcher.class.getCanonicalName(), JobConstants.DAEMON_JOB_GROUP_NAME).build();
+        job.getJobDataMap().put(JobConstants.PROP_ENGINE_CONTEXT, engineConfig);
+
+        Trigger trigger = TriggerBuilder.newTrigger().startNow().withSchedule(SimpleScheduleBuilder.simpleSchedule().withIntervalInSeconds(intervalInSeconds).repeatForever()).build();
+
+        try {
+            this.scheduler.scheduleJob(job, trigger);
+        } catch (SchedulerException e) {
+            throw new JobException(e);
+        }
+    }
+
+    public boolean interrupt(JobInstance jobInstance, JobStep jobStep) throws JobException, IOException {
+        JobKey jobKey = new JobKey(JobInstance.getStepIdentity(jobInstance, jobStep), JobConstants.CUBE_JOB_GROUP_NAME);
+
+        boolean res = false;
+        try {
+            JobDetail jobDetail = this.scheduler.getJobDetail(jobKey);
+
+            IJobCommand iJobStepCmd = (IJobCommand) jobDetail.getJobDataMap().get(JobConstants.PROP_JOB_CMD_EXECUTOR);
+            if (null != iJobStepCmd) {
+                iJobStepCmd.cancel();
+            }
+
+            jobDetail.getJobDataMap().put(JobConstants.PROP_JOB_KILLED, true);
+            this.scheduler.addJob(jobDetail, true, true);
+
+            @SuppressWarnings("unchecked")
+            ConcurrentHashMap<String, JobFlow> jobFlows = (ConcurrentHashMap<String, JobFlow>) this.scheduler.getContext().get(JobConstants.PROP_JOB_RUNTIME_FLOWS);
+            jobFlows.remove(JobInstance.getJobIdentity(jobInstance));
+        } catch (UnableToInterruptJobException e) {
+            log.error(e.getLocalizedMessage(), e);
+            throw new JobException(e);
+        } catch (SchedulerException e) {
+            log.error(e.getLocalizedMessage(), e);
+            throw new JobException(e);
+        }
+        return res;
+    }
+
+    public void stop() throws JobException {
+        try {
+            this.scheduler.standby();
+        } catch (SchedulerException e) {
+            throw new JobException(e);
+        }
+    }
+
+    public Scheduler getScheduler() {
+        return this.scheduler;
+    }
+
+    // // metrics
+
+    public int getThreadPoolSize() {
+        try {
+            return scheduler.getMetaData().getThreadPoolSize();
+        } catch (SchedulerException e) {
+            log.error("Can't get scheduler metadata!", e);
+            return 0;
+        }
+    }
+
+    public int getRunningJobs() {
+        try {
+            return this.scheduler.getCurrentlyExecutingJobs().size();
+        } catch (SchedulerException e) {
+            log.error("Can't get scheduler metadata!", e);
+            return 0;
+        }
+    }
+
+    public int getIdleSlots() {
+        try {
+            return this.scheduler.getMetaData().getThreadPoolSize() - this.scheduler.getCurrentlyExecutingJobs().size();
+        } catch (SchedulerException e) {
+            log.error("Can't get scheduler metadata!", e);
+            return 0;
+        }
+    }
+
+    public int getScheduledJobs() {
+        int allTriggersCount = 0;
+        try {
+            for (String groupName : scheduler.getJobGroupNames()) {
+                allTriggersCount += scheduler.getJobKeys(GroupMatcher.jobGroupEquals(groupName)).size();
+            }
+        } catch (SchedulerException e) {
+            log.error("Can't get scheduler metadata!", e);
+        }
+        return allTriggersCount;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/job/src/main/java/com/kylinolap/job/exception/InvalidJobInstanceException.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/com/kylinolap/job/exception/InvalidJobInstanceException.java b/job/src/main/java/com/kylinolap/job/exception/InvalidJobInstanceException.java
new file mode 100644
index 0000000..1b404cb
--- /dev/null
+++ b/job/src/main/java/com/kylinolap/job/exception/InvalidJobInstanceException.java
@@ -0,0 +1,30 @@
+/*
+ * 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.job.exception;
+
+/**
+ * @author ysong1
+ * 
+ */
+public class InvalidJobInstanceException extends Exception {
+
+    private static final long serialVersionUID = 2045169570038227895L;
+
+    public InvalidJobInstanceException(String message) {
+        super(message);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/job/src/main/java/com/kylinolap/job/exception/InvalidJobStatusException.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/com/kylinolap/job/exception/InvalidJobStatusException.java b/job/src/main/java/com/kylinolap/job/exception/InvalidJobStatusException.java
new file mode 100644
index 0000000..d1050d0
--- /dev/null
+++ b/job/src/main/java/com/kylinolap/job/exception/InvalidJobStatusException.java
@@ -0,0 +1,33 @@
+/*
+ * 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.job.exception;
+
+/**
+ * @author xduo
+ * 
+ */
+public class InvalidJobStatusException extends Exception {
+
+    /**
+     * @param string
+     */
+    public InvalidJobStatusException(String string) {
+    }
+
+    private static final long serialVersionUID = -8549756520626114000L;
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/job/src/main/java/com/kylinolap/job/exception/JobException.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/com/kylinolap/job/exception/JobException.java b/job/src/main/java/com/kylinolap/job/exception/JobException.java
new file mode 100644
index 0000000..6c4bc6a
--- /dev/null
+++ b/job/src/main/java/com/kylinolap/job/exception/JobException.java
@@ -0,0 +1,56 @@
+/*
+ * 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.job.exception;
+
+/**
+ * @author xduo
+ * 
+ */
+public class JobException extends Exception {
+
+    private static final long serialVersionUID = 1L;
+
+    /**
+     * 
+     */
+    public JobException() {
+        super();
+    }
+
+    /**
+     * @param message
+     * @param cause
+     */
+    public JobException(String message, Throwable cause) {
+        super(message, cause);
+    }
+
+    /**
+     * @param message
+     */
+    public JobException(String message) {
+        super(message);
+    }
+
+    /**
+     * @param cause
+     */
+    public JobException(Throwable cause) {
+        super(cause);
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/job/src/main/java/com/kylinolap/job/flow/AsyncJobFlowNode.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/com/kylinolap/job/flow/AsyncJobFlowNode.java b/job/src/main/java/com/kylinolap/job/flow/AsyncJobFlowNode.java
new file mode 100644
index 0000000..1b8777e
--- /dev/null
+++ b/job/src/main/java/com/kylinolap/job/flow/AsyncJobFlowNode.java
@@ -0,0 +1,119 @@
+/*
+ * 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.job.flow;
+
+import java.util.HashSet;
+import java.util.Set;
+
+import org.quartz.DateBuilder;
+import org.quartz.DateBuilder.IntervalUnit;
+import org.quartz.JobDataMap;
+import org.quartz.JobExecutionContext;
+import org.quartz.JobExecutionException;
+import org.quartz.SchedulerException;
+import org.quartz.Trigger;
+import org.quartz.TriggerBuilder;
+
+import com.kylinolap.common.KylinConfig;
+import com.kylinolap.job.JobDAO;
+import com.kylinolap.job.JobInstance;
+import com.kylinolap.job.JobInstance.JobStep;
+import com.kylinolap.job.cmd.ICommandOutput;
+import com.kylinolap.job.cmd.JobCommandFactory;
+import com.kylinolap.job.constant.JobConstants;
+import com.kylinolap.job.constant.JobStepStatusEnum;
+import com.kylinolap.job.engine.JobEngineConfig;
+import com.kylinolap.job.exception.JobException;
+
+/**
+ * @author xduo
+ * 
+ */
+public class AsyncJobFlowNode extends JobFlowNode {
+
+    @Override
+    public void execute(JobExecutionContext context) throws JobExecutionException {
+        this.currentJobDetail = context.getJobDetail();
+        JobDataMap data = this.currentJobDetail.getJobDataMap();
+        JobFlow jobFlow = (JobFlow) data.get(JobConstants.PROP_JOB_FLOW);
+        JobEngineConfig engineConfig = jobFlow.getJobengineConfig();
+        KylinConfig config = engineConfig.getConfig();
+        String jobInstanceID = data.getString(JobConstants.PROP_JOBINSTANCE_UUID);
+        int jobStepID = data.getInt(JobConstants.PROP_JOBSTEP_SEQ_ID);
+        ICommandOutput output = (ICommandOutput) data.get(JobConstants.PROP_JOB_CMD_OUTPUT);
+
+        try {
+            if (data.getBoolean(JobConstants.PROP_JOB_KILLED)) {
+                log.info(this.currentJobDetail.getKey() + " is killed");
+                return;
+            }
+
+            if (output == null) {
+                JobInstance jobInstance = updateJobStep(jobInstanceID, jobStepID, config, JobStepStatusEnum.RUNNING, System.currentTimeMillis(), null, null);
+
+                String command = data.getString(JobConstants.PROP_COMMAND);
+                jobCmd = JobCommandFactory.getJobCommand(command, jobInstance, jobStepID, engineConfig);
+                output = jobCmd.execute();
+                data.put(JobConstants.PROP_JOB_CMD_OUTPUT, output);
+                data.put(JobConstants.PROP_JOB_CMD_EXECUTOR, jobCmd);
+                context.getScheduler().addJob(this.currentJobDetail, true, true);
+
+                JobStepStatusEnum stepStatus = output.getStatus();
+                updateJobStep(jobInstanceID, jobStepID, config, stepStatus, null, stepStatus.isComplete() ? System.currentTimeMillis() : null, output.getOutput());
+
+                context.setResult(output.getExitCode());
+                scheduleStatusChecker(context);
+                log.debug("Start async job " + currentJobDetail.getKey());
+            } else {
+                JobInstance jobInstance = JobDAO.getInstance(engineConfig.getConfig()).getJob(jobInstanceID);
+                JobStep jobStep = jobInstance.getSteps().get(jobStepID);
+
+                log.debug("Start to check hadoop job status of " + currentJobDetail.getKey());
+                JobStepStatusEnum stepStatus = output.getStatus();
+
+                if ((System.currentTimeMillis() - jobStep.getExecStartTime()) / 1000 >= engineConfig.getJobStepTimeout()) {
+                    throw new JobException("Job step " + jobStep.getName() + " timeout.");
+                }
+
+                updateJobStep(jobInstance.getUuid(), jobStepID, config, stepStatus, null, stepStatus.isComplete() ? System.currentTimeMillis() : null, output.getOutput());
+
+                if (!stepStatus.isComplete()) {
+                    scheduleStatusChecker(context);
+                }
+
+                context.setResult(0);
+                log.debug("Status of async job " + currentJobDetail.getKey() + ":" + stepStatus);
+            }
+        } catch (Throwable t) {
+            handleException(jobInstanceID, jobStepID, config, t);
+        }
+
+    }
+
+    private void scheduleStatusChecker(JobExecutionContext context) throws SchedulerException {
+        JobDataMap jobDataMap = this.currentJobDetail.getJobDataMap();
+        JobFlow jobFlow = (JobFlow) jobDataMap.get(JobConstants.PROP_JOB_FLOW);
+        JobEngineConfig engineConfig = jobFlow.getJobengineConfig();
+        int interval = engineConfig.getAsyncJobCheckInterval();
+        log.debug("Trigger a status check job in " + interval + " seconds for job " + currentJobDetail.getKey());
+
+        Trigger trigger = TriggerBuilder.newTrigger().startAt(DateBuilder.futureDate(interval, IntervalUnit.SECOND)).build();
+        Set<Trigger> triggers = new HashSet<Trigger>();
+        triggers.add(trigger);
+        context.getScheduler().scheduleJob(currentJobDetail, triggers, true);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/job/src/main/java/com/kylinolap/job/flow/JobFlow.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/com/kylinolap/job/flow/JobFlow.java b/job/src/main/java/com/kylinolap/job/flow/JobFlow.java
new file mode 100644
index 0000000..9a58107
--- /dev/null
+++ b/job/src/main/java/com/kylinolap/job/flow/JobFlow.java
@@ -0,0 +1,159 @@
+/*
+ * 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.job.flow;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.Collections;
+import java.util.LinkedList;
+import java.util.List;
+
+import org.apache.commons.io.FileUtils;
+import org.apache.commons.lang3.StringUtils;
+import org.quartz.JobBuilder;
+import org.quartz.JobDetail;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.kylinolap.common.KylinConfig;
+import com.kylinolap.job.JobInstance;
+import com.kylinolap.job.JobInstance.JobStep;
+import com.kylinolap.job.constant.JobConstants;
+import com.kylinolap.job.constant.JobStepCmdTypeEnum;
+import com.kylinolap.job.engine.JobEngineConfig;
+
+/**
+ * @author xduo
+ */
+public class JobFlow {
+
+    private static Logger log = LoggerFactory.getLogger(JobFlow.class);
+
+    private final List<JobDetail> flowNodes;
+    private final JobInstance jobInstance;
+    private final JobEngineConfig engineConfig;
+
+    public JobFlow(JobInstance job, JobEngineConfig context) {
+        this.engineConfig = context;
+        this.jobInstance = job;
+        // validate job instance
+        List<JobStep> sortedSteps = jobInstance.getSteps();
+
+        if (sortedSteps == null || sortedSteps.size() == 0) {
+            throw new IllegalStateException("Steps of job " + jobInstance.getUuid() + " is null or empty!");
+        }
+
+        // sort the steps by step_sequenceID
+        Collections.sort(sortedSteps);
+        // find the 1st runnable job
+        int firstStepIndex = findFirstStep(sortedSteps);
+
+        log.info("Job " + jobInstance.getUuid() + " will be started at step " + firstStepIndex + " (sequence number)");
+
+        flowNodes = new LinkedList<JobDetail>();
+        for (int i = firstStepIndex; i < sortedSteps.size(); i++) {
+            JobDetail node = createJobFlowNode(jobInstance, i);
+            flowNodes.add(node);
+        }
+    }
+
+    public JobInstance getJobInstance() {
+        return jobInstance;
+    }
+
+    public JobEngineConfig getJobengineConfig() {
+        return engineConfig;
+    }
+
+    public JobDetail getFirst() {
+        if (flowNodes.isEmpty()) {
+            return null;
+        }
+        return flowNodes.get(0);
+    }
+
+    public JobDetail getNext(JobDetail jobFlowNode) {
+        int targetIndex = -1;
+        for (int index = 0; index < flowNodes.size(); index++) {
+            if (flowNodes.get(index).equals(jobFlowNode)) {
+                targetIndex = index;
+            }
+        }
+
+        if (targetIndex != -1 && flowNodes.size() > targetIndex + 1) {
+            return flowNodes.get(targetIndex + 1);
+        }
+        return null;
+    }
+
+    private int findFirstStep(List<JobStep> stepList) {
+        int firstJobIndex = 0;
+        for (int i = 0; i < stepList.size(); i++) {
+            JobStep currentStep = stepList.get(i);
+            if (currentStep.getStatus().isRunable() == false) {
+                continue;
+            } else {
+                firstJobIndex = i;
+                break;
+            }
+        }
+        return firstJobIndex;
+    }
+
+    private JobDetail createJobFlowNode(final JobInstance jobInstance, final int stepSeqId) {
+        JobStep step = jobInstance.getSteps().get(stepSeqId);
+
+        if (jobInstance.getName() == null || step.getName() == null) {
+            throw new IllegalArgumentException("JobInstance name or JobStep name cannot be null!");
+        }
+
+        // submit job the different groups based on isRunAsync property
+        JobDetail jobFlowNode = JobBuilder.newJob(step.isRunAsync() ? AsyncJobFlowNode.class : JobFlowNode.class).withIdentity(JobInstance.getStepIdentity(jobInstance, step), JobConstants.CUBE_JOB_GROUP_NAME).storeDurably().build();
+
+        // add job flow to node
+        jobFlowNode.getJobDataMap().put(JobConstants.PROP_JOB_FLOW, this);
+
+        // add command to flow node
+        String execCmd = (step.getCmdType() == JobStepCmdTypeEnum.SHELL_CMD || step.getCmdType() == JobStepCmdTypeEnum.SHELL_CMD_HADOOP) ? wrapExecCmd(jobInstance, step.getExecCmd(), String.valueOf(step.getSequenceID())) : step.getExecCmd();
+        jobFlowNode.getJobDataMap().put(JobConstants.PROP_COMMAND, execCmd);
+
+        // add job instance and step sequenceID to flow node
+        jobFlowNode.getJobDataMap().put(JobConstants.PROP_JOBINSTANCE_UUID, jobInstance.getUuid());
+        jobFlowNode.getJobDataMap().put(JobConstants.PROP_JOBSTEP_SEQ_ID, step.getSequenceID());
+
+        // add async flag to flow node
+        jobFlowNode.getJobDataMap().put(JobConstants.PROP_JOB_ASYNC, step.isRunAsync());
+
+        return jobFlowNode;
+    }
+
+    private String wrapExecCmd(JobInstance job, String cmd, String suffix) {
+        if (StringUtils.isBlank(cmd))
+            return cmd;
+
+        KylinConfig config = KylinConfig.getInstanceFromEnv();
+        try {
+            FileUtils.forceMkdir(new File(config.getKylinJobLogDir()));
+        } catch (IOException e) {
+            throw new RuntimeException("Create log dir " + config.getKylinJobLogDir() + " failed.", e);
+        }
+        String log = config.getKylinJobLogDir() + "/" + job.getUuid() + "_" + suffix + ".log";
+
+        String mkLogDir = "mkdir -p " + config.getKylinJobLogDir();
+        return mkLogDir + ";" + "set -o pipefail; " + cmd + " 2>&1 | tee " + log;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/job/src/main/java/com/kylinolap/job/flow/JobFlowListener.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/com/kylinolap/job/flow/JobFlowListener.java b/job/src/main/java/com/kylinolap/job/flow/JobFlowListener.java
new file mode 100644
index 0000000..8274d44
--- /dev/null
+++ b/job/src/main/java/com/kylinolap/job/flow/JobFlowListener.java
@@ -0,0 +1,419 @@
+/*
+ * 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.job.flow;
+
+import java.io.IOException;
+import java.net.InetAddress;
+import java.net.UnknownHostException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Date;
+import java.util.List;
+import java.util.concurrent.ConcurrentHashMap;
+
+import com.kylinolap.cube.CubeSegmentStatusEnum;
+
+import org.apache.commons.lang.exception.ExceptionUtils;
+import org.quartz.JobDataMap;
+import org.quartz.JobDetail;
+import org.quartz.JobExecutionContext;
+import org.quartz.JobExecutionException;
+import org.quartz.JobListener;
+import org.quartz.SchedulerException;
+import org.quartz.Trigger;
+import org.quartz.TriggerBuilder;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.kylinolap.common.KylinConfig;
+import com.kylinolap.common.util.MailService;
+import com.kylinolap.cube.CubeInstance;
+import com.kylinolap.cube.CubeManager;
+import com.kylinolap.cube.CubeSegment;
+import com.kylinolap.cube.exception.CubeIntegrityException;
+import com.kylinolap.job.JobDAO;
+import com.kylinolap.job.JobInstance;
+import com.kylinolap.job.JobInstance.JobStep;
+import com.kylinolap.job.constant.JobConstants;
+import com.kylinolap.job.constant.JobStatusEnum;
+import com.kylinolap.job.constant.JobStepStatusEnum;
+import com.kylinolap.job.engine.JobEngineConfig;
+
+/**
+ * Handle kylin job and cube change update.
+ * 
+ * @author George Song (ysong1), xduo
+ * 
+ */
+public class JobFlowListener implements JobListener {
+
+    private static Logger log = LoggerFactory.getLogger(JobFlowListener.class);
+
+    private String name;
+
+    public JobFlowListener(String name) {
+        if (name == null) {
+            throw new IllegalArgumentException("Listener name cannot be null!");
+        }
+        this.name = name;
+    }
+
+    public String getName() {
+        return name;
+    }
+
+    @Override
+    public void jobWasExecuted(JobExecutionContext context, JobExecutionException jobException) {
+        log.info(context.getJobDetail().getKey() + " was executed.");
+        JobDataMap jobDataMap = context.getJobDetail().getJobDataMap();
+        JobFlow jobFlow = (JobFlow) jobDataMap.get(JobConstants.PROP_JOB_FLOW);
+        JobEngineConfig engineConfig = jobFlow.getJobengineConfig();
+        String jobUuid = jobDataMap.getString(JobConstants.PROP_JOBINSTANCE_UUID);
+        int stepSeqID = jobDataMap.getInt(JobConstants.PROP_JOBSTEP_SEQ_ID);
+        KylinConfig config = engineConfig.getConfig();
+
+        JobInstance jobInstance = null;
+        JobStep jobStep = null;
+        try {
+            jobInstance = JobDAO.getInstance(config).getJob(jobUuid);
+            jobStep = jobInstance.getSteps().get(stepSeqID);
+            CubeInstance cube = CubeManager.getInstance(config).getCube(jobInstance.getRelatedCube());
+
+            log.info(context.getJobDetail().getKey() + " status: " + jobStep.getStatus());
+            switch (jobStep.getStatus()) {
+            case FINISHED:
+                // Ensure we are using the latest metadata
+                CubeManager.getInstance(config).loadCubeCache(cube);
+                updateKylinJobOnSuccess(jobInstance, stepSeqID, engineConfig);
+                updateCubeSegmentInfoOnSucceed(jobInstance, engineConfig);
+                notifyUsers(jobInstance, engineConfig);
+                scheduleNextJob(context, jobInstance);
+                break;
+            case ERROR:
+                updateKylinJobStatus(jobInstance, stepSeqID, engineConfig);
+                notifyUsers(jobInstance, engineConfig);
+                break;
+            case DISCARDED:
+                // Ensure we are using the latest metadata
+                CubeManager.getInstance(config).loadCubeCache(cube);
+                updateCubeSegmentInfoOnDiscard(jobInstance, engineConfig);
+                notifyUsers(jobInstance, engineConfig);
+                break;
+            default:
+                break;
+            }
+        } catch (Exception e) {
+            log.error(e.getMessage(), e);
+            handleException(jobUuid, stepSeqID, config, e);
+        } finally {
+            if (null != jobInstance && jobInstance.getStatus().isComplete()) {
+                try {
+                    context.getScheduler().deleteJob(context.getJobDetail().getKey());
+                    @SuppressWarnings("unchecked")
+                    ConcurrentHashMap<String, JobFlow> jobFlows = (ConcurrentHashMap<String, JobFlow>) context.getScheduler().getContext().get(JobConstants.PROP_JOB_RUNTIME_FLOWS);
+                    jobFlows.remove(JobInstance.getJobIdentity(jobInstance));
+                } catch (SchedulerException e) {
+                    log.error(e.getMessage(), e);
+                }
+            }
+        }
+    }
+
+    /*
+     * (non-Javadoc)
+     * 
+     * @see
+     * org.quartz.JobListener#jobToBeExecuted(org.quartz.JobExecutionContext)
+     */
+    @Override
+    public void jobToBeExecuted(JobExecutionContext context) {
+    }
+
+    /*
+     * (non-Javadoc)
+     * 
+     * @see
+     * org.quartz.JobListener#jobExecutionVetoed(org.quartz.JobExecutionContext)
+     */
+    @Override
+    public void jobExecutionVetoed(JobExecutionContext context) {
+    }
+
+    /**
+     * @param context
+     * @param jobInstance
+     */
+    protected void scheduleNextJob(JobExecutionContext context, JobInstance jobInstance) {
+        try {
+            // schedule next job
+            JobDataMap jobDataMap = context.getJobDetail().getJobDataMap();
+            JobFlow jobFlow = (JobFlow) jobDataMap.get(JobConstants.PROP_JOB_FLOW);
+            JobDetail nextJob = (JobDetail) jobFlow.getNext(context.getJobDetail());
+            if (nextJob != null) {
+                try {
+                    Trigger trigger = TriggerBuilder.newTrigger().startNow().build();
+                    log.debug("Job " + context.getJobDetail().getKey() + " will now chain to Job " + nextJob.getKey() + "");
+
+                    context.getScheduler().scheduleJob(nextJob, trigger);
+
+                } catch (SchedulerException se) {
+                    log.error("Error encountered during chaining to Job " + nextJob.getKey() + "", se);
+                }
+            }
+
+            context.getScheduler().deleteJob(context.getJobDetail().getKey());
+        } catch (SchedulerException e) {
+            log.error(e.getLocalizedMessage(), e);
+            throw new RuntimeException(e);
+        }
+    }
+
+    /**
+     * @param jobInstance
+     * @param stepId
+     */
+    private void updateKylinJobStatus(JobInstance jobInstance, int stepId, JobEngineConfig engineConfig) {
+        validate(jobInstance);
+        List<JobStep> steps = jobInstance.getSteps();
+        Collections.sort(steps);
+
+        JobStep jobStep = jobInstance.getSteps().get(stepId);
+
+        long duration = jobStep.getExecEndTime() - jobStep.getExecStartTime();
+        jobInstance.setDuration(jobInstance.getDuration() + (duration > 0 ? duration : 0) / 1000);
+        jobInstance.setMrWaiting(jobInstance.getMrWaiting() + jobStep.getExecWaitTime());
+
+        try {
+            JobDAO.getInstance(engineConfig.getConfig()).updateJobInstance(jobInstance);
+        } catch (IOException e) {
+            e.printStackTrace();
+            log.error(e.getLocalizedMessage(), e);
+        }
+    }
+
+    private void updateKylinJobOnSuccess(JobInstance jobInstance, int stepId, JobEngineConfig engineConfig) {
+        validate(jobInstance);
+        List<JobStep> steps = jobInstance.getSteps();
+        Collections.sort(steps);
+
+        JobStep jobStep = jobInstance.getSteps().get(stepId);
+        jobInstance.setExecStartTime(steps.get(0).getExecStartTime());
+
+        long duration = jobStep.getExecEndTime() - jobStep.getExecStartTime();
+        jobInstance.setDuration(jobInstance.getDuration() + (duration > 0 ? duration / 1000 : 0));
+        jobInstance.setMrWaiting(jobInstance.getMrWaiting() + jobStep.getExecWaitTime());
+        if (jobInstance.getStatus().equals(JobStatusEnum.FINISHED)) {
+            jobInstance.setExecEndTime(steps.get(steps.size() - 1).getExecEndTime());
+        }
+
+        try {
+            JobDAO.getInstance(engineConfig.getConfig()).updateJobInstance(jobInstance);
+        } catch (IOException e) {
+            e.printStackTrace();
+            log.error(e.getLocalizedMessage(), e);
+        }
+    }
+
+    private void updateCubeSegmentInfoOnDiscard(JobInstance jobInstance, JobEngineConfig engineConfig) throws IOException, CubeIntegrityException {
+        CubeManager cubeMgr = CubeManager.getInstance(engineConfig.getConfig());
+        CubeInstance cubeInstance = cubeMgr.getCube(jobInstance.getRelatedCube());
+        cubeMgr.updateSegmentOnJobDiscard(cubeInstance, jobInstance.getRelatedSegment());
+    }
+
+    private void updateCubeSegmentInfoOnSucceed(JobInstance jobInstance, JobEngineConfig engineConfig) throws CubeIntegrityException, IOException {
+        if (jobInstance.getStatus().equals(JobStatusEnum.FINISHED)) {
+            validate(jobInstance);
+
+            log.info("Updating cube segment " + jobInstance.getRelatedSegment() + " for cube " + jobInstance.getRelatedCube());
+
+            long cubeSize = 0;
+            JobStep convertToHFileStep = jobInstance.findStep(JobConstants.STEP_NAME_CONVERT_CUBOID_TO_HFILE);
+            if (null != convertToHFileStep) {
+                String cubeSizeString = convertToHFileStep.getInfo(JobInstance.HDFS_BYTES_WRITTEN);
+                if (cubeSizeString == null || cubeSizeString.equals("")) {
+                    throw new RuntimeException("Can't get cube segment size.");
+                }
+                cubeSize = Long.parseLong(cubeSizeString) / 1024;
+            } else {
+                log.info("No step with name '" + JobConstants.STEP_NAME_CONVERT_CUBOID_TO_HFILE + "' is found");
+            }
+
+            CubeManager cubeMgr = CubeManager.getInstance(engineConfig.getConfig());
+            CubeInstance cubeInstance = cubeMgr.getCube(jobInstance.getRelatedCube());
+            CubeSegment newSegment = cubeInstance.getSegmentById(jobInstance.getUuid());
+
+            long sourceCount = 0;
+            long sourceSize = 0;
+            switch (jobInstance.getType()) {
+            case BUILD:
+                JobStep baseCuboidStep = jobInstance.findStep(JobConstants.STEP_NAME_BUILD_BASE_CUBOID);
+                if (null != baseCuboidStep) {
+                    String sourceRecordsCount = baseCuboidStep.getInfo(JobInstance.SOURCE_RECORDS_COUNT);
+                    if (sourceRecordsCount == null || sourceRecordsCount.equals("")) {
+                        throw new RuntimeException("Can't get cube source record count.");
+                    }
+                    sourceCount = Long.parseLong(sourceRecordsCount);
+                } else {
+                    log.info("No step with name '" + JobConstants.STEP_NAME_BUILD_BASE_CUBOID + "' is found");
+                }
+
+                JobStep createFlatTableStep = jobInstance.findStep(JobConstants.STEP_NAME_CREATE_FLAT_HIVE_TABLE);
+                if (null != createFlatTableStep) {
+                    String sourceRecordsSize = createFlatTableStep.getInfo(JobInstance.SOURCE_RECORDS_SIZE);
+                    if (sourceRecordsSize == null || sourceRecordsSize.equals("")) {
+                        throw new RuntimeException("Can't get cube source record size.");
+                    }
+                    sourceSize = Long.parseLong(sourceRecordsSize);
+                } else {
+                    log.info("No step with name '" + JobConstants.STEP_NAME_CREATE_FLAT_HIVE_TABLE + "' is found");
+                }
+
+                if (cubeInstance.needMergeImmediatelyAfterBuild(newSegment)) {
+                    for (CubeSegment seg : cubeInstance.getSegment(CubeSegmentStatusEnum.READY)) {
+                        sourceCount += seg.getSourceRecords();
+                        sourceSize += seg.getSourceRecordsSize();
+                    }
+                }
+                break;
+            case MERGE:
+                for (CubeSegment seg : cubeInstance.getMergingSegments()) {
+                    sourceCount += seg.getSourceRecords();
+                    sourceSize += seg.getSourceRecordsSize();
+                }
+                break;
+            }
+
+            cubeMgr.updateSegmentOnJobSucceed(cubeInstance, jobInstance.getType(), jobInstance.getRelatedSegment(), jobInstance.getUuid(), jobInstance.getExecEndTime(), cubeSize, sourceCount, sourceSize);
+            log.info("Update cube segment succeed" + jobInstance.getRelatedSegment() + " for cube " + jobInstance.getRelatedCube());
+        }
+    }
+
+    private void validate(JobInstance jobInstance) {
+        List<JobStep> steps = jobInstance.getSteps();
+        if (steps == null || steps.size() == 0) {
+            throw new RuntimeException("Steps of job " + jobInstance.getUuid() + " is null or empty!");
+        }
+    }
+
+    private void handleException(String jobInstanceUuid, int jobInstanceStepSeqId, KylinConfig config, Throwable t) {
+        log.error(t.getLocalizedMessage(), t);
+        String exceptionMsg = "Failed with Exception:" + ExceptionUtils.getFullStackTrace(t);
+        try {
+            JobInstance jobInstance = JobDAO.getInstance(config).getJob(jobInstanceUuid);
+            jobInstance.getSteps().get(jobInstanceStepSeqId).setStatus(JobStepStatusEnum.ERROR);
+            // String output =
+            // jobInstance.getSteps().get(jobInstanceStepSeqId).getCmdOutput();
+            // jobInstance.getSteps().get(jobInstanceStepSeqId).setCmdOutput(output
+            // + "\n" + exceptionMsg);
+            jobInstance.getSteps().get(jobInstanceStepSeqId).setExecEndTime(System.currentTimeMillis());
+            JobDAO.getInstance(config).updateJobInstance(jobInstance);
+
+            String output = JobDAO.getInstance(config).getJobOutput(jobInstanceUuid, jobInstanceStepSeqId).getOutput();
+            output = output + "\n" + exceptionMsg;
+            JobDAO.getInstance(config).saveJobOutput(jobInstanceUuid, jobInstanceStepSeqId, output);
+        } catch (IOException e1) {
+            log.error(e1.getLocalizedMessage(), e1);
+        }
+    }
+
+    /**
+     * @param jobInstance
+     */
+    protected void notifyUsers(JobInstance jobInstance, JobEngineConfig engineConfig) {
+        KylinConfig config = engineConfig.getConfig();
+        String cubeName = jobInstance.getRelatedCube();
+        CubeInstance cubeInstance = CubeManager.getInstance(config).getCube(cubeName);
+        String finalStatus = null;
+        String content = JobConstants.NOTIFY_EMAIL_TEMPLATE;
+        String logMsg = "";
+
+        switch (jobInstance.getStatus()) {
+        case FINISHED:
+            finalStatus = "SUCCESS";
+            break;
+        case ERROR:
+            for (JobStep step : jobInstance.getSteps()) {
+                if (step.getStatus() == JobStepStatusEnum.ERROR) {
+                    try {
+                        logMsg = JobDAO.getInstance(config).getJobOutput(step).getOutput();
+                    } catch (IOException e) {
+                        log.error(e.getLocalizedMessage(), e);
+                    }
+                }
+            }
+            finalStatus = "FAILED";
+            break;
+        case DISCARDED:
+            finalStatus = "DISCARDED";
+        default:
+            break;
+        }
+
+        if (null == finalStatus) {
+            return;
+        }
+
+        try {
+            InetAddress inetAddress = InetAddress.getLocalHost();
+            content = content.replaceAll("\\$\\{job_engine\\}", inetAddress.getCanonicalHostName());
+        } catch (UnknownHostException e) {
+            log.error(e.getLocalizedMessage(), e);
+        }
+
+        content = content.replaceAll("\\$\\{job_name\\}", jobInstance.getName());
+        content = content.replaceAll("\\$\\{result\\}", finalStatus);
+        content = content.replaceAll("\\$\\{cube_name\\}", cubeName);
+        content = content.replaceAll("\\$\\{start_time\\}", new Date(jobInstance.getExecStartTime()).toString());
+        content = content.replaceAll("\\$\\{duration\\}", jobInstance.getDuration() / 60 + "mins");
+        content = content.replaceAll("\\$\\{mr_waiting\\}", jobInstance.getMrWaiting() / 60 + "mins");
+        content = content.replaceAll("\\$\\{last_update_time\\}", new Date(jobInstance.getLastModified()).toString());
+        content = content.replaceAll("\\$\\{submitter\\}", jobInstance.getSubmitter());
+        content = content.replaceAll("\\$\\{error_log\\}", logMsg);
+
+        
+        MailService mailService = new MailService();
+        try {
+            List<String> users = new ArrayList<String>();
+
+            if (null != cubeInstance.getDescriptor().getNotifyList()) {
+                users.addAll(cubeInstance.getDescriptor().getNotifyList());
+            }
+
+            if (null != engineConfig.getAdminDls()) {
+                String[] adminDls = engineConfig.getAdminDls().split(",");
+
+                for (String adminDl : adminDls) {
+                    users.add(adminDl);
+                }
+            }
+
+            log.info("prepare to send email to:"+users);
+            
+            log.info("job name:"+jobInstance.getName());
+            
+            log.info("submitter:"+jobInstance.getSubmitter());
+            
+            if (users.size() > 0) {
+                log.info("notify list:"+users);
+                mailService.sendMail(users, "["+ finalStatus + "] - [Kylin Cube Build Job]-" + cubeName, content);
+                log.info("notified users:"+users);
+            }
+        } catch (IOException e) {
+            log.error(e.getLocalizedMessage(), e);
+        }
+
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/job/src/main/java/com/kylinolap/job/flow/JobFlowNode.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/com/kylinolap/job/flow/JobFlowNode.java b/job/src/main/java/com/kylinolap/job/flow/JobFlowNode.java
new file mode 100644
index 0000000..e539121
--- /dev/null
+++ b/job/src/main/java/com/kylinolap/job/flow/JobFlowNode.java
@@ -0,0 +1,165 @@
+/*
+ * 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.job.flow;
+
+import java.io.IOException;
+
+import org.apache.commons.lang.exception.ExceptionUtils;
+import org.quartz.InterruptableJob;
+import org.quartz.JobDataMap;
+import org.quartz.JobDetail;
+import org.quartz.JobExecutionContext;
+import org.quartz.JobExecutionException;
+import org.quartz.UnableToInterruptJobException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.kylinolap.common.KylinConfig;
+import com.kylinolap.job.JobDAO;
+import com.kylinolap.job.JobInstance;
+import com.kylinolap.job.JobInstance.JobStep;
+import com.kylinolap.job.cmd.ICommandOutput;
+import com.kylinolap.job.cmd.IJobCommand;
+import com.kylinolap.job.cmd.JobCommandFactory;
+import com.kylinolap.job.constant.JobConstants;
+import com.kylinolap.job.constant.JobStepStatusEnum;
+import com.kylinolap.job.engine.JobEngine;
+import com.kylinolap.job.engine.JobEngineConfig;
+
+/**
+ * @author xduo
+ * 
+ */
+public class JobFlowNode implements InterruptableJob {
+
+    protected static final Logger log = LoggerFactory.getLogger(JobFlowNode.class);
+
+    protected JobDetail currentJobDetail;
+    protected IJobCommand jobCmd;
+
+    /*
+     * (non-Javadoc)
+     * 
+     * @see org.quartz.Job#execute(org.quartz.JobExecutionContext)
+     */
+    @Override
+    public void execute(JobExecutionContext context) throws JobExecutionException {
+        this.currentJobDetail = context.getJobDetail();
+        JobDataMap data = this.currentJobDetail.getJobDataMap();
+        JobFlow jobFlow = (JobFlow) data.get(JobConstants.PROP_JOB_FLOW);
+        JobEngineConfig engineConfig = jobFlow.getJobengineConfig();
+        String jobInstanceID = data.getString(JobConstants.PROP_JOBINSTANCE_UUID);
+        int jobStepID = data.getInt(JobConstants.PROP_JOBSTEP_SEQ_ID);
+        String command = data.getString(JobConstants.PROP_COMMAND);
+        KylinConfig config = engineConfig.getConfig();
+
+        try {
+            JobInstance jobInstance = updateJobStep(jobInstanceID, jobStepID, config, JobStepStatusEnum.RUNNING, System.currentTimeMillis(), null, null);
+
+            jobCmd = JobCommandFactory.getJobCommand(command, jobInstance, jobStepID, engineConfig);
+            data.put(JobConstants.PROP_JOB_CMD_EXECUTOR, jobCmd);
+            context.getScheduler().addJob(this.currentJobDetail, true, true);
+
+            ICommandOutput output = jobCmd.execute();
+
+            if (data.getBoolean(JobConstants.PROP_JOB_KILLED)) {
+                return;
+            }
+
+            int exitCode = output.getExitCode();
+            updateJobStep(jobInstanceID, jobStepID, config, output.getStatus(), null, System.currentTimeMillis(), output.getOutput());
+            context.setResult(exitCode);
+
+            log.info("Job status for " + context.getJobDetail().getKey() + " has been updated.");
+            log.info("cmd:" + command);
+            log.info("output:" + output.getOutput());
+            log.info("exitCode:" + exitCode);
+
+        } catch (Throwable t) {
+            handleException(jobInstanceID, jobStepID, config, t);
+        }
+    }
+
+    @Override
+    public void interrupt() throws UnableToInterruptJobException {
+    }
+
+    protected JobInstance updateJobStep(String jobInstanceUuid, int jobInstanceStepSeqId, KylinConfig config, JobStepStatusEnum newStatus, Long execStartTime, Long execEndTime, String output) throws IOException {
+        // set step status to running
+        JobInstance jobInstance = JobDAO.getInstance(config).getJob(jobInstanceUuid);
+        JobStep currentStep = null;
+
+        try {
+            currentStep = jobInstance.getSteps().get(jobInstanceStepSeqId);
+            JobStepStatusEnum currentStatus = currentStep.getStatus();
+            boolean hasChange = false;
+
+            if (null != execStartTime) {
+                hasChange = true;
+                currentStep.setExecStartTime(execStartTime);
+            }
+            if (null != execEndTime) {
+                hasChange = true;
+                currentStep.setExecEndTime(execEndTime);
+            }
+            if (null != output) {
+                hasChange = true;
+                // currentStep.setCmdOutput(output);
+                JobDAO.getInstance(config).saveJobOutput(currentStep, output);
+            }
+            if (JobStepStatusEnum.WAITING == currentStatus && (JobStepStatusEnum.RUNNING == newStatus || JobStepStatusEnum.FINISHED == newStatus)) {
+                hasChange = true;
+                currentStep.setExecWaitTime((System.currentTimeMillis() - currentStep.getExecStartTime()) / 1000);
+            }
+            if (null != newStatus) {
+                hasChange = true;
+                currentStep.setStatus(newStatus);
+            }
+
+            if (hasChange) {
+                JobDAO.getInstance(config).updateJobInstance(jobInstance);
+            }
+        } catch (IOException e) {
+            log.error(e.getLocalizedMessage(), e);
+        }
+
+        if (null != execEndTime) {
+            JobEngine.JOB_DURATION.put(JobInstance.getStepIdentity(jobInstance, currentStep) + " - " + String.valueOf(currentStep.getExecStartTime()), (double) (currentStep.getExecEndTime() - currentStep.getExecStartTime()) / 1000);
+        }
+
+        return jobInstance;
+    }
+
+    protected void handleException(String jobInstanceUuid, int jobInstanceStepSeqId, KylinConfig config, Throwable t) {
+        log.error(t.getLocalizedMessage(), t);
+        String exceptionMsg = "Failed with Exception:" + ExceptionUtils.getFullStackTrace(t);
+        try {
+            JobDAO dao = JobDAO.getInstance(config);
+            JobInstance jobInstance = dao.getJob(jobInstanceUuid);
+            JobStep jobStep = jobInstance.getSteps().get(jobInstanceStepSeqId);
+            jobStep.setStatus(JobStepStatusEnum.ERROR);
+            jobStep.setExecEndTime(System.currentTimeMillis());
+            dao.updateJobInstance(jobInstance);
+
+            String output = dao.getJobOutput(jobInstanceUuid, jobInstanceStepSeqId).getOutput();
+            output = output + "\n" + exceptionMsg;
+            dao.saveJobOutput(jobInstanceUuid, jobInstanceStepSeqId, output);
+        } catch (IOException e1) {
+            log.error(e1.getLocalizedMessage(), e1);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/job/src/main/java/com/kylinolap/job/hadoop/AbstractHadoopJob.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/com/kylinolap/job/hadoop/AbstractHadoopJob.java b/job/src/main/java/com/kylinolap/job/hadoop/AbstractHadoopJob.java
new file mode 100644
index 0000000..4cd59a9
--- /dev/null
+++ b/job/src/main/java/com/kylinolap/job/hadoop/AbstractHadoopJob.java
@@ -0,0 +1,291 @@
+/*
+ * 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.job.hadoop;
+
+/**
+ * @author George Song (ysong1)
+ * 
+ */
+import java.io.File;
+import java.io.IOException;
+import java.io.InputStream;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.Map;
+
+import org.apache.commons.cli.Option;
+import org.apache.commons.cli.OptionBuilder;
+import org.apache.commons.cli.Options;
+import org.apache.commons.cli.ParseException;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.conf.Configured;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.mapreduce.Counters;
+import org.apache.hadoop.mapreduce.InputFormat;
+import org.apache.hadoop.mapreduce.InputSplit;
+import org.apache.hadoop.mapreduce.Job;
+import org.apache.hadoop.mapreduce.lib.input.FileInputFormat;
+import org.apache.hadoop.util.ReflectionUtils;
+import org.apache.hadoop.util.StringUtils;
+import org.apache.hadoop.util.Tool;
+import org.apache.hadoop.util.ToolRunner;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.kylinolap.common.KylinConfig;
+import com.kylinolap.common.persistence.ResourceStore;
+import com.kylinolap.common.util.StringSplitter;
+import com.kylinolap.cube.CubeInstance;
+import com.kylinolap.cube.CubeSegment;
+import com.kylinolap.job.JobInstance;
+import com.kylinolap.job.exception.JobException;
+import com.kylinolap.job.tools.OptionsHelper;
+import com.kylinolap.metadata.model.schema.TableDesc;
+
+@SuppressWarnings("static-access")
+public abstract class AbstractHadoopJob extends Configured implements Tool {
+    protected static final Logger log = LoggerFactory.getLogger(AbstractHadoopJob.class);
+
+    protected static final Option OPTION_JOB_NAME = OptionBuilder.withArgName("name").hasArg().isRequired(true).withDescription("Job name. For exmaple, Kylin_Cuboid_Builder-clsfd_v2_Step_22-D)").create("jobname");
+    protected static final Option OPTION_CUBE_NAME = OptionBuilder.withArgName("name").hasArg().isRequired(true).withDescription("Cube name. For exmaple, flat_item_cube").create("cubename");
+    protected static final Option OPTION_SEGMENT_NAME = OptionBuilder.withArgName("name").hasArg().isRequired(true).withDescription("Cube segment name)").create("segmentname");
+    protected static final Option OPTION_TABLE_NAME = OptionBuilder.withArgName("name").hasArg().isRequired(true).withDescription("Hive table name.").create("tablename");
+    protected static final Option OPTION_INPUT_PATH = OptionBuilder.withArgName("path").hasArg().isRequired(true).withDescription("Input path").create("input");
+    protected static final Option OPTION_INPUT_FORMAT = OptionBuilder.withArgName("inputformat").hasArg().isRequired(false).withDescription("Input format").create("inputformat");
+    protected static final Option OPTION_INPUT_DELIM = OptionBuilder.withArgName("inputdelim").hasArg().isRequired(false).withDescription("Input delimeter").create("inputdelim");
+    protected static final Option OPTION_OUTPUT_PATH = OptionBuilder.withArgName("path").hasArg().isRequired(true).withDescription("Output path").create("output");
+    protected static final Option OPTION_NCUBOID_LEVEL = OptionBuilder.withArgName("level").hasArg().isRequired(true).withDescription("N-Cuboid build level, e.g. 1, 2, 3...").create("level");
+    protected static final Option OPTION_PARTITION_FILE_PATH = OptionBuilder.withArgName("path").hasArg().isRequired(true).withDescription("Partition file path.").create("input");
+    protected static final Option OPTION_HTABLE_NAME = OptionBuilder.withArgName("htable name").hasArg().isRequired(true).withDescription("HTable name").create("htablename");
+    protected static final Option OPTION_KEY_COLUMN_PERCENTAGE = OptionBuilder.withArgName("rowkey column percentage").hasArg().isRequired(true).withDescription("Percentage of row key columns").create("columnpercentage");
+    protected static final Option OPTION_KEY_SPLIT_NUMBER = OptionBuilder.withArgName("key split number").hasArg().isRequired(true).withDescription("Number of key split range").create("splitnumber");
+
+    protected String name;
+    protected String description;
+    protected boolean isAsync = false;
+    protected OptionsHelper optionsHelper = new OptionsHelper();
+
+    protected Job job;
+
+    protected void parseOptions(Options options, String[] args) throws ParseException {
+        optionsHelper.parseOptions(options, args);
+    }
+
+    public void printUsage(Options options) {
+        optionsHelper.printUsage(getClass().getSimpleName(), options);
+    }
+
+    public Option[] getOptions() {
+        return optionsHelper.getOptions();
+    }
+
+    public String getOptionsAsString() {
+        return optionsHelper.getOptionsAsString();
+    }
+
+    protected String getOptionValue(Option option) {
+        return optionsHelper.getOptionValue(option);
+    }
+
+    protected boolean hasOption(Option option) {
+        return optionsHelper.hasOption(option);
+    }
+
+    protected int waitForCompletion(Job job) throws IOException, InterruptedException, ClassNotFoundException {
+        int retVal = 0;
+        long start = System.nanoTime();
+
+        if (isAsync) {
+            job.submit();
+        } else {
+            job.waitForCompletion(true);
+            retVal = job.isSuccessful() ? 0 : 1;
+        }
+
+        log.debug("Job '" + job.getJobName() + "' finished " + (job.isSuccessful() ? "successfully in " : "with failures.  Time taken ") + StringUtils.formatTime((System.nanoTime() - start) / 1000000L));
+
+        return retVal;
+    }
+
+    protected static void runJob(Tool job, String[] args) {
+        try {
+            int exitCode = ToolRunner.run(job, args);
+            System.exit(exitCode);
+        } catch (Exception e) {
+            e.printStackTrace(System.err);
+            System.exit(5);
+        }
+    }
+
+    public void addInputDirs(String input, Job job) throws IOException {
+        for (String inp : StringSplitter.split(input, ",")) {
+            inp = inp.trim();
+            if (inp.endsWith("/*")) {
+                inp = inp.substring(0, inp.length() - 2);
+                FileSystem fs = FileSystem.get(job.getConfiguration());
+                Path path = new Path(inp);
+                FileStatus[] fileStatuses = fs.listStatus(path);
+                boolean hasDir = false;
+                for (FileStatus stat : fileStatuses) {
+                    if (stat.isDirectory()) {
+                        hasDir = true;
+                        addInputDirs(stat.getPath().toString(), job);
+                    }
+                }
+                if (fileStatuses.length > 0 && !hasDir) {
+                    addInputDirs(path.toString(), job);
+                }
+            } else {
+                System.out.println("Add input " + inp);
+                FileInputFormat.addInputPath(job, new Path(inp));
+            }
+        }
+    }
+
+    protected void attachKylinPropsAndMetadata(CubeInstance cube, Configuration conf) throws IOException {
+        File tmp = File.createTempFile("kylin_job_meta", "");
+        tmp.delete(); // we need a directory, so delete the file first
+
+        File metaDir = new File(tmp, "meta");
+        metaDir.mkdirs();
+        metaDir.getParentFile().deleteOnExit();
+
+        // write kylin.properties
+        KylinConfig kylinConfig = KylinConfig.getInstanceFromEnv();
+        File kylinPropsFile = new File(metaDir, "kylin.properties");
+        kylinConfig.writeProperties(kylinPropsFile);
+
+        // write cube / cube_desc / dict / table
+        ArrayList<String> dumpList = new ArrayList<String>();
+        dumpList.add(cube.getResourcePath());
+        dumpList.add(cube.getDescriptor().getResourcePath());
+        if (cube.isInvertedIndex()) {
+            dumpList.add(cube.getInvertedIndexDesc().getResourcePath());
+        }
+        for (TableDesc table : cube.getDescriptor().listTables()) {
+            dumpList.add(table.getResourcePath());
+        }
+
+        for (CubeSegment segment : cube.getSegments()) {
+            dumpList.addAll(segment.getDictionaryPaths());
+        }
+
+        dumpResources(kylinConfig, metaDir, dumpList);
+
+        // hadoop distributed cache
+        conf.set("tmpfiles", "file:///" + OptionsHelper.convertToFileURL(metaDir.getAbsolutePath()));
+    }
+
+    private void dumpResources(KylinConfig kylinConfig, File metaDir, ArrayList<String> dumpList) throws IOException {
+        ResourceStore from = ResourceStore.getStore(kylinConfig);
+        KylinConfig localConfig = KylinConfig.createInstanceFromUri(metaDir.getAbsolutePath());
+        ResourceStore to = ResourceStore.getStore(localConfig);
+        for (String path : dumpList) {
+            InputStream in = from.getResource(path);
+            if (in == null)
+                throw new IllegalStateException("No resource found at -- " + path);
+            long ts = from.getResourceTimestamp(path);
+            to.putResource(path, in, ts);
+            log.info("Dumped resource " + path + " to " + metaDir.getAbsolutePath());
+        }
+    }
+
+    protected void deletePath(Configuration conf, Path path) throws IOException {
+        FileSystem fs = FileSystem.get(conf);
+        if (fs.exists(path)) {
+            fs.delete(path, true);
+        }
+    }
+
+    protected double getTotalMapInputMB() throws ClassNotFoundException, IOException, InterruptedException, JobException {
+        if (job == null) {
+            throw new JobException("Job is null");
+        }
+
+        long mapInputBytes = 0;
+        InputFormat<?, ?> input = ReflectionUtils.newInstance(job.getInputFormatClass(), job.getConfiguration());
+        for (InputSplit split : input.getSplits(job)) {
+            mapInputBytes += split.getLength();
+        }
+        if (mapInputBytes == 0) {
+            throw new IllegalArgumentException("Map input splits are 0 bytes, something is wrong!");
+        }
+        double totalMapInputMB = (double) mapInputBytes / 1024 / 1024;
+        return totalMapInputMB;
+    }
+
+    protected int getMapInputSplitCount() throws ClassNotFoundException, JobException, IOException, InterruptedException {
+        if (job == null) {
+            throw new JobException("Job is null");
+        }
+        InputFormat<?, ?> input = ReflectionUtils.newInstance(job.getInputFormatClass(), job.getConfiguration());
+        return input.getSplits(job).size();
+    }
+
+    public static KylinConfig loadKylinPropsAndMetadata(Configuration conf) throws IOException {
+        File metaDir = new File("meta");
+        System.setProperty(KylinConfig.KYLIN_CONF, metaDir.getAbsolutePath());
+        System.out.println("The absolute path for meta dir is " + metaDir.getAbsolutePath());
+        KylinConfig kylinConfig = KylinConfig.getInstanceFromEnv();
+        kylinConfig.setMetadataUrl(metaDir.getCanonicalPath());
+        return kylinConfig;
+    }
+
+    public void kill() throws JobException {
+        if (job != null) {
+            try {
+                job.killJob();
+            } catch (IOException e) {
+                throw new JobException(e);
+            }
+        }
+    }
+
+    public Map<String, String> getInfo() throws JobException {
+        if (job != null) {
+            Map<String, String> status = new HashMap<String, String>();
+            if (null != job.getJobID()) {
+                status.put(JobInstance.MR_JOB_ID, job.getJobID().toString());
+            }
+            if (null != job.getTrackingURL()) {
+                status.put(JobInstance.YARN_APP_URL, job.getTrackingURL().toString());
+            }
+
+            return status;
+        } else {
+            throw new JobException("Job is null");
+        }
+    }
+
+    public Counters getCounters() throws JobException {
+        if (job != null) {
+            try {
+                return job.getCounters();
+            } catch (IOException e) {
+                throw new JobException(e);
+            }
+        } else {
+            throw new JobException("Job is null");
+        }
+    }
+
+    public void setAsync(boolean isAsync) {
+        this.isAsync = isAsync;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/job/src/main/java/com/kylinolap/job/hadoop/cardinality/ColumnCardinalityMapper.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/com/kylinolap/job/hadoop/cardinality/ColumnCardinalityMapper.java b/job/src/main/java/com/kylinolap/job/hadoop/cardinality/ColumnCardinalityMapper.java
new file mode 100644
index 0000000..e25ec36
--- /dev/null
+++ b/job/src/main/java/com/kylinolap/job/hadoop/cardinality/ColumnCardinalityMapper.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.job.hadoop.cardinality;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.Map;
+import java.util.StringTokenizer;
+
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.io.BytesWritable;
+import org.apache.hadoop.io.IntWritable;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.mapreduce.Mapper;
+
+import com.kylinolap.common.hll.HyperLogLogPlusCounter;
+import com.kylinolap.cube.kv.RowConstants;
+
+/**
+ * @author Jack
+ * 
+ */
+public class ColumnCardinalityMapper<T> extends Mapper<T, Text, IntWritable, BytesWritable> {
+
+    private Map<Integer, HyperLogLogPlusCounter> hllcMap = new HashMap<Integer, HyperLogLogPlusCounter>();
+    public static final String DEFAULT_DELIM = ",";
+
+    @Override
+    public void map(T key, Text value, Context context) throws IOException, InterruptedException {
+        String delim = context.getConfiguration().get(HiveColumnCardinalityJob.KEY_INPUT_DELIM);
+        if (delim == null) {
+            delim = DEFAULT_DELIM;
+        }
+        String line = value.toString();
+        StringTokenizer tokenizer = new StringTokenizer(line, delim);
+        int i = 1;
+        while (tokenizer.hasMoreTokens()) {
+            String temp = tokenizer.nextToken();
+            getHllc(i).add(Bytes.toBytes(temp));
+            i++;
+        }
+    }
+
+    private HyperLogLogPlusCounter getHllc(Integer key) {
+        if (!hllcMap.containsKey(key)) {
+            hllcMap.put(key, new HyperLogLogPlusCounter());
+        }
+        return hllcMap.get(key);
+    }
+
+    @Override
+    protected void cleanup(Context context) throws IOException, InterruptedException {
+        Iterator<Integer> it = hllcMap.keySet().iterator();
+        while (it.hasNext()) {
+            int key = it.next();
+            HyperLogLogPlusCounter hllc = hllcMap.get(key);
+            ByteBuffer buf = ByteBuffer.allocate(RowConstants.ROWVALUE_BUFFER_SIZE);
+            buf.clear();
+            hllc.writeRegisters(buf);
+            buf.flip();
+            context.write(new IntWritable(key), new BytesWritable(buf.array()));
+        }
+    }
+
+}


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

Posted by lu...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/examples/test_case_data/localmeta/data/TEST_CAL_DT.csv
----------------------------------------------------------------------
diff --git a/examples/test_case_data/localmeta/data/TEST_CAL_DT.csv b/examples/test_case_data/localmeta/data/TEST_CAL_DT.csv
new file mode 100644
index 0000000..67efe23
--- /dev/null
+++ b/examples/test_case_data/localmeta/data/TEST_CAL_DT.csv
@@ -0,0 +1,731 @@
+2012-08-16,2012-01-01,2012-07-01,2012-08-01,2012-08-11,0,-1,-3,-15,-103,0,-1,-4,-15,-15,41501,228,47,16,6,33,5928,2,8,1364,3,455,114,2012-12-31,2012-09-30,2012-08-31,2012-08-17,16-Aug-2012,Aug 16th 2012,Fri 08-16-13,1,0,0,0,2012-06-21,365,92,31,7,2012-12-30,2012-06-30,2012-07-28,2012-08-11,2012-08-12,2012-08-16,Fri       ,2012M08,Aug-2012,N,2012M08   ,N,Year 2012 - Quarter 03,2012Q03   ,N,33,2012,2012-08-11,2012-08-17,N,Wk.33 - 13,2012-08-11 00:00:00,2012-08-17 00:00:00,2012W33   ,2012W33   ,08/11/13 - 08/17/13,08/11 - 08/17,2012,N,2012-08-16,2011-08-16,2012-05-16,2012-02-16,2012-07-16,2012-06-16,2012-08-09,2012-08-02,0,0,0,0,0,0,0,0,8,3,33,3,1,1,1,2005-09-07,USER_X  ,2012-11-27 00:16:56,USER_X
+2012-01-03,2012-01-01,2012-01-01,2012-01-01,2012-01-01,0,-3,-10,-47,-328,0,-3,-11,-47,-47,41276,3,3,3,5,1,5896,1,1,1357,1,453,114,2012-12-31,2012-03-31,2012-01-31,2012-01-05,03-Jan-2012,Jan 3rd 2012,Thu 01-03-13,1,0,0,0,2012-12-21,365,90,31,5,2012-12-30,2012-12-30,2012-12-30,2012-12-30,2012-12-31,2012-01-03,Thu       ,2012M01,Jan-2012,N,2012M01   ,N,Year 2012 - Quarter 01,2012Q01   ,N,1,2012,2012-12-30,2012-01-05,N,Wk.01 - 13,2012-01-01 00:00:00,2012-01-05 00:00:00,2012W01   ,2012W01   ,01/01/13 - 01/05/13,01/01 - 01/05,2012,N,2012-01-03,2011-01-03,2012-10-03,2012-07-03,2012-12-03,2012-11-03,2012-12-27,2012-12-20,0,0,0,0,0,0,0,0,1,1,1,1,1,1,1,2005-09-07,USER_X  ,2012-11-27 00:16:56,USER_X
+2012-04-10,2012-01-01,2012-04-01,2012-04-01,2012-04-07,0,-2,-7,-33,-231,0,-2,-8,-33,-33,41373,100,10,10,4,15,5910,1,4,1360,2,454,114,2012-12-31,2012-06-30,2012-04-30,2012-04-13,10-Apr-2012,Apr 10th 2012,Wed 04-10-13,1,0,0,0,2012-03-21,365,91,30,7,2012-12-30,2012-03-31,2012-03-31,2012-04-07,2012-04-08,2012-04-10,Wed       ,2012M04,Apr-2012,N,2012M04   ,N,Year 2012 - Quarter 02,2012Q02   ,N,15,2012,2012-04-07,2012-04-13,N,Wk.15 - 13,2012-04-07 00:00:00,2012-04-13 00:00:00,2012W15   ,2012W15   ,04/07/13 - 04/13/13,04/07 - 04/13,2012,N,2012-04-10,2011-04-10,2012-01-10,2012-10-10,2012-03-10,2012-02-10,2012-04-03,2012-03-27,0,0,0,0,0,0,0,0,4,2,15,2,1,1,1,2005-09-07,USER_X  ,2012-11-27 00:16:56,USER_X
+2012-06-12,2012-01-01,2012-04-01,2012-06-01,2012-06-09,0,-2,-5,-24,-168,0,-2,-6,-24,-24,41436,163,73,12,4,24,5919,3,6,1362,2,454,114,2012-12-31,2012-06-30,2012-06-30,2012-06-15,12-Jun-2012,Jun 12th 2012,Wed 06-12-13,1,0,0,0,2012-03-21,365,91,30,7,2012-12-30,2012-03-31,2012-05-26,2012-06-09,2012-06-10,2012-06-12,Wed       ,2012M06,Jun-2012,N,2012M06   ,N,Year 2012 - Quarter 02,2012Q02   ,N,24,2012,2012-06-09,2012-06-15,N,Wk.24 - 13,2012-06-09 00:00:00,2012-06-15 00:00:00,2012W24   ,2012W24   ,06/09/13 - 06/15/13,06/09 - 06/15,2012,N,2012-06-12,2011-06-12,2012-03-12,2012-12-12,2012-05-12,2012-04-12,2012-06-05,2012-05-29,0,0,0,0,0,0,0,0,6,2,24,2,1,1,1,2005-09-07,USER_X  ,2012-11-27 00:16:56,USER_X
+2012-04-27,2012-01-01,2012-04-01,2012-04-01,2012-04-21,0,-2,-7,-31,-214,0,-2,-8,-31,-31,41390,117,27,27,7,17,5912,1,4,1360,2,454,114,2012-12-31,2012-06-30,2012-04-30,2012-04-27,27-Apr-2012,Apr 27th 2012,Sat 04-27-13,1,0,0,0,2012-03-21,365,91,30,7,2012-12-30,2012-03-31,2012-03-31,2012-04-21,2012-04-22,2012-04-27,Sat       ,2012M04,Apr-2012,N,2012M04   ,N,Year 2012 - Quarter 02,2012Q02   ,N,17,2012,2012-04-21,2012-04-27,N,Wk.17 - 13,2012-04-21 00:00:00,2012-04-27 00:00:00,2012W17   ,2012W17   ,04/21/13 - 04/27/13,04/21 - 04/27,2012,N,2012-04-27,2011-04-27,2012-01-27,2012-10-27,2012-03-27,2012-02-27,2012-04-20,2012-04-13,0,0,0,0,0,0,0,0,4,2,17,2,1,1,1,2005-09-07,USER_X  ,2012-11-27 00:16:56,USER_X
+2012-09-19,2012-01-01,2012-07-01,2012-09-01,2012-09-15,0,-1,-2,-10,-69,0,-1,-3,-10,-10,41535,262,81,19,5,38,5933,3,9,1365,3,455,114,2012-12-31,2012-09-30,2012-09-30,2012-09-21,19-Sep-2012,Sep 19th 2012,Thu 09-19-13,1,0,0,0,2012-06-21,365,92,30,7,2012-12-30,2012-06-30,2012-08-25,2012-09-15,2012-09-16,2012-09-19,Thu       ,2012M09,Sep-2012,N,2012M09   ,N,Year 2012 - Quarter 03,2012Q03   ,N,38,2012,2012-09-15,2012-09-21,N,Wk.38 - 13,2012-09-15 00:00:00,2012-09-21 00:00:00,2012W38   ,2012W38   ,09/15/13 - 09/21/13,09/15 - 09/21,2012,N,2012-09-19,2011-09-19,2012-06-19,2012-03-19,2012-08-19,2012-07-19,2012-09-12,2012-09-05,0,0,0,0,0,0,0,0,9,3,38,3,1,1,1,2005-09-07,USER_X  ,2012-11-27 00:16:56,USER_X
+2012-03-07,2012-01-01,2012-01-01,2012-03-01,2012-03-03,0,-3,-8,-38,-265,0,-3,-9,-38,-38,41339,66,66,7,5,10,5905,3,3,1359,1,453,114,2012-12-31,2012-03-31,2012-03-31,2012-03-09,07-Mar-2012,Mar 7th 2012,Thu 03-07-13,1,0,0,0,2012-12-21,365,90,31,7,2012-12-30,2012-12-30,2012-02-24,2012-03-03,2012-03-04,2012-03-07,Thu       ,2012M03,Mar-2012,N,2012M03   ,N,Year 2012 - Quarter 01,2012Q01   ,N,10,2012,2012-03-03,2012-03-09,N,Wk.10 - 13,2012-03-03 00:00:00,2012-03-09 00:00:00,2012W10   ,2012W10   ,03/03/13 - 03/09/13,03/03 - 03/09,2012,N,2012-03-07,2011-03-07,2012-12-07,2012-09-07,2012-02-07,2012-01-07,2012-02-28,2012-02-21,0,0,0,0,0,0,0,0,3,1,10,1,1,1,1,2005-09-07,USER_X  ,2012-11-27 00:16:56,USER_X
+2012-03-28,2012-01-01,2012-01-01,2012-03-01,2012-03-24,0,-3,-8,-35,-244,0,-3,-9,-35,-35,41360,87,87,28,5,13,5908,3,3,1359,1,453,114,2012-12-31,2012-03-31,2012-03-31,2012-03-30,28-Mar-2012,Mar 28th 2012,Thu 03-28-13,1,0,0,0,2012-03-21,365,90,31,7,2012-12-30,2012-12-30,2012-02-24,2012-03-24,2012-03-25,2012-03-28,Thu       ,2012M03,Mar-2012,N,2012M03   ,N,Year 2012 - Quarter 01,2012Q01   ,N,13,2012,2012-03-24,2012-03-30,N,Wk.13 - 13,2012-03-24 00:00:00,2012-03-30 00:00:00,2012W13   ,2012W13   ,03/24/13 - 03/30/13,03/24 - 03/30,2012,N,2012-03-28,2011-03-28,2012-12-28,2012-09-28,2012-02-28,2012-01-28,2012-03-21,2012-03-14,0,0,0,0,0,0,0,0,3,1,13,2,1,1,1,2005-09-07,USER_X  ,2012-11-27 00:16:56,USER_X
+2012-11-23,2012-01-01,2012-10-01,2012-11-01,2012-11-17,0,0,0,-1,-4,0,0,-1,-1,-1,41600,327,54,23,7,47,5942,2,11,1367,4,456,114,2012-12-31,2012-12-31,2012-11-30,2012-11-23,23-Nov-2012,Nov 23rd 2012,Sat 11-23-13,1,1,1,0,2012-09-21,365,92,30,7,2012-12-30,2012-09-29,2012-10-27,2012-11-17,2012-11-18,2012-11-23,Sat       ,2012M11,Nov-2012,N,2012M11   ,N,Year 2012 - Quarter 04,2012Q04   ,N,47,2012,2012-11-17,2012-11-23,N,Wk.47 - 13,2012-11-17 00:00:00,2012-11-23 00:00:00,2012W47   ,2012W47   ,11/17/13 - 11/23/13,11/17 - 11/23,2012,N,2012-11-23,2011-11-23,2012-08-23,2012-05-23,2012-10-23,2012-09-23,2012-11-16,2012-11-09,0,0,0,0,0,0,0,0,11,4,47,4,1,1,1,2005-09-07,USER_X  ,2012-11-27 00:16:56,USER_X
+2012-03-09,2012-01-01,2012-01-01,2012-03-01,2012-03-03,0,-3,-8,-38,-263,0,-3,-9,-38,-38,41341,68,68,9,7,10,5905,3,3,1359,1,453,114,2012-12-31,2012-03-31,2012-03-31,2012-03-09,09-Mar-2012,Mar 9th 2012,Sat 03-09-13,1,0,0,0,2012-12-21,365,90,31,7,2012-12-30,2012-12-30,2012-02-24,2012-03-03,2012-03-04,2012-03-09,Sat       ,2012M03,Mar-2012,N,2012M03   ,N,Year 2012 - Quarter 01,2012Q01   ,N,10,2012,2012-03-03,2012-03-09,N,Wk.10 - 13,2012-03-03 00:00:00,2012-03-09 00:00:00,2012W10   ,2012W10   ,03/03/13 - 03/09/13,03/03 - 03/09,2012,N,2012-03-09,2011-03-09,2012-12-09,2012-09-09,2012-02-09,2012-01-09,2012-03-02,2012-02-23,0,0,0,0,0,0,0,0,3,1,10,1,1,1,1,2005-09-07,USER_X  ,2012-11-27 00:16:56,USER_X
+2012-11-21,2012-01-01,2012-10-01,2012-11-01,2012-11-17,0,0,0,-1,-6,0,0,-1,-1,-1,41598,325,52,21,5,47,5942,2,11,1367,4,456,114,2012-12-31,2012-12-31,2012-11-30,2012-11-23,21-Nov-2012,Nov 21st 2012,Thu 11-21-13,1,1,1,0,2012-09-21,365,92,30,7,2012-12-30,2012-09-29,2012-10-27,2012-11-17,2012-11-18,2012-11-21,Thu       ,2012M11,Nov-2012,N,2012M11   ,N,Year 2012 - Quarter 04,2012Q04   ,N,47,2012,2012-11-17,2012-11-23,N,Wk.47 - 13,2012-11-17 00:00:00,2012-11-23 00:00:00,2012W47   ,2012W47   ,11/17/13 - 11/23/13,11/17 - 11/23,2012,N,2012-11-21,2011-11-21,2012-08-21,2012-05-21,2012-10-21,2012-09-21,2012-11-14,2012-11-07,0,0,0,0,0,0,0,0,11,4,47,4,1,1,1,2005-09-07,USER_X  ,2012-11-27 00:16:56,USER_X
+2012-03-26,2012-01-01,2012-01-01,2012-03-01,2012-03-24,0,-3,-8,-35,-246,0,-3,-9,-35,-35,41358,85,85,26,3,13,5908,3,3,1359,1,453,114,2012-12-31,2012-03-31,2012-03-31,2012-03-30,26-Mar-2012,Mar 26th 2012,Tue 03-26-13,1,0,0,0,2012-03-21,365,90,31,7,2012-12-30,2012-12-30,2012-02-24,2012-03-24,2012-03-25,2012-03-26,Tue       ,2012M03,Mar-2012,N,2012M03   ,N,Year 2012 - Quarter 01,2012Q01   ,N,13,2012,2012-03-24,2012-03-30,N,Wk.13 - 13,2012-03-24 00:00:00,2012-03-30 00:00:00,2012W13   ,2012W13   ,03/24/13 - 03/30/13,03/24 - 03/30,2012,N,2012-03-26,2011-03-26,2012-12-26,2012-09-26,2012-02-26,2012-01-26,2012-03-19,2012-03-12,0,0,0,0,0,0,0,0,3,1,13,2,1,1,1,2005-09-07,USER_X  ,2012-11-27 00:16:56,USER_X
+2012-01-24,2012-01-01,2012-01-01,2012-01-01,2012-01-20,0,-3,-10,-44,-307,0,-3,-11,-44,-44,41297,24,24,24,5,4,5899,1,1,1357,1,453,114,2012-12-31,2012-03-31,2012-01-31,2012-01-26,24-Jan-2012,Jan 24th 2012,Thu 01-24-13,1,0,0,0,2012-12-21,365,90,31,7,2012-12-30,2012-12-30,2012-12-30,2012-01-20,2012-01-21,2012-01-24,Thu       ,2012M01,Jan-2012,N,2012M01   ,N,Year 2012 - Quarter 01,2012Q01   ,N,4,2012,2012-01-20,2012-01-26,N,Wk.04 - 13,2012-01-20 00:00:00,2012-01-26 00:00:00,2012W04   ,2012W04   ,01/20/13 - 01/26/13,01/20 - 01/26,2012,N,2012-01-24,2011-01-24,2012-10-24,2012-07-24,2012-12-24,2012-11-24,2012-01-17,2012-01-10,0,0,0,0,0,0,0,0,1,1,4,1,1,1,1,2005-09-07,USER_X  ,2012-11-27 00:16:56,USER_X
+2012-01-01,2012-01-01,2012-01-01,2012-01-01,2012-01-01,0,-3,-10,-47,-330,0,-3,-11,-47,-47,41274,1,1,1,3,1,5896,1,1,1357,1,453,114,2012-12-31,2012-03-31,2012-01-31,2012-01-05,01-Jan-2012,Jan 1st 2012,Tue 01-01-13,1,0,0,0,2012-12-21,365,90,31,5,2012-12-30,2012-12-30,2012-12-30,2012-12-30,2012-12-31,2012-01-01,Tue       ,2012M01,Jan-2012,Y,2012M01   ,Y,Year 2012 - Quarter 01,2012Q01   ,Y,1,2012,2012-12-30,2012-01-05,Y,Wk.01 - 13,2012-01-01 00:00:00,2012-01-05 00:00:00,2012W01   ,2012W01   ,01/01/13 - 01/05/13,01/01 - 01/05,2012,Y,2012-01-01,2011-01-01,2012-10-01,2012-07-01,2012-12-01,2012-11-01,2012-12-25,2012-12-18,0,0,0,0,0,0,0,0,1,1,1,1,1,1,1,2005-09-07,USER_X  ,2012-11-27 00:16:56,USER_X
+2012-04-29,2012-01-01,2012-04-01,2012-04-01,2012-04-28,0,-2,-7,-30,-212,0,-2,-7,-30,-30,41392,119,29,29,2,18,5913,1,4,1360,2,454,114,2012-12-31,2012-06-30,2012-04-30,2012-05-04,29-Apr-2012,Apr 29th 2012,Mon 04-29-13,1,0,0,0,2012-03-21,365,91,30,7,2012-12-30,2012-03-31,2012-04-28,2012-04-28,2012-04-29,2012-04-29,Mon       ,2012M04,Apr-2012,N,2012M04   ,N,Year 2012 - Quarter 02,2012Q02   ,N,18,2012,2012-04-28,2012-05-04,N,Wk.18 - 13,2012-04-28 00:00:00,2012-05-04 00:00:00,2012W18   ,2012W18   ,04/28/13 - 05/04/13,04/28 - 05/04,2012,N,2012-04-29,2011-04-29,2012-01-29,2012-10-29,2012-03-29,2012-02-28,2012-04-22,2012-04-15,0,0,0,0,0,0,0,0,5,2,18,2,1,1,1,2005-09-07,USER_X  ,2012-11-27 00:16:56,USER_X
+2012-05-11,2012-01-01,2012-04-01,2012-05-01,2012-05-05,0,-2,-6,-29,-200,0,-2,-7,-29,-29,41404,131,41,11,7,19,5914,2,5,1361,2,454,114,2012-12-31,2012-06-30,2012-05-31,2012-05-11,11-May-2012,May 11th 2012,Sat 05-11-13,1,0,0,0,2012-03-21,365,91,31,7,2012-12-30,2012-03-31,2012-04-28,2012-05-05,2012-05-06,2012-05-11,Sat       ,2012M05,May-2012,N,2012M05   ,N,Year 2012 - Quarter 02,2012Q02   ,N,19,2012,2012-05-05,2012-05-11,N,Wk.19 - 13,2012-05-05 00:00:00,2012-05-11 00:00:00,2012W19   ,2012W19   ,05/05/13 - 05/11/13,05/05 - 05/11,2012,N,2012-05-11,2011-05-11,2012-02-11,2012-11-11,2012-04-11,2012-03-11,2012-05-04,2012-04-27,0,0,0,0,0,0,0,0,5,2,19,2,1,1,1,2005-09-07,USER_X  ,2012-11-27 00:16:56,USER_X
+2012-10-01,2012-01-01,2012-10-01,2012-10-01,2012-09-29,0,0,-1,-8,-57,0,0,-2,-8,-8,41547,274,1,1,3,40,5935,1,10,1366,4,456,114,2012-12-31,2012-12-31,2012-10-31,2012-10-05,01-Oct-2012,Oct 1st 2012,Tue 10-01-13,1,1,0,0,2012-09-21,365,92,31,7,2012-12-30,2012-09-29,2012-09-29,2012-09-29,2012-09-30,2012-10-01,Tue       ,2012M10,Oct-2012,Y,2012M10   ,Y,Year 2012 - Quarter 04,2012Q04   ,Y,40,2012,2012-09-29,2012-10-05,N,Wk.40 - 13,2012-09-29 00:00:00,2012-10-05 00:00:00,2012W40   ,2012W40   ,09/29/13 - 10/05/13,09/29 - 10/05,2012,N,2012-10-01,2011-10-01,2012-07-01,2012-04-01,2012-09-01,2012-08-01,2012-09-24,2012-09-17,0,0,0,0,0,0,0,0,10,4,40,4,1,1,1,2005-09-07,USER_X  ,2012-11-27 00:16:56,USER_X
+2012-12-05,2012-01-01,2012-10-01,2012-12-01,2012-12-01,0,0,1,1,8,0,0,0,1,1,41612,339,66,5,5,49,5944,3,12,1368,4,456,114,2012-12-31,2012-12-31,2012-12-31,2012-12-07,05-Dec-2012,Dec 5th 2012,Thu 12-05-13,0,0,0,0,2012-09-21,365,92,31,7,2012-12-30,2012-09-29,2012-11-24,2012-12-01,2012-12-02,2012-12-05,Thu       ,2012M12,Dec-2012,N,2012M12   ,N,Year 2012 - Quarter 04,2012Q04   ,N,49,2012,2012-12-01,2012-12-07,N,Wk.49 - 13,2012-12-01 00:00:00,2012-12-07 00:00:00,2012W49   ,2012W49   ,12/01/13 - 12/07/13,12/01 - 12/07,2012,N,2012-12-05,2011-12-05,2012-09-05,2012-06-05,2012-11-05,2012-10-05,2012-11-28,2012-11-21,0,0,0,0,0,0,0,0,12,4,49,4,0,1,0,2005-09-07,USER_X  ,2012-11-27 00:16:56,USER_X
+2012-03-05,2012-01-01,2012-01-01,2012-03-01,2012-03-03,0,-3,-8,-38,-267,0,-3,-9,-38,-38,41337,64,64,5,3,10,5905,3,3,1359,1,453,114,2012-12-31,2012-03-31,2012-03-31,2012-03-09,05-Mar-2012,Mar 5th 2012,Tue 03-05-13,1,0,0,0,2012-12-21,365,90,31,7,2012-12-30,2012-12-30,2012-02-24,2012-03-03,2012-03-04,2012-03-05,Tue       ,2012M03,Mar-2012,N,2012M03   ,N,Year 2012 - Quarter 01,2012Q01   ,N,10,2012,2012-03-03,2012-03-09,N,Wk.10 - 13,2012-03-03 00:00:00,2012-03-09 00:00:00,2012W10   ,2012W10   ,03/03/13 - 03/09/13,03/03 - 03/09,2012,N,2012-03-05,2011-03-05,2012-12-05,2012-09-05,2012-02-05,2012-01-05,2012-02-26,2012-02-19,0,0,0,0,0,0,0,0,3,1,10,1,1,1,1,2005-09-07,USER_X  ,2012-11-27 00:16:56,USER_X
+2012-10-03,2012-01-01,2012-10-01,2012-10-01,2012-09-29,0,0,-1,-8,-55,0,0,-2,-8,-8,41549,276,3,3,5,40,5935,1,10,1366,4,456,114,2012-12-31,2012-12-31,2012-10-31,2012-10-05,03-Oct-2012,Oct 3rd 2012,Thu 10-03-13,1,1,0,0,2012-09-21,365,92,31,7,2012-12-30,2012-09-29,2012-09-29,2012-09-29,2012-09-30,2012-10-03,Thu       ,2012M10,Oct-2012,N,2012M10   ,N,Year 2012 - Quarter 04,2012Q04   ,N,40,2012,2012-09-29,2012-10-05,N,Wk.40 - 13,2012-09-29 00:00:00,2012-10-05 00:00:00,2012W40   ,2012W40   ,09/29/13 - 10/05/13,09/29 - 10/05,2012,N,2012-10-03,2011-10-03,2012-07-03,2012-04-03,2012-09-03,2012-08-03,2012-09-26,2012-09-19,0,0,0,0,0,0,0,0,10,4,40,4,1,1,1,2005-09-07,USER_X  ,2012-11-27 00:16:56,USER_X
+2012-01-22,2012-01-01,2012-01-01,2012-01-01,2012-01-20,0,-3,-10,-44,-309,0,-3,-11,-44,-44,41295,22,22,22,3,4,5899,1,1,1357,1,453,114,2012-12-31,2012-03-31,2012-01-31,2012-01-26,22-Jan-2012,Jan 22nd 2012,Tue 01-22-13,1,0,0,0,2012-12-21,365,90,31,7,2012-12-30,2012-12-30,2012-12-30,2012-01-20,2012-01-21,2012-01-22,Tue       ,2012M01,Jan-2012,N,2012M01   ,N,Year 2012 - Quarter 01,2012Q01   ,N,4,2012,2012-01-20,2012-01-26,N,Wk.04 - 13,2012-01-20 00:00:00,2012-01-26 00:00:00,2012W04   ,2012W04   ,01/20/13 - 01/26/13,01/20 - 01/26,2012,N,2012-01-22,2011-01-22,2012-10-22,2012-07-22,2012-12-22,2012-11-22,2012-01-15,2012-01-08,0,0,0,0,0,0,0,0,1,1,4,1,1,1,1,2005-09-07,USER_X  ,2012-11-27 00:16:56,USER_X
+2012-12-26,2012-01-01,2012-10-01,2012-12-01,2012-12-22,0,0,1,4,29,0,0,0,4,4,41633,360,87,26,5,52,5947,3,12,1368,4,456,114,2012-12-31,2012-12-31,2012-12-31,2012-12-28,26-Dec-2012,Dec 26th 2012,Thu 12-26-13,0,0,0,0,2012-12-21,365,92,31,7,2012-12-30,2012-09-29,2012-11-24,2012-12-22,2012-12-23,2012-12-26,Thu       ,2012M12,Dec-2012,N,2012M12   ,N,Year 2012 - Quarter 04,2012Q04   ,N,52,2012,2012-12-22,2012-12-28,N,Wk.52 - 13,2012-12-22 00:00:00,2012-12-28 00:00:00,2012W52   ,2012W52   ,12/22/13 - 12/28/13,12/22 - 12/28,2012,N,2012-12-26,2011-12-26,2012-09-26,2012-06-26,2012-11-26,2012-10-26,2012-12-19,2012-12-12,0,0,0,0,0,0,0,0,12,4,52,1,0,1,0,2005-09-07,USER_X  ,2012-11-27 00:16:56,USER_X
+2012-09-17,2012-01-01,2012-07-01,2012-09-01,2012-09-15,0,-1,-2,-10,-71,0,-1,-3,-10,-10,41533,260,79,17,3,38,5933,3,9,1365,3,455,114,2012-12-31,2012-09-30,2012-09-30,2012-09-21,17-Sep-2012,Sep 17th 2012,Tue 09-17-13,1,0,0,0,2012-06-21,365,92,30,7,2012-12-30,2012-06-30,2012-08-25,2012-09-15,2012-09-16,2012-09-17,Tue       ,2012M09,Sep-2012,N,2012M09   ,N,Year 2012 - Quarter 03,2012Q03   ,N,38,2012,2012-09-15,2012-09-21,N,Wk.38 - 13,2012-09-15 00:00:00,2012-09-21 00:00:00,2012W38   ,2012W38   ,09/15/13 - 09/21/13,09/15 - 09/21,2012,N,2012-09-17,2011-09-17,2012-06-17,2012-03-17,2012-08-17,2012-07-17,2012-09-10,2012-09-03,0,0,0,0,0,0,0,0,9,3,38,3,1,1,1,2005-09-07,USER_X  ,2012-11-27 00:16:56,USER_X
+2012-10-20,2012-01-01,2012-10-01,2012-10-01,2012-10-20,0,0,-1,-5,-38,0,0,-2,-5,-6,41566,293,20,20,1,43,5938,1,10,1366,4,456,114,2012-12-31,2012-12-31,2012-10-31,2012-10-26,20-Oct-2012,Oct 20th 2012,Sun 10-20-13,1,1,0,0,2012-09-21,365,92,31,7,2012-12-30,2012-09-29,2012-09-29,2012-10-20,2012-10-14,2012-10-20,Sun       ,2012M10,Oct-2012,N,2012M10   ,N,Year 2012 - Quarter 04,2012Q04   ,N,43,2012,2012-10-20,2012-10-26,Y,Wk.43 - 13,2012-10-20 00:00:00,2012-10-26 00:00:00,2012W43   ,2012W43   ,10/20/13 - 10/26/13,10/20 - 10/26,2012,N,2012-10-20,2011-10-20,2012-07-20,2012-04-20,2012-09-20,2012-08-20,2012-10-13,2012-10-06,0,0,0,0,0,0,0,0,10,4,43,4,1,1,1,2005-09-07,USER_X  ,2012-11-27 00:16:56,USER_X
+2012-11-25,2012-01-01,2012-10-01,2012-11-01,2012-11-24,0,0,0,0,-2,0,0,0,0,0,41602,329,56,25,2,48,5943,2,11,1367,4,456,114,2012-12-31,2012-12-31,2012-11-30,2012-11-30,25-Nov-2012,Nov 25th 2012,Mon 11-25-13,1,1,1,1,2012-09-21,365,92,30,7,2012-12-30,2012-09-29,2012-11-24,2012-11-24,2012-11-25,2012-11-25,Mon       ,2012M11,Nov-2012,N,2012M11   ,N,Year 2012 - Quarter 04,2012Q04   ,N,48,2012,2012-11-24,2012-11-30,N,Wk.48 - 13,2012-11-24 00:00:00,2012-11-30 00:00:00,2012W48   ,2012W48   ,11/24/13 - 11/30/13,11/24 - 11/30,2012,N,2012-11-25,2011-11-25,2012-08-25,2012-05-25,2012-10-25,2012-09-25,2012-11-18,2012-11-11,0,0,0,0,0,0,0,0,12,4,48,4,1,1,1,2005-09-07,USER_X  ,2012-11-27 00:16:56,USER_X
+2012-01-26,2012-01-01,2012-01-01,2012-01-01,2012-01-20,0,-3,-10,-44,-305,0,-3,-11,-44,-44,41299,26,26,26,7,4,5899,1,1,1357,1,453,114,2012-12-31,2012-03-31,2012-01-31,2012-01-26,26-Jan-2012,Jan 26th 2012,Sat 01-26-13,1,0,0,0,2012-12-21,365,90,31,7,2012-12-30,2012-12-30,2012-12-30,2012-01-20,2012-01-21,2012-01-26,Sat       ,2012M01,Jan-2012,N,2012M01   ,N,Year 2012 - Quarter 01,2012Q01   ,N,4,2012,2012-01-20,2012-01-26,N,Wk.04 - 13,2012-01-20 00:00:00,2012-01-26 00:00:00,2012W04   ,2012W04   ,01/20/13 - 01/26/13,01/20 - 01/26,2012,N,2012-01-26,2011-01-26,2012-10-26,2012-07-26,2012-12-26,2012-11-26,2012-01-19,2012-01-12,0,0,0,0,0,0,0,0,1,1,4,1,1,1,1,2005-09-07,USER_X  ,2012-11-27 00:16:56,USER_X
+2012-12-24,2012-01-01,2012-10-01,2012-12-01,2012-12-22,0,0,1,4,27,0,0,0,4,4,41631,358,85,24,3,52,5947,3,12,1368,4,456,114,2012-12-31,2012-12-31,2012-12-31,2012-12-28,24-Dec-2012,Dec 24th 2012,Tue 12-24-13,0,0,0,0,2012-12-21,365,92,31,7,2012-12-30,2012-09-29,2012-11-24,2012-12-22,2012-12-23,2012-12-24,Tue       ,2012M12,Dec-2012,N,2012M12   ,N,Year 2012 - Quarter 04,2012Q04   ,N,52,2012,2012-12-22,2012-12-28,N,Wk.52 - 13,2012-12-22 00:00:00,2012-12-28 00:00:00,2012W52   ,2012W52   ,12/22/13 - 12/28/13,12/22 - 12/28,2012,N,2012-12-24,2011-12-24,2012-09-24,2012-06-24,2012-11-24,2012-10-24,2012-12-17,2012-12-10,0,0,0,0,0,0,0,0,12,4,52,1,0,1,0,2005-09-07,USER_X  ,2012-11-27 00:16:56,USER_X
+2012-02-04,2012-01-01,2012-01-01,2012-02-01,2012-02-03,0,-3,-9,-42,-296,0,-3,-10,-42,-42,41308,35,35,4,2,6,5901,2,2,1358,1,453,114,2012-12-31,2012-03-31,2012-02-28,2012-02-09,04-Feb-2012,Feb 4th 2012,Mon 02-04-13,1,0,0,0,2012-12-21,365,90,28,7,2012-12-30,2012-12-30,2012-01-27,2012-02-03,2012-02-04,2012-02-04,Mon       ,2012M02,Feb-2012,N,2012M02   ,N,Year 2012 - Quarter 01,2012Q01   ,N,6,2012,2012-02-03,2012-02-09,N,Wk.06 - 13,2012-02-03 00:00:00,2012-02-09 00:00:00,2012W06   ,2012W06   ,02/03/13 - 02/09/13,02/03 - 02/09,2012,N,2012-02-04,2011-02-04,2012-11-04,2012-08-04,2012-01-04,2012-12-04,2012-01-28,2012-01-21,0,0,0,0,0,0,0,0,2,1,6,1,1,1,1,2005-09-07,USER_X  ,2012-11-27 00:16:56,USER_X
+2012-05-30,2012-01-01,2012-04-01,2012-05-01,2012-05-26,0,-2,-6,-26,-181,0,-2,-6,-26,-26,41423,150,60,30,5,22,5917,2,5,1361,2,454,114,2012-12-31,2012-06-30,2012-05-31,2012-06-01,30-May-2012,May 30th 2012,Thu 05-30-13,1,0,0,0,2012-03-21,365,91,31,7,2012-12-30,2012-03-31,2012-05-26,2012-05-26,2012-05-27,2012-05-30,Thu       ,2012M05,May-2012,N,2012M05   ,N,Year 2012 - Quarter 02,2012Q02   ,N,22,2012,2012-05-26,2012-06-01,N,Wk.22 - 13,2012-05-26 00:00:00,2012-06-01 00:00:00,2012W22   ,2012W22   ,05/26/13 - 06/01/13,05/26 - 06/01,2012,N,2012-05-30,2011-05-30,2012-02-28,2012-11-30,2012-04-30,2012-03-30,2012-05-23,2012-05-16,0,0,0,0,0,0,0,0,6,2,22,2,1,1,1,2005-09-07,USER_X  ,2012-11-27 00:16:56,USER_X
+2012-04-12,2012-01-01,2012-04-01,2012-04-01,2012-04-07,0,-2,-7,-33,-229,0,-2,-8,-33,-33,41375,102,12,12,6,15,5910,1,4,1360,2,454,114,2012-12-31,2012-06-30,2012-04-30,2012-04-13,12-Apr-2012,Apr 12th 2012,Fri 04-12-13,1,0,0,0,2012-03-21,365,91,30,7,2012-12-30,2012-03-31,2012-03-31,2012-04-07,2012-04-08,2012-04-12,Fri       ,2012M04,Apr-2012,N,2012M04   ,N,Year 2012 - Quarter 02,2012Q02   ,N,15,2012,2012-04-07,2012-04-13,N,Wk.15 - 13,2012-04-07 00:00:00,2012-04-13 00:00:00,2012W15   ,2012W15   ,04/07/13 - 04/13/13,04/07 - 04/13,2012,N,2012-04-12,2011-04-12,2012-01-12,2012-10-12,2012-03-12,2012-02-12,2012-04-05,2012-03-29,0,0,0,0,0,0,0,0,4,2,15,2,1,1,1,2005-09-07,USER_X  ,2012-11-27 00:16:56,USER_X
+2012-04-08,2012-01-01,2012-04-01,2012-04-01,2012-04-07,0,-2,-7,-33,-233,0,-2,-8,-33,-33,41371,98,8,8,2,15,5910,1,4,1360,2,454,114,2012-12-31,2012-06-30,2012-04-30,2012-04-13,08-Apr-2012,Apr 8th 2012,Mon 04-08-13,1,0,0,0,2012-03-21,365,91,30,7,2012-12-30,2012-03-31,2012-03-31,2012-04-07,2012-04-08,2012-04-08,Mon       ,2012M04,Apr-2012,N,2012M04   ,N,Year 2012 - Quarter 02,2012Q02   ,N,15,2012,2012-04-07,2012-04-13,N,Wk.15 - 13,2012-04-07 00:00:00,2012-04-13 00:00:00,2012W15   ,2012W15   ,04/07/13 - 04/13/13,04/07 - 04/13,2012,N,2012-04-08,2011-04-08,2012-01-08,2012-10-08,2012-03-08,2012-02-08,2012-04-01,2012-03-25,0,0,0,0,0,0,0,0,4,2,15,2,1,1,1,2005-09-07,USER_X  ,2012-11-27 00:16:56,USER_X
+2012-12-28,2012-01-01,2012-10-01,2012-12-01,2012-12-22,0,0,1,4,31,0,0,0,4,4,41635,362,89,28,7,52,5947,3,12,1368,4,456,114,2012-12-31,2012-12-31,2012-12-31,2012-12-28,28-Dec-2012,Dec 28th 2012,Sat 12-28-13,0,0,0,0,2012-12-21,365,92,31,7,2012-12-30,2012-09-29,2012-11-24,2012-12-22,2012-12-23,2012-12-28,Sat       ,2012M12,Dec-2012,N,2012M12   ,N,Year 2012 - Quarter 04,2012Q04   ,N,52,2012,2012-12-22,2012-12-28,N,Wk.52 - 13,2012-12-22 00:00:00,2012-12-28 00:00:00,2012W52   ,2012W52   ,12/22/13 - 12/28/13,12/22 - 12/28,2012,N,2012-12-28,2011-12-28,2012-09-28,2012-06-28,2012-11-28,2012-10-28,2012-12-21,2012-12-14,0,0,0,0,0,0,0,0,12,4,52,1,0,1,0,2005-09-07,USER_X  ,2012-11-27 00:16:56,USER_X
+2012-12-03,2012-01-01,2012-10-01,2012-12-01,2012-12-01,0,0,1,1,6,0,0,0,1,1,41610,337,64,3,3,49,5944,3,12,1368,4,456,114,2012-12-31,2012-12-31,2012-12-31,2012-12-07,03-Dec-2012,Dec 3rd 2012,Tue 12-03-13,0,0,0,0,2012-09-21,365,92,31,7,2012-12-30,2012-09-29,2012-11-24,2012-12-01,2012-12-02,2012-12-03,Tue       ,2012M12,Dec-2012,N,2012M12   ,N,Year 2012 - Quarter 04,2012Q04   ,N,49,2012,2012-12-01,2012-12-07,N,Wk.49 - 13,2012-12-01 00:00:00,2012-12-07 00:00:00,2012W49   ,2012W49   ,12/01/13 - 12/07/13,12/01 - 12/07,2012,N,2012-12-03,2011-12-03,2012-09-03,2012-06-03,2012-11-03,2012-10-03,2012-11-26,2012-11-19,0,0,0,0,0,0,0,0,12,4,49,4,0,1,0,2005-09-07,USER_X  ,2012-11-27 00:16:56,USER_X
+2012-07-15,2012-01-01,2012-07-01,2012-07-01,2012-07-14,0,-1,-4,-19,-135,0,-1,-5,-19,-19,41469,196,15,15,2,29,5924,1,7,1363,3,455,114,2012-12-31,2012-09-30,2012-07-31,2012-07-20,15-Jul-2012,Jul 15th 2012,Mon 07-15-13,1,0,0,0,2012-06-21,365,92,31,7,2012-12-30,2012-06-30,2012-06-30,2012-07-14,2012-07-15,2012-07-15,Mon       ,2012M07,Jul-2012,N,2012M07   ,N,Year 2012 - Quarter 03,2012Q03   ,N,29,2012,2012-07-14,2012-07-20,N,Wk.29 - 13,2012-07-14 00:00:00,2012-07-20 00:00:00,2012W29   ,2012W29   ,07/14/13 - 07/20/13,07/14 - 07/20,2012,N,2012-07-15,2011-07-15,2012-04-15,2012-01-15,2012-06-15,2012-05-15,2012-07-08,2012-07-01,0,0,0,0,0,0,0,0,7,3,29,3,1,1,1,2005-09-07,USER_X  ,2012-11-27 00:16:56,USER_X
+2012-05-13,2012-01-01,2012-04-01,2012-05-01,2012-05-12,0,-2,-6,-28,-198,0,-2,-7,-28,-28,41406,133,43,13,2,20,5915,2,5,1361,2,454,114,2012-12-31,2012-06-30,2012-05-31,2012-05-18,13-May-2012,May 13th 2012,Mon 05-13-13,1,0,0,0,2012-03-21,365,91,31,7,2012-12-30,2012-03-31,2012-04-28,2012-05-12,2012-05-13,2012-05-13,Mon       ,2012M05,May-2012,N,2012M05   ,N,Year 2012 - Quarter 02,2012Q02   ,N,20,2012,2012-05-12,2012-05-18,N,Wk.20 - 13,2012-05-12 00:00:00,2012-05-18 00:00:00,2012W20   ,2012W20   ,05/12/13 - 05/18/13,05/12 - 05/18,2012,N,2012-05-13,2011-05-13,2012-02-13,2012-11-13,2012-04-13,2012-03-13,2012-05-06,2012-04-29,0,0,0,0,0,0,0,0,5,2,20,2,1,1,1,2005-09-07,USER_X  ,2012-11-27 00:16:56,USER_X
+2012-03-30,2012-01-01,2012-01-01,2012-03-01,2012-03-24,0,-3,-8,-35,-242,0,-3,-9,-35,-35,41362,89,89,30,7,13,5908,3,3,1359,1,453,114,2012-12-31,2012-03-31,2012-03-31,2012-03-30,30-Mar-2012,Mar 30th 2012,Sat 03-30-13,1,0,0,0,2012-03-21,365,90,31,7,2012-12-30,2012-12-30,2012-02-24,2012-03-24,2012-03-25,2012-03-30,Sat       ,2012M03,Mar-2012,N,2012M03   ,N,Year 2012 - Quarter 01,2012Q01   ,N,13,2012,2012-03-24,2012-03-30,N,Wk.13 - 13,2012-03-24 00:00:00,2012-03-30 00:00:00,2012W13   ,2012W13   ,03/24/13 - 03/30/13,03/24 - 03/30,2012,N,2012-03-30,2011-03-30,2012-12-30,2012-09-30,2012-02-28,2012-01-30,2012-03-23,2012-03-16,0,0,0,0,0,0,0,0,3,1,13,2,1,1,1,2005-09-07,USER_X  ,2012-11-27 00:16:56,USER_X
+2012-12-09,2012-01-01,2012-10-01,2012-12-01,2012-12-08,0,0,1,2,12,0,0,0,2,2,41616,343,70,9,2,50,5945,3,12,1368,4,456,114,2012-12-31,2012-12-31,2012-12-31,2012-12-14,09-Dec-2012,Dec 9th 2012,Mon 12-09-13,0,0,0,0,2012-09-21,365,92,31,7,2012-12-30,2012-09-29,2012-11-24,2012-12-08,2012-12-09,2012-12-09,Mon       ,2012M12,Dec-2012,N,2012M12   ,N,Year 2012 - Quarter 04,2012Q04   ,N,50,2012,2012-12-08,2012-12-14,N,Wk.50 - 13,2012-12-08 00:00:00,2012-12-14 00:00:00,2012W50   ,2012W50   ,12/08/13 - 12/14/13,12/08 - 12/14,2012,N,2012-12-09,2011-12-09,2012-09-09,2012-06-09,2012-11-09,2012-10-09,2012-12-02,2012-11-25,0,0,0,0,0,0,0,0,12,4,50,4,0,1,0,2005-09-07,USER_X  ,2012-11-27 00:16:56,USER_X
+2012-02-25,2012-01-01,2012-01-01,2012-02-01,2012-02-24,0,-3,-9,-39,-275,0,-3,-9,-39,-39,41329,56,56,25,2,9,5904,2,2,1358,1,453,114,2012-12-31,2012-03-31,2012-02-28,2012-03-02,25-Feb-2012,Feb 25th 2012,Mon 02-25-13,1,0,0,0,2012-12-21,365,90,28,7,2012-12-30,2012-12-30,2012-02-24,2012-02-24,2012-02-25,2012-02-25,Mon       ,2012M02,Feb-2012,N,2012M02   ,N,Year 2012 - Quarter 01,2012Q01   ,N,9,2012,2012-02-24,2012-03-02,N,Wk.09 - 13,2012-02-24 00:00:00,2012-03-02 00:00:00,2012W09   ,2012W09   ,02/24/13 - 03/02/13,02/24 - 03/02,2012,N,2012-02-25,2011-02-25,2012-11-25,2012-08-25,2012-01-25,2012-12-25,2012-02-18,2012-02-11,0,0,0,0,0,0,0,0,3,1,9,1,1,1,1,2005-09-07,USER_X  ,2012-11-27 00:16:56,USER_X
+2012-11-04,2012-01-01,2012-10-01,2012-11-01,2012-11-03,0,0,0,-3,-23,0,0,-1,-3,-3,41581,308,35,4,2,45,5940,2,11,1367,4,456,114,2012-12-31,2012-12-31,2012-11-30,2012-11-09,04-Nov-2012,Nov 4th 2012,Mon 11-04-13,1,1,1,0,2012-09-21,365,92,30,7,2012-12-30,2012-09-29,2012-10-27,2012-11-03,2012-11-04,2012-11-04,Mon       ,2012M11,Nov-2012,N,2012M11   ,N,Year 2012 - Quarter 04,2012Q04   ,N,45,2012,2012-11-03,2012-11-09,N,Wk.45 - 13,2012-11-03 00:00:00,2012-11-09 00:00:00,2012W45   ,2012W45   ,11/03/13 - 11/09/13,11/03 - 11/09,2012,N,2012-11-04,2011-11-04,2012-08-04,2012-05-04,2012-10-04,2012-09-04,2012-10-28,2012-10-21,0,0,0,0,0,0,0,0,11,4,45,4,1,1,1,2005-09-07,USER_X  ,2012-11-27 00:16:56,USER_X
+2012-11-08,2012-01-01,2012-10-01,2012-11-01,2012-11-03,0,0,0,-3,-19,0,0,-1,-3,-3,41585,312,39,8,6,45,5940,2,11,1367,4,456,114,2012-12-31,2012-12-31,2012-11-30,2012-11-09,08-Nov-2012,Nov 8th 2012,Fri 11-08-13,1,1,1,0,2012-09-21,365,92,30,7,2012-12-30,2012-09-29,2012-10-27,2012-11-03,2012-11-04,2012-11-08,Fri       ,2012M11,Nov-2012,N,2012M11   ,N,Year 2012 - Quarter 04,2012Q04   ,N,45,2012,2012-11-03,2012-11-09,N,Wk.45 - 13,2012-11-03 00:00:00,2012-11-09 00:00:00,2012W45   ,2012W45   ,11/03/13 - 11/09/13,11/03 - 11/09,2012,N,2012-11-08,2011-11-08,2012-08-08,2012-05-08,2012-10-08,2012-09-08,2012-11-01,2012-10-25,0,0,0,0,0,0,0,0,11,4,45,4,1,1,1,2005-09-07,USER_X  ,2012-11-27 00:16:56,USER_X
+2012-08-22,2012-01-01,2012-07-01,2012-08-01,2012-08-18,0,-1,-3,-14,-97,0,-1,-4,-14,-14,41507,234,53,22,5,34,5929,2,8,1364,3,455,114,2012-12-31,2012-09-30,2012-08-31,2012-08-24,22-Aug-2012,Aug 22nd 2012,Thu 08-22-13,1,0,0,0,2012-06-21,365,92,31,7,2012-12-30,2012-06-30,2012-07-28,2012-08-18,2012-08-19,2012-08-22,Thu       ,2012M08,Aug-2012,N,2012M08   ,N,Year 2012 - Quarter 03,2012Q03   ,N,34,2012,2012-08-18,2012-08-24,N,Wk.34 - 13,2012-08-18 00:00:00,2012-08-24 00:00:00,2012W34   ,2012W34   ,08/18/13 - 08/24/13,08/18 - 08/24,2012,N,2012-08-22,2011-08-22,2012-05-22,2012-02-22,2012-07-22,2012-06-22,2012-08-15,2012-08-08,0,0,0,0,0,0,0,0,8,3,34,3,1,1,1,2005-09-07,USER_X  ,2012-11-27 00:16:56,USER_X
+2012-06-18,2012-01-01,2012-04-01,2012-06-01,2012-06-16,0,-2,-5,-23,-162,0,-2,-6,-23,-23,41442,169,79,18,3,25,5920,3,6,1362,2,454,114,2012-12-31,2012-06-30,2012-06-30,2012-06-22,18-Jun-2012,Jun 18th 2012,Tue 06-18-13,1,0,0,0,2012-03-21,365,91,30,7,2012-12-30,2012-03-31,2012-05-26,2012-06-16,2012-06-17,2012-06-18,Tue       ,2012M06,Jun-2012,N,2012M06   ,N,Year 2012 - Quarter 02,2012Q02   ,N,25,2012,2012-06-16,2012-06-22,N,Wk.25 - 13,2012-06-16 00:00:00,2012-06-22 00:00:00,2012W25   ,2012W25   ,06/16/13 - 06/22/13,06/16 - 06/22,2012,N,2012-06-18,2011-06-18,2012-03-18,2012-12-18,2012-05-18,2012-04-18,2012-06-11,2012-06-04,0,0,0,0,0,0,0,0,6,2,25,2,1,1,1,2005-09-07,USER_X  ,2012-11-27 00:16:56,USER_X
+2012-07-19,2012-01-01,2012-07-01,2012-07-01,2012-07-14,0,-1,-4,-19,-131,0,-1,-5,-19,-19,41473,200,19,19,6,29,5924,1,7,1363,3,455,114,2012-12-31,2012-09-30,2012-07-31,2012-07-20,19-Jul-2012,Jul 19th 2012,Fri 07-19-13,1,0,0,0,2012-06-21,365,92,31,7,2012-12-30,2012-06-30,2012-06-30,2012-07-14,2012-07-15,2012-07-19,Fri       ,2012M07,Jul-2012,N,2012M07   ,N,Year 2012 - Quarter 03,2012Q03   ,N,29,2012,2012-07-14,2012-07-20,N,Wk.29 - 13,2012-07-14 00:00:00,2012-07-20 00:00:00,2012W29   ,2012W29   ,07/14/13 - 07/20/13,07/14 - 07/20,2012,N,2012-07-19,2011-07-19,2012-04-19,2012-01-19,2012-06-19,2012-05-19,2012-07-12,2012-07-05,0,0,0,0,0,0,0,0,7,3,29,3,1,1,1,2005-09-07,USER_X  ,2012-11-27 00:16:56,USER_X
+2012-08-20,2012-01-01,2012-07-01,2012-08-01,2012-08-18,0,-1,-3,-14,-99,0,-1,-4,-14,-14,41505,232,51,20,3,34,5929,2,8,1364,3,455,114,2012-12-31,2012-09-30,2012-08-31,2012-08-24,20-Aug-2012,Aug 20th 2012,Tue 08-20-13,1,0,0,0,2012-06-21,365,92,31,7,2012-12-30,2012-06-30,2012-07-28,2012-08-18,2012-08-19,2012-08-20,Tue       ,2012M08,Aug-2012,N,2012M08   ,N,Year 2012 - Quarter 03,2012Q03   ,N,34,2012,2012-08-18,2012-08-24,N,Wk.34 - 13,2012-08-18 00:00:00,2012-08-24 00:00:00,2012W34   ,2012W34   ,08/18/13 - 08/24/13,08/18 - 08/24,2012,N,2012-08-20,2011-08-20,2012-05-20,2012-02-20,2012-07-20,2012-06-20,2012-08-13,2012-08-06,0,0,0,0,0,0,0,0,8,3,34,3,1,1,1,2005-09-07,USER_X  ,2012-11-27 00:16:56,USER_X
+2012-08-24,2012-01-01,2012-07-01,2012-08-01,2012-08-18,0,-1,-3,-14,-95,0,-1,-4,-14,-14,41509,236,55,24,7,34,5929,2,8,1364,3,455,114,2012-12-31,2012-09-30,2012-08-31,2012-08-24,24-Aug-2012,Aug 24th 2012,Sat 08-24-13,1,0,0,0,2012-06-21,365,92,31,7,2012-12-30,2012-06-30,2012-07-28,2012-08-18,2012-08-19,2012-08-24,Sat       ,2012M08,Aug-2012,N,2012M08   ,N,Year 2012 - Quarter 03,2012Q03   ,N,34,2012,2012-08-18,2012-08-24,N,Wk.34 - 13,2012-08-18 00:00:00,2012-08-24 00:00:00,2012W34   ,2012W34   ,08/18/13 - 08/24/13,08/18 - 08/24,2012,N,2012-08-24,2011-08-24,2012-05-24,2012-02-24,2012-07-24,2012-06-24,2012-08-17,2012-08-10,0,0,0,0,0,0,0,0,8,3,34,3,1,1,1,2005-09-07,USER_X  ,2012-11-27 00:16:56,USER_X
+2012-07-13,2012-01-01,2012-07-01,2012-07-01,2012-07-07,0,-1,-4,-20,-137,0,-1,-5,-20,-20,41467,194,13,13,7,28,5923,1,7,1363,3,455,114,2012-12-31,2012-09-30,2012-07-31,2012-07-13,13-Jul-2012,Jul 13th 2012,Sat 07-13-13,1,0,0,0,2012-06-21,365,92,31,7,2012-12-30,2012-06-30,2012-06-30,2012-07-07,2012-07-08,2012-07-13,Sat       ,2012M07,Jul-2012,N,2012M07   ,N,Year 2012 - Quarter 03,2012Q03   ,N,28,2012,2012-07-07,2012-07-13,N,Wk.28 - 13,2012-07-07 00:00:00,2012-07-13 00:00:00,2012W28   ,2012W28   ,07/07/13 - 07/13/13,07/07 - 07/13,2012,N,2012-07-13,2011-07-13,2012-04-13,2012-01-13,2012-06-13,2012-05-13,2012-07-06,2012-06-29,0,0,0,0,0,0,0,0,7,3,28,3,1,1,1,2005-09-07,USER_X  ,2012-11-27 00:16:56,USER_X
+2012-11-06,2012-01-01,2012-10-01,2012-11-01,2012-11-03,0,0,0,-3,-21,0,0,-1,-3,-3,41583,310,37,6,4,45,5940,2,11,1367,4,456,114,2012-12-31,2012-12-31,2012-11-30,2012-11-09,06-Nov-2012,Nov 6th 2012,Wed 11-06-13,1,1,1,0,2012-09-21,365,92,30,7,2012-12-30,2012-09-29,2012-10-27,2012-11-03,2012-11-04,2012-11-06,Wed       ,2012M11,Nov-2012,N,2012M11   ,N,Year 2012 - Quarter 04,2012Q04   ,N,45,2012,2012-11-03,2012-11-09,N,Wk.45 - 13,2012-11-03 00:00:00,2012-11-09 00:00:00,2012W45   ,2012W45   ,11/03/13 - 11/09/13,11/03 - 11/09,2012,N,2012-11-06,2011-11-06,2012-08-06,2012-05-06,2012-10-06,2012-09-06,2012-10-30,2012-10-23,0,0,0,0,0,0,0,0,11,4,45,4,1,1,1,2005-09-07,USER_X  ,2012-11-27 00:16:56,USER_X
+2012-12-07,2012-01-01,2012-10-01,2012-12-01,2012-12-01,0,0,1,1,10,0,0,0,1,1,41614,341,68,7,7,49,5944,3,12,1368,4,456,114,2012-12-31,2012-12-31,2012-12-31,2012-12-07,07-Dec-2012,Dec 7th 2012,Sat 12-07-13,0,0,0,0,2012-09-21,365,92,31,7,2012-12-30,2012-09-29,2012-11-24,2012-12-01,2012-12-02,2012-12-07,Sat       ,2012M12,Dec-2012,N,2012M12   ,N,Year 2012 - Quarter 04,2012Q04   ,N,49,2012,2012-12-01,2012-12-07,N,Wk.49 - 13,2012-12-01 00:00:00,2012-12-07 00:00:00,2012W49   ,2012W49   ,12/01/13 - 12/07/13,12/01 - 12/07,2012,N,2012-12-07,2011-12-07,2012-09-07,2012-06-07,2012-11-07,2012-10-07,2012-11-30,2012-11-23,0,0,0,0,0,0,0,0,12,4,49,4,0,1,0,2005-09-07,USER_X  ,2012-11-27 00:16:56,USER_X
+2012-07-04,2012-01-01,2012-07-01,2012-07-01,2012-06-30,0,-1,-4,-21,-146,0,-1,-5,-21,-21,41458,185,4,4,5,27,5922,1,7,1363,3,455,114,2012-12-31,2012-09-30,2012-07-31,2012-07-06,04-Jul-2012,Jul 4th 2012,Thu 07-04-13,1,0,0,0,2012-06-21,365,92,31,7,2012-12-30,2012-06-30,2012-06-30,2012-06-30,2012-07-01,2012-07-04,Thu       ,2012M07,Jul-2012,N,2012M07   ,N,Year 2012 - Quarter 03,2012Q03   ,N,27,2012,2012-06-30,2012-07-06,N,Wk.27 - 13,2012-06-30 00:00:00,2012-07-06 00:00:00,2012W27   ,2012W27   ,06/30/13 - 07/06/13,06/30 - 07/06,2012,N,2012-07-04,2011-07-04,2012-04-04,2012-01-04,2012-06-04,2012-05-04,2012-06-27,2012-06-20,0,0,0,0,0,0,0,0,7,3,27,3,1,1,1,2005-09-07,USER_X  ,2012-11-27 00:16:56,USER_X
+2012-10-07,2012-01-01,2012-10-01,2012-10-01,2012-10-06,0,0,-1,-7,-51,0,0,-2,-7,-7,41553,280,7,7,2,41,5936,1,10,1366,4,456,114,2012-12-31,2012-12-31,2012-10-31,2012-10-12,07-Oct-2012,Oct 7th 2012,Mon 10-07-13,1,1,0,0,2012-09-21,365,92,31,7,2012-12-30,2012-09-29,2012-09-29,2012-10-06,2012-10-07,2012-10-07,Mon       ,2012M10,Oct-2012,N,2012M10   ,N,Year 2012 - Quarter 04,2012Q04   ,N,41,2012,2012-10-06,2012-10-12,N,Wk.41 - 13,2012-10-06 00:00:00,2012-10-12 00:00:00,2012W41   ,2012W41   ,10/06/13 - 10/12/13,10/06 - 10/12,2012,N,2012-10-07,2011-10-07,2012-07-07,2012-04-07,2012-09-07,2012-08-07,2012-09-30,2012-09-23,0,0,0,0,0,0,0,0,10,4,41,4,1,1,1,2005-09-07,USER_X  ,2012-11-27 00:16:56,USER_X
+2012-11-10,2012-01-01,2012-10-01,2012-11-01,2012-11-10,0,0,0,-2,-17,0,0,-1,-2,-3,41587,314,41,10,1,46,5941,2,11,1367,4,456,114,2012-12-31,2012-12-31,2012-11-30,2012-11-16,10-Nov-2012,Nov 10th 2012,Sun 11-10-13,1,1,1,0,2012-09-21,365,92,30,7,2012-12-30,2012-09-29,2012-10-27,2012-11-10,2012-11-04,2012-11-10,Sun       ,2012M11,Nov-2012,N,2012M11   ,N,Year 2012 - Quarter 04,2012Q04   ,N,46,2012,2012-11-10,2012-11-16,Y,Wk.46 - 13,2012-11-10 00:00:00,2012-11-16 00:00:00,2012W46   ,2012W46   ,11/10/13 - 11/16/13,11/10 - 11/16,2012,N,2012-11-10,2011-11-10,2012-08-10,2012-05-10,2012-10-10,2012-09-10,2012-11-03,2012-10-27,0,0,0,0,0,0,0,0,11,4,46,4,1,1,1,2005-09-07,USER_X  ,2012-11-27 00:16:56,USER_X
+2012-08-07,2012-01-01,2012-07-01,2012-08-01,2012-08-04,0,-1,-3,-16,-112,0,-1,-4,-16,-16,41492,219,38,7,4,32,5927,2,8,1364,3,455,114,2012-12-31,2012-09-30,2012-08-31,2012-08-10,07-Aug-2012,Aug 7th 2012,Wed 08-07-13,1,0,0,0,2012-06-21,365,92,31,7,2012-12-30,2012-06-30,2012-07-28,2012-08-04,2012-08-05,2012-08-07,Wed       ,2012M08,Aug-2012,N,2012M08   ,N,Year 2012 - Quarter 03,2012Q03   ,N,32,2012,2012-08-04,2012-08-10,N,Wk.32 - 13,2012-08-04 00:00:00,2012-08-10 00:00:00,2012W32   ,2012W32   ,08/04/13 - 08/10/13,08/04 - 08/10,2012,N,2012-08-07,2011-08-07,2012-05-07,2012-02-07,2012-07-07,2012-06-07,2012-07-31,2012-07-24,0,0,0,0,0,0,0,0,8,3,32,3,1,1,1,2005-09-07,USER_X  ,2012-11-27 00:16:56,USER_X
+2012-09-25,2012-01-01,2012-07-01,2012-09-01,2012-09-22,0,-1,-2,-9,-63,0,-1,-3,-9,-9,41541,268,87,25,4,39,5934,3,9,1365,3,455,114,2012-12-31,2012-09-30,2012-09-30,2012-09-28,25-Sep-2012,Sep 25th 2012,Wed 09-25-13,1,0,0,0,2012-09-21,365,92,30,7,2012-12-30,2012-06-30,2012-08-25,2012-09-22,2012-09-23,2012-09-25,Wed       ,2012M09,Sep-2012,N,2012M09   ,N,Year 2012 - Quarter 03,2012Q03   ,N,39,2012,2012-09-22,2012-09-28,N,Wk.39 - 13,2012-09-22 00:00:00,2012-09-28 00:00:00,2012W39   ,2012W39   ,09/22/13 - 09/28/13,09/22 - 09/28,2012,N,2012-09-25,2011-09-25,2012-06-25,2012-03-25,2012-08-25,2012-07-25,2012-09-18,2012-09-11,0,0,0,0,0,0,0,0,9,3,39,4,1,1,1,2005-09-07,USER_X  ,2012-11-27 00:16:56,USER_X
+2012-10-05,2012-01-01,2012-10-01,2012-10-01,2012-09-29,0,0,-1,-8,-53,0,0,-2,-8,-8,41551,278,5,5,7,40,5935,1,10,1366,4,456,114,2012-12-31,2012-12-31,2012-10-31,2012-10-05,05-Oct-2012,Oct 5th 2012,Sat 10-05-13,1,1,0,0,2012-09-21,365,92,31,7,2012-12-30,2012-09-29,2012-09-29,2012-09-29,2012-09-30,2012-10-05,Sat       ,2012M10,Oct-2012,N,2012M10   ,N,Year 2012 - Quarter 04,2012Q04   ,N,40,2012,2012-09-29,2012-10-05,N,Wk.40 - 13,2012-09-29 00:00:00,2012-10-05 00:00:00,2012W40   ,2012W40   ,09/29/13 - 10/05/13,09/29 - 10/05,2012,N,2012-10-05,2011-10-05,2012-07-05,2012-04-05,2012-09-05,2012-08-05,2012-09-28,2012-09-21,0,0,0,0,0,0,0,0,10,4,40,4,1,1,1,2005-09-07,USER_X  ,2012-11-27 00:16:56,USER_X
+2012-09-02,2012-01-01,2012-07-01,2012-09-01,2012-09-01,0,-1,-2,-12,-86,0,-1,-3,-12,-12,41518,245,64,2,2,36,5931,3,9,1365,3,455,114,2012-12-31,2012-09-30,2012-09-30,2012-09-07,02-Sep-2012,Sep 2nd 2012,Mon 09-02-13,1,0,0,0,2012-06-21,365,92,30,7,2012-12-30,2012-06-30,2012-08-25,2012-09-01,2012-09-02,2012-09-02,Mon       ,2012M09,Sep-2012,N,2012M09   ,N,Year 2012 - Quarter 03,2012Q03   ,N,36,2012,2012-09-01,2012-09-07,N,Wk.36 - 13,2012-09-01 00:00:00,2012-09-07 00:00:00,2012W36   ,2012W36   ,09/01/13 - 09/07/13,09/01 - 09/07,2012,N,2012-09-02,2011-09-02,2012-06-02,2012-03-02,2012-08-02,2012-07-02,2012-08-26,2012-08-19,0,0,0,0,0,0,0,0,9,3,36,3,1,1,1,2005-09-07,USER_X  ,2012-11-27 00:16:56,USER_X
+2012-05-15,2012-01-01,2012-04-01,2012-05-01,2012-05-12,0,-2,-6,-28,-196,0,-2,-7,-28,-28,41408,135,45,15,4,20,5915,2,5,1361,2,454,114,2012-12-31,2012-06-30,2012-05-31,2012-05-18,15-May-2012,May 15th 2012,Wed 05-15-13,1,0,0,0,2012-03-21,365,91,31,7,2012-12-30,2012-03-31,2012-04-28,2012-05-12,2012-05-13,2012-05-15,Wed       ,2012M05,May-2012,N,2012M05   ,N,Year 2012 - Quarter 02,2012Q02   ,N,20,2012,2012-05-12,2012-05-18,N,Wk.20 - 13,2012-05-12 00:00:00,2012-05-18 00:00:00,2012W20   ,2012W20   ,05/12/13 - 05/18/13,05/12 - 05/18,2012,N,2012-05-15,2011-05-15,2012-02-15,2012-11-15,2012-04-15,2012-03-15,2012-05-08,2012-05-01,0,0,0,0,0,0,0,0,5,2,20,2,1,1,1,2005-09-07,USER_X  ,2012-11-27 00:16:56,USER_X
+2012-02-10,2012-01-01,2012-01-01,2012-02-01,2012-02-10,0,-3,-9,-41,-290,0,-3,-10,-41,-42,41314,41,41,10,1,7,5902,2,2,1358,1,453,114,2012-12-31,2012-03-31,2012-02-28,2012-02-16,10-Feb-2012,Feb 10th 2012,Sun 02-10-13,1,0,0,0,2012-12-21,365,90,28,7,2012-12-30,2012-12-30,2012-01-27,2012-02-10,2012-02-04,2012-02-10,Sun       ,2012M02,Feb-2012,N,2012M02   ,N,Year 2012 - Quarter 01,2012Q01   ,N,7,2012,2012-02-10,2012-02-16,Y,Wk.07 - 13,2012-02-10 00:00:00,2012-02-16 00:00:00,2012W07   ,2012W07   ,02/10/13 - 02/16/13,02/10 - 02/16,2012,N,2012-02-10,2011-02-10,2012-11-10,2012-08-10,2012-01-10,2012-12-10,2012-02-03,2012-01-27,0,0,0,0,0,0,0,0,2,1,7,1,1,1,1,2005-09-07,USER_X  ,2012-11-27 00:16:56,USER_X
+2012-12-21,2012-01-01,2012-10-01,2012-12-01,2012-12-15,0,0,1,3,24,0,0,0,3,3,41628,355,82,21,7,51,5946,3,12,1368,4,456,114,2012-12-31,2012-12-31,2012-12-31,2012-12-21,21-Dec-2012,Dec 21st 2012,Sat 12-21-13,0,0,0,0,2012-12-21,365,92,31,7,2012-12-30,2012-09-29,2012-11-24,2012-12-15,2012-12-16,2012-12-21,Sat       ,2012M12,Dec-2012,N,2012M12   ,N,Year 2012 - Quarter 04,2012Q04   ,N,51,2012,2012-12-15,2012-12-21,N,Wk.51 - 13,2012-12-15 00:00:00,2012-12-21 00:00:00,2012W51   ,2012W51   ,12/15/13 - 12/21/13,12/15 - 12/21,2012,N,2012-12-21,2011-12-21,2012-09-21,2012-06-21,2012-11-21,2012-10-21,2012-12-14,2012-12-07,0,0,0,0,0,0,0,0,12,4,51,1,0,1,0,2005-09-07,USER_X  ,2012-11-27 00:16:56,USER_X
+2012-10-22,2012-01-01,2012-10-01,2012-10-01,2012-10-20,0,0,-1,-5,-36,0,0,-2,-5,-5,41568,295,22,22,3,43,5938,1,10,1366,4,456,114,2012-12-31,2012-12-31,2012-10-31,2012-10-26,22-Oct-2012,Oct 22nd 2012,Tue 10-22-13,1,1,0,0,2012-09-21,365,92,31,7,2012-12-30,2012-09-29,2012-09-29,2012-10-20,2012-10-21,2012-10-22,Tue       ,2012M10,Oct-2012,N,2012M10   ,N,Year 2012 - Quarter 04,2012Q04   ,N,43,2012,2012-10-20,2012-10-26,N,Wk.43 - 13,2012-10-20 00:00:00,2012-10-26 00:00:00,2012W43   ,2012W43   ,10/20/13 - 10/26/13,10/20 - 10/26,2012,N,2012-10-22,2011-10-22,2012-07-22,2012-04-22,2012-09-22,2012-08-22,2012-10-15,2012-10-08,0,0,0,0,0,0,0,0,10,4,43,4,1,1,1,2005-09-07,USER_X  ,2012-11-27 00:16:56,USER_X
+2012-10-24,2012-01-01,2012-10-01,2012-10-01,2012-10-20,0,0,-1,-5,-34,0,0,-2,-5,-5,41570,297,24,24,5,43,5938,1,10,1366,4,456,114,2012-12-31,2012-12-31,2012-10-31,2012-10-26,24-Oct-2012,Oct 24th 2012,Thu 10-24-13,1,1,0,0,2012-09-21,365,92,31,7,2012-12-30,2012-09-29,2012-09-29,2012-10-20,2012-10-21,2012-10-24,Thu       ,2012M10,Oct-2012,N,2012M10   ,N,Year 2012 - Quarter 04,2012Q04   ,N,43,2012,2012-10-20,2012-10-26,N,Wk.43 - 13,2012-10-20 00:00:00,2012-10-26 00:00:00,2012W43   ,2012W43   ,10/20/13 - 10/26/13,10/20 - 10/26,2012,N,2012-10-24,2011-10-24,2012-07-24,2012-04-24,2012-09-24,2012-08-24,2012-10-17,2012-10-10,0,0,0,0,0,0,0,0,10,4,43,4,1,1,1,2005-09-07,USER_X  ,2012-11-27 00:16:56,USER_X
+2012-07-10,2012-01-01,2012-07-01,2012-07-01,2012-07-07,0,-1,-4,-20,-140,0,-1,-5,-20,-20,41464,191,10,10,4,28,5923,1,7,1363,3,455,114,2012-12-31,2012-09-30,2012-07-31,2012-07-13,10-Jul-2012,Jul 10th 2012,Wed 07-10-13,1,0,0,0,2012-06-21,365,92,31,7,2012-12-30,2012-06-30,2012-06-30,2012-07-07,2012-07-08,2012-07-10,Wed       ,2012M07,Jul-2012,N,2012M07   ,N,Year 2012 - Quarter 03,2012Q03   ,N,28,2012,2012-07-07,2012-07-13,N,Wk.28 - 13,2012-07-07 00:00:00,2012-07-13 00:00:00,2012W28   ,2012W28   ,07/07/13 - 07/13/13,07/07 - 07/13,2012,N,2012-07-10,2011-07-10,2012-04-10,2012-01-10,2012-06-10,2012-05-10,2012-07-03,2012-06-26,0,0,0,0,0,0,0,0,7,3,28,3,1,1,1,2005-09-07,USER_X  ,2012-11-27 00:16:56,USER_X
+2012-02-27,2012-01-01,2012-01-01,2012-02-01,2012-02-24,0,-3,-9,-39,-273,0,-3,-9,-39,-39,41331,58,58,27,4,9,5904,2,2,1358,1,453,114,2012-12-31,2012-03-31,2012-02-28,2012-03-02,27-Feb-2012,Feb 27th 2012,Wed 02-27-13,1,0,0,0,2012-12-21,365,90,28,7,2012-12-30,2012-12-30,2012-02-24,2012-02-24,2012-02-25,2012-02-27,Wed       ,2012M02,Feb-2012,N,2012M02   ,N,Year 2012 - Quarter 01,2012Q01   ,N,9,2012,2012-02-24,2012-03-02,N,Wk.09 - 13,2012-02-24 00:00:00,2012-03-02 00:00:00,2012W09   ,2012W09   ,02/24/13 - 03/02/13,02/24 - 03/02,2012,N,2012-02-27,2011-02-27,2012-11-27,2012-08-27,2012-01-27,2012-12-27,2012-02-20,2012-02-13,0,0,0,0,0,0,0,0,3,1,9,1,1,1,1,2005-09-07,USER_X  ,2012-11-27 00:16:56,USER_X
+2012-03-02,2012-01-01,2012-01-01,2012-03-01,2012-02-24,0,-3,-8,-39,-270,0,-3,-9,-39,-39,41334,61,61,2,7,9,5904,3,3,1359,1,453,114,2012-12-31,2012-03-31,2012-03-31,2012-03-02,02-Mar-2012,Mar 2nd 2012,Sat 03-02-13,1,0,0,0,2012-12-21,365,90,31,7,2012-12-30,2012-12-30,2012-02-24,2012-02-24,2012-02-25,2012-03-02,Sat       ,2012M03,Mar-2012,N,2012M03   ,N,Year 2012 - Quarter 01,2012Q01   ,N,9,2012,2012-02-24,2012-03-02,N,Wk.09 - 13,2012-02-24 00:00:00,2012-03-02 00:00:00,2012W09   ,2012W09   ,02/24/13 - 03/02/13,02/24 - 03/02,2012,N,2012-03-02,2011-03-02,2012-12-02,2012-09-02,2012-02-02,2012-01-02,2012-02-23,2012-02-16,0,0,0,0,0,0,0,0,3,1,9,1,1,1,1,2005-09-07,USER_X  ,2012-11-27 00:16:56,USER_X
+2012-11-02,2012-01-01,2012-10-01,2012-11-01,2012-10-27,0,0,0,-4,-25,0,0,-1,-4,-4,41579,306,33,2,7,44,5939,2,11,1367,4,456,114,2012-12-31,2012-12-31,2012-11-30,2012-11-02,02-Nov-2012,Nov 2nd 2012,Sat 11-02-13,1,1,1,0,2012-09-21,365,92,30,7,2012-12-30,2012-09-29,2012-10-27,2012-10-27,2012-10-28,2012-11-02,Sat       ,2012M11,Nov-2012,N,2012M11   ,N,Year 2012 - Quarter 04,2012Q04   ,N,44,2012,2012-10-27,2012-11-02,N,Wk.44 - 13,2012-10-27 00:00:00,2012-11-02 00:00:00,2012W44   ,2012W44   ,10/27/13 - 11/02/13,10/27 - 11/02,2012,N,2012-11-02,2011-11-02,2012-08-02,2012-05-02,2012-10-02,2012-09-02,2012-10-26,2012-10-19,0,0,0,0,0,0,0,0,11,4,44,4,1,1,1,2005-09-07,USER_X  ,2012-11-27 00:16:56,USER_X
+2012-03-21,2012-01-01,2012-01-01,2012-03-01,2012-03-17,0,-3,-8,-36,-251,0,-3,-9,-36,-36,41353,80,80,21,5,12,5907,3,3,1359,1,453,114,2012-12-31,2012-03-31,2012-03-31,2012-03-23,21-Mar-2012,Mar 21st 2012,Thu 03-21-13,1,0,0,0,2012-03-21,365,90,31,7,2012-12-30,2012-12-30,2012-02-24,2012-03-17,2012-03-18,2012-03-21,Thu       ,2012M03,Mar-2012,N,2012M03   ,N,Year 2012 - Quarter 01,2012Q01   ,N,12,2012,2012-03-17,2012-03-23,N,Wk.12 - 13,2012-03-17 00:00:00,2012-03-23 00:00:00,2012W12   ,2012W12   ,03/17/13 - 03/23/13,03/17 - 03/23,2012,N,2012-03-21,2011-03-21,2012-12-21,2012-09-21,2012-02-21,2012-01-21,2012-03-14,2012-03-07,0,0,0,0,0,0,0,0,3,1,12,2,1,1,1,2005-09-07,USER_X  ,2012-11-27 00:16:56,USER_X
+2012-04-03,2012-01-01,2012-04-01,2012-04-01,2012-03-31,0,-2,-7,-34,-238,0,-2,-8,-34,-34,41366,93,3,3,4,14,5909,1,4,1360,2,454,114,2012-12-31,2012-06-30,2012-04-30,2012-04-06,03-Apr-2012,Apr 3rd 2012,Wed 04-03-13,1,0,0,0,2012-03-21,365,91,30,7,2012-12-30,2012-03-31,2012-03-31,2012-03-31,2012-04-01,2012-04-03,Wed       ,2012M04,Apr-2012,N,2012M04   ,N,Year 2012 - Quarter 02,2012Q02   ,N,14,2012,2012-03-31,2012-04-06,N,Wk.14 - 13,2012-03-31 00:00:00,2012-04-06 00:00:00,2012W14   ,2012W14   ,03/31/13 - 04/06/13,03/31 - 04/06,2012,N,2012-04-03,2011-04-03,2012-01-03,2012-10-03,2012-03-03,2012-02-03,2012-03-27,2012-03-20,0,0,0,0,0,0,0,0,4,2,14,2,1,1,1,2005-09-07,USER_X  ,2012-11-27 00:16:56,USER_X
+2012-07-06,2012-01-01,2012-07-01,2012-07-01,2012-06-30,0,-1,-4,-21,-144,0,-1,-5,-21,-21,41460,187,6,6,7,27,5922,1,7,1363,3,455,114,2012-12-31,2012-09-30,2012-07-31,2012-07-06,06-Jul-2012,Jul 6th 2012,Sat 07-06-13,1,0,0,0,2012-06-21,365,92,31,7,2012-12-30,2012-06-30,2012-06-30,2012-06-30,2012-07-01,2012-07-06,Sat       ,2012M07,Jul-2012,N,2012M07   ,N,Year 2012 - Quarter 03,2012Q03   ,N,27,2012,2012-06-30,2012-07-06,N,Wk.27 - 13,2012-06-30 00:00:00,2012-07-06 00:00:00,2012W27   ,2012W27   ,06/30/13 - 07/06/13,06/30 - 07/06,2012,N,2012-07-06,2011-07-06,2012-04-06,2012-01-06,2012-06-06,2012-05-06,2012-06-29,2012-06-22,0,0,0,0,0,0,0,0,7,3,27,3,1,1,1,2005-09-07,USER_X  ,2012-11-27 00:16:56,USER_X
+2012-06-26,2012-01-01,2012-04-01,2012-06-01,2012-06-23,0,-2,-5,-22,-154,0,-2,-6,-22,-22,41450,177,87,26,4,26,5921,3,6,1362,2,454,114,2012-12-31,2012-06-30,2012-06-30,2012-06-29,26-Jun-2012,Jun 26th 2012,Wed 06-26-13,1,0,0,0,2012-06-21,365,91,30,7,2012-12-30,2012-03-31,2012-05-26,2012-06-23,2012-06-24,2012-06-26,Wed       ,2012M06,Jun-2012,N,2012M06   ,N,Year 2012 - Quarter 02,2012Q02   ,N,26,2012,2012-06-23,2012-06-29,N,Wk.26 - 13,2012-06-23 00:00:00,2012-06-29 00:00:00,2012W26   ,2012W26   ,06/23/13 - 06/29/13,06/23 - 06/29,2012,N,2012-06-26,2011-06-26,2012-03-26,2012-12-26,2012-05-26,2012-04-26,2012-06-19,2012-06-12,0,0,0,0,0,0,0,0,6,2,26,3,1,1,1,2005-09-07,USER_X  ,2012-11-27 00:16:56,USER_X
+2012-08-26,2012-01-01,2012-07-01,2012-08-01,2012-08-25,0,-1,-3,-13,-93,0,-1,-3,-13,-13,41511,238,57,26,2,35,5930,2,8,1364,3,455,114,2012-12-31,2012-09-30,2012-08-31,2012-08-31,26-Aug-2012,Aug 26th 2012,Mon 08-26-13,1,0,0,0,2012-06-21,365,92,31,7,2012-12-30,2012-06-30,2012-08-25,2012-08-25,2012-08-26,2012-08-26,Mon       ,2012M08,Aug-2012,N,2012M08   ,N,Year 2012 - Quarter 03,2012Q03   ,N,35,2012,2012-08-25,2012-08-31,N,Wk.35 - 13,2012-08-25 00:00:00,2012-08-31 00:00:00,2012W35   ,2012W35   ,08/25/13 - 08/31/13,08/25 - 08/31,2012,N,2012-08-26,2011-08-26,2012-05-26,2012-02-26,2012-07-26,2012-06-26,2012-08-19,2012-08-12,0,0,0,0,0,0,0,0,9,3,35,3,1,1,1,2005-09-07,USER_X  ,2012-11-27 00:16:56,USER_X
+2012-12-13,2012-01-01,2012-10-01,2012-12-01,2012-12-08,0,0,1,2,16,0,0,0,2,2,41620,347,74,13,6,50,5945,3,12,1368,4,456,114,2012-12-31,2012-12-31,2012-12-31,2012-12-14,13-Dec-2012,Dec 13th 2012,Fri 12-13-13,0,0,0,0,2012-09-21,365,92,31,7,2012-12-30,2012-09-29,2012-11-24,2012-12-08,2012-12-09,2012-12-13,Fri       ,2012M12,Dec-2012,N,2012M12   ,N,Year 2012 - Quarter 04,2012Q04   ,N,50,2012,2012-12-08,2012-12-14,N,Wk.50 - 13,2012-12-08 00:00:00,2012-12-14 00:00:00,2012W50   ,2012W50   ,12/08/13 - 12/14/13,12/08 - 12/14,2012,N,2012-12-13,2011-12-13,2012-09-13,2012-06-13,2012-11-13,2012-10-13,2012-12-06,2012-11-29,0,0,0,0,0,0,0,0,12,4,50,4,0,1,0,2005-09-07,USER_X  ,2012-11-27 00:16:56,USER_X
+2012-07-02,2012-01-01,2012-07-01,2012-07-01,2012-06-30,0,-1,-4,-21,-148,0,-1,-5,-21,-21,41456,183,2,2,3,27,5922,1,7,1363,3,455,114,2012-12-31,2012-09-30,2012-07-31,2012-07-06,02-Jul-2012,Jul 2nd 2012,Tue 07-02-13,1,0,0,0,2012-06-21,365,92,31,7,2012-12-30,2012-06-30,2012-06-30,2012-06-30,2012-07-01,2012-07-02,Tue       ,2012M07,Jul-2012,N,2012M07   ,N,Year 2012 - Quarter 03,2012Q03   ,N,27,2012,2012-06-30,2012-07-06,N,Wk.27 - 13,2012-06-30 00:00:00,2012-07-06 00:00:00,2012W27   ,2012W27   ,06/30/13 - 07/06/13,06/30 - 07/06,2012,N,2012-07-02,2011-07-02,2012-04-02,2012-01-02,2012-06-02,2012-05-02,2012-06-25,2012-06-18,0,0,0,0,0,0,0,0,7,3,27,3,1,1,1,2005-09-07,USER_X  ,2012-11-27 00:16:56,USER_X
+2012-08-05,2012-01-01,2012-07-01,2012-08-01,2012-08-04,0,-1,-3,-16,-114,0,-1,-4,-16,-16,41490,217,36,5,2,32,5927,2,8,1364,3,455,114,2012-12-31,2012-09-30,2012-08-31,2012-08-10,05-Aug-2012,Aug 5th 2012,Mon 08-05-13,1,0,0,0,2012-06-21,365,92,31,7,2012-12-30,2012-06-30,2012-07-28,2012-08-04,2012-08-05,2012-08-05,Mon       ,2012M08,Aug-2012,N,2012M08   ,N,Year 2012 - Quarter 03,2012Q03   ,N,32,2012,2012-08-04,2012-08-10,N,Wk.32 - 13,2012-08-04 00:00:00,2012-08-10 00:00:00,2012W32   ,2012W32   ,08/04/13 - 08/10/13,08/04 - 08/10,2012,N,2012-08-05,2011-08-05,2012-05-05,2012-02-05,2012-07-05,2012-06-05,2012-07-29,2012-07-22,0,0,0,0,0,0,0,0,8,3,32,3,1,1,1,2005-09-07,USER_X  ,2012-11-27 00:16:56,USER_X
+2012-02-08,2012-01-01,2012-01-01,2012-02-01,2012-02-03,0,-3,-9,-42,-292,0,-3,-10,-42,-42,41312,39,39,8,6,6,5901,2,2,1358,1,453,114,2012-12-31,2012-03-31,2012-02-28,2012-02-09,08-Feb-2012,Feb 8th 2012,Fri 02-08-13,1,0,0,0,2012-12-21,365,90,28,7,2012-12-30,2012-12-30,2012-01-27,2012-02-03,2012-02-04,2012-02-08,Fri       ,2012M02,Feb-2012,N,2012M02   ,N,Year 2012 - Quarter 01,2012Q01   ,N,6,2012,2012-02-03,2012-02-09,N,Wk.06 - 13,2012-02-03 00:00:00,2012-02-09 00:00:00,2012W06   ,2012W06   ,02/03/13 - 02/09/13,02/03 - 02/09,2012,N,2012-02-08,2011-02-08,2012-11-08,2012-08-08,2012-01-08,2012-12-08,2012-02-01,2012-01-25,0,0,0,0,0,0,0,0,2,1,6,1,1,1,1,2005-09-07,USER_X  ,2012-11-27 00:16:56,USER_X
+2012-02-14,2012-01-01,2012-01-01,2012-02-01,2012-02-10,0,-3,-9,-41,-286,0,-3,-10,-41,-41,41318,45,45,14,5,7,5902,2,2,1358,1,453,114,2012-12-31,2012-03-31,2012-02-28,2012-02-16,14-Feb-2012,Feb 14th 2012,Thu 02-14-13,1,0,0,0,2012-12-21,365,90,28,7,2012-12-30,2012-12-30,2012-01-27,2012-02-10,2012-02-11,2012-02-14,Thu       ,2012M02,Feb-2012,N,2012M02   ,N,Year 2012 - Quarter 01,2012Q01   ,N,7,2012,2012-02-10,2012-02-16,N,Wk.07 - 13,2012-02-10 00:00:00,2012-02-16 00:00:00,2012W07   ,2012W07   ,02/10/13 - 02/16/13,02/10 - 02/16,2012,N,2012-02-14,2011-02-14,2012-11-14,2012-08-14,2012-01-14,2012-12-14,2012-02-07,2012-01-31,0,0,0,0,0,0,0,0,2,1,7,1,1,1,1,2005-09-07,USER_X  ,2012-11-27 00:16:56,USER_X
+2012-09-21,2012-01-01,2012-07-01,2012-09-01,2012-09-15,0,-1,-2,-10,-67,0,-1,-3,-10,-10,41537,264,83,21,7,38,5933,3,9,1365,3,455,114,2012-12-31,2012-09-30,2012-09-30,2012-09-21,21-Sep-2012,Sep 21st 2012,Sat 09-21-13,1,0,0,0,2012-09-21,365,92,30,7,2012-12-30,2012-06-30,2012-08-25,2012-09-15,2012-09-16,2012-09-21,Sat       ,2012M09,Sep-2012,N,2012M09   ,N,Year 2012 - Quarter 03,2012Q03   ,N,38,2012,2012-09-15,2012-09-21,N,Wk.38 - 13,2012-09-15 00:00:00,2012-09-21 00:00:00,2012W38   ,2012W38   ,09/15/13 - 09/21/13,09/15 - 09/21,2012,N,2012-09-21,2011-09-21,2012-06-21,2012-03-21,2012-08-21,2012-07-21,2012-09-14,2012-09-07,0,0,0,0,0,0,0,0,9,3,38,4,1,1,1,2005-09-07,USER_X  ,2012-11-27 00:16:56,USER_X
+2012-03-19,2012-01-01,2012-01-01,2012-03-01,2012-03-17,0,-3,-8,-36,-253,0,-3,-9,-36,-36,41351,78,78,19,3,12,5907,3,3,1359,1,453,114,2012-12-31,2012-03-31,2012-03-31,2012-03-23,19-Mar-2012,Mar 19th 2012,Tue 03-19-13,1,0,0,0,2012-12-21,365,90,31,7,2012-12-30,2012-12-30,2012-02-24,2012-03-17,2012-03-18,2012-03-19,Tue       ,2012M03,Mar-2012,N,2012M03   ,N,Year 2012 - Quarter 01,2012Q01   ,N,12,2012,2012-03-17,2012-03-23,N,Wk.12 - 13,2012-03-17 00:00:00,2012-03-23 00:00:00,2012W12   ,2012W12   ,03/17/13 - 03/23/13,03/17 - 03/23,2012,N,2012-03-19,2011-03-19,2012-12-19,2012-09-19,2012-02-19,2012-01-19,2012-03-12,2012-03-05,0,0,0,0,0,0,0,0,3,1,12,1,1,1,1,2005-09-07,USER_X  ,2012-11-27 00:16:56,USER_X
+2012-01-28,2012-01-01,2012-01-01,2012-01-01,2012-01-27,0,-3,-10,-43,-303,0,-3,-10,-43,-43,41301,28,28,28,2,5,5900,1,1,1357,1,453,114,2012-12-31,2012-03-31,2012-01-31,2012-02-02,28-Jan-2012,Jan 28th 2012,Mon 01-28-13,1,0,0,0,2012-12-21,365,90,31,7,2012-12-30,2012-12-30,2012-01-27,2012-01-27,2012-01-28,2012-01-28,Mon       ,2012M01,Jan-2012,N,2012M01   ,N,Year 2012 - Quarter 01,2012Q01   ,N,5,2012,2012-01-27,2012-02-02,N,Wk.05 - 13,2012-01-27 00:00:00,2012-02-02 00:00:00,2012W05   ,2012W05   ,01/27/13 - 02/02/13,01/27 - 02/02,2012,N,2012-01-28,2011-01-28,2012-10-28,2012-07-28,2012-12-28,2012-11-28,2012-01-21,2012-01-14,0,0,0,0,0,0,0,0,2,1,5,1,1,1,1,2005-09-07,USER_X  ,2012-11-27 00:16:56,USER_X
+2012-08-03,2012-01-01,2012-07-01,2012-08-01,2012-07-28,0,-1,-3,-17,-116,0,-1,-4,-17,-17,41488,215,34,3,7,31,5926,2,8,1364,3,455,114,2012-12-31,2012-09-30,2012-08-31,2012-08-03,03-Aug-2012,Aug 3rd 2012,Sat 08-03-13,1,0,0,0,2012-06-21,365,92,31,7,2012-12-30,2012-06-30,2012-07-28,2012-07-28,2012-07-29,2012-08-03,Sat       ,2012M08,Aug-2012,N,2012M08   ,N,Year 2012 - Quarter 03,2012Q03   ,N,31,2012,2012-07-28,2012-08-03,N,Wk.31 - 13,2012-07-28 00:00:00,2012-08-03 00:00:00,2012W31   ,2012W31   ,07/28/13 - 08/03/13,07/28 - 08/03,2012,N,2012-08-03,2011-08-03,2012-05-03,2012-02-03,2012-07-03,2012-06-03,2012-07-27,2012-07-20,0,0,0,0,0,0,0,0,8,3,31,3,1,1,1,2005-09-07,USER_X  ,2012-11-27 00:16:56,USER_X
+2012-06-14,2012-01-01,2012-04-01,2012-06-01,2012-06-09,0,-2,-5,-24,-166,0,-2,-6,-24,-24,41438,165,75,14,6,24,5919,3,6,1362,2,454,114,2012-12-31,2012-06-30,2012-06-30,2012-06-15,14-Jun-2012,Jun 14th 2012,Fri 06-14-13,1,0,0,0,2012-03-21,365,91,30,7,2012-12-30,2012-03-31,2012-05-26,2012-06-09,2012-06-10,2012-06-14,Fri       ,2012M06,Jun-2012,N,2012M06   ,N,Year 2012 - Quarter 02,2012Q02   ,N,24,2012,2012-06-09,2012-06-15,N,Wk.24 - 13,2012-06-09 00:00:00,2012-06-15 00:00:00,2012W24   ,2012W24   ,06/09/13 - 06/15/13,06/09 - 06/15,2012,N,2012-06-14,2011-06-14,2012-03-14,2012-12-14,2012-05-14,2012-04-14,2012-06-07,2012-05-31,0,0,0,0,0,0,0,0,6,2,24,2,1,1,1,2005-09-07,USER_X  ,2012-11-27 00:16:56,USER_X
+2012-07-08,2012-01-01,2012-07-01,2012-07-01,2012-07-07,0,-1,-4,-20,-142,0,-1,-5,-20,-20,41462,189,8,8,2,28,5923,1,7,1363,3,455,114,2012-12-31,2012-09-30,2012-07-31,2012-07-13,08-Jul-2012,Jul 8th 2012,Mon 07-08-13,1,0,0,0,2012-06-21,365,92,31,7,2012-12-30,2012-06-30,2012-06-30,2012-07-07,2012-07-08,2012-07-08,Mon       ,2012M07,Jul-2012,N,2012M07   ,N,Year 2012 - Quarter 03,2012Q03   ,N,28,2012,2012-07-07,2012-07-13,N,Wk.28 - 13,2012-07-07 00:00:00,2012-07-13 00:00:00,2012W28   ,2012W28   ,07/07/13 - 07/13/13,07/07 - 07/13,2012,N,2012-07-08,2011-07-08,2012-04-08,2012-01-08,2012-06-08,2012-05-08,2012-07-01,2012-06-24,0,0,0,0,0,0,0,0,7,3,28,3,1,1,1,2005-09-07,USER_X  ,2012-11-27 00:16:56,USER_X
+2012-11-30,2012-01-01,2012-10-01,2012-11-01,2012-11-24,0,0,0,0,3,0,0,0,0,0,41607,334,61,30,7,48,5943,2,11,1367,4,456,114,2012-12-31,2012-12-31,2012-11-30,2012-11-30,30-Nov-2012,Nov 30th 2012,Sat 11-30-13,0,0,0,0,2012-09-21,365,92,30,7,2012-12-30,2012-09-29,2012-11-24,2012-11-24,2012-11-25,2012-11-30,Sat       ,2012M11,Nov-2012,N,2012M11   ,N,Year 2012 - Quarter 04,2012Q04   ,N,48,2012,2012-11-24,2012-11-30,N,Wk.48 - 13,2012-11-24 00:00:00,2012-11-30 00:00:00,2012W48   ,2012W48   ,11/24/13 - 11/30/13,11/24 - 11/30,2012,N,2012-11-30,2011-11-30,2012-08-30,2012-05-30,2012-10-30,2012-09-30,2012-11-23,2012-11-16,0,0,0,0,0,0,0,0,12,4,48,4,1,1,1,2005-09-07,USER_X  ,2012-11-27 00:16:56,USER_X
+2012-01-10,2012-01-01,2012-01-01,2012-01-01,2012-01-06,0,-3,-10,-46,-321,0,-3,-11,-46,-46,41283,10,10,10,5,2,5897,1,1,1357,1,453,114,2012-12-31,2012-03-31,2012-01-31,2012-01-12,10-Jan-2012,Jan 10th 2012,Thu 01-10-13,1,0,0,0,2012-12-21,365,90,31,7,2012-12-30,2012-12-30,2012-12-30,2012-01-06,2012-01-07,2012-01-10,Thu       ,2012M01,Jan-2012,N,2012M01   ,N,Year 2012 - Quarter 01,2012Q01   ,N,2,2012,2012-01-06,2012-01-12,N,Wk.02 - 13,2012-01-06 00:00:00,2012-01-12 00:00:00,2012W02   ,2012W02   ,01/06/13 - 01/12/13,01/06 - 01/12,2012,N,2012-01-10,2011-01-10,2012-10-10,2012-07-10,2012-12-10,2012-11-10,2012-01-03,2012-12-27,0,0,0,0,0,0,0,0,1,1,2,1,1,1,1,2005-09-07,USER_X  ,2012-11-27 00:16:56,USER_X
+2012-05-18,2012-01-01,2012-04-01,2012-05-01,2012-05-12,0,-2,-6,-28,-193,0,-2,-7,-28,-28,41411,138,48,18,7,20,5915,2,5,1361,2,454,114,2012-12-31,2012-06-30,2012-05-31,2012-05-18,18-May-2012,May 18th 2012,Sat 05-18-13,1,0,0,0,2012-03-21,365,91,31,7,2012-12-30,2012-03-31,2012-04-28,2012-05-12,2012-05-13,2012-05-18,Sat       ,2012M05,May-2012,N,2012M05   ,N,Year 2012 - Quarter 02,2012Q02   ,N,20,2012,2012-05-12,2012-05-18,N,Wk.20 - 13,2012-05-12 00:00:00,2012-05-18 00:00:00,2012W20   ,2012W20   ,05/12/13 - 05/18/13,05/12 - 05/18,2012,N,2012-05-18,2011-05-18,2012-02-18,2012-11-18,2012-04-18,2012-03-18,2012-05-11,2012-05-04,0,0,0,0,0,0,0,0,5,2,20,2,1,1,1,2005-09-07,USER_X  ,2012-11-27 00:16:56,USER_X
+2012-04-15,2012-01-01,2012-04-01,2012-04-01,2012-04-14,0,-2,-7,-32,-226,0,-2,-8,-32,-32,41378,105,15,15,2,16,5911,1,4,1360,2,454,114,2012-12-31,2012-06-30,2012-04-30,2012-04-20,15-Apr-2012,Apr 15th 2012,Mon 04-15-13,1,0,0,0,2012-03-21,365,91,30,7,2012-12-30,2012-03-31,2012-03-31,2012-04-14,2012-04-15,2012-04-15,Mon       ,2012M04,Apr-2012,N,2012M04   ,N,Year 2012 - Quarter 02,2012Q02   ,N,16,2012,2012-04-14,2012-04-20,N,Wk.16 - 13,2012-04-14 00:00:00,2012-04-20 00:00:00,2012W16   ,2012W16   ,04/14/13 - 04/20/13,04/14 - 04/20,2012,N,2012-04-15,2011-04-15,2012-01-15,2012-10-15,2012-03-15,2012-02-15,2012-04-08,2012-04-01,0,0,0,0,0,0,0,0,4,2,16,2,1,1,1,2005-09-07,USER_X  ,2012-11-27 00:16:56,USER_X
+2012-03-12,2012-01-01,2012-01-01,2012-03-01,2012-03-10,0,-3,-8,-37,-260,0,-3,-9,-37,-37,41344,71,71,12,3,11,5906,3,3,1359,1,453,114,2012-12-31,2012-03-31,2012-03-31,2012-03-16,12-Mar-2012,Mar 12th 2012,Tue 03-12-13,1,0,0,0,2012-12-21,365,90,31,7,2012-12-30,2012-12-30,2012-02-24,2012-03-10,2012-03-11,2012-03-12,Tue       ,2012M03,Mar-2012,N,2012M03   ,N,Year 2012 - Quarter 01,2012Q01   ,N,11,2012,2012-03-10,2012-03-16,N,Wk.11 - 13,2012-03-10 00:00:00,2012-03-16 00:00:00,2012W11   ,2012W11   ,03/10/13 - 03/16/13,03/10 - 03/16,2012,N,2012-03-12,2011-03-12,2012-12-12,2012-09-12,2012-02-12,2012-01-12,2012-03-05,2012-02-26,0,0,0,0,0,0,0,0,3,1,11,1,1,1,1,2005-09-07,USER_X  ,2012-11-27 00:16:56,USER_X
+2012-11-09,2012-01-01,2012-10-01,2012-11-01,2012-11-03,0,0,0,-3,-18,0,0,-1,-3,-3,41586,313,40,9,7,45,5940,2,11,1367,4,456,114,2012-12-31,2012-12-31,2012-11-30,2012-11-09,09-Nov-2012,Nov 9th 2012,Sat 11-09-13,1,1,1,0,2012-09-21,365,92,30,7,2012-12-30,2012-09-29,2012-10-27,2012-11-03,2012-11-04,2012-11-09,Sat       ,2012M11,Nov-2012,N,2012M11   ,N,Year 2012 - Quarter 04,2012Q04   ,N,45,2012,2012-11-03,2012-11-09,N,Wk.45 - 13,2012-11-03 00:00:00,2012-11-09 00:00:00,2012W45   ,2012W45   ,11/03/13 - 11/09/13,11/03 - 11/09,2012,N,2012-11-09,2011-11-09,2012-08-09,2012-05-09,2012-10-09,2012-09-09,2012-11-02,2012-10-26,0,0,0,0,0,0,0,0,11,4,45,4,1,1,1,2005-09-07,USER_X  ,2012-11-27 00:16:56,USER_X
+2012-03-14,2012-01-01,2012-01-01,2012-03-01,2012-03-10,0,-3,-8,-37,-258,0,-3,-9,-37,-37,41346,73,73,14,5,11,5906,3,3,1359,1,453,114,2012-12-31,2012-03-31,2012-03-31,2012-03-16,14-Mar-2012,Mar 14th 2012,Thu 03-14-13,1,0,0,0,2012-12-21,365,90,31,7,2012-12-30,2012-12-30,2012-02-24,2012-03-10,2012-03-11,2012-03-14,Thu       ,2012M03,Mar-2012,N,2012M03   ,N,Year 2012 - Quarter 01,2012Q01   ,N,11,2012,2012-03-10,2012-03-16,N,Wk.11 - 13,2012-03-10 00:00:00,2012-03-16 00:00:00,2012W11   ,2012W11   ,03/10/13 - 03/16/13,03/10 - 03/16,2012,N,2012-03-14,2011-03-14,2012-12-14,2012-09-14,2012-02-14,2012-01-14,2012-03-07,2012-02-28,0,0,0,0,0,0,0,0,3,1,11,1,1,1,1,2005-09-07,USER_X  ,2012-11-27 00:16:56,USER_X
+2012-09-03,2012-01-01,2012-07-01,2012-09-01,2012-09-01,0,-1,-2,-12,-85,0,-1,-3,-12,-12,41519,246,65,3,3,36,5931,3,9,1365,3,455,114,2012-12-31,2012-09-30,2012-09-30,2012-09-07,03-Sep-2012,Sep 3rd 2012,Tue 09-03-13,1,0,0,0,2012-06-21,365,92,30,7,2012-12-30,2012-06-30,2012-08-25,2012-09-01,2012-09-02,2012-09-03,Tue       ,2012M09,Sep-2012,N,2012M09   ,N,Year 2012 - Quarter 03,2012Q03   ,N,36,2012,2012-09-01,2012-09-07,N,Wk.36 - 13,2012-09-01 00:00:00,2012-09-07 00:00:00,2012W36   ,2012W36   ,09/01/13 - 09/07/13,09/01 - 09/07,2012,N,2012-09-03,2011-09-03,2012-06-03,2012-03-03,2012-08-03,2012-07-03,2012-08-27,2012-08-20,0,0,0,0,0,0,0,0,9,3,36,3,1,1,1,2005-09-07,USER_X  ,2012-11-27 00:16:56,USER_X
+2012-05-16,2012-01-01,2012-04-01,2012-05-01,2012-05-12,0,-2,-6,-28,-195,0,-2,-7,-28,-28,41409,136,46,16,5,20,5915,2,5,1361,2,454,114,2012-12-31,2012-06-30,2012-05-31,2012-05-18,16-May-2012,May 16th 2012,Thu 05-16-13,1,0,0,0,2012-03-21,365,91,31,7,2012-12-30,2012-03-31,2012-04-28,2012-05-12,2012-05-13,2012-05-16,Thu       ,2012M05,May-2012,N,2012M05   ,N,Year 2012 - Quarter 02,2012Q02   ,N,20,2012,2012-05-12,2012-05-18,N,Wk.20 - 13,2012-05-12 00:00:00,2012-05-18 00:00:00,2012W20   ,2012W20   ,05/12/13 - 05/18/13,05/12 - 05/18,2012,N,2012-05-16,2011-05-16,2012-02-16,2012-11-16,2012-04-16,2012-03-16,2012-05-09,2012-05-02,0,0,0,0,0,0,0,0,5,2,20,2,1,1,1,2005-09-07,USER_X  ,2012-11-27 00:16:56,USER_X
+2012-10-27,2012-01-01,2012-10-01,2012-10-01,2012-10-27,0,0,-1,-4,-31,0,0,-1,-4,-5,41573,300,27,27,1,44,5939,1,10,1366,4,456,114,2012-12-31,2012-12-31,2012-10-31,2012-11-02,27-Oct-2012,Oct 27th 2012,Sun 10-27-13,1,1,0,0,2012-09-21,365,92,31,7,2012-12-30,2012-09-29,2012-10-27,2012-10-27,2012-10-21,2012-10-27,Sun       ,2012M10,Oct-2012,N,2012M10   ,N,Year 2012 - Quarter 04,2012Q04   ,N,44,2012,2012-10-27,2012-11-02,Y,Wk.44 - 13,2012-10-27 00:00:00,2012-11-02 00:00:00,2012W44   ,2012W44   ,10/27/13 - 11/02/13,10/27 - 11/02,2012,N,2012-10-27,2011-10-27,2012-07-27,2012-04-27,2012-09-27,2012-08-27,2012-10-20,2012-10-13,0,0,0,0,1,0,0,0,11,4,44,4,1,1,1,2005-09-07,USER_X  ,2012-11-27 00:16:56,USER_X
+2012-08-21,2012-01-01,2012-07-01,2012-08-01,2012-08-18,0,-1,-3,-14,-98,0,-1,-4,-14,-14,41506,233,52,21,4,34,5929,2,8,1364,3,455,114,2012-12-31,2012-09-30,2012-08-31,2012-08-24,21-Aug-2012,Aug 21st 2012,Wed 08-21-13,1,0,0,0,2012-06-21,365,92,31,7,2012-12-30,2012-06-30,2012-07-28,2012-08-18,2012-08-19,2012-08-21,Wed       ,2012M08,Aug-2012,N,2012M08   ,N,Year 2012 - Quarter 03,2012Q03   ,N,34,2012,2012-08-18,2012-08-24,N,Wk.34 - 13,2012-08-18 00:00:00,2012-08-24 00:00:00,2012W34   ,2012W34   ,08/18/13 - 08/24/13,08/18 - 08/24,2012,N,2012-08-21,2011-08-21,2012-05-21,2012-02-21,2012-07-21,2012-06-21,2012-08-14,2012-08-07,0,0,0,0,0,0,0,0,8,3,34,3,1,1,1,2005-09-07,USER_X  ,2012-11-27 00:16:56,USER_X
+2012-10-25,2012-01-01,2012-10-01,2012-10-01,2012-10-20,0,0,-1,-5,-33,0,0,-2,-5,-5,41571,298,25,25,6,43,5938,1,10,1366,4,456,114,2012-12-31,2012-12-31,2012-10-31,2012-10-26,25-Oct-2012,Oct 25th 2012,Fri 10-25-13,1,1,0,0,2012-09-21,365,92,31,7,2012-12-30,2012-09-29,2012-09-29,2012-10-20,2012-10-21,2012-10-25,Fri       ,2012M10,Oct-2012,N,2012M10   ,N,Year 2012 - Quarter 04,2012Q04   ,N,43,2012,2012-10-20,2012-10-26,N,Wk.43 - 13,2012-10-20 00:00:00,2012-10-26 00:00:00,2012W43   ,2012W43   ,10/20/13 - 10/26/13,10/20 - 10/26,2012,N,2012-10-25,2011-10-25,2012-07-25,2012-04-25,2012-09-25,2012-08-25,2012-10-18,2012-10-11,0,0,0,0,0,0,0,0,10,4,43,4,1,1,1,2005-09-07,USER_X  ,2012-11-27 00:16:56,USER_X
+2012-09-24,2012-01-01,2012-07-01,2012-09-01,2012-09-22,0,-1,-2,-9,-64,0,-1,-3,-9,-9,41540,267,86,24,3,39,5934,3,9,1365,3,455,114,2012-12-31,2012-09-30,2012-09-30,2012-09-28,24-Sep-2012,Sep 24th 2012,Tue 09-24-13,1,0,0,0,2012-09-21,365,92,30,7,2012-12-30,2012-06-30,2012-08-25,2012-09-22,2012-09-23,2012-09-24,Tue       ,2012M09,Sep-2012,N,2012M09   ,N,Year 2012 - Quarter 03,2012Q03   ,N,39,2012,2012-09-22,2012-09-28,N,Wk.39 - 13,2012-09-22 00:00:00,2012-09-28 00:00:00,2012W39   ,2012W39   ,09/22/13 - 09/28/13,09/22 - 09/28,2012,N,2012-09-24,2011-09-24,2012-06-24,2012-03-24,2012-08-24,2012-07-24,2012-09-17,2012-09-10,0,0,0,0,0,0,0,0,9,3,39,4,1,1,1,2005-09-07,USER_X  ,2012-11-27 00:16:56,USER_X
+2012-01-27,2012-01-01,2012-01-01,2012-01-01,2012-01-27,0,-3,-10,-43,-304,0,-3,-10,-43,-44,41300,27,27,27,1,5,5900,1,1,1357,1,453,114,2012-12-31,2012-03-31,2012-01-31,2012-02-02,27-Jan-2012,Jan 27th 2012,Sun 01-27-13,1,0,0,0,2012-12-21,365,90,31,7,2012-12-30,2012-12-30,2012-01-27,2012-01-27,2012-01-21,2012-01-27,Sun       ,2012M01,Jan-2012,N,2012M01   ,N,Year 2012 - Quarter 01,2012Q01   ,N,5,2012,2012-01-27,2012-02-02,Y,Wk.05 - 13,2012-01-27 00:00:00,2012-02-02 00:00:00,2012W05   ,2012W05   ,01/27/13 - 02/02/13,01/27 - 02/02,2012,N,2012-01-27,2011-01-27,2012-10-27,2012-07-27,2012-12-27,2012-11-27,2012-01-20,2012-01-13,0,0,0,0,0,0,0,0,2,1,5,1,1,1,1,2005-09-07,USER_X  ,2012-11-27 00:16:56,USER_X
+2012-03-10,2012-01-01,2012-01-01,2012-03-01,2012-03-10,0,-3,-8,-37,-262,0,-3,-9,-37,-38,41342,69,69,10,1,11,5906,3,3,1359,1,453,114,2012-12-31,2012-03-31,2012-03-31,2012-03-16,10-Mar-2012,Mar 10th 2012,Sun 03-10-13,1,0,0,0,2012-12-21,365,90,31,7,2012-12-30,2012-12-30,2012-02-24,2012-03-10,2012-03-04,2012-03-10,Sun       ,2012M03,Mar-2012,N,2012M03   ,N,Year 2012 - Quarter 01,2012Q01   ,N,11,2012,2012-03-10,2012-03-16,Y,Wk.11 - 13,2012-03-10 00:00:00,2012-03-16 00:00:00,2012W11   ,2012W11   ,03/10/13 - 03/16/13,03/10 - 03/16,2012,N,2012-03-10,2011-03-10,2012-12-10,2012-09-10,2012-02-10,2012-01-10,2012-03-03,2012-02-24,0,0,0,0,0,0,0,0,3,1,11,1,1,1,1,2005-09-07,USER_X  ,2012-11-27 00:16:56,USER_X
+2012-05-14,2012-01-01,2012-04-01,2012-05-01,2012-05-12,0,-2,-6,-28,-197,0,-2,-7,-28,-28,41407,134,44,14,3,20,5915,2,5,1361,2,454,114,2012-12-31,2012-06-30,2012-05-31,2012-05-18,14-May-2012,May 14th 2012,Tue 05-14-13,1,0,0,0,2012-03-21,365,91,31,7,2012-12-30,2012-03-31,2012-04-28,2012-05-12,2012-05-13,2012-05-14,Tue       ,2012M05,May-2012,N,2012M05   ,N,Year 2012 - Quarter 02,2012Q02   ,N,20,2012,2012-05-12,2012-05-18,N,Wk.20 - 13,2012-05-12 00:00:00,2012-05-18 00:00:00,2012W20   ,2012W20   ,05/12/13 - 05/18/13,05/12 - 05/18,2012,N,2012-05-14,2011-05-14,2012-02-14,2012-11-14,2012-04-14,2012-03-14,2012-05-07,2012-04-30,0,0,0,0,0,0,0,0,5,2,20,2,1,1,1,2005-09-07,USER_X  ,2012-11-27 00:16:56,USER_X
+2012-12-31,2012-01-01,2012-10-01,2012-12-01,2012-12-29,0,0,1,5,34,1,1,1,5,5,41638,365,92,31,3,53,5948,3,12,1368,4,456,114,2012-12-31,2012-12-31,2012-12-31,2012-12-31,31-Dec-2012,Dec 31st 2012,Tue 12-31-13,0,0,0,0,2012-12-21,365,92,31,3,2012-12-29,2012-12-29,2012-12-29,2012-12-29,2012-12-30,2012-12-31,Tue       ,2012M12,Dec-2012,N,2012M12   ,N,Year 2012 - Quarter 04,2012Q04   ,N,1,2014,2012-12-29,2014-01-04,N,Wk.53 - 13,2012-12-29 00:00:00,2012-12-31 00:00:00,2012W53   ,2012W53   ,12/29/13 - 12/31/13,12/29 - 12/31,2012,N,2012-12-31,2011-12-31,2012-09-30,2012-06-30,2012-11-30,2012-10-31,2012-12-24,2012-12-17,0,0,0,0,0,0,0,0,1,1,1,1,0,1,0,2005-09-07,USER_X  ,2012-11-27 00:16:56,USER_X
+2012-09-01,2012-01-01,2012-07-01,2012-09-01,2012-09-01,0,-1,-2,-12,-87,0,-1,-3,-12,-13,41517,244,63,1,1,36,5931,3,9,1365,3,455,114,2012-12-31,2012-09-30,2012-09-30,2012-09-07,01-Sep-2012,Sep 1st 2012,Sun 09-01-13,1,0,0,0,2012-06-21,365,92,30,7,2012-12-30,2012-06-30,2012-08-25,2012-09-01,2012-08-26,2012-09-01,Sun       ,2012M09,Sep-2012,Y,2012M09   ,Y,Year 2012 - Quarter 03,2012Q03   ,N,36,2012,2012-09-01,2012-09-07,Y,Wk.36 - 13,2012-09-01 00:00:00,2012-09-07 00:00:00,2012W36   ,2012W36   ,09/01/13 - 09/07/13,09/01 - 09/07,2012,N,2012-09-01,2011-09-01,2012-06-01,2012-03-01,2012-08-01,2012-07-01,2012-08-25,2012-08-18,0,0,0,0,0,0,0,0,9,3,36,3,1,1,1,2005-09-07,USER_X  ,2012-11-27 00:16:56,USER_X
+2012-04-13,2012-01-01,2012-04-01,2012-04-01,2012-04-07,0,-2,-7,-33,-228,0,-2,-8,-33,-33,41376,103,13,13,7,15,5910,1,4,1360,2,454,114,2012-12-31,2012-06-30,2012-04-30,2012-04-13,13-Apr-2012,Apr 13th 2012,Sat 04-13-13,1,0,0,0,2012-03-21,365,91,30,7,2012-12-30,2012-03-31,2012-03-31,2012-04-07,2012-04-08,2012-04-13,Sat       ,2012M04,Apr-2012,N,2012M04   ,N,Year 2012 - Quarter 02,2012Q02   ,N,15,2012,2012-04-07,2012-04-13,N,Wk.15 - 13,2012-04-07 00:00:00,2012-04-13 00:00:00,2012W15   ,2012W15   ,04/07/13 - 04/13/13,04/07 - 04/13,2012,N,2012-04-13,2011-04-13,2012-01-13,2012-10-13,2012-03-13,2012-02-13,2012-04-06,2012-03-30,0,0,0,0,0,0,0,0,4,2,15,2,1,1,1,2005-09-07,USER_X  ,2012-11-27 00:16:56,USER_X
+2012-02-09,2012-01-01,2012-01-01,2012-02-01,2012-02-03,0,-3,-9,-42,-291,0,-3,-10,-42,-42,41313,40,40,9,7,6,5901,2,2,1358,1,453,114,2012-12-31,2012-03-31,2012-02-28,2012-02-09,09-Feb-2012,Feb 9th 2012,Sat 02-09-13,1,0,0,0,2012-12-21,365,90,28,7,2012-12-30,2012-12-30,2012-01-27,2012-02-03,2012-02-04,2012-02-09,Sat       ,2012M02,Feb-2012,N,2012M02   ,N,Year 2012 - Quarter 01,2012Q01   ,N,6,2012,2012-02-03,2012-02-09,N,Wk.06 - 13,2012-02-03 00:00:00,2012-02-09 00:00:00,2012W06   ,2012W06   ,02/03/13 - 02/09/13,02/03 - 02/09,2012,N,2012-02-09,2011-02-09,2012-11-09,2012-08-09,2012-01-09,2012-12-09,2012-02-02,2012-01-26,0,0,0,0,0,0,0,0,2,1,6,1,1,1,1,2005-09-07,USER_X  ,2012-11-27 00:16:56,USER_X
+2012-12-27,2012-01-01,2012-10-01,2012-12-01,2012-12-22,0,0,1,4,30,0,0,0,4,4,41634,361,88,27,6,52,5947,3,12,1368,4,456,114,2012-12-31,2012-12-31,2012-12-31,2012-12-28,27-Dec-2012,Dec 27th 2012,Fri 12-27-13,0,0,0,0,2012-12-21,365,92,31,7,2012-12-30,2012-09-29,2012-11-24,2012-12-22,2012-12-23,2012-12-27,Fri       ,2012M12,Dec-2012,N,2012M12   ,N,Year 2012 - Quarter 04,2012Q04   ,N,52,2012,2012-12-22,2012-12-28,N,Wk.52 - 13,2012-12-22 00:00:00,2012-12-28 00:00:00,2012W52   ,2012W52   ,12/22/13 - 12/28/13,12/22 - 12/28,2012,N,2012-12-27,2011-12-27,2012-09-27,2012-06-27,2012-11-27,2012-10-27,2012-12-20,2012-12-13,0,0,0,0,0,0,0,0,12,4,52,1,0,1,0,2005-09-07,USER_X  ,2012-11-27 00:16:56,USER_X
+2012-08-19,2012-01-01,2012-07-01,2012-08-01,2012-08-18,0,-1,-3,-14,-100,0,-1,-4,-14,-14,41504,231,50,19,2,34,5929,2,8,1364,3,455,114,2012-12-31,2012-09-30,2012-08-31,2012-08-24,19-Aug-2012,Aug 19th 2012,Mon 08-19-13,1,0,0,0,2012-06-21,365,92,31,7,2012-12-30,2012-06-30,2012-07-28,2012-08-18,2012-08-19,2012-08-19,Mon       ,2012M08,Aug-2012,N,2012M08   ,N,Year 2012 - Quarter 03,2012Q03   ,N,34,2012,2012-08-18,2012-08-24,N,Wk.34 - 13,2012-08-18 00:00:00,2012-08-24 00:00:00,2012W34   ,2012W34   ,08/18/13 - 08/24/13,08/18 - 08/24,2012,N,2012-08-19,2011-08-19,2012-05-19,2012-02-19,2012-07-19,2012-06-19,2012-08-12,2012-08-05,0,0,0,0,0,0,0,0,8,3,34,3,1,1,1,2005-09-07,USER_X  ,2012-11-27 00:16:56,USER_X
+2012-04-11,2012-01-01,2012-04-01,2012-04-01,2012-04-07,0,-2,-7,-33,-230,0,-2,-8,-33,-33,41374,101,11,11,5,15,5910,1,4,1360,2,454,114,2012-12-31,2012-06-30,2012-04-30,2012-04-13,11-Apr-2012,Apr 11th 2012,Thu 04-11-13,1,0,0,0,2012-03-21,365,91,30,7,2012-12-30,2012-03-31,2012-03-31,2012-04-07,2012-04-08,2012-04-11,Thu       ,2012M04,Apr-2012,N,2012M04   ,N,Year 2012 - Quarter 02,2012Q02   ,N,15,2012,2012-04-07,2012-04-13,N,Wk.15 - 13,2012-04-07 00:00:00,2012-04-13 00:00:00,2012W15   ,2012W15   ,04/07/13 - 04/13/13,04/07 - 04/13,2012,N,2012-04-11,2011-04-11,2012-01-11,2012-10-11,2012-03-11,2012-02-11,2012-04-04,2012-03-28,0,0,0,0,0,0,0,0,4,2,15,2,1,1,1,2005-09-07,USER_X  ,2012-11-27 00:16:56,USER_X
+2012-12-06,2012-01-01,2012-10-01,2012-12-01,2012-12-01,0,0,1,1,9,0,0,0,1,1,41613,340,67,6,6,49,5944,3,12,1368,4,456,114,2012-12-31,2012-12-31,2012-12-31,2012-12-07,06-Dec-2012,Dec 6th 2012,Fri 12-06-13,0,0,0,0,2012-09-21,365,92,31,7,2012-12-30,2012-09-29,2012-11-24,2012-12-01,2012-12-02,2012-12-06,Fri       ,2012M12,Dec-2012,N,2012M12   ,N,Year 2012 - Quarter 04,2012Q04   ,N,49,2012,2012-12-01,2012-12-07,N,Wk.49 - 13,2012-12-01 00:00:00,2012-12-07 00:00:00,2012W49   ,2012W49   ,12/01/13 - 12/07/13,12/01 - 12/07,2012,N,2012-12-06,2011-12-06,2012-09-06,2012-06-06,2012-11-06,2012-10-06,2012-11-29,2012-11-22,0,0,0,0,0,0,0,0,12,4,49,4,0,1,0,2005-09-07,USER_X  ,2012-11-27 00:16:56,USER_X
+2012-03-31,2012-01-01,2012-01-01,2012-03-01,2012-03-31,0,-3,-8,-34,-241,0,-2,-8,-34,-35,41363,90,90,31,1,14,5909,3,3,1359,1,453,114,2012-12-31,2012-03-31,2012-03-31,2012-04-06,31-Mar-2012,Mar 31st 2012,Sun 03-31-13,1,0,0,0,2012-03-21,365,90,31,7,2012-12-30,2012-03-31,2012-03-31,2012-03-31,2012-03-25,2012-03-31,Sun       ,2012M03,Mar-2012,N,2012M03   ,N,Year 2012 - Quarter 01,2012Q01   ,N,14,2012,2012-03-31,2012-04-06,Y,Wk.14 - 13,2012-03-31 00:00:00,2012-04-06 00:00:00,2012W14   ,2012W14   ,03/31/13 - 04/06/13,03/31 - 04/06,2012,N,2012-03-31,2011-03-31,2012-12-31,2012-09-30,2012-02-28,2012-01-31,2012-03-24,2012-03-17,0,0,0,0,0,0,0,0,4,2,14,2,1,1,1,2005-09-07,USER_X  ,2012-11-27 00:16:56,USER_X
+2012-09-22,2012-01-01,2012-07-01,2012-09-01,2012-09-22,0,-1,-2,-9,-66,0,-1,-3,-9,-10,41538,265,84,22,1,39,5934,3,9,1365,3,455,114,2012-12-31,2012-09-30,2012-09-30,2012-09-28,22-Sep-2012,Sep 22nd 2012,Sun 09-22-13,1,0,0,0,2012-09-21,365,92,30,7,2012-12-30,2012-06-30,2012-08-25,2012-09-22,2012-09-16,2012-09-22,Sun       ,2012M09,Sep-2012,N,2012M09   ,N,Year 2012 - Quarter 03,2012Q03   ,N,39,2012,2012-09-22,2012-09-28,Y,Wk.39 - 13,2012-09-22 00:00:00,2012-09-28 00:00:00,2012W39   ,2012W39   ,09/22/13 - 09/28/13,09/22 - 09/28,2012,N,2012-09-22,2011-09-22,2012-06-22,2012-03-22,2012-08-22,2012-07-22,2012-09-15,2012-09-08,0,0,0,0,0,0,0,0,9,3,39,4,1,1,1,2005-09-07,USER_X  ,2012-11-27 00:16:56,USER_X
+2012-01-06,2012-01-01,2012-01-01,2012-01-01,2012-01-06,0,-3,-10,-46,-325,0,-3,-11,-46,-47,41279,6,6,6,1,2,5897,1,1,1357,1,453,114,2012-12-31,2012-03-31,2012-01-31,2012-01-12,06-Jan-2012,Jan 6th 2012,Sun 01-06-13,1,0,0,0,2012-12-21,365,90,31,7,2012-12-30,2012-12-30,2012-12-30,2012-01-06,2012-12-31,2012-01-06,Sun       ,2012M01,Jan-2012,N,2012M01   ,N,Year 2012 - Quarter 01,2012Q01   ,N,2,2012,2012-01-06,2012-01-12,Y,Wk.02 - 13,2012-01-06 00:00:00,2012-01-12 00:00:00,2012W02   ,2012W02   ,01/06/13 - 01/12/13,01/06 - 01/12,2012,N,2012-01-06,2011-01-06,2012-10-06,2012-07-06,2012-12-06,2012-11-06,2012-12-30,2012-12-23,0,0,0,0,0,0,0,0,1,1,2,1,1,1,1,2005-09-07,USER_X  ,2012-11-27 00:16:56,USER_X
+2012-01-04,2012-01-01,2012-01-01,2012-01-01,2012-01-01,0,-3,-10,-47,-327,0,-3,-11,-47,-47,41277,4,4,4,6,1,5896,1,1,1357,1,453,114,2012-12-31,2012-03-31,2012-01-31,2012-01-05,04-Jan-2012,Jan 4th 2012,Fri 01-04-13,1,0,0,0,2012-12-21,365,90,31,5,2012-12-30,2012-12-30,2012-12-30,2012-12-30,2012-12-31,2012-01-04,Fri       ,2012M01,Jan-2012,N,2012M01   ,N,Year 2012 - Quarter 01,2012Q01   ,N,1,2012,2012-12-30,2012-01-05,N,Wk.01 - 13,2012-01-01 00:00:00,2012-01-05 00:00:00,2012W01   ,2012W01   ,01/01/13 - 01/05/13,01/01 - 01/05,2012,N,2012-01-04,2011-01-04,2012-10-04,2012-07-04,2012-12-04,2012-11-04,2012-12-28,2012-12-21,0,0,0,0,0,0,0,0,1,1,1,1,1,1,1,2005-09-07,USER_X  ,2012-11-27 00:16:56,USER_X
+2012-07-16,2012-01-01,2012-07-01,2012-07-01,2012-07-14,0,-1,-4,-19,-134,0,-1,-5,-19,-19,41470,197,16,16,3,29,5924,1,7,1363,3,455,114,2012-12-31,2012-09-30,2012-07-31,2012-07-20,16-Jul-2012,Jul 16th 2012,Tue 07-16-13,1,0,0,0,2012-06-21,365,92,31,7,2012-12-30,2012-06-30,2012-06-30,2012-07-14,2012-07-15,2012-07-16,Tue       ,2012M07,Jul-2012,N,2012M07   ,N,Year 2012 - Quarter 03,2012Q03   ,N,29,2012,2012-07-14,2012-07-20,N,Wk.29 - 13,2012-07-14 00:00:00,2012-07-20 00:00:00,2012W29   ,2012W29   ,07/14/13 - 07/20/13,07/14 - 07/20,2012,N,2012-07-16,2011-07-16,2012-04-16,2012-01-16,2012-06-16,2012-05-16,2012-07-09,2012-07-02,0,0,0,0,0,0,0,0,7,3,29,3,1,1,1,2005-09-07,USER_X  ,2012-11-27 00:16:56,USER_X
+2012-11-03,2012-01-01,2012-10-01,2012-11-01,2012-11-03,0,0,0,-3,-24,0,0,-1,-3,-4,41580,307,34,3,1,45,5940,2,11,1367,4,456,114,2012-12-31,2012-12-31,2012-11-30,2012-11-09,03-Nov-2012,Nov 3rd 2012,Sun 11-03-13,1,1,1,0,2012-09-21,365,92,30,7,2012-12-30,2012-09-29,2012-10-27,2012-11-03,2012-10-28,2012-11-03,Sun       ,2012M11,Nov-2012,N,2012M11   ,N,Year 2012 - Quarter 04,2012Q04   ,N,45,2012,2012-11-03,2012-11-09,Y,Wk.45 - 13,2012-11-03 00:00:00,2012-11-09 00:00:00,2012W45   ,2012W45   ,11/03/13 - 11/09/13,11/03 - 11/09,2012,N,2012-11-03,2011-11-03,2012-08-03,2012-05-03,2012-10-03,2012-09-03,2012-10-27,2012-10-20,0,0,0,0,0,0,0,0,11,4,45,4,1,1,1,2005-09-07,USER_X  ,2012-11-27 00:16:56,USER_X
+2012-08-25,2012-01-01,2012-07-01,2012-08-01,2012-08-25,0,-1,-3,-13,-94,0,-1,-3,-13,-14,41510,237,56,25,1,35,5930,2,8,1364,3,455,114,2012-12-31,2012-09-30,2012-08-31,2012-08-31,25-Aug-2012,Aug 25th 2012,Sun 08-25-13,1,0,0,0,2012-06-21,365,92,31,7,2012-12-30,2012-06-30,2012-08-25,2012-08-25,2012-08-19,2012-08-25,Sun       ,2012M08,Aug-2012,N,2012M08   ,N,Year 2012 - Quarter 03,2012Q03   ,N,35,2012,2012-08-25,2012-08-31,Y,Wk.35 - 13,2012-08-25 00:00:00,2012-08-31 00:00:00,2012W35   ,2012W35   ,08/25/13 - 08/31/13,08/25 - 08/31,2012,N,2012-08-25,2011-08-25,2012-05-25,2012-02-25,2012-07-25,2012-06-25,2012-08-18,2012-08-11,0,0,0,0,0,0,0,0,9,3,35,3,1,1,1,2005-09-07,USER_X  ,2012-11-27 00:16:56,USER_X
+2012-11-24,2012-01-01,2012-10-01,2012-11-01,2012-11-24,0,0,0,0,-3,0,0,0,0,-1,41601,328,55,24,1,48,5943,2,11,1367,4,456,114,2012-12-31,2012-12-31,2012-11-30,2012-11-30,24-Nov-2012,Nov 24th 2012,Sun 11-24-13,1,1,1,1,2012-09-21,365,92,30,7,2012-12-30,2012-09-29,2012-11-24,2012-11-24,2012-11-18,2012-11-24,Sun       ,2012M11,Nov-2012,N,2012M11   ,N,Year 2012 - Quarter 04,2012Q04   ,N,48,2012,2012-11-24,2012-11-30,Y,Wk.48 - 13,2012-11-24 00:00:00,2012-11-30 00:00:00,2012W48   ,2012W48   ,11/24/13 - 11/30/13,11/24 - 11/30,2012,N,2012-11-24,2011-11-24,2012-08-24,2012-05-24,2012-10-24,2012-09-24,2012-11-17,2012-11-10,0,0,0,0,0,0,0,0,12,4,48,4,1,1,1,2005-09-07,USER_X  ,2012-11-27 00:16:56,USER_X
+2012-05-12,2012-01-01,2012-04-01,2012-05-01,2012-05-12,0,-2,-6,-28,-199,0,-2,-7,-28,-29,41405,132,42,12,1,20,5915,2,5,1361,2,454,114,2012-12-31,2012-06-30,2012-05-31,2012-05-18,12-May-2012,May 12th 2012,Sun 05-12-13,1,0,0,0,2012-03-21,365,91,31,7,2012-12-30,2012-03-31,2012-04-28,2012-05-12,2012-05-06,2012-05-12,Sun       ,2012M05,May-2012,N,2012M05   ,N,Year 2012 - Quarter 02,2012Q02   ,N,20,2012,2012-05-12,2012-05-18,Y,Wk.20 - 13,2012-05-12 00:00:00,2012-05-18 00:00:00,2012W20   ,2012W20   ,05/12/13 - 05/18/13,05/12 - 05/18,2012,N,2012-05-12,2011-05-12,2012-02-12,2012-11-12,2012-04-12,2012-03-12,2012-05-05,2012-04-28,0,0,0,0,0,0,0,0,5,2,20,2,1,1,1,2005-09-07,USER_X  ,2012-11-27 00:16:56,USER_X
+2012-08-17,2012-01-01,2012-07-01,2012-08-01,2012-08-11,0,-1,-3,-15,-102,0,-1,-4,-15,-15,41502,229,48,17,7,33,5928,2,8,1364,3,455,114,2012-12-31,2012-09-30,2012-08-31,2012-08-17,17-Aug-2012,Aug 17th 2012,Sat 08-17-13,1,0,0,0,2012-06-21,365,92,31,7,2012-12-30,2012-06-30,2012-07-28,2012-08-11,2012-08-12,2012-08-17,Sat       ,2012M08,Aug-2012,N,2012M08   ,N,Year 2012 - Quarter 03,2012Q03   ,N,33,2012,2012-08-11,2012-08-17,N,Wk.33 - 13,2012-08-11 00:00:00,2012-08-17 00:00:00,2012W33   ,2012W33   ,08/11/13 - 08/17/13,08/11 - 08/17,2012,N,2012-08-17,2011-08-17,2012-05-17,2012-02-17,2012-07-17,2012-06-17,2012-08-10,2012-08-03,0,0,0,0,0,0,0,0,8,3,33,3,1,1,1,2005-09-07,USER_X  ,2012-11-27 00:16:56,USER_X
+2012-10-21,2012-01-01,2012-10-01,2012-10-01,2012-10-20,0,0,-1,-5,-37,0,0,-2,-5,-5,41567,294,21,21,2,43,5938,1,10,1366,4,456,114,2012-12-31,2012-12-31,2012-10-31,2012-10-26,21-Oct-2012,Oct 21st 2012,Mon 10-21-13,1,1,0,0,2012-09-21,365,92,31,7,2012-12-30,2012-09-29,2012-09-29,2012-10-20,2012-10-21,2012-10-21,Mon       ,2012M10,Oct-2012,N,2012M10   ,N,Year 2012 - Quarter 04,2012Q04   ,N,43,2012,2012-10-20,2012-10-26,N,Wk.43 - 13,2012-10-20 00:00:00,2012-10-26 00:00:00,2012W43   ,2012W43   ,10/20/13 - 10/26/13,10/20 - 10/26,2012,N,2012-10-21,2011-10-21,2012-07-21,2012-04-21,2012-09-21,2012-08-21,2012-10-14,2012-10-07,0,0,0,0,0,0,0,0,10,4,43,4,1,1,1,2005-09-07,USER_X  ,2012-11-27 00:16:56,USER_X
+2012-03-27,2012-01-01,2012-01-01,2012-03-01,2012-03-24,0,-3,-8,-35,-245,0,-3,-9,-35,-35,41359,86,86,27,4,13,5908,3,3,1359,1,453,114,2012-12-31,2012-03-31,2012-03-31,2012-03-30,27-Mar-2012,Mar 27th 2012,Wed 03-27-13,1,0,0,0,2012-03-21,365,90,31,7,2012-12-30,2012-12-30,2012-02-24,2012-03-24,2012-03-25,2012-03-27,Wed       ,2012M03,Mar-2012,N,2012M03   ,N,Year 2012 - Quarter 01,2012Q01   ,N,13,2012,2012-03-24,2012-03-30,N,Wk.13 - 13,2012-03-24 00:00:00,2012-03-30 00:00:00,2012W13   ,2012W13   ,03/24/13 - 03/30/13,03/24 - 03/30,2012,N,2012-03-27,2011-03-27,2012-12-27,2012-09-27,2012-02-27,2012-01-27,2012-03-20,2012-03-13,0,0,0,0,0,0,0,0,3,1,13,2,1,1,1,2005-09-07,USER_X  ,2012-11-27 00:16:56,USER_X
+2012-11-22,2012-01-01,2012-10-01,2012-11-01,2012-11-17,0,0,0,-1,-5,0,0,-1,-1,-1,41599,326,53,22,6,47,5942,2,11,1367,4,456,114,2012-12-31,2012-12-31,2012-11-30,2012-11-23,22-Nov-2012,Nov 22nd 2012,Fri 11-22-13,1,1,1,0,2012-09-21,365,92,30,7,2012-12-30,2012-09-29,2012-10-27,2012-11-17,2012-11-18,2012-11-22,Fri       ,2012M11,Nov-2012,N,2012M11   ,N,Year 2012 - Quarter 04,2012Q04   ,N,47,2012,2012-11-17,2012-11-23,N,Wk.47 - 13,2012-11-17 00:00:00,2012-11-23 00:00:00,2012W47   ,2012W47   ,11/17/13 - 11/23/13,11/17 - 11/23,2012,N,2012-11-22,2011-11-22,2012-08-22,2012-05-22,2012-10-22,2012-09-22,2012-11-15,2012-11-08,0,0,0,0,0,0,0,0,11,4,47,4,1,1,1,2005-09-07,USER_X  ,2012-11-27 00:16:56,USER_X
+2012-07-07,2012-01-01,2012-07-01,2012-07-01,2012-07-07,0,-1,-4,-20,-143,0,-1,-5,-20,-21,41461,188,7,7,1,28,5923,1,7,1363,3,455,114,2012-12-31,2012-09-30,2012-07-31,2012-07-13,07-Jul-2012,Jul 7th 2012,Sun 07-07-13,1,0,0,0,2012-06-21,365,92,31,7,2012-12-30,2012-06-30,2012-06-30,2012-07-07,2012-07-01,2012-07-07,Sun       ,2012M07,Jul-2012,N,2012M07   ,N,Year 2012 - Quarter 03,2012Q03   ,N,28,2012,2012-07-07,2012-07-13,Y,Wk.28 - 13,2012-07-07 00:00:00,2012-07-13 00:00:00,2012W28   ,2012W28   ,07/07/13 - 07/13/13,07/07 - 07/13,2012,N,2012-07-07,2011-07-07,2012-04-07,2012-01-07,2012-06-07,2012-05-07,2012-06-30,2012-06-23,0,0,0,0,0,0,0,0,7,3,28,3,1,1,1,2005-09-07,USER_X  ,2012-11-27 00:16:56,USER_X
+2012-09-11,2012-01-01,2012-07-01,2012-09-01,2012-09-08,0,-1,-2,-11,-77,0,-1,-3,-11,-11,41527,254,73,11,4,37,5932,3,9,1365,3,455,114,2012-12-31,2012-09-30,2012-09-30,2012-09-14,11-Sep-2012,Sep 11th 2012,Wed 09-11-13,1,0,0,0,2012-06-21,365,92,30,7,2012-12-30,2012-06-30,2012-08-25,2012-09-08,2012-09-09,2012-09-11,Wed       ,2012M09,Sep-2012,N,2012M09   ,N,Year 2012 - Quarter 03,2012Q03   ,N,37,2012,2012-09-08,2012-09-14,N,Wk.37 - 13,2012-09-08 00:00:00,2012-09-14 00:00:00,2012W37   ,2012W37   ,09/08/13 - 09/14/13,09/08 - 09/14,2012,N,2012-09-11,2011-09-11,2012-06-11,2012-03-11,2012-08-11,2012-07-11,2012-09-04,2012-08-28,0,0,0,0,0,0,0,0,9,3,37,3,1,1,1,2005-09-07,USER_X  ,2012-11-27 00:16:56,USER_X
+2012-10-19,2012-01-01,2012-10-01,2012-10-01,2012-10-13,0,0,-1,-6,-39,0,0,-2,-6,-6,41565,292,19,19,7,42,5937,1,10,1366,4,456,114,2012-12-31,2012-12-31,2012-10-31,2012-10-19,19-Oct-2012,Oct 19th 2012,Sat 10-19-13,1,1,0,0,2012-09-21,365,92,31,7,2012-12-30,2012-09-29,2012-09-29,2012-10-13,2012-10-14,2012-10-19,Sat       ,2012M10,Oct-2012,N,2012M10   ,N,Year 2012 - Quarter 04,2012Q04   ,N,42,2012,2012-10-13,2012-10-19,N,Wk.42 - 13,2012-10-13 00:00:00,2012-10-19 00:00:00,2012W42   ,2012W42   ,10/13/13 - 10/19/13,10/13 - 10/19,2012,N,2012-10-19,2011-10-19,2012-07-19,2012-04-19,2012-09-19,2012-08-19,2012-10-12,2012-10-05,0,0,0,0,0,0,0,0,10,4,42,4,1,1,1,2005-09-07,USER_X  ,2012-11-27 00:16:56,USER_X
+2012-12-23,2012-01-01,2012-10-01,2012-12-01,2012-12-22,0,0,1,4,26,0,0,0,4,4,41630,357,84,23,2,52,5947,3,12,1368,4,456,114,2012-12-31,2012-12-31,2012-12-31,2012-12-28,23-Dec-2012,Dec 23rd 2012,Mon 12-23-13,0,0,0,0,2012-12-21,365,92,31,7,2012-12-30,2012-09-29,2012-11-24,2012-12-22,2012-12-23,2012-12-23,Mon       ,2012M12,Dec-2012,N,2012M12   ,N,Year 2012 - Quarter 04,2012Q04   ,N,52,2012,2012-12-22,2012-12-28,N,Wk.52 - 13,2012-12-22 00:00:00,2012-12-28 00:00:00,2012W52   ,2012W52   ,12/22/13 - 12/28/13,12/22 - 12/28,2012,N,2012-12-23,2011-12-23,2012-09-23,2012-06-23,2012-11-23,2012-10-23,2012-12-16,2012-12-09,0,0,0,0,0,0,0,0,12,4,52,1,0,1,0,2005-09-07,USER_X  ,2012-11-27 00:16:56,USER_X
+2012-05-24,2012-01-01,2012-04-01,2012-05-01,2012-05-19,0,-2,-6,-27,-187,0,-2,-7,-27,-27,41417,144,54,24,6,21,5916,2,5,1361,2,454,114,2012-12-31,2012-06-30,2012-05-31,2012-05-25,24-May-2012,May 24th 2012,Fri 05-24-13,1,0,0,0,2012-03-21,365,91,31,7,2012-12-30,2012-03-31,2012-04-28,2012-05-19,2012-05-20,2012-05-24,Fri       ,2012M05,May-2012,N,2012M05   ,N,Year 2012 - Quarter 02,2012Q02   ,N,21,2012,2012-05-19,2012-05-25,N,Wk.21 - 13,2012-05-19 00:00:00,2012-05-25 00:00:00,2012W21   ,2012W21   ,05/19/13 - 05/25/13,05/19 - 05/25,2012,N,2012-05-24,2011-05-24,2012-02-24,2012-11-24,2012-04-24,2012-03-24,2012-05-17,2012-05-10,0,0,0,0,0,0,0,0,5,2,21,2,1,1,1,2005-09-07,USER_X  ,2012-11-27 00:16:56,USER_X
+2012-11-05,2012-01-01,2012-10-01,2012-11-01,2012-11-03,0,0,0,-3,-22,0,0,-1,-3,-3,41582,309,36,5,3,45,5940,2,11,1367,4,456,114,2012-12-31,2012-12-31,2012-11-30,2012-11-09,05-Nov-2012,Nov 5th 2012,Tue 11-05-13,1,1,1,0,2012-09-21,365,92,30,7,2012-12-30,2012-09-29,2012-10-27,2012-11-03,2012-11-04,2012-11-05,Tue       ,2012M11,Nov-2012,N,2012M11   ,N,Year 2012 - Quarter 04,2012Q04   ,N,45,2012,2012-11-03,2012-11-09,N,Wk.45 - 13,2012-11-03 00:00:00,2012-11-09 00:00:00,2012W45   ,2012W45   ,11/03/13 - 11/09/13,11/03 - 11/09,2012,N,2012-11-05,2011-11-05,2012-08-05,2012-05-05,2012-10-05,2012-09-05,2012-10-29,2012-10-22,0,0,0,0,0,0,0,0,11,4,45,4,1,1,1,2005-09-07,USER_X  ,2012-11-27 00:16:56,USER_X
+2012-05-29,2012-01-01,2012-04-01,2012-05-01,2012-05-26,0,-2,-6,-26,-182,0,-2,-6,-26,-26,41422,149,59,29,4,22,5917,2,5,1361,2,454,114,2012-12-31,2012-06-30,2012-05-31,2012-06-01,29-May-2012,May 29th 2012,Wed 05-29-13,1,0,0,0,2012-03-21,365,91,31,7,2012-12-30,2012-03-31,2012-05-26,2012-05-26,2012-05-27,2012-05-29,Wed       ,2012M05,May-2012,N,2012M05   ,N,Year 2012 - Quarter 02,2012Q02   ,N,22,2012,2012-05-26,2012-06-01,N,Wk.22 - 13,2012-05-26 00:00:00,2012-06-01 00:00:00,2012W22   ,2012W22   ,05/26/13 - 06/01/13,05/26 - 06/01,2012,N,2012-05-29,2011-05-29,2012-02-28,2012-11-29,2012-04-29,2012-03-29,2012-05-22,2012-05-15,0,0,0,0,0,0,0,0,6,2,22,2,1,1,1,2005-09-07,USER_X  ,2012-11-27 00:16:56,USER_X
+2012-01-02,2012-01-01,2012-01-01,2012-01-01,2012-01-01,0,-3,-10,-47,-329,0,-3,-11,-47,-47,41275,2,2,2,4,1,5896,1,1,1357,1,453,114,2012-12-31,2012-03-31,2012-01-31,2012-01-05,02-Jan-2012,Jan 2nd 2012,Wed 01-02-13,1,0,0,0,2012-12-21,365,90,31,5,2012-12-30,2012-12-30,2012-12-30,2012-12-30,2012-12-31,2012-01-02,Wed       ,2012M01,Jan-2012,N,2012M01   ,N,Year 2012 - Quarter 01,2012Q01   ,N,1,2012,2012-12-30,2012-01-05,N,Wk.01 - 13,2012-01-01 00:00:00,2012-01-05 00:00:00,2012W01   ,2012W01   ,01/01/13 - 01/05/13,01/01 - 01/05,2012,N,2012-01-02,2011-01-02,2012-10-02,2012-07-02,2012-12-02,2012-11-02,2012-12-26,2012-12-19,0,0,0,0,0,0,0,0,1,1,1,1,1,1,1,2005-09-07,USER_X  ,2012-11-27 00:16:56,USER_X
+2012-12-02,2012-01-01,2012-10-01,2012-12-01,2012-12-01,0,0,1,1,5,0,0,0,1,1,41609,336,63,2,2,49,5944,3,12,1368,4,456,114,2012-12-31,2012-12-31,2012-12-31,2012-12-07,02-Dec-2012,Dec 2nd 2012,Mon 12-02-13,0,0,0,0,2012-09-21,365,92,31,7,2012-12-30,2012-09-29,2012-11-24,2012-12-01,2012-12-02,2012-12-02,Mon       ,2012M12,Dec-2012,N,2012M12   ,N,Year 2012 - Quarter 04,2012Q04   ,N,49,2012,2012-12-01,2012-12-07,N,Wk.49 - 13,2012-12-01 00:00:00,2012-12-07 00:00:00,2012W49   ,2012W49   ,12/01/13 - 12/07/13,12/01 - 12/07,2012,N,2012-12-02,2011-12-02,2012-09-02,2012-06-02,2012-11-02,2012-10-02,2012-11-25,2012-11-18,0,0,0,0,0,0,0,0,12,4,49,4,0,1,0,2005-09-07,USER_X  ,2012-11-27 00:16:56,USER_X
+2012-02-24,2012-01-01,2012-01-01,2012-02-01,2012-02-24,0,-3,-9,-39,-276,0,-3,-9,-39,-40,41328,55,55,24,1,9,5904,2,2,1358,1,453,114,2012-12-31,2012-03-31,2012-02-28,2012-03-02,24-Feb-2012,Feb 24th 2012,Sun 02-24-13,1,0,0,0,2012-12-21,365,90,28,7,2012-12-30,2012-12-30,2012-02-24,2012-02-24,2012-02-18,2012-02-24,Sun       ,2012M02,Feb-2012,N,2012M02   ,N,Year 2012 - Quarter 01,2012Q01   ,N,9,2012,2012-02-24,2012-03-02,Y,Wk.09 - 13,2012-02-24 00:00:00,2012-03-02 00:00:00,2012W09   ,2012W09   ,02/24/13 - 03/02/13,02/24 - 03/02,2012,N,2012-02-24,

<TRUNCATED>

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

Posted by lu...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/server/src/main/java/com/kylinolap/rest/service/AclService.java
----------------------------------------------------------------------
diff --git a/server/src/main/java/com/kylinolap/rest/service/AclService.java b/server/src/main/java/com/kylinolap/rest/service/AclService.java
new file mode 100644
index 0000000..0873dd2
--- /dev/null
+++ b/server/src/main/java/com/kylinolap/rest/service/AclService.java
@@ -0,0 +1,465 @@
+/*
+ * 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.rest.service;
+
+import java.io.IOException;
+import java.io.Serializable;
+import java.lang.reflect.Field;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.NavigableMap;
+
+import org.apache.commons.io.IOUtils;
+import org.apache.hadoop.hbase.client.Delete;
+import org.apache.hadoop.hbase.client.Get;
+import org.apache.hadoop.hbase.client.HTableInterface;
+import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.client.Result;
+import org.apache.hadoop.hbase.client.ResultScanner;
+import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.filter.CompareFilter.CompareOp;
+import org.apache.hadoop.hbase.filter.SingleColumnValueFilter;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.springframework.beans.factory.annotation.Autowired;
+import org.springframework.security.acls.domain.AccessControlEntryImpl;
+import org.springframework.security.acls.domain.AclAuthorizationStrategy;
+import org.springframework.security.acls.domain.AclImpl;
+import org.springframework.security.acls.domain.AuditLogger;
+import org.springframework.security.acls.domain.GrantedAuthoritySid;
+import org.springframework.security.acls.domain.ObjectIdentityImpl;
+import org.springframework.security.acls.domain.PermissionFactory;
+import org.springframework.security.acls.domain.PrincipalSid;
+import org.springframework.security.acls.model.AccessControlEntry;
+import org.springframework.security.acls.model.Acl;
+import org.springframework.security.acls.model.AlreadyExistsException;
+import org.springframework.security.acls.model.ChildrenExistException;
+import org.springframework.security.acls.model.MutableAcl;
+import org.springframework.security.acls.model.MutableAclService;
+import org.springframework.security.acls.model.NotFoundException;
+import org.springframework.security.acls.model.ObjectIdentity;
+import org.springframework.security.acls.model.PermissionGrantingStrategy;
+import org.springframework.security.acls.model.Sid;
+import org.springframework.security.core.Authentication;
+import org.springframework.security.core.context.SecurityContextHolder;
+import org.springframework.security.util.FieldUtils;
+import org.springframework.stereotype.Component;
+import org.springframework.util.Assert;
+
+import com.fasterxml.jackson.core.JsonParseException;
+import com.fasterxml.jackson.databind.JsonMappingException;
+import com.kylinolap.common.KylinConfig;
+import com.kylinolap.common.persistence.HBaseConnection;
+import com.kylinolap.rest.util.Serializer;
+
+/**
+ * @author xduo
+ * 
+ */
+@Component("aclService")
+public class AclService implements MutableAclService {
+
+    private static final Logger logger = LoggerFactory.getLogger(AclService.class);
+
+    public static final String ACL_INFO_FAMILY = "i";
+    public static final String ACL_ACES_FAMILY = "a";
+    private static final String DEFAULT_TABLE_PREFIX = "kylin_metadata";
+    private static final String ACL_TABLE_NAME = "_acl";
+    private static final String ACL_INFO_FAMILY_TYPE_COLUMN = "t";
+    private static final String ACL_INFO_FAMILY_OWNER_COLUMN = "o";
+    private static final String ACL_INFO_FAMILY_PARENT_COLUMN = "p";
+    private static final String ACL_INFO_FAMILY_ENTRY_INHERIT_COLUMN = "i";
+
+    private Serializer<SidInfo> sidSerializer = new Serializer<SidInfo>(SidInfo.class);
+    private Serializer<DomainObjectInfo> domainObjSerializer = new Serializer<DomainObjectInfo>(DomainObjectInfo.class);
+    private Serializer<AceInfo> aceSerializer = new Serializer<AceInfo>(AceInfo.class);
+
+    private String hbaseUrl = null;
+    private String tableNameBase = null;
+    private String aclTableName = null;
+
+    private final Field fieldAces = FieldUtils.getField(AclImpl.class, "aces");
+    private final Field fieldAcl = FieldUtils.getField(AccessControlEntryImpl.class, "acl");
+
+    @Autowired
+    protected PermissionGrantingStrategy permissionGrantingStrategy;
+
+    @Autowired
+    protected PermissionFactory aclPermissionFactory;
+
+    @Autowired
+    protected AclAuthorizationStrategy aclAuthorizationStrategy;
+
+    @Autowired
+    protected AuditLogger auditLogger;
+
+    public AclService() throws IOException {
+        String metadataUrl = KylinConfig.getInstanceFromEnv().getMetadataUrl();
+        // split TABLE@HBASE_URL
+        int cut = metadataUrl.indexOf('@');
+        tableNameBase = cut < 0 ? DEFAULT_TABLE_PREFIX : metadataUrl.substring(0, cut);
+        hbaseUrl = cut < 0 ? metadataUrl : metadataUrl.substring(cut + 1);
+        aclTableName = tableNameBase + ACL_TABLE_NAME;
+
+        fieldAces.setAccessible(true);
+        fieldAcl.setAccessible(true);
+
+        HBaseConnection.createHTableIfNeeded(hbaseUrl, aclTableName, ACL_INFO_FAMILY, ACL_ACES_FAMILY);
+    }
+
+    @Override
+    public List<ObjectIdentity> findChildren(ObjectIdentity parentIdentity) {
+        List<ObjectIdentity> oids = new ArrayList<ObjectIdentity>();
+        HTableInterface htable = null;
+        try {
+            htable = HBaseConnection.get(hbaseUrl).getTable(aclTableName);
+
+            Scan scan = new Scan();
+            SingleColumnValueFilter parentFilter = new SingleColumnValueFilter(Bytes.toBytes(ACL_INFO_FAMILY), Bytes.toBytes(ACL_INFO_FAMILY_PARENT_COLUMN), CompareOp.EQUAL, domainObjSerializer.serialize(new DomainObjectInfo(parentIdentity)));
+            parentFilter.setFilterIfMissing(true);
+            scan.setFilter(parentFilter);
+
+            ResultScanner scanner = htable.getScanner(scan);
+            for (Result result = scanner.next(); result != null; result = scanner.next()) {
+                String id = Bytes.toString(result.getRow());
+                String type = Bytes.toString(result.getValue(Bytes.toBytes(ACL_INFO_FAMILY), Bytes.toBytes(ACL_INFO_FAMILY_TYPE_COLUMN)));
+
+                oids.add(new ObjectIdentityImpl(type, id));
+            }
+        } catch (IOException e) {
+            throw new RuntimeException(e.getMessage(), e);
+        } finally {
+            IOUtils.closeQuietly(htable);
+        }
+
+        return oids;
+    }
+
+    @Override
+    public Acl readAclById(ObjectIdentity object) throws NotFoundException {
+        Map<ObjectIdentity, Acl> aclsMap = readAclsById(Arrays.asList(object), null);
+        //        Assert.isTrue(aclsMap.containsKey(object), "There should have been an Acl entry for ObjectIdentity " + object);
+
+        return aclsMap.get(object);
+    }
+
+    @Override
+    public Acl readAclById(ObjectIdentity object, List<Sid> sids) throws NotFoundException {
+        Map<ObjectIdentity, Acl> aclsMap = readAclsById(Arrays.asList(object), sids);
+        Assert.isTrue(aclsMap.containsKey(object), "There should have been an Acl entry for ObjectIdentity " + object);
+
+        return aclsMap.get(object);
+    }
+
+    @Override
+    public Map<ObjectIdentity, Acl> readAclsById(List<ObjectIdentity> objects) throws NotFoundException {
+        return readAclsById(objects, null);
+    }
+
+    @Override
+    public Map<ObjectIdentity, Acl> readAclsById(List<ObjectIdentity> oids, List<Sid> sids) throws NotFoundException {
+        Map<ObjectIdentity, Acl> aclMaps = new HashMap<ObjectIdentity, Acl>();
+        HTableInterface htable = null;
+        Result result = null;
+        try {
+            htable = HBaseConnection.get(hbaseUrl).getTable(aclTableName);
+
+            for (ObjectIdentity oid : oids) {
+                result = htable.get(new Get(Bytes.toBytes(String.valueOf(oid.getIdentifier()))));
+
+                if (null != result && !result.isEmpty()) {
+                    SidInfo owner = sidSerializer.deserialize(result.getValue(Bytes.toBytes(ACL_INFO_FAMILY), Bytes.toBytes(ACL_INFO_FAMILY_OWNER_COLUMN)));
+                    Sid ownerSid = (null == owner) ? null : (owner.isPrincipal() ? new PrincipalSid(owner.getSid()) : new GrantedAuthoritySid(owner.getSid()));
+                    boolean entriesInheriting = Bytes.toBoolean(result.getValue(Bytes.toBytes(ACL_INFO_FAMILY), Bytes.toBytes(ACL_INFO_FAMILY_ENTRY_INHERIT_COLUMN)));
+
+                    Acl parentAcl = null;
+                    DomainObjectInfo parentInfo = domainObjSerializer.deserialize(result.getValue(Bytes.toBytes(ACL_INFO_FAMILY), Bytes.toBytes(ACL_INFO_FAMILY_PARENT_COLUMN)));
+                    if (null != parentInfo) {
+                        ObjectIdentity parentObj = new ObjectIdentityImpl(parentInfo.getType(), parentInfo.getId());
+                        parentAcl = readAclById(parentObj, null);
+                    }
+
+                    AclImpl acl = new AclImpl(oid, oid.getIdentifier(), aclAuthorizationStrategy, permissionGrantingStrategy, parentAcl, null, entriesInheriting, ownerSid);
+                    genAces(sids, result, acl);
+
+                    aclMaps.put(oid, acl);
+                } else {
+                    throw new NotFoundException("Unable to find ACL information for object identity '" + oid + "'");
+                }
+            }
+        } catch (IOException e) {
+            throw new RuntimeException(e.getMessage(), e);
+        } finally {
+            IOUtils.closeQuietly(htable);
+        }
+
+        return aclMaps;
+    }
+
+    @Override
+    public MutableAcl createAcl(ObjectIdentity objectIdentity) throws AlreadyExistsException {
+        Acl acl = null;
+
+        try {
+            acl = readAclById(objectIdentity);
+        } catch (NotFoundException e) {
+        }
+        if (null != acl) {
+            throw new AlreadyExistsException("ACL of " + objectIdentity + " exists!");
+        }
+
+        Authentication auth = SecurityContextHolder.getContext().getAuthentication();
+        PrincipalSid sid = new PrincipalSid(auth);
+
+        HTableInterface htable = null;
+        try {
+            htable = HBaseConnection.get(hbaseUrl).getTable(aclTableName);
+            Put put = new Put(Bytes.toBytes(String.valueOf(objectIdentity.getIdentifier())));
+            put.add(Bytes.toBytes(ACL_INFO_FAMILY), Bytes.toBytes(ACL_INFO_FAMILY_TYPE_COLUMN), Bytes.toBytes(objectIdentity.getType()));
+            put.add(Bytes.toBytes(ACL_INFO_FAMILY), Bytes.toBytes(ACL_INFO_FAMILY_OWNER_COLUMN), sidSerializer.serialize(new SidInfo(sid)));
+            put.add(Bytes.toBytes(ACL_INFO_FAMILY), Bytes.toBytes(ACL_INFO_FAMILY_ENTRY_INHERIT_COLUMN), Bytes.toBytes(true));
+
+            htable.put(put);
+            htable.flushCommits();
+
+            logger.debug("ACL of " + objectIdentity + " created successfully.");
+        } catch (IOException e) {
+            throw new RuntimeException(e.getMessage(), e);
+        } finally {
+            IOUtils.closeQuietly(htable);
+        }
+
+        return (MutableAcl) readAclById(objectIdentity);
+    }
+
+    @Override
+    public void deleteAcl(ObjectIdentity objectIdentity, boolean deleteChildren) throws ChildrenExistException {
+        HTableInterface htable = null;
+        try {
+            htable = HBaseConnection.get(hbaseUrl).getTable(aclTableName);
+            Delete delete = new Delete(Bytes.toBytes(String.valueOf(objectIdentity.getIdentifier())));
+
+            List<ObjectIdentity> children = findChildren(objectIdentity);
+            if (!deleteChildren && children.size() > 0) {
+                throw new ChildrenExistException("Children exists for " + objectIdentity);
+            }
+
+            for (ObjectIdentity oid : children) {
+                deleteAcl(oid, deleteChildren);
+            }
+
+            htable.delete(delete);
+            htable.flushCommits();
+
+            logger.debug("ACL of " + objectIdentity + " deleted successfully.");
+        } catch (IOException e) {
+            throw new RuntimeException(e.getMessage(), e);
+        } finally {
+            IOUtils.closeQuietly(htable);
+        }
+    }
+
+    @Override
+    public MutableAcl updateAcl(MutableAcl acl) throws NotFoundException {
+        try {
+            readAclById(acl.getObjectIdentity());
+        } catch (NotFoundException e) {
+            throw e;
+        }
+
+        HTableInterface htable = null;
+        try {
+            htable = HBaseConnection.get(hbaseUrl).getTable(aclTableName);
+            Delete delete = new Delete(Bytes.toBytes(String.valueOf(acl.getObjectIdentity().getIdentifier())));
+            delete.deleteFamily(Bytes.toBytes(ACL_ACES_FAMILY));
+            htable.delete(delete);
+
+            Put put = new Put(Bytes.toBytes(String.valueOf(acl.getObjectIdentity().getIdentifier())));
+
+            if (null != acl.getParentAcl()) {
+                put.add(Bytes.toBytes(ACL_INFO_FAMILY), Bytes.toBytes(ACL_INFO_FAMILY_PARENT_COLUMN), domainObjSerializer.serialize(new DomainObjectInfo(acl.getParentAcl().getObjectIdentity())));
+            }
+
+            for (AccessControlEntry ace : acl.getEntries()) {
+                AceInfo aceInfo = new AceInfo(ace);
+                put.add(Bytes.toBytes(ACL_ACES_FAMILY), Bytes.toBytes(aceInfo.getSidInfo().getSid()), aceSerializer.serialize(aceInfo));
+            }
+
+            if (!put.isEmpty()) {
+                htable.put(put);
+                htable.flushCommits();
+
+                logger.debug("ACL of " + acl.getObjectIdentity() + " updated successfully.");
+            }
+        } catch (IOException e) {
+            throw new RuntimeException(e.getMessage(), e);
+        } finally {
+            IOUtils.closeQuietly(htable);
+        }
+
+        return (MutableAcl) readAclById(acl.getObjectIdentity());
+    }
+
+    private void genAces(List<Sid> sids, Result result, AclImpl acl) throws JsonParseException, JsonMappingException, IOException {
+        List<AceInfo> aceInfos = new ArrayList<AceInfo>();
+        if (null != sids) {
+            // Just return aces in sids
+            for (Sid sid : sids) {
+                String sidName = null;
+                if (sid instanceof PrincipalSid) {
+                    sidName = ((PrincipalSid) sid).getPrincipal();
+                } else if (sid instanceof GrantedAuthoritySid) {
+                    sidName = ((GrantedAuthoritySid) sid).getGrantedAuthority();
+                }
+
+                AceInfo aceInfo = aceSerializer.deserialize(result.getValue(Bytes.toBytes(ACL_ACES_FAMILY), Bytes.toBytes(sidName)));
+                if (null != aceInfo) {
+                    aceInfos.add(aceInfo);
+                }
+            }
+        } else {
+            NavigableMap<byte[], byte[]> familyMap = result.getFamilyMap(Bytes.toBytes(ACL_ACES_FAMILY));
+            for (byte[] qualifier : familyMap.keySet()) {
+                AceInfo aceInfo = aceSerializer.deserialize(familyMap.get(qualifier));
+
+                if (null != aceInfo) {
+                    aceInfos.add(aceInfo);
+                }
+            }
+        }
+
+        List<AccessControlEntry> newAces = new ArrayList<AccessControlEntry>();
+        for (int i = 0; i < aceInfos.size(); i++) {
+            AceInfo aceInfo = aceInfos.get(i);
+
+            if (null != aceInfo) {
+                Sid sid = aceInfo.getSidInfo().isPrincipal() ? new PrincipalSid(aceInfo.getSidInfo().getSid()) : new GrantedAuthoritySid(aceInfo.getSidInfo().getSid());
+                AccessControlEntry ace = new AccessControlEntryImpl(Long.valueOf(i), acl, sid, aclPermissionFactory.buildFromMask(aceInfo.getPermissionMask()), true, false, false);
+                newAces.add(ace);
+            }
+        }
+
+        this.setAces(acl, newAces);
+    }
+
+    private void setAces(AclImpl acl, List<AccessControlEntry> aces) {
+        try {
+            fieldAces.set(acl, aces);
+        } catch (IllegalAccessException e) {
+            throw new IllegalStateException("Could not set AclImpl entries", e);
+        }
+    }
+
+    protected static class DomainObjectInfo {
+        private String id;
+        private String type;
+
+        public DomainObjectInfo() {
+        }
+
+        public DomainObjectInfo(ObjectIdentity oid) {
+            super();
+            this.id = (String) oid.getIdentifier();
+            this.type = oid.getType();
+        }
+
+        public Serializable getId() {
+            return id;
+        }
+
+        public void setId(String id) {
+            this.id = id;
+        }
+
+        public String getType() {
+            return type;
+        }
+
+        public void setType(String type) {
+            this.type = type;
+        }
+    }
+
+    protected static class SidInfo {
+        private String sid;
+        private boolean isPrincipal;
+
+        public SidInfo() {
+        }
+
+        public SidInfo(Sid sid) {
+            if (sid instanceof PrincipalSid) {
+                this.sid = ((PrincipalSid) sid).getPrincipal();
+                this.isPrincipal = true;
+            } else if (sid instanceof GrantedAuthoritySid) {
+                this.sid = ((GrantedAuthoritySid) sid).getGrantedAuthority();
+                this.isPrincipal = false;
+            }
+        }
+
+        public String getSid() {
+            return sid;
+        }
+
+        public void setSid(String sid) {
+            this.sid = sid;
+        }
+
+        public boolean isPrincipal() {
+            return isPrincipal;
+        }
+
+        public void setPrincipal(boolean isPrincipal) {
+            this.isPrincipal = isPrincipal;
+        }
+    }
+
+    protected static class AceInfo {
+        private SidInfo sidInfo;
+        private int permissionMask;
+
+        public AceInfo() {
+        }
+
+        public AceInfo(AccessControlEntry ace) {
+            super();
+            this.sidInfo = new SidInfo(ace.getSid());
+            this.permissionMask = ace.getPermission().getMask();
+        }
+
+        public SidInfo getSidInfo() {
+            return sidInfo;
+        }
+
+        public void setSidInfo(SidInfo sidInfo) {
+            this.sidInfo = sidInfo;
+        }
+
+        public int getPermissionMask() {
+            return permissionMask;
+        }
+
+        public void setPermissionMask(int permissionMask) {
+            this.permissionMask = permissionMask;
+        }
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/server/src/main/java/com/kylinolap/rest/service/AdminService.java
----------------------------------------------------------------------
diff --git a/server/src/main/java/com/kylinolap/rest/service/AdminService.java b/server/src/main/java/com/kylinolap/rest/service/AdminService.java
new file mode 100644
index 0000000..f7a0211
--- /dev/null
+++ b/server/src/main/java/com/kylinolap/rest/service/AdminService.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.rest.service;
+
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+import java.util.Map;
+import java.util.Properties;
+
+import org.apache.commons.configuration.ConfigurationException;
+import org.apache.commons.configuration.PropertiesConfiguration;
+import org.apache.hadoop.util.ToolRunner;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.springframework.security.access.prepost.PreAuthorize;
+import org.springframework.stereotype.Component;
+
+import com.kylinolap.common.KylinConfig;
+import com.kylinolap.job.hadoop.cube.StorageCleanupJob;
+import com.kylinolap.rest.constant.Constant;
+import com.kylinolap.rest.exception.InternalErrorException;
+
+/**
+ * @author jianliu
+ */
+@Component("adminService")
+public class AdminService extends BasicService {
+    private static final Logger logger = LoggerFactory.getLogger(AdminService.class);
+
+    /**
+     * Get Java Env info as string
+     *
+     * @return
+     */
+    @PreAuthorize(Constant.ACCESS_HAS_ROLE_ADMIN)
+    public String getEnv() {
+        logger.debug("Get Kylin Runtime environment");
+        PropertiesConfiguration tempConfig = new PropertiesConfiguration();
+
+        // Add Java Env
+
+        try {
+            String content = "";
+            ByteArrayOutputStream baos = new ByteArrayOutputStream();
+            // env
+            Map<String, String> env = System.getenv();
+
+            for (Map.Entry<String, String> entry : env.entrySet()) {
+                tempConfig.addProperty(entry.getKey(), entry.getValue());
+            }
+
+            // properties
+            Properties proterties = System.getProperties();
+
+            for(Map.Entry<Object,Object> entry : proterties.entrySet())
+            {
+                tempConfig.setProperty((String) entry.getKey(), entry.getValue());
+            }
+
+
+            // do save
+            tempConfig.save(baos);
+            content = baos.toString();
+            return content;
+        } catch (ConfigurationException e) {
+            throw new InternalErrorException("Failed to get Kylin env Config", e);
+        }
+    }
+
+    /**
+     * Get Java config info as String
+     *
+     * @return
+     */
+    // @PreAuthorize(Constant.ACCESS_HAS_ROLE_ADMIN)
+    public String getConfigAsString() {
+        logger.debug("Get Kylin Runtime Config");
+
+        try {
+            return KylinConfig.getInstanceFromEnv().getConfigAsString();
+        } catch (IOException e) {
+            throw new InternalErrorException("Failed to get Kylin Runtime Config", e);
+        }
+    }
+
+    public void cleanupStorage() {
+        StorageCleanupJob job = new StorageCleanupJob();
+        String[] args = new String[] { "-delete", "true" };
+        try {
+            ToolRunner.run(job, args);
+        } catch (Exception e) {
+            throw new InternalErrorException(e.getMessage(), e);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/server/src/main/java/com/kylinolap/rest/service/BasicService.java
----------------------------------------------------------------------
diff --git a/server/src/main/java/com/kylinolap/rest/service/BasicService.java b/server/src/main/java/com/kylinolap/rest/service/BasicService.java
new file mode 100644
index 0000000..812c6aa
--- /dev/null
+++ b/server/src/main/java/com/kylinolap/rest/service/BasicService.java
@@ -0,0 +1,193 @@
+/*
+ * 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.rest.service;
+
+import java.io.File;
+import java.io.IOException;
+import java.net.InetAddress;
+import java.net.UnknownHostException;
+import java.nio.charset.Charset;
+import java.sql.Connection;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.sql.Statement;
+import java.util.List;
+import java.util.Properties;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+
+import javax.sql.DataSource;
+
+import org.apache.commons.lang3.StringUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.springframework.cache.annotation.CacheEvict;
+import org.springframework.cache.annotation.Caching;
+import org.springframework.jdbc.datasource.DriverManagerDataSource;
+
+import com.google.common.io.Files;
+import com.kylinolap.common.KylinConfig;
+import com.kylinolap.cube.CubeManager;
+import com.kylinolap.cube.project.ProjectInstance;
+import com.kylinolap.cube.project.ProjectManager;
+import com.kylinolap.job.JobManager;
+import com.kylinolap.job.engine.JobEngineConfig;
+import com.kylinolap.job.exception.JobException;
+import com.kylinolap.metadata.MetadataManager;
+import com.kylinolap.query.enumerator.OLAPQuery;
+import com.kylinolap.query.relnode.OLAPContext;
+import com.kylinolap.query.schema.OLAPSchemaFactory;
+import com.kylinolap.rest.controller.QueryController;
+
+public abstract class BasicService {
+
+    private static final Logger logger = LoggerFactory.getLogger(BasicService.class);
+
+    private static ConcurrentMap<String, DataSource> olapDataSources = new ConcurrentHashMap<String, DataSource>();
+
+//    @Autowired
+//    protected JdbcTemplate jdbcTemplate;
+
+    public KylinConfig getConfig() {
+        return KylinConfig.getInstanceFromEnv();
+    }
+
+    public void removeOLAPDataSource(String project) {
+        if (StringUtils.isEmpty(project))
+            throw new IllegalArgumentException("removeOLAPDataSource: project name not given");
+
+        project = ProjectInstance.getNormalizedProjectName(project);
+        olapDataSources.remove(project);
+    }
+
+    public static void resetOLAPDataSources() {
+        // brutal, yet simplest way
+        logger.info("resetOLAPDataSources is called.");
+        olapDataSources = new ConcurrentHashMap<String, DataSource>();
+    }
+
+    public DataSource getOLAPDataSource(String project) {
+
+        project = ProjectInstance.getNormalizedProjectName(project);
+
+        DataSource ret = olapDataSources.get(project);
+        if (ret == null) {
+            logger.debug("Creating a new data source");
+            logger.debug("OLAP data source pointing to " + getConfig());
+
+            File modelJson = OLAPSchemaFactory.createTempOLAPJson(project, getConfig());
+
+            try {
+                List<String> text = Files.readLines(modelJson, Charset.defaultCharset());
+                logger.debug("The new temp olap json is :");
+                for (String line : text)
+                    logger.debug(line);
+            } catch (IOException e) {
+                e.printStackTrace(); // logging failure is not critical
+            }
+
+            DriverManagerDataSource ds = new DriverManagerDataSource();
+            Properties props = new Properties();
+            props.setProperty(OLAPQuery.PROP_SCAN_THRESHOLD, String.valueOf(KylinConfig.getInstanceFromEnv().getScanThreshold()));
+            ds.setConnectionProperties(props);
+            ds.setDriverClassName("net.hydromatic.optiq.jdbc.Driver");
+            ds.setUrl("jdbc:calcite:model=" + modelJson.getAbsolutePath());
+
+            ret = olapDataSources.putIfAbsent(project, ds);
+            if (ret == null) {
+                ret = ds;
+            }
+        }
+        return ret;
+    }
+
+    /**
+     * Reload changed cube into cache
+     * 
+     * @param name
+     * @throws IOException
+     */
+    @Caching(evict = { @CacheEvict(value = QueryController.SUCCESS_QUERY_CACHE, allEntries = true), @CacheEvict(value = QueryController.EXCEPTION_QUERY_CACHE, allEntries = true) })
+    public void cleanDataCache() {
+        CubeManager.removeInstance(getConfig());
+        ProjectManager.removeInstance(getConfig());
+        BasicService.resetOLAPDataSources();
+    }
+
+    /**
+     * Reload the cube desc with name {name} into cache
+     * 
+     * @param name
+     */
+    public void reloadMetadataCache() {
+        MetadataManager.getInstance(getConfig()).reload();
+    }
+
+    public KylinConfig getKylinConfig() {
+        KylinConfig kylinConfig = KylinConfig.getInstanceFromEnv();
+
+        if (kylinConfig == null) {
+            throw new IllegalArgumentException("Failed to load kylin config instance");
+        }
+
+        return kylinConfig;
+    }
+
+    public MetadataManager getMetadataManager() {
+        return MetadataManager.getInstance(getConfig());
+    }
+
+    public CubeManager getCubeManager() {
+        return CubeManager.getInstance(getConfig());
+    }
+
+    public ProjectManager getProjectManager() {
+        return ProjectManager.getInstance(getConfig());
+    }
+
+    public JobManager getJobManager() throws JobException, UnknownHostException {
+        KylinConfig config = KylinConfig.getInstanceFromEnv();
+
+        JobEngineConfig engineCntx = new JobEngineConfig(config);
+
+        InetAddress ia = InetAddress.getLocalHost();
+        return new JobManager(ia.getCanonicalHostName(), engineCntx);
+    }
+
+    protected static void close(ResultSet resultSet, Statement stat, Connection conn) {
+        OLAPContext.clearParameter();
+
+        if (resultSet != null)
+            try {
+                resultSet.close();
+            } catch (SQLException e) {
+                logger.error("failed to close", e);
+            }
+        if (stat != null)
+            try {
+                stat.close();
+            } catch (SQLException e) {
+                logger.error("failed to close", e);
+            }
+        if (conn != null)
+            try {
+                conn.close();
+            } catch (SQLException e) {
+                logger.error("failed to close", e);
+            }
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/server/src/main/java/com/kylinolap/rest/service/CubeService.java
----------------------------------------------------------------------
diff --git a/server/src/main/java/com/kylinolap/rest/service/CubeService.java b/server/src/main/java/com/kylinolap/rest/service/CubeService.java
new file mode 100644
index 0000000..0215504
--- /dev/null
+++ b/server/src/main/java/com/kylinolap/rest/service/CubeService.java
@@ -0,0 +1,598 @@
+/*
+ * 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.rest.service;
+
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.net.UnknownHostException;
+import java.text.DateFormat;
+import java.text.ParseException;
+import java.text.SimpleDateFormat;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Date;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.commons.lang.StringUtils;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.client.HTable;
+import org.apache.hadoop.util.ToolRunner;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.springframework.beans.factory.annotation.Autowired;
+import org.springframework.cache.annotation.CacheEvict;
+import org.springframework.cache.annotation.Caching;
+import org.springframework.security.access.prepost.PostFilter;
+import org.springframework.security.access.prepost.PreAuthorize;
+import org.springframework.security.core.context.SecurityContextHolder;
+import org.springframework.stereotype.Component;
+
+import com.fasterxml.jackson.core.JsonGenerationException;
+import com.fasterxml.jackson.databind.JsonMappingException;
+import com.kylinolap.common.KylinConfig;
+import com.kylinolap.common.persistence.ResourceStore;
+import com.kylinolap.common.util.HBaseRegionSizeCalculator;
+import com.kylinolap.common.util.HadoopUtil;
+import com.kylinolap.common.util.JsonUtil;
+import com.kylinolap.cube.CubeInstance;
+import com.kylinolap.cube.CubeManager;
+import com.kylinolap.cube.CubeSegment;
+import com.kylinolap.cube.CubeSegmentStatusEnum;
+import com.kylinolap.cube.CubeStatusEnum;
+import com.kylinolap.cube.cuboid.CuboidCLI;
+import com.kylinolap.cube.exception.CubeIntegrityException;
+import com.kylinolap.cube.project.ProjectInstance;
+import com.kylinolap.job.JobDAO;
+import com.kylinolap.job.JobInstance;
+import com.kylinolap.job.JobInstance.JobStep;
+import com.kylinolap.job.constant.JobStatusEnum;
+import com.kylinolap.job.constant.JobStepStatusEnum;
+import com.kylinolap.job.exception.JobException;
+import com.kylinolap.job.hadoop.cardinality.HiveColumnCardinalityJob;
+import com.kylinolap.metadata.MetadataConstances;
+import com.kylinolap.metadata.model.cube.CubeDesc;
+import com.kylinolap.metadata.model.schema.ColumnDesc;
+import com.kylinolap.metadata.model.schema.TableDesc;
+import com.kylinolap.metadata.tool.HiveSourceTableLoader;
+import com.kylinolap.rest.constant.Constant;
+import com.kylinolap.rest.controller.QueryController;
+import com.kylinolap.rest.exception.InternalErrorException;
+import com.kylinolap.rest.request.MetricsRequest;
+import com.kylinolap.rest.response.HBaseResponse;
+import com.kylinolap.rest.response.MetricsResponse;
+import com.kylinolap.rest.security.AclPermission;
+
+/**
+ * Stateless & lightweight service facade of cube management functions.
+ *
+ * @author yangli9
+ */
+@Component("cubeMgmtService")
+public class CubeService extends BasicService {
+    private static final String DESC_SUFFIX = "_desc";
+
+    private static final Logger logger = LoggerFactory.getLogger(CubeService.class);
+
+    @Autowired
+    private AccessService accessService;
+
+    @PostFilter(Constant.ACCESS_POST_FILTER_READ)
+    public List<CubeInstance> listAllCubes(final String cubeName, final String projectName) {
+        List<CubeInstance> cubeInstances = null;
+        ProjectInstance project = (null != projectName) ? getProjectManager().getProject(projectName) : null;
+
+        if (null == project) {
+            cubeInstances = getCubeManager().listAllCubes();
+        } else {
+            cubeInstances = getProjectManager().listAllCubes(projectName);
+        }
+
+        List<CubeInstance> filterCubes = new ArrayList<CubeInstance>();
+        for (CubeInstance cubeInstance : cubeInstances) {
+            boolean isCubeMatch = (null == cubeName) || cubeInstance.getName().toLowerCase().contains(cubeName.toLowerCase());
+
+            if (isCubeMatch) {
+                filterCubes.add(cubeInstance);
+            }
+        }
+
+        return filterCubes;
+    }
+
+    public List<CubeInstance> getCubes(final String cubeName, final String projectName, final Integer limit, final Integer offset) {
+        int climit = (null == limit) ? 30 : limit;
+        int coffset = (null == offset) ? 0 : offset;
+
+        List<CubeInstance> cubes;
+        cubes = listAllCubes(cubeName, projectName);
+
+        if (cubes.size() <= coffset) {
+            return Collections.emptyList();
+        }
+
+        if ((cubes.size() - coffset) < climit) {
+            return cubes.subList(coffset, cubes.size());
+        }
+
+        return cubes.subList(coffset, coffset + climit);
+    }
+
+    @PreAuthorize(Constant.ACCESS_HAS_ROLE_ADMIN)
+    public CubeInstance updateCubeCost(String cubeName, int cost) throws IOException, CubeIntegrityException {
+        CubeInstance cube = getCubeManager().getCube(cubeName);
+        if (cube == null) {
+            throw new IOException("Cannot find cube " + cubeName);
+        }
+        if (cube.getCost() == cost) {
+            // Do nothing
+            return cube;
+        }
+        cube.setCost(cost);
+
+        String owner = SecurityContextHolder.getContext().getAuthentication().getName();
+        cube.setOwner(owner);
+
+        return getCubeManager().updateCube(cube);
+    }
+
+    public CubeInstance createCubeAndDesc(String cubeName, String projectName, CubeDesc desc) throws IOException {
+        if (getCubeManager().getCube(cubeName) != null) {
+            throw new InternalErrorException("The cube named " + cubeName + " already exists");
+        }
+
+        String owner = SecurityContextHolder.getContext().getAuthentication().getName();
+        CubeDesc createdDesc = null;
+        CubeInstance createdCube = null;
+
+        createdDesc = getMetadataManager().createCubeDesc(desc);
+
+        if (!createdDesc.getError().isEmpty()) {
+            getMetadataManager().removeCubeDesc(createdDesc);
+            throw new InternalErrorException(createdDesc.getError().get(0));
+        }
+
+        try {
+            int cuboidCount = CuboidCLI.simulateCuboidGeneration(createdDesc);
+            logger.info("New cube " + cubeName + " has " + cuboidCount + " cuboids");
+        } catch (Exception e) {
+            getMetadataManager().removeCubeDesc(createdDesc);
+            throw new InternalErrorException("Failed to deal with the request."+e.getLocalizedMessage(), e);
+        }
+
+        createdCube = getCubeManager().createCube(cubeName, projectName, createdDesc, owner);
+        accessService.init(createdCube, AclPermission.ADMINISTRATION);
+
+        ProjectInstance project = getProjectManager().getProject(projectName);
+        accessService.inherit(createdCube, project);
+
+        return createdCube;
+    }
+
+    @PreAuthorize(Constant.ACCESS_HAS_ROLE_ADMIN + " or hasPermission(#cube, 'ADMINISTRATION') or hasPermission(#cube, 'MANAGEMENT')")
+    public CubeDesc updateCubeAndDesc(CubeInstance cube, CubeDesc desc, String newProjectName) throws Exception {
+        List<JobInstance> jobInstances = this.getJobManager().listJobs(cube.getName(), null);
+        for (JobInstance jobInstance : jobInstances) {
+            if (jobInstance.getStatus() == JobStatusEnum.PENDING || jobInstance.getStatus() == JobStatusEnum.RUNNING) {
+                throw new JobException("Cube schema shouldn't be changed with running job.");
+            }
+        }
+
+        if (!cube.getDescriptor().calculateSignature().equals(cube.getDescriptor().getSignature())) {
+            this.releaseAllSegments(cube);
+        }
+
+        CubeDesc updatedCubeDesc = getMetadataManager().updateCubeDesc(desc);
+        if (updatedCubeDesc.getError().size() > 0)
+            return updatedCubeDesc;
+
+        int cuboidCount = CuboidCLI.simulateCuboidGeneration(updatedCubeDesc);
+        logger.info("Updated cube " + cube.getName() + " has " + cuboidCount + " cuboids");
+
+        if (!getProjectManager().isCubeInProject(newProjectName, cube)) {
+            String owner = SecurityContextHolder.getContext().getAuthentication().getName();
+            ProjectInstance newProject = getProjectManager().updateCubeToProject(cube.getName(), newProjectName, owner);
+            accessService.inherit(cube, newProject);
+        }
+
+        return updatedCubeDesc;
+    }
+
+    @PreAuthorize(Constant.ACCESS_HAS_ROLE_ADMIN + " or hasPermission(#cube, 'ADMINISTRATION') or hasPermission(#cube, 'MANAGEMENT')")
+    public void deleteCube(CubeInstance cube) throws IOException, JobException, CubeIntegrityException {
+        List<JobInstance> jobInstances = this.getJobManager().listJobs(cube.getName(), null);
+        for (JobInstance jobInstance : jobInstances) {
+            if (jobInstance.getStatus() == JobStatusEnum.PENDING || jobInstance.getStatus() == JobStatusEnum.RUNNING) {
+                throw new JobException("The cube " + cube.getName() + " has running job, please discard it and try again.");
+            }
+        }
+
+        this.releaseAllSegments(cube);
+        getCubeManager().dropCube(cube.getName(), true);
+        accessService.clean(cube, true);
+    }
+
+    public boolean isCubeEditable(CubeInstance ci) {
+        return ci.getStatus() == CubeStatusEnum.DISABLED;
+    }
+
+    public boolean isCubeDescEditable(CubeDesc cd) {
+        List<CubeInstance> list = getCubeManager().getCubesByDesc(cd.getName());
+        if (list.isEmpty()) {
+            return true;
+        }
+        Iterator<CubeInstance> it = list.iterator();
+        while (it.hasNext()) {
+            if (!isCubeEditable(it.next())) {
+                return false;
+            }
+        }
+        return true;
+    }
+
+    public static String getCubeDescNameFromCube(String cubeName) {
+        return cubeName + DESC_SUFFIX;
+    }
+
+    public static String getCubeNameFromDesc(String descName) {
+        if (descName.toLowerCase().endsWith(DESC_SUFFIX)) {
+            return descName.substring(0, descName.toLowerCase().indexOf(DESC_SUFFIX));
+        } else {
+            return descName;
+        }
+    }
+
+    public void reloadCubeCache(String cubeName) {
+        CubeInstance cube = CubeManager.getInstance(this.getConfig()).getCube(cubeName);
+        CubeManager.getInstance(this.getConfig()).loadCubeCache(cube);
+    }
+
+    public void removeCubeCache(String cubeName) {
+        CubeInstance cube = CubeManager.getInstance(this.getConfig()).getCube(cubeName);
+        CubeManager.getInstance(this.getConfig()).removeCubeCache(cube);
+    }
+
+    /**
+     * Stop all jobs belonging to this cube and clean out all segments
+     *
+     * @param cube
+     * @return
+     * @throws IOException
+     * @throws CubeIntegrityException
+     * @throws JobException
+     */
+    @PreAuthorize(Constant.ACCESS_HAS_ROLE_ADMIN + " or hasPermission(#cube, 'ADMINISTRATION') or hasPermission(#cube, 'OPERATION') or hasPermission(#cube, 'MANAGEMENT')")
+    @Caching(evict = { @CacheEvict(value = QueryController.SUCCESS_QUERY_CACHE, allEntries = true), @CacheEvict(value = QueryController.EXCEPTION_QUERY_CACHE, allEntries = true) })
+    public CubeInstance purgeCube(CubeInstance cube) throws IOException, CubeIntegrityException, JobException {
+        String cubeName = cube.getName();
+
+        CubeStatusEnum ostatus = cube.getStatus();
+        if (null != ostatus && !CubeStatusEnum.DISABLED.equals(ostatus)) {
+            throw new InternalErrorException("Only disabled cube can be purged, status of " + cubeName + " is " + ostatus);
+        }
+
+        try {
+            this.releaseAllSegments(cube);
+            return cube;
+        } catch (IOException e) {
+            throw e;
+        }
+
+    }
+
+    /**
+     * Update a cube status from ready to disabled.
+     *
+     * @return
+     * @throws CubeIntegrityException
+     * @throws IOException
+     * @throws JobException
+     */
+    @PreAuthorize(Constant.ACCESS_HAS_ROLE_ADMIN + " or hasPermission(#cube, 'ADMINISTRATION') or hasPermission(#cube, 'OPERATION') or hasPermission(#cube, 'MANAGEMENT')")
+    @Caching(evict = { @CacheEvict(value = QueryController.SUCCESS_QUERY_CACHE, allEntries = true), @CacheEvict(value = QueryController.EXCEPTION_QUERY_CACHE, allEntries = true) })
+    public CubeInstance disableCube(CubeInstance cube) throws IOException, CubeIntegrityException, JobException {
+        String cubeName = cube.getName();
+
+        CubeStatusEnum ostatus = cube.getStatus();
+        if (null != ostatus && !CubeStatusEnum.READY.equals(ostatus)) {
+            throw new InternalErrorException("Only ready cube can be disabled, status of " + cubeName + " is " + ostatus);
+        }
+
+        cube.setStatus(CubeStatusEnum.DISABLED);
+
+        try {
+            return getCubeManager().updateCube(cube);
+        } catch (IOException e) {
+            cube.setStatus(ostatus);
+            throw e;
+        }
+    }
+
+    /**
+     * Update a cube status from disable to ready.
+     *
+     * @return
+     * @throws CubeIntegrityException
+     * @throws IOException
+     * @throws JobException
+     */
+    @PreAuthorize(Constant.ACCESS_HAS_ROLE_ADMIN + " or hasPermission(#cube, 'ADMINISTRATION') or hasPermission(#cube, 'OPERATION')  or hasPermission(#cube, 'MANAGEMENT')")
+    public CubeInstance enableCube(CubeInstance cube) throws IOException, CubeIntegrityException, JobException {
+        String cubeName = cube.getName();
+
+        CubeStatusEnum ostatus = cube.getStatus();
+        if (!cube.getStatus().equals(CubeStatusEnum.DISABLED)) {
+            throw new InternalErrorException("Only disabled cube can be enabled, status of " + cubeName + " is " + ostatus);
+        }
+
+        if (cube.getSegments(CubeSegmentStatusEnum.READY).size() == 0) {
+            throw new InternalErrorException("Cube " + cubeName + " dosen't contain any READY segment");
+        }
+
+        List<JobInstance> jobInstances = this.getJobManager().listJobs(cube.getName(), null);
+        for (JobInstance jobInstance : jobInstances) {
+            if (jobInstance.getStatus() == JobStatusEnum.PENDING || jobInstance.getStatus() == JobStatusEnum.RUNNING) {
+                throw new JobException("Enable is not allowed with a running job.");
+            }
+        }
+        if (!cube.getDescriptor().calculateSignature().equals(cube.getDescriptor().getSignature())) {
+            this.releaseAllSegments(cube);
+        }
+
+        cube.setStatus(CubeStatusEnum.READY);
+        try {
+            return getCubeManager().updateCube(cube);
+        } catch (IOException e) {
+            cube.setStatus(ostatus);
+            throw e;
+        }
+    }
+
+    public MetricsResponse calculateMetrics(MetricsRequest request) {
+        DateFormat format = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss z");
+        List<CubeInstance> cubes = this.getCubeManager().listAllCubes();
+        MetricsResponse metrics = new MetricsResponse();
+        Date startTime = (null == request.getStartTime()) ? new Date(-1) : request.getStartTime();
+        Date endTime = (null == request.getEndTime()) ? new Date() : request.getEndTime();
+        metrics.increase("totalCubes", (float) 0);
+        metrics.increase("totalStorage", (float) 0);
+
+        for (CubeInstance cube : cubes) {
+            Date createdDate = new Date(-1);
+            try {
+                createdDate = (null == cube.getCreateTime()) ? createdDate : format.parse(cube.getCreateTime());
+            } catch (ParseException e) {
+                logger.error("", e);
+            }
+
+            if (createdDate.getTime() > startTime.getTime() && createdDate.getTime() < endTime.getTime()) {
+                metrics.increase("totalCubes");
+            }
+        }
+
+        metrics.increase("aveStorage", (metrics.get("totalCubes") == 0) ? 0 : metrics.get("totalStorage") / metrics.get("totalCubes"));
+
+        return metrics;
+    }
+
+    /**
+     * Calculate size of each region for given table and other info of the
+     * table.
+     *
+     * @param tableName The table name.
+     * @return The HBaseResponse object contains table size, region count. null
+     * if error happens.
+     * @throws IOException Exception when HTable resource is not closed correctly.
+     */
+    public HBaseResponse getHTableInfo(String tableName) throws IOException {
+        // Get HBase storage conf.
+        String hbaseUrl = KylinConfig.getInstanceFromEnv().getStorageUrl();
+        Configuration hconf = HadoopUtil.newHBaseConfiguration(hbaseUrl);
+
+        HTable table = null;
+        HBaseResponse hr = null;
+        long tableSize = 0;
+        int regionCount = 0;
+
+        try {
+            table = new HTable(hconf, tableName);
+
+            HBaseRegionSizeCalculator cal = new HBaseRegionSizeCalculator(table);
+            Map<byte[], Long> sizeMap = cal.getRegionSizeMap();
+
+            for (long s : sizeMap.values()) {
+                tableSize += s;
+            }
+
+            regionCount = sizeMap.size();
+
+            // Set response.
+            hr = new HBaseResponse();
+            hr.setTableSize(tableSize);
+            hr.setRegionCount(regionCount);
+        } finally {
+            if (null != table) {
+                table.close();
+            }
+        }
+
+        return hr;
+    }
+
+    /**
+     * Generate cardinality for table This will trigger a hadoop job and nothing
+     * The result will be merged into table exd info
+     *
+     * @param tableName
+     * @param delimiter
+     * @param format
+     */
+    public void generateCardinality(String tableName, String format, String delimiter) {
+        TableDesc table = getMetadataManager().getTableDesc(tableName);
+        Map<String, String> tableExd = getMetadataManager().getTableDescExd(tableName);
+        if (tableExd == null || table == null) {
+            IllegalArgumentException e = new IllegalArgumentException("Cannot find table descirptor " + tableName);
+            logger.error("Cannot find table descirptor " + tableName, e);
+            throw e;
+        }
+        Map<String, String> exd = getMetadataManager().getTableDescExd(tableName);
+        if (exd == null || !Boolean.valueOf(exd.get(MetadataConstances.TABLE_EXD_STATUS_KEY))) {
+            throw new IllegalArgumentException("Table " + tableName + " does not exist.");
+        }
+        String location = exd.get(MetadataConstances.TABLE_EXD_LOCATION);
+        if (location == null || MetadataConstances.TABLE_EXD_DEFAULT_VALUE.equals(location)) {
+            throw new IllegalArgumentException("Cannot get table " + tableName + " location, the location is " + location);
+        }
+        String inputFormat = exd.get(MetadataConstances.TABLE_EXD_IF);
+        if (inputFormat == null || MetadataConstances.TABLE_EXD_DEFAULT_VALUE.equals(inputFormat)) {
+            throw new IllegalArgumentException("Cannot get table " + tableName + " input format, the format is " + inputFormat);
+        }
+        String delim = exd.get(MetadataConstances.TABLE_EXD_DELIM);
+        if (delimiter != null) {
+            delim = delimiter;
+        }
+        String jarPath = getKylinConfig().getKylinJobJarPath();
+        String outPath = HiveColumnCardinalityJob.OUTPUT_PATH + "/" + tableName;
+        String[] args = null;
+        if (delim == null) {
+            args = new String[] { "-input", location, "-output", outPath, "-iformat", inputFormat };
+        } else {
+            args = new String[] { "-input", location, "-output", outPath, "-iformat", inputFormat, "-idelim", delim };
+        }
+        HiveColumnCardinalityJob job = new HiveColumnCardinalityJob(jarPath, null);
+        int hresult = 0;
+        try {
+            hresult = ToolRunner.run(job, args);
+        } catch (Exception e) {
+            logger.error("Cardinality calculation failed. ", e);
+            throw new IllegalArgumentException("Hadoop job failed with exception ", e);
+        }
+
+        // Get calculate result;
+        if (hresult != 0) {
+            throw new IllegalArgumentException("Hadoop job failed with result " + hresult);
+        }
+        List<String> columns = null;
+        try {
+            columns = job.readLines(new Path(outPath), job.getConf());
+        } catch (IllegalArgumentException e) {
+            logger.error("Failed to resolve cardinality for " + tableName + " from " + outPath, e);
+            return;
+        } catch (Exception e) {
+            logger.error("Failed to resolve cardinality for " + tableName + " from " + outPath, e);
+            return;
+        }
+        StringBuffer cardi = new StringBuffer();
+        ColumnDesc[] cols = table.getColumns();
+        if (columns.isEmpty() || cols.length != columns.size()) {
+            logger.error("The hadoop cardinality column size " + columns.size() + " is not equal metadata column size " + cols.length + ". Table " + tableName);
+        }
+        Iterator<String> it = columns.iterator();
+        while (it.hasNext()) {
+            String string = (String) it.next();
+            String[] ss = StringUtils.split(string, "\t");
+
+            if (ss.length != 2) {
+                logger.error("The hadoop cardinality value is not valid " + string);
+                continue;
+            }
+            cardi.append(ss[1]);
+            cardi.append(",");
+        }
+        String scardi = cardi.toString();
+        scardi = scardi.substring(0, scardi.length() - 1);
+        tableExd.put(MetadataConstances.TABLE_EXD_CARDINALITY, scardi);
+
+        try {
+            ByteArrayOutputStream bos = new ByteArrayOutputStream();
+            JsonUtil.writeValueIndent(bos, tableExd);
+            System.out.println(bos.toString());
+            ByteArrayInputStream bis = new ByteArrayInputStream(bos.toByteArray());
+            String xPath = ResourceStore.TABLE_EXD_RESOURCE_ROOT + "/" + tableName.toUpperCase() + "." + HiveSourceTableLoader.OUTPUT_SURFIX;
+            writeResource(bis, KylinConfig.getInstanceFromEnv(), xPath);
+        } catch (JsonGenerationException e) {
+            e.printStackTrace();
+        } catch (JsonMappingException e) {
+            e.printStackTrace();
+        } catch (IOException e) {
+            e.printStackTrace();
+        }
+
+        getMetadataManager().reload();
+    }
+
+    private static void writeResource(InputStream source, KylinConfig dstConfig, String path) throws IOException {
+        ResourceStore store = ResourceStore.getStore(dstConfig);
+        store.putResource(path, source, System.currentTimeMillis());
+    }
+
+    @PreAuthorize(Constant.ACCESS_HAS_ROLE_ADMIN + " or hasPermission(#cube, 'ADMINISTRATION') or hasPermission(#cube, 'OPERATION')  or hasPermission(#cube, 'MANAGEMENT')")
+    public void updateCubeNotifyList(CubeInstance cube, List<String> notifyList) throws IOException, CubeIntegrityException {
+        CubeDesc desc = cube.getDescriptor();
+        desc.setNotifyList(notifyList);
+        getMetadataManager().updateCubeDesc(desc);
+    }
+
+    public CubeInstance rebuildLookupSnapshot(String cubeName, String segmentName, String lookupTable) throws IOException {
+        CubeManager cubeMgr = getCubeManager();
+        CubeInstance cube = cubeMgr.getCube(cubeName);
+        CubeSegment seg = cube.getSegment(segmentName, CubeSegmentStatusEnum.READY);
+        cubeMgr.buildSnapshotTable(seg, lookupTable);
+
+        return cube;
+    }
+
+    /**
+     * purge the cube
+     *
+     * @throws IOException
+     * @throws JobException
+     * @throws UnknownHostException
+     * @throws CubeIntegrityException
+     */
+    private void releaseAllSegments(CubeInstance cube) throws IOException, JobException, UnknownHostException, CubeIntegrityException {
+        for (JobInstance jobInstance : this.getJobManager().listJobs(cube.getName(), null)) {
+            if (jobInstance.getStatus() != JobStatusEnum.FINISHED && jobInstance.getStatus() != JobStatusEnum.DISCARDED) {
+                for (JobStep jobStep : jobInstance.getSteps()) {
+                    if (jobStep.getStatus() != JobStepStatusEnum.FINISHED) {
+                        jobStep.setStatus(JobStepStatusEnum.DISCARDED);
+                    }
+                }
+                JobDAO.getInstance(this.getConfig()).updateJobInstance(jobInstance);
+            }
+        }
+
+        cube.getSegments().clear();
+        CubeManager.getInstance(getConfig()).updateCube(cube);
+    }
+
+    @PreAuthorize(Constant.ACCESS_HAS_ROLE_MODELER)
+    public String[] reloadHiveTable(String tables) throws IOException {
+        Set<String> loaded = HiveSourceTableLoader.reloadHiveTables(tables.split(","), getConfig());
+        getMetadataManager().reload();
+        return (String[]) loaded.toArray(new String[loaded.size()]);
+    }
+
+    @PreAuthorize(Constant.ACCESS_HAS_ROLE_ADMIN)
+    public void syncTableToProject(String[] tables, String project) throws IOException {
+        getProjectManager().updateTableToProject(tables, project);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/server/src/main/java/com/kylinolap/rest/service/JobService.java
----------------------------------------------------------------------
diff --git a/server/src/main/java/com/kylinolap/rest/service/JobService.java b/server/src/main/java/com/kylinolap/rest/service/JobService.java
new file mode 100644
index 0000000..af4f16f
--- /dev/null
+++ b/server/src/main/java/com/kylinolap/rest/service/JobService.java
@@ -0,0 +1,184 @@
+/*
+ * 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.rest.service;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Date;
+import java.util.List;
+
+import com.google.common.collect.Lists;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.springframework.beans.factory.annotation.Autowired;
+import org.springframework.security.access.prepost.PreAuthorize;
+import org.springframework.stereotype.Component;
+
+import com.kylinolap.cube.CubeBuildTypeEnum;
+import com.kylinolap.cube.CubeInstance;
+import com.kylinolap.cube.CubeSegment;
+import com.kylinolap.cube.exception.CubeIntegrityException;
+import com.kylinolap.job.JobInstance;
+import com.kylinolap.job.constant.JobStatusEnum;
+import com.kylinolap.job.exception.InvalidJobInstanceException;
+import com.kylinolap.job.exception.JobException;
+import com.kylinolap.rest.constant.Constant;
+import com.kylinolap.rest.exception.InternalErrorException;
+import com.kylinolap.rest.request.MetricsRequest;
+import com.kylinolap.rest.response.MetricsResponse;
+
+/**
+ * @author ysong1
+ */
+@Component("jobService")
+public class JobService extends BasicService {
+
+    private static final Logger logger = LoggerFactory.getLogger(CubeService.class);
+
+    @Autowired
+    private AccessService permissionService;
+
+    public List<JobInstance> listAllJobs(final String cubeName, final String projectName, final List<JobStatusEnum> statusList, final Integer limitValue, final Integer offsetValue) throws IOException, JobException {
+        Integer limit = (null == limitValue) ? 30 : limitValue;
+        Integer offset = (null == offsetValue) ? 0 : offsetValue;
+        List<JobInstance> jobs = listAllJobs(cubeName, projectName, statusList);
+        Collections.sort(jobs);
+
+        if (jobs.size() <= offset) {
+            return Collections.emptyList();
+        }
+
+        if ((jobs.size() - offset) < limit) {
+            return jobs.subList(offset, jobs.size());
+        }
+
+        return jobs.subList(offset, offset + limit);
+    }
+
+    public List<JobInstance> listAllJobs(String cubeName, String projectName, List<JobStatusEnum> statusList) throws IOException, JobException {
+        List<JobInstance> jobs = new ArrayList<JobInstance>();
+        jobs.addAll(this.getJobManager().listJobs(cubeName, projectName));
+
+        if (null == jobs || jobs.size() == 0) {
+            return jobs;
+        }
+
+        List<JobInstance> results = new ArrayList<JobInstance>();
+
+        for (JobInstance job : jobs) {
+            if (null != statusList && statusList.size() > 0) {
+                for (JobStatusEnum status : statusList) {
+                    if (job.getStatus() == status) {
+                        results.add(job);
+                    }
+                }
+            } else {
+                results.add(job);
+            }
+        }
+
+        return results;
+    }
+
+    @PreAuthorize(Constant.ACCESS_HAS_ROLE_ADMIN + " or hasPermission(#cube, 'ADMINISTRATION') or hasPermission(#cube, 'OPERATION') or hasPermission(#cube, 'MANAGEMENT')")
+    public String submitJob(CubeInstance cube, long startDate, long endDate, CubeBuildTypeEnum buildType,String submitter) throws IOException, JobException, InvalidJobInstanceException {
+
+        List<JobInstance> jobInstances = this.getJobManager().listJobs(cube.getName(), null);
+        for (JobInstance jobInstance : jobInstances) {
+            if (jobInstance.getStatus() == JobStatusEnum.PENDING || jobInstance.getStatus() == JobStatusEnum.RUNNING) {
+                throw new JobException("The cube " + cube.getName() + " has running job(" + jobInstance.getUuid() + ") please discard it and try again.");
+            }
+        }
+
+        String uuid = null;
+        try {
+            List<CubeSegment> cubeSegments = this.getCubeManager().allocateSegments(cube, buildType, startDate, endDate);
+            List<JobInstance> jobs = Lists.newArrayListWithExpectedSize(cubeSegments.size());
+            for (CubeSegment segment : cubeSegments) {
+                uuid = segment.getUuid();
+                JobInstance job = this.getJobManager().createJob(cube.getName(), segment.getName(), segment.getUuid(), buildType,submitter);
+                segment.setLastBuildJobID(uuid);
+                jobs.add(job);
+            }
+            getCubeManager().updateCube(cube);
+            for (JobInstance job: jobs) {
+                this.getJobManager().submitJob(job);
+                permissionService.init(job, null);
+                permissionService.inherit(job, cube);
+            }
+        } catch (CubeIntegrityException e) {
+            throw new InternalErrorException(e.getLocalizedMessage(), e);
+        }
+
+        return uuid;
+    }
+
+    public JobInstance getJobInstance(String uuid) throws IOException, JobException {
+        return this.getJobManager().getJob(uuid);
+    }
+
+    @PreAuthorize(Constant.ACCESS_HAS_ROLE_ADMIN + " or hasPermission(#job, 'ADMINISTRATION') or hasPermission(#job, 'OPERATION') or hasPermission(#job, 'MANAGEMENT')")
+    public void resumeJob(JobInstance job) throws IOException, JobException {
+        this.getJobManager().resumeJob(job.getUuid());
+    }
+
+    @PreAuthorize(Constant.ACCESS_HAS_ROLE_ADMIN + " or hasPermission(#job, 'ADMINISTRATION') or hasPermission(#job, 'OPERATION') or hasPermission(#job, 'MANAGEMENT')")
+    public void cancelJob(JobInstance job) throws IOException, JobException, CubeIntegrityException {
+        CubeInstance cube = this.getCubeManager().getCube(job.getRelatedCube());
+        List<JobInstance> jobs = this.getJobManager().listJobs(cube.getName(), null);
+        for (JobInstance jobInstance : jobs) {
+            if (jobInstance.getStatus() != JobStatusEnum.DISCARDED && jobInstance.getStatus() != JobStatusEnum.FINISHED) {
+                this.getJobManager().discardJob(jobInstance.getUuid());
+            }
+        }
+    }
+
+    public MetricsResponse calculateMetrics(MetricsRequest request) {
+        List<JobInstance> jobs = new ArrayList<JobInstance>();
+
+        try {
+            jobs.addAll(getJobManager().listJobs(null, null));
+        } catch (IOException e) {
+            logger.error("", e);
+        } catch (JobException e) {
+            logger.error("", e);
+        }
+
+        MetricsResponse metrics = new MetricsResponse();
+        int successCount = 0;
+        long totalTime = 0;
+        Date startTime = (null == request.getStartTime()) ? new Date(-1) : request.getStartTime();
+        Date endTime = (null == request.getEndTime()) ? new Date() : request.getEndTime();
+
+        for (JobInstance job : jobs) {
+            if (job.getExecStartTime() > startTime.getTime() && job.getExecStartTime() < endTime.getTime()) {
+                metrics.increase("total");
+                metrics.increase(job.getStatus().name());
+
+                if (job.getStatus() == JobStatusEnum.FINISHED) {
+                    successCount++;
+                    totalTime += (job.getExecEndTime() - job.getExecStartTime());
+                }
+            }
+        }
+
+        metrics.increase("aveExecTime", ((successCount == 0) ? 0 : totalTime / (float) successCount));
+
+        return metrics;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/server/src/main/java/com/kylinolap/rest/service/MetricsService.java
----------------------------------------------------------------------
diff --git a/server/src/main/java/com/kylinolap/rest/service/MetricsService.java b/server/src/main/java/com/kylinolap/rest/service/MetricsService.java
new file mode 100644
index 0000000..9d8dc4e
--- /dev/null
+++ b/server/src/main/java/com/kylinolap/rest/service/MetricsService.java
@@ -0,0 +1,60 @@
+/*
+ * 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.rest.service;
+
+import org.springframework.beans.factory.InitializingBean;
+import org.springframework.beans.factory.annotation.Autowired;
+import org.springframework.beans.factory.annotation.Qualifier;
+import org.springframework.stereotype.Component;
+
+import com.codahale.metrics.MetricRegistry;
+import com.kylinolap.job.JobManager;
+import com.kylinolap.rest.metrics.JobMetrics;
+import com.kylinolap.rest.metrics.QueryMetrics;
+
+/**
+ * @author xduo
+ * 
+ */
+@Component("metricsService")
+public class MetricsService implements InitializingBean {
+
+    @Autowired
+    @Qualifier("metrics")
+    private MetricRegistry metricRegistry;
+
+    public void registerJobMetrics(final JobManager jobManager) {
+        JobMetrics jobMetrics = JobMetrics.getInstance();
+        jobMetrics.setJobManager(jobManager);
+        metricRegistry.register("JobMetrics", jobMetrics);
+    }
+
+    public void registerQueryMetrics() {
+        metricRegistry.register("QueryMetrics", QueryMetrics.getInstance());
+    }
+
+    /*
+     * (non-Javadoc)
+     * 
+     * @see
+     * org.springframework.beans.factory.InitializingBean#afterPropertiesSet()
+     */
+    @Override
+    public void afterPropertiesSet() throws Exception {
+        registerQueryMetrics();
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/server/src/main/java/com/kylinolap/rest/service/ProjectService.java
----------------------------------------------------------------------
diff --git a/server/src/main/java/com/kylinolap/rest/service/ProjectService.java b/server/src/main/java/com/kylinolap/rest/service/ProjectService.java
new file mode 100644
index 0000000..73c7302
--- /dev/null
+++ b/server/src/main/java/com/kylinolap/rest/service/ProjectService.java
@@ -0,0 +1,126 @@
+/*
+ * 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.rest.service;
+
+import java.io.IOException;
+import java.util.Collections;
+import java.util.List;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.springframework.beans.factory.annotation.Autowired;
+import org.springframework.security.access.prepost.PreAuthorize;
+import org.springframework.security.core.context.SecurityContextHolder;
+import org.springframework.stereotype.Component;
+
+import com.kylinolap.cube.project.ProjectInstance;
+import com.kylinolap.rest.constant.Constant;
+import com.kylinolap.rest.exception.InternalErrorException;
+import com.kylinolap.rest.request.CreateProjectRequest;
+import com.kylinolap.rest.request.UpdateProjectRequest;
+import com.kylinolap.rest.security.AclPermission;
+
+/**
+ * @author xduo
+ * 
+ */
+@Component("projectService")
+public class ProjectService extends BasicService {
+
+    private static final Logger logger = LoggerFactory.getLogger(ProjectService.class);
+
+    @Autowired
+    private AccessService accessService;
+
+    public ProjectInstance createProject(CreateProjectRequest projectRequest) throws IOException {
+        String projectName = projectRequest.getName();
+        String description = projectRequest.getDescription();
+        ProjectInstance currentProject = getProjectManager().getProject(projectName);
+
+        if (currentProject != null) {
+            throw new InternalErrorException("The project named " + projectName + " already exists");
+        }
+        String owner = SecurityContextHolder.getContext().getAuthentication().getName();
+        ProjectInstance createdProject = getProjectManager().createProject(projectName, owner, description);
+        accessService.init(createdProject, AclPermission.ADMINISTRATION);
+        logger.debug("New project created.");
+
+        return createdProject;
+    }
+
+    @PreAuthorize(Constant.ACCESS_HAS_ROLE_ADMIN + " or hasPermission(#cube, 'ADMINISTRATION') or hasPermission(#cube, 'MANAGEMENT')")
+    public ProjectInstance updateProject(UpdateProjectRequest projectRequest) throws IOException {
+        String formerProjectName = projectRequest.getFormerProjectName();
+        String newProjectName = projectRequest.getNewProjectName();
+        String newDescription = projectRequest.getNewDescription();
+
+        ProjectInstance currentProject = getProjectManager().getProject(formerProjectName);
+
+        if (currentProject == null) {
+            throw new InternalErrorException("The project named " + formerProjectName + " does not exists");
+        }
+
+        ProjectInstance updatedProject = getProjectManager().updateProject(currentProject, newProjectName, newDescription);
+
+        logger.debug("Project updated.");
+
+        return updatedProject;
+    }
+
+    public List<ProjectInstance> listAllProjects(final Integer limit, final Integer offset) {
+        List<ProjectInstance> projects = getProjectManager().listAllProjects();
+
+        int climit = (null == limit) ? 30 : limit;
+        int coffset = (null == offset) ? 0 : offset;
+
+        if (projects.size() <= coffset) {
+            return Collections.emptyList();
+        }
+
+        if ((projects.size() - coffset) < climit) {
+            return projects.subList(coffset, projects.size());
+        }
+
+        return projects.subList(coffset, coffset + climit);
+    }
+
+    @PreAuthorize(Constant.ACCESS_HAS_ROLE_ADMIN + " or hasPermission(#cube, 'ADMINISTRATION') or hasPermission(#cube, 'MANAGEMENT')")
+    public void deleteProject(String projectName) throws IOException {
+        ProjectInstance project = getProjectManager().getProject(projectName);
+        getProjectManager().dropProject(projectName);
+
+        accessService.clean(project, true);
+    }
+
+    /**
+     * @param name
+     * @throws IOException
+     */
+    public void reloadProjectCache(String name) throws IOException {
+        ProjectInstance project = this.getProjectManager().getProject(name);
+        this.getProjectManager().loadProjectCache(project, false);
+    }
+
+    /**
+     * @param name
+     */
+    public void removeProjectCache(String name) {
+        ProjectInstance project = this.getProjectManager().getProject(name);
+        this.getProjectManager().removeProjectCache(project);
+    }
+
+}